From 2d3068c01fd6a5d3523f91de3dd2c951da0e88f3 Mon Sep 17 00:00:00 2001 From: Marcus Eriksson <marcuse@apache.org> Date: Thu, 10 Oct 2024 10:42:58 +0200 Subject: [PATCH 001/225] During TCM upgrade, retain all properties of existing system tables Patch by Abe Ratnofsky and marcuse; reviewed by Sam Tunnicliffe for CASSANDRA-19992 Co-authored-by: Abe Ratnofsky <abe@aber.io> --- CHANGES.txt | 1 + .../cassandra/schema/DistributedSchema.java | 7 +- .../schema/DistributedSchemaTest.java | 80 +++++++++++++++++++ 3 files changed, 86 insertions(+), 2 deletions(-) create mode 100644 test/unit/org/apache/cassandra/schema/DistributedSchemaTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 276969d4d7af..0269f479b00e 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * During TCM upgrade, retain all properties of existing system tables (CASSANDRA-19992) * Properly cancel in-flight futures and reject requests in EpochAwareDebounce during shutdown (CASSANDRA-19848) * Provide clearer exception message on failing commitlog_disk_access_mode combinations (CASSANDRA-19812) * Add total space used for a keyspace to nodetool tablestats (CASSANDRA-19671) diff --git a/src/java/org/apache/cassandra/schema/DistributedSchema.java b/src/java/org/apache/cassandra/schema/DistributedSchema.java index 327a2c1a93bc..e10c1e00495c 100644 --- a/src/java/org/apache/cassandra/schema/DistributedSchema.java +++ b/src/java/org/apache/cassandra/schema/DistributedSchema.java @@ -123,7 +123,7 @@ public static DistributedSchema fromSystemTables(Keyspaces keyspaces, Set<String AuthKeyspace.metadata()); for (KeyspaceMetadata ksm : keyspaces) // on disk keyspaces kss = kss.withAddedOrUpdated(kss.get(ksm.name) - .map(k -> merged(k, ksm)) + .map(k -> merged(ksm, k)) .orElse(ksm)); keyspaces = kss; } @@ -134,11 +134,14 @@ public static DistributedSchema fromSystemTables(Keyspaces keyspaces, Set<String * merges any tables in `mergeFrom` to `mergeTo` unless they already exist there. * * This method is only called when creating the initial cluster metadata on upgrade + * + * mergeTo is the on disk schema, mergeFrom is the hard coded KSM + * if a table exists in the on disk schema, keep it as-is, otherwise add the hard coded one */ private static KeyspaceMetadata merged(KeyspaceMetadata mergeTo, KeyspaceMetadata mergeFrom) { KeyspaceMetadata newKsm = KeyspaceMetadata.create(mergeTo.name, - mergeFrom.params, + mergeTo.params, mergeTo.tables, mergeTo.views, mergeTo.types, diff --git a/test/unit/org/apache/cassandra/schema/DistributedSchemaTest.java b/test/unit/org/apache/cassandra/schema/DistributedSchemaTest.java new file mode 100644 index 000000000000..4b74f5c38b6e --- /dev/null +++ b/test/unit/org/apache/cassandra/schema/DistributedSchemaTest.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.schema; + +import java.util.HashMap; +import java.util.Set; + +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.ServerTestUtils; +import org.apache.cassandra.auth.AuthKeyspace; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.service.reads.PercentileSpeculativeRetryPolicy; +import org.apache.cassandra.service.reads.repair.ReadRepairStrategy; + +import static org.assertj.core.api.Assertions.assertThat; + +public class DistributedSchemaTest +{ + + @BeforeClass + public static void beforeClass() + { + ServerTestUtils.prepareServerNoRegister(); + } + + @Test + public void fromSystemTablesPreservesExistingSchemaProperties() + { + TableMetadata defaultTable = AuthKeyspace.metadata().tables.getNullable(AuthKeyspace.ROLES); + TableMetadata modifiedTable = defaultTable.unbuild() + .id(TableId.fromLong(0)) + .comment("Testing") + .gcGraceSeconds(60) + .compaction(CompactionParams.lcs(new HashMap<>())) + .speculativeRetry(new PercentileSpeculativeRetryPolicy(50.0)) + .build(); + assertThat(modifiedTable).isNotEqualTo(defaultTable); + + TableMetadata nonstandard = TableMetadata.builder(SchemaConstants.AUTH_KEYSPACE_NAME, "nonstandard") + .addPartitionKeyColumn("pk", Int32Type.instance) + .comment("A non-standard table that should be preserved") + .params(TableParams.builder().readRepair(ReadRepairStrategy.NONE).build()) + .build(); + KeyspaceMetadata km = KeyspaceMetadata.create(SchemaConstants.AUTH_KEYSPACE_NAME, + KeyspaceParams.simple(3), + Tables.of(modifiedTable, nonstandard)); + + DistributedSchema schema = DistributedSchema.fromSystemTables(Keyspaces.of(km), + Set.of(DatabaseDescriptor.getLocalDataCenter())); + KeyspaceMetadata merged = schema.getKeyspaceMetadata(SchemaConstants.AUTH_KEYSPACE_NAME); + assertThat(merged.getTableOrViewNullable(AuthKeyspace.ROLES)).isEqualTo(modifiedTable); + assertThat(merged.getTableOrViewNullable("nonstandard")).isEqualTo(nonstandard); + + // check all other default tables in the auth keyspace were included in the merged schema + AuthKeyspace.metadata().tables.forEach(tm -> { + if (!tm.name.equals(AuthKeyspace.ROLES)) + assertThat(merged.getTableOrViewNullable(tm.name)).isEqualTo(tm); + }); + + } +} From a9a83dc884621f0bfdf2c37be9e7e8fcf0081da1 Mon Sep 17 00:00:00 2001 From: Sam Tunnicliffe <samt@apache.org> Date: Mon, 14 Oct 2024 16:21:52 +0100 Subject: [PATCH 002/225] Simulation test fixes * During replacement, correctly set token on the joining instance * In bootstrap, we were not correctly stepping through the join operation * When investigating bootstrap issues, it was found that we were not running repairPaxosForTopologyChange on this path. Patch by Sam Tunnicliffe; reviewed by Marcus Eriksson for CASSANDRA-19997 --- CHANGES.txt | 1 + .../org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java | 2 ++ .../org/apache/cassandra/simulator/cluster/KeyspaceActions.java | 2 +- .../org/apache/cassandra/simulator/cluster/OnClusterJoin.java | 2 +- 4 files changed, 5 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 0269f479b00e..0e454fa19808 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * In simulation tests, correctly set the tokens of replacement nodes (CASSANDRA-19997) * During TCM upgrade, retain all properties of existing system tables (CASSANDRA-19992) * Properly cancel in-flight futures and reject requests in EpochAwareDebounce during shutdown (CASSANDRA-19848) * Provide clearer exception message on failing commitlog_disk_access_mode combinations (CASSANDRA-19812) diff --git a/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java b/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java index 7d316e1ea1b4..15182fc92b2a 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java +++ b/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java @@ -353,6 +353,8 @@ public static boolean bootstrap(final Collection<Token> tokens, logger.info("Resetting bootstrap progress to start fresh"); SystemKeyspace.resetAvailableStreamedRanges(); } + + StorageService.instance.repairPaxosForTopologyChange("bootstrap"); Future<StreamState> bootstrapStream = StorageService.instance.startBootstrap(metadata, beingReplaced, movements, strictMovements); try { diff --git a/test/simulator/main/org/apache/cassandra/simulator/cluster/KeyspaceActions.java b/test/simulator/main/org/apache/cassandra/simulator/cluster/KeyspaceActions.java index 1cff2ab08f7c..4b6caa0164c8 100644 --- a/test/simulator/main/org/apache/cassandra/simulator/cluster/KeyspaceActions.java +++ b/test/simulator/main/org/apache/cassandra/simulator/cluster/KeyspaceActions.java @@ -281,8 +281,8 @@ private Action next() joined.add(join); joined.remove(leave); left.add(leave); - TokenPlacementModel.ReplicatedRanges placementsAfter = placements(joined, currentRf); nodeLookup.setTokenOf(join, nodeLookup.tokenOf(leave)); + TokenPlacementModel.ReplicatedRanges placementsAfter = placements(joined, currentRf); Topology during = recomputeTopology(placementsBefore, placementsAfter); updateTopology(during); Topology after = recomputeTopology(placementsAfter, placementsAfter); diff --git a/test/simulator/main/org/apache/cassandra/simulator/cluster/OnClusterJoin.java b/test/simulator/main/org/apache/cassandra/simulator/cluster/OnClusterJoin.java index 53484ab318b2..bb36ad72188d 100644 --- a/test/simulator/main/org/apache/cassandra/simulator/cluster/OnClusterJoin.java +++ b/test/simulator/main/org/apache/cassandra/simulator/cluster/OnClusterJoin.java @@ -111,7 +111,7 @@ private ExecuteNextStep(ClusterActions actions, int on, int kind) BootstrapAndJoin bootstrapAndJoin = ((BootstrapAndJoin) sequence); assert bootstrapAndJoin.next.ordinal() == kind : String.format("Expected next step to be %s, but got %s", Transformation.Kind.values()[kind], bootstrapAndJoin.next); - boolean res = bootstrapAndJoin.finishJoiningRing().executeNext().isContinuable(); + boolean res = bootstrapAndJoin.executeNext().isContinuable(); assert res; }); } From e8891be04a681fbd87e2bec2c07ed22e55725a04 Mon Sep 17 00:00:00 2001 From: rwelgosh <raymond.welgosh@gmail.com> Date: Mon, 14 Oct 2024 00:01:49 -0400 Subject: [PATCH 003/225] Fix type check for referenced duration type for nested types Patch by Raymond Welgosh; Reviewed by David Capwell, Yifan Cai for CASSANDRA-19890 --- CHANGES.txt | 1 + .../cassandra/db/marshal/AbstractType.java | 7 +++++ .../cassandra/db/marshal/ReversedType.java | 6 +++++ .../db/marshal/AbstractTypeTest.java | 27 +++++++++++++++++++ 4 files changed, 41 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index 0e454fa19808..239c04f4c942 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Fix type check for referenced duration type for nested types (CASSANDRA-19890) * In simulation tests, correctly set the tokens of replacement nodes (CASSANDRA-19997) * During TCM upgrade, retain all properties of existing system tables (CASSANDRA-19992) * Properly cancel in-flight futures and reject requests in EpochAwareDebounce during shutdown (CASSANDRA-19848) diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractType.java b/src/java/org/apache/cassandra/db/marshal/AbstractType.java index c55d9afd440d..3c382e781be1 100644 --- a/src/java/org/apache/cassandra/db/marshal/AbstractType.java +++ b/src/java/org/apache/cassandra/db/marshal/AbstractType.java @@ -641,6 +641,13 @@ public AbstractType<?> expandUserTypes() public boolean referencesDuration() { + for (AbstractType<?> type : subTypes()) + { + if (type.referencesDuration()) + { + return true; + } + } return false; } diff --git a/src/java/org/apache/cassandra/db/marshal/ReversedType.java b/src/java/org/apache/cassandra/db/marshal/ReversedType.java index 3d5e11d0d7f6..b567fe348126 100644 --- a/src/java/org/apache/cassandra/db/marshal/ReversedType.java +++ b/src/java/org/apache/cassandra/db/marshal/ReversedType.java @@ -164,6 +164,12 @@ public AbstractType<?> expandUserTypes() return getInstance(baseType.expandUserTypes()); } + @Override + public boolean referencesDuration() + { + return baseType.referencesDuration(); + } + @Override public ReversedType<?> withUpdatedUserType(UserType udt) { diff --git a/test/unit/org/apache/cassandra/db/marshal/AbstractTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/AbstractTypeTest.java index 35ecfab60d72..ed2046fe5df6 100644 --- a/test/unit/org/apache/cassandra/db/marshal/AbstractTypeTest.java +++ b/test/unit/org/apache/cassandra/db/marshal/AbstractTypeTest.java @@ -460,6 +460,33 @@ public void nested() }); } + @Test + @SuppressWarnings("rawtypes") + public void nestedDuration() + { + qt().forAll(AbstractTypeGenerators.builder() + .withoutTypeKinds(COUNTER) + .withPrimitives(DurationType.instance) + .build()) + .checkAssert(type -> { + assertThat(type.referencesDuration()).isTrue(); + assertThat(ReversedType.getInstance(type).referencesDuration()).isTrue(); + }); + } + + @Test + public void nestedWithoutDuration() + { + qt().forAll(AbstractTypeGenerators.builder() + .withoutTypeKinds(PRIMITIVE, COUNTER) + .withoutPrimitive(DurationType.instance) + .build()) + .checkAssert(type -> { + assertThat(type.referencesDuration()).isFalse(); + assertThat(ReversedType.getInstance(type).referencesDuration()).isFalse(); + }); + } + /** * @see <pre>CASSANDRA-18526: TupleType getString and fromString are not safe with string types</pre> */ From 5e4ff921afb9d8cb2deacbfd24db78ea4e2ccbe1 Mon Sep 17 00:00:00 2001 From: Bereng <berenguerblasi@gmail.com> Date: Tue, 22 Oct 2024 14:27:49 +0200 Subject: [PATCH 004/225] Add extra compaction junit patch by Berenguer Blasi; reviewed by Branimir Lambov, Michael Semb Wever for CASSANDRA-19863 --- ...pactionStrategyManagerPendingRepairTest.java | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerPendingRepairTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerPendingRepairTest.java index 028d28303ecb..d809644e1ca2 100644 --- a/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerPendingRepairTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerPendingRepairTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.Iterables; +import org.apache.cassandra.Util; import org.apache.cassandra.repair.consistent.LocalSession; import org.junit.Assert; @@ -374,15 +375,29 @@ public void testFinalizedAndCompactionRace() throws NoSuchRepairSessionException System.out.println("Live sstables: " + cfs.getLiveSSTables().size()); System.out.println("*********************************************************************************************"); + // Run compaction again. It should pick up the pending repair sstable compactionTask = csm.getNextBackgroundTask(FBUtilities.nowInSeconds()); if (compactionTask != null) { Assert.assertSame(PendingRepairManager.RepairFinishedCompactionTask.class, compactionTask.getClass()); compactionTask.execute(ActiveCompactionsTracker.NOOP); - Assert.assertEquals(1, cfs.getLiveSSTables().size()); + + while ((compactionTask = csm.getNextBackgroundTask(FBUtilities.nowInSeconds())) != null) + compactionTask.execute(ActiveCompactionsTracker.NOOP); } + // Make sure you consume all pending compactions + Util.spinAssertEquals(Boolean.FALSE, + () -> { + AbstractCompactionTask ctask; + while ((ctask = csm.getNextBackgroundTask(FBUtilities.nowInSeconds())) != null) + ctask.execute(ActiveCompactionsTracker.NOOP); + + return hasPendingStrategiesFor(repairID); + }, + 30); + System.out.println("*********************************************************************************************"); System.out.println(compactedSSTable); System.out.println("Pending repair UUID: " + compactedSSTable.getPendingRepair()); From c8854af03064ae894e4d69253723a1f99f001450 Mon Sep 17 00:00:00 2001 From: Cheng <chengw@netflix.com> Date: Tue, 22 Oct 2024 14:27:10 -0700 Subject: [PATCH 005/225] Add -H option for human-friendly output in nodetool compactionhistory patch by Cheng Wang; reviewed by Jordan West, Stefan Miklosovic for CASSANDRA-20015 --- CHANGES.txt | 1 + .../apache/cassandra/io/util/FileUtils.java | 8 +++++ .../tools/nodetool/CompactionHistory.java | 7 +++- .../stats/CompactionHistoryHolder.java | 13 +++++--- .../nodetool/stats/TableStatsHolder.java | 33 ++++++++----------- 5 files changed, 37 insertions(+), 25 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 239c04f4c942..7e5d6fede41d 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Add -H option for human-friendly output in nodetool compactionhistory (CASSANDRA-20015) * Fix type check for referenced duration type for nested types (CASSANDRA-19890) * In simulation tests, correctly set the tokens of replacement nodes (CASSANDRA-19997) * During TCM upgrade, retain all properties of existing system tables (CASSANDRA-19992) diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java b/src/java/org/apache/cassandra/io/util/FileUtils.java index e32b059e68b3..cf6ea52be16e 100644 --- a/src/java/org/apache/cassandra/io/util/FileUtils.java +++ b/src/java/org/apache/cassandra/io/util/FileUtils.java @@ -426,6 +426,14 @@ else if (value.endsWith(" bytes")) } } + public static String stringifyFileSize(long bytes, boolean humanReadable) + { + if (humanReadable) + return stringifyFileSize(bytes); + else + return Long.toString(bytes); + } + public static String stringifyFileSize(double value) { double d; diff --git a/src/java/org/apache/cassandra/tools/nodetool/CompactionHistory.java b/src/java/org/apache/cassandra/tools/nodetool/CompactionHistory.java index d1a506187c24..e963d7a1c42d 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/CompactionHistory.java +++ b/src/java/org/apache/cassandra/tools/nodetool/CompactionHistory.java @@ -34,6 +34,11 @@ public class CompactionHistory extends NodeToolCmd description = "Output format (json, yaml)") private String outputFormat = ""; + @Option(title = "human_readable", + name = {"-H", "--human-readable"}, + description = "Display bytes in human readable form, i.e. KiB, MiB, GiB, TiB") + private boolean humanReadable = false; + @Override public void execute(NodeProbe probe) { @@ -41,7 +46,7 @@ public void execute(NodeProbe probe) { throw new IllegalArgumentException("arguments for -F are json,yaml only."); } - StatsHolder data = new CompactionHistoryHolder(probe); + StatsHolder data = new CompactionHistoryHolder(probe, humanReadable); StatsPrinter printer = CompactionHistoryPrinter.from(outputFormat); printer.print(data, probe.output().out); } diff --git a/src/java/org/apache/cassandra/tools/nodetool/stats/CompactionHistoryHolder.java b/src/java/org/apache/cassandra/tools/nodetool/stats/CompactionHistoryHolder.java index 362bc67c3443..189500bc9e76 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/stats/CompactionHistoryHolder.java +++ b/src/java/org/apache/cassandra/tools/nodetool/stats/CompactionHistoryHolder.java @@ -29,16 +29,19 @@ import java.util.Set; import javax.management.openmbean.TabularData; +import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.tools.NodeProbe; public class CompactionHistoryHolder implements StatsHolder { public final NodeProbe probe; public List<String> indexNames; + private final boolean humanReadable; - public CompactionHistoryHolder(NodeProbe probe) + public CompactionHistoryHolder(NodeProbe probe, boolean humanReadable) { this.probe = probe; + this.humanReadable = humanReadable; } /** @@ -73,7 +76,7 @@ public int compareTo(CompactionHistoryHolder.CompactionHistoryRow chr) return Long.signum(chr.compactedAt - this.compactedAt); } - private HashMap<String, Object> getAllAsMap() + private HashMap<String, Object> getAllAsMap(boolean humanReadable) { HashMap<String, Object> compaction = new HashMap<>(); compaction.put("id", this.id); @@ -82,8 +85,8 @@ private HashMap<String, Object> getAllAsMap() Instant instant = Instant.ofEpochMilli(this.compactedAt); LocalDateTime ldt = LocalDateTime.ofInstant(instant, ZoneId.systemDefault()); compaction.put("compacted_at", ldt.toString()); - compaction.put("bytes_in", this.bytesIn); - compaction.put("bytes_out", this.bytesOut); + compaction.put("bytes_in", FileUtils.stringifyFileSize(this.bytesIn, humanReadable)); + compaction.put("bytes_out", FileUtils.stringifyFileSize(this.bytesOut, humanReadable)); compaction.put("rows_merged", this.rowMerged); compaction.put("compaction_properties", this.compactionProperties); return compaction; @@ -122,7 +125,7 @@ public Map<String, Object> convert2Map() Collections.sort(chrList); for (CompactionHistoryHolder.CompactionHistoryRow chr : chrList) { - compactions.add(chr.getAllAsMap()); + compactions.add(chr.getAllAsMap(humanReadable)); } result.put("CompactionHistory", compactions); return result; diff --git a/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsHolder.java b/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsHolder.java index d32d88bffadf..338151a0d89d 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsHolder.java +++ b/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsHolder.java @@ -82,8 +82,8 @@ private Map<String, Object> convertAllToMap() mpKeyspace.put("write_count", keyspace.writeCount); mpKeyspace.put("write_latency_ms", keyspace.writeLatency()); mpKeyspace.put("pending_flushes", keyspace.pendingFlushes); - mpKeyspace.put("space_used_live", format(keyspace.spaceUsedLive, humanReadable)); - mpKeyspace.put("space_used_total", format(keyspace.spaceUsedTotal, humanReadable)); + mpKeyspace.put("space_used_live", FileUtils.stringifyFileSize(keyspace.spaceUsedLive, humanReadable)); + mpKeyspace.put("space_used_total", FileUtils.stringifyFileSize(keyspace.spaceUsedTotal, humanReadable)); // store each table's metrics to map List<StatsTable> tables = keyspace.tables; @@ -260,7 +260,7 @@ private void initializeKeyspaces(NodeProbe probe, boolean ignore, List<String> t for (int level = 0; level < leveledSSTablesBytes.length; level++) { long size = leveledSSTablesBytes[level]; - statsTable.sstableBytesInEachLevel.add(format(size, humanReadable)); + statsTable.sstableBytesInEachLevel.add(FileUtils.stringifyFileSize(size, humanReadable)); } } @@ -300,16 +300,16 @@ private void initializeKeyspaces(NodeProbe probe, boolean ignore, List<String> t throw e; } - statsTable.spaceUsedLive = format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "LiveDiskSpaceUsed"), humanReadable); - statsTable.spaceUsedTotal = format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "TotalDiskSpaceUsed"), humanReadable); - statsTable.spaceUsedBySnapshotsTotal = format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "SnapshotsSize"), humanReadable); + statsTable.spaceUsedLive = FileUtils.stringifyFileSize((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "LiveDiskSpaceUsed"), humanReadable); + statsTable.spaceUsedTotal = FileUtils.stringifyFileSize((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "TotalDiskSpaceUsed"), humanReadable); + statsTable.spaceUsedBySnapshotsTotal = FileUtils.stringifyFileSize((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "SnapshotsSize"), humanReadable); maybeAddTWCSWindowWithMaxDuration(statsTable, probe, keyspaceName, tableName); if (offHeapSize != null) { statsTable.offHeapUsed = true; - statsTable.offHeapMemoryUsedTotal = format(offHeapSize, humanReadable); + statsTable.offHeapMemoryUsedTotal = FileUtils.stringifyFileSize(offHeapSize, humanReadable); } if (percentRepaired != null) @@ -330,11 +330,11 @@ private void initializeKeyspaces(NodeProbe probe, boolean ignore, List<String> t statsTable.numberOfPartitionsEstimate = estimatedPartitionCount; statsTable.memtableCellCount = probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableColumnsCount"); - statsTable.memtableDataSize = format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableLiveDataSize"), humanReadable); + statsTable.memtableDataSize = FileUtils.stringifyFileSize((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableLiveDataSize"), humanReadable); if (memtableOffHeapSize != null) { statsTable.memtableOffHeapUsed = true; - statsTable.memtableOffHeapMemoryUsed = format(memtableOffHeapSize, humanReadable); + statsTable.memtableOffHeapMemoryUsed = FileUtils.stringifyFileSize(memtableOffHeapSize, humanReadable); } statsTable.memtableSwitchCount = probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableSwitchCount"); statsTable.speculativeRetries = probe.getColumnFamilyMetric(keyspaceName, tableName, "SpeculativeRetries"); @@ -356,23 +356,23 @@ private void initializeKeyspaces(NodeProbe probe, boolean ignore, List<String> t statsTable.bloomFilterFalsePositives = probe.getColumnFamilyMetric(keyspaceName, tableName, "BloomFilterFalsePositives"); statsTable.bloomFilterFalseRatio = bloomFilterFalseRatio != null ? bloomFilterFalseRatio : Double.NaN; - statsTable.bloomFilterSpaceUsed = format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "BloomFilterDiskSpaceUsed"), humanReadable); + statsTable.bloomFilterSpaceUsed = FileUtils.stringifyFileSize((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "BloomFilterDiskSpaceUsed"), humanReadable); if (bloomFilterOffHeapSize != null) { statsTable.bloomFilterOffHeapUsed = true; - statsTable.bloomFilterOffHeapMemoryUsed = format(bloomFilterOffHeapSize, humanReadable); + statsTable.bloomFilterOffHeapMemoryUsed = FileUtils.stringifyFileSize(bloomFilterOffHeapSize, humanReadable); } if (indexSummaryOffHeapSize != null) { statsTable.indexSummaryOffHeapUsed = true; - statsTable.indexSummaryOffHeapMemoryUsed = format(indexSummaryOffHeapSize, humanReadable); + statsTable.indexSummaryOffHeapMemoryUsed = FileUtils.stringifyFileSize(indexSummaryOffHeapSize, humanReadable); } if (compressionMetadataOffHeapSize != null) { statsTable.compressionMetadataOffHeapUsed = true; - statsTable.compressionMetadataOffHeapMemoryUsed = format(compressionMetadataOffHeapSize, humanReadable); + statsTable.compressionMetadataOffHeapMemoryUsed = FileUtils.stringifyFileSize(compressionMetadataOffHeapSize, humanReadable); } statsTable.compactedPartitionMinimumBytes = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MinPartitionSize"); statsTable.compactedPartitionMaximumBytes = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MaxPartitionSize"); @@ -420,16 +420,11 @@ private void maybeAddTWCSWindowWithMaxDuration(StatsTable statsTable, NodeProbe statsTable.twcs = String.format("%s %s, max duration: %s", size, unit, maxDuration); } - private String format(long bytes, boolean humanReadable) - { - return humanReadable ? FileUtils.stringifyFileSize(bytes) : Long.toString(bytes); - } - private Map<String, String> format(Map<String, Long> map, boolean humanReadable) { LinkedHashMap<String, String> retMap = new LinkedHashMap<>(); for (Map.Entry<String, Long> entry : map.entrySet()) - retMap.put(entry.getKey(), format(entry.getValue(), humanReadable)); + retMap.put(entry.getKey(), FileUtils.stringifyFileSize(entry.getValue(), humanReadable)); return retMap; } From 46b36f23cd865ee905742ff1d3da05fc8a467758 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ata=20=C4=B0lhan=20K=C3=B6kt=C3=BCrk?= <atailhan2006@gmail.com> Date: Sun, 29 Sep 2024 12:31:34 +0800 Subject: [PATCH 006/225] Explicitly localize strings to Locale.US for internal implementation MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit There is inconsistent usage of toLowerCase/toUpperCase methods in the codebase. Somewhere we already use Locale.US, somewhere not. That might cause various issues in runtime as shown in CASSANDRA-19953 ticket when an environment Cassandra runs in has different locale from expected. In this patch, all such method calls explicitly set their locale to Locale.US. This patch also contains a checkstyle rule which fails the compilation if toLowerCase or toUpperCase is used. We prefer calls to methods in LocalizeString class which use US locale. patch by Ata İlhan Köktürk; reviewed by Stefan Miklosovic, Brandon Williams for CASSANDRA-19953 Co-authored-by: Maxwell Guo <cclive1601@gmail.com> Co-authored-by: Stefan Miklosovic <smiklosovic@apacheorg> --- .build/checkstyle.xml | 16 ++++- CHANGES.txt | 1 + src/antlr/Cql.g | 1 + src/antlr/Parser.g | 6 +- .../cassandra/audit/AuditLogManager.java | 4 +- .../cassandra/audit/AuditLogOptions.java | 12 ++-- .../apache/cassandra/concurrent/Stage.java | 3 +- .../config/CassandraRelevantProperties.java | 12 ++-- .../apache/cassandra/config/DataRateSpec.java | 7 ++- .../cassandra/config/DataStorageSpec.java | 7 ++- .../cassandra/config/DatabaseDescriptor.java | 6 +- .../apache/cassandra/config/DurationSpec.java | 10 ++-- .../cassandra/config/EncryptionOptions.java | 12 ++-- .../org/apache/cassandra/cql3/CQL3Type.java | 3 +- .../cassandra/cql3/ColumnIdentifier.java | 7 ++- .../org/apache/cassandra/cql3/Duration.java | 3 +- .../cassandra/cql3/FieldIdentifier.java | 5 +- .../cassandra/cql3/PasswordObfuscator.java | 6 +- .../apache/cassandra/cql3/QualifiedName.java | 5 +- .../cassandra/cql3/ReservedKeywords.java | 4 +- .../org/apache/cassandra/cql3/RoleName.java | 4 +- .../apache/cassandra/cql3/SchemaElement.java | 5 +- .../cassandra/cql3/functions/CastFcts.java | 3 +- .../cql3/functions/FunctionResolver.java | 3 +- .../functions/masking/MaskingFunction.java | 4 +- .../cql3/functions/types/DataType.java | 4 +- .../cql3/functions/types/Duration.java | 3 +- .../cql3/functions/types/Metadata.java | 4 +- .../cassandra/cql3/selection/Selectable.java | 3 +- .../cassandra/cql3/selection/Selection.java | 4 +- .../cql3/statements/PropertyDefinitions.java | 3 +- .../apache/cassandra/db/ConsistencyLevel.java | 6 +- .../org/apache/cassandra/db/Directories.java | 6 +- .../db/compaction/OperationType.java | 4 +- .../db/compaction/unified/Controller.java | 9 +-- .../cassandra/db/lifecycle/LogRecord.java | 4 +- .../cassandra/db/marshal/CollectionType.java | 7 ++- .../cassandra/db/marshal/EmptyType.java | 3 +- .../apache/cassandra/db/marshal/UserType.java | 3 +- .../cassandra/db/virtual/ClientsTable.java | 4 +- .../CollectionVirtualTableAdapter.java | 3 +- .../cassandra/db/virtual/GossipInfoTable.java | 9 +-- .../db/virtual/LocalRepairTables.java | 6 +- .../db/virtual/SSTableTasksTable.java | 6 +- .../db/virtual/StreamingVirtualTable.java | 8 ++- .../db/virtual/VirtualSchemaKeyspace.java | 5 +- .../sai/analyzer/filter/BasicFilters.java | 4 +- .../index/sai/disk/v1/IndexWriterConfig.java | 5 +- .../index/sai/disk/v1/vector/OptimizeFor.java | 3 +- .../analyzer/filter/BasicResultFilters.java | 7 ++- .../index/sasi/disk/OnDiskIndexBuilder.java | 4 +- .../metrics/CassandraMetricsRegistry.java | 6 +- .../cassandra/schema/CQLTypeParser.java | 8 ++- .../cassandra/schema/CachingParams.java | 3 +- .../cassandra/schema/CompactionParams.java | 8 ++- .../cassandra/schema/IndexMetadata.java | 5 +- .../cassandra/schema/KeyspaceParams.java | 4 +- .../cassandra/schema/SchemaConstants.java | 8 ++- .../cassandra/schema/SchemaKeyspace.java | 16 ++--- .../apache/cassandra/schema/TableParams.java | 3 +- .../cassandra/security/JKSKeyProvider.java | 4 +- .../apache/cassandra/security/SSLFactory.java | 3 +- .../serializers/CollectionSerializer.java | 5 +- .../cassandra/service/StartupChecks.java | 3 +- .../cassandra/service/StorageProxy.java | 3 +- .../service/paxos/ContentionStrategy.java | 3 +- .../reads/HybridSpeculativeRetryPolicy.java | 4 +- .../reads/repair/ReadRepairStrategy.java | 4 +- .../streaming/StreamResultFuture.java | 3 +- .../cassandra/streaming/StreamingState.java | 5 +- .../apache/cassandra/tcm/MetadataKeys.java | 4 +- .../apache/cassandra/tools/RepairRunner.java | 5 +- .../cassandra/tools/StandaloneScrubber.java | 6 +- .../cassandra/tools/nodetool/ProfileLoad.java | 3 +- .../apache/cassandra/transport/CBUtil.java | 5 +- .../apache/cassandra/transport/Client.java | 13 ++-- .../transport/messages/StartupMessage.java | 7 ++- .../apache/cassandra/utils/FBUtilities.java | 7 ++- .../apache/cassandra/utils/GuidGenerator.java | 3 +- .../org/apache/cassandra/utils/JsonUtils.java | 4 +- .../cassandra/utils/LocalizeString.java | 60 +++++++++++++++++++ .../apache/cassandra/utils/NativeLibrary.java | 5 +- .../utils/NativeSSTableLoaderClient.java | 8 ++- .../apache/cassandra/utils/binlog/BinLog.java | 3 +- .../apache/cassandra/distributed/Cluster.java | 4 +- .../distributed/test/FailingRepairTest.java | 4 +- .../test/ReadDigestConsistencyTest.java | 4 +- .../test/RepairCoordinatorBase.java | 3 +- .../RepairCoordinatorFailingMessageTest.java | 3 +- .../test/SSTableIdGenerationTest.java | 5 +- .../compaction/CompactionAllocationTest.java | 10 ++-- .../apache/cassandra/simulator/Action.java | 2 +- .../cassandra/simulator/SimulationRunner.java | 5 +- .../org/apache/cassandra/SchemaLoader.java | 3 +- .../auth/CassandraNetworkAuthorizerTest.java | 3 +- .../cassandra/config/DataStorageSpecTest.java | 3 +- .../config/DatabaseDescriptorRefTest.java | 1 + .../org/apache/cassandra/cql3/CQLTester.java | 8 +-- .../cassandra/cql3/KeywordTestBase.java | 4 +- .../cql3/functions/NativeFunctionsTest.java | 8 ++- .../masking/PartialMaskingFunctionTest.java | 3 +- .../entities/SecondaryIndexTest.java | 12 ++-- .../db/compaction/unified/ControllerTest.java | 3 +- .../db/guardrails/GuardrailPasswordTest.java | 3 +- .../cassandra/db/tries/TrieToDotTest.java | 4 +- .../cassandra/db/tries/TrieToMermaidTest.java | 4 +- .../db/virtual/LocalRepairTablesTest.java | 4 +- .../db/virtual/SSTableTasksTableTest.java | 3 +- .../db/virtual/StreamingVirtualTableTest.java | 5 +- .../sai/analyzer/filter/BasicFiltersTest.java | 3 +- .../sasi/analyzer/DelimiterAnalyzerTest.java | 5 +- .../analyzer/NonTokenizingAnalyzerTest.java | 6 +- .../reads/repair/AbstractReadRepairTest.java | 3 +- .../cassandra/tools/TopPartitionsTest.java | 4 +- .../apache/cassandra/stress/StressAction.java | 3 +- .../apache/cassandra/stress/StressGraph.java | 4 +- .../cassandra/stress/StressProfile.java | 17 +++--- .../cassandra/stress/settings/CliOption.java | 6 +- .../cassandra/stress/settings/Command.java | 12 ++-- .../stress/settings/OptionDistribution.java | 8 ++- .../settings/OptionEnumProbabilities.java | 3 +- .../stress/settings/SettingsCommand.java | 15 +++-- .../settings/SettingsCommandPreDefined.java | 6 +- .../stress/settings/SettingsLog.java | 4 +- .../stress/settings/SettingsMisc.java | 8 ++- .../stress/settings/SettingsMode.java | 3 +- .../stress/settings/SettingsPopulation.java | 4 +- .../stress/settings/StressSettings.java | 4 +- 128 files changed, 493 insertions(+), 243 deletions(-) create mode 100644 src/java/org/apache/cassandra/utils/LocalizeString.java diff --git a/.build/checkstyle.xml b/.build/checkstyle.xml index 8b81f21281d6..bc9b5c02dec8 100644 --- a/.build/checkstyle.xml +++ b/.build/checkstyle.xml @@ -62,7 +62,13 @@ <property name="idFormat" value="blockPathToFile"/> <property name="influenceFormat" value="0"/> </module> - + + <module name="SuppressWithNearbyCommentFilter"> + <property name="commentFormat" value="checkstyle: permit this invocation"/> + <property name="idFormat" value="blockToCases"/> + <property name="influenceFormat" value="0"/> + </module> + <module name="RegexpSinglelineJava"> <!-- block system time --> <property name="id" value="blockSystemClock"/> @@ -151,6 +157,14 @@ <property name="message" value="Avoid Short() and use Short.valueOf()" /> </module> + <module name="RegexpSinglelineJava"> + <!-- block toLowerCase and toUpperCase --> + <property name="id" value="blockToCases"/> + <property name="format" value="toLowerCase\(|toUpperCase\("/> + <property name="ignoreComments" value="true"/> + <property name="message" value="Avoid toLowerCase() or toUpperCase() and use LocalizeString class instead" /> + </module> + <module name="SuppressionCommentFilter"> <property name="offCommentFormat" value="checkstyle: suppress below '([\w\|]+)'"/> <property name="idFormat" value="$1"/> diff --git a/CHANGES.txt b/CHANGES.txt index 7e5d6fede41d..f4e9d0f5fab4 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Explicitly localize strings to Locale.US for internal implementation (CASSANDRA-19953) * Add -H option for human-friendly output in nodetool compactionhistory (CASSANDRA-20015) * Fix type check for referenced duration type for nested types (CASSANDRA-19890) * In simulation tests, correctly set the tokens of replacement nodes (CASSANDRA-19997) diff --git a/src/antlr/Cql.g b/src/antlr/Cql.g index b25f3944d6cf..5cf8c88a3143 100644 --- a/src/antlr/Cql.g +++ b/src/antlr/Cql.g @@ -50,6 +50,7 @@ import Parser,Lexer; import org.apache.cassandra.exceptions.SyntaxException; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.utils.Pair; + import org.apache.cassandra.utils.LocalizeString; } @members { diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g index 099feaa1db2e..15c2a0140bbd 100644 --- a/src/antlr/Parser.g +++ b/src/antlr/Parser.g @@ -722,7 +722,7 @@ createFunctionStatement returns [CreateFunctionStatement.Raw stmt] K_LANGUAGE language = IDENT K_AS body = STRING_LITERAL { $stmt = new CreateFunctionStatement.Raw( - fn, argNames, argTypes, returnType, calledOnNullInput, $language.text.toLowerCase(), $body.text, orReplace, ifNotExists); + fn, argNames, argTypes, returnType, calledOnNullInput, LocalizeString.toLowerCaseLocalized($language.text), $body.text, orReplace, ifNotExists); } ; @@ -1127,7 +1127,7 @@ listPermissionsStatement returns [ListPermissionsStatement stmt] permission returns [Permission perm] : p=(K_CREATE | K_ALTER | K_DROP | K_SELECT | K_MODIFY | K_AUTHORIZE | K_DESCRIBE | K_EXECUTE | K_UNMASK | K_SELECT_MASKED) - { $perm = Permission.valueOf($p.text.toUpperCase()); } + { $perm = Permission.valueOf(LocalizeString.toUpperCaseLocalized($p.text)); } ; permissionOrAll returns [Set<Permission> perms] @@ -1642,7 +1642,7 @@ functionName returns [FunctionName s] ; allowedFunctionName returns [String s] - : f=IDENT { $s = $f.text.toLowerCase(); } + : f=IDENT { $s = LocalizeString.toLowerCaseLocalized($f.text); } | f=QUOTED_NAME { $s = $f.text; } | u=unreserved_function_keyword { $s = u; } | K_TOKEN { $s = "token"; } diff --git a/src/java/org/apache/cassandra/audit/AuditLogManager.java b/src/java/org/apache/cassandra/audit/AuditLogManager.java index 85f754f019c8..0f49a540603d 100644 --- a/src/java/org/apache/cassandra/audit/AuditLogManager.java +++ b/src/java/org/apache/cassandra/audit/AuditLogManager.java @@ -51,6 +51,8 @@ import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.MBeanWrapper; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + /** * Central location for managing the logging of client/user-initated actions (like queries, log in commands, and so on). * @@ -387,7 +389,7 @@ private String obfuscatePasswordInformation(Exception e, List<String> queries) { for (String query : queries) { - if (query.toLowerCase().contains(PasswordObfuscator.PASSWORD_TOKEN)) + if (toLowerCaseLocalized(query).contains(PasswordObfuscator.PASSWORD_TOKEN)) return "Syntax Exception. Obscured for security reasons."; } } diff --git a/src/java/org/apache/cassandra/audit/AuditLogOptions.java b/src/java/org/apache/cassandra/audit/AuditLogOptions.java index e9e31c9040c5..196bd9d0ef22 100644 --- a/src/java/org/apache/cassandra/audit/AuditLogOptions.java +++ b/src/java/org/apache/cassandra/audit/AuditLogOptions.java @@ -34,6 +34,8 @@ import org.apache.cassandra.io.util.File; import org.apache.cassandra.utils.binlog.BinLogOptions; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; + public class AuditLogOptions extends BinLogOptions { public volatile boolean enabled = false; @@ -143,13 +145,13 @@ public Builder withExcludedKeyspaces(final String excludedKeyspaces) public Builder withIncludedCategories(final String includedCategories) { - sanitise(includedCategories).map(v -> this.includedCategories = v.toUpperCase()); + sanitise(includedCategories).map(v -> this.includedCategories = toUpperCaseLocalized(v)); return this; } public Builder withExcludedCategories(final String excludedCategories) { - sanitise(excludedCategories).map(v -> this.excludedCategories = v.toUpperCase()); + sanitise(excludedCategories).map(v -> this.excludedCategories = toUpperCaseLocalized(v)); return this; } @@ -173,7 +175,7 @@ public Builder withAuditLogDir(final String auditLogDir) public Builder withRollCycle(final String rollCycle) { - sanitise(rollCycle).map(v -> this.rollCycle = v.toUpperCase()); + sanitise(rollCycle).map(v -> this.rollCycle = toUpperCaseLocalized(v)); return this; } @@ -230,8 +232,8 @@ public AuditLogOptions build() opts.logger = this.logger; sanitise(this.includedKeyspaces).map(v -> opts.included_keyspaces = v); sanitise(this.excludedKeyspaces).map(v -> opts.excluded_keyspaces = v); - sanitise(this.includedCategories).map(v -> opts.included_categories = v.toUpperCase()); - sanitise(this.excludedCategories).map(v -> opts.excluded_categories = v.toUpperCase()); + sanitise(this.includedCategories).map(v -> opts.included_categories = toUpperCaseLocalized(v)); + sanitise(this.excludedCategories).map(v -> opts.excluded_categories = toUpperCaseLocalized(v)); sanitise(this.includedUsers).map(v -> opts.included_users = v); sanitise(this.excludedUsers).map(v -> opts.excluded_users = v); opts.roll_cycle = this.rollCycle; diff --git a/src/java/org/apache/cassandra/concurrent/Stage.java b/src/java/org/apache/cassandra/concurrent/Stage.java index 4def1774229a..23f80b5a575f 100644 --- a/src/java/org/apache/cassandra/concurrent/Stage.java +++ b/src/java/org/apache/cassandra/concurrent/Stage.java @@ -39,6 +39,7 @@ import static java.util.stream.Collectors.toMap; import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; public enum Stage { @@ -79,7 +80,7 @@ public enum Stage private static String normalizeName(String stageName) { // Handle discrepancy between JMX names and actual pool names - String upperStageName = stageName.toUpperCase(); + String upperStageName = toUpperCaseLocalized(stageName); if (upperStageName.endsWith("STAGE")) { upperStageName = upperStageName.substring(0, stageName.length() - 5); diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java index b98d1b2d9d6a..fcf9f7d0e688 100644 --- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java +++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java @@ -32,6 +32,8 @@ import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.StorageCompatibilityMode; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; + // checkstyle: suppress below 'blockSystemPropertyUsage' /** A class that extracts system properties for the cassandra node it runs within. */ @@ -918,7 +920,7 @@ public Long setLong(long value) } /** - * Gets the value of a system property as a enum, calling {@link String#toUpperCase()} first. + * Gets the value of a system property as an enum, calling {@link org.apache.cassandra.utils.LocalizeString#toUpperCaseLocalized(String)} first. * * @param defaultValue to return when not defined * @param <T> type @@ -930,7 +932,7 @@ public <T extends Enum<T>> T getEnum(T defaultValue) } /** - * Gets the value of a system property as a enum, optionally calling {@link String#toUpperCase()} first. + * Gets the value of a system property as an enum, optionally calling {@link org.apache.cassandra.utils.LocalizeString#toUpperCaseLocalized(String)} first. * * @param toUppercase before converting to enum * @param defaultValue to return when not defined @@ -942,11 +944,11 @@ public <T extends Enum<T>> T getEnum(boolean toUppercase, T defaultValue) String value = System.getProperty(key); if (value == null) return defaultValue; - return Enum.valueOf(defaultValue.getDeclaringClass(), toUppercase ? value.toUpperCase() : value); + return Enum.valueOf(defaultValue.getDeclaringClass(), toUppercase ? toUpperCaseLocalized(value) : value); } /** - * Gets the value of a system property as an enum, optionally calling {@link String#toUpperCase()} first. + * Gets the value of a system property as an enum, optionally calling {@link org.apache.cassandra.utils.LocalizeString#toLowerCaseLocalized(String)} first. * If the value is missing, the default value for this property is used * * @param toUppercase before converting to enum @@ -957,7 +959,7 @@ public <T extends Enum<T>> T getEnum(boolean toUppercase, T defaultValue) public <T extends Enum<T>> T getEnum(boolean toUppercase, Class<T> enumClass) { String value = System.getProperty(key, defaultVal); - return Enum.valueOf(enumClass, toUppercase ? value.toUpperCase() : value); + return Enum.valueOf(enumClass, toUppercase ? toUpperCaseLocalized(value) : value); } /** diff --git a/src/java/org/apache/cassandra/config/DataRateSpec.java b/src/java/org/apache/cassandra/config/DataRateSpec.java index d496cfa67169..a89cd8413ec8 100644 --- a/src/java/org/apache/cassandra/config/DataRateSpec.java +++ b/src/java/org/apache/cassandra/config/DataRateSpec.java @@ -27,6 +27,7 @@ import com.google.common.primitives.Ints; import static org.apache.cassandra.config.DataRateSpec.DataRateUnit.BYTES_PER_SECOND; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; /** * Represents a data rate type used for cassandra configuration. It supports the opportunity for the users to be able to @@ -76,7 +77,7 @@ private static void validateQuantity(String value, double quantity, DataRateUnit // negatives are not allowed by the regex pattern if (minUnit.convert(quantity, unit) >= max) throw new IllegalArgumentException("Invalid data rate: " + value + ". It shouldn't be more than " + - (max - 1) + " in " + minUnit.name().toLowerCase()); + (max - 1) + " in " + toLowerCaseLocalized(minUnit.name())); } private static void validateQuantity(double quantity, DataRateUnit unit, DataRateUnit minUnit, long max) @@ -86,8 +87,8 @@ private static void validateQuantity(double quantity, DataRateUnit unit, DataRat if (minUnit.convert(quantity, unit) >= max) throw new IllegalArgumentException(String.format("Invalid data rate: %s %s. It shouldn't be more than %d in %s", - quantity, unit.name().toLowerCase(), - max - 1, minUnit.name().toLowerCase())); + quantity, toLowerCaseLocalized(unit.name()), + max - 1, toLowerCaseLocalized(minUnit.name()))); } // get vs no-get prefix is not consistent in the code base, but for classes involved with config parsing, it is diff --git a/src/java/org/apache/cassandra/config/DataStorageSpec.java b/src/java/org/apache/cassandra/config/DataStorageSpec.java index b5d4374088ae..53d71717eff3 100644 --- a/src/java/org/apache/cassandra/config/DataStorageSpec.java +++ b/src/java/org/apache/cassandra/config/DataStorageSpec.java @@ -28,6 +28,7 @@ import static org.apache.cassandra.config.DataStorageSpec.DataStorageUnit.BYTES; import static org.apache.cassandra.config.DataStorageSpec.DataStorageUnit.KIBIBYTES; import static org.apache.cassandra.config.DataStorageSpec.DataStorageUnit.MEBIBYTES; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; /** * Represents an amount of data storage. Wrapper class for Cassandra configuration parameters, providing to the @@ -99,7 +100,7 @@ private static void validateQuantity(String value, long quantity, DataStorageUni if (minUnit.convert(quantity, sourceUnit) >= max) throw new IllegalArgumentException("Invalid data storage: " + value + ". It shouldn't be more than " + - (max - 1) + " in " + minUnit.name().toLowerCase()); + (max - 1) + " in " + toLowerCaseLocalized(minUnit.name())); } private static void validateQuantity(long quantity, DataStorageUnit sourceUnit, DataStorageUnit minUnit, long max) @@ -109,8 +110,8 @@ private static void validateQuantity(long quantity, DataStorageUnit sourceUnit, if (minUnit.convert(quantity, sourceUnit) >= max) throw new IllegalArgumentException(String.format("Invalid data storage: %d %s. It shouldn't be more than %d in %s", - quantity, sourceUnit.name().toLowerCase(), - max - 1, minUnit.name().toLowerCase())); + quantity, toLowerCaseLocalized(sourceUnit.name()), + max - 1, toLowerCaseLocalized(minUnit.name()))); } // get vs no-get prefix is not consistent in the code base, but for classes involved with config parsing, it is diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 7699a809a463..784a07932034 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -62,7 +62,7 @@ import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import com.google.common.util.concurrent.RateLimiter; -import org.apache.cassandra.utils.Pair; + import org.apache.commons.lang3.ArrayUtils; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; @@ -116,6 +116,7 @@ import org.apache.cassandra.service.paxos.Paxos; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.MBeanWrapper; +import org.apache.cassandra.utils.Pair; import org.apache.cassandra.utils.StorageCompatibilityMode; import static org.apache.cassandra.config.CassandraRelevantProperties.ALLOCATE_TOKENS_FOR_KEYSPACE; @@ -156,6 +157,7 @@ import static org.apache.cassandra.io.util.FileUtils.ONE_GIB; import static org.apache.cassandra.io.util.FileUtils.ONE_MIB; import static org.apache.cassandra.utils.Clock.Global.logInitializationOutcome; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; public class DatabaseDescriptor { @@ -1785,7 +1787,7 @@ public static ICIDRAuthorizer.CIDRAuthorizerMode getCidrAuthorizerMode() if (cidrAuthorizerMode == null || cidrAuthorizerMode.isEmpty()) return defaultCidrAuthorizerMode; - return ICIDRAuthorizer.CIDRAuthorizerMode.valueOf(cidrAuthorizerMode.toUpperCase()); + return ICIDRAuthorizer.CIDRAuthorizerMode.valueOf(toUpperCaseLocalized(cidrAuthorizerMode)); } public static int getCidrGroupsCacheRefreshInterval() diff --git a/src/java/org/apache/cassandra/config/DurationSpec.java b/src/java/org/apache/cassandra/config/DurationSpec.java index 2522d86124f5..3a9a03ab1fb2 100644 --- a/src/java/org/apache/cassandra/config/DurationSpec.java +++ b/src/java/org/apache/cassandra/config/DurationSpec.java @@ -34,6 +34,8 @@ import static java.util.concurrent.TimeUnit.NANOSECONDS; import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + /** * Represents a positive time duration. Wrapper class for Cassandra duration configuration parameters, providing to the * users the opportunity to be able to provide config with a unit of their choice in cassandra.yaml as per the available @@ -108,7 +110,7 @@ private static void validateQuantity(String value, long quantity, TimeUnit sourc if (minUnit.convert(quantity, sourceUnit) >= max) throw new IllegalArgumentException("Invalid duration: " + value + ". It shouldn't be more than " + - (max - 1) + " in " + minUnit.name().toLowerCase()); + (max - 1) + " in " + toLowerCaseLocalized(minUnit.name())); } private static void validateQuantity(long quantity, TimeUnit sourceUnit, TimeUnit minUnit, long max) @@ -118,8 +120,8 @@ private static void validateQuantity(long quantity, TimeUnit sourceUnit, TimeUni if (minUnit.convert(quantity, sourceUnit) >= max) throw new IllegalArgumentException(String.format("Invalid duration: %d %s. It shouldn't be more than %d in %s", - quantity, sourceUnit.name().toLowerCase(), - max - 1, minUnit.name().toLowerCase())); + quantity, toLowerCaseLocalized(sourceUnit.name()), + max - 1, toLowerCaseLocalized(minUnit.name()))); } // get vs no-get prefix is not consistent in the code base, but for classes involved with config parsing, it is @@ -141,7 +143,7 @@ public TimeUnit unit() */ static TimeUnit fromSymbol(String symbol) { - switch (symbol.toLowerCase()) + switch (toLowerCaseLocalized(symbol)) { case "d": return DAYS; case "h": return HOURS; diff --git a/src/java/org/apache/cassandra/config/EncryptionOptions.java b/src/java/org/apache/cassandra/config/EncryptionOptions.java index 6471eea9e34e..fdc5c0efc6ae 100644 --- a/src/java/org/apache/cassandra/config/EncryptionOptions.java +++ b/src/java/org/apache/cassandra/config/EncryptionOptions.java @@ -38,6 +38,8 @@ import org.apache.cassandra.security.ISslContextFactory; import org.apache.cassandra.utils.FBUtilities; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + /** * This holds various options used for enabling SSL/TLS encryption. * Examples of such options are: supported cipher-suites, ssl protocol with version, accepted protocols, end-point @@ -78,7 +80,7 @@ public enum ClientAuth for (ClientAuth clientAuth : ClientAuth.values()) { VALUES.put(clientAuth.value, clientAuth); - VALUES.put(clientAuth.name().toLowerCase(), clientAuth); + VALUES.put(toLowerCaseLocalized(clientAuth.name()), clientAuth); } } @@ -89,9 +91,9 @@ public enum ClientAuth public static ClientAuth from(String value) { - if (VALUES.containsKey(value.toLowerCase())) + if (VALUES.containsKey(toLowerCaseLocalized(value))) { - return VALUES.get(value.toLowerCase()); + return VALUES.get(toLowerCaseLocalized(value)); } throw new ConfigurationException(value + " is not a valid ClientAuth option"); } @@ -183,7 +185,7 @@ static Set<String> asSet() Set<String> valueSet = new HashSet<>(); ConfigKey[] values = values(); for(ConfigKey key: values) { - valueSet.add(key.getKeyName().toLowerCase()); + valueSet.add(toLowerCaseLocalized(key.getKeyName())); } return valueSet; } @@ -302,7 +304,7 @@ private void prepareSslContextFactoryParameterizedKeys(Map<String,Object> sslCon Set<String> configKeys = ConfigKey.asSet(); for (Map.Entry<String, String> entry : ssl_context_factory.parameters.entrySet()) { - if(configKeys.contains(entry.getKey().toLowerCase())) + if(configKeys.contains(toLowerCaseLocalized(entry.getKey()))) { throw new IllegalArgumentException("SslContextFactory "+ssl_context_factory.class_name+" should " + "configure '"+entry.getKey()+"' as encryption_options instead of" + diff --git a/src/java/org/apache/cassandra/cql3/CQL3Type.java b/src/java/org/apache/cassandra/cql3/CQL3Type.java index 7eaa109f0288..6e2fdcc22e25 100644 --- a/src/java/org/apache/cassandra/cql3/CQL3Type.java +++ b/src/java/org/apache/cassandra/cql3/CQL3Type.java @@ -42,6 +42,7 @@ import org.apache.cassandra.utils.ByteBufferUtil; import static java.util.stream.Collectors.toList; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; public interface CQL3Type { @@ -139,7 +140,7 @@ public String toCQLLiteral(ByteBuffer buffer) @Override public String toString() { - return super.toString().toLowerCase(); + return toLowerCaseLocalized(super.toString()); } } diff --git a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java index e4da92218b81..77cad3dd3226 100644 --- a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java +++ b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java @@ -19,7 +19,6 @@ import java.nio.ByteBuffer; import java.util.List; -import java.util.Locale; import java.util.concurrent.ConcurrentMap; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -34,6 +33,8 @@ import org.apache.cassandra.utils.ObjectSizes; import org.apache.cassandra.utils.memory.ByteBufferCloner; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + /** * Represents an identifer for a CQL column definition. * TODO : should support light-weight mode without text representation for when not interned @@ -109,7 +110,7 @@ private static long prefixComparison(ByteBuffer bytes) public ColumnIdentifier(String rawText, boolean keepCase) { - this.text = keepCase ? rawText : rawText.toLowerCase(Locale.US); + this.text = keepCase ? rawText : toLowerCaseLocalized(rawText); this.bytes = ByteBufferUtil.bytes(this.text); this.prefixComparison = prefixComparison(bytes); this.interned = false; @@ -140,7 +141,7 @@ public static ColumnIdentifier getInterned(ByteBuffer bytes, AbstractType<?> typ public static ColumnIdentifier getInterned(String rawText, boolean keepCase) { - String text = keepCase ? rawText : rawText.toLowerCase(Locale.US); + String text = keepCase ? rawText : toLowerCaseLocalized(rawText); ByteBuffer bytes = ByteBufferUtil.bytes(text); return getInterned(UTF8Type.instance, bytes, text); } diff --git a/src/java/org/apache/cassandra/cql3/Duration.java b/src/java/org/apache/cassandra/cql3/Duration.java index 1d6b9f7b98f6..7eb600658165 100644 --- a/src/java/org/apache/cassandra/cql3/Duration.java +++ b/src/java/org/apache/cassandra/cql3/Duration.java @@ -30,6 +30,7 @@ import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse; import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue; import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; import static org.apache.commons.lang3.time.DateUtils.MILLIS_PER_DAY; import io.netty.util.concurrent.FastThreadLocal; @@ -251,7 +252,7 @@ private static long groupAsLong(Matcher matcher, int group) private static Builder add(Builder builder, long number, String symbol) { - switch (symbol.toLowerCase()) + switch (toLowerCaseLocalized(symbol)) { case "y": return builder.addYears(number); case "mo": return builder.addMonths(number); diff --git a/src/java/org/apache/cassandra/cql3/FieldIdentifier.java b/src/java/org/apache/cassandra/cql3/FieldIdentifier.java index bdde98d3b079..c46208d474e2 100644 --- a/src/java/org/apache/cassandra/cql3/FieldIdentifier.java +++ b/src/java/org/apache/cassandra/cql3/FieldIdentifier.java @@ -17,13 +17,14 @@ */ package org.apache.cassandra.cql3; -import java.util.Locale; import java.nio.ByteBuffer; import org.apache.cassandra.db.marshal.UTF8Type; import org.apache.cassandra.exceptions.SyntaxException; import org.apache.cassandra.serializers.MarshalException; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + /** * Identifies a field in a UDT. */ @@ -41,7 +42,7 @@ public FieldIdentifier(ByteBuffer bytes) */ public static FieldIdentifier forUnquoted(String text) { - return new FieldIdentifier(convert(text == null ? null : text.toLowerCase(Locale.US))); + return new FieldIdentifier(convert(text == null ? null : toLowerCaseLocalized(text))); } /** diff --git a/src/java/org/apache/cassandra/cql3/PasswordObfuscator.java b/src/java/org/apache/cassandra/cql3/PasswordObfuscator.java index 8e18f34611f4..9b28db70e1ba 100644 --- a/src/java/org/apache/cassandra/cql3/PasswordObfuscator.java +++ b/src/java/org/apache/cassandra/cql3/PasswordObfuscator.java @@ -23,13 +23,15 @@ import org.apache.cassandra.auth.PasswordAuthenticator; import org.apache.cassandra.auth.RoleOptions; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + /** * Obfuscates passwords in a given string */ public class PasswordObfuscator { public static final String OBFUSCATION_TOKEN = "*******"; - public static final String PASSWORD_TOKEN = PasswordAuthenticator.PASSWORD_KEY.toLowerCase(); + public static final String PASSWORD_TOKEN = toLowerCaseLocalized(PasswordAuthenticator.PASSWORD_KEY); /** * Obfuscates everything after the first appearance password token @@ -42,7 +44,7 @@ public static String obfuscate(String sourceString) if (null == sourceString) return null; - int passwordTokenStartIndex = sourceString.toLowerCase().indexOf(PASSWORD_TOKEN); + int passwordTokenStartIndex = toLowerCaseLocalized(sourceString).indexOf(PASSWORD_TOKEN); if (passwordTokenStartIndex < 0) return sourceString; diff --git a/src/java/org/apache/cassandra/cql3/QualifiedName.java b/src/java/org/apache/cassandra/cql3/QualifiedName.java index a3f70d5d1ed5..51413d4df27a 100644 --- a/src/java/org/apache/cassandra/cql3/QualifiedName.java +++ b/src/java/org/apache/cassandra/cql3/QualifiedName.java @@ -17,9 +17,10 @@ */ package org.apache.cassandra.cql3; -import java.util.Locale; import java.util.Objects; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + /** * Class for the names of the keyspace-prefixed elements (e.g. table, index, view names) */ @@ -123,6 +124,6 @@ public boolean equals(Object o) */ private static String toInternalName(String name, boolean keepCase) { - return keepCase ? name : name.toLowerCase(Locale.US); + return keepCase ? name : toLowerCaseLocalized(name); } } diff --git a/src/java/org/apache/cassandra/cql3/ReservedKeywords.java b/src/java/org/apache/cassandra/cql3/ReservedKeywords.java index 39221b572523..49d196d11c85 100644 --- a/src/java/org/apache/cassandra/cql3/ReservedKeywords.java +++ b/src/java/org/apache/cassandra/cql3/ReservedKeywords.java @@ -30,6 +30,8 @@ import org.apache.cassandra.exceptions.ConfigurationException; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; + public final class ReservedKeywords { private static final String FILE_NAME = "reserved_keywords.txt"; @@ -58,6 +60,6 @@ private static Set<String> getFromResource() public static boolean isReserved(String text) { - return reservedKeywords.contains(text.toUpperCase()); + return reservedKeywords.contains(toUpperCaseLocalized(text)); } } diff --git a/src/java/org/apache/cassandra/cql3/RoleName.java b/src/java/org/apache/cassandra/cql3/RoleName.java index b50c17d71aeb..15fff135bbe8 100644 --- a/src/java/org/apache/cassandra/cql3/RoleName.java +++ b/src/java/org/apache/cassandra/cql3/RoleName.java @@ -17,7 +17,7 @@ */ package org.apache.cassandra.cql3; -import java.util.Locale; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; public class RoleName { @@ -25,7 +25,7 @@ public class RoleName public void setName(String name, boolean keepCase) { - this.name = keepCase ? name : (name == null ? name : name.toLowerCase(Locale.US)); + this.name = keepCase ? name : (name == null ? name : toLowerCaseLocalized(name)); } public boolean hasName() diff --git a/src/java/org/apache/cassandra/cql3/SchemaElement.java b/src/java/org/apache/cassandra/cql3/SchemaElement.java index 2520ecb529a8..4fd96f84b59f 100644 --- a/src/java/org/apache/cassandra/cql3/SchemaElement.java +++ b/src/java/org/apache/cassandra/cql3/SchemaElement.java @@ -18,7 +18,8 @@ package org.apache.cassandra.cql3; import java.util.Comparator; -import java.util.Locale; + +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; /** * A schema element (keyspace, udt, udf, uda, table, index, view). @@ -43,7 +44,7 @@ enum SchemaElementType @Override public String toString() { - return super.toString().toLowerCase(Locale.US); + return toLowerCaseLocalized(super.toString()); } } diff --git a/src/java/org/apache/cassandra/cql3/functions/CastFcts.java b/src/java/org/apache/cassandra/cql3/functions/CastFcts.java index 02a29e1f9bd2..f32d4798e7c6 100644 --- a/src/java/org/apache/cassandra/cql3/functions/CastFcts.java +++ b/src/java/org/apache/cassandra/cql3/functions/CastFcts.java @@ -45,6 +45,7 @@ import org.apache.cassandra.transport.ProtocolVersion; import static org.apache.cassandra.cql3.functions.TimeFcts.*; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; import org.apache.commons.lang3.StringUtils; @@ -196,7 +197,7 @@ private static <O, I> CastFunction<?, O> wrapJavaFunction(AbstractType<I> inputT private static String toLowerCaseString(CQL3Type type) { - return type.toString().toLowerCase(); + return toLowerCaseLocalized(type.toString()); } /** diff --git a/src/java/org/apache/cassandra/cql3/functions/FunctionResolver.java b/src/java/org/apache/cassandra/cql3/functions/FunctionResolver.java index 11cf24362c6b..960f049d9ac2 100644 --- a/src/java/org/apache/cassandra/cql3/functions/FunctionResolver.java +++ b/src/java/org/apache/cassandra/cql3/functions/FunctionResolver.java @@ -35,6 +35,7 @@ import static java.util.stream.Collectors.joining; import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; public final class FunctionResolver { @@ -46,7 +47,7 @@ public static ColumnSpecification makeArgSpec(String receiverKeyspace, String re { return new ColumnSpecification(receiverKeyspace, receiverTable, - new ColumnIdentifier("arg" + i + '(' + fun.name().toString().toLowerCase() + ')', true), + new ColumnIdentifier("arg" + i + '(' + toLowerCaseLocalized(fun.name().toString()) + ')', true), fun.argTypes().get(i)); } diff --git a/src/java/org/apache/cassandra/cql3/functions/masking/MaskingFunction.java b/src/java/org/apache/cassandra/cql3/functions/masking/MaskingFunction.java index 37bc79171199..88d9c24afa01 100644 --- a/src/java/org/apache/cassandra/cql3/functions/masking/MaskingFunction.java +++ b/src/java/org/apache/cassandra/cql3/functions/masking/MaskingFunction.java @@ -26,6 +26,8 @@ import org.apache.cassandra.cql3.functions.NativeScalarFunction; import org.apache.cassandra.db.marshal.AbstractType; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + /** * A {@link NativeScalarFunction} that totally or partially replaces the original value of a column value, * meant to obscure the real value of the column. @@ -56,7 +58,7 @@ protected static abstract class Factory extends FunctionFactory { public Factory(String name, FunctionParameter... parameters) { - super(NAME_PREFIX + name.toLowerCase(), parameters); + super(NAME_PREFIX + toLowerCaseLocalized(name), parameters); } } } diff --git a/src/java/org/apache/cassandra/cql3/functions/types/DataType.java b/src/java/org/apache/cassandra/cql3/functions/types/DataType.java index fafc721b99de..8a5dfd763152 100644 --- a/src/java/org/apache/cassandra/cql3/functions/types/DataType.java +++ b/src/java/org/apache/cassandra/cql3/functions/types/DataType.java @@ -23,6 +23,8 @@ import org.apache.cassandra.transport.ProtocolVersion; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + /** * Data types supported by cassandra. */ @@ -122,7 +124,7 @@ public boolean isCompatibleWith(Name that) @Override public String toString() { - return super.toString().toLowerCase(); + return toLowerCaseLocalized(super.toString()); } } diff --git a/src/java/org/apache/cassandra/cql3/functions/types/Duration.java b/src/java/org/apache/cassandra/cql3/functions/types/Duration.java index 71d3e931df7e..f1dd6e37be33 100644 --- a/src/java/org/apache/cassandra/cql3/functions/types/Duration.java +++ b/src/java/org/apache/cassandra/cql3/functions/types/Duration.java @@ -23,6 +23,7 @@ import com.google.common.base.Objects; import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; /** * Represents a duration. A duration stores separately months, days, and seconds due to the fact @@ -237,7 +238,7 @@ private static long groupAsLong(Matcher matcher, int group) private static Builder add(Builder builder, long number, String symbol) { - String s = symbol.toLowerCase(); + String s = toLowerCaseLocalized(symbol); if (s.equals("y")) { return builder.addYears(number); diff --git a/src/java/org/apache/cassandra/cql3/functions/types/Metadata.java b/src/java/org/apache/cassandra/cql3/functions/types/Metadata.java index 24ab05b0fef7..dd806abc09bf 100644 --- a/src/java/org/apache/cassandra/cql3/functions/types/Metadata.java +++ b/src/java/org/apache/cassandra/cql3/functions/types/Metadata.java @@ -19,6 +19,8 @@ import org.apache.cassandra.cql3.ColumnIdentifier; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + /** * Keeps metadata on the connected cluster, including known nodes and schema definitions. */ @@ -65,7 +67,7 @@ else if (!((c >= 48 && c <= 57) // 0-9 } if (isAlphanumeric) { - return id.toLowerCase(); + return toLowerCaseLocalized(id); } // Check if it's enclosed in quotes. If it is, remove them and unescape internal double quotes diff --git a/src/java/org/apache/cassandra/cql3/selection/Selectable.java b/src/java/org/apache/cassandra/cql3/selection/Selectable.java index 1fade9fa99de..9bb5ea0f91c6 100644 --- a/src/java/org/apache/cassandra/cql3/selection/Selectable.java +++ b/src/java/org/apache/cassandra/cql3/selection/Selectable.java @@ -42,6 +42,7 @@ import static org.apache.cassandra.cql3.selection.SelectorFactories.createFactoriesAndCollectColumnDefinitions; import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; public interface Selectable extends AssignmentTestable { @@ -447,7 +448,7 @@ public WithCast(Selectable arg, CQL3Type type) @Override public String toString() { - return String.format("cast(%s as %s)", arg, type.toString().toLowerCase()); + return String.format("cast(%s as %s)", arg, toLowerCaseLocalized(type.toString())); } public Selector.Factory newSelectorFactory(TableMetadata table, AbstractType<?> expectedType, List<ColumnMetadata> defs, VariableSpecifications boundNames) diff --git a/src/java/org/apache/cassandra/cql3/selection/Selection.java b/src/java/org/apache/cassandra/cql3/selection/Selection.java index da87f2619a3c..743da6934e05 100644 --- a/src/java/org/apache/cassandra/cql3/selection/Selection.java +++ b/src/java/org/apache/cassandra/cql3/selection/Selection.java @@ -37,6 +37,8 @@ import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.utils.JsonUtils; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + public abstract class Selection { /** @@ -327,7 +329,7 @@ private static List<ByteBuffer> rowToJson(List<ByteBuffer> row, sb.append(", "); String columnName = spec.name.toString(); - if (!columnName.equals(columnName.toLowerCase(Locale.US))) + if (!columnName.equals(toLowerCaseLocalized(columnName))) columnName = "\"" + columnName + "\""; sb.append('"'); diff --git a/src/java/org/apache/cassandra/cql3/statements/PropertyDefinitions.java b/src/java/org/apache/cassandra/cql3/statements/PropertyDefinitions.java index a80e9ae693de..9a51877c06eb 100644 --- a/src/java/org/apache/cassandra/cql3/statements/PropertyDefinitions.java +++ b/src/java/org/apache/cassandra/cql3/statements/PropertyDefinitions.java @@ -28,6 +28,7 @@ import org.apache.cassandra.exceptions.SyntaxException; import static java.lang.String.format; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; public class PropertyDefinitions { @@ -113,7 +114,7 @@ public static boolean parseBoolean(String key, String value) throws SyntaxExcept if (null == value) throw new IllegalArgumentException("value argument can't be null"); - String lowerCasedValue = value.toLowerCase(); + String lowerCasedValue = toLowerCaseLocalized(value); if (POSITIVE_PATTERN.matcher(lowerCasedValue).matches()) return true; diff --git a/src/java/org/apache/cassandra/db/ConsistencyLevel.java b/src/java/org/apache/cassandra/db/ConsistencyLevel.java index 7c21c1287a7a..73cdeb5c5c3d 100644 --- a/src/java/org/apache/cassandra/db/ConsistencyLevel.java +++ b/src/java/org/apache/cassandra/db/ConsistencyLevel.java @@ -17,9 +17,6 @@ */ package org.apache.cassandra.db; - -import java.util.Locale; - import com.carrotsearch.hppc.ObjectIntHashMap; import org.apache.cassandra.locator.Endpoints; import org.apache.cassandra.locator.InOurDc; @@ -31,6 +28,7 @@ import org.apache.cassandra.transport.ProtocolException; import static org.apache.cassandra.locator.Replicas.addToCountPerDc; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; public enum ConsistencyLevel { @@ -85,7 +83,7 @@ public static ConsistencyLevel fromCode(int code) public static ConsistencyLevel fromString(String str) { - return valueOf(str.toUpperCase(Locale.US)); + return valueOf(toUpperCaseLocalized(str)); } public static int quorumFor(AbstractReplicationStrategy replicationStrategy) diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java index cbe9d9032225..c6e30c961d79 100644 --- a/src/java/org/apache/cassandra/db/Directories.java +++ b/src/java/org/apache/cassandra/db/Directories.java @@ -79,6 +79,8 @@ import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.Pair; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + /** * Encapsulate handling of paths to the data files. * @@ -755,11 +757,11 @@ public static Optional<File> getBackupsDirectoryIfExists(File location) */ public static boolean isStoredInLocalSystemKeyspacesDataLocation(String keyspace, String table) { - String keyspaceName = keyspace.toLowerCase(); + String keyspaceName = toLowerCaseLocalized(keyspace); return SchemaConstants.LOCAL_SYSTEM_KEYSPACE_NAMES.contains(keyspaceName) && !(SchemaConstants.SYSTEM_KEYSPACE_NAME.equals(keyspaceName) - && SystemKeyspace.TABLES_SPLIT_ACROSS_MULTIPLE_DISKS.contains(table.toLowerCase())); + && SystemKeyspace.TABLES_SPLIT_ACROSS_MULTIPLE_DISKS.contains(toLowerCaseLocalized(table))); } public static class DataDirectory diff --git a/src/java/org/apache/cassandra/db/compaction/OperationType.java b/src/java/org/apache/cassandra/db/compaction/OperationType.java index 2a5ffc61e678..461067b45846 100644 --- a/src/java/org/apache/cassandra/db/compaction/OperationType.java +++ b/src/java/org/apache/cassandra/db/compaction/OperationType.java @@ -17,6 +17,8 @@ */ package org.apache.cassandra.db.compaction; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + public enum OperationType { /** Each modification here should be also applied to {@link org.apache.cassandra.tools.nodetool.Stop#compactionType} */ @@ -69,7 +71,7 @@ public enum OperationType OperationType(String type, boolean writesData, int priority) { this.type = type; - this.fileName = type.toLowerCase().replace(" ", ""); + this.fileName = toLowerCaseLocalized(type).replace(" ", ""); this.writesData = writesData; this.priority = priority; } diff --git a/src/java/org/apache/cassandra/db/compaction/unified/Controller.java b/src/java/org/apache/cassandra/db/compaction/unified/Controller.java index cd7a35d44d92..5faab2702787 100644 --- a/src/java/org/apache/cassandra/db/compaction/unified/Controller.java +++ b/src/java/org/apache/cassandra/db/compaction/unified/Controller.java @@ -25,18 +25,19 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; -import org.apache.cassandra.config.CassandraRelevantProperties; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.compaction.UnifiedCompactionStrategy; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.utils.Overlaps; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.MonotonicClock; +import org.apache.cassandra.utils.Overlaps; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; /** * The controller provides compaction parameters to the unified compaction strategy */ @@ -442,7 +443,7 @@ public static Controller fromOptions(ColumnFamilyStore cfs, Map<String, String> sstableGrowthModifier = FBUtilities.parsePercent(options.get(SSTABLE_GROWTH_OPTION)); Overlaps.InclusionMethod inclusionMethod = options.containsKey(OVERLAP_INCLUSION_METHOD_OPTION) - ? Overlaps.InclusionMethod.valueOf(options.get(OVERLAP_INCLUSION_METHOD_OPTION).toUpperCase()) + ? Overlaps.InclusionMethod.valueOf(toUpperCaseLocalized(options.get(OVERLAP_INCLUSION_METHOD_OPTION))) : DEFAULT_OVERLAP_INCLUSION_METHOD; return new Controller(cfs, @@ -583,7 +584,7 @@ public static Map<String, String> validateOptions(Map<String, String> options) t { try { - Overlaps.InclusionMethod.valueOf(s.toUpperCase()); + Overlaps.InclusionMethod.valueOf(toUpperCaseLocalized(s)); } catch (IllegalArgumentException e) { diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java b/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java index 34fd0daf91bf..5f45156eb6ad 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java +++ b/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java @@ -46,6 +46,8 @@ import org.apache.cassandra.io.util.PathUtils; import org.apache.cassandra.utils.FBUtilities; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; + /** * A decoded line in a transaction log file replica. * @@ -63,7 +65,7 @@ public enum Type public static Type fromPrefix(String prefix) { - return valueOf(prefix.toUpperCase()); + return valueOf(toUpperCaseLocalized(prefix)); } public boolean hasFile() diff --git a/src/java/org/apache/cassandra/db/marshal/CollectionType.java b/src/java/org/apache/cassandra/db/marshal/CollectionType.java index b4bcc78d4095..8c39dbab4f5b 100644 --- a/src/java/org/apache/cassandra/db/marshal/CollectionType.java +++ b/src/java/org/apache/cassandra/db/marshal/CollectionType.java @@ -25,7 +25,6 @@ import java.util.Iterator; import java.util.Objects; import java.util.function.Consumer; -import java.util.Locale; import org.apache.cassandra.cql3.CQL3Type; import org.apache.cassandra.cql3.ColumnSpecification; @@ -46,6 +45,8 @@ import org.apache.cassandra.utils.bytecomparable.ByteSource; import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + /** * The abstract validator that is the base for maps, sets and lists (both frozen and non-frozen). * @@ -85,7 +86,7 @@ public ColumnSpecification makeCollectionReceiver(ColumnSpecification collection @Override public String toString() { - return super.toString().toLowerCase(Locale.US); + return toLowerCaseLocalized(super.toString()); } } @@ -142,7 +143,7 @@ public boolean isCollection() public <V> void validate(V value, ValueAccessor<V> accessor) throws MarshalException { if (accessor.isEmpty(value)) - throw new MarshalException("Not enough bytes to read a " + kind.name().toLowerCase()); + throw new MarshalException("Not enough bytes to read a " + toLowerCaseLocalized(kind.name())); super.validate(value, accessor); } diff --git a/src/java/org/apache/cassandra/db/marshal/EmptyType.java b/src/java/org/apache/cassandra/db/marshal/EmptyType.java index 07ea3ff3738f..69ae2c2d4e37 100644 --- a/src/java/org/apache/cassandra/db/marshal/EmptyType.java +++ b/src/java/org/apache/cassandra/db/marshal/EmptyType.java @@ -39,6 +39,7 @@ import org.apache.cassandra.utils.NoSpamLogger; import static org.apache.cassandra.config.CassandraRelevantProperties.SERIALIZATION_EMPTY_TYPE_NONEMPTY_BEHAVIOR; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; /** * A type that only accept empty data. @@ -59,7 +60,7 @@ private static NonEmptyWriteBehavior parseNonEmptyWriteBehavior() return NonEmptyWriteBehavior.FAIL; try { - return NonEmptyWriteBehavior.valueOf(value.toUpperCase().trim()); + return NonEmptyWriteBehavior.valueOf(toUpperCaseLocalized(value).trim()); } catch (Exception e) { diff --git a/src/java/org/apache/cassandra/db/marshal/UserType.java b/src/java/org/apache/cassandra/db/marshal/UserType.java index 7addfef63d9f..bbd16e36d199 100644 --- a/src/java/org/apache/cassandra/db/marshal/UserType.java +++ b/src/java/org/apache/cassandra/db/marshal/UserType.java @@ -50,6 +50,7 @@ import static com.google.common.collect.Iterables.transform; import static org.apache.cassandra.config.CassandraRelevantProperties.TYPE_UDT_CONFLICT_BEHAVIOR; import static org.apache.cassandra.cql3.ColumnIdentifier.maybeQuote; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; /** * A user defined type. @@ -274,7 +275,7 @@ public String toJSONString(ByteBuffer buffer, ProtocolVersion protocolVersion) sb.append(", "); String name = stringFieldNames.get(i); - if (!name.equals(name.toLowerCase(Locale.US))) + if (!name.equals(toLowerCaseLocalized(name))) name = "\"" + name + "\""; sb.append('"'); diff --git a/src/java/org/apache/cassandra/db/virtual/ClientsTable.java b/src/java/org/apache/cassandra/db/virtual/ClientsTable.java index 8f99f3a77112..370808eec86b 100644 --- a/src/java/org/apache/cassandra/db/virtual/ClientsTable.java +++ b/src/java/org/apache/cassandra/db/virtual/ClientsTable.java @@ -32,6 +32,8 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.transport.ConnectedClient; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + final class ClientsTable extends AbstractVirtualTable { private static final String ADDRESS = "address"; @@ -88,7 +90,7 @@ public DataSet data() result.row(remoteAddress.getAddress(), remoteAddress.getPort()) .column(HOSTNAME, remoteAddress.getHostName()) .column(USERNAME, client.username().orElse(null)) - .column(CONNECTION_STAGE, client.stage().toString().toLowerCase()) + .column(CONNECTION_STAGE, toLowerCaseLocalized(client.stage().toString())) .column(PROTOCOL_VERSION, client.protocolVersion()) .column(CLIENT_OPTIONS, client.clientOptions().orElse(null)) .column(DRIVER_NAME, client.driverName().orElse(null)) diff --git a/src/java/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapter.java b/src/java/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapter.java index 56eacf5b3eca..c5079d73c0f0 100644 --- a/src/java/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapter.java +++ b/src/java/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapter.java @@ -85,6 +85,7 @@ import static org.apache.cassandra.db.rows.Cell.NO_DELETION_TIME; import static org.apache.cassandra.utils.FBUtilities.camelToSnake; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; /** * This is a virtual table that iteratively builds rows using a data set provided by internal collection. @@ -255,7 +256,7 @@ private static String camelToSnakeWithAbbreviations(String camel) Pattern pattern = Pattern.compile("^[A-Z1-9_]+$"); // Contains only uppercase letters, numbers and underscores, so it's already snake case. if (pattern.matcher(camel).matches()) - return camel.toLowerCase(); + return toLowerCaseLocalized(camel); // Some special cases must be handled manually. String modifiedCamel = camel; diff --git a/src/java/org/apache/cassandra/db/virtual/GossipInfoTable.java b/src/java/org/apache/cassandra/db/virtual/GossipInfoTable.java index bd612c6c21df..2051e6ef7b3a 100644 --- a/src/java/org/apache/cassandra/db/virtual/GossipInfoTable.java +++ b/src/java/org/apache/cassandra/db/virtual/GossipInfoTable.java @@ -35,6 +35,7 @@ import org.apache.cassandra.schema.TableMetadata; import static org.apache.cassandra.gms.ApplicationState.TOKENS; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; /** * A {@link VirtualTable} that return the Gossip information in tabular format. @@ -101,10 +102,10 @@ public DataSet data() .column(HEARTBEAT, getHeartBeat(localState)); for (ApplicationState state : STATES_FOR_VALUES) - dataSet.column(state.name().toLowerCase(), getValue(localState, state)); + dataSet.column(toLowerCaseLocalized(state.name()), getValue(localState, state)); for (ApplicationState state : STATES_FOR_VERSIONS) - dataSet.column(state.name().toLowerCase() + "_version", getVersion(localState, state)); + dataSet.column(toLowerCaseLocalized(state.name()) + "_version", getVersion(localState, state)); } return result; } @@ -178,10 +179,10 @@ private static TableMetadata buildTableMetadata(String keyspace) .addRegularColumn(HEARTBEAT, Int32Type.instance); for (ApplicationState state : STATES_FOR_VALUES) - builder.addRegularColumn(state.name().toLowerCase(), UTF8Type.instance); + builder.addRegularColumn(toLowerCaseLocalized(state.name()), UTF8Type.instance); for (ApplicationState state : STATES_FOR_VERSIONS) - builder.addRegularColumn(state.name().toLowerCase() + "_version", Int32Type.instance); + builder.addRegularColumn(toLowerCaseLocalized(state.name()) + "_version", Int32Type.instance); return builder.build(); } diff --git a/src/java/org/apache/cassandra/db/virtual/LocalRepairTables.java b/src/java/org/apache/cassandra/db/virtual/LocalRepairTables.java index b88502988e94..1c8c24b1716a 100644 --- a/src/java/org/apache/cassandra/db/virtual/LocalRepairTables.java +++ b/src/java/org/apache/cassandra/db/virtual/LocalRepairTables.java @@ -49,6 +49,8 @@ import org.apache.cassandra.service.ActiveRepairService; import org.apache.cassandra.utils.TimeUUID; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + public class LocalRepairTables { private LocalRepairTables() @@ -385,7 +387,7 @@ private void updateDataset(SimpleDataSet result, ValidationState state) private static String timestampColumnName(Enum<?> e) { - return timestampColumnName(e.name().toLowerCase()); + return timestampColumnName(toLowerCaseLocalized(e.name())); } private static String timestampColumnName(String e) @@ -431,7 +433,7 @@ private static <T extends Enum<T>> void addState(SimpleDataSet ds, State<T, ?> s T currentState = state.getStatus(); State.Result result = state.getResult(); - ds.column("status", result != null ? result.kind.name().toLowerCase() : currentState == null ? "init" : currentState.name().toLowerCase()); + ds.column("status", result != null ? toLowerCaseLocalized(result.kind.name()) : currentState == null ? "init" : toLowerCaseLocalized(currentState.name())); for (Map.Entry<T, Long> e : state.getStateTimesMillis().entrySet()) { if (e.getValue().longValue() != 0) diff --git a/src/java/org/apache/cassandra/db/virtual/SSTableTasksTable.java b/src/java/org/apache/cassandra/db/virtual/SSTableTasksTable.java index e2f38f8e9201..a2b80d9b8ff4 100644 --- a/src/java/org/apache/cassandra/db/virtual/SSTableTasksTable.java +++ b/src/java/org/apache/cassandra/db/virtual/SSTableTasksTable.java @@ -27,6 +27,8 @@ import org.apache.cassandra.dht.LocalPartitioner; import org.apache.cassandra.schema.TableMetadata; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + final class SSTableTasksTable extends AbstractVirtualTable { private final static String KEYSPACE_NAME = "keyspace_name"; @@ -74,11 +76,11 @@ public DataSet data() task.getTable().orElse("*"), task.getTaskId()) .column(COMPLETION_RATIO, completionRatio) - .column(KIND, task.getTaskType().toString().toLowerCase()) + .column(KIND, toLowerCaseLocalized(task.getTaskType().toString())) .column(PROGRESS, completed) .column(SSTABLES, task.getSSTables().size()) .column(TOTAL, total) - .column(UNIT, task.getUnit().toString().toLowerCase()) + .column(UNIT, toLowerCaseLocalized(task.getUnit().toString())) .column(TARGET_DIRECTORY, task.targetDirectory()); } diff --git a/src/java/org/apache/cassandra/db/virtual/StreamingVirtualTable.java b/src/java/org/apache/cassandra/db/virtual/StreamingVirtualTable.java index f01e79956db6..acfcede37403 100644 --- a/src/java/org/apache/cassandra/db/virtual/StreamingVirtualTable.java +++ b/src/java/org/apache/cassandra/db/virtual/StreamingVirtualTable.java @@ -28,6 +28,8 @@ import org.apache.cassandra.streaming.StreamingState; import org.apache.cassandra.utils.TimeUUID; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + import static org.apache.cassandra.cql3.statements.schema.CreateTableStatement.parse; public class StreamingVirtualTable extends AbstractVirtualTable @@ -60,7 +62,7 @@ private static String stateColumns() { StringBuilder sb = new StringBuilder(); for (StreamingState.Status state : StreamingState.Status.values()) - sb.append(" status_").append(state.name().toLowerCase()).append("_timestamp timestamp,\n"); + sb.append(" status_").append(toLowerCaseLocalized(state.name())).append("_timestamp timestamp,\n"); return sb.toString(); } @@ -91,13 +93,13 @@ private void updateDataSet(SimpleDataSet ds, StreamingState state) ds.column("follower", state.follower()); ds.column("operation", state.operation().getDescription()); ds.column("peers", state.peers().stream().map(Object::toString).collect(Collectors.toList())); - ds.column("status", state.status().name().toLowerCase()); + ds.column("status", toLowerCaseLocalized(state.status().name())); ds.column("progress_percentage", round(state.progress() * 100)); ds.column("duration_millis", state.durationMillis()); ds.column("failure_cause", state.failureCause()); ds.column("success_message", state.successMessage()); for (Map.Entry<StreamingState.Status, Long> e : state.stateTimesMillis().entrySet()) - ds.column("status_" + e.getKey().name().toLowerCase() + "_timestamp", new Date(e.getValue())); + ds.column("status_" + toLowerCaseLocalized(e.getKey().name()) + "_timestamp", new Date(e.getValue())); state.sessions().update(ds); } diff --git a/src/java/org/apache/cassandra/db/virtual/VirtualSchemaKeyspace.java b/src/java/org/apache/cassandra/db/virtual/VirtualSchemaKeyspace.java index bb5a430f0d86..43e373d6e3bc 100644 --- a/src/java/org/apache/cassandra/db/virtual/VirtualSchemaKeyspace.java +++ b/src/java/org/apache/cassandra/db/virtual/VirtualSchemaKeyspace.java @@ -29,6 +29,7 @@ import static org.apache.cassandra.schema.SchemaConstants.VIRTUAL_SCHEMA; import static org.apache.cassandra.schema.TableMetadata.builder; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; public final class VirtualSchemaKeyspace extends VirtualKeyspace { @@ -136,9 +137,9 @@ public DataSet data() for (ColumnMetadata column : table.columns()) { result.row(column.ksName, column.cfName, column.name.toString()) - .column(CLUSTERING_ORDER, column.clusteringOrder().toString().toLowerCase()) + .column(CLUSTERING_ORDER, toLowerCaseLocalized(column.clusteringOrder().toString())) .column(COLUMN_NAME_BYTES, column.name.bytes) - .column(KIND, column.kind.toString().toLowerCase()) + .column(KIND, toLowerCaseLocalized(column.kind.toString())) .column(POSITION, column.position()) .column(TYPE, column.type.asCQL3Type().toString()); } diff --git a/src/java/org/apache/cassandra/index/sai/analyzer/filter/BasicFilters.java b/src/java/org/apache/cassandra/index/sai/analyzer/filter/BasicFilters.java index b70fbae9da82..f81824f8eaf0 100644 --- a/src/java/org/apache/cassandra/index/sai/analyzer/filter/BasicFilters.java +++ b/src/java/org/apache/cassandra/index/sai/analyzer/filter/BasicFilters.java @@ -23,6 +23,8 @@ import org.apache.lucene.analysis.miscellaneous.ASCIIFoldingFilter; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + public class BasicFilters { private static final Locale DEFAULT_LOCALE = Locale.getDefault(); @@ -39,7 +41,7 @@ public LowerCase() @Override public String process(String input) { - return input.toLowerCase(locale); + return toLowerCaseLocalized(input, locale); } } diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/IndexWriterConfig.java b/src/java/org/apache/cassandra/index/sai/disk/v1/IndexWriterConfig.java index fe9be1c98df1..c6b060743715 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v1/IndexWriterConfig.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/IndexWriterConfig.java @@ -28,6 +28,7 @@ import org.apache.cassandra.index.sai.utils.IndexTermType; import static org.apache.cassandra.config.CassandraRelevantProperties.SAI_VECTOR_SEARCH_MAX_TOP_K; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; /** * Per-index config for storage-attached index writers. @@ -151,7 +152,7 @@ public static IndexWriterConfig fromOptions(String indexName, IndexTermType inde } if (options.containsKey(SIMILARITY_FUNCTION)) { - String option = options.get(SIMILARITY_FUNCTION).toUpperCase(); + String option = toUpperCaseLocalized(options.get(SIMILARITY_FUNCTION)); try { similarityFunction = VectorSimilarityFunction.valueOf(option); @@ -164,7 +165,7 @@ public static IndexWriterConfig fromOptions(String indexName, IndexTermType inde } if (options.containsKey(OPTIMIZE_FOR)) { - String option = options.get(OPTIMIZE_FOR).toUpperCase(); + String option = toUpperCaseLocalized(options.get(OPTIMIZE_FOR)); try { optimizeFor = OptimizeFor.valueOf(option); diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/vector/OptimizeFor.java b/src/java/org/apache/cassandra/index/sai/disk/v1/vector/OptimizeFor.java index 51e58b37df78..e2a566ea8ba0 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v1/vector/OptimizeFor.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/vector/OptimizeFor.java @@ -21,6 +21,7 @@ import java.util.function.Function; import static java.lang.Math.pow; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; /** * Allows the vector index searches to be optimised for latency or recall. This is used by the @@ -47,6 +48,6 @@ public int topKFor(int limit) public static OptimizeFor fromString(String value) { - return valueOf(value.toUpperCase()); + return valueOf(toUpperCaseLocalized(value)); } } diff --git a/src/java/org/apache/cassandra/index/sasi/analyzer/filter/BasicResultFilters.java b/src/java/org/apache/cassandra/index/sasi/analyzer/filter/BasicResultFilters.java index 2b949b898b25..bf0776410ccf 100644 --- a/src/java/org/apache/cassandra/index/sasi/analyzer/filter/BasicResultFilters.java +++ b/src/java/org/apache/cassandra/index/sasi/analyzer/filter/BasicResultFilters.java @@ -19,6 +19,9 @@ import java.util.Locale; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; + /** * Basic/General Token Filters */ @@ -42,7 +45,7 @@ public LowerCase() public String process(String input) throws Exception { - return input.toLowerCase(locale); + return toLowerCaseLocalized(input, locale); } } @@ -62,7 +65,7 @@ public UpperCase() public String process(String input) throws Exception { - return input.toUpperCase(locale); + return toUpperCaseLocalized(input, locale); } } diff --git a/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java b/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java index 9071e1088419..6930055048e4 100644 --- a/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java +++ b/src/java/org/apache/cassandra/index/sasi/disk/OnDiskIndexBuilder.java @@ -43,6 +43,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; + public class OnDiskIndexBuilder { private static final Logger logger = LoggerFactory.getLogger(OnDiskIndexBuilder.class); @@ -62,7 +64,7 @@ public enum Mode public static Mode mode(String mode) { - return Mode.valueOf(mode.toUpperCase()); + return Mode.valueOf(toUpperCaseLocalized(mode)); } public boolean supports(Op op) diff --git a/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java b/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java index dcd883a7187f..8cf83f520870 100644 --- a/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java +++ b/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java @@ -22,7 +22,6 @@ import java.util.Collection; import java.util.Collections; import java.util.List; -import java.util.Locale; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -74,6 +73,7 @@ import static org.apache.cassandra.db.virtual.CollectionVirtualTableAdapter.createSinglePartitionedKeyFiltered; import static org.apache.cassandra.db.virtual.CollectionVirtualTableAdapter.createSinglePartitionedValueFiltered; import static org.apache.cassandra.schema.SchemaConstants.VIRTUAL_METRICS; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; /** * Dropwizard metrics registry extension for Cassandra, as of for now uses the latest version of Dropwizard metrics @@ -810,7 +810,7 @@ public String getRateUnit() private String calculateRateUnit(TimeUnit unit) { - final String s = unit.toString().toLowerCase(Locale.US); + final String s = toLowerCaseLocalized(unit.toString()); return s.substring(0, s.length() - 1); } } @@ -861,7 +861,7 @@ private JmxTimer(Timer metric, { super(metric, objectName, rateUnit); this.metric = metric; - this.durationUnit = durationUnit.toString().toLowerCase(Locale.US); + this.durationUnit = toLowerCaseLocalized(durationUnit.toString()); } @Override diff --git a/src/java/org/apache/cassandra/schema/CQLTypeParser.java b/src/java/org/apache/cassandra/schema/CQLTypeParser.java index c79de881550e..ef1af7fda3a2 100644 --- a/src/java/org/apache/cassandra/schema/CQLTypeParser.java +++ b/src/java/org/apache/cassandra/schema/CQLTypeParser.java @@ -24,6 +24,8 @@ import org.apache.cassandra.db.marshal.UserType; import static org.apache.cassandra.utils.ByteBufferUtil.bytes; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; public final class CQLTypeParser { @@ -33,17 +35,17 @@ public final class CQLTypeParser { ImmutableSet.Builder<String> builder = ImmutableSet.builder(); for (CQL3Type.Native primitive : CQL3Type.Native.values()) - builder.add(primitive.name().toLowerCase()); + builder.add(toLowerCaseLocalized(primitive.name())); PRIMITIVE_TYPES = builder.build(); } public static AbstractType<?> parse(String keyspace, String unparsed, Types userTypes) { - String lowercased = unparsed.toLowerCase(); + String lowercased = toLowerCaseLocalized(unparsed); // fast path for the common case of a primitive type if (PRIMITIVE_TYPES.contains(lowercased)) - return CQL3Type.Native.valueOf(unparsed.toUpperCase()).getType(); + return CQL3Type.Native.valueOf(toUpperCaseLocalized(unparsed)).getType(); // special-case top-level UDTs UserType udt = userTypes.getNullable(bytes(lowercased)); diff --git a/src/java/org/apache/cassandra/schema/CachingParams.java b/src/java/org/apache/cassandra/schema/CachingParams.java index 1976835c89dd..f06eadb8db18 100644 --- a/src/java/org/apache/cassandra/schema/CachingParams.java +++ b/src/java/org/apache/cassandra/schema/CachingParams.java @@ -28,6 +28,7 @@ import org.apache.cassandra.exceptions.ConfigurationException; import static java.lang.String.format; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; // CQL: {'keys' : 'ALL'|'NONE', 'rows_per_partition': '200'|'NONE'|'ALL'} public final class CachingParams @@ -40,7 +41,7 @@ public enum Option @Override public String toString() { - return name().toLowerCase(); + return toLowerCaseLocalized(name()); } } diff --git a/src/java/org/apache/cassandra/schema/CompactionParams.java b/src/java/org/apache/cassandra/schema/CompactionParams.java index 7da6b50280eb..e082be487ba9 100644 --- a/src/java/org/apache/cassandra/schema/CompactionParams.java +++ b/src/java/org/apache/cassandra/schema/CompactionParams.java @@ -42,6 +42,8 @@ import static java.lang.String.format; import static org.apache.cassandra.config.CassandraRelevantProperties.DEFAULT_PROVIDE_OVERLAPPING_TOMBSTONES; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; public final class CompactionParams { @@ -58,7 +60,7 @@ public enum Option @Override public String toString() { - return name().toLowerCase(); + return toLowerCaseLocalized(name()); } } @@ -123,8 +125,8 @@ public static CompactionParams create(Class<? extends AbstractCompactionStrategy boolean isEnabled = options.containsKey(Option.ENABLED.toString()) ? Boolean.parseBoolean(options.get(Option.ENABLED.toString())) : DEFAULT_ENABLED; - String overlappingTombstoneParm = options.getOrDefault(Option.PROVIDE_OVERLAPPING_TOMBSTONES.toString(), - DEFAULT_PROVIDE_OVERLAPPING_TOMBSTONES_PROPERTY_VALUE.toString()).toUpperCase(); + String overlappingTombstoneParm = toUpperCaseLocalized(options.getOrDefault(Option.PROVIDE_OVERLAPPING_TOMBSTONES.toString(), + DEFAULT_PROVIDE_OVERLAPPING_TOMBSTONES_PROPERTY_VALUE.toString())); Optional<TombstoneOption> tombstoneOptional = TombstoneOption.forName(overlappingTombstoneParm); if (!tombstoneOptional.isPresent()) { diff --git a/src/java/org/apache/cassandra/schema/IndexMetadata.java b/src/java/org/apache/cassandra/schema/IndexMetadata.java index fb94c56f017a..1f73a5f98cb1 100644 --- a/src/java/org/apache/cassandra/schema/IndexMetadata.java +++ b/src/java/org/apache/cassandra/schema/IndexMetadata.java @@ -49,6 +49,7 @@ import org.apache.cassandra.utils.UUIDSerializer; import static org.apache.cassandra.db.TypeSizes.sizeof; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; /** * An immutable representation of secondary index metadata. @@ -72,7 +73,7 @@ public final class IndexMetadata static { indexNameAliases.put(StorageAttachedIndex.NAME, StorageAttachedIndex.class.getCanonicalName()); - indexNameAliases.put(StorageAttachedIndex.class.getSimpleName().toLowerCase(), StorageAttachedIndex.class.getCanonicalName()); + indexNameAliases.put(toLowerCaseLocalized(StorageAttachedIndex.class.getSimpleName()), StorageAttachedIndex.class.getCanonicalName()); indexNameAliases.put(SASIIndex.class.getSimpleName(), SASIIndex.class.getCanonicalName()); } @@ -159,7 +160,7 @@ public String getIndexClassName() if (isCustom()) { String className = options.get(IndexTarget.CUSTOM_INDEX_OPTION_NAME); - return indexNameAliases.getOrDefault(className.toLowerCase(), className); + return indexNameAliases.getOrDefault(toLowerCaseLocalized(className), className); } return CassandraIndex.class.getName(); } diff --git a/src/java/org/apache/cassandra/schema/KeyspaceParams.java b/src/java/org/apache/cassandra/schema/KeyspaceParams.java index 9cfaaa17f65d..76516334b8d5 100644 --- a/src/java/org/apache/cassandra/schema/KeyspaceParams.java +++ b/src/java/org/apache/cassandra/schema/KeyspaceParams.java @@ -32,6 +32,8 @@ import org.apache.cassandra.tcm.serialization.MetadataSerializer; import org.apache.cassandra.tcm.serialization.Version; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + /** * An immutable class representing keyspace parameters (durability and replication). */ @@ -57,7 +59,7 @@ public enum Option @Override public String toString() { - return name().toLowerCase(); + return toLowerCaseLocalized(name()); } } diff --git a/src/java/org/apache/cassandra/schema/SchemaConstants.java b/src/java/org/apache/cassandra/schema/SchemaConstants.java index bc733d367e1f..9a5e8844b2c3 100644 --- a/src/java/org/apache/cassandra/schema/SchemaConstants.java +++ b/src/java/org/apache/cassandra/schema/SchemaConstants.java @@ -32,6 +32,8 @@ import org.apache.cassandra.db.SystemKeyspace; import org.apache.cassandra.tracing.TraceKeyspace; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + /** * When adding new String keyspace names here, double check if it needs to be added to PartitionDenylist.canDenylistKeyspace */ @@ -93,7 +95,7 @@ public static boolean isValidName(String name) */ public static boolean isLocalSystemKeyspace(String keyspaceName) { - return LOCAL_SYSTEM_KEYSPACE_NAMES.contains(keyspaceName.toLowerCase()) || isVirtualSystemKeyspace(keyspaceName); + return LOCAL_SYSTEM_KEYSPACE_NAMES.contains(toLowerCaseLocalized(keyspaceName)) || isVirtualSystemKeyspace(keyspaceName); } /** @@ -101,7 +103,7 @@ public static boolean isLocalSystemKeyspace(String keyspaceName) */ public static boolean isReplicatedSystemKeyspace(String keyspaceName) { - return REPLICATED_SYSTEM_KEYSPACE_NAMES.contains(keyspaceName.toLowerCase()); + return REPLICATED_SYSTEM_KEYSPACE_NAMES.contains(toLowerCaseLocalized(keyspaceName)); } /** @@ -110,7 +112,7 @@ public static boolean isReplicatedSystemKeyspace(String keyspaceName) */ public static boolean isVirtualSystemKeyspace(String keyspaceName) { - return VIRTUAL_SYSTEM_KEYSPACE_NAMES.contains(keyspaceName.toLowerCase()); + return VIRTUAL_SYSTEM_KEYSPACE_NAMES.contains(toLowerCaseLocalized(keyspaceName)); } /** diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java index 8b748050e321..a6877cc6318c 100644 --- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java @@ -27,7 +27,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.*; -import com.google.common.collect.Maps; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,6 +62,8 @@ import static org.apache.cassandra.cql3.QueryProcessor.executeInternal; import static org.apache.cassandra.cql3.QueryProcessor.executeOnceInternal; import static org.apache.cassandra.schema.SchemaKeyspaceTables.*; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; import static org.apache.cassandra.utils.Simulate.With.GLOBAL_CLOCK; /** @@ -694,9 +696,9 @@ private static void addColumnToSchemaMutation(TableMetadata table, ColumnMetadat builder.update(Columns) .row(table.name, column.name.toString()) .add("column_name_bytes", column.name.bytes) - .add("kind", column.kind.toString().toLowerCase()) + .add("kind", toLowerCaseLocalized(column.kind.toString())) .add("position", column.position()) - .add("clustering_order", column.clusteringOrder().toString().toLowerCase()) + .add("clustering_order", toLowerCaseLocalized(column.clusteringOrder().toString())) .add("type", type.asCQL3Type().toString()); ColumnMask mask = column.getMask(); @@ -758,7 +760,7 @@ private static void addDroppedColumnToSchemaMutation(TableMetadata table, Droppe .row(table.name, column.column.name.toString()) .add("dropped_time", new Date(TimeUnit.MICROSECONDS.toMillis(column.droppedTime))) .add("type", column.column.type.asCQL3Type().toString()) - .add("kind", column.column.kind.toString().toLowerCase()); + .add("kind", toLowerCaseLocalized(column.column.kind.toString())); } private static void dropDroppedColumnFromSchemaMutation(TableMetadata table, DroppedColumn column, Mutation.SimpleBuilder builder) @@ -1075,10 +1077,10 @@ public static ColumnMetadata createColumnFromRow(UntypedResultSet.Row row, Types String keyspace = row.getString("keyspace_name"); String table = row.getString("table_name"); - ColumnMetadata.Kind kind = ColumnMetadata.Kind.valueOf(row.getString("kind").toUpperCase()); + ColumnMetadata.Kind kind = ColumnMetadata.Kind.valueOf(toUpperCaseLocalized(row.getString("kind"))); int position = row.getInt("position"); - ClusteringOrder order = ClusteringOrder.valueOf(row.getString("clustering_order").toUpperCase()); + ClusteringOrder order = ClusteringOrder.valueOf(toUpperCaseLocalized(row.getString("clustering_order"))); AbstractType<?> type = CQLTypeParser.parse(keyspace, row.getString("type"), types); if (order == ClusteringOrder.DESC) @@ -1159,7 +1161,7 @@ private static DroppedColumn createDroppedColumnFromRow(UntypedResultSet.Row row */ AbstractType<?> type = CQLTypeParser.parse(keyspace, row.getString("type"), org.apache.cassandra.schema.Types.none()); ColumnMetadata.Kind kind = row.has("kind") - ? ColumnMetadata.Kind.valueOf(row.getString("kind").toUpperCase()) + ? ColumnMetadata.Kind.valueOf(toUpperCaseLocalized(row.getString("kind"))) : ColumnMetadata.Kind.REGULAR; assert kind == ColumnMetadata.Kind.REGULAR || kind == ColumnMetadata.Kind.STATIC : "Unexpected dropped column kind: " + kind; diff --git a/src/java/org/apache/cassandra/schema/TableParams.java b/src/java/org/apache/cassandra/schema/TableParams.java index 2da45e5f97b3..6903179525b7 100644 --- a/src/java/org/apache/cassandra/schema/TableParams.java +++ b/src/java/org/apache/cassandra/schema/TableParams.java @@ -44,6 +44,7 @@ import static java.util.stream.Collectors.toMap; import static org.apache.cassandra.schema.TableParams.Option.*; import static org.apache.cassandra.db.TypeSizes.sizeof; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; public final class TableParams { @@ -73,7 +74,7 @@ public enum Option @Override public String toString() { - return name().toLowerCase(); + return toLowerCaseLocalized(name()); } } diff --git a/src/java/org/apache/cassandra/security/JKSKeyProvider.java b/src/java/org/apache/cassandra/security/JKSKeyProvider.java index 2fddf5e55f5a..eedacba39fc0 100644 --- a/src/java/org/apache/cassandra/security/JKSKeyProvider.java +++ b/src/java/org/apache/cassandra/security/JKSKeyProvider.java @@ -29,6 +29,8 @@ import org.apache.cassandra.config.TransparentDataEncryptionOptions; import org.apache.cassandra.io.util.File; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + /** * A {@code KeyProvider} that retrieves keys from a java keystore. */ @@ -64,7 +66,7 @@ public Key getSecretKey(String keyAlias) throws IOException { // there's a lovely behavior with jceks files that all aliases are lower-cased if (isJceks) - keyAlias = keyAlias.toLowerCase(); + keyAlias = toLowerCaseLocalized(keyAlias); Key key; try diff --git a/src/java/org/apache/cassandra/security/SSLFactory.java b/src/java/org/apache/cassandra/security/SSLFactory.java index c55e970b4f8b..a9b4be9d5c39 100644 --- a/src/java/org/apache/cassandra/security/SSLFactory.java +++ b/src/java/org/apache/cassandra/security/SSLFactory.java @@ -48,6 +48,7 @@ import static org.apache.cassandra.config.CassandraRelevantProperties.DISABLE_TCACTIVE_OPENSSL; import static org.apache.cassandra.config.EncryptionOptions.ClientAuth.REQUIRED; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; /** * A Factory for providing and setting up client {@link SSLSocket}s. Also provides @@ -337,7 +338,7 @@ public String[] filterCipherSuites(Iterable<String> ciphers, List<String> defaul if (settingDescription != null) { logger.warn("Dropping unsupported cipher_suite {} from {} configuration", - c, settingDescription.toLowerCase()); + c, toLowerCaseLocalized(settingDescription)); } } } diff --git a/src/java/org/apache/cassandra/serializers/CollectionSerializer.java b/src/java/org/apache/cassandra/serializers/CollectionSerializer.java index 74b790bf2983..6d38f2a781f9 100644 --- a/src/java/org/apache/cassandra/serializers/CollectionSerializer.java +++ b/src/java/org/apache/cassandra/serializers/CollectionSerializer.java @@ -23,7 +23,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; -import java.util.Locale; import java.util.function.Consumer; import com.google.common.collect.Range; @@ -34,6 +33,8 @@ import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.utils.ByteBufferUtil; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + public abstract class CollectionSerializer<T> extends TypeSerializer<T> { protected abstract List<ByteBuffer> serializeValues(T value); @@ -110,7 +111,7 @@ public <V> List<V> unpack(V input, ValueAccessor<V> accessor) */ private String getCollectionName() { - return getType().getSimpleName().toLowerCase(Locale.US); + return toLowerCaseLocalized(getType().getSimpleName()); } /** diff --git a/src/java/org/apache/cassandra/service/StartupChecks.java b/src/java/org/apache/cassandra/service/StartupChecks.java index 5dc11a6c6b7e..91a95a4e1069 100644 --- a/src/java/org/apache/cassandra/service/StartupChecks.java +++ b/src/java/org/apache/cassandra/service/StartupChecks.java @@ -80,6 +80,7 @@ import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_VERSION; import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_VM_NAME; import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; /** * Verifies that the system and environment is in a fit state to be started. @@ -217,7 +218,7 @@ public void execute(StartupChecksOptions startupChecksOptions) throws StartupExc { try { - if (affectedFileSystemTypes.contains(Files.getFileStore(path).type().toLowerCase())) + if (affectedFileSystemTypes.contains(toLowerCaseLocalized(Files.getFileStore(path).type()))) affectedPaths.add(path); } catch (IOException e) diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java index 055ce391c752..e62c1bb3bd81 100644 --- a/src/java/org/apache/cassandra/service/StorageProxy.java +++ b/src/java/org/apache/cassandra/service/StorageProxy.java @@ -177,6 +177,7 @@ import static org.apache.cassandra.service.paxos.v1.ProposeVerbHandler.doPropose; import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; import static org.apache.cassandra.utils.Clock.Global.nanoTime; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; import static org.apache.cassandra.utils.TimeUUID.Generator.nextTimeUUID; import static org.apache.cassandra.utils.concurrent.CountDownLatch.newCountDownLatch; import static org.apache.commons.lang3.StringUtils.join; @@ -2835,7 +2836,7 @@ public String getIdealConsistencyLevel() public String setIdealConsistencyLevel(String cl) { ConsistencyLevel original = DatabaseDescriptor.getIdealConsistencyLevel(); - ConsistencyLevel newCL = ConsistencyLevel.valueOf(cl.trim().toUpperCase()); + ConsistencyLevel newCL = ConsistencyLevel.valueOf(toUpperCaseLocalized(cl.trim())); DatabaseDescriptor.setIdealConsistencyLevel(newCL); return String.format("Updating ideal consistency level new value: %s old value %s", newCL, original.toString()); } diff --git a/src/java/org/apache/cassandra/service/paxos/ContentionStrategy.java b/src/java/org/apache/cassandra/service/paxos/ContentionStrategy.java index 7f38567f6a15..59ee5505123e 100644 --- a/src/java/org/apache/cassandra/service/paxos/ContentionStrategy.java +++ b/src/java/org/apache/cassandra/service/paxos/ContentionStrategy.java @@ -52,6 +52,7 @@ import static org.apache.cassandra.metrics.ClientRequestsMetricsHolder.casWriteMetrics; import static org.apache.cassandra.utils.Clock.Global.nanoTime; import static org.apache.cassandra.utils.Clock.waitUntil; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; /** * <p>A strategy for making back-off decisions for Paxos operations that fail to make progress because of other paxos operations. @@ -369,7 +370,7 @@ public enum Type Type(String traceTitle) { this.traceTitle = traceTitle; - this.lowercase = name().toLowerCase(); + this.lowercase = toLowerCaseLocalized(name()); } } diff --git a/src/java/org/apache/cassandra/service/reads/HybridSpeculativeRetryPolicy.java b/src/java/org/apache/cassandra/service/reads/HybridSpeculativeRetryPolicy.java index fedec23f409c..226f4f144190 100644 --- a/src/java/org/apache/cassandra/service/reads/HybridSpeculativeRetryPolicy.java +++ b/src/java/org/apache/cassandra/service/reads/HybridSpeculativeRetryPolicy.java @@ -27,6 +27,8 @@ import org.apache.cassandra.metrics.SnapshottingTimer; import org.apache.cassandra.schema.TableParams; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; + public class HybridSpeculativeRetryPolicy implements SpeculativeRetryPolicy { private static final Pattern PATTERN = @@ -128,7 +130,7 @@ static HybridSpeculativeRetryPolicy fromString(String str) SpeculativeRetryPolicy policy1 = value1 instanceof PercentileSpeculativeRetryPolicy ? value1 : value2; SpeculativeRetryPolicy policy2 = value1 instanceof FixedSpeculativeRetryPolicy ? value1 : value2; - Function function = Function.valueOf(matcher.group("fun").toUpperCase()); + Function function = Function.valueOf(toUpperCaseLocalized(matcher.group("fun"))); return new HybridSpeculativeRetryPolicy((PercentileSpeculativeRetryPolicy) policy1, (FixedSpeculativeRetryPolicy) policy2, function); } diff --git a/src/java/org/apache/cassandra/service/reads/repair/ReadRepairStrategy.java b/src/java/org/apache/cassandra/service/reads/repair/ReadRepairStrategy.java index 975b8217d20b..22615494a748 100644 --- a/src/java/org/apache/cassandra/service/reads/repair/ReadRepairStrategy.java +++ b/src/java/org/apache/cassandra/service/reads/repair/ReadRepairStrategy.java @@ -23,6 +23,8 @@ import org.apache.cassandra.locator.ReplicaPlan; import org.apache.cassandra.transport.Dispatcher; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; + public enum ReadRepairStrategy implements ReadRepair.Factory { NONE @@ -45,6 +47,6 @@ ReadRepair<E, P> create(ReadCommand command, ReplicaPlan.Shared<E, P> replicaPla public static ReadRepairStrategy fromString(String s) { - return valueOf(s.toUpperCase()); + return valueOf(toUpperCaseLocalized(s)); } } diff --git a/src/java/org/apache/cassandra/streaming/StreamResultFuture.java b/src/java/org/apache/cassandra/streaming/StreamResultFuture.java index 5277b9de73d9..6d3e1ca6c1f5 100644 --- a/src/java/org/apache/cassandra/streaming/StreamResultFuture.java +++ b/src/java/org/apache/cassandra/streaming/StreamResultFuture.java @@ -36,6 +36,7 @@ import static org.apache.cassandra.streaming.StreamingChannel.Factory.Global.streamingFactory; import static org.apache.cassandra.utils.Clock.Global.nanoTime; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; /** * A future on the result ({@link StreamState}) of a streaming plan. @@ -198,7 +199,7 @@ void handleSessionPrepared(StreamSession session, StreamSession.PrepareDirection void handleSessionComplete(StreamSession session) { - logger.info("[Stream #{}] Session with {} is {}", session.planId(), session.peer, session.state().name().toLowerCase()); + logger.info("[Stream #{}] Session with {} is {}", session.planId(), session.peer, toLowerCaseLocalized(session.state().name())); fireStreamEvent(new StreamEvent.SessionCompleteEvent(session)); SessionInfo sessionInfo = session.getSessionInfo(); coordinator.addSessionInfo(sessionInfo); diff --git a/src/java/org/apache/cassandra/streaming/StreamingState.java b/src/java/org/apache/cassandra/streaming/StreamingState.java index 385596077308..fbd66d1bb6e7 100644 --- a/src/java/org/apache/cassandra/streaming/StreamingState.java +++ b/src/java/org/apache/cassandra/streaming/StreamingState.java @@ -42,6 +42,7 @@ import org.apache.cassandra.utils.ObjectSizes; import org.apache.cassandra.utils.TimeUUID; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; import static org.apache.cassandra.utils.TimeUUID.Generator.nextTimeUUID; public class StreamingState implements StreamEventHandler, IMeasurableMemory @@ -214,14 +215,14 @@ public String toString() { TableBuilder table = new TableBuilder(); table.add("id", id.toString()); - table.add("status", status().name().toLowerCase()); + table.add("status", toLowerCaseLocalized(status().name())); table.add("progress", (progress() * 100) + "%"); table.add("duration_ms", Long.toString(durationMillis())); table.add("last_updated_ms", Long.toString(lastUpdatedAtMillis())); table.add("failure_cause", failureCause()); table.add("success_message", successMessage()); for (Map.Entry<Status, Long> e : stateTimesMillis().entrySet()) - table.add("status_" + e.getKey().name().toLowerCase() + "_ms", e.toString()); + table.add("status_" + toLowerCaseLocalized(e.getKey().name()) + "_ms", e.toString()); return table.toString(); } diff --git a/src/java/org/apache/cassandra/tcm/MetadataKeys.java b/src/java/org/apache/cassandra/tcm/MetadataKeys.java index 80280078152b..bead377c0bfb 100644 --- a/src/java/org/apache/cassandra/tcm/MetadataKeys.java +++ b/src/java/org/apache/cassandra/tcm/MetadataKeys.java @@ -29,9 +29,11 @@ import org.apache.cassandra.tcm.extensions.ExtensionKey; import org.apache.cassandra.tcm.extensions.ExtensionValue; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + public class MetadataKeys { - public static final String CORE_NS = MetadataKeys.class.getPackage().getName().toLowerCase(Locale.ROOT); + public static final String CORE_NS = toLowerCaseLocalized(MetadataKeys.class.getPackage().getName(), Locale.ROOT); public static final MetadataKey SCHEMA = make(CORE_NS, "schema", "dist_schema"); public static final MetadataKey NODE_DIRECTORY = make(CORE_NS, "membership", "node_directory"); diff --git a/src/java/org/apache/cassandra/tools/RepairRunner.java b/src/java/org/apache/cassandra/tools/RepairRunner.java index cd09c57344be..01aa5201852b 100644 --- a/src/java/org/apache/cassandra/tools/RepairRunner.java +++ b/src/java/org/apache/cassandra/tools/RepairRunner.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.Map; +import org.apache.cassandra.service.ActiveRepairService.ParentRepairStatus; import org.apache.cassandra.service.StorageServiceMBean; import org.apache.cassandra.utils.concurrent.Condition; @@ -32,10 +33,10 @@ import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; import static java.util.concurrent.TimeUnit.SECONDS; -import static org.apache.cassandra.service.ActiveRepairService.ParentRepairStatus; import static org.apache.cassandra.service.ActiveRepairService.ParentRepairStatus.FAILED; import static org.apache.cassandra.service.ActiveRepairService.ParentRepairStatus.valueOf; import static org.apache.cassandra.tools.NodeProbe.JMX_NOTIFICATION_POLL_INTERVAL_SECONDS; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition; import static org.apache.cassandra.utils.progress.ProgressEventType.*; @@ -165,7 +166,7 @@ private void queryForCompletedRepair(String triggeringCondition) case FAILED: printMessage(String.format("%s %s discovered repair %s.", triggeringCondition, - queriedString, parentRepairStatus.name().toLowerCase())); + queriedString, toLowerCaseLocalized(parentRepairStatus.name()))); if (parentRepairStatus == FAILED) { error = new IOException(messages.get(0)); diff --git a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java index e7afc255c0a1..fe12e6d723c9 100644 --- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java +++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java @@ -58,6 +58,8 @@ import static org.apache.cassandra.config.CassandraRelevantProperties.TEST_UTIL_ALLOW_TOOL_REINIT_FOR_TEST; import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; public class StandaloneScrubber { @@ -241,12 +243,12 @@ enum HeaderFixMode static HeaderFixMode fromCommandLine(String value) { - return valueOf(value.replace('-', '_').toUpperCase().trim()); + return valueOf(toUpperCaseLocalized(value.replace('-', '_')).trim()); } String asCommandLineOption() { - return name().toLowerCase().replace('_', '-'); + return toLowerCaseLocalized(name()).replace('_', '-'); } } diff --git a/src/java/org/apache/cassandra/tools/nodetool/ProfileLoad.java b/src/java/org/apache/cassandra/tools/nodetool/ProfileLoad.java index 45cade7560f3..01af271fcbc9 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/ProfileLoad.java +++ b/src/java/org/apache/cassandra/tools/nodetool/ProfileLoad.java @@ -41,6 +41,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; import static org.apache.commons.lang3.StringUtils.join; @Command(name = "profileload", description = "Low footprint profiling of activity for a period of time") @@ -113,7 +114,7 @@ else if (args.size() == 1) Set<String> available = Arrays.stream(SamplerType.values()).map(Enum::toString).collect(Collectors.toSet()); for (String s : samplers.split(",")) { - String sampler = s.trim().toUpperCase(); + String sampler = toUpperCaseLocalized(s.trim()); checkArgument(available.contains(sampler), String.format("'%s' sampler is not available from: %s", s, Arrays.toString(SamplerType.values()))); targets.add(sampler); } diff --git a/src/java/org/apache/cassandra/transport/CBUtil.java b/src/java/org/apache/cassandra/transport/CBUtil.java index d28d9c189e27..eb0d5a16f6b2 100644 --- a/src/java/org/apache/cassandra/transport/CBUtil.java +++ b/src/java/org/apache/cassandra/transport/CBUtil.java @@ -47,6 +47,7 @@ import org.apache.cassandra.utils.memory.MemoryUtil; import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_NETTY_USE_HEAP_ALLOCATOR; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; /** * ByteBuf utility methods. @@ -285,7 +286,7 @@ public static <T extends Enum<T>> T readEnumValue(Class<T> enumType, ByteBuf cb) String value = CBUtil.readString(cb); try { - return Enum.valueOf(enumType, value.toUpperCase()); + return Enum.valueOf(enumType, toUpperCaseLocalized(value)); } catch (IllegalArgumentException e) { @@ -399,7 +400,7 @@ public static Map<String, List<String>> readStringToStringListMap(ByteBuf cb) Map<String, List<String>> m = new HashMap<String, List<String>>(length); for (int i = 0; i < length; i++) { - String k = readString(cb).toUpperCase(); + String k = toUpperCaseLocalized(readString(cb)); List<String> v = readStringList(cb); m.put(k, v); } diff --git a/src/java/org/apache/cassandra/transport/Client.java b/src/java/org/apache/cassandra/transport/Client.java index 45f5e1f2fad4..96fea832a857 100644 --- a/src/java/org/apache/cassandra/transport/Client.java +++ b/src/java/org/apache/cassandra/transport/Client.java @@ -38,6 +38,9 @@ import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.MD5Digest; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; + public class Client extends SimpleClient { private final SimpleEventHandler eventHandler = new SimpleEventHandler(); @@ -100,7 +103,7 @@ private Message.Request parseLine(String line) Iterator<String> iter = splitter.split(line).iterator(); if (!iter.hasNext()) return null; - String msgType = iter.next().toUpperCase(); + String msgType = toUpperCaseLocalized(iter.next()); if (msgType.equals("STARTUP")) { Map<String, String> options = new HashMap<String, String>(); @@ -108,17 +111,17 @@ private Message.Request parseLine(String line) while (iter.hasNext()) { String next = iter.next(); - if (next.toLowerCase().equals("snappy")) + if (toLowerCaseLocalized(next).equals("snappy")) { options.put(StartupMessage.COMPRESSION, "snappy"); connection.setCompressor(Compressor.SnappyCompressor.instance); } - if (next.toLowerCase().equals("lz4")) + if (toLowerCaseLocalized(next).equals("lz4")) { options.put(StartupMessage.COMPRESSION, "lz4"); connection.setCompressor(Compressor.LZ4Compressor.instance); } - if (next.toLowerCase().equals("throw_on_overload")) + if (toLowerCaseLocalized(next).equals("throw_on_overload")) { options.put(StartupMessage.THROW_ON_OVERLOAD, "1"); connection.setThrowOnOverload(true); @@ -198,7 +201,7 @@ else if (msgType.equals("AUTHENTICATE")) } else if (msgType.equals("REGISTER")) { - String type = line.substring(9).toUpperCase(); + String type = toUpperCaseLocalized(line.substring(9)); try { return new RegisterMessage(Collections.singletonList(Enum.valueOf(Event.Type.class, type))); diff --git a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java index 5ec3dc27ca06..570b0975d625 100644 --- a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java +++ b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java @@ -29,6 +29,9 @@ import org.apache.cassandra.transport.*; import org.apache.cassandra.utils.CassandraVersion; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; + /** * The initial message of the protocol. * Sets up a number of connection options. @@ -89,7 +92,7 @@ protected Message.Response execute(QueryState state, Dispatcher.RequestTime requ if (options.containsKey(COMPRESSION)) { - String compression = options.get(COMPRESSION).toLowerCase(); + String compression = toLowerCaseLocalized(options.get(COMPRESSION)); if (compression.equals("snappy")) { if (Compressor.SnappyCompressor.instance == null) @@ -158,7 +161,7 @@ private static Map<String, String> upperCaseKeys(Map<String, String> options) { Map<String, String> newMap = new HashMap<String, String>(options.size()); for (Map.Entry<String, String> entry : options.entrySet()) - newMap.put(entry.getKey().toUpperCase(), entry.getValue()); + newMap.put(toUpperCaseLocalized(entry.getKey()), entry.getValue()); return newMap; } diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java index fd5c94c7ddd3..d52cfeb30b29 100644 --- a/src/java/org/apache/cassandra/utils/FBUtilities.java +++ b/src/java/org/apache/cassandra/utils/FBUtilities.java @@ -103,6 +103,7 @@ import static org.apache.cassandra.config.CassandraRelevantProperties.USER_HOME; import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; import static org.apache.cassandra.utils.Clock.Global.nanoTime; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; public class FBUtilities { @@ -118,7 +119,7 @@ public class FBUtilities public static final BigInteger TWO = new BigInteger("2"); private static final String DEFAULT_TRIGGER_DIR = "triggers"; - private static final String OPERATING_SYSTEM = OS_NAME.getString().toLowerCase(); + private static final String OPERATING_SYSTEM = toLowerCaseLocalized(OS_NAME.getString()); public static final boolean isLinux = OPERATING_SYSTEM.contains("linux"); private static volatile InetAddress localInetAddress; @@ -1364,7 +1365,7 @@ public static void preventIllegalAccessWarnings() public static String camelToSnake(String camel) { if (camel.chars().allMatch(Character::isUpperCase)) - return camel.toLowerCase(); + return toLowerCaseLocalized(camel); StringBuilder sb = new StringBuilder(); for (char c : camel.toCharArray()) @@ -1374,7 +1375,7 @@ public static String camelToSnake(String camel) // if first char is uppercase, then avoid adding the _ prefix if (sb.length() > 0) sb.append('_'); - sb.append(Character.toLowerCase(c)); + sb.append(Character.toLowerCase(c)); // checkstyle: permit this invocation } else { diff --git a/src/java/org/apache/cassandra/utils/GuidGenerator.java b/src/java/org/apache/cassandra/utils/GuidGenerator.java index e06270fa1d94..1038f29c1a20 100644 --- a/src/java/org/apache/cassandra/utils/GuidGenerator.java +++ b/src/java/org/apache/cassandra/utils/GuidGenerator.java @@ -23,6 +23,7 @@ import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_SECURITY_EGD; import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; public class GuidGenerator { @@ -101,7 +102,7 @@ public static ByteBuffer guidAsBytes() private static String convertToStandardFormat(String valueAfterMD5) { - String raw = valueAfterMD5.toUpperCase(); + String raw = toUpperCaseLocalized(valueAfterMD5); StringBuilder sb = new StringBuilder(); sb.append(raw.substring(0, 8)) .append("-") diff --git a/src/java/org/apache/cassandra/utils/JsonUtils.java b/src/java/org/apache/cassandra/utils/JsonUtils.java index 1cdc13c55cba..aa0963bf5859 100644 --- a/src/java/org/apache/cassandra/utils/JsonUtils.java +++ b/src/java/org/apache/cassandra/utils/JsonUtils.java @@ -23,7 +23,6 @@ import java.io.OutputStream; import java.util.ArrayList; import java.util.List; -import java.util.Locale; import java.util.Map; import com.fasterxml.jackson.core.JsonFactory; @@ -38,6 +37,7 @@ import org.apache.cassandra.serializers.MarshalException; import static org.apache.cassandra.io.util.File.WriteMode.OVERWRITE; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; public final class JsonUtils { @@ -203,7 +203,7 @@ public static void handleCaseSensitivity(Map<String, Object> valueMap) } // otherwise, lowercase it if needed - String lowered = mapKey.toLowerCase(Locale.US); + String lowered = toLowerCaseLocalized(mapKey); if (!mapKey.equals(lowered)) valueMap.put(lowered, valueMap.remove(mapKey)); } diff --git a/src/java/org/apache/cassandra/utils/LocalizeString.java b/src/java/org/apache/cassandra/utils/LocalizeString.java new file mode 100644 index 000000000000..d7d96cd7c98d --- /dev/null +++ b/src/java/org/apache/cassandra/utils/LocalizeString.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils; + +import java.util.Locale; + +public class LocalizeString +{ + /** + * Convert the String to lower case, using {@link java.util.Locale#US} by default + */ + public static String toLowerCaseLocalized(String input) + { + return toLowerCaseLocalized(input, Locale.US); + } + + /** + * @param input The string to be converted to lower case. + * @param locale The locale to use for the conversion. + * @return String itself with lowercase and localized by your selection of {@link java.util.Locale}. + */ + public static String toLowerCaseLocalized(String input, Locale locale) + { + return input.toLowerCase(locale); // checkstyle: permit this invocation + } + + /** + * Convert the String to upper case, using {@link java.util.Locale#US} by default + */ + public static String toUpperCaseLocalized(String input) + { + return toUpperCaseLocalized(input, Locale.US); + } + + /** + * @param input The string to be converted to uppercase. + * @param locale The locale to use for the conversion. This parameter is optional. + * @return String itself with uppercase and localized by your selection of {@link java.util.Locale}. + */ + public static String toUpperCaseLocalized(String input, Locale locale) + { + return input.toUpperCase(locale); // checkstyle: permit this invocation + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/utils/NativeLibrary.java b/src/java/org/apache/cassandra/utils/NativeLibrary.java index 0f7172b5ae87..84232dfabfb2 100644 --- a/src/java/org/apache/cassandra/utils/NativeLibrary.java +++ b/src/java/org/apache/cassandra/utils/NativeLibrary.java @@ -35,6 +35,7 @@ import static org.apache.cassandra.config.CassandraRelevantProperties.IGNORE_MISSING_NATIVE_FILE_HINTS; import static org.apache.cassandra.config.CassandraRelevantProperties.OS_ARCH; import static org.apache.cassandra.config.CassandraRelevantProperties.OS_NAME; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; import static org.apache.cassandra.utils.NativeLibrary.OSType.LINUX; import static org.apache.cassandra.utils.NativeLibrary.OSType.MAC; import static org.apache.cassandra.utils.NativeLibrary.OSType.AIX; @@ -102,7 +103,7 @@ public enum OSType default: wrappedLibrary = new NativeLibraryLinux(); } - if (OS_ARCH.getString().toLowerCase().contains("ppc")) + if (toLowerCaseLocalized(OS_ARCH.getString()).contains("ppc")) { if (osType == LINUX) { @@ -134,7 +135,7 @@ private NativeLibrary() {} */ private static OSType getOsType() { - String osName = OS_NAME.getString().toLowerCase(); + String osName = toLowerCaseLocalized(OS_NAME.getString()); if (osName.contains("linux")) return LINUX; else if (osName.contains("mac")) diff --git a/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java b/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java index 19dcc23dcf3a..6ce840efbb5e 100644 --- a/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java +++ b/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java @@ -35,6 +35,8 @@ import org.apache.cassandra.io.sstable.SSTableLoader; import org.apache.cassandra.schema.TableMetadata; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; + public class NativeSSTableLoaderClient extends SSTableLoader.Client { protected final Map<String, TableMetadataRef> tables; @@ -203,7 +205,7 @@ private static TableMetadataRef createTableMetadata(String keyspace, private static ColumnMetadata createDefinitionFromRow(Row row, String keyspace, String table, Types types) { - ClusteringOrder order = ClusteringOrder.valueOf(row.getString("clustering_order").toUpperCase()); + ClusteringOrder order = ClusteringOrder.valueOf(toUpperCaseLocalized(row.getString("clustering_order"))); AbstractType<?> type = CQLTypeParser.parse(keyspace, row.getString("type"), types); if (order == ClusteringOrder.DESC) type = ReversedType.getInstance(type); @@ -211,7 +213,7 @@ private static ColumnMetadata createDefinitionFromRow(Row row, String keyspace, ColumnIdentifier name = new ColumnIdentifier(row.getBytes("column_name_bytes"), row.getString("column_name")); int position = row.getInt("position"); - org.apache.cassandra.schema.ColumnMetadata.Kind kind = ColumnMetadata.Kind.valueOf(row.getString("kind").toUpperCase()); + org.apache.cassandra.schema.ColumnMetadata.Kind kind = ColumnMetadata.Kind.valueOf(toUpperCaseLocalized(row.getString("kind"))); return new ColumnMetadata(keyspace, table, name, type, position, kind, null); } @@ -219,7 +221,7 @@ private static DroppedColumn createDroppedColumnFromRow(Row row, String keyspace { String name = row.getString("column_name"); AbstractType<?> type = CQLTypeParser.parse(keyspace, row.getString("type"), Types.none()); - ColumnMetadata.Kind kind = ColumnMetadata.Kind.valueOf(row.getString("kind").toUpperCase()); + ColumnMetadata.Kind kind = ColumnMetadata.Kind.valueOf(toUpperCaseLocalized(row.getString("kind"))); ColumnMetadata column = new ColumnMetadata(keyspace, table, ColumnIdentifier.getInterned(name, true), type, ColumnMetadata.NO_POSITION, kind, null); long droppedTime = row.getTimestamp("dropped_time").getTime(); return new DroppedColumn(column, droppedTime); diff --git a/src/java/org/apache/cassandra/utils/binlog/BinLog.java b/src/java/org/apache/cassandra/utils/binlog/BinLog.java index 8b0e02570058..313d6b45498d 100644 --- a/src/java/org/apache/cassandra/utils/binlog/BinLog.java +++ b/src/java/org/apache/cassandra/utils/binlog/BinLog.java @@ -54,6 +54,7 @@ import static java.lang.String.format; import static org.apache.cassandra.config.CassandraRelevantProperties.CHRONICLE_ANNOUNCER_DISABLE; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; /** * Bin log is a is quick and dirty binary log that is kind of a NIH version of binary logging with a traditional logging @@ -389,7 +390,7 @@ public Builder path(Path path) public Builder rollCycle(String rollCycle) { Preconditions.checkNotNull(rollCycle, "rollCycle was null"); - rollCycle = rollCycle.toUpperCase(); + rollCycle = toUpperCaseLocalized(rollCycle); Preconditions.checkNotNull(RollCycles.valueOf(rollCycle), "unrecognized roll cycle"); this.rollCycle = rollCycle; return this; diff --git a/test/distributed/org/apache/cassandra/distributed/Cluster.java b/test/distributed/org/apache/cassandra/distributed/Cluster.java index 788171dd9e83..4effc4e9c49a 100644 --- a/test/distributed/org/apache/cassandra/distributed/Cluster.java +++ b/test/distributed/org/apache/cassandra/distributed/Cluster.java @@ -29,6 +29,8 @@ import org.apache.cassandra.distributed.shared.Versions; import org.apache.cassandra.net.Message; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + /** * A simple cluster supporting only the 'current' Cassandra version, offering easy access to the convenience methods * of IInvokableInstance on each node. @@ -81,7 +83,7 @@ public void enableMessageLogging() { get(1).acceptsOnInstance((IIsolatedExecutor.SerializableConsumer<IMessage>) (msgPassed) -> { Message decoded = Instance.deserializeMessage(msgPassed); - if (!decoded.verb().toString().toLowerCase().contains("gossip")) + if (!toLowerCaseLocalized(decoded.verb().toString()).contains("gossip")) System.out.println(String.format("MSG %d -> %d: %s | %s", from, to, decoded, decoded.payload)); }).accept(msg); } diff --git a/test/distributed/org/apache/cassandra/distributed/test/FailingRepairTest.java b/test/distributed/org/apache/cassandra/distributed/test/FailingRepairTest.java index 405279aae609..5727a020b6fe 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/FailingRepairTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/FailingRepairTest.java @@ -76,6 +76,8 @@ import org.apache.cassandra.service.StorageService; import org.awaitility.Awaitility; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + @RunWith(Parameterized.class) public class FailingRepairTest extends TestBaseImpl implements Serializable { @@ -132,7 +134,7 @@ private static SerializableRunnable failingReaders(Verb type, RepairParallelism private static String getCfName(Verb type, RepairParallelism parallelism, boolean withTracing) { - return type.name().toLowerCase() + "_" + parallelism.name().toLowerCase() + "_" + withTracing; + return toLowerCaseLocalized(type.name()) + "_" + toLowerCaseLocalized(parallelism.name()) + "_" + withTracing; } @BeforeClass diff --git a/test/distributed/org/apache/cassandra/distributed/test/ReadDigestConsistencyTest.java b/test/distributed/org/apache/cassandra/distributed/test/ReadDigestConsistencyTest.java index 85c2783161ba..907ce0df0461 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/ReadDigestConsistencyTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/ReadDigestConsistencyTest.java @@ -33,6 +33,8 @@ import org.apache.cassandra.utils.Throwables; import org.apache.cassandra.utils.TimeUUID; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + public class ReadDigestConsistencyTest extends TestBaseImpl { private final static Logger logger = LoggerFactory.getLogger(ReadDigestConsistencyTest.class); @@ -101,7 +103,7 @@ public static void checkTraceForDigestMismatch(ICoordinator coordinator, String Arrays.toString(boundValues), coordinator.instance().broadcastAddress(), coordinator.instance().getReleaseVersionString()), - activity.toLowerCase().contains("mismatch for key")); + toLowerCaseLocalized(activity).contains("mismatch for key")); } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorBase.java b/test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorBase.java index 0fc2554b0139..6967b9f119cd 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorBase.java +++ b/test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorBase.java @@ -35,6 +35,7 @@ import org.apache.cassandra.distributed.test.DistributedRepairUtils.RepairType; import static org.apache.cassandra.config.CassandraRelevantProperties.NODETOOL_JMX_NOTIFICATION_POLL_INTERVAL_SECONDS; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; public class RepairCoordinatorBase extends TestBaseImpl { @@ -98,7 +99,7 @@ protected String tableName(String prefix) { protected String postfix() { - return repairType.name().toLowerCase() + "_" + parallelism.name().toLowerCase() + "_" + withNotifications; + return toLowerCaseLocalized(repairType.name()) + "_" + toLowerCaseLocalized(parallelism.name()) + "_" + withNotifications; } protected NodeToolResult repair(int node, String... args) { diff --git a/test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorFailingMessageTest.java b/test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorFailingMessageTest.java index 0d046497b967..dbd2680ad9b5 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorFailingMessageTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/RepairCoordinatorFailingMessageTest.java @@ -42,6 +42,7 @@ import static java.lang.String.format; import static org.apache.cassandra.distributed.api.IMessageFilters.Matcher.of; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; @RunWith(Parameterized.class) @Ignore("Until CASSANDRA-15566 is in these tests all time out") @@ -100,7 +101,7 @@ private String tableName(String prefix) { private String postfix() { - return repairType.name().toLowerCase(); + return toLowerCaseLocalized(repairType.name()); } private NodeToolResult repair(int node, String... args) { diff --git a/test/distributed/org/apache/cassandra/distributed/test/SSTableIdGenerationTest.java b/test/distributed/org/apache/cassandra/distributed/test/SSTableIdGenerationTest.java index 9514faf9eadc..a44532ee156f 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/SSTableIdGenerationTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/SSTableIdGenerationTest.java @@ -67,6 +67,7 @@ import static org.apache.cassandra.db.SystemKeyspace.SSTABLE_ACTIVITY_V2; import static org.apache.cassandra.distributed.shared.FutureUtils.waitOn; import static org.apache.cassandra.distributed.test.ExecUtil.rethrow; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; import static org.assertj.core.api.Assertions.assertThat; public class SSTableIdGenerationTest extends TestBaseImpl @@ -176,7 +177,7 @@ private final void testCompactionStrategiesWithMixedSSTables(final Class<? exten // create a table and two sstables with sequential id for each strategy, the sstables will contain overlapping partitions for (Class<? extends AbstractCompactionStrategy> compactionStrategyClass : compactionStrategyClasses) { - String tableName = "tbl_" + compactionStrategyClass.getSimpleName().toLowerCase(); + String tableName = "tbl_" + toLowerCaseLocalized(compactionStrategyClass.getSimpleName()); cluster.schemaChange(createTableStmt(KEYSPACE, tableName, compactionStrategyClass)); createSSTables(cluster.get(1), KEYSPACE, tableName, 1, 2); @@ -189,7 +190,7 @@ private final void testCompactionStrategiesWithMixedSSTables(final Class<? exten // create another two sstables with uuid for each previously created table for (Class<? extends AbstractCompactionStrategy> compactionStrategyClass : compactionStrategyClasses) { - String tableName = "tbl_" + compactionStrategyClass.getSimpleName().toLowerCase(); + String tableName = "tbl_" + toLowerCaseLocalized(compactionStrategyClass.getSimpleName()); createSSTables(cluster.get(1), KEYSPACE, tableName, 3, 4); diff --git a/test/memory/org/apache/cassandra/db/compaction/CompactionAllocationTest.java b/test/memory/org/apache/cassandra/db/compaction/CompactionAllocationTest.java index e2a4edeca533..c800963c014c 100644 --- a/test/memory/org/apache/cassandra/db/compaction/CompactionAllocationTest.java +++ b/test/memory/org/apache/cassandra/db/compaction/CompactionAllocationTest.java @@ -78,6 +78,8 @@ import org.apache.cassandra.utils.ObjectSizes; import org.apache.cassandra.utils.concurrent.Refs; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + public class CompactionAllocationTest { private static final Logger logger = LoggerFactory.getLogger(CompactionAllocationTest.class); @@ -494,7 +496,7 @@ private static void runQuery(ReadQuery query, TableMetadata metadata) private static void testTinyPartitions(String name, int numSSTable, int sstablePartitions, boolean overlap) throws Throwable { - String ksname = "ks_" + name.toLowerCase(); + String ksname = "ks_" + toLowerCaseLocalized(name); SchemaLoader.createKeyspace(ksname, KeyspaceParams.simple(1), CreateTableStatement.parse("CREATE TABLE tbl (k INT PRIMARY KEY, v INT)", ksname).build()); @@ -603,7 +605,7 @@ public static String makeRandomString(int length, int seed) private static void testMediumPartitions(String name, int numSSTable, int sstablePartitions, boolean overlap, boolean overlapCK) throws Throwable { - String ksname = "ks_" + name.toLowerCase(); + String ksname = "ks_" + toLowerCaseLocalized(name); SchemaLoader.createKeyspace(ksname, KeyspaceParams.simple(1), CreateTableStatement.parse("CREATE TABLE tbl (k text, c text, v1 text, v2 text, v3 text, v4 text, PRIMARY KEY (k, c))", ksname).build()); @@ -702,7 +704,7 @@ public void mediumPartitionsOverlappingRows9() throws Throwable private static void testWidePartitions(String name, int numSSTable, int sstablePartitions, boolean overlap, boolean overlapCK) throws Throwable { - String ksname = "ks_" + name.toLowerCase(); + String ksname = "ks_" + toLowerCaseLocalized(name); SchemaLoader.createKeyspace(ksname, KeyspaceParams.simple(1), CreateTableStatement.parse("CREATE TABLE tbl (k text, c text, v1 text, v2 text, v3 text, v4 text, PRIMARY KEY (k, c))", ksname).build()); @@ -806,7 +808,7 @@ private static void testIndexingWidePartitions(String name, int sstablePartitions, IndexDef...indexes) throws Throwable { - String ksname = "ks_" + name.toLowerCase(); + String ksname = "ks_" + toLowerCaseLocalized(name); SchemaLoader.createKeyspace(ksname, KeyspaceParams.simple(1), CreateTableStatement.parse("CREATE TABLE tbl (k text, c text, v1 text, v2 text, v3 text, v4 text, PRIMARY KEY (k, c))", ksname).build()); diff --git a/test/simulator/main/org/apache/cassandra/simulator/Action.java b/test/simulator/main/org/apache/cassandra/simulator/Action.java index edb661fae123..76c22cef4bd5 100644 --- a/test/simulator/main/org/apache/cassandra/simulator/Action.java +++ b/test/simulator/main/org/apache/cassandra/simulator/Action.java @@ -897,7 +897,7 @@ private String describeModifiers() continue; if (!transitive.is(modifier)) builder.append(modifier.displayId); - else builder.append(Character.toUpperCase(modifier.displayId)); + else builder.append(Character.toUpperCase(modifier.displayId)); // checkstyle: permit this invocation } boolean hasTransitiveOnly = false; diff --git a/test/simulator/main/org/apache/cassandra/simulator/SimulationRunner.java b/test/simulator/main/org/apache/cassandra/simulator/SimulationRunner.java index 5542d0d1e2ba..deaa462df34a 100644 --- a/test/simulator/main/org/apache/cassandra/simulator/SimulationRunner.java +++ b/test/simulator/main/org/apache/cassandra/simulator/SimulationRunner.java @@ -81,6 +81,7 @@ import static org.apache.cassandra.simulator.debug.SelfReconcile.reconcileWithSelf; import static org.apache.cassandra.simulator.utils.IntRange.parseRange; import static org.apache.cassandra.simulator.utils.LongRange.parseNanosRange; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; @SuppressWarnings({ "ZeroLengthArrayAllocation", "CodeBlock2Expr", "SameParameterValue", "DynamicRegexReplaceableByCompiledPattern", "CallToSystemGC" }) public class SimulationRunner @@ -281,7 +282,7 @@ protected void propagate(B builder) Optional.ofNullable(topologyChanges).ifPresent(topologyChanges -> { builder.topologyChanges(stream(topologyChanges.split(",")) .filter(v -> !v.isEmpty()) - .map(v -> TopologyChange.valueOf(v.toUpperCase())) + .map(v -> TopologyChange.valueOf(toUpperCaseLocalized(v))) .toArray(TopologyChange[]::new)); }); parseNanosRange(Optional.ofNullable(topologyChangeInterval)).ifPresent(builder::topologyChangeIntervalNanos); @@ -289,7 +290,7 @@ protected void propagate(B builder) Optional.ofNullable(priority).ifPresent(kinds -> { builder.scheduler(stream(kinds.split(",")) .filter(v -> !v.isEmpty()) - .map(v -> RunnableActionScheduler.Kind.valueOf(v.toUpperCase())) + .map(v -> RunnableActionScheduler.Kind.valueOf(toUpperCaseLocalized(v))) .toArray(RunnableActionScheduler.Kind[]::new)); }); diff --git a/test/unit/org/apache/cassandra/SchemaLoader.java b/test/unit/org/apache/cassandra/SchemaLoader.java index d18dd0aa9641..73ef7285b389 100644 --- a/test/unit/org/apache/cassandra/SchemaLoader.java +++ b/test/unit/org/apache/cassandra/SchemaLoader.java @@ -52,6 +52,7 @@ import static org.apache.cassandra.config.CassandraRelevantProperties.TEST_COMPRESSION; import static org.apache.cassandra.config.CassandraRelevantProperties.TEST_COMPRESSION_ALGO; import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; public class SchemaLoader { @@ -745,7 +746,7 @@ public static void cleanupSavedCaches() private static CompressionParams compressionParams(int chunkLength) { - String algo = TEST_COMPRESSION_ALGO.getString().toLowerCase(); + String algo = toLowerCaseLocalized(TEST_COMPRESSION_ALGO.getString()); switch (algo) { case "deflate": diff --git a/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java b/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java index b255ad59ff65..6ec69d5c8591 100644 --- a/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java +++ b/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java @@ -41,6 +41,7 @@ import static org.apache.cassandra.auth.AuthTestUtils.auth; import static org.apache.cassandra.auth.AuthTestUtils.getRolesReadCount; import static org.apache.cassandra.schema.SchemaConstants.AUTH_KEYSPACE_NAME; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; @@ -100,7 +101,7 @@ private static void assertDcPermRow(String username, String... dcs) private static String createName() { - return RandomStringUtils.randomAlphabetic(8).toLowerCase(); + return toLowerCaseLocalized(RandomStringUtils.randomAlphabetic(8)); } private static DCPermissions dcPerms(String username) diff --git a/test/unit/org/apache/cassandra/config/DataStorageSpecTest.java b/test/unit/org/apache/cassandra/config/DataStorageSpecTest.java index 334e33f259b1..ff89693d5c88 100644 --- a/test/unit/org/apache/cassandra/config/DataStorageSpecTest.java +++ b/test/unit/org/apache/cassandra/config/DataStorageSpecTest.java @@ -28,6 +28,7 @@ import static org.apache.cassandra.config.DataStorageSpec.DataStorageUnit.GIBIBYTES; import static org.apache.cassandra.config.DataStorageSpec.DataStorageUnit.KIBIBYTES; import static org.apache.cassandra.config.DataStorageSpec.DataStorageUnit.MEBIBYTES; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.Assert.*; import static org.quicktheories.QuickTheory.qt; @@ -242,7 +243,7 @@ public void thereAndBackLongBytesBound() { qt().forAll(gen()).check(there -> { DataStorageSpec.LongBytesBound back = new DataStorageSpec.LongBytesBound(there.toString()); - DataStorageSpec.LongBytesBound BACK = new DataStorageSpec.LongBytesBound(there.toString().toUpperCase(Locale.ROOT).replace("I", "i")); + DataStorageSpec.LongBytesBound BACK = new DataStorageSpec.LongBytesBound(toUpperCaseLocalized(there.toString(), Locale.ROOT).replace("I", "i")); return there.equals(back) && there.equals(BACK); }); } diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java index 27d4d19528c4..dea0d867eefe 100644 --- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java +++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java @@ -282,6 +282,7 @@ public class DatabaseDescriptorRefTest "org.apache.cassandra.utils.CloseableIterator", "org.apache.cassandra.utils.FBUtilities", "org.apache.cassandra.utils.FBUtilities$1", + "org.apache.cassandra.utils.LocalizeString", "org.apache.cassandra.utils.SystemInfo", "org.apache.cassandra.utils.Pair", "org.apache.cassandra.utils.binlog.BinLogOptions", diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index 624b069e9c15..54171f9a0885 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -39,7 +39,6 @@ import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.List; -import java.util.Locale; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -204,6 +203,7 @@ import static org.apache.cassandra.cql3.SchemaElement.SchemaElementType.TYPE; import static org.apache.cassandra.metrics.CassandraMetricsRegistry.createMetricsKeyspaceTables; import static org.apache.cassandra.schema.SchemaConstants.VIRTUAL_METRICS; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; @@ -1046,7 +1046,7 @@ protected String createKeyspaceName() private String createSchemaElementName(SchemaElement.SchemaElementType type, String keyspace) { String prefix = keyspace == null ? "" : keyspace + '.'; - String typeName = type == MATERIALIZED_VIEW ? "mv" : type.name().toLowerCase(Locale.US); + String typeName = type == MATERIALIZED_VIEW ? "mv" : toLowerCaseLocalized(type.name()); int sequence = seqNumber.getAndIncrement(); int usedSpaceSoFar = prefix.length() + typeName.length() + Math.max(2, numberOfDigits(sequence)) + 1; String testMethodName = StringUtils.truncate(getTestMethodName(), SchemaConstants.NAME_LENGTH - usedSpaceSoFar); @@ -1329,7 +1329,7 @@ protected static Pair<String, String> getCreateIndexName(String keyspace, String index = ParseUtils.isQuoted(index, '\"') ? ParseUtils.unDoubleQuote(index) - : index.toLowerCase(); + : toLowerCaseLocalized(index); return Pair.create(keyspace, index); } @@ -2738,7 +2738,7 @@ protected static Gauge<Integer> getPausedConnectionsGauge() private String getTestMethodName() { - return decorateCQLWithTestNames && testName.getMethodName() != null ? '_' + testName.getMethodName().toLowerCase().replaceAll("[^\\w]", "_") + return decorateCQLWithTestNames && testName.getMethodName() != null ? '_' + toLowerCaseLocalized(testName.getMethodName()).replaceAll("[^\\w]", "_") : ""; } diff --git a/test/unit/org/apache/cassandra/cql3/KeywordTestBase.java b/test/unit/org/apache/cassandra/cql3/KeywordTestBase.java index aa6e508fa47d..274041e27c9c 100644 --- a/test/unit/org/apache/cassandra/cql3/KeywordTestBase.java +++ b/test/unit/org/apache/cassandra/cql3/KeywordTestBase.java @@ -31,6 +31,8 @@ import org.apache.cassandra.exceptions.SyntaxException; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + /** * This class tests all keywords which took a long time. Hence it was split into multiple * KeywordTestSplitN to prevent CI timing out. If timeouts reappear split it further @@ -105,7 +107,7 @@ public void test() throws Throwable logger.info(selectStatement); rs = execute(selectStatement); row = rs.one(); - String value = row.getString(keyword.toLowerCase()); + String value = row.getString(toLowerCaseLocalized(keyword)); Assert.assertEquals(keyword, value); /* Make a materialized view using the fields (cannot re-use the name as MV must be in same keyspace). diff --git a/test/unit/org/apache/cassandra/cql3/functions/NativeFunctionsTest.java b/test/unit/org/apache/cassandra/cql3/functions/NativeFunctionsTest.java index be55d17038e0..4ecfc7d99be8 100644 --- a/test/unit/org/apache/cassandra/cql3/functions/NativeFunctionsTest.java +++ b/test/unit/org/apache/cassandra/cql3/functions/NativeFunctionsTest.java @@ -29,6 +29,8 @@ import org.apache.cassandra.schema.UserFunctions; import org.assertj.core.api.Assertions; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + public class NativeFunctionsTest { /** @@ -154,7 +156,7 @@ public void testDeprectedFunctionNames() Assertions.assertThat(function.argTypes()).isEqualTo(newFunction.argTypes()); Assertions.assertThat(function.returnType()).isEqualTo(newFunction.returnType()); Assertions.assertThat(function.getClass()).isEqualTo(newFunction.getClass()); - Assertions.assertThat(function.name().name.toLowerCase()) + Assertions.assertThat(toLowerCaseLocalized(function.name().name)) .isEqualTo(StringUtils.remove(newFunction.name().name, '_')); } } @@ -190,7 +192,7 @@ public void testDeprectedFunctionFactoryNames() Assertions.assertThat(factory.name).isNotEqualTo(newFactory.name); Assertions.assertThat(factory.parameters).isEqualTo(newFactory.parameters); Assertions.assertThat(factory.getClass()).isEqualTo(newFactory.getClass()); - Assertions.assertThat(factory.name().name.toLowerCase()) + Assertions.assertThat(toLowerCaseLocalized(factory.name().name)) .isEqualTo(StringUtils.remove(newFactory.name().name, '_')); } } @@ -198,7 +200,7 @@ public void testDeprectedFunctionFactoryNames() private static boolean satisfiesConventions(FunctionName functionName) { String name = functionName.name; - return name.equals(name.toLowerCase()) && + return name.equals(toLowerCaseLocalized(name)) && !LEGACY_FUNCTION_NAMES.containsKey(name); } } diff --git a/test/unit/org/apache/cassandra/cql3/functions/masking/PartialMaskingFunctionTest.java b/test/unit/org/apache/cassandra/cql3/functions/masking/PartialMaskingFunctionTest.java index 89d20d0d5821..3a1d45516953 100644 --- a/test/unit/org/apache/cassandra/cql3/functions/masking/PartialMaskingFunctionTest.java +++ b/test/unit/org/apache/cassandra/cql3/functions/masking/PartialMaskingFunctionTest.java @@ -31,6 +31,7 @@ import org.assertj.core.api.Assertions; import static java.lang.String.format; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; /** * Tests for {@link PartialMaskingFunction}. @@ -46,7 +47,7 @@ protected void testMaskingOnColumn(String name, CQL3Type type, Object value) thr protected void testMaskingOnColumn(PartialMaskingFunction.Kind masker, String name, CQL3Type type, Object value) throws Throwable { - String functionName = SchemaConstants.SYSTEM_KEYSPACE_NAME + ".mask_" + masker.name().toLowerCase(); + String functionName = SchemaConstants.SYSTEM_KEYSPACE_NAME + ".mask_" + toLowerCaseLocalized(masker.name()); if (type.getType() instanceof StringType) { diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java index 2d2a626e97fe..c1365e4cc36d 100644 --- a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java +++ b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java @@ -19,7 +19,6 @@ import java.nio.ByteBuffer; import java.util.HashMap; -import java.util.Locale; import java.util.Map; import java.util.UUID; import java.util.concurrent.Callable; @@ -64,6 +63,7 @@ import static org.apache.cassandra.utils.ByteBufferUtil.EMPTY_BYTE_BUFFER; import static org.apache.cassandra.utils.ByteBufferUtil.bytes; import static org.apache.cassandra.utils.Clock.Global.nanoTime; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -113,7 +113,7 @@ private void testCreateAndDropIndex(String indexName, boolean addKeyspaceOnDrop) { assertInvalidMessage(format("Index '%s.%s' doesn't exist", KEYSPACE, - removeQuotes(indexName.toLowerCase(Locale.US))), + removeQuotes(toLowerCaseLocalized(indexName))), format("DROP INDEX %s.%s", KEYSPACE, indexName)); createTable("CREATE TABLE %s (a int primary key, b int);"); @@ -121,7 +121,7 @@ private void testCreateAndDropIndex(String indexName, boolean addKeyspaceOnDrop) createIndexAsync("CREATE INDEX IF NOT EXISTS " + indexName + " ON %s(b);"); assertInvalidMessage(format("Index '%s' already exists", - removeQuotes(indexName.toLowerCase(Locale.US))), + removeQuotes(toLowerCaseLocalized(indexName))), "CREATE INDEX " + indexName + " ON %s(b)"); // IF NOT EXISTS should apply in cases where the new index differs from an existing one in name only @@ -130,8 +130,8 @@ private void testCreateAndDropIndex(String indexName, boolean addKeyspaceOnDrop) createIndexAsync("CREATE INDEX IF NOT EXISTS " + otherIndexName + " ON %s(b)"); assertEquals(1, getCurrentColumnFamilyStore().metadata().indexes.size()); assertInvalidMessage(format("Index %s is a duplicate of existing index %s", - removeQuotes(otherIndexName.toLowerCase(Locale.US)), - removeQuotes(indexName.toLowerCase(Locale.US))), + removeQuotes(toLowerCaseLocalized(otherIndexName)), + removeQuotes(toLowerCaseLocalized(indexName))), "CREATE INDEX " + otherIndexName + " ON %s(b)"); execute("INSERT INTO %s (a, b) values (?, ?);", 0, 0); @@ -156,7 +156,7 @@ private void testCreateAndDropIndex(String indexName, boolean addKeyspaceOnDrop) dropIndex(format("DROP INDEX IF EXISTS %s.%s", KEYSPACE, indexName)); assertInvalidMessage(format("Index '%s.%s' doesn't exist", KEYSPACE, - removeQuotes(indexName.toLowerCase(Locale.US))), + removeQuotes(toLowerCaseLocalized(indexName))), format("DROP INDEX %s.%s", KEYSPACE, indexName)); } diff --git a/test/unit/org/apache/cassandra/db/compaction/unified/ControllerTest.java b/test/unit/org/apache/cassandra/db/compaction/unified/ControllerTest.java index 162e7fc8a5af..7b8f0477e792 100644 --- a/test/unit/org/apache/cassandra/db/compaction/unified/ControllerTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/unified/ControllerTest.java @@ -45,6 +45,7 @@ import org.mockito.MockitoAnnotations; import static java.lang.String.format; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; import static org.assertj.core.api.Assertions.assertThatExceptionOfType; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; @@ -142,7 +143,7 @@ private static void addOptions(boolean useIntegers, Map<String, String> options) options.putIfAbsent(Controller.TARGET_SSTABLE_SIZE_OPTION, FBUtilities.prettyPrintMemory(100 << 20)); // The below value is based on the value in the above statement. Decreasing the above statement should result in a decrease below. options.putIfAbsent(Controller.MIN_SSTABLE_SIZE_OPTION, "70.710MiB"); - options.putIfAbsent(Controller.OVERLAP_INCLUSION_METHOD_OPTION, Overlaps.InclusionMethod.SINGLE.toString().toLowerCase()); + options.putIfAbsent(Controller.OVERLAP_INCLUSION_METHOD_OPTION, toLowerCaseLocalized(Overlaps.InclusionMethod.SINGLE.toString())); options.putIfAbsent(Controller.SSTABLE_GROWTH_OPTION, "0.5"); } diff --git a/test/unit/org/apache/cassandra/db/guardrails/GuardrailPasswordTest.java b/test/unit/org/apache/cassandra/db/guardrails/GuardrailPasswordTest.java index d06a1696bb6c..774868079d7e 100644 --- a/test/unit/org/apache/cassandra/db/guardrails/GuardrailPasswordTest.java +++ b/test/unit/org/apache/cassandra/db/guardrails/GuardrailPasswordTest.java @@ -36,6 +36,7 @@ import static java.util.Collections.singletonList; import static org.apache.cassandra.db.guardrails.CassandraPasswordConfiguration.LENGTH_FAIL_KEY; import static org.apache.cassandra.db.guardrails.CassandraPasswordConfiguration.LENGTH_WARN_KEY; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -121,7 +122,7 @@ public void testAllSpecialCharactersArePossibleToUseInCQLQuery() private String getEntityName(String name) { - return (name + entity).toLowerCase(); + return toLowerCaseLocalized(name + entity); } private void testPasswordGuardrailInternal() throws Throwable diff --git a/test/unit/org/apache/cassandra/db/tries/TrieToDotTest.java b/test/unit/org/apache/cassandra/db/tries/TrieToDotTest.java index b4955dbc0875..92d59269d6b2 100644 --- a/test/unit/org/apache/cassandra/db/tries/TrieToDotTest.java +++ b/test/unit/org/apache/cassandra/db/tries/TrieToDotTest.java @@ -22,6 +22,8 @@ import org.apache.cassandra.io.compress.BufferType; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + public class TrieToDotTest { @Test @@ -30,7 +32,7 @@ public void testToDotContent() throws Exception InMemoryTrie<String> trie = new InMemoryTrie<>(BufferType.OFF_HEAP); String s = "Trie node types and manipulation mechanisms. The main purpose of this is to allow for handling tries directly as" + " they are on disk without any serialization, and to enable the creation of such files."; - s = s.toLowerCase(); + s = toLowerCaseLocalized(s); for (String word : s.split("[^a-z]+")) trie.putRecursive(InMemoryTrieTestBase.comparable(word), word, (x, y) -> y); diff --git a/test/unit/org/apache/cassandra/db/tries/TrieToMermaidTest.java b/test/unit/org/apache/cassandra/db/tries/TrieToMermaidTest.java index 61b5f4f8931d..ed1afa462e03 100644 --- a/test/unit/org/apache/cassandra/db/tries/TrieToMermaidTest.java +++ b/test/unit/org/apache/cassandra/db/tries/TrieToMermaidTest.java @@ -22,6 +22,8 @@ import org.apache.cassandra.io.compress.BufferType; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + public class TrieToMermaidTest { @Test @@ -30,7 +32,7 @@ public void testToMermaidContent() throws Exception InMemoryTrie<String> trie = new InMemoryTrie<>(BufferType.OFF_HEAP); // This was used as a basis the graphs in BTIFormat.md String s = "a allow an and any are as node of on the this to trie types with without"; - s = s.toLowerCase(); + s = toLowerCaseLocalized(s); for (String word : s.split("[^a-z]+")) trie.putRecursive(InMemoryTrieTestBase.comparable(word), word, (x, y) -> y); diff --git a/test/unit/org/apache/cassandra/db/virtual/LocalRepairTablesTest.java b/test/unit/org/apache/cassandra/db/virtual/LocalRepairTablesTest.java index 97dea78484e9..96f09400ecaf 100644 --- a/test/unit/org/apache/cassandra/db/virtual/LocalRepairTablesTest.java +++ b/test/unit/org/apache/cassandra/db/virtual/LocalRepairTablesTest.java @@ -56,6 +56,8 @@ import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.TimeUUID; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + public class LocalRepairTablesTest extends CQLTester { private static final String KS_NAME = "vts"; @@ -245,7 +247,7 @@ private void assertInit(String table, State<?, ?> state) throws Throwable private <T extends Enum<T>> void assertState(String table, State<?, ?> state, T expectedState) throws Throwable { assertRowsIgnoringOrder(execute(t("SELECT id, completed, status, failure_cause, success_message FROM %s." + table + " WHERE id = ?"), state.getId()), - row(state.getId(), false, expectedState.name().toLowerCase(), null, null)); + row(state.getId(), false, toLowerCaseLocalized(expectedState.name()), null, null)); } private void assertSuccess(String table, State<?, ?> state) throws Throwable diff --git a/test/unit/org/apache/cassandra/db/virtual/SSTableTasksTableTest.java b/test/unit/org/apache/cassandra/db/virtual/SSTableTasksTableTest.java index e6de58d37a67..5d387f1d9e75 100644 --- a/test/unit/org/apache/cassandra/db/virtual/SSTableTasksTableTest.java +++ b/test/unit/org/apache/cassandra/db/virtual/SSTableTasksTableTest.java @@ -37,6 +37,7 @@ import org.apache.cassandra.schema.MockSchema; import org.apache.cassandra.utils.TimeUUID; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; import static org.apache.cassandra.utils.TimeUUID.Generator.nextTimeUUID; public class SSTableTasksTableTest extends CQLTester @@ -91,7 +92,7 @@ public boolean isGlobal() CompactionManager.instance.active.beginCompaction(compactionHolder); UntypedResultSet result = execute("SELECT * FROM vts.sstable_tasks"); assertRows(result, row(CQLTester.KEYSPACE, currentTable(), compactionId, 1.0 * bytesCompacted / bytesTotal, - OperationType.COMPACTION.toString().toLowerCase(), bytesCompacted, sstables.size(), + toLowerCaseLocalized(OperationType.COMPACTION.toString()), bytesCompacted, sstables.size(), directory, bytesTotal, CompactionInfo.Unit.BYTES.toString())); CompactionManager.instance.active.finishCompaction(compactionHolder); diff --git a/test/unit/org/apache/cassandra/db/virtual/StreamingVirtualTableTest.java b/test/unit/org/apache/cassandra/db/virtual/StreamingVirtualTableTest.java index 07f2815c8698..30a70338f94b 100644 --- a/test/unit/org/apache/cassandra/db/virtual/StreamingVirtualTableTest.java +++ b/test/unit/org/apache/cassandra/db/virtual/StreamingVirtualTableTest.java @@ -54,6 +54,7 @@ import org.apache.cassandra.utils.FBUtilities; import org.assertj.core.util.Throwables; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; import static org.apache.cassandra.utils.TimeUUID.Generator.nextTimeUUID; public class StreamingVirtualTableTest extends CQLTester @@ -185,7 +186,7 @@ private static long progressEvent(StreamingState state, SessionInfo s, List<Stre long fileSize = summary.totalSize / summary.files; for (int i = 0; i < summary.files - 1; i++) { - String fileName = summary.tableId + "-" + direction.name().toLowerCase() + "-" + i; + String fileName = summary.tableId + "-" + toLowerCaseLocalized(direction.name()) + "-" + i; state.handleStreamEvent(new ProgressEvent(state.id(), new ProgressInfo((InetAddressAndPort) s.peer, 0, fileName, direction, fileSize, fileSize, fileSize))); counter += fileSize; } @@ -199,7 +200,7 @@ private static long completeEvent(StreamingState state, SessionInfo s, List<Stre for (StreamSummary summary : summaries) { long fileSize = summary.totalSize / summary.files; - String fileName = summary.tableId + "-" + direction.name().toLowerCase() + "-" + summary.files; + String fileName = summary.tableId + "-" + toLowerCaseLocalized(direction.name()) + "-" + summary.files; state.handleStreamEvent(new ProgressEvent(state.id(), new ProgressInfo((InetAddressAndPort) s.peer, 0, fileName, direction, fileSize, fileSize, fileSize))); counter += fileSize; } diff --git a/test/unit/org/apache/cassandra/index/sai/analyzer/filter/BasicFiltersTest.java b/test/unit/org/apache/cassandra/index/sai/analyzer/filter/BasicFiltersTest.java index 01faf488c11d..9e7886e9bb82 100644 --- a/test/unit/org/apache/cassandra/index/sai/analyzer/filter/BasicFiltersTest.java +++ b/test/unit/org/apache/cassandra/index/sai/analyzer/filter/BasicFiltersTest.java @@ -25,6 +25,7 @@ import org.apache.cassandra.index.sai.SAITester; import org.apache.lucene.analysis.miscellaneous.ASCIIFoldingFilter; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; import static org.junit.Assert.assertEquals; public class BasicFiltersTest @@ -37,7 +38,7 @@ public void testLowerCase() for (int count = 0; count < SAITester.getRandom().nextIntBetween(100, 1000); count++) { String actual = SAITester.getRandom().nextTextString(10, 50); - assertEquals(actual.toLowerCase(), lowerCase.process(actual)); + assertEquals(toLowerCaseLocalized(actual), lowerCase.process(actual)); } } diff --git a/test/unit/org/apache/cassandra/index/sasi/analyzer/DelimiterAnalyzerTest.java b/test/unit/org/apache/cassandra/index/sasi/analyzer/DelimiterAnalyzerTest.java index c3589d11fb60..0cb57a607b0f 100644 --- a/test/unit/org/apache/cassandra/index/sasi/analyzer/DelimiterAnalyzerTest.java +++ b/test/unit/org/apache/cassandra/index/sasi/analyzer/DelimiterAnalyzerTest.java @@ -34,6 +34,7 @@ import org.junit.Assert; import org.junit.Test; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; import static org.junit.Assert.assertEquals; public class DelimiterAnalyzerTest @@ -59,7 +60,7 @@ public void caseSensitiveAnalizer() throws Exception output.append(ByteBufferUtil.string(analyzer.next()) + (analyzer.hasNext() ? ' ' : "")); Assert.assertEquals(testString, output.toString()); - Assert.assertFalse(testString.toLowerCase().equals(output.toString())); + Assert.assertFalse(toLowerCaseLocalized(testString).equals(output.toString())); } @Test @@ -82,7 +83,7 @@ public void testBlankEntries() throws Exception output.append(ByteBufferUtil.string(analyzer.next()) + (analyzer.hasNext() ? ',' : "")); Assert.assertEquals("Nip,it,in,the,bud", output.toString()); - Assert.assertFalse(testString.toLowerCase().equals(output.toString())); + Assert.assertFalse(toLowerCaseLocalized(testString).equals(output.toString())); } @Test(expected = ConfigurationException.class) diff --git a/test/unit/org/apache/cassandra/index/sasi/analyzer/NonTokenizingAnalyzerTest.java b/test/unit/org/apache/cassandra/index/sasi/analyzer/NonTokenizingAnalyzerTest.java index ba67853c4389..abe497813286 100644 --- a/test/unit/org/apache/cassandra/index/sasi/analyzer/NonTokenizingAnalyzerTest.java +++ b/test/unit/org/apache/cassandra/index/sasi/analyzer/NonTokenizingAnalyzerTest.java @@ -26,6 +26,8 @@ import org.junit.Assert; import org.junit.Test; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + /** * Tests for the non-tokenizing analyzer */ @@ -45,7 +47,7 @@ public void caseInsensitiveAnalizer() throws Exception ByteBuffer analyzed = null; while (analyzer.hasNext()) analyzed = analyzer.next(); - Assert.assertTrue(testString.toLowerCase().equals(ByteBufferUtil.string(analyzed))); + Assert.assertTrue(toLowerCaseLocalized(testString).equals(ByteBufferUtil.string(analyzed))); } @Test @@ -61,7 +63,7 @@ public void caseSensitiveAnalizer() throws Exception ByteBuffer analyzed = null; while (analyzer.hasNext()) analyzed = analyzer.next(); - Assert.assertFalse(testString.toLowerCase().equals(ByteBufferUtil.string(analyzed))); + Assert.assertFalse(toLowerCaseLocalized(testString).equals(ByteBufferUtil.string(analyzed))); } @Test diff --git a/test/unit/org/apache/cassandra/service/reads/repair/AbstractReadRepairTest.java b/test/unit/org/apache/cassandra/service/reads/repair/AbstractReadRepairTest.java index fa6783566c3f..b2ff6723eea2 100644 --- a/test/unit/org/apache/cassandra/service/reads/repair/AbstractReadRepairTest.java +++ b/test/unit/org/apache/cassandra/service/reads/repair/AbstractReadRepairTest.java @@ -83,6 +83,7 @@ import static org.apache.cassandra.locator.ReplicaUtils.FULL_RANGE; import static org.apache.cassandra.net.Verb.INTERNAL_RSP; import static org.apache.cassandra.utils.Clock.Global.nanoTime; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; @Ignore public abstract class AbstractReadRepairTest @@ -288,7 +289,7 @@ public String getDatacenter(InetAddressAndPort endpoint) String ksName = "ks"; String ddl = String.format("CREATE TABLE tbl (k int primary key, v text) WITH read_repair='%s'", - repairStrategy.toString().toLowerCase()); + toLowerCaseLocalized(repairStrategy.toString())); cfm = CreateTableStatement.parse(ddl, ksName).build(); assert cfm.params.readRepair == repairStrategy; diff --git a/test/unit/org/apache/cassandra/tools/TopPartitionsTest.java b/test/unit/org/apache/cassandra/tools/TopPartitionsTest.java index 90b431cc7f1a..9a218727b977 100644 --- a/test/unit/org/apache/cassandra/tools/TopPartitionsTest.java +++ b/test/unit/org/apache/cassandra/tools/TopPartitionsTest.java @@ -43,9 +43,9 @@ import org.apache.cassandra.service.StorageService; import org.apache.cassandra.Util; - import static java.lang.String.format; import static org.apache.cassandra.cql3.QueryProcessor.executeInternal; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; import static org.hamcrest.Matchers.greaterThan; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -56,7 +56,7 @@ */ public class TopPartitionsTest { - public static String KEYSPACE = TopPartitionsTest.class.getSimpleName().toLowerCase(); + public static String KEYSPACE = toLowerCaseLocalized(TopPartitionsTest.class.getSimpleName()); public static String TABLE = "test"; @BeforeClass diff --git a/tools/stress/src/org/apache/cassandra/stress/StressAction.java b/tools/stress/src/org/apache/cassandra/stress/StressAction.java index 5cf429bcdc7c..729090709f78 100644 --- a/tools/stress/src/org/apache/cassandra/stress/StressAction.java +++ b/tools/stress/src/org/apache/cassandra/stress/StressAction.java @@ -40,6 +40,7 @@ import com.google.common.util.concurrent.Uninterruptibles; import static org.apache.cassandra.utils.Clock.Global.nanoTime; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; public class StressAction implements Runnable { @@ -217,7 +218,7 @@ private StressMetrics run(OpDistributionFactory operations, output.println(String.format("Running %s with %d threads %s", operations.desc(), threadCount, - durationUnits != null ? duration + " " + durationUnits.toString().toLowerCase() + durationUnits != null ? duration + " " + toLowerCaseLocalized(durationUnits.toString()) : opCount > 0 ? "for " + opCount + " iteration" : "until stderr of mean < " + settings.command.targetUncertainty)); final WorkManager workManager; diff --git a/tools/stress/src/org/apache/cassandra/stress/StressGraph.java b/tools/stress/src/org/apache/cassandra/stress/StressGraph.java index 9b38bb80f26e..76940525998d 100644 --- a/tools/stress/src/org/apache/cassandra/stress/StressGraph.java +++ b/tools/stress/src/org/apache/cassandra/stress/StressGraph.java @@ -40,6 +40,8 @@ import org.apache.cassandra.stress.settings.StressSettings; import org.apache.cassandra.utils.JsonUtils; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + public class StressGraph { private StressSettings stressSettings; @@ -200,7 +202,7 @@ else if (mode == ReadingMode.AGGREGATES) continue; } // the graphing js expects lower case names - json.put(parts[0].trim().toLowerCase(), parts[1].trim()); + json.put(toLowerCaseLocalized(parts[0].trim()), parts[1].trim()); } else if (mode == ReadingMode.NEXTITERATION) { diff --git a/tools/stress/src/org/apache/cassandra/stress/StressProfile.java b/tools/stress/src/org/apache/cassandra/stress/StressProfile.java index 5fe3b9dca703..eb95560a352c 100644 --- a/tools/stress/src/org/apache/cassandra/stress/StressProfile.java +++ b/tools/stress/src/org/apache/cassandra/stress/StressProfile.java @@ -62,6 +62,9 @@ import org.yaml.snakeyaml.constructor.Constructor; import org.yaml.snakeyaml.error.YAMLException; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; + public class StressProfile implements Serializable { private String keyspaceCql; @@ -357,7 +360,7 @@ public Operation getQuery(String name, StressSettings settings, boolean isWarmup) { - name = name.toLowerCase(); + name = toLowerCaseLocalized(name); if (!queries.containsKey(name)) throw new IllegalArgumentException("No query defined with name " + name); @@ -373,10 +376,10 @@ public Operation getQuery(String name, Map<String, SchemaStatement.ArgSelect> args = new HashMap<>(); for (Map.Entry<String, StressYaml.QueryDef> e : queries.entrySet()) { - stmts.put(e.getKey().toLowerCase(), jclient.prepare(e.getValue().cql)); - args.put(e.getKey().toLowerCase(), e.getValue().fields == null + stmts.put(toLowerCaseLocalized(e.getKey()), jclient.prepare(e.getValue().cql)); + args.put(toLowerCaseLocalized(e.getKey()), e.getValue().fields == null ? SchemaStatement.ArgSelect.MULTIROW - : SchemaStatement.ArgSelect.valueOf(e.getValue().fields.toUpperCase())); + : SchemaStatement.ArgSelect.valueOf(toUpperCaseLocalized(e.getValue().fields))); } queryStatements = stmts; argSelects = args; @@ -395,7 +398,7 @@ static boolean dynamicConditionExists(PreparedStatement statement) throws Illega if (statement == null) return false; - if (!statement.getQueryString().toUpperCase().startsWith("UPDATE")) + if (!toUpperCaseLocalized(statement.getQueryString()).startsWith("UPDATE")) return false; ModificationStatement.Parsed modificationStatement; @@ -758,7 +761,7 @@ Generator getGenerator() static Generator getGenerator(final String name, final String type, final String collectionType, GeneratorConfig config) { - switch (type.toUpperCase()) + switch (toUpperCaseLocalized(type)) { case "ASCII": case "TEXT": @@ -848,7 +851,7 @@ static <V> void lowerCase(Map<String, V> map) } } for (Map.Entry<String, V> e : reinsert) - map.put(e.getKey().toLowerCase(), e.getValue()); + map.put(toLowerCaseLocalized(e.getKey()), e.getValue()); } /* Quote a identifier if it contains uppercase letters */ diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/CliOption.java b/tools/stress/src/org/apache/cassandra/stress/settings/CliOption.java index 550467171db0..a3f649d6b727 100644 --- a/tools/stress/src/org/apache/cassandra/stress/settings/CliOption.java +++ b/tools/stress/src/org/apache/cassandra/stress/settings/CliOption.java @@ -24,6 +24,8 @@ import java.util.HashMap; import java.util.Map; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + public enum CliOption { POP("Population distribution and intra-partition visit order", SettingsPopulation.helpPrinter()), @@ -50,7 +52,7 @@ public enum CliOption final Map<String, CliOption> lookup = new HashMap<>(); for (CliOption cmd : values()) { - lookup.put("-" + cmd.toString().toLowerCase(), cmd); + lookup.put("-" + toLowerCaseLocalized(cmd.toString()), cmd); if (cmd.extraName != null) lookup.put(cmd.extraName, cmd); } @@ -59,7 +61,7 @@ public enum CliOption public static CliOption get(String command) { - return LOOKUP.get(command.toLowerCase()); + return LOOKUP.get(toLowerCaseLocalized(command)); } public final String extraName; diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/Command.java b/tools/stress/src/org/apache/cassandra/stress/settings/Command.java index c99afabbedee..849335c309e3 100644 --- a/tools/stress/src/org/apache/cassandra/stress/settings/Command.java +++ b/tools/stress/src/org/apache/cassandra/stress/settings/Command.java @@ -28,6 +28,8 @@ import com.google.common.collect.ImmutableList; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + public enum Command { @@ -78,7 +80,7 @@ public enum Command public static Command get(String command) { - return LOOKUP.get(command.toLowerCase()); + return LOOKUP.get(toLowerCaseLocalized(command)); } public final boolean updates; @@ -98,12 +100,12 @@ public static Command get(String command) this.updates = updates; this.category = category; List<String> names = new ArrayList<>(); - names.add(this.toString().toLowerCase()); - names.add(this.toString().replaceAll("_", "").toLowerCase()); + names.add(toLowerCaseLocalized(this.toString())); + names.add(toLowerCaseLocalized(this.toString().replaceAll("_", ""))); if (extra != null) { - names.add(extra.toLowerCase()); - names.add(extra.replaceAll("_", "").toLowerCase()); + names.add(toLowerCaseLocalized(extra)); + names.add(toLowerCaseLocalized(extra.replaceAll("_", ""))); } this.names = ImmutableList.copyOf(names); this.description = description; diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/OptionDistribution.java b/tools/stress/src/org/apache/cassandra/stress/settings/OptionDistribution.java index cc93323c8803..040e9f2a1d26 100644 --- a/tools/stress/src/org/apache/cassandra/stress/settings/OptionDistribution.java +++ b/tools/stress/src/org/apache/cassandra/stress/settings/OptionDistribution.java @@ -34,6 +34,8 @@ import org.apache.cassandra.stress.generate.*; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + /** * For selecting a mathematical distribution */ @@ -73,7 +75,7 @@ public OptionDistribution(String prefix, String defaultSpec, String description, @Override public boolean accept(String param) { - if (!param.toLowerCase().startsWith(prefix)) + if (!toLowerCaseLocalized(param).startsWith(prefix)) return false; spec = param.substring(prefix.length()); return true; @@ -86,7 +88,7 @@ public static DistributionFactory get(String spec) throw new IllegalArgumentException("Illegal distribution specification: " + spec); boolean inverse = m.group(1).equals("~"); String name = m.group(2); - Impl impl = LOOKUP.get(name.toLowerCase()); + Impl impl = LOOKUP.get(toLowerCaseLocalized(name)); if (impl == null) throw new IllegalArgumentException("Illegal distribution type: " + name); List<String> params = new ArrayList<>(); @@ -181,7 +183,7 @@ private static interface Impl public static long parseLong(String value) { long multiplier = 1; - value = value.trim().toLowerCase(); + value = toLowerCaseLocalized(value.trim()); switch (value.charAt(value.length() - 1)) { case 'b': diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/OptionEnumProbabilities.java b/tools/stress/src/org/apache/cassandra/stress/settings/OptionEnumProbabilities.java index c218176469e0..17cc5cff2e4a 100644 --- a/tools/stress/src/org/apache/cassandra/stress/settings/OptionEnumProbabilities.java +++ b/tools/stress/src/org/apache/cassandra/stress/settings/OptionEnumProbabilities.java @@ -26,6 +26,7 @@ import java.util.List; import java.util.Map; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; public final class OptionEnumProbabilities<T> extends OptionMulti { @@ -48,7 +49,7 @@ private static final class OptMatcher<T> extends OptionSimple final T opt; OptMatcher(T opt, String defaultValue) { - super(opt.toString().toLowerCase() + "=", "[0-9]+(\\.[0-9]+)?", defaultValue, "Performs this many " + opt + " operations out of total", false); + super(toLowerCaseLocalized(opt.toString()) + "=", "[0-9]+(\\.[0-9]+)?", defaultValue, "Performs this many " + opt + " operations out of total", false); this.opt = opt; } } diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommand.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommand.java index af10719a6f11..234cb415e1a2 100644 --- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommand.java +++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommand.java @@ -34,6 +34,9 @@ import org.apache.cassandra.stress.util.ResultLogger; import org.apache.cassandra.db.ConsistencyLevel; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; + // Generic command settings - common to read/write/etc public abstract class SettingsCommand implements Serializable { @@ -68,9 +71,9 @@ public SettingsCommand(Command type, GroupedOptions options) public SettingsCommand(Command type, Options options, Count count, Duration duration, Uncertainty uncertainty) { this.type = type; - this.consistencyLevel = ConsistencyLevel.valueOf(options.consistencyLevel.value().toUpperCase()); + this.consistencyLevel = ConsistencyLevel.valueOf(toUpperCaseLocalized(options.consistencyLevel.value())); this.noWarmup = options.noWarmup.setByUser(); - this.truncate = TruncateWhen.valueOf(options.truncate.value().toUpperCase()); + this.truncate = TruncateWhen.valueOf(toUpperCaseLocalized(options.truncate.value())); if (count != null) { @@ -85,7 +88,7 @@ else if (duration != null) { this.count = -1; this.duration = Long.parseLong(duration.duration.value().substring(0, duration.duration.value().length() - 1)); - switch (duration.duration.value().toLowerCase().charAt(duration.duration.value().length() - 1)) + switch (toLowerCaseLocalized(duration.duration.value()).charAt(duration.duration.value().length() - 1)) { case 's': this.durationUnits = TimeUnit.SECONDS; @@ -178,7 +181,7 @@ protected void truncateTables(StressSettings settings, String ks, String ... tab public void printSettings(ResultLogger out) { - out.printf(" Type: %s%n", type.toString().toLowerCase()); + out.printf(" Type: %s%n", toLowerCaseLocalized(type.toString())); out.printf(" Count: %,d%n", count); if (durationUnits != null) { @@ -226,11 +229,11 @@ static SettingsCommand get(Map<String, String[]> clArgs) static void printHelp(Command type) { - printHelp(type.toString().toLowerCase()); + printHelp(toLowerCaseLocalized(type.toString())); } static void printHelp(String type) { - GroupedOptions.printOptions(System.out, type.toLowerCase(), new Uncertainty(), new Count(), new Duration()); + GroupedOptions.printOptions(System.out, toLowerCaseLocalized(type), new Uncertainty(), new Count(), new Duration()); } } diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandPreDefined.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandPreDefined.java index 88755765db1e..d366d963b960 100644 --- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandPreDefined.java +++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsCommandPreDefined.java @@ -41,6 +41,8 @@ import org.apache.cassandra.stress.report.Timer; import org.apache.cassandra.stress.util.ResultLogger; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + // Settings unique to the mixed command type public class SettingsCommandPreDefined extends SettingsCommand { @@ -146,12 +148,12 @@ public static SettingsCommandPreDefined build(Command type, String[] params) static void printHelp(Command type) { - printHelp(type.toString().toLowerCase()); + printHelp(toLowerCaseLocalized(type.toString())); } static void printHelp(String type) { - GroupedOptions.printOptions(System.out, type.toLowerCase(), new Uncertainty(), new Count(), new Duration()); + GroupedOptions.printOptions(System.out, toLowerCaseLocalized(type), new Uncertainty(), new Count(), new Duration()); } static Runnable helpPrinter(final Command type) diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsLog.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsLog.java index ae77e0a7809a..338d2572e2fa 100644 --- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsLog.java +++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsLog.java @@ -29,6 +29,8 @@ import org.apache.cassandra.stress.util.MultiResultLogger; import org.apache.cassandra.stress.util.ResultLogger; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; + public class SettingsLog implements Serializable { public static enum Level @@ -66,7 +68,7 @@ else if (interval.endsWith("s")) intervalMillis = 1000 * Integer.parseInt(interval); if (intervalMillis <= 0) throw new IllegalArgumentException("Log interval must be greater than zero"); - level = Level.valueOf(options.level.value().toUpperCase()); + level = Level.valueOf(toUpperCaseLocalized(options.level.value())); } public MultiResultLogger getOutput() throws FileNotFoundException diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMisc.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMisc.java index 1f21a61da264..2cb5491276e2 100644 --- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMisc.java +++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMisc.java @@ -36,6 +36,8 @@ import org.apache.cassandra.stress.generate.Distribution; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + class SettingsMisc implements Serializable { @@ -164,13 +166,13 @@ public static void printHelp() System.out.println("---Commands---"); for (Command cmd : Command.values()) { - System.out.println(String.format("%-20s : %s", cmd.toString().toLowerCase(), cmd.description)); + System.out.println(String.format("%-20s : %s", toLowerCaseLocalized(cmd.toString()), cmd.description)); } System.out.println(); System.out.println("---Options---"); for (CliOption cmd : CliOption.values()) { - System.out.println(String.format("-%-20s : %s", cmd.toString().toLowerCase(), cmd.description)); + System.out.println(String.format("-%-20s : %s", toLowerCaseLocalized(cmd.toString()), cmd.description)); } } @@ -201,7 +203,7 @@ static Runnable helpHelpPrinter() System.out.println(" " + cmd.names.toString().replaceAll("\\[|\\]", "")); System.out.println("Options:"); for (CliOption op : CliOption.values()) - System.out.println(" -" + op.toString().toLowerCase() + (op.extraName != null ? ", " + op.extraName : "")); + System.out.println(" -" + toLowerCaseLocalized(op.toString()) + (op.extraName != null ? ", " + op.extraName : "")); }; } diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMode.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMode.java index 5f2e7f68b91b..a8303ed64ff0 100644 --- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMode.java +++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMode.java @@ -36,6 +36,7 @@ import static java.lang.String.format; import static org.apache.cassandra.stress.settings.SettingsCredentials.CQL_PASSWORD_PROPERTY_KEY; import static org.apache.cassandra.stress.settings.SettingsCredentials.CQL_USERNAME_PROPERTY_KEY; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; public class SettingsMode implements Serializable { @@ -79,7 +80,7 @@ public SettingsMode(GroupedOptions options, SettingsCredentials credentials) : ProtocolVersion.fromInt(Integer.parseInt(opts.protocolVersion.value())); api = ConnectionAPI.JAVA_DRIVER_NATIVE; style = opts.useUnPrepared.setByUser() ? ConnectionStyle.CQL : ConnectionStyle.CQL_PREPARED; - compression = ProtocolOptions.Compression.valueOf(opts.useCompression.value().toUpperCase()).name(); + compression = ProtocolOptions.Compression.valueOf(toUpperCaseLocalized(opts.useCompression.value())).name(); username = opts.user.setByUser() ? opts.user.value() : credentials.cqlUsername; password = opts.password.setByUser() ? opts.password.value() : credentials.cqlPassword; maxPendingPerConnection = opts.maxPendingPerConnection.value().isEmpty() ? null : Integer.valueOf(opts.maxPendingPerConnection.value()); diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsPopulation.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsPopulation.java index 66984edb3580..b25b7d2e1687 100644 --- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsPopulation.java +++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsPopulation.java @@ -32,6 +32,8 @@ import org.apache.cassandra.stress.generate.PartitionGenerator; import org.apache.cassandra.stress.util.ResultLogger; +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; + public class SettingsPopulation implements Serializable { @@ -48,7 +50,7 @@ public static enum GenerateOrder private SettingsPopulation(GenerateOptions options, DistributionOptions dist, SequentialOptions pop) { - this.order = !options.contents.setByUser() ? PartitionGenerator.Order.ARBITRARY : PartitionGenerator.Order.valueOf(options.contents.value().toUpperCase()); + this.order = !options.contents.setByUser() ? PartitionGenerator.Order.ARBITRARY : PartitionGenerator.Order.valueOf(toUpperCaseLocalized(options.contents.value())); if (dist != null) { this.distribution = dist.seed.get(); diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/StressSettings.java b/tools/stress/src/org/apache/cassandra/stress/settings/StressSettings.java index 9fbec4139de7..6aea048b4cc0 100644 --- a/tools/stress/src/org/apache/cassandra/stress/settings/StressSettings.java +++ b/tools/stress/src/org/apache/cassandra/stress/settings/StressSettings.java @@ -29,6 +29,8 @@ import org.apache.cassandra.stress.util.ResultLogger; import org.apache.cassandra.transport.SimpleClient; +import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; + public class StressSettings implements Serializable { public final SettingsCommand command; @@ -247,7 +249,7 @@ private static Map<String, String[]> parseMap(String[] args) { if (i > 0) putParam(key, params.toArray(new String[0]), r); - key = args[i].toLowerCase(); + key = toLowerCaseLocalized(args[i]); params.clear(); } else From 3550b88307e87fa6d980f321359ccfcc6067b5a3 Mon Sep 17 00:00:00 2001 From: mck <mck@apache.org> Date: Thu, 24 Oct 2024 15:09:18 +0200 Subject: [PATCH 007/225] Suppress CVE-2024-45772 lucene-core-9.7.0.jar CVE in Lucene Replicator which the project doesn't use. patch by Mick Semb Wever; reviewed by Brandon Williams, Caleb Rackliffe for CASSANDRA-20024 --- .build/owasp/dependency-check-suppressions.xml | 6 ++++++ CHANGES.txt | 1 + 2 files changed, 7 insertions(+) diff --git a/.build/owasp/dependency-check-suppressions.xml b/.build/owasp/dependency-check-suppressions.xml index 3f2233d324b5..994f3cb39562 100644 --- a/.build/owasp/dependency-check-suppressions.xml +++ b/.build/owasp/dependency-check-suppressions.xml @@ -57,4 +57,10 @@ <cve>CVE-2023-6378</cve> <cve>CVE-2023-6481</cve> </suppress> + + <!-- https://issues.apache.org/jira/browse/CASSANDRA-20024 --> + <suppress> + <packageUrl regex="true">^pkg:maven/org\.apache\.lucene/lucene\-.*@9.7.0$</packageUrl> + <cve>CVE-2024-45772</cve> + </suppress> </suppressions> diff --git a/CHANGES.txt b/CHANGES.txt index e9cc7672b9c8..3c96c434bcec 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.3 + * Suppress CVE-2024-45772 from lucene-core-9.7.0.jar (CASSANDRA-20024) Merged from 4.1: Merged from 4.0: * Add configurable batchlog endpoint strategies: random_remote, prefer_local, dynamic_remote, and dynamic (CASSANDRA-18120) From 32030e4fa35813803ebaa93e1aa54964ae2b0cf5 Mon Sep 17 00:00:00 2001 From: Caleb Rackliffe <calebrackliffe@gmail.com> Date: Tue, 29 Oct 2024 10:11:24 -0500 Subject: [PATCH 008/225] Reintroduce RestrictionSet#iterator() optimization around multi-column restrictions patch by Caleb Rackliffe; reviewed by Benjamin Lerer for CASSANDRA-20034 --- CHANGES.txt | 1 + .../cql3/restrictions/RestrictionSet.java | 62 +++++++++++++++++-- 2 files changed, 59 insertions(+), 4 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index d81ffaebb0bd..fb216698c490 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Reintroduce RestrictionSet#iterator() optimization around multi-column restrictions (CASSANDRA-20034) * Explicitly localize strings to Locale.US for internal implementation (CASSANDRA-19953) * Add -H option for human-friendly output in nodetool compactionhistory (CASSANDRA-20015) * Fix type check for referenced duration type for nested types (CASSANDRA-19890) diff --git a/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java b/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java index 9b3c7600804f..b14d3debefe1 100644 --- a/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java +++ b/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java @@ -23,12 +23,15 @@ import java.util.Comparator; import java.util.HashSet; import java.util.Iterator; -import java.util.LinkedHashSet; import java.util.List; import java.util.NavigableMap; import java.util.Set; import java.util.TreeMap; +import com.google.common.collect.AbstractIterator; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + import org.apache.cassandra.cql3.QueryOptions; import org.apache.cassandra.cql3.functions.Function; import org.apache.cassandra.db.filter.RowFilter; @@ -58,13 +61,18 @@ public int compare(ColumnMetadata column, ColumnMetadata otherColumn) }; private static final RestrictionSet EMPTY = new RestrictionSet(Collections.unmodifiableNavigableMap(new TreeMap<>(COLUMN_DEFINITION_COMPARATOR)), - false, false, false,false); + false, false, false, false,false); /** * The restrictions per column. */ private final NavigableMap<ColumnMetadata, SingleRestriction> restrictions; + /** + * {@code true} if it contains multi-column restrictions, {@code false} otherwise. + */ + private final boolean hasMultiColumnRestrictions; + private final boolean hasSlice; private final boolean hasIn; @@ -83,12 +91,14 @@ public static RestrictionSet empty() } private RestrictionSet(NavigableMap<ColumnMetadata, SingleRestriction> restrictions, + boolean hasMultiColumnRestrictions, boolean hasIn, boolean hasSlice, boolean hasAnn, boolean needsFilteringOrIndexing) { this.restrictions = restrictions; + this.hasMultiColumnRestrictions = hasMultiColumnRestrictions; this.hasIn = hasIn; this.hasSlice = hasSlice; this.hasAnn = hasAnn; @@ -184,6 +194,7 @@ public RestrictionSet addRestriction(SingleRestriction restriction) boolean newNeedsFilteringOrIndexing = needsFilteringOrIndexing || restriction.needsFilteringOrIndexing(); return new RestrictionSet(mergeRestrictions(newRestricitons, restriction), + hasMultiColumnRestrictions || restriction.isMultiColumn(), newHasIN, newHasSlice, newHasANN, @@ -260,8 +271,8 @@ public boolean needsFiltering(Index.Group indexGroup) @Override public Iterator<SingleRestriction> iterator() { - // We need to eliminate duplicates in the case where we have multi-column restrictions. - return new LinkedHashSet<>(restrictions.values()).iterator(); + Iterator<SingleRestriction> iterator = restrictions.values().iterator(); + return hasMultiColumnRestrictions ? new DistinctIterator<>(iterator) : iterator; } @Override @@ -301,4 +312,47 @@ SingleRestriction lastRestriction() { return restrictions.lastEntry().getValue(); } + + /** + * {@code Iterator} decorator that removes duplicates in an ordered one. + * + * @param <E> the iterator element type. + */ + private static final class DistinctIterator<E> extends AbstractIterator<E> + { + /** + * The decorated iterator. + */ + private final Iterator<E> iterator; + + /** + * The previous element. + */ + private E previous; + + public DistinctIterator(Iterator<E> iterator) + { + this.iterator = iterator; + } + + protected E computeNext() + { + while(iterator.hasNext()) + { + E next = iterator.next(); + if (!next.equals(previous)) + { + previous = next; + return next; + } + } + return endOfData(); + } + } + + @Override + public String toString() + { + return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE); + } } From 194e632ab3781cd10f8bc10f150ada526b186d5a Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Wed, 30 Oct 2024 16:29:15 +0100 Subject: [PATCH 009/225] Ban the usage of "var" instead of full types in the production code patch by Stefan Miklosovic; reviewed by Brandon Williams for CASSANDRA-20038 --- .build/checkstyle.xml | 4 +++ CHANGES.txt | 1 + .../restrictions/StatementRestrictions.java | 28 +++++++++---------- .../cql3/statements/SelectStatement.java | 2 +- .../index/sai/VectorQueryContext.java | 3 +- .../disk/v1/segment/SegmentTrieBuffer.java | 2 +- .../segment/VectorIndexSegmentSearcher.java | 17 +++++------ .../index/sai/disk/v1/vector/BitsUtil.java | 4 +-- .../v1/vector/CompactionVectorValues.java | 4 +-- .../index/sai/disk/v1/vector/DiskAnn.java | 17 +++++------ .../sai/disk/v1/vector/OnDiskOrdinalsMap.java | 10 +++---- .../index/sai/disk/v1/vector/OnHeapGraph.java | 22 ++++++++------- .../v1/vector/RandomAccessReaderAdapter.java | 13 +++++---- .../disk/v1/vector/VectorPostingsWriter.java | 21 +++++++------- .../iterators/KeyRangeOrderingIterator.java | 2 +- .../index/sai/memory/VectorMemoryIndex.java | 25 +++++++++-------- .../cassandra/index/sai/plan/Operation.java | 6 ++-- .../index/sai/plan/QueryController.java | 8 +++--- .../index/sai/utils/AtomicRatio.java | 2 +- 19 files changed, 102 insertions(+), 89 deletions(-) diff --git a/.build/checkstyle.xml b/.build/checkstyle.xml index 8b81f21281d6..2ec5ecab1610 100644 --- a/.build/checkstyle.xml +++ b/.build/checkstyle.xml @@ -169,6 +169,10 @@ <property name="message" value="Use the CassandraRelevantProperties or CassandraRelevantEnv instead." /> </module> + <module name="IllegalType"> <!-- usage of var check --> + <property name="illegalClassNames" value="var"/> + </module> + <module name="RedundantImport"/> <module name="UnusedImports"/> diff --git a/CHANGES.txt b/CHANGES.txt index 3c96c434bcec..e9ca7b0f757a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.3 + * Ban the usage of "var" instead of full types in the production code (CASSANDRA-20038) * Suppress CVE-2024-45772 from lucene-core-9.7.0.jar (CASSANDRA-20024) Merged from 4.1: Merged from 4.0: diff --git a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java index d5b6a2a6fd1b..adc4e65c4823 100644 --- a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java +++ b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java @@ -293,11 +293,11 @@ else if (relation.isLIKE()) Joiner.on(", ").join(nonPrimaryKeyColumns)); } - var annRestriction = Streams.stream(nonPrimaryKeyRestrictions).filter(SingleRestriction::isANN).findFirst(); + Optional<SingleRestriction> annRestriction = Streams.stream(nonPrimaryKeyRestrictions).filter(SingleRestriction::isANN).findFirst(); if (annRestriction.isPresent()) { // If there is an ANN restriction then it must be for a vector<float, n> column, and it must have an index - var annColumn = annRestriction.get().getFirstColumn(); + ColumnMetadata annColumn = annRestriction.get().getFirstColumn(); if (!annColumn.type.isVector() || !(((VectorType<?>)annColumn.type).elementType instanceof FloatType)) throw invalidRequest(StatementRestrictions.ANN_ONLY_SUPPORTED_ON_VECTOR_MESSAGE); @@ -307,16 +307,16 @@ else if (relation.isLIKE()) if (partitionKeyRestrictions.needFiltering(table)) throw invalidRequest(StatementRestrictions.ANN_REQUIRES_INDEXED_FILTERING_MESSAGE); // We do not allow ANN query filtering using non-indexed columns - var nonAnnColumns = Streams.stream(nonPrimaryKeyRestrictions) - .filter(r -> !r.isANN()) - .map(Restriction::getFirstColumn) - .collect(Collectors.toList()); - var clusteringColumns = clusteringColumnsRestrictions.getColumnDefinitions(); + List<ColumnMetadata> nonAnnColumns = Streams.stream(nonPrimaryKeyRestrictions) + .filter(r -> !r.isANN()) + .map(Restriction::getFirstColumn) + .collect(Collectors.toList()); + Collection<ColumnMetadata> clusteringColumns = clusteringColumnsRestrictions.getColumnDefinitions(); if (!nonAnnColumns.isEmpty() || !clusteringColumns.isEmpty()) { - var nonIndexedColumns = Stream.concat(nonAnnColumns.stream(), clusteringColumns.stream()) - .filter(c -> indexRegistry.listIndexes().stream().noneMatch(i -> i.dependsOn(c))) - .collect(Collectors.toList()); + List<ColumnMetadata> nonIndexedColumns = Stream.concat(nonAnnColumns.stream(), clusteringColumns.stream()) + .filter(c -> indexRegistry.listIndexes().stream().noneMatch(i -> i.dependsOn(c))) + .collect(Collectors.toList()); if (!nonIndexedColumns.isEmpty()) { @@ -331,10 +331,10 @@ else if (relation.isLIKE()) else { // We do not support indexed vector restrictions that are not part of an ANN ordering - var vectorColumn = nonPrimaryKeyRestrictions.getColumnDefs() - .stream() - .filter(c -> c.type.isVector()) - .findFirst(); + Optional<ColumnMetadata> vectorColumn = nonPrimaryKeyRestrictions.getColumnDefs() + .stream() + .filter(c -> c.type.isVector()) + .findFirst(); if (vectorColumn.isPresent() && indexRegistry.listIndexes().stream().anyMatch(i -> i.dependsOn(vectorColumn.get()))) throw invalidRequest(StatementRestrictions.VECTOR_INDEXES_ANN_ONLY_MESSAGE); } diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java index 5178b84658b3..f72befd22be9 100644 --- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java @@ -1561,7 +1561,7 @@ private boolean isReversed(TableMetadata table, Map<ColumnMetadata, Ordering> or return false; Boolean[] reversedMap = new Boolean[table.clusteringColumns().size()]; int i = 0; - for (var entry : orderingColumns.entrySet()) + for (Map.Entry<ColumnMetadata, Ordering> entry : orderingColumns.entrySet()) { ColumnMetadata def = entry.getKey(); Ordering ordering = entry.getValue(); diff --git a/src/java/org/apache/cassandra/index/sai/VectorQueryContext.java b/src/java/org/apache/cassandra/index/sai/VectorQueryContext.java index f33686e13fd4..d65723b79bc5 100644 --- a/src/java/org/apache/cassandra/index/sai/VectorQueryContext.java +++ b/src/java/org/apache/cassandra/index/sai/VectorQueryContext.java @@ -19,6 +19,7 @@ package org.apache.cassandra.index.sai; import java.io.IOException; +import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.NavigableSet; @@ -181,7 +182,7 @@ public IgnoredKeysBits(OnHeapGraph<PrimaryKey> graph, NavigableSet<PrimaryKey> i @Override public boolean get(int ordinal) { - var keys = graph.keysFromOrdinal(ordinal); + Collection<PrimaryKey> keys = graph.keysFromOrdinal(ordinal); return keys.stream().anyMatch(k -> !ignored.contains(k)); } diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/segment/SegmentTrieBuffer.java b/src/java/org/apache/cassandra/index/sai/disk/v1/segment/SegmentTrieBuffer.java index 72c9add3132c..5852fb87b39d 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v1/segment/SegmentTrieBuffer.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/segment/SegmentTrieBuffer.java @@ -79,7 +79,7 @@ public long add(ByteComparable term, int termLength, int segmentRowId) public Iterator<IndexEntry> iterator() { - var iterator = trie.entrySet().iterator(); + Iterator<Map.Entry<ByteComparable, PackedLongValues.Builder>> iterator = trie.entrySet().iterator(); return new Iterator<>() { diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/segment/VectorIndexSegmentSearcher.java b/src/java/org/apache/cassandra/index/sai/disk/v1/segment/VectorIndexSegmentSearcher.java index dc6b8fa7065a..27310777a194 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v1/segment/VectorIndexSegmentSearcher.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/segment/VectorIndexSegmentSearcher.java @@ -38,6 +38,7 @@ import org.apache.cassandra.index.sai.VectorQueryContext; import org.apache.cassandra.index.sai.disk.PrimaryKeyMap; import org.apache.cassandra.index.sai.disk.v1.PerColumnIndexFiles; +import org.apache.cassandra.index.sai.disk.v1.postings.VectorPostingList; import org.apache.cassandra.index.sai.disk.v1.vector.DiskAnn; import org.apache.cassandra.index.sai.disk.v1.vector.OptimizeFor; import org.apache.cassandra.index.sai.iterators.KeyRangeIterator; @@ -102,7 +103,7 @@ public KeyRangeIterator search(Expression exp, AbstractBounds<PartitionPosition> return toPrimaryKeyIterator(bitsOrPostingList.postingList(), context); float[] queryVector = index.termType().decomposeVector(exp.lower().value.raw.duplicate()); - var vectorPostings = graph.search(queryVector, topK, limit, bitsOrPostingList.getBits()); + VectorPostingList vectorPostings = graph.search(queryVector, topK, limit, bitsOrPostingList.getBits()); if (bitsOrPostingList.expectedNodesVisited >= 0) updateExpectedNodes(vectorPostings.getVisitedCount(), bitsOrPostingList.expectedNodesVisited); return toPrimaryKeyIterator(vectorPostings, context); @@ -139,7 +140,7 @@ private BitsOrPostingList bitsOrPostingListForKeyRange(VectorQueryContext contex // If num of matches are not bigger than limit, skip ANN. // (nRows should not include shadowed rows, but context doesn't break those out by segment, // so we will live with the inaccuracy.) - var nRows = Math.toIntExact(maxSSTableRowId - minSSTableRowId + 1); + int nRows = Math.toIntExact(maxSSTableRowId - minSSTableRowId + 1); int maxBruteForceRows = min(globalBruteForceRows, maxBruteForceRows(limit, nRows, graph.size())); logger.trace("Search range covers {} rows; max brute force rows is {} for sstable index with {} nodes, LIMIT {}", nRows, maxBruteForceRows, graph.size(), limit); @@ -202,7 +203,7 @@ private long getMaxSSTableRowId(PrimaryKeyMap primaryKeyMap, PartitionPosition r private SparseFixedBitSet bitSetForSearch() { - var bits = cachedBitSets.get(); + SparseFixedBitSet bits = cachedBitSets.get(); bits.clear(); return bits; } @@ -226,9 +227,9 @@ public KeyRangeIterator limitToTopKResults(QueryContext context, List<PrimaryKey { // the iterator represents keys from the whole table -- we'll only pull of those that // are from our own token range, so we can use row ids to order the results by vector similarity. - var maxSegmentRowId = metadata.toSegmentRowId(metadata.maxSSTableRowId); + int maxSegmentRowId = metadata.toSegmentRowId(metadata.maxSSTableRowId); SparseFixedBitSet bits = bitSetForSearch(); - var rowIds = new IntArrayList(); + IntArrayList rowIds = new IntArrayList(); try (var ordinalsView = graph.getOrdinalsView()) { for (PrimaryKey primaryKey : keysInRange) @@ -258,7 +259,7 @@ public KeyRangeIterator limitToTopKResults(QueryContext context, List<PrimaryKey // else ask the index to perform a search limited to the bits we created float[] queryVector = index.termType().decomposeVector(expression.lower().value.raw.duplicate()); - var results = graph.search(queryVector, topK, limit, bits); + VectorPostingList results = graph.search(queryVector, topK, limit, bits); updateExpectedNodes(results.getVisitedCount(), expectedNodesVisited(topK, maxSegmentRowId, graph.size())); return toPrimaryKeyIterator(results, context); } @@ -267,7 +268,7 @@ public KeyRangeIterator limitToTopKResults(QueryContext context, List<PrimaryKey private boolean shouldUseBruteForce(int topK, int limit, int numRows) { // if we have a small number of results then let TopK processor do exact NN computation - var maxBruteForceRows = min(globalBruteForceRows, maxBruteForceRows(topK, numRows, graph.size())); + int maxBruteForceRows = min(globalBruteForceRows, maxBruteForceRows(topK, numRows, graph.size())); logger.trace("SAI materialized {} rows; max brute force rows is {} for sstable index with {} nodes, LIMIT {}", numRows, maxBruteForceRows, graph.size(), limit); Tracing.trace("SAI materialized {} rows; max brute force rows is {} for sstable index with {} nodes, LIMIT {}", @@ -287,7 +288,7 @@ private int maxBruteForceRows(int limit, int nPermittedOrdinals, int graphSize) private int expectedNodesVisited(int limit, int nPermittedOrdinals, int graphSize) { - var observedRatio = actualExpectedRatio.getUpdateCount() >= 10 ? actualExpectedRatio.get() : 1.0; + double observedRatio = actualExpectedRatio.getUpdateCount() >= 10 ? actualExpectedRatio.get() : 1.0; return (int) (observedRatio * VectorMemoryIndex.expectedNodesVisited(limit, nPermittedOrdinals, graphSize)); } diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/vector/BitsUtil.java b/src/java/org/apache/cassandra/index/sai/disk/v1/vector/BitsUtil.java index 40aff6b49ee3..603efeb21abf 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v1/vector/BitsUtil.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/vector/BitsUtil.java @@ -94,7 +94,7 @@ public NoDeletedPostings(NonBlockingHashMapLong<VectorPostings<T>> postings) @Override public boolean get(int i) { - var p = postings.get(i); + VectorPostings<T> p = postings.get(i); assert p != null : "No postings for ordinal " + i; return !p.isEmpty(); } @@ -114,7 +114,7 @@ public NoDeletedIntersectingPostings(Bits toAccept, NonBlockingHashMapLong<Vecto @Override public boolean get(int i) { - var p = postings.get(i); + VectorPostings<T> p = postings.get(i); assert p != null : "No postings for ordinal " + i; return !p.isEmpty() && toAccept.get(i); } diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/vector/CompactionVectorValues.java b/src/java/org/apache/cassandra/index/sai/disk/v1/vector/CompactionVectorValues.java index 8974752086dc..af23a666a91b 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v1/vector/CompactionVectorValues.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/vector/CompactionVectorValues.java @@ -79,8 +79,8 @@ public long write(SequentialWriter writer) throws IOException writer.writeInt(size()); writer.writeInt(dimension()); - for (var i = 0; i < size(); i++) { - var bb = values.get(i); + for (int i = 0; i < size(); i++) { + ByteBuffer bb = values.get(i); assert bb != null : "null vector at index " + i + " of " + size(); writer.write(bb); } diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/vector/DiskAnn.java b/src/java/org/apache/cassandra/index/sai/disk/v1/vector/DiskAnn.java index 93f50df54f57..3e476a7581d1 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v1/vector/DiskAnn.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/vector/DiskAnn.java @@ -27,6 +27,7 @@ import io.github.jbellis.jvector.disk.CachingGraphIndex; import io.github.jbellis.jvector.disk.OnDiskGraphIndex; +import io.github.jbellis.jvector.graph.GraphIndex; import io.github.jbellis.jvector.graph.GraphSearcher; import io.github.jbellis.jvector.graph.NeighborSimilarity; import io.github.jbellis.jvector.graph.SearchResult; @@ -64,7 +65,7 @@ public DiskAnn(SegmentMetadata.ComponentMetadataMap componentMetadatas, PerColum try (var pqFileHandle = indexFiles.compressedVectors(); var reader = new RandomAccessReaderAdapter(pqFileHandle)) { reader.seek(pqSegmentOffset); - var containsCompressedVectors = reader.readBoolean(); + boolean containsCompressedVectors = reader.readBoolean(); if (containsCompressedVectors) compressedVectors = CompressedVectors.load(reader, reader.getFilePointer()); else @@ -92,8 +93,8 @@ public VectorPostingList search(float[] queryVector, int topK, int limit, Bits a { OnHeapGraph.validateIndexable(queryVector, similarityFunction); - var view = graph.getView(); - var searcher = new GraphSearcher.Builder<>(view).build(); + GraphIndex.View<float[]> view = graph.getView(); + GraphSearcher<float[]> searcher = new GraphSearcher.Builder<>(view).build(); NeighborSimilarity.ScoreFunction scoreFunction; NeighborSimilarity.ReRanker<float[]> reRanker; if (compressedVectors == null) @@ -107,10 +108,10 @@ public VectorPostingList search(float[] queryVector, int topK, int limit, Bits a scoreFunction = compressedVectors.approximateScoreFunctionFor(queryVector, similarityFunction); reRanker = (i, map) -> similarityFunction.compare(queryVector, map.get(i)); } - var result = searcher.search(scoreFunction, - reRanker, - topK, - ordinalsMap.ignoringDeleted(acceptBits)); + SearchResult result = searcher.search(scoreFunction, + reRanker, + topK, + ordinalsMap.ignoringDeleted(acceptBits)); Tracing.trace("DiskANN search visited {} nodes to return {} results", result.getVisitedCount(), result.getNodes().length); return annRowIdsToPostings(result, limit); } @@ -134,7 +135,7 @@ public boolean hasNext() { try { - var ordinal = it.next().node; + int ordinal = it.next().node; segmentRowIdIterator = Arrays.stream(rowIdsView.getSegmentRowIdsMatching(ordinal)).iterator(); } catch (IOException e) diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/vector/OnDiskOrdinalsMap.java b/src/java/org/apache/cassandra/index/sai/disk/v1/vector/OnDiskOrdinalsMap.java index 121ab01ed024..673dcc3170eb 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v1/vector/OnDiskOrdinalsMap.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/vector/OnDiskOrdinalsMap.java @@ -48,7 +48,7 @@ public OnDiskOrdinalsMap(FileHandle fh, long segmentOffset, long segmentLength) { reader.seek(segmentOffset); int deletedCount = reader.readInt(); - for (var i = 0; i < deletedCount; i++) + for (int i = 0; i < deletedCount; i++) { deletedOrdinals.add(reader.readInt()); } @@ -93,7 +93,7 @@ public int[] getSegmentRowIdsMatching(int vectorOrdinal) throws IOException throw new RuntimeException(String.format("Error seeking to index offset for ordinal %d with ordToRowOffset %d", vectorOrdinal, ordToRowOffset), e); } - var offset = reader.readLong(); + long offset = reader.readLong(); // seek to and read rowIds try { @@ -104,9 +104,9 @@ public int[] getSegmentRowIdsMatching(int vectorOrdinal) throws IOException throw new RuntimeException(String.format("Error seeking to rowIds offset for ordinal %d with ordToRowOffset %d", vectorOrdinal, ordToRowOffset), e); } - var postingsSize = reader.readInt(); - var rowIds = new int[postingsSize]; - for (var i = 0; i < rowIds.length; i++) + int postingsSize = reader.readInt(); + int[] rowIds = new int[postingsSize]; + for (int i = 0; i < rowIds.length; i++) { rowIds[i] = reader.readInt(); } diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/vector/OnHeapGraph.java b/src/java/org/apache/cassandra/index/sai/disk/v1/vector/OnHeapGraph.java index 903058108369..f11db645fdcc 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v1/vector/OnHeapGraph.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/vector/OnHeapGraph.java @@ -25,6 +25,7 @@ import java.util.HashSet; import java.util.Map; import java.util.PriorityQueue; +import java.util.Set; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicInteger; @@ -41,6 +42,7 @@ import io.github.jbellis.jvector.graph.GraphSearcher; import io.github.jbellis.jvector.graph.NeighborSimilarity; import io.github.jbellis.jvector.graph.RandomAccessVectorValues; +import io.github.jbellis.jvector.graph.SearchResult; import io.github.jbellis.jvector.pq.CompressedVectors; import io.github.jbellis.jvector.pq.ProductQuantization; import io.github.jbellis.jvector.util.Bits; @@ -132,7 +134,7 @@ public long add(ByteBuffer term, T key, InvalidVectorBehavior behavior) { assert term != null && term.remaining() != 0; - var vector = vectorType.composeAsFloat(term); + float[] vector = vectorType.composeAsFloat(term); if (behavior == InvalidVectorBehavior.IGNORE) { try @@ -151,7 +153,7 @@ public long add(ByteBuffer term, T key, InvalidVectorBehavior behavior) validateIndexable(vector, similarityFunction); } - var bytesUsed = 0L; + long bytesUsed = 0L; VectorPostings<T> postings = postingsMap.get(vector); // if the vector is already in the graph, all that happens is that the postings list is updated // otherwise, we add the vector in this order: @@ -167,7 +169,7 @@ public long add(ByteBuffer term, T key, InvalidVectorBehavior behavior) if (postingsMap.putIfAbsent(vector, postings) == null) { // we won the race to add the new entry; assign it an ordinal and add to the other structures - var ordinal = nextOrdinal.getAndIncrement(); + int ordinal = nextOrdinal.getAndIncrement(); postings.setOrdinal(ordinal); bytesUsed += RamEstimation.concurrentHashMapRamUsed(1); // the new posting Map entry bytesUsed += (vectorValues instanceof ConcurrentVectorValues) @@ -242,8 +244,8 @@ public long remove(ByteBuffer term, T key) { assert term != null && term.remaining() != 0; - var vector = vectorType.composeAsFloat(term); - var postings = postingsMap.get(vector); + float[] vector = vectorType.composeAsFloat(term); + VectorPostings<T> postings = postingsMap.get(vector); if (postings == null) { // it's possible for this to be called against a different memtable than the one @@ -269,11 +271,11 @@ public PriorityQueue<T> search(float[] queryVector, int limit, Bits toAccept) Bits bits = hasDeletions ? BitsUtil.bitsIgnoringDeleted(toAccept, postingsByOrdinal) : toAccept; GraphIndex<float[]> graph = builder.getGraph(); - var searcher = new GraphSearcher.Builder<>(graph.getView()).withConcurrentUpdates().build(); + GraphSearcher<float[]> searcher = new GraphSearcher.Builder<>(graph.getView()).withConcurrentUpdates().build(); NeighborSimilarity.ExactScoreFunction scoreFunction = node2 -> vectorCompareFunction(queryVector, node2); - var result = searcher.search(scoreFunction, null, limit, bits); + SearchResult result = searcher.search(scoreFunction, null, limit, bits); Tracing.trace("ANN search visited {} in-memory nodes to return {} results", result.getVisitedCount(), result.getNodes().length); - var a = result.getNodes(); + SearchResult.NodeScore[] a = result.getNodes(); PriorityQueue<T> keyQueue = new PriorityQueue<>(); for (int i = 0; i < a.length; i++) keyQueue.addAll(keysFromOrdinal(a[i].node)); @@ -305,7 +307,7 @@ public SegmentMetadata.ComponentMetadataMap writeData(IndexDescriptor indexDescr long pqPosition = writePQ(pqOutput.asSequentialWriter()); long pqLength = pqPosition - pqOffset; - var deletedOrdinals = new HashSet<Integer>(); + Set<Integer> deletedOrdinals = new HashSet<>(); postingsMap.values().stream().filter(VectorPostings::isEmpty).forEach(vectorPostings -> deletedOrdinals.add(vectorPostings.getOrdinal())); // remove ordinals that don't have corresponding row ids due to partition/range deletion for (VectorPostings<T> vectorPostings : postingsMap.values()) @@ -371,7 +373,7 @@ private long writePQ(SequentialWriter writer) throws IOException .mapToObj(i -> pq.encode(vectorValues.vectorValue(i))) .toArray(byte[][]::new); } - var cv = new CompressedVectors(pq, encoded); + CompressedVectors cv = new CompressedVectors(pq, encoded); // save cv.write(writer); return writer.position(); diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/vector/RandomAccessReaderAdapter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/vector/RandomAccessReaderAdapter.java index b63d1a21fc6f..95ea81f3234e 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v1/vector/RandomAccessReaderAdapter.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/vector/RandomAccessReaderAdapter.java @@ -28,6 +28,7 @@ import io.github.jbellis.jvector.disk.ReaderSupplier; import org.apache.cassandra.io.util.FileHandle; import org.apache.cassandra.io.util.RandomAccessReader; +import org.apache.cassandra.io.util.Rebufferer.BufferHolder; public class RandomAccessReaderAdapter extends RandomAccessReader implements io.github.jbellis.jvector.disk.RandomAccessReader { @@ -44,7 +45,7 @@ static ReaderSupplier createSupplier(FileHandle fileHandle) @Override public void readFully(float[] dest) throws IOException { - var bh = bufferHolder; + BufferHolder bh = bufferHolder; long position = getPosition(); FloatBuffer floatBuffer; @@ -60,7 +61,7 @@ public void readFully(float[] dest) throws IOException { // offset is non-zero, and probably not aligned to Float.BYTES, so // set the position before converting to FloatBuffer. - var bb = bh.buffer(); + ByteBuffer bb = bh.buffer(); bb.position(Ints.checkedCast(position - bh.offset())); floatBuffer = bb.asFloatBuffer(); } @@ -68,7 +69,7 @@ public void readFully(float[] dest) throws IOException if (dest.length > floatBuffer.remaining()) { // slow path -- desired slice is across region boundaries - var bb = ByteBuffer.allocate(Float.BYTES * dest.length); + ByteBuffer bb = ByteBuffer.allocate(Float.BYTES * dest.length); readFully(bb); floatBuffer = bb.asFloatBuffer(); } @@ -92,7 +93,7 @@ public void read(int[] dest, int offset, int count) throws IOException if (count == 0) return; - var bh = bufferHolder; + BufferHolder bh = bufferHolder; long position = getPosition(); IntBuffer intBuffer; @@ -108,7 +109,7 @@ public void read(int[] dest, int offset, int count) throws IOException { // offset is non-zero, and probably not aligned to Integer.BYTES, so // set the position before converting to IntBuffer. - var bb = bh.buffer(); + ByteBuffer bb = bh.buffer(); bb.position(Ints.checkedCast(position - bh.offset())); intBuffer = bb.asIntBuffer(); } @@ -116,7 +117,7 @@ public void read(int[] dest, int offset, int count) throws IOException if (count > intBuffer.remaining()) { // slow path -- desired slice is across region boundaries - var bb = ByteBuffer.allocate(Integer.BYTES * count); + ByteBuffer bb = ByteBuffer.allocate(Integer.BYTES * count); readFully(bb); intBuffer = bb.asIntBuffer(); } diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/vector/VectorPostingsWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/vector/VectorPostingsWriter.java index fd92bf656dd7..b62575e6f3e9 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v1/vector/VectorPostingsWriter.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/vector/VectorPostingsWriter.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.Set; +import org.agrona.collections.IntArrayList; import org.apache.cassandra.io.util.SequentialWriter; import org.apache.cassandra.utils.Pair; @@ -45,7 +46,7 @@ public long writePostings(SequentialWriter writer, private void writeDeletedOrdinals(SequentialWriter writer, Set<Integer> deletedOrdinals) throws IOException { writer.writeInt(deletedOrdinals.size()); - for (var ordinal : deletedOrdinals) { + for (int ordinal : deletedOrdinals) { writer.writeInt(ordinal); } } @@ -60,21 +61,21 @@ public void writeNodeOrdinalToRowIdMapping(SequentialWriter writer, writer.writeInt(vectorValues.size()); // Write the offsets of the postings for each ordinal - var offsetsStartAt = ordToRowOffset + 4L + 8L * vectorValues.size(); - var nextOffset = offsetsStartAt; - for (var i = 0; i < vectorValues.size(); i++) { + long offsetsStartAt = ordToRowOffset + 4L + 8L * vectorValues.size(); + long nextOffset = offsetsStartAt; + for (int i = 0; i < vectorValues.size(); i++) { // (ordinal is implied; don't need to write it) writer.writeLong(nextOffset); - var rowIds = postingsMap.get(vectorValues.vectorValue(i)).getRowIds(); + IntArrayList rowIds = postingsMap.get(vectorValues.vectorValue(i)).getRowIds(); nextOffset += 4 + (rowIds.size() * 4L); // 4 bytes for size and 4 bytes for each integer in the list } assert writer.position() == offsetsStartAt : "writer.position()=" + writer.position() + " offsetsStartAt=" + offsetsStartAt; // Write postings lists - for (var i = 0; i < vectorValues.size(); i++) { + for (int i = 0; i < vectorValues.size(); i++) { VectorPostings<T> postings = postingsMap.get(vectorValues.vectorValue(i)); - var rowIds = postings.getRowIds(); + IntArrayList rowIds = postings.getRowIds(); writer.writeInt(rowIds.size()); for (int r = 0; r < rowIds.size(); r++) writer.writeInt(rowIds.getInt(r)); @@ -89,8 +90,8 @@ public void writeRowIdToNodeOrdinalMapping(SequentialWriter writer, List<Pair<Integer, Integer>> pairs = new ArrayList<>(); // Collect all (rowId, vectorOrdinal) pairs - for (var i = 0; i < vectorValues.size(); i++) { - var rowIds = postingsMap.get(vectorValues.vectorValue(i)).getRowIds(); + for (int i = 0; i < vectorValues.size(); i++) { + IntArrayList rowIds = postingsMap.get(vectorValues.vectorValue(i)).getRowIds(); for (int r = 0; r < rowIds.size(); r++) pairs.add(Pair.create(rowIds.getInt(r), i)); } @@ -100,7 +101,7 @@ public void writeRowIdToNodeOrdinalMapping(SequentialWriter writer, // Write the pairs to the file long startOffset = writer.position(); - for (var pair : pairs) { + for (Pair<Integer, Integer> pair : pairs) { writer.writeInt(pair.left); writer.writeInt(pair.right); } diff --git a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeOrderingIterator.java b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeOrderingIterator.java index c63c79fddb3c..6c6a8dd8140f 100644 --- a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeOrderingIterator.java +++ b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeOrderingIterator.java @@ -65,7 +65,7 @@ public PrimaryKey computeNext() } while (nextKeys.size() < chunkSize && input.hasNext()); // Get the next iterator before closing this one to prevent releasing the resource. - var previousIterator = nextIterator; + KeyRangeIterator previousIterator = nextIterator; // If this results in an exception, previousIterator is closed in close() method. nextIterator = nextRangeFunction.apply(nextKeys); if (previousIterator != null) diff --git a/src/java/org/apache/cassandra/index/sai/memory/VectorMemoryIndex.java b/src/java/org/apache/cassandra/index/sai/memory/VectorMemoryIndex.java index bea5cb877fa0..39690de089c2 100644 --- a/src/java/org/apache/cassandra/index/sai/memory/VectorMemoryIndex.java +++ b/src/java/org/apache/cassandra/index/sai/memory/VectorMemoryIndex.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.NavigableSet; @@ -80,8 +81,8 @@ public synchronized long add(DecoratedKey key, Clustering<?> clustering, ByteBuf if (value == null || value.remaining() == 0 || !index.validateTermSize(key, value, false, null)) return 0; - var primaryKey = index.hasClustering() ? index.keyFactory().create(key, clustering) - : index.keyFactory().create(key); + PrimaryKey primaryKey = index.hasClustering() ? index.keyFactory().create(key, clustering) + : index.keyFactory().create(key); return index(primaryKey, value); } @@ -116,8 +117,8 @@ public long update(DecoratedKey key, Clustering<?> clustering, ByteBuffer oldVal long bytesUsed = 0; if (different) { - var primaryKey = index.hasClustering() ? index.keyFactory().create(key, clustering) - : index.keyFactory().create(key); + PrimaryKey primaryKey = index.hasClustering() ? index.keyFactory().create(key, clustering) + : index.keyFactory().create(key); // update bounds because only rows with vectors are included in the key bounds, // so if the vector was null before, we won't have included it updateKeyBounds(primaryKey); @@ -154,7 +155,7 @@ public KeyRangeIterator search(QueryContext queryContext, Expression expr, Abstr VectorQueryContext vectorQueryContext = queryContext.vectorContext(); - var buffer = expr.lower().value.raw; + ByteBuffer buffer = expr.lower().value.raw; float[] qv = index.termType().decomposeVector(buffer); Bits bits; @@ -191,7 +192,7 @@ public KeyRangeIterator search(QueryContext queryContext, Expression expr, Abstr bits = queryContext.vectorContext().bitsetForShadowedPrimaryKeys(graph); } - var keyQueue = graph.search(qv, queryContext.vectorContext().limit(), bits); + PriorityQueue<PrimaryKey> keyQueue = graph.search(qv, queryContext.vectorContext().limit(), bits); if (keyQueue.isEmpty()) return KeyRangeIterator.empty(); return new ReorderingRangeIterator(keyQueue); @@ -221,8 +222,8 @@ public KeyRangeIterator limitToTopResults(List<PrimaryKey> primaryKeys, Expressi ByteBuffer buffer = expression.lower().value.raw; float[] qv = index.termType().decomposeVector(buffer); - var bits = new KeyFilteringBits(results); - var keyQueue = graph.search(qv, limit, bits); + KeyFilteringBits bits = new KeyFilteringBits(results); + PriorityQueue<PrimaryKey> keyQueue = graph.search(qv, limit, bits); if (keyQueue.isEmpty()) return KeyRangeIterator.empty(); return new ReorderingRangeIterator(keyQueue); @@ -247,8 +248,8 @@ public static int expectedNodesVisited(int limit, int nPermittedOrdinals, int gr { // constants are computed by Code Interpreter based on observed comparison counts in tests // https://chat.openai.com/share/2b1d7195-b4cf-4a45-8dce-1b9b2f893c75 - var sizeRestriction = min(nPermittedOrdinals, graphSize); - var raw = (int) (0.7 * pow(log(graphSize), 2) * + int sizeRestriction = min(nPermittedOrdinals, graphSize); + int raw = (int) (0.7 * pow(log(graphSize), 2) * pow(graphSize, 0.33) * pow(log(limit), 2) * pow(log((double) graphSize / sizeRestriction), 2) / pow(sizeRestriction, 0.13)); @@ -309,7 +310,7 @@ public boolean get(int ordinal) if (bits != null && !bits.get(ordinal)) return false; - var keys = graph.keysFromOrdinal(ordinal); + Collection<PrimaryKey> keys = graph.keysFromOrdinal(ordinal); return keys.stream().anyMatch(k -> keyRange.contains(k.partitionKey())); } @@ -362,7 +363,7 @@ public KeyFilteringBits(List<PrimaryKey> results) @Override public boolean get(int i) { - var pk = graph.keysFromOrdinal(i); + Collection<PrimaryKey> pk = graph.keysFromOrdinal(i); return results.stream().anyMatch(pk::contains); } diff --git a/src/java/org/apache/cassandra/index/sai/plan/Operation.java b/src/java/org/apache/cassandra/index/sai/plan/Operation.java index c7f313863ceb..5b1824184323 100644 --- a/src/java/org/apache/cassandra/index/sai/plan/Operation.java +++ b/src/java/org/apache/cassandra/index/sai/plan/Operation.java @@ -258,13 +258,13 @@ private static int getPriority(Operator op) */ static KeyRangeIterator buildIterator(QueryController controller) { - var orderings = controller.indexFilter().getExpressions() - .stream().filter(e -> e.operator() == Operator.ANN).collect(Collectors.toList()); + List<RowFilter.Expression> orderings = controller.indexFilter().getExpressions() + .stream().filter(e -> e.operator() == Operator.ANN).collect(Collectors.toList()); assert orderings.size() <= 1; if (controller.indexFilter().getExpressions().size() == 1 && orderings.size() == 1) // If we only have one expression, we just use the ANN index to order and limit. return controller.getTopKRows(orderings.get(0)); - var iterator = Node.buildTree(controller.indexFilter()).analyzeTree(controller).rangeIterator(controller); + KeyRangeIterator iterator = Node.buildTree(controller.indexFilter()).analyzeTree(controller).rangeIterator(controller); if (orderings.isEmpty()) return iterator; return controller.getTopKRows(iterator, orderings.get(0)); diff --git a/src/java/org/apache/cassandra/index/sai/plan/QueryController.java b/src/java/org/apache/cassandra/index/sai/plan/QueryController.java index 4f8efb824814..91adc5d17e4b 100644 --- a/src/java/org/apache/cassandra/index/sai/plan/QueryController.java +++ b/src/java/org/apache/cassandra/index/sai/plan/QueryController.java @@ -309,7 +309,7 @@ public KeyRangeIterator getTopKRows(RowFilter.Expression expression) assert expression.operator() == Operator.ANN; StorageAttachedIndex index = indexFor(expression); assert index != null; - var planExpression = Expression.create(index).add(Operator.ANN, expression.getIndexValue().duplicate()); + Expression planExpression = Expression.create(index).add(Operator.ANN, expression.getIndexValue().duplicate()); // search memtable before referencing sstable indexes; otherwise we may miss newly flushed memtable index KeyRangeIterator memtableResults = index.memtableIndexManager().searchMemtableIndexes(queryContext, planExpression, mergeRange); @@ -345,10 +345,10 @@ private KeyRangeIterator getTopKRows(List<PrimaryKey> rawSourceKeys, RowFilter.E // Filter out PKs now. Each PK is passed to every segment of the ANN index, so filtering shadowed keys // eagerly can save some work when going from PK to row id for on disk segments. // Since the result is shared with multiple streams, we use an unmodifiable list. - var sourceKeys = rawSourceKeys.stream().filter(vectorQueryContext::shouldInclude).collect(Collectors.toList()); + List<PrimaryKey> sourceKeys = rawSourceKeys.stream().filter(vectorQueryContext::shouldInclude).collect(Collectors.toList()); StorageAttachedIndex index = indexFor(expression); assert index != null : "Cannot do ANN ordering on an unindexed column"; - var planExpression = Expression.create(index); + Expression planExpression = Expression.create(index); planExpression.add(Operator.ANN, expression.getIndexValue().duplicate()); // search memtable before referencing sstable indexes; otherwise we may miss newly flushed memtable index @@ -388,7 +388,7 @@ private KeyRangeIterator getTopKRows(List<PrimaryKey> rawSourceKeys, RowFilter.E */ private KeyRangeIterator createRowIdIterator(Pair<Expression, Collection<SSTableIndex>> indexExpression) { - var subIterators = indexExpression.right + List<KeyRangeIterator> subIterators = indexExpression.right .stream() .map(index -> { diff --git a/src/java/org/apache/cassandra/index/sai/utils/AtomicRatio.java b/src/java/org/apache/cassandra/index/sai/utils/AtomicRatio.java index d7348ea91132..76cab3d302ae 100644 --- a/src/java/org/apache/cassandra/index/sai/utils/AtomicRatio.java +++ b/src/java/org/apache/cassandra/index/sai/utils/AtomicRatio.java @@ -54,7 +54,7 @@ public void update(long numerator, long denominator) public double get() { - var current = ratio.get(); + Ratio current = ratio.get(); return (double) current.numerator / current.denominator; } From a27e09930dea92574edd48a0c7a6098af4e7c081 Mon Sep 17 00:00:00 2001 From: Sam Tunnicliffe <samt@apache.org> Date: Tue, 1 Oct 2024 15:19:46 +0100 Subject: [PATCH 010/225] Nodetool cms reconfigure reports streaming failures correctly Patch by Sam Tunnicliffe; reviewed by Marcus Eriksson for CASSANDRA-19972 --- CHANGES.txt | 1 + .../tcm/sequences/ReconfigureCMS.java | 6 +- .../ReconfigureCMSStreamingFailureTest.java | 100 ++++++++++++++++++ 3 files changed, 105 insertions(+), 2 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/test/log/ReconfigureCMSStreamingFailureTest.java diff --git a/CHANGES.txt b/CHANGES.txt index d5ec59c1ec8d..94c5a9835fc6 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Nodetool reconfigure cms has correct return code when streaming fails (CASSANDRA-19972) * Reintroduce RestrictionSet#iterator() optimization around multi-column restrictions (CASSANDRA-20034) * Explicitly localize strings to Locale.US for internal implementation (CASSANDRA-19953) * Add -H option for human-friendly output in nodetool compactionhistory (CASSANDRA-20015) diff --git a/src/java/org/apache/cassandra/tcm/sequences/ReconfigureCMS.java b/src/java/org/apache/cassandra/tcm/sequences/ReconfigureCMS.java index fb6d9998c487..57b5e68e807e 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/ReconfigureCMS.java +++ b/src/java/org/apache/cassandra/tcm/sequences/ReconfigureCMS.java @@ -182,8 +182,10 @@ public SequenceState executeNext() } catch (Throwable t) { - logger.error("Could not finish adding the node to the Cluster Metadata Service", t); - return SequenceState.blocked(); + String message = "Some data streaming failed. Use nodetool to check CMS reconfiguration state and resume. " + + "For more, see `nodetool help cms reconfigure`."; + logger.warn(message); + return SequenceState.error(new RuntimeException(message)); } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/ReconfigureCMSStreamingFailureTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/ReconfigureCMSStreamingFailureTest.java new file mode 100644 index 000000000000..e0364736e675 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/log/ReconfigureCMSStreamingFailureTest.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.log; + +import java.io.IOException; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.junit.Test; + +import net.bytebuddy.ByteBuddy; +import net.bytebuddy.dynamic.loading.ClassLoadingStrategy; +import net.bytebuddy.implementation.MethodDelegation; +import net.bytebuddy.implementation.bind.annotation.SuperCall; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.test.TestBaseImpl; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.Replica; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.sequences.ReconfigureCMS; +import org.apache.cassandra.tcm.transformations.cms.PrepareCMSReconfiguration; + +import static net.bytebuddy.matcher.ElementMatchers.named; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + + +public class ReconfigureCMSStreamingFailureTest extends TestBaseImpl +{ + @Test + public void testNodetoolFailureWhenStreamingErrorOccurs() throws IOException, ExecutionException, InterruptedException + { + try (Cluster cluster = init(builder().withNodes(3) + .withConfig(c -> c.with(Feature.NETWORK)) + .withInstanceInitializer(BB::install) + .start())) + { + cluster.forEach(i -> i.runOnInstance(() -> BB.failStreaming.set(true))); + cluster.get(1) + .nodetoolResult("cms", "reconfigure", "3") + .asserts() + .failure() + .stderrContains("Some data streaming failed. Use nodetool to check CMS reconfiguration state and resume."); + String status = cluster.get(1).nodetoolResult("cms", "reconfigure", "--status").getStdout(); + assertTrue(status.contains("ACTIVE: [/127.0.0")); + assertTrue(status.contains("ADDITIONS: [/127.0.0")); + cluster.forEach(i -> assertTrue(i.callOnInstance(() -> PrepareCMSReconfiguration.needsReconfiguration(ClusterMetadata.current())))); + + cluster.forEach(i -> i.runOnInstance(() -> BB.failStreaming.set(false))); + cluster.get(1).nodetoolResult("cms", "reconfigure", "--resume").asserts().success(); + cluster.forEach(i -> assertFalse(i.callOnInstance(() -> PrepareCMSReconfiguration.needsReconfiguration(ClusterMetadata.current())))); + assertTrue(cluster.get(1).callOnInstance(() -> { + ClusterMetadata metadata = ClusterMetadata.current(); + return metadata.fullCMSMemberIds().containsAll(metadata.directory.peerIds()); + })); + } + } + + public static class BB + { + public static AtomicBoolean failStreaming = new AtomicBoolean(false); + public static void install(ClassLoader cl, int i) + { + new ByteBuddy().rebase(ReconfigureCMS.class) + .method(named("streamRanges")) + .intercept(MethodDelegation.to(BB.class)) + .make() + .load(cl, ClassLoadingStrategy.Default.INJECTION); + } + + public static void streamRanges(Replica replicaForStreaming, + Set<InetAddressAndPort> streamCandidates, + @SuperCall Callable<Void> zuper) throws Exception + { + if (failStreaming.get()) + throw new IOException("failed to connect to " + replicaForStreaming.endpoint() + " for streaming data"); + + zuper.call(); + } + } +} From 9ff31e806070873449ceb6e47aaa5e25118b466f Mon Sep 17 00:00:00 2001 From: Sam Tunnicliffe <samt@apache.org> Date: Wed, 18 Sep 2024 13:21:39 +0100 Subject: [PATCH 011/225] Equality check for Paxos.Electorate should not depend on collection types Patch by Sam Tunnicliffe; reviewed by Marcus Eriksson for CASSANDRA-19935 --- CHANGES.txt | 1 + .../apache/cassandra/service/paxos/Paxos.java | 5 +- .../service/paxos/PaxosElectorateTest.java | 55 +++++++++++++++++++ 3 files changed, 60 insertions(+), 1 deletion(-) create mode 100644 test/unit/org/apache/cassandra/service/paxos/PaxosElectorateTest.java diff --git a/CHANGES.txt b/CHANGES.txt index f27d82f87bea..6d6da13c10fa 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.1.8 + * Equality check for Paxos.Electorate should not depend on collection types (CASSANDRA-19935) * Fix race condition in DecayingEstimatedHistogramReservoir during rescale (CASSANDRA-19365) Merged from 4.0: * Add configurable batchlog endpoint strategies: random_remote, prefer_local, dynamic_remote, and dynamic (CASSANDRA-18120) diff --git a/src/java/org/apache/cassandra/service/paxos/Paxos.java b/src/java/org/apache/cassandra/service/paxos/Paxos.java index 36968d81604f..1e6f0c813c77 100644 --- a/src/java/org/apache/cassandra/service/paxos/Paxos.java +++ b/src/java/org/apache/cassandra/service/paxos/Paxos.java @@ -273,7 +273,10 @@ public boolean equals(Object o) if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; Electorate that = (Electorate) o; - return natural.equals(that.natural) && pending.equals(that.pending); + return natural.size() == that.natural.size() && + pending.size() == that.pending.size() && + natural.containsAll(that.natural) && + pending.containsAll(that.pending); } public int hashCode() diff --git a/test/unit/org/apache/cassandra/service/paxos/PaxosElectorateTest.java b/test/unit/org/apache/cassandra/service/paxos/PaxosElectorateTest.java new file mode 100644 index 000000000000..67756d963508 --- /dev/null +++ b/test/unit/org/apache/cassandra/service/paxos/PaxosElectorateTest.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.paxos; + +import java.net.UnknownHostException; +import java.util.HashSet; + +import org.junit.Test; + +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.locator.EndpointsForToken; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.Replica; +import org.apache.cassandra.service.paxos.Paxos.Electorate; + +import static org.junit.Assert.assertEquals; + +public class PaxosElectorateTest +{ + + @Test + public void compareElectoratesWithDifferentCollectionTypes() throws UnknownHostException + { + Token t = new Murmur3Partitioner.LongToken(0L); + EndpointsForToken natural = EndpointsForToken.of(t, replica(1),replica(2), replica(3)); + EndpointsForToken pending = EndpointsForToken.of(t, replica(4)); + Electorate first = new Electorate(natural.endpointList(), pending.endpointList()); + Electorate second = new Electorate(new HashSet<>(natural.endpoints()), new HashSet<>(pending.endpoints())); + assertEquals(first, second); + } + + private static Replica replica(int i) throws UnknownHostException + { + return Replica.fullReplica(InetAddressAndPort.getByName("127.0.0." + i), + Murmur3Partitioner.instance.getMinimumToken(), + Murmur3Partitioner.instance.getMinimumToken()); + } +} From 28759d8bbc16daf45ca07ee27b4a8dfedc8cf92b Mon Sep 17 00:00:00 2001 From: David Capwell <dcapwell@apache.org> Date: Tue, 13 Dec 2022 13:37:40 -0800 Subject: [PATCH 012/225] CEP-10: Simulator Java11 Support (backport to cassandra-4.1) patch by David Capwell; reviewed by Benedict Elliott Smith, Ekaterina Dimitrova for CASSANDRA-17178 backported to 4.1 in CASSANDRA-19935 --- CHANGES.txt | 1 + .../simulator/asm/ClassTransformer.java | 51 ++++++++++++++++++- .../simulator/asm/InterceptAgent.java | 6 ++- .../simulator/asm/InterceptClasses.java | 1 + .../apache/cassandra/simulator/asm/Utils.java | 8 +++ 5 files changed, 64 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 6d6da13c10fa..bcb9dec299d3 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.1.8 + * Backport Java 11 support for Simulator (CASSANDRA-17178/CASSANDRA-19935) * Equality check for Paxos.Electorate should not depend on collection types (CASSANDRA-19935) * Fix race condition in DecayingEstimatedHistogramReservoir during rescale (CASSANDRA-19365) Merged from 4.0: diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/ClassTransformer.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/ClassTransformer.java index 6e6b0d30de9e..778e44c80894 100644 --- a/test/simulator/asm/org/apache/cassandra/simulator/asm/ClassTransformer.java +++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/ClassTransformer.java @@ -110,6 +110,8 @@ public void visitLocalVariable(String name, String descriptor, String signature, private final EnumSet<Flag> flags; private final Consumer<String> dependentTypes; + private boolean updateVisibility = false; + ClassTransformer(int api, String className, EnumSet<Flag> flags, Consumer<String> dependentTypes) { this(api, new ClassWriter(0), className, flags, null, null, null, null, dependentTypes); @@ -137,12 +139,58 @@ private ClassTransformer(int api, ClassWriter classWriter, String className, Enu this.methodLogger = MethodLogger.log(api, className); } + public void setUpdateVisibility(boolean updateVisibility) + { + this.updateVisibility = updateVisibility; + } + + /** + * Java 11 changed the way that classes defined in the same source file get access to private state (see https://openjdk.org/jeps/181), + * rather than trying to adapt to this, this method attempts to make the field/method/class public so that access + * is not restricted. + */ + private int makePublic(int access) + { + if (!updateVisibility) + return access; + // leave non-user created methods/fields/etc. alone + if (contains(access, Opcodes.ACC_BRIDGE) || contains(access, Opcodes.ACC_SYNTHETIC)) + return access; + if (contains(access, Opcodes.ACC_PRIVATE)) + { + access &= ~Opcodes.ACC_PRIVATE; + access |= Opcodes.ACC_PUBLIC; + } + else if (contains(access, Opcodes.ACC_PROTECTED)) + { + access &= ~Opcodes.ACC_PROTECTED; + access |= Opcodes.ACC_PUBLIC; + } + else if (!contains(access, Opcodes.ACC_PUBLIC)) // package-protected + { + access |= Opcodes.ACC_PUBLIC; + } + return access; + } + + private static boolean contains(int value, int mask) + { + return (value & mask) != 0; + } + + @Override + public void visit(int version, int access, String name, String signature, String superName, String[] interfaces) + { + super.visit(version, makePublic(access), name, signature, superName, interfaces); + + } + @Override public FieldVisitor visitField(int access, String name, String descriptor, String signature, Object value) { if (dependentTypes != null) Utils.visitIfRefType(descriptor, dependentTypes); - return super.visitField(access, name, descriptor, signature, value); + return super.visitField(makePublic(access), name, descriptor, signature, value); } @Override @@ -176,6 +224,7 @@ public MethodVisitor visitMethod(int access, String name, String descriptor, Str isToString = true; } + access = makePublic(access); MethodVisitor visitor; if (flags.contains(MONITORS) && (access & Opcodes.ACC_SYNCHRONIZED) != 0) { diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/InterceptAgent.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/InterceptAgent.java index 87cfab0f6f73..ffa9b180e596 100644 --- a/test/simulator/asm/org/apache/cassandra/simulator/asm/InterceptAgent.java +++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/InterceptAgent.java @@ -296,12 +296,14 @@ public MethodVisitor visitMethod(int access, String name, String descriptor, Str visitor.visitMethodInsn(INVOKESTATIC, "java/lang/Thread", "currentThread", "()Ljava/lang/Thread;", false); visitor.visitFieldInsn(GETSTATIC, "java/util/concurrent/ThreadLocalRandom", "SEED", "J"); visitor.visitMethodInsn(INVOKESTATIC, "org/apache/cassandra/simulator/systems/InterceptorOfSystemMethods$Global", "randomSeed", "()J", false); - visitor.visitMethodInsn(INVOKEVIRTUAL, "sun/misc/Unsafe", "putLong", "(Ljava/lang/Object;JJ)V", false); + + String unsafeClass = Utils.descriptorToClassName(unsafeDescriptor); + visitor.visitMethodInsn(INVOKEVIRTUAL, unsafeClass, "putLong", "(Ljava/lang/Object;JJ)V", false); visitor.visitFieldInsn(GETSTATIC, "java/util/concurrent/ThreadLocalRandom", unsafeFieldName, unsafeDescriptor); visitor.visitMethodInsn(INVOKESTATIC, "java/lang/Thread", "currentThread", "()Ljava/lang/Thread;", false); visitor.visitFieldInsn(GETSTATIC, "java/util/concurrent/ThreadLocalRandom", "PROBE", "J"); visitor.visitLdcInsn(0); - visitor.visitMethodInsn(INVOKEVIRTUAL, "sun/misc/Unsafe", "putInt", "(Ljava/lang/Object;JI)V", false); + visitor.visitMethodInsn(INVOKEVIRTUAL, unsafeClass, "putInt", "(Ljava/lang/Object;JI)V", false); visitor.visitInsn(RETURN); visitor.visitLabel(new Label()); visitor.visitMaxs(6, 1); diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/InterceptClasses.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/InterceptClasses.java index a57074db27dc..473cc27032b8 100644 --- a/test/simulator/asm/org/apache/cassandra/simulator/asm/InterceptClasses.java +++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/InterceptClasses.java @@ -246,6 +246,7 @@ private byte[] transform(String internalName, String externalName, PeerGroup pee } ClassTransformer transformer = new ClassTransformer(api, internalName, flags, monitorDelayChance, new NemesisGenerator(api, internalName, nemesisChance), nemesisFieldSelector, hashcode, dependentTypes); + transformer.setUpdateVisibility(true); transformer.readAndTransform(input); if (!transformer.isTransformed()) diff --git a/test/simulator/asm/org/apache/cassandra/simulator/asm/Utils.java b/test/simulator/asm/org/apache/cassandra/simulator/asm/Utils.java index be2ef6c5d8e6..d710ffe417d4 100644 --- a/test/simulator/asm/org/apache/cassandra/simulator/asm/Utils.java +++ b/test/simulator/asm/org/apache/cassandra/simulator/asm/Utils.java @@ -287,4 +287,12 @@ public static void visitIfRefType(String descriptor, Consumer<String> forEach) forEach.accept(descriptor.substring(i + 1, descriptor.length() - 1)); } } + + public static String descriptorToClassName(String desc) + { + // samples: "Ljdk/internal/misc/Unsafe;", "Lsun/misc/Unsafe;" + if (!(desc.startsWith("L") && desc.endsWith(";"))) + throw new IllegalArgumentException("Unable to parse descriptor: " + desc); + return desc.substring(1, desc.length() - 1); + } } From ff54fd5ffd03846326862951e2a6de9b52de01f6 Mon Sep 17 00:00:00 2001 From: Bereng <berenguerblasi@gmail.com> Date: Mon, 14 Oct 2024 10:34:21 +0200 Subject: [PATCH 013/225] CASSANDRA-19986 Prepared statements cache computation improvements patch by Berenguer Blasi; reviewed by Maxwell Guo, Stefan Miklosovic for CASSANDRA-19986 --- .../apache/cassandra/cql3/QueryHandler.java | 3 ++ .../apache/cassandra/cql3/QueryProcessor.java | 30 +++++++++++++------ .../org/apache/cassandra/utils/MD5Digest.java | 6 ++++ .../cassandra/cql3/PstmtPersistenceTest.java | 4 ++- 4 files changed, 33 insertions(+), 10 deletions(-) diff --git a/src/java/org/apache/cassandra/cql3/QueryHandler.java b/src/java/org/apache/cassandra/cql3/QueryHandler.java index 8be7184d4484..e0480a6e4bb2 100644 --- a/src/java/org/apache/cassandra/cql3/QueryHandler.java +++ b/src/java/org/apache/cassandra/cql3/QueryHandler.java @@ -58,6 +58,9 @@ ResultMessage processBatch(BatchStatement statement, public static class Prepared { + // CASSANDRA-19986 Precomputed size might be available + public volatile int pstmntSize = -1; + public final CQLStatement statement; public final MD5Digest resultMetadataId; diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java index 66ce2ef5a62f..910f8c79f8e5 100644 --- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java +++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java @@ -93,7 +93,7 @@ public class QueryProcessor implements QueryHandler preparedStatements = Caffeine.newBuilder() .executor(MoreExecutors.directExecutor()) .maximumWeight(capacityToBytes(DatabaseDescriptor.getPreparedStatementsCacheSizeMB())) - .weigher(QueryProcessor::measure) + .weigher(QueryProcessor::getSizeOfPreparedStatementForCache) .removalListener((key, prepared, cause) -> { MD5Digest md5Digest = (MD5Digest) key; if (cause.wasEvicted()) @@ -411,10 +411,14 @@ public static Prepared parseAndPrepare(String query, ClientState clientState, bo CQLStatement statement = raw.prepare(clientState); statement.validate(clientState); + Prepared res; if (isInternal) - return new Prepared(statement, "", fullyQualified, keyspace); + res = new Prepared(statement, "", fullyQualified, keyspace); else - return new Prepared(statement, query, fullyQualified, keyspace); + res = new Prepared(statement, query, fullyQualified, keyspace); + res.pstmntSize = measurePstmnt(res); + + return res; } public static UntypedResultSet executeInternal(String query, Object... values) @@ -674,14 +678,14 @@ public static ResultMessage.Prepared storePreparedStatement(String queryString, { // Concatenate the current keyspace so we don't mix prepared statements between keyspace (#5352). // (if the keyspace is null, queryString has to have a fully-qualified keyspace so it's fine. - long statementSize = ObjectSizes.measureDeep(prepared.statement); + MD5Digest statementId = computeId(queryString, keyspace); // don't execute the statement if it's bigger than the allowed threshold - if (statementSize > capacityToBytes(DatabaseDescriptor.getPreparedStatementsCacheSizeMB())) + if (getSizeOfPreparedStatementForCache(statementId, prepared) > capacityToBytes(DatabaseDescriptor.getPreparedStatementsCacheSizeMB())) throw new InvalidRequestException(String.format("Prepared statement of size %d bytes is larger than allowed maximum of %d MB: %s...", - statementSize, + prepared.pstmntSize, DatabaseDescriptor.getPreparedStatementsCacheSizeMB(), queryString.substring(0, 200))); - MD5Digest statementId = computeId(queryString, keyspace); + Prepared previous = preparedStatements.get(statementId, (ignored_) -> prepared); if (previous == prepared) SystemKeyspace.writePreparedStatement(keyspace, statementId, queryString); @@ -797,9 +801,17 @@ public static CQLStatement.Raw parseStatement(String queryStr) throws SyntaxExce } } - private static int measure(Object key, Prepared value) + private static int measurePstmnt(Prepared value) { - return Ints.checkedCast(ObjectSizes.measureDeep(key) + ObjectSizes.measureDeep(value)); + return Ints.checkedCast(ObjectSizes.measureDeep(value)); + } + + private static int getSizeOfPreparedStatementForCache(MD5Digest key, Prepared value) + { + if (value.pstmntSize < 0) + throw new IllegalStateException("Precomputed prepared statement size not available"); + + return Ints.checkedCast(key.size() + value.pstmntSize); } /** diff --git a/src/java/org/apache/cassandra/utils/MD5Digest.java b/src/java/org/apache/cassandra/utils/MD5Digest.java index d5429918d52d..1f3d0a52d8c8 100644 --- a/src/java/org/apache/cassandra/utils/MD5Digest.java +++ b/src/java/org/apache/cassandra/utils/MD5Digest.java @@ -113,4 +113,10 @@ public static MessageDigest threadLocalMD5Digest() { return localMD5Digest.get(); } + + public int size() + { + return bytes.length + //bytes + 4; // int hashCode + } } diff --git a/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java b/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java index 8e3f7f4c3e65..c0d499426243 100644 --- a/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java +++ b/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java @@ -21,7 +21,9 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.concurrent.TimeUnit; +import org.apache.cassandra.Util; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -153,7 +155,7 @@ public void testPstmtInvalidation() throws Throwable prepareStatement("INSERT INTO %s (key, val) VALUES (?, ?) USING TIMESTAMP " + cnt2, clientState); // each new prepared statement should have caused an eviction - assertEquals("eviction count didn't increase by the expected number", numberOfEvictedStatements(), 10); + Util.spinAssertEquals("eviction count didn't increase by the expected number", 10L, this::numberOfEvictedStatements, 5, TimeUnit.SECONDS); assertEquals("Number of statements in table and in cache don't match", numberOfStatementsInMemory(), numberOfStatementsOnDisk()); return; From 73d8ee1a7114d96e879ea3e968e7dd03b830d221 Mon Sep 17 00:00:00 2001 From: Marcus Eriksson <marcuse@apache.org> Date: Tue, 1 Oct 2024 15:29:06 +0200 Subject: [PATCH 014/225] Bump the correct metric when doing progress barrier retries Patch by marcuse; reviewed by Sam Tunnicliffe for CASSANDRA-20016 --- .../cassandra/config/CassandraRelevantProperties.java | 10 ---------- .../cassandra/tcm/sequences/ProgressBarrier.java | 3 +-- .../cassandra/tcm/sequences/ProgressBarrierTest.java | 3 +++ 3 files changed, 4 insertions(+), 12 deletions(-) diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java index fcf9f7d0e688..9862f227822d 100644 --- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java +++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java @@ -529,16 +529,6 @@ public enum CassandraRelevantProperties // but it makes sense to group logically related properties together TCM_ALLOW_TRANSFORMATIONS_DURING_UPGRADES("cassandra.allow_transformations_during_upgrades", "false"), - /** - * for obtaining acknowlegement from peers to make progress in multi-step operations - */ - TCM_PROGRESS_BARRIER_BACKOFF_MILLIS("cassandra.progress_barrier_backoff_ms", "1000"), - TCM_PROGRESS_BARRIER_TIMEOUT_MILLIS("cassandra.progress_barrier_timeout_ms", "3600000"), - /** - * size of in-memory index of max epoch -> sealed period - */ - TCM_RECENTLY_SEALED_PERIOD_INDEX_SIZE("cassandra.recently_sealed_period_index_size", "10"), - /** * for testing purposes disable the automatic CMS reconfiguration after a bootstrap/replace/move operation */ diff --git a/src/java/org/apache/cassandra/tcm/sequences/ProgressBarrier.java b/src/java/org/apache/cassandra/tcm/sequences/ProgressBarrier.java index 28f94cacf855..af504d35d362 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/ProgressBarrier.java +++ b/src/java/org/apache/cassandra/tcm/sequences/ProgressBarrier.java @@ -198,7 +198,7 @@ public boolean await(ConsistencyLevel cl, ClusterMetadata metadata) Retry.Deadline deadline = Retry.Deadline.after(TimeUnit.MILLISECONDS.toNanos(TIMEOUT_MILLIS), new Retry.Backoff(DatabaseDescriptor.getCmsDefaultRetryMaxTries(), (int) BACKOFF_MILLIS, - TCMMetrics.instance.fetchLogRetries)); + TCMMetrics.instance.progressBarrierRetries)); while (!deadline.reachedMax()) { for (WatermarkRequest request : requests) @@ -523,7 +523,6 @@ private static class WatermarkRequest implements RequestCallbackWithFailure<Epoc public WatermarkRequest(InetAddressAndPort to, MessageDelivery messagingService, Epoch waitFor) { - this.to = to; this.messagingService = messagingService; this.waitFor = waitFor; diff --git a/test/unit/org/apache/cassandra/tcm/sequences/ProgressBarrierTest.java b/test/unit/org/apache/cassandra/tcm/sequences/ProgressBarrierTest.java index c5bf2c129b3f..91bd26d479c7 100644 --- a/test/unit/org/apache/cassandra/tcm/sequences/ProgressBarrierTest.java +++ b/test/unit/org/apache/cassandra/tcm/sequences/ProgressBarrierTest.java @@ -45,6 +45,7 @@ import org.apache.cassandra.harry.gen.rng.RngUtils; import org.apache.cassandra.harry.sut.TokenPlacementModel; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.metrics.TCMMetrics; import org.apache.cassandra.net.ConnectionType; import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessageDelivery; @@ -320,7 +321,9 @@ public <V> void respond(V response, Message<?> message) {} .advance(metadata.epoch) .barrier() .withMessagingService(delivery); + long before = TCMMetrics.instance.progressBarrierRetries.getCount(); progressBarrier.await(); + Assert.assertTrue(TCMMetrics.instance.progressBarrierRetries.getCount() - before > 0); Assert.assertTrue(responded.size() == 1); } } From c679b4730332ef67102ec7e47db891be2f8feabf Mon Sep 17 00:00:00 2001 From: Marcus Eriksson <marcuse@apache.org> Date: Tue, 22 Oct 2024 11:27:44 +0200 Subject: [PATCH 015/225] =?UTF-8?q?Don=E2=80=99t=20finish=20ongoing=20deco?= =?UTF-8?q?mmission=20and=20move=20operations=20during=20startup?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Patch by marcuse; reviewed by Sam Tunnicliffe for CASSANDRA-20040 --- CHANGES.txt | 1 + .../cassandra/service/StorageService.java | 38 +++- .../service/StorageServiceMBean.java | 2 + .../cassandra/tcm/MultiStepOperation.java | 5 + .../org/apache/cassandra/tcm/Startup.java | 13 +- .../tcm/sequences/InProgressSequences.java | 7 + .../apache/cassandra/tcm/sequences/Move.java | 14 +- .../tcm/sequences/SingleNodeSequences.java | 57 +++++- .../tcm/sequences/UnbootstrapAndLeave.java | 6 + .../org/apache/cassandra/tools/NodeProbe.java | 10 ++ .../apache/cassandra/tools/nodetool/Move.java | 29 ++- .../cassandra/distributed/impl/Instance.java | 1 + .../distributed/test/DecommissionTest.java | 165 +++++++++--------- .../distributed/test/FailingMoveTest.java | 140 +++++++++++++++ .../test/jmx/JMXGetterCheckTest.java | 2 + 15 files changed, 391 insertions(+), 99 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/test/FailingMoveTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 94c5a9835fc6..f3b9468f622d 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Don’t finish ongoing decommission and move operations during startup (CASSANDRA-20040) * Nodetool reconfigure cms has correct return code when streaming fails (CASSANDRA-19972) * Reintroduce RestrictionSet#iterator() optimization around multi-column restrictions (CASSANDRA-20034) * Explicitly localize strings to Locale.US for internal implementation (CASSANDRA-19953) diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 3b502f5a4807..ab8ddfe9bbb1 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -253,6 +253,8 @@ import static org.apache.cassandra.service.StorageService.Mode.DECOMMISSIONED; import static org.apache.cassandra.service.StorageService.Mode.DECOMMISSION_FAILED; import static org.apache.cassandra.service.StorageService.Mode.JOINING_FAILED; +import static org.apache.cassandra.service.StorageService.Mode.LEAVING; +import static org.apache.cassandra.service.StorageService.Mode.MOVE_FAILED; import static org.apache.cassandra.service.StorageService.Mode.NORMAL; import static org.apache.cassandra.tcm.membership.NodeState.BOOTSTRAPPING; import static org.apache.cassandra.tcm.membership.NodeState.BOOT_REPLACING; @@ -454,8 +456,7 @@ public static List<Range<Token>> getAllRanges(List<Token> sortedTokens) /* the probability for tracing any particular request, 0 disables tracing and 1 enables for all */ private double traceProbability = 0.0; - public enum Mode { STARTING, NORMAL, JOINING, JOINING_FAILED, LEAVING, DECOMMISSIONED, DECOMMISSION_FAILED, MOVING, DRAINING, DRAINED } - private volatile Mode operationMode = Mode.STARTING; + public enum Mode { STARTING, NORMAL, JOINING, JOINING_FAILED, LEAVING, DECOMMISSIONED, DECOMMISSION_FAILED, MOVING, MOVE_FAILED, DRAINING, DRAINED } /* Can currently hold DECOMMISSIONED, DECOMMISSION_FAILED, DRAINING, DRAINED for legacy compatibility. */ private volatile Optional<Mode> transientMode = Optional.empty(); @@ -763,7 +764,16 @@ public void runMayThrow() throws InterruptedException, ExecutionException, IOExc }); if (SystemKeyspace.wasDecommissioned()) - throw new ConfigurationException("This node was decommissioned and will not rejoin the ring unless cassandra.override_decommission=true has been set, or all existing data is removed and the node is bootstrapped again"); + { + if (CassandraRelevantProperties.OVERRIDE_DECOMMISSION.getBoolean()) + { + logger.warn("This node was decommissioned, but overriding by operator request."); + } + else + { + throw new ConfigurationException("This node was decommissioned and will not rejoin the ring unless cassandra.override_decommission=true has been set, or all existing data is removed and the node is bootstrapped again"); + } + } if (DatabaseDescriptor.getReplaceTokens().size() > 0 || DatabaseDescriptor.getReplaceNode() != null) throw new RuntimeException("Replace method removed; use cassandra.replace_address instead"); @@ -3669,6 +3679,18 @@ public void move(String newToken) SingleNodeSequences.move(getTokenFactory().fromString(newToken)); } + @Override + public void resumeMove() + { + SingleNodeSequences.resumeMove(); + } + + @Override + public void abortMove() + { + SingleNodeSequences.abortMove(); + } + public String getRemovalStatus() { return getRemovalStatus(false); @@ -3776,6 +3798,12 @@ public void markBootstrapFailed() transientMode = Optional.of(JOINING_FAILED); } + public void markMoveFailed() + { + logger.info(MOVE_FAILED.toString()); + transientMode = Optional.of(MOVE_FAILED); + } + /* - Use system_views.local to get information about the node (todo: we might still need a jmx endpoint for that since you can't run cql queries on drained etc nodes) */ @@ -3845,7 +3873,7 @@ public boolean isNormal() public boolean isDecommissioned() { - return operationMode == DECOMMISSIONED; + return operationMode() == DECOMMISSIONED; } public boolean isDecommissionFailed() @@ -3855,7 +3883,7 @@ public boolean isDecommissionFailed() public boolean isDecommissioning() { - return operationMode == Mode.LEAVING || operationMode == DECOMMISSION_FAILED; + return operationMode() == LEAVING; } public boolean isBootstrapFailed() diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java index 8beeb32ba541..f23b3b454382 100644 --- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java +++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java @@ -530,6 +530,8 @@ default int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, * This node will unload its data onto its neighbors, and bootstrap to the new token. */ public void move(String newToken) throws IOException; + public void resumeMove(); + public void abortMove(); /** * removeToken removes token (and all data associated with diff --git a/src/java/org/apache/cassandra/tcm/MultiStepOperation.java b/src/java/org/apache/cassandra/tcm/MultiStepOperation.java index 5fffbd2c8412..019086dccd69 100644 --- a/src/java/org/apache/cassandra/tcm/MultiStepOperation.java +++ b/src/java/org/apache/cassandra/tcm/MultiStepOperation.java @@ -98,6 +98,11 @@ protected MultiStepOperation(int currentStep, Epoch latestModification) this.latestModification = latestModification; } + public boolean finishDuringStartup() + { + return true; + } + /** * Unique identifier for the type of operation, e.g. JOIN, LEAVE, MOVE * @return the specific kind of this operation diff --git a/src/java/org/apache/cassandra/tcm/Startup.java b/src/java/org/apache/cassandra/tcm/Startup.java index 459566ff803f..2b60c1561d13 100644 --- a/src/java/org/apache/cassandra/tcm/Startup.java +++ b/src/java/org/apache/cassandra/tcm/Startup.java @@ -396,7 +396,7 @@ public static void startup(Supplier<Transformation> initialTransformation, boole NodeId self = metadata.myNodeId(); // finish in-progress sequences first - InProgressSequences.finishInProgressSequences(self); + InProgressSequences.finishInProgressSequences(self, true); metadata = ClusterMetadata.current(); switch (metadata.directory.peerState(self)) @@ -407,8 +407,7 @@ public static void startup(Supplier<Transformation> initialTransformation, boole ReconfigureCMS.maybeReconfigureCMS(metadata, DatabaseDescriptor.getReplaceAddress()); ClusterMetadataService.instance().commit(initialTransformation.get()); - - InProgressSequences.finishInProgressSequences(self); + InProgressSequences.finishInProgressSequences(self, true); // potentially finish the MSO committed above metadata = ClusterMetadata.current(); if (metadata.directory.peerState(self) == JOINED) @@ -437,6 +436,14 @@ public static void startup(Supplier<Transformation> initialTransformation, boole "Can't proceed from the state " + metadata.directory.peerState(self)); } break; + case LEAVING: + logger.info("Node is currently being decommissioned, resume with `nodetool decommission`"); + StorageService.instance.markDecommissionFailed(); + break; + case MOVING: + logger.info("Node is currently moving, resume with nodetool move --resume or abort with nodetool move --abort"); + StorageService.instance.markMoveFailed(); + break; default: throw new IllegalStateException("Can't proceed from the state " + metadata.directory.peerState(self)); } diff --git a/src/java/org/apache/cassandra/tcm/sequences/InProgressSequences.java b/src/java/org/apache/cassandra/tcm/sequences/InProgressSequences.java index 86a8ec019ade..735a7f693571 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/InProgressSequences.java +++ b/src/java/org/apache/cassandra/tcm/sequences/InProgressSequences.java @@ -60,6 +60,11 @@ private InProgressSequences(Epoch lastModified, ImmutableMap<MultiStepOperation. } public static void finishInProgressSequences(MultiStepOperation.SequenceKey sequenceKey) + { + finishInProgressSequences(sequenceKey, false); + } + + public static void finishInProgressSequences(MultiStepOperation.SequenceKey sequenceKey, boolean onlyStartupSafeSequences) { ClusterMetadata metadata = ClusterMetadata.current(); while (true) @@ -67,6 +72,8 @@ public static void finishInProgressSequences(MultiStepOperation.SequenceKey sequ MultiStepOperation<?> sequence = metadata.inProgressSequences.get(sequenceKey); if (sequence == null) break; + if (onlyStartupSafeSequences && !sequence.finishDuringStartup()) + break; if (isLeave(sequence)) StorageService.instance.maybeInitializeServices(); if (resume(sequence)) diff --git a/src/java/org/apache/cassandra/tcm/sequences/Move.java b/src/java/org/apache/cassandra/tcm/sequences/Move.java index 7375aedc7899..09811fba80f7 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/Move.java +++ b/src/java/org/apache/cassandra/tcm/sequences/Move.java @@ -151,6 +151,12 @@ private Move(Move current, Epoch latestModification) this.streamData = current.streamData; } + @Override + public boolean finishDuringStartup() + { + return false; + } + @Override public Kind kind() { @@ -199,7 +205,7 @@ public SequenceState executeNext() catch (Throwable t) { JVMStabilityInspector.inspectThrowable(t); - return continuable() ; + return continuable(); } break; case MID_MOVE: @@ -251,8 +257,14 @@ else if (destination.isSelf()) } catch (ExecutionException e) { + StorageService.instance.markMoveFailed(); throw new RuntimeException("Unable to move", e); } + catch (Exception e) + { + StorageService.instance.markMoveFailed(); + throw e; + } try { diff --git a/src/java/org/apache/cassandra/tcm/sequences/SingleNodeSequences.java b/src/java/org/apache/cassandra/tcm/sequences/SingleNodeSequences.java index 69c071120f3b..7813fb14920b 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/SingleNodeSequences.java +++ b/src/java/org/apache/cassandra/tcm/sequences/SingleNodeSequences.java @@ -33,6 +33,7 @@ import org.apache.cassandra.tcm.MultiStepOperation; import org.apache.cassandra.tcm.membership.NodeId; import org.apache.cassandra.tcm.membership.NodeState; +import org.apache.cassandra.tcm.transformations.CancelInProgressSequence; import org.apache.cassandra.tcm.transformations.PrepareLeave; import org.apache.cassandra.tcm.transformations.PrepareMove; @@ -75,13 +76,17 @@ static void decommission(boolean shutdownNetworking, boolean force) if (inProgress == null) { - logger.info("starting decom with {} {}", metadata.epoch, self); + logger.info("starting decommission with {} {}", metadata.epoch, self); ClusterMetadataService.instance().commit(new PrepareLeave(self, force, ClusterMetadataService.instance().placementProvider(), LeaveStreams.Kind.UNBOOTSTRAP)); } - else if (!InProgressSequences.isLeave(inProgress)) + else if (InProgressSequences.isLeave(inProgress)) + { + logger.info("Resuming decommission @ {} (current epoch = {}): {}", inProgress.latestModification, metadata.epoch, inProgress.status()); + } + else { throw new IllegalArgumentException("Can not decommission a node that has an in-progress sequence"); } @@ -165,4 +170,52 @@ static void move(Token newToken) logger.debug("Successfully moved to new token {}", StorageService.instance.getLocalTokens().iterator().next()); } + static void resumeMove() + { + if (ClusterMetadataService.instance().isMigrating() || ClusterMetadataService.state() == ClusterMetadataService.State.GOSSIP) + throw new IllegalStateException("This cluster is migrating to cluster metadata, can't move until that is done."); + + ClusterMetadata metadata = ClusterMetadata.current(); + NodeId self = metadata.myNodeId(); + MultiStepOperation<?> sequence = metadata.inProgressSequences.get(self); + if (sequence == null || sequence.kind() != MultiStepOperation.Kind.MOVE) + { + String msg = "No move operation in progress, can't resume"; + logger.info(msg); + throw new IllegalStateException(msg); + } + if (StorageService.instance.operationMode() != StorageService.Mode.MOVE_FAILED) + { + String msg = "Can't resume a move operation unless it has failed"; + logger.info(msg); + throw new IllegalStateException(msg); + } + StorageService.instance.clearTransientMode(); + InProgressSequences.finishInProgressSequences(self); + } + + static void abortMove() + { + if (ClusterMetadataService.instance().isMigrating() || ClusterMetadataService.state() == ClusterMetadataService.State.GOSSIP) + throw new IllegalStateException("This cluster is migrating to cluster metadata, can't move until that is done."); + + ClusterMetadata metadata = ClusterMetadata.current(); + NodeId self = metadata.myNodeId(); + MultiStepOperation<?> sequence = metadata.inProgressSequences.get(self); + if (sequence == null || sequence.kind() != MultiStepOperation.Kind.MOVE) + { + String msg = "No move operation in progress, can't abort"; + logger.info(msg); + throw new IllegalStateException(msg); + } + if (StorageService.instance.operationMode() != StorageService.Mode.MOVE_FAILED) + { + String msg = "Can't abort a move operation unless it has failed"; + logger.info(msg); + throw new IllegalStateException(msg); + } + StorageService.instance.clearTransientMode(); + ClusterMetadataService.instance().commit(new CancelInProgressSequence(self)); + } + } diff --git a/src/java/org/apache/cassandra/tcm/sequences/UnbootstrapAndLeave.java b/src/java/org/apache/cassandra/tcm/sequences/UnbootstrapAndLeave.java index bd93d7b84cf5..9e1d1ea62c95 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/UnbootstrapAndLeave.java +++ b/src/java/org/apache/cassandra/tcm/sequences/UnbootstrapAndLeave.java @@ -292,6 +292,12 @@ private static Transformation.Kind indexToNext(int index) } } + @Override + public boolean finishDuringStartup() + { + return false; + } + @Override public String toString() { diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index 72ed9d3aec00..5c75985bda2e 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -1010,6 +1010,16 @@ public void move(String newToken) throws IOException ssProxy.move(newToken); } + public void resumeMove() + { + ssProxy.resumeMove(); + } + + public void abortMove() + { + ssProxy.abortMove(); + } + public void removeNode(String token) { removeNode(token, false); diff --git a/src/java/org/apache/cassandra/tools/nodetool/Move.java b/src/java/org/apache/cassandra/tools/nodetool/Move.java index 075e00850391..87c085b86bc8 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/Move.java +++ b/src/java/org/apache/cassandra/tools/nodetool/Move.java @@ -23,21 +23,46 @@ import java.io.IOException; +import io.airlift.airline.Option; import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.tools.NodeTool.NodeToolCmd; @Command(name = "move", description = "Move node on the token ring to a new token") public class Move extends NodeToolCmd { - @Arguments(usage = "<new token>", description = "The new token.", required = true) + @Arguments(usage = "<new token>", description = "The new token.") private String newToken = EMPTY; + @Option(title = "Resume an ongoing move operation", name = "--resume") + private boolean resume; + + @Option(title = "Abort an ongoing move operation", name = "--abort") + private boolean abort; + @Override public void execute(NodeProbe probe) { try { - probe.move(newToken); + if (!newToken.isEmpty()) + { + if (resume || abort) + throw new IllegalArgumentException("Can't give both a token and --resume/--abort"); + + probe.move(newToken); + } + else + { + if (abort && resume) + throw new IllegalArgumentException("Can't both resume and abort"); + + if (resume) + probe.resumeMove(); + else if (abort) + probe.abortMove(); + else + throw new IllegalArgumentException("Need to give either a token for a new move operation, or --resume/--abort for an existing one"); + } } catch (IOException e) { throw new RuntimeException("Error during moving node", e); diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java index 296f95b9b421..fe4a25be4309 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java @@ -636,6 +636,7 @@ public void startup(ICluster cluster) } catch (Throwable t) { + startedAt.set(0); if (t instanceof RuntimeException) throw (RuntimeException) t; throw new RuntimeException(t); diff --git a/test/distributed/org/apache/cassandra/distributed/test/DecommissionTest.java b/test/distributed/org/apache/cassandra/distributed/test/DecommissionTest.java index e2360aca2b6e..ddee1fef63cc 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/DecommissionTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/DecommissionTest.java @@ -18,6 +18,7 @@ package org.apache.cassandra.distributed.test; +import java.io.IOException; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; @@ -27,9 +28,13 @@ import net.bytebuddy.dynamic.loading.ClassLoadingStrategy; import net.bytebuddy.implementation.MethodDelegation; import net.bytebuddy.implementation.bind.annotation.SuperCall; +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.db.SystemKeyspace; import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.action.GossipHelper; import org.apache.cassandra.distributed.api.IInvokableInstance; import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.membership.NodeId; import org.apache.cassandra.tcm.ownership.PlacementDeltas; import org.apache.cassandra.tcm.sequences.UnbootstrapStreams; @@ -41,7 +46,6 @@ import static org.apache.cassandra.distributed.api.Feature.NETWORK; import static org.apache.cassandra.distributed.shared.ClusterUtils.stopUnchecked; import static org.apache.cassandra.service.StorageService.Mode.DECOMMISSION_FAILED; -import static org.apache.cassandra.service.StorageService.Mode.NORMAL; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -59,62 +63,31 @@ public void testDecommission() throws Throwable .start())) { IInvokableInstance instance = cluster.get(2); - + assertBootstrapState(instance, COMPLETED); + instance.nodetoolResult("decommission", "--force") + .asserts() + .failure() + .stderrContains("simulated error in prepareUnbootstrapStreaming"); + instance.runOnInstance(() -> { + assertFalse(StorageService.instance.isDecommissioning()); + assertTrue(StorageService.instance.isDecommissionFailed()); + }); + + // still COMPLETED, nothing has changed + assertBootstrapState(instance, COMPLETED); + assertOperationMode(instance, DECOMMISSION_FAILED); + instance.nodetoolResult("decommission", "--force").asserts().success(); + instance.runOnInstance(() -> { + assertFalse(StorageService.instance.isDecommissionFailed()); + assertFalse(StorageService.instance.isDecommissioning()); + }); + assertBootstrapState(instance, DECOMMISSIONED); + instance.nodetoolResult("decommission", "--force") + .asserts() + .success() + .stdoutContains("Node was already decommissioned"); + assertBootstrapState(instance, DECOMMISSIONED); instance.runOnInstance(() -> { - - assertEquals(COMPLETED.name(), StorageService.instance.getBootstrapState()); - - // pretend that decommissioning has failed in the middle - - try - { - StorageService.instance.decommission(true); - fail("the first attempt to decommission should fail"); - } - catch (Throwable t) - { - assertTrue(t.getMessage().contains("simulated error in prepareUnbootstrapStreaming")); - } - - assertFalse(StorageService.instance.isDecommissioning()); - assertTrue(StorageService.instance.isDecommissionFailed()); - - // still COMPLETED, nothing has changed - assertEquals(COMPLETED.name(), StorageService.instance.getBootstrapState()); - - String operationMode = StorageService.instance.getOperationMode(); - assertEquals(DECOMMISSION_FAILED.name(), operationMode); - - // try to decommission again, now successfully - - try - { - StorageService.instance.decommission(true); - - // decommission was successful, so we reset failed decommission mode - assertFalse(StorageService.instance.isDecommissionFailed()); - - assertEquals(DECOMMISSIONED.name(), StorageService.instance.getBootstrapState()); - assertFalse(StorageService.instance.isDecommissioning()); - } - catch (Throwable t) - { - fail("the second decommission attempt should pass but it failed on: " + t.getMessage()); - } - - assertEquals(DECOMMISSIONED.name(), StorageService.instance.getBootstrapState()); - assertFalse(StorageService.instance.isDecommissionFailed()); - - try - { - StorageService.instance.decommission(true); - fail("Should have failed since the node is in decomissioned state"); - } - catch (UnsupportedOperationException e) - { - // ignore - } - assertEquals(DECOMMISSIONED.name(), StorageService.instance.getBootstrapState()); assertFalse(StorageService.instance.isDecommissionFailed()); assertFalse(StorageService.instance.isDecommissioning()); }); @@ -137,46 +110,27 @@ public void testDecommissionAfterNodeRestart() throws Throwable .start())) { IInvokableInstance instance = cluster.get(2); - - instance.runOnInstance(() -> { - assertEquals(COMPLETED.name(), StorageService.instance.getBootstrapState()); - - // pretend that decommissioning has failed in the middle - - try - { - StorageService.instance.decommission(true); - fail("the first attempt to decommission should fail"); - } - catch (Throwable t) - { - assertTrue(t.getMessage().contains("simulated error in prepareUnbootstrapStreaming")); - } - - // node is in DECOMMISSION_FAILED mode - String operationMode = StorageService.instance.getOperationMode(); - assertEquals(DECOMMISSION_FAILED.name(), operationMode); - }); - + assertBootstrapState(instance, COMPLETED); + // pretend that decommissioning has failed in the middle + instance.nodetoolResult("decommission", "--force") + .asserts() + .failure() + .stderrContains("simulated error in prepareUnbootstrapStreaming"); + assertOperationMode(instance, DECOMMISSION_FAILED); // restart the node which we failed to decommission stopUnchecked(instance); instance.startup(); - - // it is back to normal so let's decommission again - - String oprationMode = instance.callOnInstance(() -> StorageService.instance.getOperationMode()); - assertEquals(NORMAL.name(), oprationMode); - + // it starts up as DECOMMISSION_FAILED so let's decommission again + assertOperationMode(instance, DECOMMISSION_FAILED); + instance.nodetoolResult("decommission", "--force").asserts().success(); + assertBootstrapState(instance, DECOMMISSIONED); instance.runOnInstance(() -> { - StorageService.instance.decommission(true); - assertEquals(DECOMMISSIONED.name(), StorageService.instance.getBootstrapState()); assertFalse(StorageService.instance.isDecommissionFailed()); assertFalse(StorageService.instance.isDecommissioning()); }); } } - public static class BB { public static void install(ClassLoader classLoader, Integer num) @@ -208,4 +162,43 @@ public static void execute(NodeId leaving, PlacementDeltas startLeave, Placement } } } + + @Test + public void testRestartDecommedNode() throws IOException, ExecutionException, InterruptedException + { + try (Cluster cluster = init(Cluster.build(2) + .withConfig(config -> config.with(GOSSIP) + .with(NETWORK)) + .start())) + { + cluster.get(2).nodetoolResult("decommission", "--force").asserts().success(); + cluster.get(2).shutdown().get(); + try + { + cluster.get(2).startup(); + fail(); + } + catch (Exception e) + { + cluster.get(2).runOnInstance(() -> ClusterMetadataService.unsetInstance()); + assertTrue(e.getMessage().contains("This node was decommissioned and will not rejoin the ring unless cassandra.override_decommission=true")); + } + + GossipHelper.withProperty(CassandraRelevantProperties.OVERRIDE_DECOMMISSION, true, () -> cluster.get(2).startup()); + assertBootstrapState(cluster.get(2), COMPLETED); + } + } + + private static void assertBootstrapState(IInvokableInstance i, SystemKeyspace.BootstrapState expectedState) + { + String bootstrapState = expectedState.name(); + i.runOnInstance(() -> assertEquals(bootstrapState, SystemKeyspace.getBootstrapState().name())); + } + + private static void assertOperationMode(IInvokableInstance i, StorageService.Mode mode) + { + String operationMode = mode.name(); + i.runOnInstance(() -> assertEquals(operationMode, StorageService.instance.operationMode().name())); + } + } diff --git a/test/distributed/org/apache/cassandra/distributed/test/FailingMoveTest.java b/test/distributed/org/apache/cassandra/distributed/test/FailingMoveTest.java new file mode 100644 index 000000000000..ac60f90cd399 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/FailingMoveTest.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import java.io.IOException; +import java.util.concurrent.Callable; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.junit.Test; + +import net.bytebuddy.ByteBuddy; +import net.bytebuddy.dynamic.loading.ClassLoadingStrategy; +import net.bytebuddy.implementation.MethodDelegation; +import net.bytebuddy.implementation.bind.annotation.SuperCall; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.streaming.StreamPlan; +import org.apache.cassandra.streaming.StreamResultFuture; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.membership.NodeId; + +import static net.bytebuddy.matcher.ElementMatchers.named; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +public class FailingMoveTest extends TestBaseImpl +{ + @Test + public void testResumeMove() throws IOException + { + try (Cluster cluster = init(Cluster.build(3) + .withoutVNodes() + .withConfig(c -> c.with(Feature.GOSSIP, Feature.NETWORK)) + .withInstanceInitializer(BB::install) + .start())) + { + cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl(id int primary key);")); + for (int i=0; i<30; i++) + cluster.coordinator(1).execute(withKeyspace("INSERT INTO %s.tbl (id) VALUES (?)"), + ConsistencyLevel.ALL, i); + String oldToken = getToken(cluster.get(3)); + String moveToToken = "2305843009213693949"; + assertNotEquals(oldToken, moveToToken); + cluster.get(3).nodetoolResult("move", moveToToken).asserts().failure(); + cluster.get(3).runOnInstance(() -> { + assertEquals(StorageService.Mode.MOVE_FAILED, StorageService.instance.operationMode()); + BB.shouldFail.set(false); + }); + + cluster.get(3).nodetoolResult("move", "--resume").asserts().success(); + cluster.get(3).runOnInstance(() -> assertEquals(StorageService.Mode.NORMAL, StorageService.instance.operationMode())); + assertEquals(moveToToken, getToken(cluster.get(3))); + } + } + + @Test + public void testAbortMove() throws IOException + { + try (Cluster cluster = init(Cluster.build(3) + .withoutVNodes() + .withConfig(c -> c.with(Feature.GOSSIP, Feature.NETWORK)) + .withInstanceInitializer(BB::install) + .start())) + { + cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl(id int primary key);")); + for (int i=0; i<30; i++) + cluster.coordinator(1).execute(withKeyspace("INSERT INTO %s.tbl (id) VALUES (?)"), + ConsistencyLevel.ALL, i); + String oldToken = getToken(cluster.get(3)); + String moveToToken = "2305843009213693949"; + assertNotEquals(oldToken, moveToToken); + cluster.get(3).nodetoolResult("move", moveToToken).asserts().failure(); + cluster.get(3).runOnInstance(() -> { + assertEquals(StorageService.Mode.MOVE_FAILED, StorageService.instance.operationMode()); + BB.shouldFail.set(false); + }); + + cluster.get(3).nodetoolResult("move", "--abort").asserts().success(); + cluster.get(3).runOnInstance(() -> assertEquals(StorageService.Mode.NORMAL, StorageService.instance.operationMode())); + assertNotEquals(moveToToken, getToken(cluster.get(3))); + } + } + + private String getToken(IInvokableInstance instance) + { + return instance.callsOnInstance(() -> { + NodeId self = ClusterMetadata.current().myNodeId(); + return ClusterMetadata.current().tokenMap.tokens(self).iterator().next().toString(); + }).call(); + } + + public static class BB + { + static AtomicBoolean shouldFail = new AtomicBoolean(true); + public static void install(ClassLoader classLoader, Integer num) + { + new ByteBuddy().rebase(StreamPlan.class) + .method(named("execute")) + .intercept(MethodDelegation.to(BB.class)) + .make() + .load(classLoader, ClassLoadingStrategy.Default.INJECTION); + } + + @SuppressWarnings("unused") + public static StreamResultFuture execute(@SuperCall Callable<StreamResultFuture> zuper) + { + if (shouldFail.get()) + throw new RuntimeException("failing stream"); + + try + { + return zuper.call(); + } + catch (Exception e) + { + throw new RuntimeException(e); + } + } + } + +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXGetterCheckTest.java b/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXGetterCheckTest.java index fa379ffdc1c5..ea95003093dc 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXGetterCheckTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXGetterCheckTest.java @@ -59,6 +59,8 @@ public class JMXGetterCheckTest extends TestBaseImpl "org.apache.cassandra.db:type=StorageService:clearConnectionHistory", // Throws a NullPointerException "org.apache.cassandra.db:type=StorageService:startGossiping", // causes multiple loops to fail "org.apache.cassandra.db:type=StorageService:startNativeTransport", // causes multiple loops to fail + "org.apache.cassandra.db:type=StorageService:resumeMove", // throws since there is no move in progress + "org.apache.cassandra.db:type=StorageService:abortMove", // throws since there is no move in progress "org.apache.cassandra.db:type=CIDRGroupsMappingManager:loadCidrGroupsCache", // AllowAllCIDRAuthorizer doesn't support this operation, as feature is disabled by default "org.apache.cassandra.db:type=StorageService:forceRemoveCompletion" // deprecated (TCM) ); From e5c919f273e886daf0e85b0e268ec25f48b1be75 Mon Sep 17 00:00:00 2001 From: jaydeepkumar1984 <chovatia.jaydeep@gmail.com> Date: Sun, 15 Sep 2024 14:22:55 -0700 Subject: [PATCH 016/225] Add nodetool checktokenmetadata command that checks TokenMetadata is insync with Gossip endpointState patch by Jaydeepkumar Chovatia; reviewed by Brandon Williams, Mick Semb Wever for CASSANDRA-18758 --- CHANGES.txt | 1 + .../org/apache/cassandra/gms/Gossiper.java | 30 ++++ .../apache/cassandra/gms/GossiperMBean.java | 5 + .../cassandra/service/StorageService.java | 2 +- .../org/apache/cassandra/tools/NodeProbe.java | 4 + .../org/apache/cassandra/tools/NodeTool.java | 1 + .../tools/nodetool/CheckTokenMetadata.java | 50 +++++++ .../apache/cassandra/gms/GossiperTest.java | 129 ++++++++++++++++++ 8 files changed, 221 insertions(+), 1 deletion(-) create mode 100644 src/java/org/apache/cassandra/tools/nodetool/CheckTokenMetadata.java diff --git a/CHANGES.txt b/CHANGES.txt index bcb9dec299d3..457a78868cd4 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.1.8 + * Add nodetool checktokenmetadata command that checks TokenMetadata is insync with Gossip endpointState (CASSANDRA-18758) * Backport Java 11 support for Simulator (CASSANDRA-17178/CASSANDRA-19935) * Equality check for Paxos.Electorate should not depend on collection types (CASSANDRA-19935) * Fix race condition in DecayingEstimatedHistogramReservoir during rescale (CASSANDRA-19365) diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java index ff1264d6fa03..c54eaafcd1a5 100644 --- a/src/java/org/apache/cassandra/gms/Gossiper.java +++ b/src/java/org/apache/cassandra/gms/Gossiper.java @@ -2583,4 +2583,34 @@ public void unsafeSendLocalEndpointStateTo(InetAddressAndPort ep) Message<GossipDigestAck2> message = Message.out(Verb.GOSSIP_DIGEST_ACK2, digestAck2Message); MessagingService.instance().send(message, ep); } + + public Map<String,List<String>> compareGossipAndTokenMetadata() + { + // local epstate will be part of endpointStateMap + Map<String,List<String>> mismatches = new HashMap<>(); + for (InetAddressAndPort endpoint : endpointStateMap.keySet()) + { + EndpointState ep = endpointStateMap.get(endpoint); + // check the status only for NORMAL nodes + if (ep.isNormalState()) + { + List<Token> tokensFromMetadata; + try + { + tokensFromMetadata = new ArrayList<>(StorageService.instance.getTokenMetadata().getTokens(endpoint)); + Collections.sort(tokensFromMetadata); + } + catch(AssertionError e) + { + tokensFromMetadata = Collections.EMPTY_LIST; + } + List<Token> tokensFromGossip = new ArrayList<>(StorageService.instance.getTokensFor(endpoint)); + Collections.sort(tokensFromGossip); + + if (!tokensFromMetadata.equals(tokensFromGossip)) + mismatches.put(endpoint.toString(), ImmutableList.of(tokensFromGossip.toString(), tokensFromMetadata.toString())); + } + } + return mismatches; + } } diff --git a/src/java/org/apache/cassandra/gms/GossiperMBean.java b/src/java/org/apache/cassandra/gms/GossiperMBean.java index 47d7207ef86e..cb02fe8bbda1 100644 --- a/src/java/org/apache/cassandra/gms/GossiperMBean.java +++ b/src/java/org/apache/cassandra/gms/GossiperMBean.java @@ -38,4 +38,9 @@ public interface GossiperMBean /** Returns each node's database release version */ public Map<String, List<String>> getReleaseVersionsWithPort(); + /** Returns a map of endpoints that don't have matching tokenMetadata to gossip tokens. + * Values are lists of fixed size two (as Pair is not jmx safe), + * the first is the gossip tokens, the second tokenMetadata). + */ + public Map<String, List<String>> compareGossipAndTokenMetadata(); } diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 8b4a66fc5e8c..468bf973c72a 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -2885,7 +2885,7 @@ public void setRpcReady(boolean value) Gossiper.instance.addLocalApplicationState(ApplicationState.RPC_READY, valueFactory.rpcReady(value)); } - private Collection<Token> getTokensFor(InetAddressAndPort endpoint) + public Collection<Token> getTokensFor(InetAddressAndPort endpoint) { try { diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index a0de78acd50e..b73f6dd818b2 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -1583,6 +1583,10 @@ public Map<String,long[]> getOutOfRangeOpCounts() { return ssProxy.getOutOfRangeOperationCounts(); } + public Map<String, List<String>> compareGossipAndTokenMetadata() + { + return gossProxy.compareGossipAndTokenMetadata(); + } // JMX getters for the o.a.c.metrics API below. /** diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java index 8d87c88906b1..b54cbee7384a 100644 --- a/src/java/org/apache/cassandra/tools/NodeTool.java +++ b/src/java/org/apache/cassandra/tools/NodeTool.java @@ -96,6 +96,7 @@ public int execute(String... args) CassHelp.class, CfHistograms.class, CfStats.class, + CheckTokenMetadata.class, Cleanup.class, ClearSnapshot.class, ClientStats.class, diff --git a/src/java/org/apache/cassandra/tools/nodetool/CheckTokenMetadata.java b/src/java/org/apache/cassandra/tools/nodetool/CheckTokenMetadata.java new file mode 100644 index 000000000000..1f9dd8091f3c --- /dev/null +++ b/src/java/org/apache/cassandra/tools/nodetool/CheckTokenMetadata.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.tools.nodetool; + +import java.util.List; +import java.util.Map; + +import io.airlift.airline.Command; + +import org.apache.cassandra.tools.NodeProbe; +import org.apache.cassandra.tools.NodeTool.NodeToolCmd; + +@Command(name = "checktokenmetadata", description = "compares the Gossip endpointState and TokenMetadata cache; printing any mismatches found") +public class CheckTokenMetadata extends NodeToolCmd +{ + @Override + public void execute(NodeProbe probe) + { + /** Cassandra maintains the token information in two places: 1) Gossip endpointState, and 2) TokenMetadata. + * The probabilistic view of the cluster is the Gossip endpointState. + * This then updates the TokenMetadata, a resulting topology view that's usable by hotpaths (e.g. read and write requests). + * Bugs can result in these falling out of sync. + * This command compares the Gossip endpointState and TokenMetadata cache, printing any mismatches found. + */ + StringBuilder sb = new StringBuilder(); + Map<String, List<String>> mismatches = probe.compareGossipAndTokenMetadata(); + + for (Map.Entry<String,List<String>> e : mismatches.entrySet()) + sb.append("Mismatch on : ").append(e.getKey()) + .append("\n Gossip tokens: ").append(e.getValue().get(0)) + .append("\n TokenMetadata: ").append(e.getValue().get(1)).append('\n'); + + System.out.println(sb); + } +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/gms/GossiperTest.java b/test/unit/org/apache/cassandra/gms/GossiperTest.java index 97a900876fd0..f79dc5269e12 100644 --- a/test/unit/org/apache/cassandra/gms/GossiperTest.java +++ b/test/unit/org/apache/cassandra/gms/GossiperTest.java @@ -18,11 +18,14 @@ package org.apache.cassandra.gms; +import java.io.IOException; import java.net.InetAddress; import java.net.UnknownHostException; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; @@ -35,10 +38,12 @@ import org.junit.Before; import org.junit.Test; +import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.Util; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.SystemKeyspace; import org.apache.cassandra.db.commitlog.CommitLog; +import org.apache.cassandra.dht.ByteOrderedPartitioner; import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.RandomPartitioner; import org.apache.cassandra.dht.Token; @@ -51,6 +56,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -87,6 +93,7 @@ public void setup() public void tearDown() { DatabaseDescriptor.setSeedProvider(originalSeedProvider); + Gossiper.instance.endpointStateMap.clear(); } @AfterClass @@ -487,6 +494,128 @@ public void testNotFireDuplicatedNotificationsWithUpdateContainsOldAndNewState() } } + @Test + public void testGossipAndTokenMetadataCacheMismatchExist() throws IOException + { + SchemaLoader.prepareServer(); + Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, 2); + Gossiper.instance.start(1); + EndpointState host0State = Gossiper.instance.getEndpointStateForEndpoint(hosts.get(0)); + EndpointState host1State = Gossiper.instance.getEndpointStateForEndpoint(hosts.get(1)); + + Gossiper.instance.injectApplicationState(hosts.get(1), ApplicationState.RELEASE_VERSION, new VersionedValue.VersionedValueFactory(null).releaseVersion(SystemKeyspace.CURRENT_VERSION.toString())); + host0State.addApplicationState(ApplicationState.STATUS_WITH_PORT, StorageService.instance.valueFactory.normal(new ArrayList<Token>(){{add(DatabaseDescriptor.getPartitioner().getRandomToken());}})); + host1State.addApplicationState(ApplicationState.STATUS_WITH_PORT, StorageService.instance.valueFactory.normal(new ArrayList<Token>(){{add(DatabaseDescriptor.getPartitioner().getRandomToken());}})); + Gossiper.instance.applyStateLocally(ImmutableMap.of(hosts.get(1), host0State)); + Map<String, List<String>> output = Gossiper.instance.compareGossipAndTokenMetadata(); + assertTrue(output.isEmpty()); + assertEquals(StorageService.instance.getTokenMetadata().getTokens(hosts.get(0)), StorageService.instance.getTokensFor(hosts.get(0))); + assertEquals(StorageService.instance.getTokenMetadata().getTokens(hosts.get(1)), StorageService.instance.getTokensFor(hosts.get(1))); + + // edit the tokens in the Gossip cache to create a mismatch between the two caches + host0State.addApplicationState(ApplicationState.TOKENS, StorageService.instance.valueFactory.tokens(new ArrayList<Token>(){{add(DatabaseDescriptor.getPartitioner().getRandomToken());}})); + + output = Gossiper.instance.compareGossipAndTokenMetadata(); + assertTrue(output.containsKey("/127.0.0.1:7012")); + } + + @Test + public void testGossipAndTokenMetadataCacheMismatchDoNotExist() throws IOException + { + SchemaLoader.prepareServer(); + Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, 2); + Gossiper.instance.start(1); + + Gossiper.instance.injectApplicationState(hosts.get(1), ApplicationState.RELEASE_VERSION, new VersionedValue.VersionedValueFactory(null).releaseVersion(SystemKeyspace.CURRENT_VERSION.toString())); + + assertEquals(StorageService.instance.getTokenMetadata().getTokens(hosts.get(0)), StorageService.instance.getTokensFor(hosts.get(0))); + assertEquals(StorageService.instance.getTokenMetadata().getTokens(hosts.get(1)), StorageService.instance.getTokensFor(hosts.get(1))); + // no impact to the cache if there is no cache coherence + Map<String, List<String>> output = Gossiper.instance.compareGossipAndTokenMetadata(); + assertTrue(output.isEmpty()); + assertEquals(StorageService.instance.getTokenMetadata().getTokens(hosts.get(0)), StorageService.instance.getTokensFor(hosts.get(0))); + assertEquals(StorageService.instance.getTokenMetadata().getTokens(hosts.get(1)), StorageService.instance.getTokensFor(hosts.get(1))); + } + + + @Test + public void testGossipAndTokenMetadataCacheIgnoreNonNormalNode() throws IOException + { + SchemaLoader.prepareServer(); + Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, 2); + Gossiper.instance.start(1); + + EndpointState host0State = Gossiper.instance.getEndpointStateForEndpoint(hosts.get(0)); + Gossiper.instance.injectApplicationState(hosts.get(1), ApplicationState.RELEASE_VERSION, new VersionedValue.VersionedValueFactory(null).releaseVersion(SystemKeyspace.CURRENT_VERSION.toString())); + + assertEquals(StorageService.instance.getTokenMetadata().getTokens(hosts.get(0)), StorageService.instance.getTokensFor(hosts.get(0))); + assertEquals(StorageService.instance.getTokenMetadata().getTokens(hosts.get(1)), StorageService.instance.getTokensFor(hosts.get(1))); + + // Change host1's status to non-normal + host0State.addApplicationState(ApplicationState.STATUS, StorageService.instance.valueFactory.leaving(new ArrayList<Token>(){{add(new ByteOrderedPartitioner.BytesToken(new byte[]{ 1}));}})); + // Now intentionally inject inconsistency between the Gossip cache and storage service cache + host0State.addApplicationState(ApplicationState.TOKENS, StorageService.instance.valueFactory.tokens(new ArrayList<Token>(){{add(new ByteOrderedPartitioner.BytesToken(new byte[]{1,2,3}));}})); + + assertNotEquals(StorageService.instance.getTokenMetadata().getTokens(hosts.get(0)), StorageService.instance.getTokensFor(hosts.get(0))); + assertEquals(StorageService.instance.getTokenMetadata().getTokens(hosts.get(1)), StorageService.instance.getTokensFor(hosts.get(1))); + + // Because host1 is not yet "NORMAL", it should be skipped in fixing the caches + Map<String, List<String>> output = Gossiper.instance.compareGossipAndTokenMetadata(); + assertTrue(output.isEmpty()); + assertNotEquals(StorageService.instance.getTokenMetadata().getTokens(hosts.get(0)), StorageService.instance.getTokensFor(hosts.get(0))); + assertEquals(StorageService.instance.getTokenMetadata().getTokens(hosts.get(1)), StorageService.instance.getTokensFor(hosts.get(1))); + } + + @Test + public void testCacheMismatchIfTokenMetadataCacheIsMissingTheEndpoint() throws IOException + { + SchemaLoader.prepareServer(); + Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, 2); + Gossiper.instance.start(1); + EndpointState host0State = Gossiper.instance.getEndpointStateForEndpoint(hosts.get(0)); + EndpointState host1State = Gossiper.instance.getEndpointStateForEndpoint(hosts.get(1)); + + Gossiper.instance.injectApplicationState(hosts.get(1), ApplicationState.RELEASE_VERSION, new VersionedValue.VersionedValueFactory(null).releaseVersion(SystemKeyspace.CURRENT_VERSION.toString())); + host0State.addApplicationState(ApplicationState.STATUS_WITH_PORT, StorageService.instance.valueFactory.normal(new ArrayList<Token>(){{add(DatabaseDescriptor.getPartitioner().getRandomToken());}})); + host1State.addApplicationState(ApplicationState.STATUS_WITH_PORT, StorageService.instance.valueFactory.normal(new ArrayList<Token>(){{add(DatabaseDescriptor.getPartitioner().getRandomToken());}})); + Gossiper.instance.applyStateLocally(ImmutableMap.of(hosts.get(1), host0State)); + Map<String, List<String>> output = Gossiper.instance.compareGossipAndTokenMetadata(); + assertTrue(output.isEmpty()); + // Gossip tokens should be used as the source of truth in case of a mismtach + Collection<Token> gossipTokensHost0 = getGossipCacheTokens(0); + Collection<Token> gossipTokensHost1 = getGossipCacheTokens(1); + + assertEquals(getTokenMetadataCacheTokens(0), getGossipCacheTokens(0)); + assertEquals(getTokenMetadataCacheTokens(1), getGossipCacheTokens(1)); + + // remove the tokens from TokenMetadata cache to create a mismatch between the two caches + StorageService.instance.getTokenMetadata().removeEndpoint(hosts.get(0)); + output = Gossiper.instance.compareGossipAndTokenMetadata(); + assertTrue(output.containsKey("/127.0.0.1:7012")); + try + { + getTokenMetadataCacheTokens(0); + fail("Expected AssertionError"); + } + catch (AssertionError e) + { + assertTrue(e.getMessage().contains("Unable to get tokens for /127.0.0.1:7012; it is not a member")); + } + assertEquals(getTokenMetadataCacheTokens(1), gossipTokensHost1); + assertEquals(getTokenMetadataCacheTokens(1), getGossipCacheTokens(1)); + } + + private Collection<Token> getTokenMetadataCacheTokens(int hostIndex) + { + return StorageService.instance.getTokenMetadata().getTokens(hosts.get(hostIndex)); + } + + private Collection<Token> getGossipCacheTokens(int hostIndex) + { + return StorageService.instance.getTokensFor(hosts.get(hostIndex)); + } + + static class SimpleStateChangeListener implements IEndpointStateChangeSubscriber { static class OnChangeParams From 181ddc0600cb7104ff0a9e01889f7cb013f7f802 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Mon, 4 Nov 2024 17:11:19 +0100 Subject: [PATCH 017/225] Fix CQL in snapshot's schema which did not contained UDTs used as reverse clustering columns patch by Stefan Miklosovic; reviewed by David Capwell for CASSANDRA-20036 --- CHANGES.txt | 1 + .../cassandra/schema/TableMetadata.java | 11 +- .../cassandra/db/SchemaCQLHelperTest.java | 107 ++++++++++++------ 3 files changed, 81 insertions(+), 38 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 486d8ad7d2a0..823be216cb37 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0.15 + * Fix CQL in snapshot's schema which did not contained UDTs used as reverse clustering columns (CASSANDRA-20036) * Add configurable batchlog endpoint strategies: random_remote, prefer_local, dynamic_remote, and dynamic (CASSANDRA-18120) * Fix bash-completion for debian distro (CASSANDRA-19999) * Ensure thread-safety for CommitLogArchiver in CommitLog (CASSANDRA-19960) diff --git a/src/java/org/apache/cassandra/schema/TableMetadata.java b/src/java/org/apache/cassandra/schema/TableMetadata.java index bd51349c7216..543a51fa55ec 100644 --- a/src/java/org/apache/cassandra/schema/TableMetadata.java +++ b/src/java/org/apache/cassandra/schema/TableMetadata.java @@ -1107,11 +1107,14 @@ public Set<ByteBuffer> getReferencedUserTypes() */ private static void addUserTypes(AbstractType<?> type, Set<ByteBuffer> types) { - // Reach into subtypes first, so that if the type is a UDT, it's dependencies are recreated first. - type.subTypes().forEach(t -> addUserTypes(t, types)); + AbstractType<?> unwrapped = type.unwrap(); - if (type.isUDT()) - types.add(((UserType)type).name); + if (unwrapped.isUDT()) + { + // Reach into subtypes first, so that if the type is a UDT, it's dependencies are recreated first. + unwrapped.subTypes().forEach(t -> addUserTypes(t, types)); + types.add(((UserType)unwrapped).name); + } } @Override diff --git a/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java b/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java index ed68b35c2649..d32a587582d9 100644 --- a/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java +++ b/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java @@ -45,6 +45,7 @@ import java.nio.charset.Charset; import java.util.Arrays; import java.util.Collections; +import java.util.List; import java.util.stream.Collectors; import static org.hamcrest.CoreMatchers.allOf; @@ -64,10 +65,76 @@ public void defineSchema() throws ConfigurationException @Test public void testUserTypesCQL() { - String keyspace = "cql_test_keyspace_user_types"; - String table = "test_table_user_types"; + String keyspaceForUserTypeTests = "cql_test_keyspace_user_types_1"; + String tableForUserTypeTests = "test_table_user_types_1"; + + UserType[] types = getTypes(keyspaceForUserTypeTests); + executeTest(keyspaceForUserTypeTests, tableForUserTypeTests, TableMetadata.builder(keyspaceForUserTypeTests, tableForUserTypeTests) + .addPartitionKeyColumn("pk1", IntegerType.instance) + .addClusteringColumn("ck1", IntegerType.instance) + .addRegularColumn("reg1", types[2].freeze()) // type C + .addRegularColumn("reg2", ListType.getInstance(IntegerType.instance, false)) + .addRegularColumn("reg3", MapType.getInstance(AsciiType.instance, IntegerType.instance, true)) + .build(), + types); + } + + @Test + public void testReversedClusteringUserTypeCQL() + { + String keyspaceForUserTypeTests = "cql_test_keyspace_user_types_2"; + String tableForUserTypeTests = "test_table_user_types_2"; + + UserType[] types = getTypes(keyspaceForUserTypeTests); + executeTest(keyspaceForUserTypeTests, tableForUserTypeTests, TableMetadata.builder(keyspaceForUserTypeTests, tableForUserTypeTests) + .addPartitionKeyColumn("pk1", IntegerType.instance) + .addClusteringColumn("cl1", ReversedType.getInstance(types[2].freeze())) // type C + .addRegularColumn("reg2", ListType.getInstance(IntegerType.instance, false)) + .addRegularColumn("reg3", MapType.getInstance(AsciiType.instance, IntegerType.instance, true)) + .build(), types); + } + + + private void executeTest(String keyspaceForUserTypeTests, String tableForUserTypeTests, TableMetadata cfm, UserType[] userTypes) + { + SchemaLoader.createKeyspace(keyspaceForUserTypeTests, KeyspaceParams.simple(1), Tables.of(cfm), Types.of(userTypes)); + + ColumnFamilyStore cfs = Keyspace.open(keyspaceForUserTypeTests).getColumnFamilyStore(tableForUserTypeTests); + + List<String> typeStatements = ImmutableList.of("CREATE TYPE IF NOT EXISTS " + keyspaceForUserTypeTests +".a (\n" + + " a1 varint,\n" + + " a2 varint,\n" + + " a3 varint\n" + + ");", + "CREATE TYPE IF NOT EXISTS " + keyspaceForUserTypeTests +".b (\n" + + " b1 a,\n" + + " b2 a,\n" + + " b3 a\n" + + ");", + "CREATE TYPE IF NOT EXISTS " + keyspaceForUserTypeTests +".c (\n" + + " c1 b,\n" + + " c2 b,\n" + + " c3 b\n" + + ");"); + + assertEquals(typeStatements, SchemaCQLHelper.getUserTypesAsCQL(cfs.metadata(), cfs.keyspace.getMetadata().types, true).collect(Collectors.toList())); + + List<String> allStatements = SchemaCQLHelper.reCreateStatementsForSchemaCql(cfm, Types.of(userTypes)).collect(Collectors.toList()); - UserType typeA = new UserType(keyspace, ByteBufferUtil.bytes("a"), + String createTableStatement = SchemaCQLHelper.getTableMetadataAsCQL(cfm, true, true, true); + + assertEquals(3, typeStatements.size()); + assertEquals(4, allStatements.size()); + + for (int i = 0; i < typeStatements.size(); i++) + assertEquals(allStatements.get(i), typeStatements.get(i)); + + assertEquals(createTableStatement, allStatements.get(3)); + } + + private UserType[] getTypes(String keyspaceForUserTypeTests) + { + UserType typeA = new UserType(keyspaceForUserTypeTests, ByteBufferUtil.bytes("a"), Arrays.asList(FieldIdentifier.forUnquoted("a1"), FieldIdentifier.forUnquoted("a2"), FieldIdentifier.forUnquoted("a3")), @@ -76,7 +143,7 @@ public void testUserTypesCQL() IntegerType.instance), true); - UserType typeB = new UserType(keyspace, ByteBufferUtil.bytes("b"), + UserType typeB = new UserType(keyspaceForUserTypeTests, ByteBufferUtil.bytes("b"), Arrays.asList(FieldIdentifier.forUnquoted("b1"), FieldIdentifier.forUnquoted("b2"), FieldIdentifier.forUnquoted("b3")), @@ -85,7 +152,7 @@ public void testUserTypesCQL() typeA), true); - UserType typeC = new UserType(keyspace, ByteBufferUtil.bytes("c"), + UserType typeC = new UserType(keyspaceForUserTypeTests, ByteBufferUtil.bytes("c"), Arrays.asList(FieldIdentifier.forUnquoted("c1"), FieldIdentifier.forUnquoted("c2"), FieldIdentifier.forUnquoted("c3")), @@ -94,35 +161,7 @@ public void testUserTypesCQL() typeB), true); - TableMetadata cfm = - TableMetadata.builder(keyspace, table) - .addPartitionKeyColumn("pk1", IntegerType.instance) - .addClusteringColumn("ck1", IntegerType.instance) - .addRegularColumn("reg1", typeC.freeze()) - .addRegularColumn("reg2", ListType.getInstance(IntegerType.instance, false)) - .addRegularColumn("reg3", MapType.getInstance(AsciiType.instance, IntegerType.instance, true)) - .build(); - - SchemaLoader.createKeyspace(keyspace, KeyspaceParams.simple(1), Tables.of(cfm), Types.of(typeA, typeB, typeC)); - - ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(table); - - assertEquals(ImmutableList.of("CREATE TYPE cql_test_keyspace_user_types.a (\n" + - " a1 varint,\n" + - " a2 varint,\n" + - " a3 varint\n" + - ");", - "CREATE TYPE cql_test_keyspace_user_types.b (\n" + - " b1 a,\n" + - " b2 a,\n" + - " b3 a\n" + - ");", - "CREATE TYPE cql_test_keyspace_user_types.c (\n" + - " c1 b,\n" + - " c2 b,\n" + - " c3 b\n" + - ");"), - SchemaCQLHelper.getUserTypesAsCQL(cfs.metadata(), cfs.keyspace.getMetadata().types, false).collect(Collectors.toList())); + return new UserType[] {typeA, typeB, typeC}; } @Test From 6093c2d99bd300eebe353ff05482924f38538e74 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Mon, 4 Nov 2024 14:06:42 +0100 Subject: [PATCH 018/225] Support UDTs and vectors as clustering keys in descending order patch by Stefan Miklosovic; reviewed by David Capwell for CASSANDRA-20050 --- CHANGES.txt | 1 + .../org/apache/cassandra/cql3/UserTypes.java | 12 +++++++----- .../cassandra/cql3/selection/Selectable.java | 2 +- .../cql3/validation/entities/UserTypesTest.java | 16 ++++++++++++++++ 4 files changed, 25 insertions(+), 6 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 823be216cb37..d8cab3521071 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0.15 + * Support UDTs and vectors as clustering keys in descending order (CASSANDRA-20050) * Fix CQL in snapshot's schema which did not contained UDTs used as reverse clustering columns (CASSANDRA-20036) * Add configurable batchlog endpoint strategies: random_remote, prefer_local, dynamic_remote, and dynamic (CASSANDRA-18120) * Fix bash-completion for debian distro (CASSANDRA-19999) diff --git a/src/java/org/apache/cassandra/cql3/UserTypes.java b/src/java/org/apache/cassandra/cql3/UserTypes.java index b023a8a0b8f6..df2e26fc47e8 100644 --- a/src/java/org/apache/cassandra/cql3/UserTypes.java +++ b/src/java/org/apache/cassandra/cql3/UserTypes.java @@ -41,7 +41,7 @@ private UserTypes() {} public static ColumnSpecification fieldSpecOf(ColumnSpecification column, int field) { - UserType ut = (UserType)column.type; + UserType ut = (UserType)column.type.unwrap(); return new ColumnSpecification(column.ksName, column.cfName, new ColumnIdentifier(column.name + "." + ut.fieldName(field), true), @@ -132,7 +132,7 @@ public Term prepare(String keyspace, ColumnSpecification receiver) throws Invali { validateAssignableTo(keyspace, receiver); - UserType ut = (UserType)receiver.type; + UserType ut = (UserType)receiver.type.unwrap(); boolean allTerminal = true; List<Term> values = new ArrayList<>(entries.size()); int foundValues = 0; @@ -161,16 +161,18 @@ public Term prepare(String keyspace, ColumnSpecification receiver) throws Invali } } - DelayedValue value = new DelayedValue(((UserType)receiver.type), values); + DelayedValue value = new DelayedValue(((UserType)receiver.type.unwrap()), values); return allTerminal ? value.bind(QueryOptions.DEFAULT) : value; } private void validateAssignableTo(String keyspace, ColumnSpecification receiver) throws InvalidRequestException { - if (!receiver.type.isUDT()) + AbstractType<?> unwrapped = receiver.type.unwrap(); + + if (!unwrapped.isUDT()) throw new InvalidRequestException(String.format("Invalid user type literal for %s of type %s", receiver.name, receiver.type.asCQL3Type())); - UserType ut = (UserType)receiver.type; + UserType ut = (UserType)unwrapped; for (int i = 0; i < ut.size(); i++) { FieldIdentifier field = ut.fieldName(i); diff --git a/src/java/org/apache/cassandra/cql3/selection/Selectable.java b/src/java/org/apache/cassandra/cql3/selection/Selectable.java index 6e653ba1b913..2218f89407c9 100644 --- a/src/java/org/apache/cassandra/cql3/selection/Selectable.java +++ b/src/java/org/apache/cassandra/cql3/selection/Selectable.java @@ -529,7 +529,7 @@ public Selector.Factory newSelectorFactory(TableMetadata table, AbstractType<?> } Selector.Factory factory = selected.newSelectorFactory(table, expectedUdtType, defs, boundNames); - AbstractType<?> type = factory.getReturnType(); + AbstractType<?> type = factory.getReturnType().unwrap(); if (!type.isUDT()) { throw new InvalidRequestException( diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java index f1b6b8fb874b..c6256f7a7a5c 100644 --- a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java +++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java @@ -102,6 +102,22 @@ public void testFor7684() throws Throwable ); } + @Test + public void testDescendingOrderingOfUserTypesIsSupported() throws Throwable + { + String myType = createType("CREATE TYPE %s (x double)"); + createTable("CREATE TABLE %s (k int, v frozen<" + myType + ">, b boolean static, PRIMARY KEY (k, v)) WITH CLUSTERING ORDER BY (v DESC)"); + + execute("INSERT INTO %s(k, v) VALUES (?, {x:?})", 1, -104.99251); + execute("UPDATE %s SET b = ? WHERE k = ?", true, 1); + + beforeAndAfterFlush(() -> + assertRows(execute("SELECT v.x FROM %s WHERE k = ? AND v = {x:?}", 1, -104.99251), + row(-104.99251) + ) + ); + } + @Test public void testInvalidUDTStatements() throws Throwable { From 8ea70cd1f0ba0a52cf4b19af9fac4c4289663270 Mon Sep 17 00:00:00 2001 From: Josh McKenzie <jmckenzie@apache.org> Date: Thu, 11 Aug 2022 14:02:27 -0400 Subject: [PATCH 019/225] Backport of CASSANDRA-17812: Rate-limit new client connection auth setup to avoid overwhelming bcrypt This backport differs from CASSANDRA-17812 in such a way that by default the number of auth request threads is set to 0. That will route all requests to request executor as before this change. The patch in 5.0 and later sets the default number of auth request threads to 4. patch by Josh McKenzie; reviewed by Chris Lohfink for CASSANDRA-20057 Co-authored-by: Stefan Miklosovic <smiklosovic@apache.org> --- CHANGES.txt | 1 + conf/cassandra.yaml | 3 + .../org/apache/cassandra/config/Config.java | 2 + .../cassandra/config/DatabaseDescriptor.java | 16 ++ .../cassandra/transport/Dispatcher.java | 42 +++- .../transport/MessageDispatcherTest.java | 180 ++++++++++++++++++ 6 files changed, 237 insertions(+), 7 deletions(-) create mode 100644 test/unit/org/apache/cassandra/transport/MessageDispatcherTest.java diff --git a/CHANGES.txt b/CHANGES.txt index d8cab3521071..0eb7ed5f7e49 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0.15 + * Backport of CASSANDRA-17812: Rate-limit new client connection auth setup to avoid overwhelming bcrypt (CASSANDRA-20057) * Support UDTs and vectors as clustering keys in descending order (CASSANDRA-20050) * Fix CQL in snapshot's schema which did not contained UDTs used as reverse clustering columns (CASSANDRA-20036) * Add configurable batchlog endpoint strategies: random_remote, prefer_local, dynamic_remote, and dynamic (CASSANDRA-18120) diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index 835a8f4fa62f..2f1184392194 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -764,6 +764,9 @@ native_transport_port: 9042 # The maximum threads for handling requests (note that idle threads are stopped # after 30 seconds so there is not corresponding minimum setting). # native_transport_max_threads: 128 +# The maximum threads for handling auth requests in a separate executor from main request executor. +# When set to 0, main executor for requests is used. +# native_transport_max_auth_threads: 0 # # The maximum size of allowed frame. Frame (requests) larger than this will # be rejected as invalid. The default is 256MB. If you're changing this parameter, diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 995aaa69d272..57388eed685c 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -195,6 +195,8 @@ public class Config public Integer native_transport_port_ssl = null; public int native_transport_max_threads = 128; public int native_transport_max_frame_size_in_mb = 256; + /** do bcrypt hashing in a limited pool to prevent cpu load spikes; 0 means that all requests will go to default request executor**/ + public int native_transport_max_auth_threads = 0; public volatile long native_transport_max_concurrent_connections = -1L; public volatile long native_transport_max_concurrent_connections_per_ip = -1L; public boolean native_transport_flush_in_batches_legacy = false; diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index d425e5380791..bf96a3f8856e 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -2259,6 +2259,22 @@ public static void setNativeTransportMaxThreads(int max_threads) conf.native_transport_max_threads = max_threads; } + public static Integer getNativeTransportMaxAuthThreads() + { + return conf.native_transport_max_auth_threads; + } + + /** + * If this value is set to <= 0 it will move auth requests to the standard request pool regardless of the current + * size of the authExecutor in {@link org.apache.cassandra.transport.Dispatcher}'s active size. + * + * see {@link org.apache.cassandra.transport.Dispatcher#dispatch} for executor selection + */ + public static void setNativeTransportMaxAuthThreads(int threads) + { + conf.native_transport_max_auth_threads = threads; + } + public static int getNativeTransportMaxFrameSize() { return (int) ByteUnit.MEBI_BYTES.toBytes(conf.native_transport_max_frame_size_in_mb); diff --git a/src/java/org/apache/cassandra/transport/Dispatcher.java b/src/java/org/apache/cassandra/transport/Dispatcher.java index 05b55e801052..e1318d1afe18 100644 --- a/src/java/org/apache/cassandra/transport/Dispatcher.java +++ b/src/java/org/apache/cassandra/transport/Dispatcher.java @@ -22,6 +22,8 @@ import java.util.concurrent.ConcurrentMap; import java.util.function.Consumer; +import com.google.common.annotations.VisibleForTesting; + import io.netty.channel.Channel; import io.netty.channel.EventLoop; import io.netty.util.AttributeKey; @@ -39,11 +41,33 @@ public class Dispatcher { - private static final LocalAwareExecutorService requestExecutor = SHARED.newExecutor(DatabaseDescriptor.getNativeTransportMaxThreads(), + @VisibleForTesting + static final LocalAwareExecutorService requestExecutor = SHARED.newExecutor(DatabaseDescriptor.getNativeTransportMaxThreads(), DatabaseDescriptor::setNativeTransportMaxThreads, "transport", "Native-Transport-Requests"); + /** CASSANDRA-17812: Rate-limit new client connection setup to avoid overwhelming during bcrypt + * + * Backported by CASSANDRA-20057 + * + * authExecutor is a separate thread pool for handling requests on connections that need to be authenticated. + * Calls to AUTHENTICATE can be expensive if the number of rounds for bcrypt is configured to a high value, + * so during a connection storm checking the password hash would starve existing connected clients for CPU and + * trigger timeouts if on the same thread pool as standard requests. + * + * Moving authentication requests to a small, separate pool prevents starvation handling all other + * requests. If the authExecutor pool backs up, it may cause authentication timeouts but the clients should + * back off and retry while the rest of the system continues to make progress. + * + * Setting less than 1 will service auth requests on the standard {@link Dispatcher#requestExecutor} + */ + @VisibleForTesting + static final LocalAwareExecutorService authExecutor = SHARED.newExecutor(Math.max(1, DatabaseDescriptor.getNativeTransportMaxAuthThreads()), + DatabaseDescriptor::setNativeTransportMaxAuthThreads, + "transport", + "Native-Transport-Auth-Requests"); + private static final ConcurrentMap<EventLoop, Flusher> flusherLookup = new ConcurrentHashMap<>(); private final boolean useLegacyFlusher; @@ -67,7 +91,14 @@ public Dispatcher(boolean useLegacyFlusher) public void dispatch(Channel channel, Message.Request request, FlushItemConverter forFlusher) { - requestExecutor.submit(() -> processRequest(channel, request, forFlusher)); + // if native_transport_max_auth_threads is < 1, don't delegate to new pool on auth messages + boolean isAuthQuery = DatabaseDescriptor.getNativeTransportMaxAuthThreads() > 0 && + (request.type == Message.Type.AUTH_RESPONSE || request.type == Message.Type.CREDENTIALS); + + // Importantly, the authExecutor will handle the AUTHENTICATE message which may be CPU intensive. + LocalAwareExecutorService executor = isAuthQuery ? authExecutor : requestExecutor; + + executor.submit(() -> processRequest(channel, request, forFlusher)); } /** @@ -140,13 +171,10 @@ private void flush(FlushItem<?> item) public static void shutdown() { - if (requestExecutor != null) - { - requestExecutor.shutdown(); - } + requestExecutor.shutdown(); + authExecutor.shutdown(); } - /** * Dispatcher for EventMessages. In {@link Server.ConnectionTracker#send(Event)}, the strategy * for delivering events to registered clients is dependent on protocol version and the configuration diff --git a/test/unit/org/apache/cassandra/transport/MessageDispatcherTest.java b/test/unit/org/apache/cassandra/transport/MessageDispatcherTest.java new file mode 100644 index 000000000000..244cd2f4fe2d --- /dev/null +++ b/test/unit/org/apache/cassandra/transport/MessageDispatcherTest.java @@ -0,0 +1,180 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.transport; + +import java.util.Collections; +import java.util.concurrent.Callable; +import java.util.concurrent.TimeUnit; + +import com.google.common.util.concurrent.Uninterruptibles; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import io.netty.channel.Channel; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.metrics.ClientMetrics; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.transport.messages.AuthResponse; + +public class MessageDispatcherTest +{ + static final Message.Request AUTH_RESPONSE_REQUEST = new AuthResponse(new byte[0]) + { + public Response execute(QueryState queryState, long queryStartNanoTime, boolean traceRequest) + { + return null; + } + }; + + private static AuthTestDispatcher dispatch; + private static int maxAuthThreadsBeforeTests; + + @BeforeClass + public static void init() throws Exception + { + DatabaseDescriptor.daemonInitialization(); + ClientMetrics.instance.init(Collections.emptyList()); + maxAuthThreadsBeforeTests = DatabaseDescriptor.getNativeTransportMaxAuthThreads(); + dispatch = new AuthTestDispatcher(); + } + + @AfterClass + public static void restoreAuthSize() + { + DatabaseDescriptor.setNativeTransportMaxAuthThreads(maxAuthThreadsBeforeTests); + } + + @Test + public void testAuthRateLimiterTurnedOffByDefault() throws Exception + { + // All requests were executed on main request executor + Assert.assertEquals(0, tryAuth(this::completedAuth)); + Assert.assertEquals(1, completedRequests()); + } + + @Test + public void testAuthRateLimiter() throws Exception + { + long startRequests = completedRequests(); + + DatabaseDescriptor.setNativeTransportMaxAuthThreads(1); + long auths = tryAuth(this::completedAuth); + Assert.assertEquals(auths, 1); + + DatabaseDescriptor.setNativeTransportMaxAuthThreads(100); + auths = tryAuth(this::completedAuth); + Assert.assertEquals(auths, 1); + + // Make sure no tasks executed on the regular pool + Assert.assertEquals(startRequests, completedRequests()); + } + + @Test + public void testAuthRateLimiterNotUsed() throws Exception + { + DatabaseDescriptor.setNativeTransportMaxAuthThreads(1); + for (Message.Type type : Message.Type.values()) + { + if (type == Message.Type.AUTH_RESPONSE || type == Message.Type.CREDENTIALS || type.direction != Message.Direction.REQUEST) + continue; + + long auths = completedAuth(); + long requests = tryAuth(this::completedRequests, new Message.Request(type) + { + public Response execute(QueryState queryState, long queryStartNanoTime, boolean traceRequest) + { + return null; + } + }); + Assert.assertEquals(requests, 1); + Assert.assertEquals(completedAuth() - auths, 0); + } + } + + @Test + public void testAuthRateLimiterDisabled() throws Exception + { + long startAuthRequests = completedAuth(); + + DatabaseDescriptor.setNativeTransportMaxAuthThreads(0); + long requests = tryAuth(this::completedRequests); + Assert.assertEquals(requests, 1); + + DatabaseDescriptor.setNativeTransportMaxAuthThreads(-1); + requests = tryAuth(this::completedRequests); + Assert.assertEquals(requests, 1); + + DatabaseDescriptor.setNativeTransportMaxAuthThreads(-1000); + requests = tryAuth(this::completedRequests); + Assert.assertEquals(requests, 1); + + // Make sure no tasks executed on the auth pool + Assert.assertEquals(startAuthRequests, completedAuth()); + } + + private long completedRequests() + { + return Dispatcher.requestExecutor.getCompletedTaskCount(); + } + + private long completedAuth() + { + return Dispatcher.authExecutor.getCompletedTaskCount(); + } + + public long tryAuth(Callable<Long> check) throws Exception + { + return tryAuth(check, AUTH_RESPONSE_REQUEST); + } + + @SuppressWarnings("UnstableApiUsage") + public long tryAuth(Callable<Long> check, Message.Request request) throws Exception + { + long start = check.call(); + dispatch.dispatch(null, request, (channel,req,response) -> null); + + // While this is timeout based, we should be *well below* a full second on any of this processing in any sane environment. + long timeout = System.currentTimeMillis(); + while(start == check.call() && System.currentTimeMillis() - timeout < 1000) + { + Uninterruptibles.sleepUninterruptibly(10, TimeUnit.MILLISECONDS); + } + return check.call() - start; + } + + public static class AuthTestDispatcher extends Dispatcher + { + public AuthTestDispatcher() + { + super(false); + } + + + + @Override + void processRequest(Channel channel, + Message.Request request, + FlushItemConverter forFlusher) + { + // noop + } + } +} From cccb9821b4e4558e16b2c4577c398f8864612687 Mon Sep 17 00:00:00 2001 From: Caleb Rackliffe <calebrackliffe@gmail.com> Date: Fri, 1 Nov 2024 16:40:49 -0500 Subject: [PATCH 020/225] Batch clusterings into single SAI partition post-filtering reads patch by Caleb Rackliffe; reviewed by Zhao Yang for CASSANDRA-19497 --- CHANGES.txt | 1 + .../index/sai/VectorQueryContext.java | 4 +- .../index/sai/plan/QueryController.java | 41 ++- .../plan/StorageAttachedIndexSearcher.java | 198 +++++++----- .../cassandra/index/sai/utils/PrimaryKey.java | 5 + .../utils/InsertionOrderedNavigableSet.java | 297 ++++++++++++++++++ 6 files changed, 456 insertions(+), 90 deletions(-) create mode 100644 src/java/org/apache/cassandra/utils/InsertionOrderedNavigableSet.java diff --git a/CHANGES.txt b/CHANGES.txt index 56128a46633b..36e021cf5695 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.3 + * Batch clusterings into single SAI partition post-filtering reads (CASSANDRA-19497) * Ban the usage of "var" instead of full types in the production code (CASSANDRA-20038) * Suppress CVE-2024-45772 from lucene-core-9.7.0.jar (CASSANDRA-20024) Merged from 4.1: diff --git a/src/java/org/apache/cassandra/index/sai/VectorQueryContext.java b/src/java/org/apache/cassandra/index/sai/VectorQueryContext.java index d65723b79bc5..a0533a0fe8c0 100644 --- a/src/java/org/apache/cassandra/index/sai/VectorQueryContext.java +++ b/src/java/org/apache/cassandra/index/sai/VectorQueryContext.java @@ -64,11 +64,11 @@ public int limit() return limit; } - public void recordShadowedPrimaryKey(PrimaryKey primaryKey) + public void recordShadowedPrimaryKeys(Set<PrimaryKey> keys) { if (shadowedPrimaryKeys == null) shadowedPrimaryKeys = new TreeSet<>(); - shadowedPrimaryKeys.add(primaryKey); + shadowedPrimaryKeys.addAll(keys); } // Returns true if the row ID will be included or false if the row ID will be shadowed diff --git a/src/java/org/apache/cassandra/index/sai/plan/QueryController.java b/src/java/org/apache/cassandra/index/sai/plan/QueryController.java index 91adc5d17e4b..b462fa3ad5f1 100644 --- a/src/java/org/apache/cassandra/index/sai/plan/QueryController.java +++ b/src/java/org/apache/cassandra/index/sai/plan/QueryController.java @@ -24,6 +24,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.NavigableSet; import java.util.stream.Collectors; import javax.annotation.Nullable; @@ -31,6 +32,7 @@ import com.google.common.collect.Lists; import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.db.Clustering; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.DataRange; import org.apache.cassandra.db.DecoratedKey; @@ -62,7 +64,7 @@ import org.apache.cassandra.net.ParamType; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.tracing.Tracing; -import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.InsertionOrderedNavigableSet; import org.apache.cassandra.utils.Pair; import org.apache.cassandra.utils.Throwables; @@ -82,6 +84,8 @@ public class QueryController private final PrimaryKey lastPrimaryKey; private final int orderChunkSize; + private final NavigableSet<Clustering<?>> nextClusterings; + public QueryController(ColumnFamilyStore cfs, ReadCommand command, RowFilter indexFilter, @@ -99,6 +103,7 @@ public QueryController(ColumnFamilyStore cfs, this.firstPrimaryKey = keyFactory.create(mergeRange.left.getToken()); this.lastPrimaryKey = keyFactory.create(mergeRange.right.getToken()); this.orderChunkSize = SAI_VECTOR_SEARCH_ORDER_CHUNK_SIZE.getInt(); + this.nextClusterings = new InsertionOrderedNavigableSet<>(cfs.metadata().comparator); } public PrimaryKey.Factory primaryKeyFactory() @@ -151,18 +156,18 @@ public boolean hasAnalyzer(RowFilter.Expression expression) return index != null && index.hasAnalyzer(); } - public UnfilteredRowIterator queryStorage(PrimaryKey key, ReadExecutionController executionController) + public UnfilteredRowIterator queryStorage(List<PrimaryKey> keys, ReadExecutionController executionController) { - if (key == null) - throw new IllegalArgumentException("non-null key required"); + if (keys.isEmpty()) + throw new IllegalArgumentException("At least one primary key is required!"); SinglePartitionReadCommand partition = SinglePartitionReadCommand.create(cfs.metadata(), command.nowInSec(), command.columnFilter(), RowFilter.none(), DataLimits.NONE, - key.partitionKey(), - makeFilter(key)); + keys.get(0).partitionKey(), + makeFilter(keys)); return partition.queryMemtableAndDisk(cfs, executionController); } @@ -409,21 +414,29 @@ private KeyRangeIterator createRowIdIterator(Pair<Expression, Collection<SSTable // Note: This method assumes that the selects method has already been called for the // key to avoid having to (potentially) call selects twice - private ClusteringIndexFilter makeFilter(PrimaryKey key) + private ClusteringIndexFilter makeFilter(List<PrimaryKey> keys) { - ClusteringIndexFilter clusteringIndexFilter = command.clusteringIndexFilter(key.partitionKey()); + PrimaryKey firstKey = keys.get(0); - assert cfs.metadata().comparator.size() == 0 && !key.kind().hasClustering || - cfs.metadata().comparator.size() > 0 && key.kind().hasClustering : - "PrimaryKey " + key + " clustering does not match table. There should be a clustering of size " + cfs.metadata().comparator.size(); + assert cfs.metadata().comparator.size() == 0 && !firstKey.kind().hasClustering || + cfs.metadata().comparator.size() > 0 && firstKey.kind().hasClustering : + "PrimaryKey " + firstKey + " clustering does not match table. There should be a clustering of size " + cfs.metadata().comparator.size(); + ClusteringIndexFilter clusteringIndexFilter = command.clusteringIndexFilter(firstKey.partitionKey()); + // If we have skinny partitions or the key is for a static row then we need to get the partition as // requested by the original query. - if (cfs.metadata().comparator.size() == 0 || key.kind() == PrimaryKey.Kind.STATIC) + if (cfs.metadata().comparator.size() == 0 || firstKey.kind() == PrimaryKey.Kind.STATIC) + { return clusteringIndexFilter; + } else - return new ClusteringIndexNamesFilter(FBUtilities.singleton(key.clustering(), cfs.metadata().comparator), - clusteringIndexFilter.isReversed()); + { + nextClusterings.clear(); + for (PrimaryKey key : keys) + nextClusterings.add(key.clustering()); + return new ClusteringIndexNamesFilter(nextClusterings, clusteringIndexFilter.isReversed()); + } } /** diff --git a/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexSearcher.java b/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexSearcher.java index 279f0d82e2de..b235c251ef8c 100644 --- a/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexSearcher.java +++ b/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexSearcher.java @@ -19,14 +19,19 @@ package org.apache.cassandra.index.sai.plan; import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; +import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; import javax.annotation.Nonnull; import javax.annotation.Nullable; +import io.netty.util.concurrent.FastThreadLocal; +import org.apache.cassandra.db.Clustering; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.DataRange; import org.apache.cassandra.db.DecoratedKey; @@ -57,11 +62,22 @@ public class StorageAttachedIndexSearcher implements Index.Searcher { + private static final int PARTITION_ROW_BATCH_SIZE = 100; + private final ReadCommand command; private final QueryController queryController; private final QueryContext queryContext; private final TableQueryMetrics tableQueryMetrics; + private static final FastThreadLocal<List<PrimaryKey>> nextKeys = new FastThreadLocal<>() + { + @Override + protected List<PrimaryKey> initialValue() + { + return new ArrayList<>(PARTITION_ROW_BATCH_SIZE); + } + }; + public StorageAttachedIndexSearcher(ColumnFamilyStore cfs, TableQueryMetrics tableQueryMetrics, ReadCommand command, @@ -129,11 +145,11 @@ private class ResultRetriever extends AbstractIterator<UnfilteredRowIterator> im private final ReadExecutionController executionController; private final PrimaryKey.Factory keyFactory; private final boolean topK; + private final int partitionRowBatchSize; private PrimaryKey lastKey; - private ResultRetriever(ReadExecutionController executionController, - boolean topK) + private ResultRetriever(ReadExecutionController executionController, boolean topK) { this.keyRanges = queryController.dataRanges().iterator(); this.currentKeyRange = keyRanges.next().keyRange(); @@ -144,6 +160,9 @@ private ResultRetriever(ReadExecutionController executionController, this.firstPrimaryKey = queryController.firstPrimaryKeyInRange(); this.lastPrimaryKey = queryController.lastPrimaryKeyInRange(); this.topK = topK; + + // Ensure we don't fetch larger batches than the provided LIMIT to avoid fetching keys we won't use: + this.partitionRowBatchSize = Math.min(PARTITION_ROW_BATCH_SIZE, command.limits().count()); } @Override @@ -164,33 +183,75 @@ public UnfilteredRowIterator computeNext() // saying this iterator must not return the same partition twice. skipToNextPartition(); - UnfilteredRowIterator iterator = nextRowIterator(this::nextSelectedKeyInRange); - return iterator != null - ? iteratePartition(iterator) - : endOfData(); + UnfilteredRowIterator iterator = nextRowIterator(this::nextSelectedKeysInRange); + return iterator != null ? iteratePartition(iterator) : endOfData(); } /** - * Tries to obtain a row iterator for one of the supplied keys by repeatedly calling + * Tries to obtain a row iterator for the supplied keys by repeatedly calling * {@link ResultRetriever#queryStorageAndFilter} until it gives a non-null result. - * The keySupplier should return the next key with every call to get() and - * null when there are no more keys to try. + * The keysSupplier should return the next batch of keys with every call to get() + * and null when there are no more keys to try. * * @return an iterator or null if all keys were tried with no success */ - private @Nullable UnfilteredRowIterator nextRowIterator(@Nonnull Supplier<PrimaryKey> keySupplier) + private @Nullable UnfilteredRowIterator nextRowIterator(@Nonnull Supplier<List<PrimaryKey>> keysSupplier) { UnfilteredRowIterator iterator = null; while (iterator == null) { - PrimaryKey key = keySupplier.get(); - if (key == null) + List<PrimaryKey> keys = keysSupplier.get(); + if (keys.isEmpty()) return null; - iterator = queryStorageAndFilter(key); + iterator = queryStorageAndFilter(keys); } return iterator; } + /** + * Retrieves the next batch of primary keys (i.e. up to {@link #partitionRowBatchSize} of them) that are + * contained by one of the query key ranges and selected by the {@link QueryController}. If the next key falls + * out of the current key range, it skips to the next key range, and so on. If no more keys accepted by + * the controller are available, and empty list is returned. + * + * @return a list of up to {@link #partitionRowBatchSize} primary keys + */ + private List<PrimaryKey> nextSelectedKeysInRange() + { + List<PrimaryKey> threadLocalNextKeys = nextKeys.get(); + threadLocalNextKeys.clear(); + PrimaryKey firstKey; + + do + { + firstKey = nextKeyInRange(); + + if (firstKey == null) + return Collections.emptyList(); + } + while (queryController.doesNotSelect(firstKey) || firstKey.equals(lastKey)); + + lastKey = firstKey; + threadLocalNextKeys.add(firstKey); + fillNextSelectedKeysInPartition(firstKey.partitionKey(), threadLocalNextKeys); + return threadLocalNextKeys; + } + + /** + * Retrieves the next batch of primary keys (i.e. up to {@link #partitionRowBatchSize} of them) that belong to + * the given partition and are selected by the query controller, advancing the underlying iterator only while + * the next key belongs to that partition. + * + * @return a list of up to {@link #partitionRowBatchSize} primary keys within the given partition + */ + private List<PrimaryKey> nextSelectedKeysInPartition(DecoratedKey partitionKey) + { + List<PrimaryKey> threadLocalNextKeys = nextKeys.get(); + threadLocalNextKeys.clear(); + fillNextSelectedKeysInPartition(partitionKey, threadLocalNextKeys); + return threadLocalNextKeys; + } + /** * Returns the next available key contained by one of the keyRanges. * If the next key falls out of the current key range, it skips to the next key range, and so on. @@ -219,48 +280,23 @@ public UnfilteredRowIterator computeNext() return key; } - /** - * Returns the next available key contained by one of the keyRanges and selected by the queryController. - * If the next key falls out of the current key range, it skips to the next key range, and so on. - * If no more keys acceptd by the controller are available, returns null. - */ - private @Nullable PrimaryKey nextSelectedKeyInRange() + private void fillNextSelectedKeysInPartition(DecoratedKey partitionKey, List<PrimaryKey> nextPrimaryKeys) { - PrimaryKey key; - do + while (resultKeyIterator.hasNext() + && resultKeyIterator.peek().partitionKey().equals(partitionKey) + && nextPrimaryKeys.size() < partitionRowBatchSize) { - key = nextKeyInRange(); - } - while (key != null && queryController.doesNotSelect(key)); - return key; - } + PrimaryKey key = nextKey(); - /** - * Retrieves the next primary key that belongs to the given partition and is selected by the query controller. - * The underlying key iterator is advanced only if the key belongs to the same partition. - * <p> - * Returns null if: - * <ul> - * <li>there are no more keys</li> - * <li>the next key is beyond the upper bound</li> - * <li>the next key belongs to a different partition</li> - * </ul> - * </p> - */ - private @Nullable PrimaryKey nextSelectedKeyInPartition(DecoratedKey partitionKey) - { - PrimaryKey key; - do - { - if (!resultKeyIterator.hasNext()) - return null; - if (!resultKeyIterator.peek().partitionKey().equals(partitionKey)) - return null; + if (key == null) + break; + + if (queryController.doesNotSelect(key) || key.equals(lastKey)) + continue; - key = nextKey(); + nextPrimaryKeys.add(key); + lastKey = key; } - while (key != null && queryController.doesNotSelect(key)); - return key; } /** @@ -315,8 +351,8 @@ private void skipToNextPartition() /** * Returns an iterator over the rows in the partition associated with the given iterator. * Initially, it retrieves the rows from the given iterator until it runs out of data. - * Then it iterates the primary keys obtained from the index until the end of the partition - * and lazily constructs new row itertors for each of the keys. At a given time, only one row iterator is open. + * Then it iterates the remaining primary keys obtained from the index in batches until the end of the + * partition, lazily constructing an itertor for each batch. Only one row iterator is open at a time. * <p> * The rows are retrieved in the order of primary keys provided by the underlying index. * The iterator is complete when the next key to be fetched belongs to different partition @@ -343,7 +379,7 @@ protected Unfiltered computeNext() while (!currentIter.hasNext()) { currentIter.close(); - currentIter = nextRowIterator(() -> nextSelectedKeyInPartition(partitionKey)); + currentIter = nextRowIterator(() -> nextSelectedKeysInPartition(partitionKey)); if (currentIter == null) return endOfData(); } @@ -359,21 +395,16 @@ public void close() }; } - private UnfilteredRowIterator queryStorageAndFilter(PrimaryKey key) + private UnfilteredRowIterator queryStorageAndFilter(List<PrimaryKey> keys) { - // Key reads are lazy, delayed all the way to this point. Skip if we've already seen this one: - if (key.equals(lastKey)) - return null; - - lastKey = key; long startTimeNanos = Clock.Global.nanoTime(); - try (UnfilteredRowIterator partition = queryController.queryStorage(key, executionController)) + try (UnfilteredRowIterator partition = queryController.queryStorage(keys, executionController)) { queryContext.partitionsRead++; queryContext.checkpoint(); - UnfilteredRowIterator filtered = applyIndexFilter(key, partition, filterTree); + UnfilteredRowIterator filtered = filterPartition(keys, partition, filterTree); // Note that we record the duration of the read after post-filtering, which actually // materializes the rows from disk. @@ -383,15 +414,14 @@ private UnfilteredRowIterator queryStorageAndFilter(PrimaryKey key) } } - private UnfilteredRowIterator applyIndexFilter(PrimaryKey key, UnfilteredRowIterator partition, FilterTree tree) + private UnfilteredRowIterator filterPartition(List<PrimaryKey> keys, UnfilteredRowIterator partition, FilterTree tree) { Row staticRow = partition.staticRow(); - List<Unfiltered> matchingRows = new ArrayList<>(); + DecoratedKey partitionKey = partition.partitionKey(); + List<Unfiltered> matches = new ArrayList<>(); boolean hasMatch = false; + Set<PrimaryKey> keysToShadow = topK ? new HashSet<>(keys) : Collections.emptySet(); - // We need to filter the partition rows before filtering on the static row. If this is done in the other - // order then we get incorrect results if we are filtering on a partition key index on a table with a - // composite partition key. while (partition.hasNext()) { Unfiltered unfiltered = partition.next(); @@ -400,28 +430,48 @@ private UnfilteredRowIterator applyIndexFilter(PrimaryKey key, UnfilteredRowIter { queryContext.rowsFiltered++; - if (tree.isSatisfiedBy(partition.partitionKey(), (Row) unfiltered, staticRow)) + if (tree.isSatisfiedBy(partitionKey, (Row) unfiltered, staticRow)) { - matchingRows.add(unfiltered); + matches.add(unfiltered); hasMatch = true; + + if (topK) + { + PrimaryKey shadowed = keyFactory.hasClusteringColumns() + ? keyFactory.create(partitionKey, ((Row) unfiltered).clustering()) + : keyFactory.create(partitionKey); + keysToShadow.remove(shadowed); + } } } } + // If any non-static rows match the filter, there should be no need to shadow the static primary key: + if (topK && hasMatch && keyFactory.hasClusteringColumns()) + keysToShadow.remove(keyFactory.create(partitionKey, Clustering.STATIC_CLUSTERING)); + + // We may not have any non-static row data to filter... if (!hasMatch) { queryContext.rowsFiltered++; - if (tree.isSatisfiedBy(key.partitionKey(), staticRow, staticRow)) + if (tree.isSatisfiedBy(partitionKey, staticRow, staticRow)) + { hasMatch = true; + + if (topK) + keysToShadow.clear(); + } } - if (!hasMatch) + if (topK && !keysToShadow.isEmpty()) { - // shadowed by expired TTL or row tombstone or range tombstone - if (topK) - queryContext.vectorContext().recordShadowedPrimaryKey(key); + // Record primary keys shadowed by expired TTLs, row tombstones, or range tombstones: + queryContext.vectorContext().recordShadowedPrimaryKeys(keysToShadow); + } + if (!hasMatch) + { // If there are no matches, return an empty partition. If reconciliation is required at the // coordinator, replica filtering protection may make a second round trip to complete its view // of the partition. @@ -429,7 +479,7 @@ private UnfilteredRowIterator applyIndexFilter(PrimaryKey key, UnfilteredRowIter } // Return all matches found, along with the static row... - return new PartitionIterator(partition, staticRow, matchingRows.iterator()); + return new PartitionIterator(partition, staticRow, matches.iterator()); } private class PartitionIterator extends AbstractUnfilteredRowIterator diff --git a/src/java/org/apache/cassandra/index/sai/utils/PrimaryKey.java b/src/java/org/apache/cassandra/index/sai/utils/PrimaryKey.java index c079a66c4c48..4e5081ad0c05 100644 --- a/src/java/org/apache/cassandra/index/sai/utils/PrimaryKey.java +++ b/src/java/org/apache/cassandra/index/sai/utils/PrimaryKey.java @@ -120,6 +120,11 @@ public PrimaryKey create(DecoratedKey partitionKey, Clustering<?> clustering) return clustering == Clustering.STATIC_CLUSTERING ? new StaticPrimaryKey(partitionKey) : new WidePrimaryKey(partitionKey, clustering); } + public boolean hasClusteringColumns() + { + return clusteringComparator != null && clusteringComparator.size() > 0; + } + /** * Create a {@link PrimaryKey} from a {@link ByteSource}. This should only be used with {@link ByteSource} instances * created by calls to {@link PrimaryKey#asComparableBytes(Version)}. diff --git a/src/java/org/apache/cassandra/utils/InsertionOrderedNavigableSet.java b/src/java/org/apache/cassandra/utils/InsertionOrderedNavigableSet.java new file mode 100644 index 000000000000..7d9d841046d4 --- /dev/null +++ b/src/java/org/apache/cassandra/utils/InsertionOrderedNavigableSet.java @@ -0,0 +1,297 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.NavigableSet; +import java.util.NoSuchElementException; +import java.util.Objects; +import java.util.SortedSet; + +import com.google.common.base.Preconditions; + +/** + * A {@link NavigableSet} that enforces in-order insertion of elements. This is helpful when we + * have an already-ordered collection with no duplicates and want constant time insertion. + * <p> + * Note: Not all methods of {@link NavigableSet} are implemented. + * + * @param <E> the type of elements maintained by this set + */ +public class InsertionOrderedNavigableSet<E> implements NavigableSet<E> +{ + private final ArrayList<E> elements; + private final Comparator<? super E> comparator; + + public InsertionOrderedNavigableSet(Comparator<? super E> comparator) + { + this.elements = new ArrayList<>(); + this.comparator = comparator; + } + + @Override + public E lower(E e) + { + throw new UnsupportedOperationException(); + } + + @Override + public E floor(E e) + { + throw new UnsupportedOperationException(); + } + + @Override + public E ceiling(E e) + { + throw new UnsupportedOperationException(); + } + + @Override + public E higher(E e) + { + throw new UnsupportedOperationException(); + } + + @Override + public E pollFirst() + { + if (isEmpty()) + return null; + + return elements.remove(0); + } + + @Override + public E pollLast() + { + if (isEmpty()) + return null; + + return elements.remove(size() - 1); + } + + @Override + public int size() + { + return elements.size(); + } + + @Override + public boolean isEmpty() + { + return elements.isEmpty(); + } + + @Override + public boolean contains(Object o) + { + return elements.contains(o); + } + + @Override + public Iterator<E> iterator() + { + return elements.iterator(); + } + + @Override + public Object[] toArray() + { + return elements.toArray(); + } + + @Override + public <T> T[] toArray(T[] a) + { + return elements.toArray(a); + } + + @Override + public boolean add(E e) + { + if (!isEmpty() && comparator.compare(e, last()) <= 0) + throw new IllegalStateException("Cannot add element " + e + " as it is not greater than the current last element " + last()); + + return elements.add(e); + } + + @Override + public boolean remove(Object o) + { + return elements.remove(o); + } + + @Override + public boolean containsAll(Collection<?> c) + { + return elements.containsAll(c); + } + + @Override + public boolean addAll(Collection<? extends E> c) + { + boolean modified = false; + for (E element : c) + if (add(element)) + modified = true; + return modified; + } + + @Override + public boolean retainAll(Collection<?> c) + { + return elements.retainAll(c); + } + + @Override + public boolean removeAll(Collection<?> c) + { + return elements.removeAll(c); + } + + @Override + public void clear() + { + elements.clear(); + } + + @Override + public NavigableSet<E> descendingSet() + { + throw new UnsupportedOperationException(); + } + + @Override + public Iterator<E> descendingIterator() + { + throw new UnsupportedOperationException(); + } + + @Override + public NavigableSet<E> subSet(E fromElement, boolean fromInclusive, E toElement, boolean toInclusive) + { + throw new UnsupportedOperationException(); + } + + @Override + public NavigableSet<E> headSet(E toElement, boolean inclusive) + { + Preconditions.checkNotNull(toElement); + + if (isEmpty()) + return Collections.emptyNavigableSet(); + + NavigableSet<E> head = new InsertionOrderedNavigableSet<>(comparator); + + for (E element : elements) + { + int comparison = comparator.compare(element, toElement); + if (comparison > 0 || comparison == 0 && !inclusive) + break; + + head.add(element); + } + + return head; + } + + @Override + public NavigableSet<E> tailSet(E fromElement, boolean inclusive) + { + Preconditions.checkNotNull(fromElement); + + if (isEmpty()) + return Collections.emptyNavigableSet(); + + NavigableSet<E> tail = new InsertionOrderedNavigableSet<>(comparator); + + for (E element : elements) + { + int comparison = comparator.compare(element, fromElement); + if (comparison < 0 || comparison == 0 && !inclusive) + continue; + + tail.add(element); + } + + return tail; + } + + @Override + public Comparator<? super E> comparator() + { + return comparator; + } + + @Override + public SortedSet<E> subSet(E fromElement, E toElement) + { + throw new UnsupportedOperationException(); + } + + @Override + public SortedSet<E> headSet(E toElement) + { + return headSet(toElement, false); + } + + @Override + public SortedSet<E> tailSet(E fromElement) + { + return tailSet(fromElement, true); + } + + @Override + public E first() + { + if (isEmpty()) + throw new NoSuchElementException(); + + return elements.get(0); + } + + @Override + public E last() + { + if (isEmpty()) + throw new NoSuchElementException(); + + return elements.get(size() - 1); + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + InsertionOrderedNavigableSet<?> that = (InsertionOrderedNavigableSet<?>) o; + return Objects.equals(elements, that.elements) && Objects.equals(comparator, that.comparator); + } + + @Override + public int hashCode() + { + return Objects.hash(elements, comparator); + } +} From 7ba691e3d3439cffe92418a204e2c0307bea59e5 Mon Sep 17 00:00:00 2001 From: Bereng <berenguerblasi@gmail.com> Date: Tue, 5 Nov 2024 10:12:51 +0100 Subject: [PATCH 021/225] Flaky PstmtPersistenceTest patch by Berenguer Blasi; reviewed by Stefan Miklosovic for CASSANDRA-20055 --- test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java b/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java index c0d499426243..e437e0f3c47f 100644 --- a/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java +++ b/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java @@ -155,7 +155,7 @@ public void testPstmtInvalidation() throws Throwable prepareStatement("INSERT INTO %s (key, val) VALUES (?, ?) USING TIMESTAMP " + cnt2, clientState); // each new prepared statement should have caused an eviction - Util.spinAssertEquals("eviction count didn't increase by the expected number", 10L, this::numberOfEvictedStatements, 5, TimeUnit.SECONDS); + Util.spinAssertEquals("eviction count didn't increase by the expected number", 10L, this::numberOfEvictedStatements, 90, TimeUnit.SECONDS); assertEquals("Number of statements in table and in cache don't match", numberOfStatementsInMemory(), numberOfStatementsOnDisk()); return; From 0b3c26ab0568656a9572bdf8cf58d8221e9ae550 Mon Sep 17 00:00:00 2001 From: Mohammad Aburadeh <mohmmad_aburadeh@hotmail.com> Date: Sun, 10 Nov 2024 14:18:09 +0000 Subject: [PATCH 022/225] Log client address when detecting unknown exception in client networking Patch by Mohammad Aburadeh, reviewed by brandonwilliams and smiklosovic for CASSANDRA-20072 --- src/java/org/apache/cassandra/transport/ExceptionHandlers.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/transport/ExceptionHandlers.java b/src/java/org/apache/cassandra/transport/ExceptionHandlers.java index e21d78e8f20b..43638ef2169e 100644 --- a/src/java/org/apache/cassandra/transport/ExceptionHandlers.java +++ b/src/java/org/apache/cassandra/transport/ExceptionHandlers.java @@ -113,7 +113,7 @@ else if (Throwables.anyCauseMatches(cause, t -> t instanceof SSLException)) else { ClientMetrics.instance.markUnknownException(); - logger.warn("Unknown exception in client networking", cause); + logger.warn("Unknown exception in client networking with peer {} {}", ctx.channel().remoteAddress(), cause.getMessage()); } } From b79f9dca873b551eee4bfae89d295e7b415a1366 Mon Sep 17 00:00:00 2001 From: Mohammad Aburadeh <mohmmad_aburadeh@hotmail.com> Date: Sun, 10 Nov 2024 14:24:45 +0000 Subject: [PATCH 023/225] Log client address when detecting unknown exception in client networking Patch by Mohammad Aburadeh, reviewed by brandonwilliams and smiklosovic for CASSANDRA-20072 --- src/java/org/apache/cassandra/transport/ExceptionHandlers.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/transport/ExceptionHandlers.java b/src/java/org/apache/cassandra/transport/ExceptionHandlers.java index 4156342fea4c..4f063924ead7 100644 --- a/src/java/org/apache/cassandra/transport/ExceptionHandlers.java +++ b/src/java/org/apache/cassandra/transport/ExceptionHandlers.java @@ -142,7 +142,7 @@ else if (Throwables.anyCauseMatches(cause, t -> t instanceof SSLException)) else { ClientMetrics.instance.markUnknownException(); - logger.warn("Unknown exception in client networking", cause); + logger.warn("Unknown exception in client networking with peer {} {}", clientAddress, cause.getMessage()); } } From 85048b20196e2c7fddc142de9095ef6e9f82e837 Mon Sep 17 00:00:00 2001 From: Sunil Ramchandra Pawar <pawar_sr@apple.com> Date: Tue, 29 Oct 2024 08:33:59 +0530 Subject: [PATCH 024/225] Add post-filtering support for the IN operator in SAI queries patch by Sunil Ramchandra Pawar; reviewed by Caleb Rackliffe and Ekaterina Dimitrova --- CHANGES.txt | 1 + .../sai/supported-query-operators-list.adoc | 4 +- .../cassandra/index/sai/plan/Expression.java | 27 +++++- .../index/sai/utils/IndexTermType.java | 3 +- .../test/sai/StrictFilteringTest.java | 10 +-- .../index/sai/cql/AllowFilteringTest.java | 86 +++++++++++++++++++ 6 files changed, 116 insertions(+), 15 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 360c17b6b63d..c5d63acb8347 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Add post-filtering support for the IN operator in SAI queries (CASSANDRA-20025) * Don’t finish ongoing decommission and move operations during startup (CASSANDRA-20040) * Nodetool reconfigure cms has correct return code when streaming fails (CASSANDRA-19972) * Reintroduce RestrictionSet#iterator() optimization around multi-column restrictions (CASSANDRA-20034) diff --git a/doc/modules/cassandra/partials/sai/supported-query-operators-list.adoc b/doc/modules/cassandra/partials/sai/supported-query-operators-list.adoc index 3c90a41703a2..a5ae372f612e 100644 --- a/doc/modules/cassandra/partials/sai/supported-query-operators-list.adoc +++ b/doc/modules/cassandra/partials/sai/supported-query-operators-list.adoc @@ -4,6 +4,6 @@ ifeval::["{evalproduct}" == "dse"] * Strings: `=`, `CONTAINS`, `CONTAINS KEY`, `AND` endif::[] ifeval::["{evalproduct}" != "dse"] -* Numerics: `=`, `<`, `>`, `<=`, `>=`, `AND`, `OR`, `IN` -* Strings: `=`, `CONTAINS`, `CONTAINS KEY`, `AND`, `OR`, `IN` +* Numerics: `=`, `<`, `>`, `<=`, `>=`, `AND` +* Strings: `=`, `CONTAINS`, `CONTAINS KEY`, `AND` endif::[] diff --git a/src/java/org/apache/cassandra/index/sai/plan/Expression.java b/src/java/org/apache/cassandra/index/sai/plan/Expression.java index 52a77b42d5a5..9823c3c0724b 100644 --- a/src/java/org/apache/cassandra/index/sai/plan/Expression.java +++ b/src/java/org/apache/cassandra/index/sai/plan/Expression.java @@ -79,7 +79,7 @@ public static boolean supportsOperator(Operator operator) public enum IndexOperator { - EQ, RANGE, CONTAINS_KEY, CONTAINS_VALUE, ANN; + EQ, RANGE, CONTAINS_KEY, CONTAINS_VALUE, ANN, IN; public static IndexOperator valueOf(Operator operator) { @@ -104,6 +104,9 @@ public static IndexOperator valueOf(Operator operator) case ANN: return ANN; + case IN: + return IN; + default: return null; } @@ -111,7 +114,7 @@ public static IndexOperator valueOf(Operator operator) public boolean isEquality() { - return this == EQ || this == CONTAINS_KEY || this == CONTAINS_VALUE; + return this == EQ || this == CONTAINS_KEY || this == CONTAINS_VALUE || this == IN; } public boolean isEqualityOrRange() @@ -169,6 +172,7 @@ public Expression add(Operator op, ByteBuffer value) case EQ: case CONTAINS: case CONTAINS_KEY: + case IN: lower = new Bound(value, indexTermType, true); upper = lower; operator = IndexOperator.valueOf(op); @@ -223,8 +227,8 @@ public Expression add(Operator op, ByteBuffer value) Value first = new Value(buffers.get(0), indexTermType); Value second = new Value(buffers.get(1), indexTermType); - // SimpleRestriction#addToRowFilter() ensures correct bounds ordering, but SAI enforces a non-arbitrary - // ordering between IPv4 and IPv6 addresses, so correction may still be necessary. + // SimpleRestriction#addToRowFilter() ensures correct bounds ordering, but SAI enforces a non-arbitrary + // ordering between IPv4 and IPv6 addresses, so correction may still be necessary. boolean outOfOrder = indexTermType.compare(first.encoded, second.encoded) > 0; lower = new Bound(outOfOrder ? second : first, true); upper = new Bound(outOfOrder ? first : second, true); @@ -236,6 +240,7 @@ public Expression add(Operator op, ByteBuffer value) lower = new Bound(value, indexTermType, true); upper = lower; break; + default: throw new IllegalArgumentException("Index does not support the " + op + " operator"); } @@ -275,6 +280,9 @@ public boolean isSatisfiedBy(ByteBuffer columnValue) if (operator == IndexOperator.EQ || operator == IndexOperator.CONTAINS_KEY || operator == IndexOperator.CONTAINS_VALUE) return cmp == 0; + if (operator == IndexOperator.IN) + return termMatches(value.raw, lower.value.raw); + if (cmp > 0 || (cmp == 0 && !lowerInclusive)) return false; } @@ -335,6 +343,17 @@ private boolean termMatches(ByteBuffer term, ByteBuffer requestedValue) case RANGE: isMatch = isLowerSatisfiedBy(term) && isUpperSatisfiedBy(term); break; + case IN: + ListType<?> type = ListType.getInstance(indexTermType.columnMetadata().type, true); + List<? extends ByteBuffer> buffers = type.unpack(requestedValue); + for (ByteBuffer value : buffers) + { + if (indexTermType.compare(term, value) == 0) + { + return true; + } + } + break; } return isMatch; } diff --git a/src/java/org/apache/cassandra/index/sai/utils/IndexTermType.java b/src/java/org/apache/cassandra/index/sai/utils/IndexTermType.java index a558d5ee82aa..c1dfad04a5ac 100644 --- a/src/java/org/apache/cassandra/index/sai/utils/IndexTermType.java +++ b/src/java/org/apache/cassandra/index/sai/utils/IndexTermType.java @@ -566,7 +566,8 @@ public boolean supports(Operator operator) operator == Operator.LIKE_CONTAINS || operator == Operator.LIKE_PREFIX || operator == Operator.LIKE_MATCHES || - operator == Operator.LIKE_SUFFIX) return false; + operator == Operator.LIKE_SUFFIX || + operator == Operator.IN) return false; // ANN is only supported against vectors, and vector indexes only support ANN if (operator == Operator.ANN) diff --git a/test/distributed/org/apache/cassandra/distributed/test/sai/StrictFilteringTest.java b/test/distributed/org/apache/cassandra/distributed/test/sai/StrictFilteringTest.java index 301336f8629d..83bfdcf94882 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/sai/StrictFilteringTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/sai/StrictFilteringTest.java @@ -21,17 +21,14 @@ import java.io.IOException; import java.util.Iterator; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; -import org.apache.cassandra.cql3.Operator; import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.api.ConsistencyLevel; import org.apache.cassandra.distributed.shared.AssertUtils; import org.apache.cassandra.distributed.test.TestBaseImpl; -import org.apache.cassandra.index.sai.plan.StorageAttachedIndexQueryPlan; import static org.apache.cassandra.distributed.api.Feature.GOSSIP; import static org.apache.cassandra.distributed.api.Feature.NETWORK; @@ -55,11 +52,10 @@ public static void setUpCluster() throws IOException } @Test - public void shouldRejectNonStrictIN() + public void shouldPostFilterNonStrictIN() { CLUSTER.schemaChange(withKeyspace("CREATE TABLE %s.reject_in (k int PRIMARY KEY, a int, b int) WITH read_repair = 'NONE'")); CLUSTER.schemaChange(withKeyspace("CREATE INDEX ON %s.reject_in(a) USING 'sai'")); - CLUSTER.schemaChange(withKeyspace("CREATE INDEX ON %s.reject_in(b) USING 'sai'")); SAIUtil.waitForIndexQueryable(CLUSTER, KEYSPACE); // insert an unrepaired row @@ -68,9 +64,7 @@ public void shouldRejectNonStrictIN() String select = withKeyspace("SELECT * FROM %s.reject_in WHERE a = 1 AND b IN (2, 3) ALLOW FILTERING"); - // This should fail, as strict filtering is not allowed: - Assertions.assertThatThrownBy(() -> CLUSTER.coordinator(1).execute(select, ConsistencyLevel.ALL)) - .hasMessageContaining(String.format(StorageAttachedIndexQueryPlan.UNSUPPORTED_NON_STRICT_OPERATOR, Operator.IN)); + assertRows(CLUSTER.coordinator(1).execute(select, ConsistencyLevel.ALL), row(0,1,2)); // Repair fixes the split row, although we still only allow the query when reconciliation is not required: CLUSTER.get(1).nodetoolResult("repair", KEYSPACE).asserts().success(); diff --git a/test/unit/org/apache/cassandra/index/sai/cql/AllowFilteringTest.java b/test/unit/org/apache/cassandra/index/sai/cql/AllowFilteringTest.java index f80d07eeb774..7a9198a7009a 100644 --- a/test/unit/org/apache/cassandra/index/sai/cql/AllowFilteringTest.java +++ b/test/unit/org/apache/cassandra/index/sai/cql/AllowFilteringTest.java @@ -296,6 +296,91 @@ public void testAllowFilteringOnClusteringAndRegularColumns() throws Throwable test("SELECT * FROM %s WHERE v1=0 AND v2=0 AND k1=0 AND k2=0 AND (c1, c2, c3, c4) = (0, 0, 0, 0) AND v3=0", true); } + @Test + public void testAllowFilteringTextWithINClause () + { + createTable("CREATE TABLE %S (k1 TEXT, k2 TEXT, k3 TEXT, PRIMARY KEY(k1))"); + createIndex("CREATE INDEX ON %s(K2) USING 'sai'"); + + execute("INSERT INTO %s (k1,k2,k3) VALUES ('s1','s11','s111')"); + execute("INSERT INTO %s (k1,k2,k3) VALUES ('s2','s11','s11')"); + execute("INSERT INTO %s (k1,k2,k3) VALUES ('s3','s22','s111')"); + execute("INSERT INTO %s (k1,k2,k3) VALUES ('s4','s22','s111')"); + execute("INSERT INTO %s (k1,k2,k3) VALUES ('s5','s31','s111')"); + + assertRowCount(execute("SELECT * FROM %s WHERE k2='s11' AND k3 IN ('s11','s111') ALLOW FILTERING"),2); + assertRowCount(execute("SELECT * FROM %s WHERE k2='s22' AND k3 IN ('s111','s111') ALLOW FILTERING"), 2); + assertRowCount(execute("SELECT * FROM %s WHERE k2='s22' AND k3 IN ('s','s1') ALLOW FILTERING"), 0); + // To test if an IN clause without an AND condition does not create a query plan and works as expected. + assertRowCount(execute("SELECT * FROM %s WHERE k2 IN ('s11','s22') ALLOW FILTERING"), 4); + + } + + @Test + public void testAllowFilteringIntWithINClause () + { + createTable("CREATE TABLE %S (k1 text, k2 int, k3 int, PRIMARY KEY(k1))"); + createIndex("CREATE INDEX ON %s(K2) USING 'sai'"); + + execute("insert into %s (k1,k2,k3) values ('s1',11,1)"); + execute("insert into %s (k1,k2,k3) values ('s2',11,11)"); + execute("insert into %s (k1,k2,k3) values ('s3',11,111)"); + execute("insert into %s (k1,k2,k3) values ('s4',22,1)"); + execute("insert into %s (k1,k2,k3) values ('s5',22,11)"); + execute("insert into %s (k1,k2,k3) values ('s6',22,111)"); + + assertRowCount(execute("SELECT * FROM %s WHERE k2=11 AND k3 IN (1,11,111) ALLOW FILTERING"),3); + assertRowCount(execute("SELECT * FROM %s WHERE k2=22 AND k3 IN (1,11,111) ALLOW FILTERING"),3); + assertRowCount(execute("SELECT * FROM %s WHERE k2=22 AND k3 IN (101,102) ALLOW FILTERING"),0); + // To test if an IN clause without an AND condition does not create a query plan and works as expected. + assertRowCount(execute("SELECT * FROM %s WHERE k2 IN (11,22) ALLOW FILTERING"), 6); + + } + + @Test + public void testAllowFilteringBigIntWithINClause () + { + createTable("CREATE TABLE %S (k1 text, k2 bigint, k3 bigint, PRIMARY KEY(k1))"); + createIndex("CREATE INDEX ON %s(K2) USING 'sai'"); + + execute("insert into %s (k1, k2, k3) values ('s1', 1001, 100)"); + execute("insert into %s (k1, k2, k3) values ('s2', 1001, 200)"); + execute("insert into %s (k1, k2, k3) values ('s3', 1001, 300)"); + execute("insert into %s (k1, k2, k3) values ('s4', 2002, 100)"); + execute("insert into %s (k1, k2, k3) values ('s5', 2002, 200)"); + execute("insert into %s (k1, k2, k3) values ('s6', 2002, 300)"); + + assertRowCount(execute("SELECT * FROM %s WHERE k2=1001 AND k3 IN (100, 200, 300) ALLOW FILTERING"), 3); + assertRowCount(execute("SELECT * FROM %s WHERE k2=2002 AND k3 IN (100, 200, 300) ALLOW FILTERING"), 3); + assertRowCount(execute("SELECT * FROM %s WHERE k2=2002 AND k3 IN (101, 201, 301) ALLOW FILTERING"), 0); + // To test if an IN clause without an AND condition does not create a query plan and works as expected. + assertRowCount(execute("SELECT * FROM %s WHERE k2 IN (1001,2002) ALLOW FILTERING"), 6); + + } + + @Test + public void testAllowFilteringBigDecimalWithINClause() + { + createTable("CREATE TABLE %S (k1 text, k2 decimal, k3 decimal, PRIMARY KEY(k1))"); + createIndex("CREATE INDEX ON %s(K2) USING 'sai'"); + + execute("insert into %s (k1, k2, k3) values ('s1', 1.1, 1.11)"); + execute("insert into %s (k1, k2, k3) values ('s2', 1.1, 1.12)"); + execute("insert into %s (k1, k2, k3) values ('s3', 1.1, 1.13)"); + execute("insert into %s (k1, k2, k3) values ('s4', 2.2, 1.11)"); + execute("insert into %s (k1, k2, k3) values ('s5', 2.2, 1.12)"); + execute("insert into %s (k1, k2, k3) values ('s6', 2.2, 1.13)"); + + assertRowCount(execute("SELECT * FROM %s WHERE k2=1.1 AND k3 IN (1.11, 1.12, 1.13) ALLOW FILTERING"), 3); + assertRowCount(execute("SELECT * FROM %s WHERE k2=2.2 AND k3 IN (1.11, 1.12, 1.13) ALLOW FILTERING"), 3); + assertRowCount(execute("SELECT * FROM %s WHERE k2=2.2 AND k3 IN (1.21, 1.22, 1.13) ALLOW FILTERING"), 1); + assertRowCount(execute("SELECT * FROM %s WHERE k2=2.2 AND k3 IN (1.21, 1.22, 1.23) ALLOW FILTERING"), 0); + // To test if an IN clause without an AND condition does not create a query plan and works as expected. + assertRowCount(execute("SELECT * FROM %s WHERE k2 IN (1.1,2.2) ALLOW FILTERING"), 6); + } + + + private void test(String query, boolean requiresAllowFiltering) throws Throwable { if (requiresAllowFiltering) @@ -305,4 +390,5 @@ private void test(String query, boolean requiresAllowFiltering) throws Throwable assertNotNull(execute(query + " ALLOW FILTERING")); } + } From 39b42ccd2facde366d6d3fdbcc1aa221a5d0c14d Mon Sep 17 00:00:00 2001 From: Brandon Williams <brandonwilliams@apache.org> Date: Wed, 13 Nov 2024 12:39:36 -0600 Subject: [PATCH 025/225] Update RMI object marshalling restrictions Patch by brandonwilliams; reviewed by paulo for CASSANDRA-20076 --- .../cassandra/utils/JMXServerUtils.java | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/src/java/org/apache/cassandra/utils/JMXServerUtils.java b/src/java/org/apache/cassandra/utils/JMXServerUtils.java index 49ff5a07d278..bbe09e86363a 100644 --- a/src/java/org/apache/cassandra/utils/JMXServerUtils.java +++ b/src/java/org/apache/cassandra/utils/JMXServerUtils.java @@ -56,6 +56,7 @@ import org.apache.cassandra.auth.jmx.AuthenticationProxy; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_VERSION; import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_ACCESS_FILE; import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_AUTHENTICATE; import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_PASSWORD_FILE; @@ -65,6 +66,7 @@ import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_PROTOCOLS; import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_NEED_CLIENT_AUTH; + public class JMXServerUtils { private static final Logger logger = LoggerFactory.getLogger(JMXServerUtils.class); @@ -100,6 +102,8 @@ public static JMXConnectorServer createJMXServer(int port, String hostname, bool // via a JAAS configuration entry, or one which delegates to the standard file based authenticator. // Authn is disabled if com.sun.management.jmxremote.authenticate=false env.putAll(configureJmxAuthentication()); + // Secure credential passing to avoid deserialization attacks + env.putAll(configureSecureCredentials()); // Configure authz - if a custom proxy class is specified an instance will be returned. // If not, but a location for the standard access file is set in system properties, the @@ -151,6 +155,19 @@ public static JMXConnectorServer createJMXServer(int port, boolean local) throws return createJMXServer(port, null, local); } + private static Map<String, Object> configureSecureCredentials() + { + Map<String, Object> env = new HashMap<>(); + if (JAVA_VERSION.getString().startsWith("1.8")) + { + env.put("jmx.remote.rmi.server.credential.types", + new String[] { String[].class.getName(), String.class.getName() }); + } else { + env.put("jmx.remote.rmi.server.credentials.filter.pattern", String.class.getName() + ";!*"); + } + return env; + } + private static Map<String, Object> configureJmxAuthentication() { Map<String, Object> env = new HashMap<>(); @@ -184,8 +201,6 @@ private static Map<String, Object> configureJmxAuthentication() env.put(JMXConnectorServer.AUTHENTICATOR, new JMXPluggableAuthenticatorWrapper(env)); } - env.put("jmx.remote.rmi.server.credential.types", - new String[] { String[].class.getName(), String.class.getName() }); return env; } From 88a5ea76cf39fb3d3a527eb1053fa06aa16dfaf2 Mon Sep 17 00:00:00 2001 From: Brandon Williams <brandonwilliams@apache.org> Date: Wed, 13 Nov 2024 12:44:20 -0600 Subject: [PATCH 026/225] Update RMI object marshalling restrictions Patch by brandonwilliams; reviewed by paulo for CASSANDRA-20076 --- .../org/apache/cassandra/utils/JMXServerUtils.java | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/src/java/org/apache/cassandra/utils/JMXServerUtils.java b/src/java/org/apache/cassandra/utils/JMXServerUtils.java index 3f26d59e6e10..78c8ced8d2fb 100644 --- a/src/java/org/apache/cassandra/utils/JMXServerUtils.java +++ b/src/java/org/apache/cassandra/utils/JMXServerUtils.java @@ -104,6 +104,8 @@ public static JMXConnectorServer createJMXServer(int port, String hostname, bool // via a JAAS configuration entry, or one which delegates to the standard file based authenticator. // Authn is disabled if com.sun.management.jmxremote.authenticate=false env.putAll(configureJmxAuthentication()); + // Secure credential passing to avoid deserialization attacks + env.putAll(configureSecureCredentials()); // Configure authz - if a custom proxy class is specified an instance will be returned. // If not, but a location for the standard access file is set in system properties, the @@ -154,6 +156,13 @@ public static JMXConnectorServer createJMXServer(int port, boolean local) throws return createJMXServer(port, null, local); } + private static Map<String, Object> configureSecureCredentials() + { + Map<String, Object> env = new HashMap<>(); + env.put("jmx.remote.rmi.server.credentials.filter.pattern", String.class.getName() + ";!*"); + return env; + } + private static Map<String, Object> configureJmxAuthentication() { Map<String, Object> env = new HashMap<>(); @@ -187,8 +196,6 @@ private static Map<String, Object> configureJmxAuthentication() env.put(JMXConnectorServer.AUTHENTICATOR, new JMXPluggableAuthenticatorWrapper(env)); } - env.put("jmx.remote.rmi.server.credential.types", - new String[] { String[].class.getName(), String.class.getName() }); return env; } From b88fe80910deccc4eb648038a92729e31976e291 Mon Sep 17 00:00:00 2001 From: Brandon Williams <brandonwilliams@apache.org> Date: Wed, 13 Nov 2024 12:50:57 -0600 Subject: [PATCH 027/225] Prepare debian changelog for 4.0.15 --- debian/changelog | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/debian/changelog b/debian/changelog index 4ba959aa21a4..92c3bb3271d3 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,3 +1,9 @@ +cassandra (4.0.15) unstable; urgency=medium + + * New release + + -- Brandon Williams <brandonwilliams@apache.org> Wed, 13 Nov 2024 12:49:46 -0600 + cassandra (4.0.14) unstable; urgency=medium * New release From 77cc238deec25a15a40970f23442ebb83d495ab9 Mon Sep 17 00:00:00 2001 From: Brandon Williams <brandonwilliams@apache.org> Date: Wed, 13 Nov 2024 14:53:26 -0600 Subject: [PATCH 028/225] Prepare debian changelog for 4.1.8 --- debian/changelog | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/debian/changelog b/debian/changelog index ab1ae98ccf1c..4900a48f7b95 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,3 +1,9 @@ +cassandra (4.1.8) unstable; urgency=medium + + * New release + + -- Brandon Williams <brandonwilliams@apache.org> Wed, 13 Nov 2024 14:51:52 -0600 + cassandra (4.1.7) unstable; urgency=medium * New release From b9a5d15deec81295113cdc703d5acda423d8a688 Mon Sep 17 00:00:00 2001 From: Brandon Williams <brandonwilliams@apache.org> Date: Wed, 13 Nov 2024 15:34:39 -0600 Subject: [PATCH 029/225] Prepare debian changelog for 5.0.3 --- debian/changelog | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/debian/changelog b/debian/changelog index c713b9a10f7e..d6f3c47d085d 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,3 +1,9 @@ +cassandra (5.0.3) unstable; urgency=medium + + * New release + + -- Brandon Williams <brandonwilliams@apache.org> Wed, 13 Nov 2024 15:34:07 -0600 + cassandra (5.0.2) unstable; urgency=medium * New release From d0c984fa32b81f3fe8058daf0fac2a3081775c9b Mon Sep 17 00:00:00 2001 From: Dmitry Konstantinov <netudima@gmail.com> Date: Tue, 12 Nov 2024 14:11:27 +0000 Subject: [PATCH 030/225] Change the resolution of AbstractCommitLogService#lastSyncedAt to nanos to be aligned with later comparisons patch by Dmitry Konstantinov; reviewed by Branimir Lambov, Stefan Miklosovic for CASSANDRA-20074 --- CHANGES.txt | 1 + .../apache/cassandra/db/commitlog/AbstractCommitLogService.java | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 0eb7ed5f7e49..2fc6e48a9466 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0.15 + * Change the resolution of AbstractCommitLogService#lastSyncedAt to nanos to be aligned with later comparisons (CASSANDRA-20074) * Backport of CASSANDRA-17812: Rate-limit new client connection auth setup to avoid overwhelming bcrypt (CASSANDRA-20057) * Support UDTs and vectors as clustering keys in descending order (CASSANDRA-20050) * Fix CQL in snapshot's schema which did not contained UDTs used as reverse clustering columns (CASSANDRA-20036) diff --git a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java index a65ef00a11bd..8b3acdae5eea 100644 --- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java +++ b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java @@ -45,7 +45,7 @@ public abstract class AbstractCommitLogService private volatile boolean shutdown = false; // all Allocations written before this time will be synced - protected volatile long lastSyncedAt = System.currentTimeMillis(); + protected volatile long lastSyncedAt = MonotonicClock.preciseTime.now(); // counts of total written, and pending, log messages private final AtomicLong written = new AtomicLong(0); From f3c29e08fae5369051d950de5b78e2367b75a17c Mon Sep 17 00:00:00 2001 From: Bereng <berenguerblasi@gmail.com> Date: Thu, 14 Nov 2024 09:02:02 +0100 Subject: [PATCH 031/225] Flaky PstmtPersistenceTest take 2 patch by Berenguer Blasi; reviewed by Stefan Miklosovic for CASSANDRA-20055 --- .../apache/cassandra/cql3/PstmtPersistenceTest.java | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java b/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java index e437e0f3c47f..df4a554e2e23 100644 --- a/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java +++ b/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java @@ -21,9 +21,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.concurrent.TimeUnit; -import org.apache.cassandra.Util; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -148,16 +146,6 @@ public void testPstmtInvalidation() throws Throwable if (numberOfEvictedStatements() > 0) { - assertEquals("Number of statements in table and in cache don't match", numberOfStatementsInMemory(), numberOfStatementsOnDisk()); - - // prepare a more statements to trigger more evictions - for (int cnt2 = 1; cnt2 < 10; cnt2++) - prepareStatement("INSERT INTO %s (key, val) VALUES (?, ?) USING TIMESTAMP " + cnt2, clientState); - - // each new prepared statement should have caused an eviction - Util.spinAssertEquals("eviction count didn't increase by the expected number", 10L, this::numberOfEvictedStatements, 90, TimeUnit.SECONDS); - assertEquals("Number of statements in table and in cache don't match", numberOfStatementsInMemory(), numberOfStatementsOnDisk()); - return; } } From a231b787e5654a5e2db0510dd9c0ae10a6926d86 Mon Sep 17 00:00:00 2001 From: Brandon Williams <brandonwilliams@apache.org> Date: Mon, 18 Nov 2024 09:00:59 -0600 Subject: [PATCH 032/225] increment version to 4.0.16 --- CHANGES.txt | 3 +++ build.xml | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 0eb7ed5f7e49..916bc5a740b0 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,3 +1,6 @@ +4.0.16 + + 4.0.15 * Backport of CASSANDRA-17812: Rate-limit new client connection auth setup to avoid overwhelming bcrypt (CASSANDRA-20057) * Support UDTs and vectors as clustering keys in descending order (CASSANDRA-20050) diff --git a/build.xml b/build.xml index 58e28aebce04..5d9b961f51fb 100644 --- a/build.xml +++ b/build.xml @@ -34,7 +34,7 @@ <property name="debuglevel" value="source,lines,vars"/> <!-- default version and SCM information --> - <property name="base.version" value="4.0.15"/> + <property name="base.version" value="4.0.16"/> <property name="scm.connection" value="scm:https://gitbox.apache.org/repos/asf/cassandra.git"/> <property name="scm.developerConnection" value="scm:https://gitbox.apache.org/repos/asf/cassandra.git"/> <property name="scm.url" value="https://gitbox.apache.org/repos/asf?p=cassandra.git"/> From bdfb3fb9d6338cf03d5ea4e21162756a8a66d91e Mon Sep 17 00:00:00 2001 From: Brandon Williams <brandonwilliams@apache.org> Date: Mon, 18 Nov 2024 09:00:59 -0600 Subject: [PATCH 033/225] increment version to 4.0.16 --- CHANGES.txt | 3 +++ build.xml | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 2fc6e48a9466..5c35bd232516 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,3 +1,6 @@ +4.0.16 + + 4.0.15 * Change the resolution of AbstractCommitLogService#lastSyncedAt to nanos to be aligned with later comparisons (CASSANDRA-20074) * Backport of CASSANDRA-17812: Rate-limit new client connection auth setup to avoid overwhelming bcrypt (CASSANDRA-20057) diff --git a/build.xml b/build.xml index 58e28aebce04..5d9b961f51fb 100644 --- a/build.xml +++ b/build.xml @@ -34,7 +34,7 @@ <property name="debuglevel" value="source,lines,vars"/> <!-- default version and SCM information --> - <property name="base.version" value="4.0.15"/> + <property name="base.version" value="4.0.16"/> <property name="scm.connection" value="scm:https://gitbox.apache.org/repos/asf/cassandra.git"/> <property name="scm.developerConnection" value="scm:https://gitbox.apache.org/repos/asf/cassandra.git"/> <property name="scm.url" value="https://gitbox.apache.org/repos/asf?p=cassandra.git"/> From b6eb5890da38642fc7af7d39c83f7ec01f33d78f Mon Sep 17 00:00:00 2001 From: Marcus Eriksson <marcuse@apache.org> Date: Mon, 18 Nov 2024 11:19:04 -0700 Subject: [PATCH 034/225] Optionally prevent tombstone purging during repair patch by Marcus Eriksson, Abe Ratnofsky; reviewed by Jon Meredith for CASSANDRA-20071 --- CHANGES.txt | 1 + .../repair/CassandraTableRepairManager.java | 4 +- .../repair/CassandraValidationIterator.java | 5 +- .../cassandra/repair/AbstractRepairTask.java | 1 + .../apache/cassandra/repair/RepairJob.java | 2 +- .../repair/RepairMessageVerbHandler.java | 4 +- .../cassandra/repair/RepairSession.java | 3 + .../cassandra/repair/TableRepairManager.java | 2 +- .../cassandra/repair/ValidationManager.java | 2 +- .../cassandra/repair/ValidationTask.java | 6 +- .../apache/cassandra/repair/Validator.java | 14 ++-- .../repair/messages/RepairOption.java | 16 +++- .../repair/messages/ValidationRequest.java | 11 ++- .../service/ActiveRepairService.java | 3 +- .../cassandra/tools/nodetool/Repair.java | 4 + .../5.1/service.ValidationRequest.bin | Bin 74 -> 75 bytes .../test/repair/NoTombstonePurgingTest.java | 72 ++++++++++++++++++ .../simulator/cluster/OnInstanceRepair.java | 2 +- ...onManagerGetSSTablesForValidationTest.java | 6 +- .../cassandra/repair/RepairJobTest.java | 6 +- ...epairMessageVerbHandlerOutOfRangeTest.java | 2 +- .../cassandra/repair/RepairSessionTest.java | 3 +- .../cassandra/repair/ValidationTaskTest.java | 2 +- .../cassandra/repair/ValidatorTest.java | 6 +- .../RepairMessageSerializationsTest.java | 6 +- .../cassandra/service/SerializationsTest.java | 3 +- 26 files changed, 150 insertions(+), 36 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/test/repair/NoTombstonePurgingTest.java diff --git a/CHANGES.txt b/CHANGES.txt index b5ba4dc2675a..91d48619427e 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Optionally prevent tombstone purging during repair (CASSANDRA-20071) * Add post-filtering support for the IN operator in SAI queries (CASSANDRA-20025) * Don’t finish ongoing decommission and move operations during startup (CASSANDRA-20040) * Nodetool reconfigure cms has correct return code when streaming fails (CASSANDRA-19972) diff --git a/src/java/org/apache/cassandra/db/repair/CassandraTableRepairManager.java b/src/java/org/apache/cassandra/db/repair/CassandraTableRepairManager.java index 4e54d6ee7782..24e79d24544b 100644 --- a/src/java/org/apache/cassandra/db/repair/CassandraTableRepairManager.java +++ b/src/java/org/apache/cassandra/db/repair/CassandraTableRepairManager.java @@ -56,9 +56,9 @@ public CassandraTableRepairManager(ColumnFamilyStore cfs, SharedContext ctx) } @Override - public ValidationPartitionIterator getValidationIterator(Collection<Range<Token>> ranges, TimeUUID parentId, TimeUUID sessionID, boolean isIncremental, long nowInSec, TopPartitionTracker.Collector topPartitionCollector) throws IOException, NoSuchRepairSessionException + public ValidationPartitionIterator getValidationIterator(Collection<Range<Token>> ranges, TimeUUID parentId, TimeUUID sessionID, boolean isIncremental, long nowInSec, boolean dontPurgeTombstones, TopPartitionTracker.Collector topPartitionCollector) throws IOException, NoSuchRepairSessionException { - return new CassandraValidationIterator(cfs, ctx, ranges, parentId, sessionID, isIncremental, nowInSec, topPartitionCollector); + return new CassandraValidationIterator(cfs, ctx, ranges, parentId, sessionID, isIncremental, nowInSec, dontPurgeTombstones, topPartitionCollector); } @Override diff --git a/src/java/org/apache/cassandra/db/repair/CassandraValidationIterator.java b/src/java/org/apache/cassandra/db/repair/CassandraValidationIterator.java index a31a7038b069..05408e91752d 100644 --- a/src/java/org/apache/cassandra/db/repair/CassandraValidationIterator.java +++ b/src/java/org/apache/cassandra/db/repair/CassandraValidationIterator.java @@ -174,7 +174,7 @@ else if (isIncremental) private final long estimatedPartitions; private final Map<Range<Token>, Long> rangePartitionCounts; - public CassandraValidationIterator(ColumnFamilyStore cfs, SharedContext ctx, Collection<Range<Token>> ranges, TimeUUID parentId, TimeUUID sessionID, boolean isIncremental, long nowInSec, TopPartitionTracker.Collector topPartitionCollector) throws IOException, NoSuchRepairSessionException + public CassandraValidationIterator(ColumnFamilyStore cfs, SharedContext ctx, Collection<Range<Token>> ranges, TimeUUID parentId, TimeUUID sessionID, boolean isIncremental, long nowInSec, boolean dontPurgeTombstones, TopPartitionTracker.Collector topPartitionCollector) throws IOException, NoSuchRepairSessionException { this.cfs = cfs; this.ctx = ctx; @@ -219,7 +219,8 @@ public CassandraValidationIterator(ColumnFamilyStore cfs, SharedContext ctx, Col cfs.getKeyspaceName(), cfs.getTableName()); - controller = new ValidationCompactionController(cfs, getDefaultGcBefore(cfs, nowInSec)); + long gcBefore = dontPurgeTombstones ? Long.MIN_VALUE : getDefaultGcBefore(cfs, nowInSec); + controller = new ValidationCompactionController(cfs, gcBefore); scanners = cfs.getCompactionStrategyManager().getScanners(sstables, ranges); ci = new ValidationCompactionIterator(scanners.scanners, controller, nowInSec, CompactionManager.instance.active, topPartitionCollector); diff --git a/src/java/org/apache/cassandra/repair/AbstractRepairTask.java b/src/java/org/apache/cassandra/repair/AbstractRepairTask.java index 94cc3545c210..f27e72deb177 100644 --- a/src/java/org/apache/cassandra/repair/AbstractRepairTask.java +++ b/src/java/org/apache/cassandra/repair/AbstractRepairTask.java @@ -76,6 +76,7 @@ private List<RepairSession> submitRepairSessions(TimeUUID parentSession, options.optimiseStreams(), options.repairPaxos(), options.paxosOnly(), + options.dontPurgeTombstones(), executor, validationScheduler, cfnames); diff --git a/src/java/org/apache/cassandra/repair/RepairJob.java b/src/java/org/apache/cassandra/repair/RepairJob.java index 424b69acd1c1..63b7b96ec518 100644 --- a/src/java/org/apache/cassandra/repair/RepairJob.java +++ b/src/java/org/apache/cassandra/repair/RepairJob.java @@ -614,7 +614,7 @@ public void onFailure(Throwable t) {} private ValidationTask newValidationTask(InetAddressAndPort endpoint, long nowInSec) { - ValidationTask task = new ValidationTask(session.ctx, desc, endpoint, nowInSec, session.previewKind); + ValidationTask task = new ValidationTask(session.ctx, desc, endpoint, nowInSec, session.previewKind, session.dontPurgeTombstones); validationTasks.add(task); return task; } diff --git a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java index ca823faa5a6f..f7771260195a 100644 --- a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java +++ b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java @@ -243,7 +243,9 @@ public void doVerb(final Message<RepairMessage> message) sendAck(message); Validator validator = new Validator(ctx, vState, validationRequest.nowInSec, - isIncremental(desc.parentSessionId), previewKind); + isIncremental(desc.parentSessionId), + previewKind, + validationRequest.dontPurgeTombstones); ctx.validationManager().submitValidation(store, validator); } catch (Throwable t) diff --git a/src/java/org/apache/cassandra/repair/RepairSession.java b/src/java/org/apache/cassandra/repair/RepairSession.java index f329bf477994..92d56390fe3a 100644 --- a/src/java/org/apache/cassandra/repair/RepairSession.java +++ b/src/java/org/apache/cassandra/repair/RepairSession.java @@ -121,6 +121,7 @@ public class RepairSession extends AsyncFuture<RepairSessionResult> implements I public final PreviewKind previewKind; public final boolean repairPaxos; public final boolean paxosOnly; + public final boolean dontPurgeTombstones; private final AtomicBoolean isFailed = new AtomicBoolean(false); @@ -161,6 +162,7 @@ public RepairSession(SharedContext ctx, boolean optimiseStreams, boolean repairPaxos, boolean paxosOnly, + boolean dontPurgeTombstones, String... cfnames) { this.ctx = ctx; @@ -174,6 +176,7 @@ public RepairSession(SharedContext ctx, this.previewKind = previewKind; this.pullRepair = pullRepair; this.optimiseStreams = optimiseStreams; + this.dontPurgeTombstones = dontPurgeTombstones; this.taskExecutor = new SafeExecutor(createExecutor(ctx)); } diff --git a/src/java/org/apache/cassandra/repair/TableRepairManager.java b/src/java/org/apache/cassandra/repair/TableRepairManager.java index 99ccff0714c0..f5bee38a0cb5 100644 --- a/src/java/org/apache/cassandra/repair/TableRepairManager.java +++ b/src/java/org/apache/cassandra/repair/TableRepairManager.java @@ -38,7 +38,7 @@ public interface TableRepairManager * data previously isolated for repair with the given parentId. nowInSec should determine whether tombstones should * be purged or not. */ - ValidationPartitionIterator getValidationIterator(Collection<Range<Token>> ranges, TimeUUID parentId, TimeUUID sessionID, boolean isIncremental, long nowInSec, TopPartitionTracker.Collector topPartitionCollector) throws IOException, NoSuchRepairSessionException; + ValidationPartitionIterator getValidationIterator(Collection<Range<Token>> ranges, TimeUUID parentId, TimeUUID sessionID, boolean isIncremental, long nowInSec, boolean dontPurgeTombstones, TopPartitionTracker.Collector topPartitionCollector) throws IOException, NoSuchRepairSessionException; /** * Begin execution of the given validation callable. Which thread pool a validation should run in is an implementation detail. diff --git a/src/java/org/apache/cassandra/repair/ValidationManager.java b/src/java/org/apache/cassandra/repair/ValidationManager.java index e3598cd38f87..ca7ad3a68eea 100644 --- a/src/java/org/apache/cassandra/repair/ValidationManager.java +++ b/src/java/org/apache/cassandra/repair/ValidationManager.java @@ -90,7 +90,7 @@ private static MerkleTrees createMerkleTrees(ValidationPartitionIterator validat private static ValidationPartitionIterator getValidationIterator(TableRepairManager repairManager, Validator validator, TopPartitionTracker.Collector topPartitionCollector) throws IOException, NoSuchRepairSessionException { RepairJobDesc desc = validator.desc; - return repairManager.getValidationIterator(desc.ranges, desc.parentSessionId, desc.sessionId, validator.isIncremental, validator.nowInSec, topPartitionCollector); + return repairManager.getValidationIterator(desc.ranges, desc.parentSessionId, desc.sessionId, validator.isIncremental, validator.nowInSec, validator.dontPurgeTombstones, topPartitionCollector); } /** diff --git a/src/java/org/apache/cassandra/repair/ValidationTask.java b/src/java/org/apache/cassandra/repair/ValidationTask.java index 322e07cd2df8..445e3880a22d 100644 --- a/src/java/org/apache/cassandra/repair/ValidationTask.java +++ b/src/java/org/apache/cassandra/repair/ValidationTask.java @@ -40,15 +40,17 @@ public class ValidationTask extends AsyncFuture<TreeResponse> implements Runnabl private final InetAddressAndPort endpoint; private final long nowInSec; private final PreviewKind previewKind; + private final boolean dontPurgeTombstones; private final SharedContext ctx; - public ValidationTask(SharedContext ctx, RepairJobDesc desc, InetAddressAndPort endpoint, long nowInSec, PreviewKind previewKind) + public ValidationTask(SharedContext ctx, RepairJobDesc desc, InetAddressAndPort endpoint, long nowInSec, PreviewKind previewKind, boolean dontPurgeTombstones) { this.ctx = ctx; this.desc = desc; this.endpoint = endpoint; this.nowInSec = nowInSec; this.previewKind = previewKind; + this.dontPurgeTombstones = dontPurgeTombstones; } /** @@ -57,7 +59,7 @@ public ValidationTask(SharedContext ctx, RepairJobDesc desc, InetAddressAndPort public void run() { RepairMessage.sendMessageWithFailureCB(ctx, notDone(this), - new ValidationRequest(desc, nowInSec), + new ValidationRequest(desc, nowInSec, dontPurgeTombstones), VALIDATION_REQ, endpoint, this::tryFailure); diff --git a/src/java/org/apache/cassandra/repair/Validator.java b/src/java/org/apache/cassandra/repair/Validator.java index d8ba929de62c..c5152aee1344 100644 --- a/src/java/org/apache/cassandra/repair/Validator.java +++ b/src/java/org/apache/cassandra/repair/Validator.java @@ -82,23 +82,24 @@ public class Validator implements Runnable private final PreviewKind previewKind; public final ValidationState state; public TopPartitionTracker.Collector topPartitionCollector; + public final boolean dontPurgeTombstones; public Validator(ValidationState state, long nowInSec, PreviewKind previewKind) { - this(SharedContext.Global.instance, state, nowInSec, false, false, previewKind); + this(SharedContext.Global.instance, state, nowInSec, false, false, previewKind, false); } - public Validator(SharedContext ctx, ValidationState state, long nowInSec, boolean isIncremental, PreviewKind previewKind) + public Validator(SharedContext ctx, ValidationState state, long nowInSec, boolean isIncremental, PreviewKind previewKind, boolean dontPurgeTombstones) { - this(ctx, state, nowInSec, false, isIncremental, previewKind); + this(ctx, state, nowInSec, false, isIncremental, previewKind, dontPurgeTombstones); } - public Validator(ValidationState state, long nowInSec, boolean isIncremental, PreviewKind previewKind) + public Validator(ValidationState state, long nowInSec, boolean isIncremental, PreviewKind previewKind, boolean dontPurgeTombstones) { - this(SharedContext.Global.instance, state, nowInSec, false, isIncremental, previewKind); + this(SharedContext.Global.instance, state, nowInSec, false, isIncremental, previewKind, dontPurgeTombstones); } - public Validator(SharedContext ctx, ValidationState state, long nowInSec, boolean evenTreeDistribution, boolean isIncremental, PreviewKind previewKind) + public Validator(SharedContext ctx, ValidationState state, long nowInSec, boolean evenTreeDistribution, boolean isIncremental, PreviewKind previewKind, boolean dontPurgeTombstones) { this.ctx = ctx; this.state = state; @@ -107,6 +108,7 @@ public Validator(SharedContext ctx, ValidationState state, long nowInSec, boolea this.nowInSec = nowInSec; this.isIncremental = isIncremental; this.previewKind = previewKind; + this.dontPurgeTombstones = dontPurgeTombstones; validated = 0; range = null; ranges = null; diff --git a/src/java/org/apache/cassandra/repair/messages/RepairOption.java b/src/java/org/apache/cassandra/repair/messages/RepairOption.java index 03097da0779d..bc9231dcc142 100644 --- a/src/java/org/apache/cassandra/repair/messages/RepairOption.java +++ b/src/java/org/apache/cassandra/repair/messages/RepairOption.java @@ -54,6 +54,8 @@ public class RepairOption public static final String IGNORE_UNREPLICATED_KS = "ignoreUnreplicatedKeyspaces"; public static final String REPAIR_PAXOS_KEY = "repairPaxos"; public static final String PAXOS_ONLY_KEY = "paxosOnly"; + public static final String NO_TOMBSTONE_PURGING = "nopurge"; + // we don't want to push nodes too much for repair public static final int MAX_JOB_THREADS = 4; @@ -185,6 +187,7 @@ public static RepairOption parse(Map<String, String> options, IPartitioner parti boolean ignoreUnreplicatedKeyspaces = Boolean.parseBoolean(options.get(IGNORE_UNREPLICATED_KS)); boolean repairPaxos = Boolean.parseBoolean(options.get(REPAIR_PAXOS_KEY)); boolean paxosOnly = Boolean.parseBoolean(options.get(PAXOS_ONLY_KEY)); + boolean dontPurgeTombstones = Boolean.parseBoolean(options.get(NO_TOMBSTONE_PURGING)); if (previewKind != PreviewKind.NONE) { @@ -209,7 +212,7 @@ public static RepairOption parse(Map<String, String> options, IPartitioner parti boolean asymmetricSyncing = Boolean.parseBoolean(options.get(OPTIMISE_STREAMS_KEY)); - RepairOption option = new RepairOption(parallelism, primaryRange, incremental, trace, jobThreads, ranges, !ranges.isEmpty(), pullRepair, force, previewKind, asymmetricSyncing, ignoreUnreplicatedKeyspaces, repairPaxos, paxosOnly); + RepairOption option = new RepairOption(parallelism, primaryRange, incremental, trace, jobThreads, ranges, !ranges.isEmpty(), pullRepair, force, previewKind, asymmetricSyncing, ignoreUnreplicatedKeyspaces, repairPaxos, paxosOnly, dontPurgeTombstones); // data centers String dataCentersStr = options.get(DATACENTERS_KEY); @@ -291,13 +294,14 @@ else if (ranges.isEmpty()) private final boolean ignoreUnreplicatedKeyspaces; private final boolean repairPaxos; private final boolean paxosOnly; + private final boolean dontPurgeTombstones; private final Collection<String> columnFamilies = new HashSet<>(); private final Collection<String> dataCenters = new HashSet<>(); private final Collection<String> hosts = new HashSet<>(); private final Collection<Range<Token>> ranges = new HashSet<>(); - public RepairOption(RepairParallelism parallelism, boolean primaryRange, boolean incremental, boolean trace, int jobThreads, Collection<Range<Token>> ranges, boolean isSubrangeRepair, boolean pullRepair, boolean forceRepair, PreviewKind previewKind, boolean optimiseStreams, boolean ignoreUnreplicatedKeyspaces, boolean repairPaxos, boolean paxosOnly) + public RepairOption(RepairParallelism parallelism, boolean primaryRange, boolean incremental, boolean trace, int jobThreads, Collection<Range<Token>> ranges, boolean isSubrangeRepair, boolean pullRepair, boolean forceRepair, PreviewKind previewKind, boolean optimiseStreams, boolean ignoreUnreplicatedKeyspaces, boolean repairPaxos, boolean paxosOnly, boolean dontPurgeTombstones) { this.parallelism = parallelism; @@ -314,6 +318,7 @@ public RepairOption(RepairParallelism parallelism, boolean primaryRange, boolean this.ignoreUnreplicatedKeyspaces = ignoreUnreplicatedKeyspaces; this.repairPaxos = repairPaxos; this.paxosOnly = paxosOnly; + this.dontPurgeTombstones = dontPurgeTombstones; } public RepairParallelism getParallelism() @@ -429,6 +434,11 @@ public boolean paxosOnly() return paxosOnly; } + public boolean dontPurgeTombstones() + { + return dontPurgeTombstones; + } + @Override public String toString() { @@ -448,6 +458,7 @@ public String toString() ", ignore unreplicated keyspaces: "+ ignoreUnreplicatedKeyspaces + ", repairPaxos: " + repairPaxos + ", paxosOnly: " + paxosOnly + + ", dontPurgeTombstones: " + dontPurgeTombstones + ')'; } @@ -470,6 +481,7 @@ public Map<String, String> asMap() options.put(OPTIMISE_STREAMS_KEY, Boolean.toString(optimiseStreams)); options.put(REPAIR_PAXOS_KEY, Boolean.toString(repairPaxos)); options.put(PAXOS_ONLY_KEY, Boolean.toString(paxosOnly)); + options.put(NO_TOMBSTONE_PURGING, Boolean.toString(dontPurgeTombstones)); return options; } } diff --git a/src/java/org/apache/cassandra/repair/messages/ValidationRequest.java b/src/java/org/apache/cassandra/repair/messages/ValidationRequest.java index 1e651a96d2d7..5c6550fac77c 100644 --- a/src/java/org/apache/cassandra/repair/messages/ValidationRequest.java +++ b/src/java/org/apache/cassandra/repair/messages/ValidationRequest.java @@ -35,11 +35,13 @@ public class ValidationRequest extends RepairMessage { public final long nowInSec; + public final boolean dontPurgeTombstones; - public ValidationRequest(RepairJobDesc desc, long nowInSec) + public ValidationRequest(RepairJobDesc desc, long nowInSec, boolean dontPurgeTombstones) { super(desc); this.nowInSec = nowInSec; + this.dontPurgeTombstones = dontPurgeTombstones; } @Override @@ -47,6 +49,7 @@ public String toString() { return "ValidationRequest{" + "nowInSec=" + nowInSec + + ", dontPurgeTombstones" + dontPurgeTombstones + "} " + super.toString(); } @@ -72,19 +75,23 @@ public void serialize(ValidationRequest message, DataOutputPlus out, int version { RepairJobDesc.serializer.serialize(message.desc, out, version); out.writeInt(version >= MessagingService.VERSION_50 ? CassandraUInt.fromLong(message.nowInSec) : (int) message.nowInSec); + if (version >= MessagingService.VERSION_51) + out.writeBoolean(message.dontPurgeTombstones); } public ValidationRequest deserialize(DataInputPlus dis, int version) throws IOException { RepairJobDesc desc = RepairJobDesc.serializer.deserialize(dis, version); long nowInsec = version >= MessagingService.VERSION_50 ? CassandraUInt.toLong(dis.readInt()) : dis.readInt(); - return new ValidationRequest(desc, nowInsec); + boolean dontPurgeTombstones = version >= MessagingService.VERSION_51 ? dis.readBoolean() : false; + return new ValidationRequest(desc, nowInsec, dontPurgeTombstones); } public long serializedSize(ValidationRequest message, int version) { long size = RepairJobDesc.serializer.serializedSize(message.desc, version); size += TypeSizes.INT_SIZE; + size += version >= MessagingService.VERSION_51 ? TypeSizes.sizeof(message.dontPurgeTombstones) : 0; return size; } }; diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java index 9682c8e9594d..b5fbc48d4374 100644 --- a/src/java/org/apache/cassandra/service/ActiveRepairService.java +++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java @@ -454,6 +454,7 @@ public RepairSession submitRepairSession(TimeUUID parentRepairSession, boolean optimiseStreams, boolean repairPaxos, boolean paxosOnly, + boolean dontPurgeTombstones, ExecutorPlus executor, Scheduler validationScheduler, String... cfnames) @@ -469,7 +470,7 @@ public RepairSession submitRepairSession(TimeUUID parentRepairSession, final RepairSession session = new RepairSession(ctx, validationScheduler, parentRepairSession, range, keyspace, parallelismDegree, isIncremental, pullRepair, - previewKind, optimiseStreams, repairPaxos, paxosOnly, cfnames); + previewKind, optimiseStreams, repairPaxos, paxosOnly, dontPurgeTombstones, cfnames); repairs.getIfPresent(parentRepairSession).register(session.state); sessions.put(session.getId(), session); diff --git a/src/java/org/apache/cassandra/tools/nodetool/Repair.java b/src/java/org/apache/cassandra/tools/nodetool/Repair.java index 35832408301c..c66992acc9a8 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/Repair.java +++ b/src/java/org/apache/cassandra/tools/nodetool/Repair.java @@ -108,6 +108,9 @@ public class Repair extends NodeToolCmd @Option(title = "ignore_unreplicated_keyspaces", name = {"-iuk","--ignore-unreplicated-keyspaces"}, description = "Use --ignore-unreplicated-keyspaces to ignore keyspaces which are not replicated, otherwise the repair will fail") private boolean ignoreUnreplicatedKeyspaces = false; + @Option(title = "no_purge", name = {"--include-gcgs-expired-tombstones"}, description = "Do not apply gc grace seconds to purge any tombstones. Only useful in rare recovery scenarios, never regular operations.") + private boolean dontPurgeTombstones = false; + private PreviewKind getPreviewKind() { if (validate) @@ -186,6 +189,7 @@ else if (dcParallel) options.put(RepairOption.IGNORE_UNREPLICATED_KS, Boolean.toString(ignoreUnreplicatedKeyspaces)); options.put(RepairOption.REPAIR_PAXOS_KEY, Boolean.toString(!skipPaxos && getPreviewKind() == PreviewKind.NONE)); options.put(RepairOption.PAXOS_ONLY_KEY, Boolean.toString(paxosOnly && getPreviewKind() == PreviewKind.NONE)); + options.put(RepairOption.NO_TOMBSTONE_PURGING, Boolean.toString(dontPurgeTombstones)); if (!startToken.isEmpty() || !endToken.isEmpty()) { diff --git a/test/data/serialization/5.1/service.ValidationRequest.bin b/test/data/serialization/5.1/service.ValidationRequest.bin index 04c492a8a1efcd9d17df4d49dced710d9acd7701..2a2aea7183af251104b6ab018059baef1f43efbe 100644 GIT binary patch delta 6 NcmebBp5VpE2ml6J0dfEU delta 4 LcmebFn&1Th19$;& diff --git a/test/distributed/org/apache/cassandra/distributed/test/repair/NoTombstonePurgingTest.java b/test/distributed/org/apache/cassandra/distributed/test/repair/NoTombstonePurgingTest.java new file mode 100644 index 000000000000..adcb2b3e2cbe --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/repair/NoTombstonePurgingTest.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.repair; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; + +import com.google.common.util.concurrent.Uninterruptibles; +import org.junit.Test; + +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.test.TestBaseImpl; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class NoTombstonePurgingTest extends TestBaseImpl +{ + @Test + public void testNp() throws IOException + { + testHelper((cluster) -> { + // full repair, with -np, tombstone gets streamed + cluster.get(1).nodetoolResult("repair", "--include-gcgs-expired-tombstones", "--full", KEYSPACE, "tbl"); + }); + } + + private void testHelper(Consumer<Cluster> repair) throws IOException + { + try (Cluster cluster = init(Cluster.build(2) + .withConfig(c -> c.set("hinted_handoff_enabled", false) + .with(Feature.values())) + .start())) + { + cluster.schemaChange(withKeyspace("create table %s.tbl (id int primary key) with gc_grace_seconds = 1 and compaction={'class':'SizeTieredCompactionStrategy', 'enabled':false}")); + cluster.get(1).executeInternal(withKeyspace("delete from %s.tbl where id = 5")); + cluster.get(1).flush(KEYSPACE); + Uninterruptibles.sleepUninterruptibly(2, TimeUnit.SECONDS); //gcgs expiry + + // incremental repair, the tombstone is purgeable, will not get included in MT calculation + cluster.get(1).nodetoolResult("repair", KEYSPACE, "tbl"); + cluster.get(2).runOnInstance(() -> assertTrue(Keyspace.open(KEYSPACE).getColumnFamilyStore("tbl").getLiveSSTables().isEmpty())); + + // full repair, tombstone still gets purged + cluster.get(1).nodetoolResult("repair", "--full", KEYSPACE, "tbl"); + cluster.get(2).runOnInstance(() -> assertTrue(Keyspace.open(KEYSPACE).getColumnFamilyStore("tbl").getLiveSSTables().isEmpty())); + + repair.accept(cluster); + + cluster.get(2).runOnInstance(() -> assertFalse(Keyspace.open(KEYSPACE).getColumnFamilyStore("tbl").getLiveSSTables().isEmpty())); + } + } +} diff --git a/test/simulator/main/org/apache/cassandra/simulator/cluster/OnInstanceRepair.java b/test/simulator/main/org/apache/cassandra/simulator/cluster/OnInstanceRepair.java index f24e639ac300..46edfb392649 100644 --- a/test/simulator/main/org/apache/cassandra/simulator/cluster/OnInstanceRepair.java +++ b/test/simulator/main/org/apache/cassandra/simulator/cluster/OnInstanceRepair.java @@ -97,7 +97,7 @@ private static void invokeRepair(String keyspaceName, boolean repairPaxos, boole { Collection<Range<Token>> ranges = rangesSupplier.call(); // no need to wait for completion, as we track all task submissions and message exchanges, and ensure they finish before continuing to next action - StorageService.instance.repair(keyspaceName, new RepairOption(RepairParallelism.SEQUENTIAL, isPrimaryRangeOnly, false, false, 1, ranges, false, false, force, PreviewKind.NONE, false, true, repairPaxos, repairOnlyPaxos), singletonList((tag, event) -> { + StorageService.instance.repair(keyspaceName, new RepairOption(RepairParallelism.SEQUENTIAL, isPrimaryRangeOnly, false, false, 1, ranges, false, false, force, PreviewKind.NONE, false, true, repairPaxos, repairOnlyPaxos, false), singletonList((tag, event) -> { if (event.getType() == ProgressEventType.COMPLETE) listener.run(); })); diff --git a/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java b/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java index 011db44d099e..4c7932888b39 100644 --- a/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java +++ b/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java @@ -143,7 +143,7 @@ public void consistentRepair() throws Exception modifySSTables(); // get sstables for repair - Validator validator = new Validator(new ValidationState(Clock.Global.clock(), desc, coordinator), FBUtilities.nowInSeconds(), true, PreviewKind.NONE); + Validator validator = new Validator(new ValidationState(Clock.Global.clock(), desc, coordinator), FBUtilities.nowInSeconds(), true, PreviewKind.NONE, false); Set<SSTableReader> sstables = Sets.newHashSet(getSSTablesToValidate(cfs, SharedContext.Global.instance, validator.desc.ranges, validator.desc.parentSessionId, validator.isIncremental)); Assert.assertNotNull(sstables); Assert.assertEquals(1, sstables.size()); @@ -158,7 +158,7 @@ public void legacyIncrementalRepair() throws Exception modifySSTables(); // get sstables for repair - Validator validator = new Validator(new ValidationState(Clock.Global.clock(), desc, coordinator), FBUtilities.nowInSeconds(), false, PreviewKind.NONE); + Validator validator = new Validator(new ValidationState(Clock.Global.clock(), desc, coordinator), FBUtilities.nowInSeconds(), false, PreviewKind.NONE, false); Set<SSTableReader> sstables = Sets.newHashSet(getSSTablesToValidate(cfs, SharedContext.Global.instance, validator.desc.ranges, validator.desc.parentSessionId, validator.isIncremental)); Assert.assertNotNull(sstables); Assert.assertEquals(2, sstables.size()); @@ -174,7 +174,7 @@ public void fullRepair() throws Exception modifySSTables(); // get sstables for repair - Validator validator = new Validator(new ValidationState(Clock.Global.clock(), desc, coordinator), FBUtilities.nowInSeconds(), false, PreviewKind.NONE); + Validator validator = new Validator(new ValidationState(Clock.Global.clock(), desc, coordinator), FBUtilities.nowInSeconds(), false, PreviewKind.NONE, false); Set<SSTableReader> sstables = Sets.newHashSet(getSSTablesToValidate(cfs, SharedContext.Global.instance, validator.desc.ranges, validator.desc.parentSessionId, validator.isIncremental)); Assert.assertNotNull(sstables); Assert.assertEquals(3, sstables.size()); diff --git a/test/unit/org/apache/cassandra/repair/RepairJobTest.java b/test/unit/org/apache/cassandra/repair/RepairJobTest.java index 36c17855e309..ea32bd750b88 100644 --- a/test/unit/org/apache/cassandra/repair/RepairJobTest.java +++ b/test/unit/org/apache/cassandra/repair/RepairJobTest.java @@ -126,11 +126,11 @@ private static class MeasureableRepairSession extends RepairSession public MeasureableRepairSession(TimeUUID parentRepairSession, CommonRange commonRange, String keyspace, RepairParallelism parallelismDegree, boolean isIncremental, boolean pullRepair, PreviewKind previewKind, boolean optimiseStreams, boolean repairPaxos, boolean paxosOnly, - String... cfnames) + boolean dontPurgeTombstones, String... cfnames) { super(SharedContext.Global.instance, new Scheduler.NoopScheduler(), parentRepairSession, commonRange, keyspace, parallelismDegree, isIncremental, pullRepair, - previewKind, optimiseStreams, repairPaxos, paxosOnly, cfnames); + previewKind, optimiseStreams, repairPaxos, paxosOnly, dontPurgeTombstones, cfnames); } @Override @@ -196,7 +196,7 @@ public void setup() this.session = new MeasureableRepairSession(parentRepairSession, new CommonRange(neighbors, emptySet(), FULL_RANGE), KEYSPACE, SEQUENTIAL, false, false, - NONE, false, true, false, CF); + NONE, false, true, false, false, CF); this.job = new RepairJob(session, CF); this.sessionJobDesc = new RepairJobDesc(session.state.parentRepairSession, session.getId(), diff --git a/test/unit/org/apache/cassandra/repair/RepairMessageVerbHandlerOutOfRangeTest.java b/test/unit/org/apache/cassandra/repair/RepairMessageVerbHandlerOutOfRangeTest.java index 6b74d0febd0c..4e7f9ba40094 100644 --- a/test/unit/org/apache/cassandra/repair/RepairMessageVerbHandlerOutOfRangeTest.java +++ b/test/unit/org/apache/cassandra/repair/RepairMessageVerbHandlerOutOfRangeTest.java @@ -262,7 +262,7 @@ private static ValidationRequest validationMsg(Range<Token> range) true, PreviewKind.NONE); return new ValidationRequest(new RepairJobDesc(parentId, uuid(), KEYSPACE, TABLE, Collections.singleton(range)), - randomInt()); + randomInt(), false); } public static TimeUUID uuid() diff --git a/test/unit/org/apache/cassandra/repair/RepairSessionTest.java b/test/unit/org/apache/cassandra/repair/RepairSessionTest.java index f4d177870e33..470a2efc538e 100644 --- a/test/unit/org/apache/cassandra/repair/RepairSessionTest.java +++ b/test/unit/org/apache/cassandra/repair/RepairSessionTest.java @@ -67,7 +67,8 @@ public void testConviction() throws Exception new CommonRange(endpoints, Collections.emptySet(), Arrays.asList(repairRange)), "Keyspace1", RepairParallelism.SEQUENTIAL, false, false, - PreviewKind.NONE, false, false, false, "Standard1"); + PreviewKind.NONE, false, false, false, false, + "Standard1"); // perform convict session.convict(remote, Double.MAX_VALUE); diff --git a/test/unit/org/apache/cassandra/repair/ValidationTaskTest.java b/test/unit/org/apache/cassandra/repair/ValidationTaskTest.java index 5639d4105cbe..e7f325de52ff 100644 --- a/test/unit/org/apache/cassandra/repair/ValidationTaskTest.java +++ b/test/unit/org/apache/cassandra/repair/ValidationTaskTest.java @@ -79,6 +79,6 @@ public void shouldReleaseTreesOnAbort() throws Exception private ValidationTask createTask() throws UnknownHostException { InetAddressAndPort addressAndPort = InetAddressAndPort.getByName("127.0.0.1"); RepairJobDesc desc = new RepairJobDesc(nextTimeUUID(), nextTimeUUID(), UUID.randomUUID().toString(), UUID.randomUUID().toString(), null); - return new ValidationTask(SharedContext.Global.instance, desc, addressAndPort, 0, PreviewKind.NONE); + return new ValidationTask(SharedContext.Global.instance, desc, addressAndPort, 0, PreviewKind.NONE, false); } } diff --git a/test/unit/org/apache/cassandra/repair/ValidatorTest.java b/test/unit/org/apache/cassandra/repair/ValidatorTest.java index 7d0317f2c1bd..9ea956b85c10 100644 --- a/test/unit/org/apache/cassandra/repair/ValidatorTest.java +++ b/test/unit/org/apache/cassandra/repair/ValidatorTest.java @@ -208,7 +208,7 @@ public void simpleValidationTest(int n) throws Exception false, PreviewKind.NONE); final CompletableFuture<Message> outgoingMessageSink = registerOutgoingMessageSink(); - Validator validator = new Validator(SharedContext.Global.instance, new ValidationState(Clock.Global.clock(), desc, host), 0, true, false, PreviewKind.NONE); + Validator validator = new Validator(SharedContext.Global.instance, new ValidationState(Clock.Global.clock(), desc, host), 0, true, false, PreviewKind.NONE, false); ValidationManager.instance.submitValidation(cfs, validator); Message message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS); @@ -265,7 +265,7 @@ public void testSizeLimiting() throws Exception false, PreviewKind.NONE); final CompletableFuture<Message> outgoingMessageSink = registerOutgoingMessageSink(); - Validator validator = new Validator(SharedContext.Global.instance, new ValidationState(Clock.Global.clock(), desc, host), 0, true, false, PreviewKind.NONE); + Validator validator = new Validator(SharedContext.Global.instance, new ValidationState(Clock.Global.clock(), desc, host), 0, true, false, PreviewKind.NONE, false); ValidationManager.instance.submitValidation(cfs, validator); Message message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS); @@ -327,7 +327,7 @@ public void testRangeSplittingTreeSizeLimit() throws Exception false, PreviewKind.NONE); final CompletableFuture<Message> outgoingMessageSink = registerOutgoingMessageSink(); - Validator validator = new Validator(SharedContext.Global.instance, new ValidationState(Clock.Global.clock(), desc, host), 0, true, false, PreviewKind.NONE); + Validator validator = new Validator(SharedContext.Global.instance, new ValidationState(Clock.Global.clock(), desc, host), 0, true, false, PreviewKind.NONE, false); ValidationManager.instance.submitValidation(cfs, validator); Message message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS); diff --git a/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java b/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java index 9886076d702e..1657ceff4870 100644 --- a/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java +++ b/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java @@ -87,9 +87,13 @@ public static void after() public void validationRequestMessage() throws IOException { RepairJobDesc jobDesc = buildRepairJobDesc(); - ValidationRequest msg = new ValidationRequest(jobDesc, GC_BEFORE); + ValidationRequest msg = new ValidationRequest(jobDesc, GC_BEFORE, false); ValidationRequest deserialized = serializeRoundTrip(msg, ValidationRequest.serializer); Assert.assertEquals(jobDesc, deserialized.desc); + + msg = new ValidationRequest(jobDesc, GC_BEFORE, true); + deserialized = serializeRoundTrip(msg, ValidationRequest.serializer); + Assert.assertEquals(jobDesc, deserialized.desc); } private RepairJobDesc buildRepairJobDesc() diff --git a/test/unit/org/apache/cassandra/service/SerializationsTest.java b/test/unit/org/apache/cassandra/service/SerializationsTest.java index d10b65dd171d..20431fc335c6 100644 --- a/test/unit/org/apache/cassandra/service/SerializationsTest.java +++ b/test/unit/org/apache/cassandra/service/SerializationsTest.java @@ -104,7 +104,7 @@ private <T extends RepairMessage> void testRepairMessageWrite(String fileName, I private void testValidationRequestWrite() throws IOException { - ValidationRequest message = new ValidationRequest(DESC, 1234); + ValidationRequest message = new ValidationRequest(DESC, 1234, true); testRepairMessageWrite("service.ValidationRequest.bin", ValidationRequest.serializer, message); } @@ -119,6 +119,7 @@ public void testValidationRequestRead() throws IOException ValidationRequest message = ValidationRequest.serializer.deserialize(in, getVersion()); assert DESC.equals(message.desc); assert message.nowInSec == 1234; + assert message.dontPurgeTombstones == true; } } From 426eebb5139ab67b30914ed6202c25d181ab239e Mon Sep 17 00:00:00 2001 From: Runtian Liu <curlylrt@gmail.com> Date: Fri, 18 Oct 2024 10:51:45 -0500 Subject: [PATCH 035/225] A node should always return back the full information about itself for gossip regardless the asked heartbeat version patch by Runtian Liu; reviewed by Brandon Williams, Michael Semb Wever for CASSANDRA-19983 --- CHANGES.txt | 1 + .../gms/GossipDigestAckVerbHandler.java | 5 +- .../org/apache/cassandra/gms/Gossiper.java | 8 +- .../distributed/test/GossipTest.java | 133 ++++++++++++++++++ 4 files changed, 145 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 5c35bd232516..1e25031aeb8e 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0.16 + * Fix gossip issue with gossip-only and bootstrapping nodes missing DC/Rack/Host ID endpoint state (CASSANDRA-19983) 4.0.15 diff --git a/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java b/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java index 0242d837e194..e8a4dca94a1a 100644 --- a/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java +++ b/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java @@ -27,6 +27,7 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.utils.FBUtilities; import static org.apache.cassandra.net.Verb.GOSSIP_DIGEST_ACK2; @@ -85,7 +86,9 @@ public void doVerb(Message<GossipDigestAck> message) for (GossipDigest gDigest : gDigestList) { InetAddressAndPort addr = gDigest.getEndpoint(); - EndpointState localEpStatePtr = Gossiper.instance.getStateForVersionBiggerThan(addr, gDigest.getMaxVersion()); + // reply everything regardless of heartbeat if remote is asking about me, adding the flag check so we have the ability to rollback to OSS behavior + int maxVersion = FBUtilities.getBroadcastAddressAndPort().equals(addr) ? HeartBeatState.EMPTY_VERSION : gDigest.getMaxVersion(); + EndpointState localEpStatePtr = Gossiper.instance.getStateForVersionBiggerThan(addr, maxVersion); if (localEpStatePtr != null) deltaEpStateMap.put(addr, localEpStatePtr); } diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java index 009e6b255fb3..9a7a9935b8e7 100644 --- a/src/java/org/apache/cassandra/gms/Gossiper.java +++ b/src/java/org/apache/cassandra/gms/Gossiper.java @@ -1142,6 +1142,12 @@ protected long getExpireTimeForEndpoint(InetAddressAndPort endpoint) return storedTime == null ? computeExpireTime() : storedTime; } + @VisibleForTesting + public boolean inJustRemovedEndpoints(InetAddressAndPort ep) + { + return justRemovedEndpoints.containsKey(ep); + } + public EndpointState getEndpointStateForEndpoint(InetAddressAndPort ep) { return endpointStateMap.get(ep); @@ -1739,7 +1745,7 @@ void examineGossiper(List<GossipDigest> gDigestList, List<GossipDigest> deltaGos /* we request everything from the gossiper */ requestAll(gDigest, deltaGossipDigestList, remoteGeneration); } - else if (remoteGeneration < localGeneration) + else if (remoteGeneration < localGeneration || FBUtilities.getBroadcastAddressAndPort().equals(gDigest.getEndpoint())) { /* send all data with generation = localgeneration and version > -1 */ sendAll(gDigest, deltaEpStateMap, HeartBeatState.EMPTY_VERSION); diff --git a/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java b/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java index 4569600776a7..f68bbc671acb 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java @@ -21,7 +21,9 @@ import java.io.Closeable; import java.net.InetSocketAddress; import java.util.Collection; +import java.util.Set; import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.LockSupport; import java.util.stream.Collectors; @@ -30,17 +32,23 @@ import com.google.common.util.concurrent.Uninterruptibles; import org.junit.Assert; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import net.bytebuddy.ByteBuddy; import net.bytebuddy.dynamic.loading.ClassLoadingStrategy; import net.bytebuddy.implementation.MethodDelegation; +import net.bytebuddy.implementation.bind.annotation.SuperCall; import org.apache.cassandra.dht.Token; import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.api.*; +import org.apache.cassandra.distributed.shared.NetworkTopology; import org.apache.cassandra.gms.ApplicationState; import org.apache.cassandra.gms.EndpointState; +import org.apache.cassandra.gms.GossipDigestSyn; import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.Message; import org.apache.cassandra.service.PendingRangeCalculatorService; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.streaming.StreamPlan; @@ -63,6 +71,8 @@ public class GossipTest extends TestBaseImpl { + private static final Logger logger = LoggerFactory.getLogger(GossipTest.class); + @Test public void nodeDownDuringMove() throws Throwable { @@ -140,6 +150,129 @@ public void nodeDownDuringMove() throws Throwable } } + public static class BBGossiper + { + public static final AtomicBoolean disableSendGossip = new AtomicBoolean(); + public static final AtomicBoolean blockGossipStageQueue = new AtomicBoolean(); + public static void install(ClassLoader cl, Integer i) + { + new ByteBuddy().rebase(Gossiper.class) + .method(named("sendGossip")) + .intercept(MethodDelegation.to(BBGossiper.class)) + .make() + .load(cl, ClassLoadingStrategy.Default.INJECTION); + } + + public static boolean sendGossip(Message<GossipDigestSyn> message, Set<InetAddressAndPort> epSet, @SuperCall Callable<Boolean> zuper) throws Exception + { + if (disableSendGossip.get()) + { + logger.info("Send gossip disabled"); + return true; + } + return zuper.call(); + } + } + + @Test + public void testBusyGossipBusyShouldNotCreateParitialGossipInfoOnOtherNodes() throws Exception + { + int originalNodeCount = 2; + int expandedNodeCount = originalNodeCount + 1; + ExecutorService es = Executors.newFixedThreadPool(1); + // set 5s as ring delay so fatclient will be removed soon + System.setProperty("cassandra.ring_delay_ms", "5000"); + try (Cluster cluster = builder().withNodes(originalNodeCount) + .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(expandedNodeCount)) + .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(expandedNodeCount, "dc0", "rack0")) + .withConfig(config -> config.with(NETWORK, GOSSIP)) + .withInstanceInitializer(BBGossiper::install) + .start()) + { + IInstanceConfig config = cluster.newInstanceConfig(); + config.set("auto_bootstrap", true); + IInvokableInstance newInstance = cluster.bootstrap(config); + withProperty("cassandra.join_ring", false, () -> newInstance.startup(cluster)); + + // wait for the new node to show in existings gossip map, HOST_ID should be there + InetSocketAddress newNodeAddress = newInstance.broadcastAddress(); + for (int i = 1 ; i <= originalNodeCount ; ++i) + { + cluster.get(i).acceptsOnInstance((InetSocketAddress address) -> { + EndpointState ep; + InetAddressAndPort endpoint = toCassandraInetAddressAndPort(address); + while (null == (ep = Gossiper.instance.getEndpointStateForEndpoint(endpoint)) + || ep.getApplicationState(ApplicationState.HOST_ID) == null) + LockSupport.parkNanos(TimeUnit.MILLISECONDS.toNanos(10L)); + }).accept(newNodeAddress); + } + + Runnable busyNewNode = newInstance.runsOnInstance( + ()-> { + // below code will trigger the simulation for large C* cluster, this new node may only send gossip to some nodes + // here we disable sned gossip so that no other nodes will be contacted + BBGossiper.disableSendGossip.set(true); + // below code will simulate the Gossip stage queue for the new node is flooded, it is busy processing tasks + BBGossiper.blockGossipStageQueue.set(true); + Gossiper.runInGossipStageBlocking( + () -> { + logger.info("Gossip stage tasks block started"); + while (BBGossiper.blockGossipStageQueue.get()) + { + try + { + Thread.sleep(1000); + } + catch (InterruptedException e) + { + e.printStackTrace(); + } + } + logger.info("Gossip stage tasks block removed"); + } + ); + } + ); + + Future<?> nodeRecovered = es.submit(busyNewNode); + + // wait the node to be removed as a fatclient && removed from getJustRemovedEndpoints + for (int i = 1 ; i <= originalNodeCount ; ++i) + { + cluster.get(i).acceptsOnInstance((InetSocketAddress address) -> { + InetAddressAndPort endpoint = toCassandraInetAddressAndPort(address); + while (null != Gossiper.instance.getEndpointStateForEndpoint(endpoint) || Gossiper.instance.inJustRemovedEndpoints(endpoint)) + { + LockSupport.parkNanos(TimeUnit.MILLISECONDS.toNanos(10L)); + } + + }).accept(newNodeAddress); + } + + // remove block for syn verb handler + newInstance.runOnInstance( + ()-> { + BBGossiper.blockGossipStageQueue.set(false); + } + ); + + // wait the two old node to get the gossip for the new node + for (int i = 1 ; i <= originalNodeCount ; ++i) + { + cluster.get(i).acceptsOnInstance((InetSocketAddress address) -> { + InetAddressAndPort endpoint = toCassandraInetAddressAndPort(address); + while (null == Gossiper.instance.getEndpointStateForEndpoint(endpoint)) + { + LockSupport.parkNanos(TimeUnit.MILLISECONDS.toNanos(10L)); + } + // HOST_ID should be there in the gossip state + Assert.assertNotNull(Gossiper.instance.getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.HOST_ID)); + }).accept(newNodeAddress); + } + nodeRecovered.get(); + } + } + public static class BBBootstrapInterceptor { final static CountDownLatch bootstrapReady = new CountDownLatch(1); From 510c20fec1f294edf1024dce9b27f19a5ce04bb6 Mon Sep 17 00:00:00 2001 From: Gottipati Gautam <38732642+GautamGottipati@users.noreply.github.com> Date: Tue, 16 Jul 2024 01:20:04 -0500 Subject: [PATCH 036/225] Fix cqlsh CAPTURE command to save query results without trace details when TRACING is ON This patch also opportunistically fixes CAPTURE OFF. patch by Gottipati Gautam; reviewed by Stefan Miklosovic, Brad Schoening for CASSANDRA-19105 --- CHANGES.txt | 1 + pylib/cqlshlib/cqlshhandling.py | 2 +- pylib/cqlshlib/cqlshmain.py | 3 ++- pylib/cqlshlib/tracing.py | 12 ++++++++++++ 4 files changed, 16 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 91d48619427e..002e57dbc5eb 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Fix cqlsh CAPTURE command to save query results without trace details when TRACING is ON (CASSANDRA-19105) * Optionally prevent tombstone purging during repair (CASSANDRA-20071) * Add post-filtering support for the IN operator in SAI queries (CASSANDRA-20025) * Don’t finish ongoing decommission and move operations during startup (CASSANDRA-20040) diff --git a/pylib/cqlshlib/cqlshhandling.py b/pylib/cqlshlib/cqlshhandling.py index ef6ad2872546..4098b5408553 100644 --- a/pylib/cqlshlib/cqlshhandling.py +++ b/pylib/cqlshlib/cqlshhandling.py @@ -145,7 +145,7 @@ def registrator(f): ''' cqlsh_capture_cmd_syntax_rules = r''' -<captureCommand> ::= "CAPTURE" ( fname=( <stringLiteral>) | "OFF" )? +<captureCommand> ::= "CAPTURE" ( switch=( <stringLiteral> | "OFF" ) )? ; ''' diff --git a/pylib/cqlshlib/cqlshmain.py b/pylib/cqlshlib/cqlshmain.py index 2cac58ef22c4..a781efc7e1d2 100755 --- a/pylib/cqlshlib/cqlshmain.py +++ b/pylib/cqlshlib/cqlshmain.py @@ -1524,7 +1524,8 @@ def do_capture(self, parsed): To inspect the current capture configuration, use CAPTURE with no arguments. """ - fname = parsed.get_binding('fname') + fname = parsed.get_binding('switch') + if fname is None: if self.shunted_query_out is not None: print("Currently capturing query output to %r." % (self.query_out.name,)) diff --git a/pylib/cqlshlib/tracing.py b/pylib/cqlshlib/tracing.py index b7ee43c833ef..8b319906baca 100644 --- a/pylib/cqlshlib/tracing.py +++ b/pylib/cqlshlib/tracing.py @@ -38,6 +38,14 @@ def print_trace(shell, trace): """ Print an already populated cassandra.query.QueryTrace instance. """ + temp_query = None + temp_color = None + if shell.shunted_query_out is not None: + temp_query = shell.query_out + shell.query_out = shell.shunted_query_out + temp_color = shell.color + shell.color = shell.shunted_color + rows = make_trace_rows(trace) if not rows: shell.printerr("No rows for session %s found." % (trace.trace_id,)) @@ -54,6 +62,10 @@ def print_trace(shell, trace): shell.print_formatted_result(formatted_names, formatted_values, with_header=True, tty=shell.tty) shell.writeresult('') + if temp_query is not None: + shell.query_out = temp_query + shell.color = temp_color + def make_trace_rows(trace): if not trace.events: From 376fe2a9fe3f13c7555c40cda6d3912d55ef63cc Mon Sep 17 00:00:00 2001 From: Caleb Rackliffe <calebrackliffe@gmail.com> Date: Tue, 12 Nov 2024 19:10:15 -0600 Subject: [PATCH 037/225] Streamline the serialized format for index status gossip messages patch by Caleb Rackliffe; reviewed by David Capwell for CASSANDRA-20058 --- CHANGES.txt | 1 + .../org/apache/cassandra/index/Index.java | 36 ++- .../cassandra/index/IndexStatusManager.java | 117 ++++++++-- .../test/sai/IndexAvailabilityTest.java | 221 ++++++++++++------ .../index/IndexStatusManagerTest.java | 28 ++- 5 files changed, 298 insertions(+), 105 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 003d870a25a3..bb5f13abce65 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.3 + * Streamline the serialized format for index status gossip messages (CASSANDRA-20058) * Batch clusterings into single SAI partition post-filtering reads (CASSANDRA-19497) * Ban the usage of "var" instead of full types in the production code (CASSANDRA-20038) * Suppress CVE-2024-45772 from lucene-core-9.7.0.jar (CASSANDRA-20024) diff --git a/src/java/org/apache/cassandra/index/Index.java b/src/java/org/apache/cassandra/index/Index.java index 8abc800e0f18..f9e2265631c4 100644 --- a/src/java/org/apache/cassandra/index/Index.java +++ b/src/java/org/apache/cassandra/index/Index.java @@ -1031,10 +1031,36 @@ default boolean isTopK() */ enum Status { - UNKNOWN, - FULL_REBUILD_STARTED, - BUILD_FAILED, - BUILD_SUCCEEDED, - DROPPED + UNKNOWN(0), + FULL_REBUILD_STARTED(1), + BUILD_FAILED(2), + BUILD_SUCCEEDED(3), + DROPPED(4); + + public final int code; + + Status(int code) + { + this.code = code; + } + + static Status fromCode(int code) + { + switch (code) + { + case 0: + return UNKNOWN; + case 1: + return FULL_REBUILD_STARTED; + case 2: + return BUILD_FAILED; + case 3: + return BUILD_SUCCEEDED; + case 4: + return DROPPED; + } + + throw new IllegalArgumentException("Unrecognized code: " + code); + } } } diff --git a/src/java/org/apache/cassandra/index/IndexStatusManager.java b/src/java/org/apache/cassandra/index/IndexStatusManager.java index 1c0f5887db1a..6ba91beb7bcf 100644 --- a/src/java/org/apache/cassandra/index/IndexStatusManager.java +++ b/src/java/org/apache/cassandra/index/IndexStatusManager.java @@ -24,6 +24,7 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; +import java.util.concurrent.TimeUnit; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -45,6 +46,7 @@ import org.apache.cassandra.locator.Replica; import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.utils.CassandraVersion; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.JsonUtils; @@ -73,8 +75,7 @@ public class IndexStatusManager */ public final Map<InetAddressAndPort, Map<String, Index.Status>> peerIndexStatus = new HashMap<>(); - private IndexStatusManager() - {} + private IndexStatusManager() {} /** * Remove endpoints whose indexes are not queryable for the specified {@link Index.QueryPlan}. @@ -148,27 +149,55 @@ public synchronized void receivePeerIndexStatus(InetAddressAndPort endpoint, Ver if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort())) return; - Map<String, String> peerStatus = JsonUtils.fromJsonMap(versionedValue.value); - Map<String, Index.Status> indexStatus = new HashMap<>(); + Map<String, Index.Status> indexStatusMap = statusMapFromString(versionedValue); - for (Map.Entry<String, String> e : peerStatus.entrySet()) - { - String keyspaceIndex = e.getKey(); - Index.Status status = Index.Status.valueOf(e.getValue()); - indexStatus.put(keyspaceIndex, status); - } - - Map<String, Index.Status> oldStatus = peerIndexStatus.put(endpoint, indexStatus); - Map<String, Index.Status> updated = updatedIndexStatuses(oldStatus, indexStatus); - Set<String> removed = removedIndexStatuses(oldStatus, indexStatus); + Map<String, Index.Status> oldStatus = peerIndexStatus.put(endpoint, indexStatusMap); + Map<String, Index.Status> updated = updatedIndexStatuses(oldStatus, indexStatusMap); + Set<String> removed = removedIndexStatuses(oldStatus, indexStatusMap); if (!updated.isEmpty() || !removed.isEmpty()) logger.debug("Received index status for peer {}:\n Updated: {}\n Removed: {}", endpoint, updated, removed); } - catch (MarshalException | IllegalArgumentException e) + catch (Exception e) { - logger.warn("Unable to parse index status: {}", e.getMessage()); + logger.error("Unable to parse index status: {}", e.getMessage()); + } + } + + private Map<String, Index.Status> statusMapFromString(VersionedValue versionedValue) + { + Map<String, Object> peerStatus = JsonUtils.fromJsonMap(versionedValue.value); + Map<String, Index.Status> indexStatusMap = new HashMap<>(); + + for (Map.Entry<String, Object> endpointStatus : peerStatus.entrySet()) + { + String keyspaceOrIndex = endpointStatus.getKey(); + Object keyspaceOrIndexStatus = endpointStatus.getValue(); + + if (keyspaceOrIndexStatus instanceof String) + { + // This is the legacy format: (fully qualified index name -> enum string) + Index.Status status = Index.Status.valueOf(keyspaceOrIndexStatus.toString()); + indexStatusMap.put(keyspaceOrIndex, status); + } + else if (keyspaceOrIndexStatus instanceof Map) + { + // This is the new format. (keyspace -> (index -> numeric enum code)) + @SuppressWarnings("unchecked") + Map<String, Integer> keyspaceIndexStatusMap = (Map<String, Integer>) keyspaceOrIndexStatus; + + for (Map.Entry<String, Integer> indexStatus : keyspaceIndexStatusMap.entrySet()) + { + Index.Status status = Index.Status.fromCode(indexStatus.getValue()); + indexStatusMap.put(identifier(keyspaceOrIndex, indexStatus.getKey()), status); + } + } + else + { + throw new MarshalException("Invalid index status format: " + endpointStatus); + } } + return indexStatusMap; } /** @@ -183,35 +212,79 @@ public synchronized void propagateLocalIndexStatus(String keyspace, String index { try { - Map<String, Index.Status> states = peerIndexStatus.computeIfAbsent(FBUtilities.getBroadcastAddressAndPort(), + Map<String, Index.Status> statusMap = peerIndexStatus.computeIfAbsent(FBUtilities.getBroadcastAddressAndPort(), k -> new HashMap<>()); String keyspaceIndex = identifier(keyspace, index); if (status == Index.Status.DROPPED) - states.remove(keyspaceIndex); + statusMap.remove(keyspaceIndex); else - states.put(keyspaceIndex, status); + statusMap.put(keyspaceIndex, status); // Don't try and propagate if the gossiper isn't enabled. This is primarily for tests where the // Gossiper has not been started. If we attempt to propagate when not started an exception is // logged and this causes a number of dtests to fail. if (Gossiper.instance.isEnabled()) { - String newStatus = JsonUtils.JSON_OBJECT_MAPPER.writeValueAsString(states); + // Versions 5.0.0 through 5.0.2 use a much more bloated format that duplicates keyspace names + // and writes full status names instead of their numeric codes. If the minimum cluster version is + // unknown or one of those 3 versions, continue to propagate the old format. + CassandraVersion minVersion = Gossiper.instance.getMinVersion(1, TimeUnit.SECONDS); + String newSerializedStatusMap = shouldWriteLegacyStatusFormat(minVersion) ? JsonUtils.writeAsJsonString(statusMap) + : toSerializedFormat(statusMap); + statusPropagationExecutor.submit(() -> { // schedule gossiper update asynchronously to avoid potential deadlock when another thread is holding // gossiper taskLock. - VersionedValue value = StorageService.instance.valueFactory.indexStatus(newStatus); + VersionedValue value = StorageService.instance.valueFactory.indexStatus(newSerializedStatusMap); Gossiper.instance.addLocalApplicationState(ApplicationState.INDEX_STATUS, value); }); } } - catch (Throwable e) + catch (Exception e) { logger.warn("Unable to propagate index status: {}", e.getMessage()); } } + private static boolean shouldWriteLegacyStatusFormat(CassandraVersion minVersion) + { + return minVersion == null || (minVersion.major == 5 && minVersion.minor == 0 && minVersion.patch < 3); + } + + /** + * Serializes as a JSON string the status of the indexes in the provided map. + * <p> + * For example, the map... + * <pre> + * { + * ks1.cf1_idx1=FULL_REBUILD_STARTED, + * ks1.cf1_idx2=FULL_REBUILD_STARTED, + * system.PaxosUncommittedIndex=BUILD_SUCCEEDED + * } + * </pre> + * ...will be converted to the string... + * <pre> + * { + * "system": {"PaxosUncommittedIndex": 3}, + * "ks1": {"cf1_idx1": 1, "cf1_idx2": 1} + * } + * </pre> + */ + public static String toSerializedFormat(Map<String, Index.Status> indexStatusMap) + { + Map<String, Map<String, Integer>> serialized = new HashMap<>(); + + for (Map.Entry<String, Index.Status> e : indexStatusMap.entrySet()) + { + String[] keyspaceAndIndex = e.getKey().split("\\."); + serialized.computeIfAbsent(keyspaceAndIndex[0], ignore -> new HashMap<>()) + .put(keyspaceAndIndex[1], e.getValue().code); + } + + return JsonUtils.writeAsJsonString(serialized); + } + @VisibleForTesting public synchronized Index.Status getIndexStatus(InetAddressAndPort peer, String keyspace, String index) { diff --git a/test/distributed/org/apache/cassandra/distributed/test/sai/IndexAvailabilityTest.java b/test/distributed/org/apache/cassandra/distributed/test/sai/IndexAvailabilityTest.java index 3a9e111badfb..5731281601a1 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/sai/IndexAvailabilityTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/sai/IndexAvailabilityTest.java @@ -29,6 +29,9 @@ import com.google.common.base.Objects; import org.junit.Test; +import net.bytebuddy.ByteBuddy; +import net.bytebuddy.dynamic.loading.ClassLoadingStrategy; +import net.bytebuddy.implementation.MethodDelegation; import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.api.IInvokableInstance; import org.apache.cassandra.distributed.test.TestBaseImpl; @@ -41,6 +44,7 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.utils.FBUtilities; +import static net.bytebuddy.matcher.ElementMatchers.named; import static org.apache.cassandra.distributed.api.Feature.GOSSIP; import static org.apache.cassandra.distributed.api.Feature.NETWORK; import static org.apache.cassandra.distributed.test.sai.SAIUtil.waitForIndexQueryable; @@ -62,88 +66,117 @@ public class IndexAvailabilityTest extends TestBaseImpl public void verifyIndexStatusPropagation() throws Exception { try (Cluster cluster = init(Cluster.build(2) - .withConfig(config -> config.with(GOSSIP) - .with(NETWORK)) + .withConfig(config -> config.with(GOSSIP).with(NETWORK)) .start())) { - String ks1 = "ks1"; - String ks2 = "ks2"; - String ks3 = "ks3"; - String cf1 = "cf1"; - String index1 = "cf1_idx1"; - String index2 = "cf1_idx2"; - - keyspaces = Arrays.asList(ks1, ks2, ks3); - indexesPerKs = Arrays.asList(index1, index2); - - // create 1 tables per keyspace, 2 indexes per table. all indexes are queryable - for (String ks : keyspaces) - { - cluster.schemaChange(String.format(CREATE_KEYSPACE, ks, 2)); - cluster.schemaChange(String.format(CREATE_TABLE, ks, cf1)); - cluster.schemaChange(String.format(CREATE_INDEX, index1, ks, cf1, "v1")); - cluster.schemaChange(String.format(CREATE_INDEX, index2, ks, cf1, "v2")); - waitForIndexQueryable(cluster, ks); - cluster.forEach(node -> { - expectedNodeIndexQueryability.put(NodeIndex.create(ks, index1, node), Index.Status.BUILD_SUCCEEDED); - expectedNodeIndexQueryability.put(NodeIndex.create(ks, index2, node), Index.Status.BUILD_SUCCEEDED); - }); - } + verifyIndexStatusPropagation(cluster); + } + } - // mark ks1 index1 as non-queryable on node1 - markIndexNonQueryable(cluster.get(1), ks1, cf1, index1); - // on node2, it observes that node1 ks1.index1 is not queryable - waitForIndexingStatus(cluster.get(2), ks1, index1, cluster.get(1), Index.Status.BUILD_FAILED); - // other indexes or keyspaces should not be affected - assertIndexingStatus(cluster); - - // mark ks2 index2 as non-queryable on node2 - markIndexNonQueryable(cluster.get(2), ks2, cf1, index2); - // on node1, it observes that node2 ks2.index2 is not queryable - waitForIndexingStatus(cluster.get(1), ks2, index2, cluster.get(2), Index.Status.BUILD_FAILED); - // other indexes or keyspaces should not be affected - assertIndexingStatus(cluster); - - // mark ks1 index1 as queryable on node1 - markIndexQueryable(cluster.get(1), ks1, cf1, index1); - // on node2, it observes that node1 ks1.index1 is queryable - waitForIndexingStatus(cluster.get(2), ks1, index1, cluster.get(1), Index.Status.BUILD_SUCCEEDED); - // other indexes or keyspaces should not be affected - assertIndexingStatus(cluster); - - // mark ks2 index2 as indexing on node1 - markIndexBuilding(cluster.get(1), ks2, cf1, index2); - // on node2, it observes that node1 ks2.index2 is not queryable - waitForIndexingStatus(cluster.get(2), ks2, index2, cluster.get(1), Index.Status.FULL_REBUILD_STARTED); - // other indexes or keyspaces should not be affected - assertIndexingStatus(cluster); - - // drop ks1, ks1 index1/index2 should be non queryable on all nodes - cluster.schemaChange("DROP KEYSPACE " + ks1); - expectedNodeIndexQueryability.keySet().forEach(k -> { - if (k.keyspace.equals(ks1)) - expectedNodeIndexQueryability.put(k, Index.Status.UNKNOWN); - }); - assertIndexingStatus(cluster); + @Test + public void verifyIndexStatusPropagationMixedPatchVersion() throws Exception + { + try (Cluster cluster = init(Cluster.build(2) + .withConfig(config -> config.with(GOSSIP).with(NETWORK)) + .withInstanceInitializer(MixedPatchVersionHelper::setVersions) + .start())) + { + verifyIndexStatusPropagation(cluster); + } + } - // drop ks2 index2, there should be no ks2 index2 status on all node - cluster.schemaChange("DROP INDEX " + ks2 + "." + index2); - expectedNodeIndexQueryability.keySet().forEach(k -> { - if (k.keyspace.equals(ks2) && k.index.equals(index2)) - expectedNodeIndexQueryability.put(k, Index.Status.UNKNOWN); - }); - assertIndexingStatus(cluster); + @Test + public void verifyIndexStatusPropagationMixedMajorVersion() throws Exception + { + try (Cluster cluster = init(Cluster.build(2) + .withConfig(config -> config.with(GOSSIP).with(NETWORK)) + .withInstanceInitializer(MixedMajorVersionHelper::setVersions) + .start())) + { + verifyIndexStatusPropagation(cluster); + } + } + + private void verifyIndexStatusPropagation(Cluster cluster) + { + String ks1 = "ks1"; + String ks2 = "ks2"; + String ks3 = "ks3"; + String cf1 = "cf1"; + String index1 = "cf1_idx1"; + String index2 = "cf1_idx2"; + + keyspaces = Arrays.asList(ks1, ks2, ks3); + indexesPerKs = Arrays.asList(index1, index2); - // drop ks3 cf1, there should be no ks3 index1/index2 status - cluster.schemaChange("DROP TABLE " + ks3 + "." + cf1); - expectedNodeIndexQueryability.keySet().forEach(k -> { - if (k.keyspace.equals(ks3)) - expectedNodeIndexQueryability.put(k, Index.Status.UNKNOWN); + // create 1 tables per keyspace, 2 indexes per table. all indexes are queryable + for (String ks : keyspaces) + { + cluster.schemaChange(String.format(CREATE_KEYSPACE, ks, 2)); + cluster.schemaChange(String.format(CREATE_TABLE, ks, cf1)); + cluster.schemaChange(String.format(CREATE_INDEX, index1, ks, cf1, "v1")); + cluster.schemaChange(String.format(CREATE_INDEX, index2, ks, cf1, "v2")); + waitForIndexQueryable(cluster, ks); + cluster.forEach(node -> { + expectedNodeIndexQueryability.put(NodeIndex.create(ks, index1, node), Index.Status.BUILD_SUCCEEDED); + expectedNodeIndexQueryability.put(NodeIndex.create(ks, index2, node), Index.Status.BUILD_SUCCEEDED); }); - assertIndexingStatus(cluster); } + + // mark ks1 index1 as non-queryable on node1 + markIndexNonQueryable(cluster.get(1), ks1, cf1, index1); + // on node2, it observes that node1 ks1.index1 is not queryable + waitForIndexingStatus(cluster.get(2), ks1, index1, cluster.get(1), Index.Status.BUILD_FAILED); + // other indexes or keyspaces should not be affected + assertIndexingStatus(cluster); + + // mark ks2 index2 as non-queryable on node2 + markIndexNonQueryable(cluster.get(2), ks2, cf1, index2); + // on node1, it observes that node2 ks2.index2 is not queryable + waitForIndexingStatus(cluster.get(1), ks2, index2, cluster.get(2), Index.Status.BUILD_FAILED); + // other indexes or keyspaces should not be affected + assertIndexingStatus(cluster); + + // mark ks1 index1 as queryable on node1 + markIndexQueryable(cluster.get(1), ks1, cf1, index1); + // on node2, it observes that node1 ks1.index1 is queryable + waitForIndexingStatus(cluster.get(2), ks1, index1, cluster.get(1), Index.Status.BUILD_SUCCEEDED); + // other indexes or keyspaces should not be affected + assertIndexingStatus(cluster); + + // mark ks2 index2 as indexing on node1 + markIndexBuilding(cluster.get(1), ks2, cf1, index2); + // on node2, it observes that node1 ks2.index2 is not queryable + waitForIndexingStatus(cluster.get(2), ks2, index2, cluster.get(1), Index.Status.FULL_REBUILD_STARTED); + // other indexes or keyspaces should not be affected + assertIndexingStatus(cluster); + + // drop ks1, ks1 index1/index2 should be non queryable on all nodes + cluster.schemaChange("DROP KEYSPACE " + ks1); + expectedNodeIndexQueryability.keySet().forEach(k -> { + if (k.keyspace.equals(ks1)) + expectedNodeIndexQueryability.put(k, Index.Status.UNKNOWN); + }); + assertIndexingStatus(cluster); + + // drop ks2 index2, there should be no ks2 index2 status on all node + cluster.schemaChange("DROP INDEX " + ks2 + '.' + index2); + expectedNodeIndexQueryability.keySet().forEach(k -> { + if (k.keyspace.equals(ks2) && k.index.equals(index2)) + expectedNodeIndexQueryability.put(k, Index.Status.UNKNOWN); + }); + assertIndexingStatus(cluster); + + // drop ks3 cf1, there should be no ks3 index1/index2 status + cluster.schemaChange("DROP TABLE " + ks3 + '.' + cf1); + expectedNodeIndexQueryability.keySet().forEach(k -> { + if (k.keyspace.equals(ks3)) + expectedNodeIndexQueryability.put(k, Index.Status.UNKNOWN); + }); + assertIndexingStatus(cluster); } + @SuppressWarnings("DataFlowIssue") private void markIndexNonQueryable(IInvokableInstance node, String keyspace, String table, String indexName) { expectedNodeIndexQueryability.put(NodeIndex.create(keyspace, indexName, node), Index.Status.BUILD_FAILED); @@ -155,6 +188,7 @@ private void markIndexNonQueryable(IInvokableInstance node, String keyspace, Str }); } + @SuppressWarnings("DataFlowIssue") private void markIndexQueryable(IInvokableInstance node, String keyspace, String table, String indexName) { expectedNodeIndexQueryability.put(NodeIndex.create(keyspace, indexName, node), Index.Status.BUILD_SUCCEEDED); @@ -166,6 +200,7 @@ private void markIndexQueryable(IInvokableInstance node, String keyspace, String }); } + @SuppressWarnings("DataFlowIssue") private void markIndexBuilding(IInvokableInstance node, String keyspace, String table, String indexName) { expectedNodeIndexQueryability.put(NodeIndex.create(keyspace, indexName, node), Index.Status.FULL_REBUILD_STARTED); @@ -285,4 +320,44 @@ public int hashCode() return Objects.hashCode(keyspace, index, node); } } + + public static class MixedMajorVersionHelper + { + @SuppressWarnings({ "unused", "resource" }) + static void setVersions(ClassLoader loader, int node) + { + if (node == 1) + new ByteBuddy().rebase(FBUtilities.class) + .method(named("getReleaseVersionString")) + .intercept(MethodDelegation.to(MixedMajorVersionHelper.class)) + .make() + .load(loader, ClassLoadingStrategy.Default.INJECTION); + } + + @SuppressWarnings("unused") + public static String getReleaseVersionString() + { + return "4.1.0"; + } + } + + public static class MixedPatchVersionHelper + { + @SuppressWarnings({ "unused", "resource" }) + static void setVersions(ClassLoader loader, int node) + { + if (node == 1) + new ByteBuddy().rebase(FBUtilities.class) + .method(named("getReleaseVersionString")) + .intercept(MethodDelegation.to(MixedPatchVersionHelper.class)) + .make() + .load(loader, ClassLoadingStrategy.Default.INJECTION); + } + + @SuppressWarnings("unused") + public static String getReleaseVersionString() + { + return "5.0.2"; + } + } } diff --git a/test/unit/org/apache/cassandra/index/IndexStatusManagerTest.java b/test/unit/org/apache/cassandra/index/IndexStatusManagerTest.java index d08fec974a19..331e079680f3 100644 --- a/test/unit/org/apache/cassandra/index/IndexStatusManagerTest.java +++ b/test/unit/org/apache/cassandra/index/IndexStatusManagerTest.java @@ -19,12 +19,16 @@ package org.apache.cassandra.index; import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.TreeSet; import java.util.stream.Collectors; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.mockito.Mockito; import org.apache.cassandra.db.ConsistencyLevel; @@ -47,8 +51,21 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.Assert.assertArrayEquals; +@RunWith(Parameterized.class) public class IndexStatusManagerTest { + @Parameterized.Parameter + public boolean legacyStatusFormat; + + @Parameterized.Parameters(name = "{0}") + public static List<Object[]> parameters() + { + List<Object[]> parameters = new ArrayList<>(); + parameters.add(new Object[] { true }); + parameters.add(new Object[] { false }); + return parameters; + } + static class Testcase { String keyspace; @@ -389,11 +406,12 @@ void runTest(Testcase testcase) .collect(Collectors.toSet()); // send indexStatus for each endpoint - testcase.indexStatus.forEach((endpoint, indexStatus) -> - IndexStatusManager.instance.receivePeerIndexStatus( - endpoint, - VersionedValue.unsafeMakeVersionedValue(JsonUtils.writeAsJsonString(indexStatus), 1) - )); + testcase.indexStatus.forEach((endpoint, indexStatusMap) -> + { + String serialized = legacyStatusFormat ? JsonUtils.writeAsJsonString(indexStatusMap) + : IndexStatusManager.toSerializedFormat(indexStatusMap); + IndexStatusManager.instance.receivePeerIndexStatus(endpoint, VersionedValue.unsafeMakeVersionedValue(serialized, 1)); + }); // sort the replicas here, so that we can assert the order later EndpointsForRange endpoints = EndpointsForRange.copyOf(new TreeSet<>(replicas)); From 659558c980c67a80287ca7ccdfc8a70b1a56b7e2 Mon Sep 17 00:00:00 2001 From: Yifan Cai <ycai@apache.org> Date: Tue, 12 Nov 2024 15:56:06 -0800 Subject: [PATCH 038/225] Enhance CQLSSTableWriter to notify client on sstable production Patch by Yifan Cai; Reviewed by Francisco Guerrero for CASSANDRA-19800 --- CHANGES.txt | 1 + .../sstable/AbstractSSTableSimpleWriter.java | 31 +++++++ .../io/sstable/CQLSSTableWriter.java | 38 +++++++- .../sstable/SSTableSimpleUnsortedWriter.java | 7 +- .../io/sstable/SSTableSimpleWriter.java | 9 +- .../io/sstable/CQLSSTableWriterTest.java | 89 ++++++++++++++++++- 6 files changed, 168 insertions(+), 7 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 1e25031aeb8e..23dcc154f8e6 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -3,6 +3,7 @@ 4.0.15 + * Enhance CQLSSTableWriter to notify clients on sstable production (CASSANDRA-19800) * Change the resolution of AbstractCommitLogService#lastSyncedAt to nanos to be aligned with later comparisons (CASSANDRA-20074) * Backport of CASSANDRA-17812: Rate-limit new client connection auth setup to avoid overwhelming bcrypt (CASSANDRA-20057) * Support UDTs and vectors as clustering keys in descending order (CASSANDRA-20050) diff --git a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java index 4eaf1fe64a01..74ebc00b7102 100644 --- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java @@ -23,14 +23,18 @@ import java.io.Closeable; import java.nio.ByteBuffer; import java.util.Collections; +import java.util.Collection; import java.util.HashSet; +import java.util.Objects; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; import org.apache.cassandra.db.*; import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.db.rows.EncodingStats; import org.apache.cassandra.io.sstable.format.SSTableFormat; +import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.schema.TableMetadataRef; import org.apache.cassandra.service.ActiveRepairService; @@ -45,6 +49,8 @@ abstract class AbstractSSTableSimpleWriter implements Closeable protected SSTableFormat.Type formatType = SSTableFormat.Type.current(); protected static AtomicInteger generation = new AtomicInteger(0); protected boolean makeRangeAware = false; + protected Consumer<Collection<SSTableReader>> sstableProducedListener; + protected boolean openSSTableOnProduced = false; protected AbstractSSTableSimpleWriter(File directory, TableMetadataRef metadata, RegularAndStaticColumns columns) { @@ -63,6 +69,31 @@ protected void setRangeAwareWriting(boolean makeRangeAware) this.makeRangeAware = makeRangeAware; } + protected void setSSTableProducedListener(Consumer<Collection<SSTableReader>> listener) + { + this.sstableProducedListener = Objects.requireNonNull(listener); + } + + protected void setShouldOpenProducedSSTable(boolean openSSTableOnProduced) + { + this.openSSTableOnProduced = openSSTableOnProduced; + } + + /** + * Indicate whether the produced sstable should be opened or not. + */ + protected boolean shouldOpenSSTables() + { + return openSSTableOnProduced; + } + + protected void notifySSTableProduced(Collection<SSTableReader> sstables) + { + if (sstableProducedListener == null) + return; + + sstableProducedListener.accept(sstables); + } protected SSTableTxnWriter createWriter() { diff --git a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java index 4c299d071ee5..3b20b7565174 100644 --- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java @@ -24,10 +24,12 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.SortedSet; import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; import java.util.stream.Collectors; import org.apache.cassandra.config.DatabaseDescriptor; @@ -50,6 +52,7 @@ import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.exceptions.SyntaxException; import org.apache.cassandra.io.sstable.format.SSTableFormat; +import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.schema.*; import org.apache.cassandra.service.ClientState; import org.apache.cassandra.transport.ProtocolVersion; @@ -355,9 +358,10 @@ public static class Builder private CreateTableStatement.Raw schemaStatement; private ModificationStatement.Parsed insertStatement; private IPartitioner partitioner; - private boolean sorted = false; private long maxSSTableSizeInMiB = -1L; + private Consumer<Collection<SSTableReader>> sstableProducedListener; + private boolean openSSTableOnProduced = false; protected Builder() { this.typeStatements = new ArrayList<>(); @@ -525,6 +529,33 @@ public Builder sorted() return this; } + /** + * Set the listener to receive notifications on sstable produced + * <p> + * Note that if listener is registered, the sstables are opened into {@link SSTableReader}. + * The consumer is responsible for releasing the {@link SSTableReader} + * + * @param sstableProducedListener receives the produced sstables + * @return this builder + */ + public Builder withSSTableProducedListener(Consumer<Collection<SSTableReader>> sstableProducedListener) + { + this.sstableProducedListener = sstableProducedListener; + return this; + } + + /** + * Whether the produced sstable should be open or not. + * By default, the writer does not open the produced sstables + * + * @return this builder + */ + public Builder openSSTableOnProduced() + { + this.openSSTableOnProduced = true; + return this; + } + @SuppressWarnings("resource") public CQLSSTableWriter build() { @@ -581,6 +612,11 @@ public CQLSSTableWriter build() if (formatType != null) writer.setSSTableFormatType(formatType); + if (sstableProducedListener != null) + writer.setSSTableProducedListener(sstableProducedListener); + + writer.setShouldOpenProducedSSTable(openSSTableOnProduced); + return new CQLSSTableWriter(writer, preparedInsert, preparedInsert.getBindVariables()); } } diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java index 87e87e95c230..457621b65bfa 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java @@ -19,6 +19,7 @@ import java.io.File; import java.io.IOException; +import java.util.Collection; import java.util.Map; import java.util.TreeMap; import java.util.concurrent.BlockingQueue; @@ -34,6 +35,7 @@ import org.apache.cassandra.db.rows.SerializationHelper; import org.apache.cassandra.db.rows.UnfilteredSerializer; import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.schema.TableMetadataRef; import org.apache.cassandra.utils.JVMStabilityInspector; @@ -208,11 +210,12 @@ public void run() if (b == SENTINEL) return; - try (SSTableTxnWriter writer = createWriter()) + try (SSTableTxnWriter writer = createWriter()) { for (Map.Entry<DecoratedKey, PartitionUpdate.Builder> entry : b.entrySet()) writer.append(entry.getValue().build().unfilteredIterator()); - writer.finish(false); + Collection<SSTableReader> finished = writer.finish(shouldOpenSSTables()); + notifySSTableProduced(finished); } } catch (Throwable e) diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java index 1cbff412add5..cd51e1ebb401 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java @@ -19,12 +19,14 @@ import java.io.File; import java.io.IOException; +import java.util.Collection; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import org.apache.cassandra.db.*; import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.schema.TableMetadataRef; /** @@ -136,8 +138,11 @@ private void maybeCloseWriter(SSTableTxnWriter writer) { try { - if (writer != null) - writer.finish(false); + if (writer == null) + return; + + Collection<SSTableReader> finished = writer.finish(shouldOpenSSTables()); + notifySSTableProduced(finished); } catch (Throwable t) { diff --git a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java index 13b6e2d92798..8295ac2468fe 100644 --- a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java @@ -25,13 +25,12 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; -import java.util.stream.Stream; import java.util.stream.StreamSupport; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.io.Files; +import org.apache.cassandra.io.sstable.format.SSTableReader; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Rule; @@ -919,6 +918,92 @@ public void testWriteWithSortedAndMaxSize() throws Exception } } + @Test + public void testNotifySSTableFinishedForSorted() throws Exception + { + testNotifySSTableFinished(true, false); + } + + @Test + public void testNotifySSTableFinishedForUnsorted() throws Exception + { + testNotifySSTableFinished(false, false); + } + + @Test + public void testCloseSortedWriterOnFirstPorducedShouldStillResultInTwoSSTables() throws Exception + { + // Writing a new partition (and exceeding the size limit) leads to closing the current writer and buffering the last partition update. + // Since there is a last partition buffered, closing the sstable writer flushes to a new sstable. + // Therefore, even though the test closes the writer immediately, there are still 2 sstables produced. + testNotifySSTableFinished(true, true); + } + + private void testNotifySSTableFinished(boolean sorted, boolean closeWriterOnFirstProduced) throws Exception + { + List<SSTableReader> produced = new ArrayList<>(); + String schema = "CREATE TABLE " + qualifiedTable + " (" + + " k int PRIMARY KEY," + + " v text )"; + CQLSSTableWriter.Builder builder = CQLSSTableWriter + .builder() + .inDirectory(dataDir) + .forTable(schema) + .using("INSERT INTO " + qualifiedTable + + " (k, v) VALUES (?, ?)") + .withMaxSSTableSizeInMiB(1) + .openSSTableOnProduced() + .withSSTableProducedListener(produced::addAll); + if (sorted) + { + builder.sorted(); + } + CQLSSTableWriter writer = builder.build(); + + int rowCount = 30_000; + // Max SSTable size is 1 MiB + // 30_000 rows should take 30_000 * (4 + 37) = 1.17 MiB > 1 MiB, i.e. producing 2 sstables + for (int i = 0; i < rowCount; i++) + { + writer.addRow(i, UUID.randomUUID().toString()); + if (closeWriterOnFirstProduced && !produced.isEmpty()) + { + // on closing writer, it flushes the last update to the new sstable + writer.close(); + break; + } + } + // the assertion is only performed for sorted because unsorted writer writes asynchrously; avoid flakiness + if (!closeWriterOnFirstProduced && sorted) + { + // while writing, one sstable should be finished + assertEquals(1, produced.size()); + } + + if (!closeWriterOnFirstProduced) + writer.close(); + // another sstable is finished on closing the writer + assertEquals(2, produced.size()); + + File[] dataFiles = dataDir.listFiles((file, name) -> name.endsWith(Component.DATA.name())); + assertNotNull(dataFiles); + assertEquals("The sorted writer should produce 2 sstables when max sstable size is configured", + 2, dataFiles.length); + Set<File> notifiedDataFileSet = produced.stream() + .map(sstable -> new File(sstable.descriptor.filenameFor(Component.DATA))) + .collect(Collectors.toSet()); + Set<File> listedDataFileSet = Arrays.stream(dataFiles) + .map(f -> { + try { + return f.getCanonicalFile(); + } catch (IOException e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toSet()); + assertEquals(notifiedDataFileSet, listedDataFileSet); + } + private static void loadSSTables(File dataDir, String ks) throws ExecutionException, InterruptedException { SSTableLoader loader = new SSTableLoader(dataDir, new SSTableLoader.Client() From 9131be9fa53e27b7e3bc332b613e00b73e9fffef Mon Sep 17 00:00:00 2001 From: Maulin Vasavada <maulin.vasavada@gmail.com> Date: Tue, 23 Jul 2024 23:27:29 -0700 Subject: [PATCH 039/225] Make JMX SSL configurable in cassandra.yaml patch by Maulin Vasavada; reviewed by Stefan Miklosovic, Doug Rohrer for CASSANDRA-18508 --- CHANGES.txt | 1 + NEWS.txt | 1 + conf/cassandra-env.sh | 5 + conf/cassandra.yaml | 18 ++ conf/cassandra_latest.yaml | 18 ++ .../pages/managing/operating/security.adoc | 68 +++++- doc/scripts/convert_yaml_to_adoc.py | 1 + ...rnetesSecretsPEMSslContextFactoryTest.java | 2 +- .../org/apache/cassandra/config/Config.java | 2 + .../cassandra/config/DatabaseDescriptor.java | 15 ++ .../cassandra/config/EncryptionOptions.java | 11 +- .../cassandra/utils/JMXServerUtils.java | 95 +++----- .../utils/RMIClientSocketFactoryImpl.java | 5 +- .../RMICloseableClientSocketFactory.java | 29 +++ .../RMICloseableServerSocketFactory.java | 29 +++ .../utils/jmx/AbstractJmxSocketFactory.java | 208 ++++++++++++++++++ .../utils/jmx/DefaultJmxSocketFactory.java | 82 +++++++ .../cassandra-jmx-disabled-sslconfig.yaml | 147 +++++++++++++ test/conf/cassandra-jmx-pem-sslconfig.yaml | 147 +++++++++++++ test/conf/cassandra-jmx-sslconfig.yaml | 72 ++++++ .../CollectingRMIServerSocketFactoryImpl.java | 7 +- ...llectingSslRMIServerSocketFactoryImpl.java | 160 ++++++++++++++ .../cassandra/distributed/impl/Instance.java | 2 +- .../distributed/impl/IsolatedJmx.java | 77 +++++-- .../impl/IsolatedJmxSocketFactory.java | 96 ++++++++ .../impl/JmxTestClientSslContextFactory.java | 135 ++++++++++++ .../impl/JmxTestClientSslSocketFactory.java | 60 +++++ .../impl/RMISslClientSocketFactoryImpl.java | 131 +++++++++++ .../distributed/shared/WithProperties.java | 7 + .../test/AbstractEncryptionOptionsImpl.java | 14 +- .../test/InternodeEncryptionOptionsTest.java | 16 +- .../NativeTransportEncryptionOptionsTest.java | 14 +- .../distributed/test/ResourceLeakTest.java | 4 +- .../SSTableLoaderEncryptionOptionsTest.java | 4 +- .../distributed/test/WithPropertiesTest.java | 44 ++++ .../distributed/test/jmx/JMXFeatureTest.java | 6 +- .../test/jmx/JMXGetterCheckTest.java | 128 +---------- .../test/jmx/JMXSslConfigDistributedTest.java | 177 +++++++++++++++ .../distributed/test/jmx/JMXTestsUtil.java | 160 ++++++++++++++ .../config/EncryptionOptionsTest.java | 2 +- .../cassandra/config/PropertiesTest.java | 4 +- .../db/virtual/SettingsTableTest.java | 1 + .../cassandra/utils/jmx/JMXSslConfigTest.java | 102 +++++++++ ...XSslConfiguredWithYamlFileOptionsTest.java | 104 +++++++++ .../JMXSslDisabledEncryptionOptionsTest.java | 102 +++++++++ .../utils/jmx/JMXSslPEMConfigTest.java | 84 +++++++ .../utils/jmx/JMXSslPropertiesUtil.java | 65 ++++++ 47 files changed, 2410 insertions(+), 252 deletions(-) create mode 100644 src/java/org/apache/cassandra/utils/RMICloseableClientSocketFactory.java create mode 100644 src/java/org/apache/cassandra/utils/RMICloseableServerSocketFactory.java create mode 100644 src/java/org/apache/cassandra/utils/jmx/AbstractJmxSocketFactory.java create mode 100644 src/java/org/apache/cassandra/utils/jmx/DefaultJmxSocketFactory.java create mode 100644 test/conf/cassandra-jmx-disabled-sslconfig.yaml create mode 100644 test/conf/cassandra-jmx-pem-sslconfig.yaml create mode 100644 test/conf/cassandra-jmx-sslconfig.yaml create mode 100644 test/distributed/org/apache/cassandra/distributed/impl/CollectingSslRMIServerSocketFactoryImpl.java create mode 100644 test/distributed/org/apache/cassandra/distributed/impl/IsolatedJmxSocketFactory.java create mode 100644 test/distributed/org/apache/cassandra/distributed/impl/JmxTestClientSslContextFactory.java create mode 100644 test/distributed/org/apache/cassandra/distributed/impl/JmxTestClientSslSocketFactory.java create mode 100644 test/distributed/org/apache/cassandra/distributed/impl/RMISslClientSocketFactoryImpl.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/WithPropertiesTest.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/jmx/JMXSslConfigDistributedTest.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/jmx/JMXTestsUtil.java create mode 100644 test/unit/org/apache/cassandra/utils/jmx/JMXSslConfigTest.java create mode 100644 test/unit/org/apache/cassandra/utils/jmx/JMXSslConfiguredWithYamlFileOptionsTest.java create mode 100644 test/unit/org/apache/cassandra/utils/jmx/JMXSslDisabledEncryptionOptionsTest.java create mode 100644 test/unit/org/apache/cassandra/utils/jmx/JMXSslPEMConfigTest.java create mode 100644 test/unit/org/apache/cassandra/utils/jmx/JMXSslPropertiesUtil.java diff --git a/CHANGES.txt b/CHANGES.txt index 8aab7a4e8c74..75dd48aa0246 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Make JMX SSL configurable in cassandra.yaml (CASSANDRA-18508) * Fix cqlsh CAPTURE command to save query results without trace details when TRACING is ON (CASSANDRA-19105) * Optionally prevent tombstone purging during repair (CASSANDRA-20071) * Add post-filtering support for the IN operator in SAI queries (CASSANDRA-20025) diff --git a/NEWS.txt b/NEWS.txt index 8d9e620801c0..b4511c3bba9f 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -88,6 +88,7 @@ New features generate a password of configured password strength policy upon role creation or alteration when 'GENERATED PASSWORD' clause is used. Character sets supported are: English, Cyrillic, modern Cyrillic, German, Polish and Czech. + - JMX SSL configuration can be now done in cassandra.yaml via jmx_encryption_options section instead of cassandra-env.sh Upgrading diff --git a/conf/cassandra-env.sh b/conf/cassandra-env.sh index 4e28e687f903..98b7ed8ccdf1 100644 --- a/conf/cassandra-env.sh +++ b/conf/cassandra-env.sh @@ -236,6 +236,11 @@ else JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true" # jmx ssl options + # Consider using the jmx_encryption_options section of cassandra.yaml instead + # to prevent sensitive information being exposed. + # In case jmx ssl options are configured in both the places - this file and cassandra.yaml, and + # if com.sun.management.jmxremote.ssl is configured to be true here and encryption_options are marked enabled in + # cassandra.yaml, then we will get exception at the startup #JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl=true" #JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.need.client.auth=true" #JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.protocols=<enabled-protocols>" diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index 9c752b7d5442..b93b41fa0bb7 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -1712,6 +1712,24 @@ client_encryption_options: # warnings will be reported during the session establishment. # certificate_validity_warn_threshold: 10d +# Configure SSL for JMX +# +# Using system properties via cassandra-env.sh to configure JMX SSL is +# considered legacy and only supported for backward compatibility. If SSL +# is enabled via both methods, a configuration error will occur at +# startup. Hot reloading of the `SSLContext` is not yet supported for the +# JMX SSL. +# Similar to `client/server_encryption_options`, you can specify PEM-based +# key material or customize the SSL configuration using `ssl_context_factory` in `jmx_encryption_options`. +#jmx_encryption_options: +# enabled: true +# cipher_suites: [TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256] +# accepted_protocols: [TLSv1.2,TLSv1.3,TLSv1.1] +# keystore: conf/cassandra_ssl.keystore +# keystore_password: cassandra +# truststore: conf/cassandra_ssl.truststore +# truststore_password: cassandra + # internode_compression controls whether traffic between nodes is # compressed. # Can be: diff --git a/conf/cassandra_latest.yaml b/conf/cassandra_latest.yaml index 39bce459ccb3..2ad1a25f0779 100644 --- a/conf/cassandra_latest.yaml +++ b/conf/cassandra_latest.yaml @@ -1676,6 +1676,24 @@ client_encryption_options: # TLS_RSA_WITH_AES_256_CBC_SHA # ] +# Configure SSL for JMX +# +# Using system properties via cassandra-env.sh to configure JMX SSL is +# considered legacy and only supported for backward compatibility. If SSL +# is enabled via both methods, a configuration error will occur at +# startup. Hot reloading of the `SSLContext` is not yet supported for the +# JMX SSL. +# Similar to `client/server_encryption_options`, you can specify PEM-based +# key material or customize the SSL configuration using `ssl_context_factory` in `jmx_encryption_options`. +#jmx_encryption_options: +# enabled: true +# cipher_suites: [TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256] +# accepted_protocols: [TLSv1.2,TLSv1.3,TLSv1.1] +# keystore: conf/cassandra_ssl.keystore +# keystore_password: cassandra +# truststore: conf/cassandra_ssl.truststore +# truststore_password: cassandra + # internode_compression controls whether traffic between nodes is # compressed. # Can be: diff --git a/doc/modules/cassandra/pages/managing/operating/security.adoc b/doc/modules/cassandra/pages/managing/operating/security.adoc index a425cab8d0f2..e7748e93fb61 100644 --- a/doc/modules/cassandra/pages/managing/operating/security.adoc +++ b/doc/modules/cassandra/pages/managing/operating/security.adoc @@ -78,7 +78,8 @@ files having the required PEM data as shown below, * Configuration: PEM keys/certs defined in-line (mind the spaces in the YAML!) -.... +[source,yaml] +---- client/server_encryption_options: ssl_context_factory: class_name: org.apache.cassandra.security.PEMBasedSslContextFactory @@ -97,17 +98,19 @@ YAML!) -----BEGIN CERTIFICATE----- <your base64 encoded certificate> -----END CERTIFICATE----- -.... +---- * Configuration: PEM keys/certs defined in files -.... + +[source,yaml] +---- client/server_encryption_options: ssl_context_factory: class_name: org.apache.cassandra.security.PEMBasedSslContextFactory keystore: <file path to the keystore file in the PEM format with the private key and the certificate chain> keystore_password: "<your password if the private key is encrypted with a password>" truststore: <file path to the truststore file in the PEM format> -.... +---- == SSL Certificate Hot Reloading @@ -564,8 +567,61 @@ See also: xref:cassandra:developing/cql/security.adoc#permissions[`Permissions`] === JMX With SSL -JMX SSL configuration is controlled by a number of system properties, -some of which are optional. To turn on SSL, edit the relevant lines in +JMX SSL configuration should be specified in `cassandra.yaml` under +`encryption_options`, similar to how `client/server_encryption_options` +are configured. Using system properties to configure JMX SSL is +considered legacy and only supported for backward compatibility. If SSL +is enabled via both methods, a configuration error will occur at +startup. Hot reloading of the `SSLContext` is not yet supported for the +JMX SSL. + +==== JMX SSL configuration in cassandra.yaml +Below is an example of configuring JMX SSL in `cassandra.yaml` + +[source,yaml] +---- +jmx_encryption_options: + enabled: true + cipher_suites: [TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256] + accepted_protocols: [TLSv1.2,TLSv1.3,TLSv1.1] + keystore: test/conf/cassandra_ssl_test.keystore + keystore_password: cassandra + truststore: test/conf/cassandra_ssl_test.truststore + truststore_password: cassandra +---- + +Similar to `client/server_encryption_options`, you can specify PEM-based +key material or customize the SSL configuration using +`ssl_context_factory` in `jmx_encryption_options`. + +Below is an example of configuring PEM based key material, +[source,yaml] +---- +jmx_encryption_options: + enabled: true + cipher_suites: [TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256] + accepted_protocols: [TLSv1.2,TLSv1.3,TLSv1.1] + ssl_context_factory: + class_name: org.apache.cassandra.security.PEMBasedSslContextFactory + parameters: + private_key: | + -----BEGIN ENCRYPTED PRIVATE KEY----- OR -----BEGIN PRIVATE KEY----- + <your base64 encoded private key> + -----END ENCRYPTED PRIVATE KEY----- OR -----END PRIVATE KEY----- + -----BEGIN CERTIFICATE----- + <your base64 encoded certificate chain> + -----END CERTIFICATE----- + + private_key_password: "<your password if the private key is encrypted with a password>" + + trusted_certificates: | + -----BEGIN CERTIFICATE----- + <your base64 encoded certificate> + -----END CERTIFICATE----- +---- + +==== (Legacy) JMX SSL configuration with system properties +To turn on SSL this way, edit the relevant lines in `cassandra-env.sh` to uncomment and set the values of these properties as required: diff --git a/doc/scripts/convert_yaml_to_adoc.py b/doc/scripts/convert_yaml_to_adoc.py index bf5e16148813..061706f83589 100644 --- a/doc/scripts/convert_yaml_to_adoc.py +++ b/doc/scripts/convert_yaml_to_adoc.py @@ -49,6 +49,7 @@ 'hints_compression', 'server_encryption_options', 'client_encryption_options', + 'jmx_encryption_options', 'transparent_data_encryption_options', 'hinted_handoff_disabled_datacenters', 'startup_checks', diff --git a/examples/ssl-factory/test/unit/org/apache/cassandra/security/KubernetesSecretsPEMSslContextFactoryTest.java b/examples/ssl-factory/test/unit/org/apache/cassandra/security/KubernetesSecretsPEMSslContextFactoryTest.java index 5ecec4ed26d3..ef20d425bb72 100644 --- a/examples/ssl-factory/test/unit/org/apache/cassandra/security/KubernetesSecretsPEMSslContextFactoryTest.java +++ b/examples/ssl-factory/test/unit/org/apache/cassandra/security/KubernetesSecretsPEMSslContextFactoryTest.java @@ -114,7 +114,7 @@ public void setup() * In order to test with real 'env' variables comment out this line and set appropriate env variable. This is * done to avoid having a dependency on env in the unit test. */ - commonConfig.put("require_client_auth", "false"; + commonConfig.put("require_client_auth", "false"); commonConfig.put("cipher_suites", Arrays.asList("TLS_RSA_WITH_AES_128_CBC_SHA")); } diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 119cfbfb7da0..ca4cf09be4a1 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -434,6 +434,7 @@ public static class SSTableConfig public EncryptionOptions.ServerEncryptionOptions server_encryption_options = new EncryptionOptions.ServerEncryptionOptions(); public EncryptionOptions client_encryption_options = new EncryptionOptions(); + public EncryptionOptions jmx_encryption_options = new EncryptionOptions(); public InternodeCompression internode_compression = InternodeCompression.none; @@ -1320,6 +1321,7 @@ public enum BatchlogEndpointStrategy private static final Set<String> SENSITIVE_KEYS = new HashSet<String>() {{ add("client_encryption_options"); add("server_encryption_options"); + add("jmx_encryption_options"); }}; public static void log(Config config) diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 784a07932034..3cd003b97b6f 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -960,6 +960,9 @@ else if (conf.commitlog_segment_size.toKibibytes() < 2 * conf.max_mutation_size. if (conf.client_encryption_options != null) conf.client_encryption_options.applyConfig(); + if (conf.jmx_encryption_options != null) + conf.jmx_encryption_options.applyConfig(); + if (conf.snapshot_links_per_second < 0) throw new ConfigurationException("snapshot_links_per_second must be >= 0"); @@ -1306,7 +1309,14 @@ public static void applySslContext() { SSLFactory.validateSslContext("Internode messaging", conf.server_encryption_options, REQUIRED, true); SSLFactory.validateSslContext("Native transport", conf.client_encryption_options, conf.client_encryption_options.getClientAuth(), true); + // For JMX SSL the validation is pretty much the same as the Native transport + SSLFactory.validateSslContext("JMX transport", conf.jmx_encryption_options, conf.jmx_encryption_options.getClientAuth(), true); SSLFactory.initHotReloading(conf.server_encryption_options, conf.client_encryption_options, false); + /* + For JMX SSL, the hot reloading of the SSLContext is out of scope for CASSANDRA-18508. + Since JMXServerUtil that initializes the JMX Server is used statically, it may require significant + effort to change that behavior unlike SSLFactory used for Native transport/Internode messaging. + */ } catch (IOException e) { @@ -3653,6 +3663,11 @@ public static EncryptionOptions getNativeProtocolEncryptionOptions() return conf.client_encryption_options; } + public static EncryptionOptions getJmxEncryptionOptions() + { + return conf.jmx_encryption_options; + } + @VisibleForTesting public static void updateNativeProtocolEncryptionOptions(Function<EncryptionOptions, EncryptionOptions> update) { diff --git a/src/java/org/apache/cassandra/config/EncryptionOptions.java b/src/java/org/apache/cassandra/config/EncryptionOptions.java index fdc5c0efc6ae..eeac859f317c 100644 --- a/src/java/org/apache/cassandra/config/EncryptionOptions.java +++ b/src/java/org/apache/cassandra/config/EncryptionOptions.java @@ -172,10 +172,10 @@ public enum ConfigKey ConfigKey(String keyName) { - this.keyName=keyName; + this.keyName = keyName; } - String getKeyName() + public String toString() { return keyName; } @@ -184,8 +184,9 @@ static Set<String> asSet() { Set<String> valueSet = new HashSet<>(); ConfigKey[] values = values(); - for(ConfigKey key: values) { - valueSet.add(toLowerCaseLocalized(key.getKeyName())); + for (ConfigKey key : values) + { + valueSet.add(toLowerCaseLocalized(key.toString())); } return valueSet; } @@ -357,7 +358,7 @@ private void initializeSslContextFactory() protected static void putSslContextFactoryParameter(Map<String, Object> existingParameters, ConfigKey configKey, Object value) { if (value != null) { - existingParameters.put(configKey.getKeyName(), value); + existingParameters.put(configKey.toString(), value); } } diff --git a/src/java/org/apache/cassandra/utils/JMXServerUtils.java b/src/java/org/apache/cassandra/utils/JMXServerUtils.java index 78c8ced8d2fb..43221f28bfb9 100644 --- a/src/java/org/apache/cassandra/utils/JMXServerUtils.java +++ b/src/java/org/apache/cassandra/utils/JMXServerUtils.java @@ -37,24 +37,25 @@ import java.rmi.server.RMIClientSocketFactory; import java.rmi.server.RMIServerSocketFactory; import java.rmi.server.UnicastRemoteObject; -import java.util.Arrays; import java.util.HashMap; import java.util.Map; -import java.util.stream.Collectors; -import javax.management.remote.*; +import javax.management.remote.JMXAuthenticator; +import javax.management.remote.JMXConnectorServer; +import javax.management.remote.JMXServiceURL; +import javax.management.remote.MBeanServerForwarder; import javax.management.remote.rmi.RMIConnectorServer; import javax.management.remote.rmi.RMIJRMPServerImpl; -import javax.rmi.ssl.SslRMIClientSocketFactory; -import javax.rmi.ssl.SslRMIServerSocketFactory; +import javax.net.ssl.SSLException; import javax.security.auth.Subject; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableMap; -import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.cassandra.auth.jmx.AuthenticationProxy; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.utils.jmx.DefaultJmxSocketFactory; import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_AUTHORIZER; import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_REMOTE_LOGIN_CONFIG; @@ -62,12 +63,6 @@ import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_AUTHENTICATE; import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_PASSWORD_FILE; import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_RMI_PORT; -import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL; -import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_CIPHER_SUITES; -import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_PROTOCOLS; -import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_NEED_CLIENT_AUTH; -import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES; -import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS; import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_RMI_SERVER_HOSTNAME; public class JMXServerUtils @@ -92,6 +87,7 @@ public static JMXConnectorServer createJMXServer(int port, String hostname, bool } // Configure the RMI client & server socket factories, including SSL config. + // CASSANDRA-18508: Make JMX SSL to be configured in cassandra.yaml env.putAll(configureJmxSocketFactories(serverAddress, local)); // configure the RMI registry @@ -226,46 +222,33 @@ private static MBeanServerForwarder configureJmxAuthorization(Map<String, Object } } - private static Map<String, Object> configureJmxSocketFactories(InetAddress serverAddress, boolean localOnly) + /** + * Configures the client and server socket factories for the JMX connection. It uses {@link DefaultJmxSocketFactory} + * for configuring this. + * + * @param serverAddress the JMX server is bound to + * @param localOnly {@code true} if the JMX server only allows local connections; {@code false} if the JMX server + * allows the remote connections. + * @return Map<String, Object@gt; containing {@code jmx.remote.rmi.client.socket.factory}, {@code jmx.remote.rmi.server.socket.factory} + * and {@code com.sun.jndi.rmi.factory.socket} properties for the client and server socket factories. + * @throws SSLException if it fails to configure the socket factories with the given input + * + * @see DefaultJmxSocketFactory + */ + @VisibleForTesting + public static Map<String, Object> configureJmxSocketFactories(InetAddress serverAddress, boolean localOnly) throws SSLException { - Map<String, Object> env = new HashMap<>(); - if (COM_SUN_MANAGEMENT_JMXREMOTE_SSL.getBoolean()) - { - boolean requireClientAuth = COM_SUN_MANAGEMENT_JMXREMOTE_SSL_NEED_CLIENT_AUTH.getBoolean(); - String[] protocols = null; - String protocolList = COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_PROTOCOLS.getString(); - if (protocolList != null) - { - JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS.setString(protocolList); - protocols = StringUtils.split(protocolList, ','); - } - - String[] ciphers = null; - String cipherList = COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_CIPHER_SUITES.getString(); - if (cipherList != null) - { - JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES.setString(cipherList); - ciphers = StringUtils.split(cipherList, ','); - } - - SslRMIClientSocketFactory clientFactory = new SslRMIClientSocketFactory(); - SslRMIServerSocketFactory serverFactory = new SslRMIServerSocketFactory(ciphers, protocols, requireClientAuth); - env.put(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE, serverFactory); - env.put(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE, clientFactory); - env.put("com.sun.jndi.rmi.factory.socket", clientFactory); - logJmxSslConfig(serverFactory); - } - else if (localOnly) - { - env.put(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE, - new RMIServerSocketFactoryImpl(serverAddress)); - } - - return env; + return new DefaultJmxSocketFactory().configure(serverAddress, localOnly, DatabaseDescriptor.getJmxEncryptionOptions()); } @VisibleForTesting public static void logJmxServiceUrl(InetAddress serverAddress, int port) + { + logger.info("Configured JMX server at: {}", getJmxServiceUrl(serverAddress, port)); + } + + @VisibleForTesting + private static String getJmxServiceUrl(InetAddress serverAddress, int port) { String urlTemplate = "service:jmx:rmi://%1$s/jndi/rmi://%1$s:%2$d/jmxrmi"; String hostName; @@ -280,23 +263,9 @@ public static void logJmxServiceUrl(InetAddress serverAddress, int port) ? '[' + serverAddress.getHostAddress() + ']' : serverAddress.getHostAddress(); } - String url = String.format(urlTemplate, hostName, port); - logger.info("Configured JMX server at: {}", url); - } - - private static void logJmxSslConfig(SslRMIServerSocketFactory serverFactory) - { - if (logger.isDebugEnabled()) - logger.debug("JMX SSL configuration. { protocols: [{}], cipher_suites: [{}], require_client_auth: {} }", - serverFactory.getEnabledProtocols() == null - ? "'JVM defaults'" - : Arrays.stream(serverFactory.getEnabledProtocols()).collect(Collectors.joining("','", "'", "'")), - serverFactory.getEnabledCipherSuites() == null - ? "'JVM defaults'" - : Arrays.stream(serverFactory.getEnabledCipherSuites()).collect(Collectors.joining("','", "'", "'")), - serverFactory.getNeedClientAuth()); + return String.format(urlTemplate, hostName, port); } - + private static class JMXPluggableAuthenticatorWrapper implements JMXAuthenticator { private static final MethodHandle ctorHandle; diff --git a/src/java/org/apache/cassandra/utils/RMIClientSocketFactoryImpl.java b/src/java/org/apache/cassandra/utils/RMIClientSocketFactoryImpl.java index 81e728eeb933..9be9e69f0409 100644 --- a/src/java/org/apache/cassandra/utils/RMIClientSocketFactoryImpl.java +++ b/src/java/org/apache/cassandra/utils/RMIClientSocketFactoryImpl.java @@ -22,7 +22,6 @@ import java.io.Serializable; import java.net.InetAddress; import java.net.Socket; -import java.rmi.server.RMIClientSocketFactory; import java.util.ArrayList; import java.util.List; import java.util.Objects; @@ -32,8 +31,9 @@ * which can otherwise be influenced by the system property "java.rmi.server.hostname" in strange and * unpredictable ways. */ -public class RMIClientSocketFactoryImpl implements RMIClientSocketFactory, Serializable +public class RMIClientSocketFactoryImpl implements Serializable, RMICloseableClientSocketFactory { + private static final long serialVersionUID = 955153017775496366L; List<Socket> sockets = new ArrayList<>(); private final InetAddress localAddress; @@ -50,6 +50,7 @@ public Socket createSocket(String host, int port) throws IOException return socket; } + @Override public void close() throws IOException { for (Socket socket: sockets) diff --git a/src/java/org/apache/cassandra/utils/RMICloseableClientSocketFactory.java b/src/java/org/apache/cassandra/utils/RMICloseableClientSocketFactory.java new file mode 100644 index 000000000000..1e2225772abe --- /dev/null +++ b/src/java/org/apache/cassandra/utils/RMICloseableClientSocketFactory.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils; + +import java.rmi.server.RMIClientSocketFactory; + +/** + * This represents closeable RMI Client Socket factory. It extends {@link AutoCloseable} and can be used with + * {@code try-with-resources}. + */ +public interface RMICloseableClientSocketFactory extends RMIClientSocketFactory, AutoCloseable +{ +} diff --git a/src/java/org/apache/cassandra/utils/RMICloseableServerSocketFactory.java b/src/java/org/apache/cassandra/utils/RMICloseableServerSocketFactory.java new file mode 100644 index 000000000000..b9d70f083ae5 --- /dev/null +++ b/src/java/org/apache/cassandra/utils/RMICloseableServerSocketFactory.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils; + +import java.rmi.server.RMIServerSocketFactory; + +/** + * This represents closeable RMI Server Socket factory. It extends {@link AutoCloseable} and can be used with + * {@code try-with-resources}. + */ +public interface RMICloseableServerSocketFactory extends RMIServerSocketFactory, AutoCloseable +{ +} diff --git a/src/java/org/apache/cassandra/utils/jmx/AbstractJmxSocketFactory.java b/src/java/org/apache/cassandra/utils/jmx/AbstractJmxSocketFactory.java new file mode 100644 index 000000000000..6ee390ee93a7 --- /dev/null +++ b/src/java/org/apache/cassandra/utils/jmx/AbstractJmxSocketFactory.java @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils.jmx; + +import java.net.InetAddress; +import java.util.HashMap; +import java.util.Map; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLException; + +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.EncryptionOptions; +import org.apache.cassandra.exceptions.ConfigurationException; + +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_CIPHER_SUITES; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_PROTOCOLS; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_NEED_CLIENT_AUTH; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS; + +/** + * Abstracts out the most common workflow in setting up the SSL client and server socket factorires for JMX. + * First, it checks the system properties (see <a href="https://docs.oracle.com/en/java/javase/17/management/monitoring-and-management-using-jmx-technology.html#GUID-F08985BB-629A-4FBF-A0CB-8762DF7590E0">Java Documentation</a> to read the SSL configuration. + * Next, it checks the provided {@code jmxEncryptionOptions} to read the SSL configuration. + * If none of them is enabled, it checks the provided {@code localOnly} flag to configure the JMX server socket + * factory for the local JMX connection. + */ +abstract public class AbstractJmxSocketFactory +{ + private static final Logger logger = LoggerFactory.getLogger(AbstractJmxSocketFactory.class); + + /** + * Configures the client and server socket factories for the JMX connection. + * Specifically it configures below properties as applicable, + * <pre> + * jmx.remote.rmi.client.socket.factory + * jmx.remote.rmi.server.socket.factory + * com.sun.jndi.rmi.factory.socket + * </pre> + * <p> + * In case of remote connection enabled, this also sets the following system properties, + * <pre> + * com.sun.management.jmxremote.ssl=true + * javax.rmi.ssl.client.enabledCipherSuites=<applicable cipher suites provided in the configuration> + * javax.rmi.ssl.client.enabledProtocols=<applicable protocols provided in the configuration> + * </pre> + * + * @param serverAddress the JMX server is bound to + * @param localOnly {@code true} if the JMX server only allows local connections; {@code false} if the JMX server + * allows the remote connections. + * @param jmxEncryptionOptions {@link EncryptionOptions} used for the SSL configuration in case of the remote + * connections. Could be {@code null} if system properties are + * used instead as per <a href="https://docs.oracle.com/en/java/javase/17/management/monitoring-and-management-using-jmx-technology.html#GUID-F08985BB-629A-4FBF-A0CB-8762DF7590E0">Java Documentation</a> + * @return Map<String, Object@gt; containing {@code jmx.remote.rmi.client.socket.factory}, {@code jmx.remote.rmi.server.socket.factory} + * and {@code com.sun.jndi.rmi.factory.socket} properties for the client and server socket factories. + * @throws SSLException if it fails to configure the socket factories with the given input + */ + public Map<String, Object> configure(InetAddress serverAddress, boolean localOnly, + EncryptionOptions jmxEncryptionOptions) throws SSLException + { + Map<String, Object> env = new HashMap<>(); + boolean jmxRemoteSslSystemConfigProvided = COM_SUN_MANAGEMENT_JMXREMOTE_SSL.getBoolean(); + // We check for the enabled jmx_encryption_options here because in case of no configuration provided in cassandra.yaml + // it will default to empty/non-null encryption options. Hence, we consider it set only if 'enabled' flag is set to true + boolean jmxEncryptionOptionsProvided = jmxEncryptionOptions != null + && jmxEncryptionOptions.getEnabled() != null + && jmxEncryptionOptions.getEnabled(); + + if (jmxRemoteSslSystemConfigProvided && jmxEncryptionOptionsProvided) + { + throw new ConfigurationException("Please specify JMX SSL configuration in either cassandra-env.sh or " + + "cassandra.yaml, not in both locations"); + } + + boolean requireClientAuth = false; + String[] ciphers = null; + String[] protocols = null; + SSLContext sslContext = null; + + if (jmxRemoteSslSystemConfigProvided) + { + logger.info("Enabling JMX SSL using environment file properties"); + logger.warn("Consider using the jmx_encryption_options section of cassandra.yaml instead to prevent " + + "sensitive information being exposed"); + requireClientAuth = COM_SUN_MANAGEMENT_JMXREMOTE_SSL_NEED_CLIENT_AUTH.getBoolean(); + String protocolList = COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_PROTOCOLS.getString(); + if (protocolList != null) + { + JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS.setString(protocolList); + protocols = StringUtils.split(protocolList, ','); + } + + String cipherList = COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_CIPHER_SUITES.getString(); + if (cipherList != null) + { + JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES.setString(cipherList); + ciphers = StringUtils.split(cipherList, ','); + } + configureSslClientSocketFactory(env, serverAddress); + configureSslServerSocketFactory(env, serverAddress, ciphers, protocols, requireClientAuth); + } + else if (jmxEncryptionOptionsProvided) + { + logger.info("Enabling JMX SSL using jmx_encryption_options from cassandra.yaml"); + // Here we can continue to use the SslRMIClientSocketFactory for client sockets. + // However, we should still set System properties for cipher_suites and enabled_protocols + // to have the same behavior as cassandra-env.sh based JMX SSL settings + setJmxSystemProperties(jmxEncryptionOptions); + + requireClientAuth = jmxEncryptionOptions.getClientAuth() == EncryptionOptions.ClientAuth.REQUIRED; + ciphers = jmxEncryptionOptions.cipherSuitesArray(); + protocols = jmxEncryptionOptions.acceptedProtocolsArray(); + sslContext = jmxEncryptionOptions.sslContextFactoryInstance + .createJSSESslContext(jmxEncryptionOptions.getClientAuth()); + configureSslClientSocketFactory(env, serverAddress); + configureSslServerSocketFactory(env, serverAddress, ciphers, protocols, requireClientAuth, sslContext); + } + else if (localOnly) + { + configureLocalSocketFactories(env, serverAddress); + } + + return env; + } + + /** + * Configures the non-SSL socket factories for the local JMX. + * + * @param env output param containing the configured socket factories + * @param serverAddress the JMX server is bound to + */ + abstract public void configureLocalSocketFactories(Map<String, Object> env, InetAddress serverAddress); + + /** + * Configures SSL based client socket factory. + * + * @param env output param containing the configured socket factories + * @param serverAddress the JMX server is bound to + */ + abstract public void configureSslClientSocketFactory(Map<String, Object> env, InetAddress serverAddress); + + /** + * Configures SSL based server socket factory based on system config for key/trust stores. + * + * @param env output param containing the configured socket factories + * @param serverAddress the JMX server is bound to + * @param enabledCipherSuites for the SSL communication + * @param enabledProtocols for the SSL communication + * @param needClientAuth {@code true} if it requires the client-auth; {@code false} otherwise + */ + abstract public void configureSslServerSocketFactory(Map<String, Object> env, InetAddress serverAddress, + String[] enabledCipherSuites, String[] enabledProtocols, + boolean needClientAuth); + + /** + * Configures SSL based server socket factory based on custom SSLContext. + * + * @param env output param containing the configured socket factories + * @param serverAddress the JMX server is bound to + * @param enabledCipherSuites for the SSL communication + * @param enabledProtocols for the SSL communication + * @param needClientAuth {@code true} if it requires the client-auth; {@code false} otherwise + * @param sslContext for the SSL communication + */ + abstract public void configureSslServerSocketFactory(Map<String, Object> env, InetAddress serverAddress, + String[] enabledCipherSuites, String[] enabledProtocols, + boolean needClientAuth, SSLContext sslContext); + + /** + * Sets the following JMX system properties. + * <pre> + * com.sun.management.jmxremote.ssl=true + * javax.rmi.ssl.client.enabledCipherSuites=<applicable cipher suites provided in the configuration> + * javax.rmi.ssl.client.enabledProtocols=<applicable protocols provided in the configuration> + * </pre> + * + * @param jmxEncryptionOptions for the SSL communication + */ + private void setJmxSystemProperties(EncryptionOptions jmxEncryptionOptions) + { + COM_SUN_MANAGEMENT_JMXREMOTE_SSL.setBoolean(true); + if (jmxEncryptionOptions.getAcceptedProtocols() != null) + JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS.setString(StringUtils.join(jmxEncryptionOptions.getAcceptedProtocols(), ",")); + + if (jmxEncryptionOptions.cipherSuitesArray() != null) + JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES.setString(StringUtils.join(jmxEncryptionOptions.cipherSuitesArray(), ",")); + } +} diff --git a/src/java/org/apache/cassandra/utils/jmx/DefaultJmxSocketFactory.java b/src/java/org/apache/cassandra/utils/jmx/DefaultJmxSocketFactory.java new file mode 100644 index 000000000000..35cc92b731e3 --- /dev/null +++ b/src/java/org/apache/cassandra/utils/jmx/DefaultJmxSocketFactory.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils.jmx; + +import java.net.InetAddress; +import java.util.Arrays; +import java.util.Map; +import java.util.stream.Collectors; +import javax.management.remote.rmi.RMIConnectorServer; +import javax.net.ssl.SSLContext; +import javax.rmi.ssl.SslRMIClientSocketFactory; +import javax.rmi.ssl.SslRMIServerSocketFactory; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.utils.RMIServerSocketFactoryImpl; + +/** + * Default implementation of the JMX Socket factory. + */ +public final class DefaultJmxSocketFactory extends AbstractJmxSocketFactory +{ + private static final Logger logger = LoggerFactory.getLogger(DefaultJmxSocketFactory.class); + + @Override + public void configureLocalSocketFactories(Map<String, Object> env, InetAddress serverAddress) + { + env.put(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE, new RMIServerSocketFactoryImpl(serverAddress)); + } + + @Override + public void configureSslClientSocketFactory(Map<String, Object> env, InetAddress serverAddress) + { + SslRMIClientSocketFactory clientFactory = new SslRMIClientSocketFactory(); + env.put(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE, clientFactory); + env.put("com.sun.jndi.rmi.factory.socket", clientFactory); + } + + @Override + public void configureSslServerSocketFactory(Map<String, Object> env, InetAddress serverAddress, String[] enabledCipherSuites, + String[] enabledProtocols, boolean needClientAuth) + { + SslRMIServerSocketFactory serverFactory = new SslRMIServerSocketFactory(enabledCipherSuites, enabledProtocols, needClientAuth); + env.put(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE, serverFactory); + logJmxSslConfig(serverFactory); + } + + @Override + public void configureSslServerSocketFactory(Map<String, Object> env, InetAddress serverAddress, String[] enabledCipherSuites, + String[] enabledProtocols, boolean needClientAuth, SSLContext sslContext) + { + SslRMIServerSocketFactory serverFactory = new SslRMIServerSocketFactory(sslContext, enabledCipherSuites, enabledProtocols, needClientAuth); + env.put(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE, serverFactory); + logJmxSslConfig(serverFactory); + } + + private void logJmxSslConfig(SslRMIServerSocketFactory serverFactory) + { + if (logger.isDebugEnabled()) + logger.debug("JMX SSL configuration. { protocols: [{}], cipher_suites: [{}], require_client_auth: {} }", + serverFactory.getEnabledProtocols() == null ? "'JVM defaults'" : Arrays.stream(serverFactory.getEnabledProtocols()).collect(Collectors.joining("','", "'", "'")), + serverFactory.getEnabledCipherSuites() == null ? "'JVM defaults'" : Arrays.stream(serverFactory.getEnabledCipherSuites()).collect(Collectors.joining("','", "'", "'")), + serverFactory.getNeedClientAuth()); + } +} diff --git a/test/conf/cassandra-jmx-disabled-sslconfig.yaml b/test/conf/cassandra-jmx-disabled-sslconfig.yaml new file mode 100644 index 000000000000..2b6338992668 --- /dev/null +++ b/test/conf/cassandra-jmx-disabled-sslconfig.yaml @@ -0,0 +1,147 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# +# Testing for pluggable ssl_context_factory option for client and server encryption options with a valid and a missing +# implementation classes. +# +cluster_name: Test Cluster +# memtable_allocation_type: heap_buffers +memtable_allocation_type: offheap_objects +commitlog_sync: batch +commitlog_segment_size: 5MiB +commitlog_directory: build/test/cassandra/commitlog +# commitlog_compression: +# - class_name: LZ4Compressor +cdc_raw_directory: build/test/cassandra/cdc_raw +cdc_enabled: false +hints_directory: build/test/cassandra/hints +partitioner: org.apache.cassandra.dht.ByteOrderedPartitioner +listen_address: 127.0.0.1 +storage_port: 7012 +ssl_storage_port: 17012 +start_native_transport: true +native_transport_port: 9042 +column_index_size: 4KiB +saved_caches_directory: build/test/cassandra/saved_caches +data_file_directories: + - build/test/cassandra/data +disk_access_mode: mmap_index_only +seed_provider: + - class_name: org.apache.cassandra.locator.SimpleSeedProvider + parameters: + - seeds: "127.0.0.1:7012" +endpoint_snitch: org.apache.cassandra.locator.SimpleSnitch +dynamic_snitch: true +jmx_encryption_options: + enabled: false + cipher_suites: [TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256] + accepted_protocols: [TLSv1.2,TLSv1.3,TLSv1.1] + ssl_context_factory: + class_name: org.apache.cassandra.security.PEMBasedSslContextFactory + parameters: + private_key: | + -----BEGIN ENCRYPTED PRIVATE KEY----- + MIIE6jAcBgoqhkiG9w0BDAEDMA4ECOWqSzq5PBIdAgIFxQSCBMjXsCK30J0aT3J/ + g5kcbmevTOY1pIhJGbf5QYYrMUPiuDK2ydxIbiPzoTE4/S+OkCeHhlqwn/YydpBl + xgjZZ1Z5rLJHO27d2biuESqanDiBVXYuVmHmaifRnFy0uUTFkStB5mjVZEiJgO29 + L83hL60uWru71EVuVriC2WCfmZ/EXp6wyYszOqCFQ8Quk/rDO6XuaBl467MJbx5V + sucGT6E9XKNd9hB14/Izb2jtVM5kqKxoiHpz1na6yhEYJiE5D1uOonznWjBnjwB/ + f0x+acpDfVDoJKTlRdz+DEcbOF7mb9lBVVjP6P/AAsmQzz6JKwHjvCrjYfQmyyN8 + RI4KRQnWgm4L3dtByLqY8HFU4ogisCMCgI+hZQ+OKMz/hoRO540YGiPcTRY3EOUR + 0bd5JxU6tCJDMTqKP9aSL2KmLoiLowdMkSPz7TCzLsZ2bGJemuCfpAs4XT1vXCHs + evrUbOnh8et1IA8mZ9auThfqsZtNagJLEXA6hWIKp1FfVL3Q49wvMKZt4eTn/zwU + tLL0m5yPo6/HAaOA3hbm/oghZS0dseshXl7PZrmZQtvYnIvjyoxEL7ducYDQCDP6 + wZ7Nzyh1QZAauSS15hl3vLFRZCA9hWAVgwQAviTvhB342O0i9qI7TQkcHk+qcTPN + K+iGNbFZ8ma1izXNKSJ2PgI/QqFNIeJWvZrb9PhJRmaZVsTJ9fERm1ewpebZqkVv + zMqMhlKgx9ggAaSKgnGZkwXwB6GrSbbzUrwRCKm3FieD1QE4VVYevaadVUU75GG5 + mrFKorJEH7kFZlic8OTjDksYnHbcgU36XZrGEXa2+ldVeGKL3CsXWciaQRcJg8yo + WQDjZpcutGI0eMJWCqUkv8pYZC2/wZU4htCve5nVJUU4t9uuo9ex7lnwlLWPvheQ + jUBMgzSRsZ+zwaIusvufAAxiKK/cJm4ubZSZPIjBbfd4U7VPxtirP4Accydu7EK6 + eG/MZwtAMFNJxfxUR+/aYzJU/q1ePw7fWVHrpt58t/22CX2SJBEiUGmSmuyER4Ny + DPw6d6mhvPUS1jRhIZ9A81ht8MOX7VL5uVp307rt7o5vRpV1mo0iPiRHzGscMpJn + AP36klEAUNTf0uLTKZa7KHiwhn5iPmsCrENHkOKJjxhRrqHjD2wy3YHs3ow2voyY + Ua4Cids+c1hvRkNEDGNHm4+rKGFOGOsG/ZU7uj/6gflO4JXxNGiyTLflqMdWBvow + Zd7hk1zCaGAAn8nZ0hPweGxQ4Q30I9IBZrimGxB0vjiUqNio9+qMf33dCHFJEuut + ZGJMaUGVaPhXQcTy4uD5hzsPZV5xcsU4H3vBYyBcZgrusJ6OOgkuZQaU7p8rWQWr + bUEVbXuZdwEmxsCe7H/vEVv5+aA4sF4kWnMMFL7/LIYaiEzkTqdJlRv/KyJJgcAH + hg2BvR3XTAq8wiX0C98CdmTbsx2eyQdj5tCU606rEohFLKUxWkJYAKxCiUbxGGpI + RheVmxkef9ErxJiq7hsAsGrSJvMtJuDKIasnD14SOEwD/7jRAq6WdL9VLpxtzlOw + pWnIl8kUCO3WoaG9Jf+ZTIv2hnxJhaSzYrdXzGPNnaWKhBlwnXJRvQEdrIxZOimP + FujZhqbKUDbYAcqTkoQ= + -----END ENCRYPTED PRIVATE KEY----- + -----BEGIN CERTIFICATE----- + MIIDkTCCAnmgAwIBAgIETxH5JDANBgkqhkiG9w0BAQsFADB5MRAwDgYDVQQGEwdV + bmtub3duMRAwDgYDVQQIEwdVbmtub3duMRAwDgYDVQQHEwdVbmtub3duMRAwDgYD + VQQKEwdVbmtub3duMRQwEgYDVQQLDAtzc2xfdGVzdGluZzEZMBcGA1UEAxMQQXBh + Y2hlIENhc3NhbmRyYTAeFw0xNjAzMTgyMTI4MDJaFw0xNjA2MTYyMTI4MDJaMHkx + EDAOBgNVBAYTB1Vua25vd24xEDAOBgNVBAgTB1Vua25vd24xEDAOBgNVBAcTB1Vu + a25vd24xEDAOBgNVBAoTB1Vua25vd24xFDASBgNVBAsMC3NzbF90ZXN0aW5nMRkw + FwYDVQQDExBBcGFjaGUgQ2Fzc2FuZHJhMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8A + MIIBCgKCAQEAjkmVX/HS49cS8Hn6o26IGwMIcEV3d7ZhH0GNcx8rnSRd10dU9F6d + ugSjbwGFMcWUQzYNejN6az0Wb8JIQyXRPTWjfgaWTyVGr0bGTnxg6vwhzfI/9jzy + q59xv29OuSY1dxmY31f0pZ9OOw3mabWksjoO2TexfKoxqsRHJ8PrM1f8E84Z4xo2 + TJXGzpuIxRkAJ+sVDqKEAhrKAfRYMSgdJ7zRt8VXv9ngjX20uA2m092NcH0Kmeto + TmuWUtK8E/qcN7ULN8xRWNUn4hu6mG6mayk4XliGRqI1VZupqh+MgNqHznuTd0bA + YrQsFPw9HaZ2hvVnJffJ5l7njAekZNOL+wIDAQABoyEwHzAdBgNVHQ4EFgQUcdiD + N6aylI91kAd34Hl2AzWY51QwDQYJKoZIhvcNAQELBQADggEBAG9q29ilUgCWQP5v + iHkZHj10gXGEoMkdfrPBf8grC7dpUcaw1Qfku/DJ7kPvMALeEsmFDk/t78roeNbh + IYBLJlzI1HZN6VPtpWQGsqxltAy5XN9Xw9mQM/tu70ShgsodGmE1UoW6eE5+/GMv + 6Fg+zLuICPvs2cFNmWUvukN5LW146tJSYCv0Q/rCPB3m9dNQ9pBxrzPUHXw4glwG + qGnGddXmOC+tSW5lDLLG1BRbKv4zxv3UlrtIjqlJtZb/sQMT6WtG2ihAz7SKOBHa + HOWUwuPTetWIuJCKP7P4mWWtmSmjLy+BFX5seNEngn3RzJ2L8uuTJQ/88OsqgGru + n3MVF9w= + -----END CERTIFICATE----- + private_key_password: "cassandra" + trusted_certificates: | + -----BEGIN CERTIFICATE----- + MIIDkTCCAnmgAwIBAgIETxH5JDANBgkqhkiG9w0BAQsFADB5MRAwDgYDVQQGEwdV + bmtub3duMRAwDgYDVQQIEwdVbmtub3duMRAwDgYDVQQHEwdVbmtub3duMRAwDgYD + VQQKEwdVbmtub3duMRQwEgYDVQQLDAtzc2xfdGVzdGluZzEZMBcGA1UEAxMQQXBh + Y2hlIENhc3NhbmRyYTAeFw0xNjAzMTgyMTI4MDJaFw0xNjA2MTYyMTI4MDJaMHkx + EDAOBgNVBAYTB1Vua25vd24xEDAOBgNVBAgTB1Vua25vd24xEDAOBgNVBAcTB1Vu + a25vd24xEDAOBgNVBAoTB1Vua25vd24xFDASBgNVBAsMC3NzbF90ZXN0aW5nMRkw + FwYDVQQDExBBcGFjaGUgQ2Fzc2FuZHJhMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8A + MIIBCgKCAQEAjkmVX/HS49cS8Hn6o26IGwMIcEV3d7ZhH0GNcx8rnSRd10dU9F6d + ugSjbwGFMcWUQzYNejN6az0Wb8JIQyXRPTWjfgaWTyVGr0bGTnxg6vwhzfI/9jzy + q59xv29OuSY1dxmY31f0pZ9OOw3mabWksjoO2TexfKoxqsRHJ8PrM1f8E84Z4xo2 + TJXGzpuIxRkAJ+sVDqKEAhrKAfRYMSgdJ7zRt8VXv9ngjX20uA2m092NcH0Kmeto + TmuWUtK8E/qcN7ULN8xRWNUn4hu6mG6mayk4XliGRqI1VZupqh+MgNqHznuTd0bA + YrQsFPw9HaZ2hvVnJffJ5l7njAekZNOL+wIDAQABoyEwHzAdBgNVHQ4EFgQUcdiD + N6aylI91kAd34Hl2AzWY51QwDQYJKoZIhvcNAQELBQADggEBAG9q29ilUgCWQP5v + iHkZHj10gXGEoMkdfrPBf8grC7dpUcaw1Qfku/DJ7kPvMALeEsmFDk/t78roeNbh + IYBLJlzI1HZN6VPtpWQGsqxltAy5XN9Xw9mQM/tu70ShgsodGmE1UoW6eE5+/GMv + 6Fg+zLuICPvs2cFNmWUvukN5LW146tJSYCv0Q/rCPB3m9dNQ9pBxrzPUHXw4glwG + qGnGddXmOC+tSW5lDLLG1BRbKv4zxv3UlrtIjqlJtZb/sQMT6WtG2ihAz7SKOBHa + HOWUwuPTetWIuJCKP7P4mWWtmSmjLy+BFX5seNEngn3RzJ2L8uuTJQ/88OsqgGru + n3MVF9w= + -----END CERTIFICATE----- +incremental_backups: true +concurrent_compactors: 4 +compaction_throughput: 0MiB/s +row_cache_class_name: org.apache.cassandra.cache.OHCProvider +row_cache_size: 16MiB +user_defined_functions_enabled: true +scripted_user_defined_functions_enabled: false +prepared_statements_cache_size: 1MiB +corrupted_tombstone_strategy: exception +stream_entire_sstables: true +stream_throughput_outbound: 24MiB/s +sasi_indexes_enabled: true +materialized_views_enabled: true +file_cache_enabled: true diff --git a/test/conf/cassandra-jmx-pem-sslconfig.yaml b/test/conf/cassandra-jmx-pem-sslconfig.yaml new file mode 100644 index 000000000000..45f58ffda272 --- /dev/null +++ b/test/conf/cassandra-jmx-pem-sslconfig.yaml @@ -0,0 +1,147 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# +# Testing for pluggable ssl_context_factory option for client and server encryption options with a valid and a missing +# implementation classes. +# +cluster_name: Test Cluster +# memtable_allocation_type: heap_buffers +memtable_allocation_type: offheap_objects +commitlog_sync: batch +commitlog_segment_size: 5MiB +commitlog_directory: build/test/cassandra/commitlog +# commitlog_compression: +# - class_name: LZ4Compressor +cdc_raw_directory: build/test/cassandra/cdc_raw +cdc_enabled: false +hints_directory: build/test/cassandra/hints +partitioner: org.apache.cassandra.dht.ByteOrderedPartitioner +listen_address: 127.0.0.1 +storage_port: 7012 +ssl_storage_port: 17012 +start_native_transport: true +native_transport_port: 9042 +column_index_size: 4KiB +saved_caches_directory: build/test/cassandra/saved_caches +data_file_directories: + - build/test/cassandra/data +disk_access_mode: mmap_index_only +seed_provider: + - class_name: org.apache.cassandra.locator.SimpleSeedProvider + parameters: + - seeds: "127.0.0.1:7012" +endpoint_snitch: org.apache.cassandra.locator.SimpleSnitch +dynamic_snitch: true +jmx_encryption_options: + enabled: true + cipher_suites: [TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256] + accepted_protocols: [TLSv1.2,TLSv1.3,TLSv1.1] + ssl_context_factory: + class_name: org.apache.cassandra.security.PEMBasedSslContextFactory + parameters: + private_key: | + -----BEGIN ENCRYPTED PRIVATE KEY----- + MIIE6jAcBgoqhkiG9w0BDAEDMA4ECOWqSzq5PBIdAgIFxQSCBMjXsCK30J0aT3J/ + g5kcbmevTOY1pIhJGbf5QYYrMUPiuDK2ydxIbiPzoTE4/S+OkCeHhlqwn/YydpBl + xgjZZ1Z5rLJHO27d2biuESqanDiBVXYuVmHmaifRnFy0uUTFkStB5mjVZEiJgO29 + L83hL60uWru71EVuVriC2WCfmZ/EXp6wyYszOqCFQ8Quk/rDO6XuaBl467MJbx5V + sucGT6E9XKNd9hB14/Izb2jtVM5kqKxoiHpz1na6yhEYJiE5D1uOonznWjBnjwB/ + f0x+acpDfVDoJKTlRdz+DEcbOF7mb9lBVVjP6P/AAsmQzz6JKwHjvCrjYfQmyyN8 + RI4KRQnWgm4L3dtByLqY8HFU4ogisCMCgI+hZQ+OKMz/hoRO540YGiPcTRY3EOUR + 0bd5JxU6tCJDMTqKP9aSL2KmLoiLowdMkSPz7TCzLsZ2bGJemuCfpAs4XT1vXCHs + evrUbOnh8et1IA8mZ9auThfqsZtNagJLEXA6hWIKp1FfVL3Q49wvMKZt4eTn/zwU + tLL0m5yPo6/HAaOA3hbm/oghZS0dseshXl7PZrmZQtvYnIvjyoxEL7ducYDQCDP6 + wZ7Nzyh1QZAauSS15hl3vLFRZCA9hWAVgwQAviTvhB342O0i9qI7TQkcHk+qcTPN + K+iGNbFZ8ma1izXNKSJ2PgI/QqFNIeJWvZrb9PhJRmaZVsTJ9fERm1ewpebZqkVv + zMqMhlKgx9ggAaSKgnGZkwXwB6GrSbbzUrwRCKm3FieD1QE4VVYevaadVUU75GG5 + mrFKorJEH7kFZlic8OTjDksYnHbcgU36XZrGEXa2+ldVeGKL3CsXWciaQRcJg8yo + WQDjZpcutGI0eMJWCqUkv8pYZC2/wZU4htCve5nVJUU4t9uuo9ex7lnwlLWPvheQ + jUBMgzSRsZ+zwaIusvufAAxiKK/cJm4ubZSZPIjBbfd4U7VPxtirP4Accydu7EK6 + eG/MZwtAMFNJxfxUR+/aYzJU/q1ePw7fWVHrpt58t/22CX2SJBEiUGmSmuyER4Ny + DPw6d6mhvPUS1jRhIZ9A81ht8MOX7VL5uVp307rt7o5vRpV1mo0iPiRHzGscMpJn + AP36klEAUNTf0uLTKZa7KHiwhn5iPmsCrENHkOKJjxhRrqHjD2wy3YHs3ow2voyY + Ua4Cids+c1hvRkNEDGNHm4+rKGFOGOsG/ZU7uj/6gflO4JXxNGiyTLflqMdWBvow + Zd7hk1zCaGAAn8nZ0hPweGxQ4Q30I9IBZrimGxB0vjiUqNio9+qMf33dCHFJEuut + ZGJMaUGVaPhXQcTy4uD5hzsPZV5xcsU4H3vBYyBcZgrusJ6OOgkuZQaU7p8rWQWr + bUEVbXuZdwEmxsCe7H/vEVv5+aA4sF4kWnMMFL7/LIYaiEzkTqdJlRv/KyJJgcAH + hg2BvR3XTAq8wiX0C98CdmTbsx2eyQdj5tCU606rEohFLKUxWkJYAKxCiUbxGGpI + RheVmxkef9ErxJiq7hsAsGrSJvMtJuDKIasnD14SOEwD/7jRAq6WdL9VLpxtzlOw + pWnIl8kUCO3WoaG9Jf+ZTIv2hnxJhaSzYrdXzGPNnaWKhBlwnXJRvQEdrIxZOimP + FujZhqbKUDbYAcqTkoQ= + -----END ENCRYPTED PRIVATE KEY----- + -----BEGIN CERTIFICATE----- + MIIDkTCCAnmgAwIBAgIETxH5JDANBgkqhkiG9w0BAQsFADB5MRAwDgYDVQQGEwdV + bmtub3duMRAwDgYDVQQIEwdVbmtub3duMRAwDgYDVQQHEwdVbmtub3duMRAwDgYD + VQQKEwdVbmtub3duMRQwEgYDVQQLDAtzc2xfdGVzdGluZzEZMBcGA1UEAxMQQXBh + Y2hlIENhc3NhbmRyYTAeFw0xNjAzMTgyMTI4MDJaFw0xNjA2MTYyMTI4MDJaMHkx + EDAOBgNVBAYTB1Vua25vd24xEDAOBgNVBAgTB1Vua25vd24xEDAOBgNVBAcTB1Vu + a25vd24xEDAOBgNVBAoTB1Vua25vd24xFDASBgNVBAsMC3NzbF90ZXN0aW5nMRkw + FwYDVQQDExBBcGFjaGUgQ2Fzc2FuZHJhMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8A + MIIBCgKCAQEAjkmVX/HS49cS8Hn6o26IGwMIcEV3d7ZhH0GNcx8rnSRd10dU9F6d + ugSjbwGFMcWUQzYNejN6az0Wb8JIQyXRPTWjfgaWTyVGr0bGTnxg6vwhzfI/9jzy + q59xv29OuSY1dxmY31f0pZ9OOw3mabWksjoO2TexfKoxqsRHJ8PrM1f8E84Z4xo2 + TJXGzpuIxRkAJ+sVDqKEAhrKAfRYMSgdJ7zRt8VXv9ngjX20uA2m092NcH0Kmeto + TmuWUtK8E/qcN7ULN8xRWNUn4hu6mG6mayk4XliGRqI1VZupqh+MgNqHznuTd0bA + YrQsFPw9HaZ2hvVnJffJ5l7njAekZNOL+wIDAQABoyEwHzAdBgNVHQ4EFgQUcdiD + N6aylI91kAd34Hl2AzWY51QwDQYJKoZIhvcNAQELBQADggEBAG9q29ilUgCWQP5v + iHkZHj10gXGEoMkdfrPBf8grC7dpUcaw1Qfku/DJ7kPvMALeEsmFDk/t78roeNbh + IYBLJlzI1HZN6VPtpWQGsqxltAy5XN9Xw9mQM/tu70ShgsodGmE1UoW6eE5+/GMv + 6Fg+zLuICPvs2cFNmWUvukN5LW146tJSYCv0Q/rCPB3m9dNQ9pBxrzPUHXw4glwG + qGnGddXmOC+tSW5lDLLG1BRbKv4zxv3UlrtIjqlJtZb/sQMT6WtG2ihAz7SKOBHa + HOWUwuPTetWIuJCKP7P4mWWtmSmjLy+BFX5seNEngn3RzJ2L8uuTJQ/88OsqgGru + n3MVF9w= + -----END CERTIFICATE----- + private_key_password: "cassandra" + trusted_certificates: | + -----BEGIN CERTIFICATE----- + MIIDkTCCAnmgAwIBAgIETxH5JDANBgkqhkiG9w0BAQsFADB5MRAwDgYDVQQGEwdV + bmtub3duMRAwDgYDVQQIEwdVbmtub3duMRAwDgYDVQQHEwdVbmtub3duMRAwDgYD + VQQKEwdVbmtub3duMRQwEgYDVQQLDAtzc2xfdGVzdGluZzEZMBcGA1UEAxMQQXBh + Y2hlIENhc3NhbmRyYTAeFw0xNjAzMTgyMTI4MDJaFw0xNjA2MTYyMTI4MDJaMHkx + EDAOBgNVBAYTB1Vua25vd24xEDAOBgNVBAgTB1Vua25vd24xEDAOBgNVBAcTB1Vu + a25vd24xEDAOBgNVBAoTB1Vua25vd24xFDASBgNVBAsMC3NzbF90ZXN0aW5nMRkw + FwYDVQQDExBBcGFjaGUgQ2Fzc2FuZHJhMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8A + MIIBCgKCAQEAjkmVX/HS49cS8Hn6o26IGwMIcEV3d7ZhH0GNcx8rnSRd10dU9F6d + ugSjbwGFMcWUQzYNejN6az0Wb8JIQyXRPTWjfgaWTyVGr0bGTnxg6vwhzfI/9jzy + q59xv29OuSY1dxmY31f0pZ9OOw3mabWksjoO2TexfKoxqsRHJ8PrM1f8E84Z4xo2 + TJXGzpuIxRkAJ+sVDqKEAhrKAfRYMSgdJ7zRt8VXv9ngjX20uA2m092NcH0Kmeto + TmuWUtK8E/qcN7ULN8xRWNUn4hu6mG6mayk4XliGRqI1VZupqh+MgNqHznuTd0bA + YrQsFPw9HaZ2hvVnJffJ5l7njAekZNOL+wIDAQABoyEwHzAdBgNVHQ4EFgQUcdiD + N6aylI91kAd34Hl2AzWY51QwDQYJKoZIhvcNAQELBQADggEBAG9q29ilUgCWQP5v + iHkZHj10gXGEoMkdfrPBf8grC7dpUcaw1Qfku/DJ7kPvMALeEsmFDk/t78roeNbh + IYBLJlzI1HZN6VPtpWQGsqxltAy5XN9Xw9mQM/tu70ShgsodGmE1UoW6eE5+/GMv + 6Fg+zLuICPvs2cFNmWUvukN5LW146tJSYCv0Q/rCPB3m9dNQ9pBxrzPUHXw4glwG + qGnGddXmOC+tSW5lDLLG1BRbKv4zxv3UlrtIjqlJtZb/sQMT6WtG2ihAz7SKOBHa + HOWUwuPTetWIuJCKP7P4mWWtmSmjLy+BFX5seNEngn3RzJ2L8uuTJQ/88OsqgGru + n3MVF9w= + -----END CERTIFICATE----- +incremental_backups: true +concurrent_compactors: 4 +compaction_throughput: 0MiB/s +row_cache_class_name: org.apache.cassandra.cache.OHCProvider +row_cache_size: 16MiB +user_defined_functions_enabled: true +scripted_user_defined_functions_enabled: false +prepared_statements_cache_size: 1MiB +corrupted_tombstone_strategy: exception +stream_entire_sstables: true +stream_throughput_outbound: 24MiB/s +sasi_indexes_enabled: true +materialized_views_enabled: true +file_cache_enabled: true diff --git a/test/conf/cassandra-jmx-sslconfig.yaml b/test/conf/cassandra-jmx-sslconfig.yaml new file mode 100644 index 000000000000..317d5236b62a --- /dev/null +++ b/test/conf/cassandra-jmx-sslconfig.yaml @@ -0,0 +1,72 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# +# Testing for pluggable ssl_context_factory option for client and server encryption options with a valid and a missing +# implementation classes. +# +cluster_name: Test Cluster +# memtable_allocation_type: heap_buffers +memtable_allocation_type: offheap_objects +commitlog_sync: batch +commitlog_segment_size: 5MiB +commitlog_directory: build/test/cassandra/commitlog +# commitlog_compression: +# - class_name: LZ4Compressor +cdc_raw_directory: build/test/cassandra/cdc_raw +cdc_enabled: false +hints_directory: build/test/cassandra/hints +partitioner: org.apache.cassandra.dht.ByteOrderedPartitioner +listen_address: 127.0.0.1 +storage_port: 7012 +ssl_storage_port: 17012 +start_native_transport: true +native_transport_port: 9042 +column_index_size: 4KiB +saved_caches_directory: build/test/cassandra/saved_caches +data_file_directories: + - build/test/cassandra/data +disk_access_mode: mmap_index_only +seed_provider: + - class_name: org.apache.cassandra.locator.SimpleSeedProvider + parameters: + - seeds: "127.0.0.1:7012" +endpoint_snitch: org.apache.cassandra.locator.SimpleSnitch +dynamic_snitch: true +jmx_encryption_options: + enabled: true + cipher_suites: [TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256] + accepted_protocols: [TLSv1.2,TLSv1.3,TLSv1.1] + keystore: test/conf/cassandra_ssl_test.keystore + keystore_password: cassandra + truststore: test/conf/cassandra_ssl_test.truststore + truststore_password: cassandra +incremental_backups: true +concurrent_compactors: 4 +compaction_throughput: 0MiB/s +row_cache_class_name: org.apache.cassandra.cache.OHCProvider +row_cache_size: 16MiB +user_defined_functions_enabled: true +scripted_user_defined_functions_enabled: false +prepared_statements_cache_size: 1MiB +corrupted_tombstone_strategy: exception +stream_entire_sstables: true +stream_throughput_outbound: 24MiB/s +sasi_indexes_enabled: true +materialized_views_enabled: true +file_cache_enabled: true diff --git a/test/distributed/org/apache/cassandra/distributed/impl/CollectingRMIServerSocketFactoryImpl.java b/test/distributed/org/apache/cassandra/distributed/impl/CollectingRMIServerSocketFactoryImpl.java index 0fc742513f6b..bba536f67f4e 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/CollectingRMIServerSocketFactoryImpl.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/CollectingRMIServerSocketFactoryImpl.java @@ -22,20 +22,21 @@ import java.net.InetAddress; import java.net.ServerSocket; import java.net.SocketException; -import java.rmi.server.RMIServerSocketFactory; import java.util.ArrayList; import java.util.List; import java.util.Objects; import javax.net.ServerSocketFactory; +import org.apache.cassandra.utils.RMICloseableServerSocketFactory; + /** * This class is used to keep track of RMI servers created during a cluster creation so we can * later close the sockets, which would otherwise be left with a thread running waiting for * connections that would never show up as the server was otherwise closed. */ -class CollectingRMIServerSocketFactoryImpl implements RMIServerSocketFactory +class CollectingRMIServerSocketFactoryImpl implements RMICloseableServerSocketFactory { private final InetAddress bindAddress; List<ServerSocket> sockets = new ArrayList<>(); @@ -62,7 +63,7 @@ public ServerSocket createServerSocket(int pPort) throws IOException return result; } - + @Override public void close() throws IOException { for (ServerSocket socket : sockets) diff --git a/test/distributed/org/apache/cassandra/distributed/impl/CollectingSslRMIServerSocketFactoryImpl.java b/test/distributed/org/apache/cassandra/distributed/impl/CollectingSslRMIServerSocketFactoryImpl.java new file mode 100644 index 000000000000..01d051e4cf07 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/impl/CollectingSslRMIServerSocketFactoryImpl.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.impl; + +import java.io.IOException; +import java.net.InetAddress; +import java.net.ServerSocket; +import java.net.Socket; +import java.net.SocketException; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLSocket; +import javax.net.ssl.SSLSocketFactory; + +import org.apache.cassandra.utils.RMICloseableServerSocketFactory; + + +/** + * This class is used to keep track of SSL based RMI servers created during a cluster creation to + * later close the sockets, which would otherwise be left with a thread running waiting for + * connections that would never show up as the server was otherwise closed. + */ +class CollectingSslRMIServerSocketFactoryImpl implements RMICloseableServerSocketFactory +{ + private final InetAddress bindAddress; + private final String[] enabledCipherSuites; + private final String[] enabledProtocols; + private final boolean needClientAuth; + private final SSLSocketFactory sslSocketFactory; + List<ServerSocket> sockets = new ArrayList<>(); + + public CollectingSslRMIServerSocketFactoryImpl(InetAddress bindAddress, String[] enabledCipherSuites, + String[] enabledProtocols, boolean needClientAuth, SSLContext sslContext) + { + this.bindAddress = bindAddress; + this.enabledCipherSuites = enabledCipherSuites; + this.enabledProtocols = enabledProtocols; + this.needClientAuth = needClientAuth; + this.sslSocketFactory = sslContext.getSocketFactory(); + } + + public CollectingSslRMIServerSocketFactoryImpl(InetAddress bindAddress, String[] enabledCipherSuites, + String[] enabledProtocols, boolean needClientAuth) + { + this.bindAddress = bindAddress; + this.enabledCipherSuites = enabledCipherSuites; + this.enabledProtocols = enabledProtocols; + this.needClientAuth = needClientAuth; + this.sslSocketFactory = getDefaultSSLSocketFactory(); + } + + public String[] getEnabledCipherSuites() + { + return enabledCipherSuites; + } + + public String[] getEnabledProtocols() + { + return enabledProtocols; + } + + public boolean isNeedClientAuth() + { + return needClientAuth; + } + + @Override + public ServerSocket createServerSocket(int pPort) throws IOException + { + ServerSocket result = createSslServerSocket(pPort); + try + { + result.setReuseAddress(true); + } + catch (SocketException e) + { + result.close(); + throw e; + } + sockets.add(result); + return result; + } + + private ServerSocket createSslServerSocket(int pPort) throws IOException + { + return new ServerSocket(pPort, 0, bindAddress) + { + public Socket accept() throws IOException + { + Socket socket = super.accept(); + SSLSocket sslSocket = (SSLSocket) sslSocketFactory.createSocket( + socket, socket.getInetAddress().getHostName(), + socket.getPort(), true); + sslSocket.setUseClientMode(false); + if (enabledCipherSuites != null) + { + sslSocket.setEnabledCipherSuites(enabledCipherSuites); + } + if (enabledProtocols != null) + { + sslSocket.setEnabledProtocols(enabledProtocols); + } + sslSocket.setNeedClientAuth(needClientAuth); + return sslSocket; + } + }; + } + + @Override + public void close() throws IOException + { + for (ServerSocket socket : sockets) + { + socket.close(); + } + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + CollectingSslRMIServerSocketFactoryImpl that = (CollectingSslRMIServerSocketFactoryImpl) o; + return Objects.equals(bindAddress, that.bindAddress); + } + + @Override + public int hashCode() + { + return Objects.hash(bindAddress); + } + + private static SSLSocketFactory defaultSSLSocketFactory = null; + + private static synchronized SSLSocketFactory getDefaultSSLSocketFactory() + { + if (defaultSSLSocketFactory == null) + defaultSSLSocketFactory = + (SSLSocketFactory) SSLSocketFactory.getDefault(); + return defaultSSLSocketFactory; + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java index fe4a25be4309..7b5501fbad5c 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java @@ -708,9 +708,9 @@ protected void partialStartup(ICluster<?> cluster) throws IOException, NoSuchFie assert config.networkTopology().contains(config.broadcastAddress()) : String.format("Network topology %s doesn't contain the address %s", config.networkTopology(), config.broadcastAddress()); DistributedTestSnitch.assign(config.networkTopology()); + DatabaseDescriptor.daemonInitialization(); if (config.has(JMX)) startJmx(); - DatabaseDescriptor.daemonInitialization(); LoggingSupportFactory.getLoggingSupport().onStartup(); logSystemInfo(inInstancelogger); diff --git a/test/distributed/org/apache/cassandra/distributed/impl/IsolatedJmx.java b/test/distributed/org/apache/cassandra/distributed/impl/IsolatedJmx.java index 7812ec89f82d..6a575972eb35 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/IsolatedJmx.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/IsolatedJmx.java @@ -22,9 +22,9 @@ import java.net.InetAddress; import java.util.HashMap; import java.util.LinkedList; +import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; - import javax.management.remote.JMXConnector; import javax.management.remote.JMXConnectorServer; import javax.management.remote.JMXServiceURL; @@ -34,12 +34,14 @@ import com.google.common.util.concurrent.Uninterruptibles; import org.slf4j.Logger; +import org.apache.cassandra.config.EncryptionOptions; import org.apache.cassandra.distributed.api.IInstance; import org.apache.cassandra.distributed.api.IInstanceConfig; import org.apache.cassandra.distributed.shared.JMXUtil; import org.apache.cassandra.utils.JMXServerUtils; import org.apache.cassandra.utils.MBeanWrapper; -import org.apache.cassandra.utils.RMIClientSocketFactoryImpl; +import org.apache.cassandra.utils.RMICloseableClientSocketFactory; +import org.apache.cassandra.utils.RMICloseableServerSocketFactory; import sun.rmi.transport.tcp.TCPEndpoint; import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_RMI_DGC_LEASE_VALUE_IN_JVM_DTEST; @@ -57,17 +59,20 @@ public class IsolatedJmx private JMXServerUtils.JmxRegistry registry; private RMIJRMPServerImpl jmxRmiServer; private MBeanWrapper.InstanceMBeanWrapper wrapper; - private RMIClientSocketFactoryImpl clientSocketFactory; - private CollectingRMIServerSocketFactoryImpl serverSocketFactory; + private RMICloseableClientSocketFactory clientSocketFactory; + private RMICloseableServerSocketFactory serverSocketFactory; private Logger inInstancelogger; private IInstanceConfig config; - public IsolatedJmx(IInstance instance, Logger inInstanceLogger) { + public IsolatedJmx(IInstance instance, Logger inInstanceLogger) + { this.config = instance.config(); this.inInstancelogger = inInstanceLogger; } - public void startJmx() { + @SuppressWarnings("unchecked") + public void startJmx() + { try { // Several RMI threads hold references to in-jvm dtest objects, and are, by default, kept @@ -84,14 +89,17 @@ public void startJmx() { String hostname = addr.getHostAddress(); wrapper = new MBeanWrapper.InstanceMBeanWrapper(hostname + ":" + jmxPort); ((MBeanWrapper.DelegatingMbeanWrapper) MBeanWrapper.instance).setDelegate(wrapper); - Map<String, Object> env = new HashMap<>(); - serverSocketFactory = new CollectingRMIServerSocketFactoryImpl(addr); - env.put(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE, - serverSocketFactory); - clientSocketFactory = new RMIClientSocketFactoryImpl(addr); - env.put(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE, - clientSocketFactory); + // CASSANDRA-18508: Sensitive JMX SSL configuration options can be easily exposed + Map<String, Object> encryptionOptionsMap = (Map<String, Object>) config.getParams().get("jmx_encryption_options"); + EncryptionOptions jmxEncryptionOptions = getJmxEncryptionOptions(encryptionOptionsMap); + // Here the `localOnly` is always passed as true as it is for the local isolated JMX testing + // However if the `jmxEncryptionOptions` are provided or JMX SSL configuration is set it will configure + // the socket factories appropriately. + Map<String, Object> socketFactories = new IsolatedJmxSocketFactory().configure(addr, true, jmxEncryptionOptions); + serverSocketFactory = (RMICloseableServerSocketFactory) socketFactories.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE); + clientSocketFactory = (RMICloseableClientSocketFactory) socketFactories.get(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE); + Map<String, Object> env = new HashMap<>(socketFactories); // configure the RMI registry registry = new JMXServerUtils.JmxRegistry(jmxPort, @@ -137,6 +145,49 @@ public void startJmx() { } } + /** + * Builds {@code EncryptionOptions} from the map based SSL configuration properties. + * + * @param encryptionOptionsMap of SSL configuration properties + * @return EncryptionOptions built object + */ + @SuppressWarnings("unchecked") + private EncryptionOptions getJmxEncryptionOptions(Map<String, Object> encryptionOptionsMap) + { + if (encryptionOptionsMap == null) + { + return null; + } + EncryptionOptions jmxEncryptionOptions = new EncryptionOptions(); + String[] cipherSuitesArray = (String[]) encryptionOptionsMap.get(EncryptionOptions.ConfigKey.CIPHER_SUITES.toString()); + if (cipherSuitesArray != null) + { + jmxEncryptionOptions = jmxEncryptionOptions.withCipherSuites(cipherSuitesArray); + } + List<String> acceptedProtocols = (List<String>) encryptionOptionsMap.get(EncryptionOptions.ConfigKey.ACCEPTED_PROTOCOLS.toString()); + if (acceptedProtocols != null) + { + jmxEncryptionOptions = jmxEncryptionOptions.withAcceptedProtocols(acceptedProtocols); + } + + Boolean requireClientAuthValue = (Boolean) encryptionOptionsMap.get(EncryptionOptions.ConfigKey.REQUIRE_CLIENT_AUTH.toString()); + EncryptionOptions.ClientAuth requireClientAuth = requireClientAuthValue == null ? + EncryptionOptions.ClientAuth.NOT_REQUIRED : + EncryptionOptions.ClientAuth.from(String.valueOf(requireClientAuthValue)); + Object enabledOption = encryptionOptionsMap.get(EncryptionOptions.ConfigKey.ENABLED.toString()); + boolean enabled = enabledOption != null ? (Boolean) encryptionOptionsMap.get(EncryptionOptions.ConfigKey.ENABLED.toString()) : false; + + //CASSANDRA-18508 NOTE - We do not populate sslContextFactory configuration here for tests, it could be enhanced + jmxEncryptionOptions = jmxEncryptionOptions + .withKeyStore((String) encryptionOptionsMap.get(EncryptionOptions.ConfigKey.KEYSTORE.toString())) + .withKeyStorePassword((String) encryptionOptionsMap.get(EncryptionOptions.ConfigKey.KEYSTORE_PASSWORD.toString())) + .withTrustStore((String) encryptionOptionsMap.get(EncryptionOptions.ConfigKey.TRUSTSTORE.toString())) + .withTrustStorePassword((String) encryptionOptionsMap.get(EncryptionOptions.ConfigKey.TRUSTSTORE_PASSWORD.toString())) + .withRequireClientAuth(requireClientAuth) + .withEnabled(enabled); + return jmxEncryptionOptions; + } + private void waitForJmxAvailability(Map<String, ?> env) { try (JMXConnector ignored = JMXUtil.getJmxConnector(config, 20, env)) diff --git a/test/distributed/org/apache/cassandra/distributed/impl/IsolatedJmxSocketFactory.java b/test/distributed/org/apache/cassandra/distributed/impl/IsolatedJmxSocketFactory.java new file mode 100644 index 000000000000..868cb8ae4b28 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/impl/IsolatedJmxSocketFactory.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.impl; + +import java.net.InetAddress; +import java.util.Arrays; +import java.util.Map; +import java.util.stream.Collectors; +import javax.management.remote.rmi.RMIConnectorServer; +import javax.net.ssl.SSLContext; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.utils.RMIClientSocketFactoryImpl; +import org.apache.cassandra.utils.jmx.AbstractJmxSocketFactory; + +/** + * JMX Socket factory used for the isolated JMX testing. + */ +public class IsolatedJmxSocketFactory extends AbstractJmxSocketFactory +{ + private static final Logger logger = LoggerFactory.getLogger(IsolatedJmxSocketFactory.class); + + @Override + public void configureLocalSocketFactories(Map<String, Object> env, InetAddress serverAddress) + { + CollectingRMIServerSocketFactoryImpl serverSocketFactory = new CollectingRMIServerSocketFactoryImpl(serverAddress); + env.put(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE, + serverSocketFactory); + RMIClientSocketFactoryImpl clientSocketFactory = new RMIClientSocketFactoryImpl(serverAddress); + env.put(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE, + clientSocketFactory); + } + + @Override + public void configureSslClientSocketFactory(Map<String, Object> env, InetAddress serverAddress) + { + RMISslClientSocketFactoryImpl clientFactory = new RMISslClientSocketFactoryImpl(serverAddress, + CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES.getString(), + CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS.getString()); + env.put(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE, clientFactory); + env.put("com.sun.jndi.rmi.factory.socket", clientFactory); + } + + @Override + public void configureSslServerSocketFactory(Map<String, Object> env, InetAddress serverAddress, String[] enabledCipherSuites, + String[] enabledProtocols, boolean needClientAuth) + { + CollectingSslRMIServerSocketFactoryImpl serverFactory = new CollectingSslRMIServerSocketFactoryImpl(serverAddress, + enabledCipherSuites, + enabledProtocols, + needClientAuth); + env.put(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE, serverFactory); + logJmxSslConfig(serverFactory); + } + + @Override + public void configureSslServerSocketFactory(Map<String, Object> env, InetAddress serverAddress, String[] enabledCipherSuites, + String[] enabledProtocols, boolean needClientAuth, SSLContext sslContext) + { + CollectingSslRMIServerSocketFactoryImpl serverFactory = new CollectingSslRMIServerSocketFactoryImpl(serverAddress, + enabledCipherSuites, + enabledProtocols, + needClientAuth, + sslContext); + env.put(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE, serverFactory); + logJmxSslConfig(serverFactory); + } + + private void logJmxSslConfig(CollectingSslRMIServerSocketFactoryImpl serverFactory) + { + if (logger.isDebugEnabled()) + logger.debug("JMX SSL configuration. { protocols: [{}], cipher_suites: [{}], require_client_auth: {} }", + serverFactory.getEnabledProtocols() == null ? "'JVM defaults'" : Arrays.stream(serverFactory.getEnabledProtocols()).collect(Collectors.joining("','", "'", "'")), + serverFactory.getEnabledCipherSuites() == null ? "'JVM defaults'" : Arrays.stream(serverFactory.getEnabledCipherSuites()).collect(Collectors.joining("','", "'", "'")), + serverFactory.isNeedClientAuth()); + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/impl/JmxTestClientSslContextFactory.java b/test/distributed/org/apache/cassandra/distributed/impl/JmxTestClientSslContextFactory.java new file mode 100644 index 000000000000..8247d8e9d4bd --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/impl/JmxTestClientSslContextFactory.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.impl; + +import java.io.InputStream; +import java.nio.file.Files; +import java.security.KeyStore; +import java.util.Map; +import javax.net.ssl.KeyManagerFactory; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLException; +import javax.net.ssl.TrustManager; +import javax.net.ssl.TrustManagerFactory; + +import org.apache.commons.lang3.StringUtils; + +import org.apache.cassandra.config.EncryptionOptions; +import org.apache.cassandra.io.util.File; + +/** + * Simplified and independent version of {@link org.apache.cassandra.security.FileBasedSslContextFactory} for + * testing SSL based JMX clients that require configuring keystore and/or truststore. + */ +public class JmxTestClientSslContextFactory +{ + private final Map<String, Object> parameters; + // keystore is not needed when the JMX server does not require client-auth + private final String keystore; + // keystore could be null in case JMX server does not require client-auth + private final String keystore_password; + private final String truststore; + private final String truststore_password; + private final String protocol; + private final String algorithm; + private final String store_type; + + public JmxTestClientSslContextFactory(Map<String, Object> parameters) + { + this.parameters = parameters; + keystore = getString(EncryptionOptions.ConfigKey.KEYSTORE.toString()); + keystore_password = getString(EncryptionOptions.ConfigKey.KEYSTORE_PASSWORD.toString()); + truststore = getString(EncryptionOptions.ConfigKey.TRUSTSTORE.toString()); + truststore_password = getString(EncryptionOptions.ConfigKey.TRUSTSTORE_PASSWORD.toString()); + protocol = getString(EncryptionOptions.ConfigKey.PROTOCOL.toString(), "TLS"); + algorithm = getString(EncryptionOptions.ConfigKey.ALGORITHM.toString()); + store_type = getString(EncryptionOptions.ConfigKey.STORE_TYPE.toString(), "JKS"); + } + + private String getString(String key, String defaultValue) + { + return parameters.get(key) == null ? defaultValue : (String) parameters.get(key); + } + + private String getString(String key) + { + return (String) parameters.get(key); + } + + private TrustManagerFactory buildTrustManagerFactory() throws SSLException + { + try (InputStream tsf = Files.newInputStream(File.getPath(truststore))) + { + final String algorithm = this.algorithm == null ? TrustManagerFactory.getDefaultAlgorithm() : this.algorithm; + TrustManagerFactory tmf = TrustManagerFactory.getInstance(algorithm); + KeyStore ts = KeyStore.getInstance(store_type); + + final char[] truststorePassword = StringUtils.isEmpty(truststore_password) ? null : truststore_password.toCharArray(); + ts.load(tsf, truststorePassword); + tmf.init(ts); + return tmf; + } + catch (Exception e) + { + throw new SSLException("failed to build trust manager store for secure connections", e); + } + } + + private KeyManagerFactory buildKeyManagerFactory() throws SSLException + { + final String algorithm = this.algorithm == null ? KeyManagerFactory.getDefaultAlgorithm() : this.algorithm; + + if (keystore != null) + { + try (InputStream ksf = Files.newInputStream(File.getPath(keystore))) + { + KeyManagerFactory kmf = KeyManagerFactory.getInstance(algorithm); + KeyStore ks = KeyStore.getInstance(store_type); + final char[] password = keystore_password.toCharArray(); + ks.load(ksf, password); + kmf.init(ks, password); + return kmf; + } + catch (Exception e) + { + throw new SSLException("failed to build key manager store for secure connections", e); + } + } + else + { + return null; + } + } + + public SSLContext createSSLContext() throws SSLException + { + TrustManager[] trustManagers = buildTrustManagerFactory().getTrustManagers(); + KeyManagerFactory kmf = buildKeyManagerFactory(); + try + { + SSLContext ctx = SSLContext.getInstance(protocol); + ctx.init(kmf != null ? kmf.getKeyManagers() : null, trustManagers, null); + return ctx; + } + catch (Exception e) + { + throw new SSLException("Error creating/initializing the SSL Context", e); + } + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/impl/JmxTestClientSslSocketFactory.java b/test/distributed/org/apache/cassandra/distributed/impl/JmxTestClientSslSocketFactory.java new file mode 100644 index 000000000000..db802c031e9b --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/impl/JmxTestClientSslSocketFactory.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.impl; + +import java.io.IOException; +import java.io.Serializable; +import java.net.Socket; +import javax.net.SocketFactory; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLSocket; +import javax.net.ssl.SSLSocketFactory; +import javax.rmi.ssl.SslRMIClientSocketFactory; + +/** + * {@code RMIClientSocketFactory} for testing SSL based JMX clients. + */ +public class JmxTestClientSslSocketFactory extends SslRMIClientSocketFactory implements Serializable +{ + private static final long serialVersionUID = 818579127759449333L; + private final SocketFactory defaultSocketFactory; + private final String[] cipherSuites; + private final String[] acceptedProtocols; + + public JmxTestClientSslSocketFactory(SSLContext sslContext, String[] cipherSuites, String[] acceptedProtocols) + { + this.cipherSuites = cipherSuites; + this.acceptedProtocols = acceptedProtocols; + defaultSocketFactory = sslContext == null ? SSLSocketFactory.getDefault() : sslContext.getSocketFactory(); + } + + @Override + public Socket createSocket(String host, int port) throws IOException + { + final SSLSocket sslSocket = (SSLSocket) defaultSocketFactory.createSocket(host, port); + + if (cipherSuites != null) + sslSocket.setEnabledCipherSuites(cipherSuites); + + if (acceptedProtocols != null) + sslSocket.setEnabledProtocols(acceptedProtocols); + + return sslSocket; + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/impl/RMISslClientSocketFactoryImpl.java b/test/distributed/org/apache/cassandra/distributed/impl/RMISslClientSocketFactoryImpl.java new file mode 100644 index 000000000000..bca6b1fb6aad --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/impl/RMISslClientSocketFactoryImpl.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.impl; + +import java.io.IOException; +import java.io.Serializable; +import java.net.InetAddress; +import java.net.Socket; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.regex.Pattern; +import javax.net.SocketFactory; +import javax.net.ssl.SSLSocket; +import javax.net.ssl.SSLSocketFactory; + +import org.apache.cassandra.utils.RMICloseableClientSocketFactory; + +/** + * {@code RMIClientSocketFactory} for testing SSL based JMX clients. + * This class is used to override the local address the JMX client calculates when trying to connect, + * which can otherwise be influenced by the system property "java.rmi.server.hostname" in strange and + * unpredictable ways. + */ +public class RMISslClientSocketFactoryImpl implements Serializable, RMICloseableClientSocketFactory +{ + private static final long serialVersionUID = 9054380061905145241L; + private static final Pattern COMMA_SPLITTER = Pattern.compile(","); + private static final List<Socket> sockets = new ArrayList<>(); + private final InetAddress localAddress; + private final String[] enabledCipherSuites; + private final String[] enabledProtocols; + + public RMISslClientSocketFactoryImpl(InetAddress localAddress, String enabledCipherSuites, String enabledProtocls) + { + this.localAddress = localAddress; + this.enabledCipherSuites = splitCommaSeparatedString(enabledCipherSuites); + this.enabledProtocols = splitCommaSeparatedString(enabledProtocls); + } + + @Override + public Socket createSocket(String host, int port) throws IOException + { + Socket socket = createSslSocket(port); + sockets.add(socket); + return socket; + } + + private Socket createSslSocket(int port) throws IOException + { + final SocketFactory sslSocketFactory = SSLSocketFactory.getDefault(); + final SSLSocket sslSocket = (SSLSocket) sslSocketFactory.createSocket(localAddress, port); + if (enabledCipherSuites != null) + { + try + { + sslSocket.setEnabledCipherSuites(enabledCipherSuites); + } + catch (IllegalArgumentException e) + { + throw new IOException(e.getMessage(), e); + } + } + if (enabledProtocols != null) + { + try + { + sslSocket.setEnabledProtocols(enabledProtocols); + } + catch (IllegalArgumentException e) + { + throw new IOException(e.getMessage(), e); + } + } + return sslSocket; + } + + @Override + public void close() throws IOException + { + for (Socket socket : sockets) + { + try + { + socket.close(); + } + catch (IOException ignored) + { + // intentionally ignored + } + } + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + RMISslClientSocketFactoryImpl that = (RMISslClientSocketFactoryImpl) o; + return Objects.equals(localAddress, that.localAddress); + } + + @Override + public int hashCode() + { + return Objects.hash(localAddress); + } + + private String[] splitCommaSeparatedString(String stringToSplit) + { + if (stringToSplit == null) + return null; + return COMMA_SPLITTER.split(stringToSplit); + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/shared/WithProperties.java b/test/distributed/org/apache/cassandra/distributed/shared/WithProperties.java index d17d3e6f3af4..9c476525f8ed 100644 --- a/test/distributed/org/apache/cassandra/distributed/shared/WithProperties.java +++ b/test/distributed/org/apache/cassandra/distributed/shared/WithProperties.java @@ -72,6 +72,13 @@ public WithProperties set(CassandraRelevantProperties prop, long value) return set(prop, () -> prop.setLong(value)); } + public WithProperties preserve(CassandraRelevantProperties prop) + { + String previous = prop.getString(); // because all properties are strings + rollback.add(previous == null ? prop::clearValue : () -> prop.setString(previous)); + return this; + } + private void with(String key, String value) { String previous = System.setProperty(key, value); // checkstyle: suppress nearby 'blockSystemPropertyUsage' diff --git a/test/distributed/org/apache/cassandra/distributed/test/AbstractEncryptionOptionsImpl.java b/test/distributed/org/apache/cassandra/distributed/test/AbstractEncryptionOptionsImpl.java index 51af668c299d..25e9e64b6326 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/AbstractEncryptionOptionsImpl.java +++ b/test/distributed/org/apache/cassandra/distributed/test/AbstractEncryptionOptionsImpl.java @@ -70,19 +70,19 @@ public class AbstractEncryptionOptionsImpl extends TestBaseImpl final static String validTrustStorePassword = TlsTestUtils.SERVER_TRUSTSTORE_PASSWORD; // Base configuration map for a valid keystore that can be opened - final static Map<String,Object> validKeystore = ImmutableMap.of("keystore", validKeyStorePath, - "keystore_password", validKeyStorePassword, - "truststore", validTrustStorePath, - "truststore_password", validTrustStorePassword); + protected final static Map<String,Object> validFileBasedKeystores = ImmutableMap.of("keystore", validKeyStorePath, + "keystore_password", validKeyStorePassword, + "truststore", validTrustStorePath, + "truststore_password", validTrustStorePassword); // Configuration with a valid keystore, but an unknown protocol final static Map<String,Object> nonExistantProtocol = ImmutableMap.<String,Object>builder() - .putAll(validKeystore) + .putAll(validFileBasedKeystores) .put("accepted_protocols", Collections.singletonList("NoProtocolIKnow")) .build(); // Configuration with a valid keystore, but an unknown cipher suite final static Map<String,Object> nonExistantCipher = ImmutableMap.<String,Object>builder() - .putAll(validKeystore) + .putAll(validFileBasedKeystores) .put("cipher_suites", Collections.singletonList("NoCipherIKnow")) .build(); @@ -340,7 +340,7 @@ void assertReceivedHandshakeException() } /* Provde the cluster cannot start with the configured options */ - void assertCannotStartDueToConfigurationException(Cluster cluster) + protected void assertCannotStartDueToConfigurationException(Cluster cluster) { Throwable tr = null; try diff --git a/test/distributed/org/apache/cassandra/distributed/test/InternodeEncryptionOptionsTest.java b/test/distributed/org/apache/cassandra/distributed/test/InternodeEncryptionOptionsTest.java index 83bcaaad3c14..a955757c9b0e 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/InternodeEncryptionOptionsTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/InternodeEncryptionOptionsTest.java @@ -53,7 +53,7 @@ public void legacySslPortProvidedWithEncryptionNoneWillNotStartTest() throws Thr c.with(Feature.NETWORK); c.set("ssl_storage_port", 7013); c.set("server_encryption_options", - ImmutableMap.builder().putAll(validKeystore) + ImmutableMap.builder().putAll(validFileBasedKeystores) .put("internode_encryption", "none") .put("optional", false) .put("legacy_ssl_storage_port_enabled", "true") @@ -87,7 +87,7 @@ public void optionalTlsConnectionAllowedWithKeystoreTest() throws Throwable { try (Cluster cluster = builder().withNodes(1).withConfig(c -> { c.with(Feature.NETWORK); - c.set("server_encryption_options", validKeystore); + c.set("server_encryption_options", validFileBasedKeystores); }).createWithoutStarting()) { InetAddress address = cluster.get(1).config().broadcastAddress().getAddress(); @@ -111,7 +111,7 @@ public void optionalTlsConnectionAllowedToStoragePortTest() throws Throwable c.set("storage_port", 7012); c.set("ssl_storage_port", 7013); c.set("server_encryption_options", - ImmutableMap.builder().putAll(validKeystore) + ImmutableMap.builder().putAll(validFileBasedKeystores) .put("internode_encryption", "none") .put("optional", true) .put("legacy_ssl_storage_port_enabled", "true") @@ -146,7 +146,7 @@ public void legacySslStoragePortEnabledWithSameRegularAndSslPortTest() throws Th c.set("storage_port", 7012); // must match in-jvm dtest assigned ports c.set("ssl_storage_port", 7012); c.set("server_encryption_options", - ImmutableMap.builder().putAll(validKeystore) + ImmutableMap.builder().putAll(validFileBasedKeystores) .put("internode_encryption", "none") .put("optional", true) .put("legacy_ssl_storage_port_enabled", "true") @@ -174,7 +174,7 @@ public void tlsConnectionRejectedWhenUnencrypted() throws Throwable try (Cluster cluster = builder().withNodes(1).withConfig(c -> { c.with(Feature.NETWORK); c.set("server_encryption_options", - ImmutableMap.builder().putAll(validKeystore) + ImmutableMap.builder().putAll(validFileBasedKeystores) .put("internode_encryption", "none") .put("optional", false) .build()); @@ -202,7 +202,7 @@ public void allInternodeEncryptionEstablishedTest() throws Throwable .with(Feature.GOSSIP) // To make sure AllMembersAliveMonitor checks gossip (which uses internode conns) .with(Feature.NATIVE_PROTOCOL); // For virtual tables c.set("server_encryption_options", - ImmutableMap.builder().putAll(validKeystore) + ImmutableMap.builder().putAll(validFileBasedKeystores) .put("internode_encryption", "all") .build()); }).start()) @@ -234,7 +234,7 @@ public void negotiatedProtocolMustBeAcceptedProtocolTest() throws Throwable try (Cluster cluster = builder().withNodes(1).withConfig(c -> { c.with(Feature.NETWORK); c.set("server_encryption_options", - ImmutableMap.builder().putAll(validKeystore) + ImmutableMap.builder().putAll(validFileBasedKeystores) .put("internode_encryption", "all") .put("accepted_protocols", ImmutableList.of("TLSv1.1", "TLSv1.2", "TLSv1.3")) .build()); @@ -272,7 +272,7 @@ public void connectionCannotAgreeOnClientAndServer() throws Throwable try (Cluster cluster = builder().withNodes(1).withConfig(c -> { c.with(Feature.NETWORK); c.set("server_encryption_options", - ImmutableMap.builder().putAll(validKeystore) + ImmutableMap.builder().putAll(validFileBasedKeystores) .put("internode_encryption", "all") .put("accepted_protocols", Collections.singletonList("TLSv1.2")) .put("cipher_suites", Collections.singletonList("TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384")) diff --git a/test/distributed/org/apache/cassandra/distributed/test/NativeTransportEncryptionOptionsTest.java b/test/distributed/org/apache/cassandra/distributed/test/NativeTransportEncryptionOptionsTest.java index 9eb7c7afeb97..b23f501e6967 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/NativeTransportEncryptionOptionsTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/NativeTransportEncryptionOptionsTest.java @@ -87,7 +87,7 @@ public void optionalTlsConnectionAllowedWithKeystoreTest() throws Throwable { try (Cluster cluster = builder().withNodes(1).withConfig(c -> { c.with(Feature.NATIVE_PROTOCOL); - c.set("client_encryption_options", validKeystore); + c.set("client_encryption_options", validFileBasedKeystores); }).createWithoutStarting()) { InetAddress address = cluster.get(1).config().broadcastAddress().getAddress(); @@ -119,7 +119,7 @@ public void negotiatedProtocolMustBeAcceptedProtocolTest() throws Throwable try (Cluster cluster = builder().withNodes(1).withConfig(c -> { c.with(Feature.NATIVE_PROTOCOL); c.set("client_encryption_options", - ImmutableMap.builder().putAll(validKeystore) + ImmutableMap.builder().putAll(validFileBasedKeystores) .put("enabled", true) .put("accepted_protocols", ImmutableList.of("TLSv1.1", "TLSv1.2", "TLSv1.3")) .build()); @@ -156,7 +156,7 @@ public void connectionCannotAgreeOnClientAndServerTest() throws Throwable try (Cluster cluster = builder().withNodes(1).withConfig(c -> { c.with(Feature.NATIVE_PROTOCOL); c.set("client_encryption_options", - ImmutableMap.builder().putAll(validKeystore) + ImmutableMap.builder().putAll(validFileBasedKeystores) .put("enabled", true) .put("accepted_protocols", Collections.singletonList("TLSv1.2")) .put("cipher_suites", Collections.singletonList("TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384")) @@ -237,7 +237,7 @@ public void testOptionalMtlsModeAllowNonSSLConnections() throws Exception try (Cluster cluster = builder().withNodes(1).withConfig(c -> { c.with(Feature.NATIVE_PROTOCOL); c.set("client_encryption_options", - ImmutableMap.builder().putAll(validKeystore) + ImmutableMap.builder().putAll(validFileBasedKeystores) .put("enabled", true) .put("require_client_auth", "optional") .put("optional", true) @@ -282,7 +282,7 @@ public void testOptionalMtlsModeDoNotAllowNonSSLConnections() throws Exception try (Cluster cluster = builder().withNodes(1).withConfig(c -> { c.with(Feature.NATIVE_PROTOCOL); c.set("client_encryption_options", - ImmutableMap.builder().putAll(validKeystore) + ImmutableMap.builder().putAll(validFileBasedKeystores) .put("enabled", true) .put("require_client_auth", "optional") .put("optional", false) @@ -329,7 +329,7 @@ public void testOptionalSSLMode() throws Exception // Server configuration for optional mTLS mode c.with(Feature.NATIVE_PROTOCOL); c.set("client_encryption_options", - ImmutableMap.builder().putAll(validKeystore) + ImmutableMap.builder().putAll(validFileBasedKeystores) .put("enabled", true) .put("require_client_auth", "false") .put("optional", true) @@ -384,7 +384,7 @@ private void testEndpointVerification(boolean requireEndpointVerification, boole try (Cluster cluster = builder().withNodes(1).withConfig(c -> { c.with(Feature.NATIVE_PROTOCOL); c.set("client_encryption_options", - ImmutableMap.builder().putAll(validKeystore) + ImmutableMap.builder().putAll(validFileBasedKeystores) .put("enabled", true) .put("require_client_auth", true) .put("require_endpoint_verification", requireEndpointVerification) diff --git a/test/distributed/org/apache/cassandra/distributed/test/ResourceLeakTest.java b/test/distributed/org/apache/cassandra/distributed/test/ResourceLeakTest.java index d1a5d0815246..03cb62d5cba6 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/ResourceLeakTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/ResourceLeakTest.java @@ -49,7 +49,7 @@ import static org.apache.cassandra.distributed.api.Feature.JMX; import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL; import static org.apache.cassandra.distributed.api.Feature.NETWORK; -import static org.apache.cassandra.distributed.test.jmx.JMXGetterCheckTest.testAllValidGetters; +import static org.apache.cassandra.distributed.test.jmx.JMXTestsUtil.testAllValidGetters; import static org.apache.cassandra.utils.FBUtilities.now; import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.startsWith; @@ -167,7 +167,7 @@ static void testJmx(Cluster cluster) Assert.assertThat(defaultDomain, startsWith(JMXUtil.getJmxHost(config) + ":" + config.jmxPort())); } } - testAllValidGetters(cluster); + testAllValidGetters(cluster, null); } catch (Exception e) { diff --git a/test/distributed/org/apache/cassandra/distributed/test/SSTableLoaderEncryptionOptionsTest.java b/test/distributed/org/apache/cassandra/distributed/test/SSTableLoaderEncryptionOptionsTest.java index 94ea1d04416b..ea3c98058605 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/SSTableLoaderEncryptionOptionsTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/SSTableLoaderEncryptionOptionsTest.java @@ -61,12 +61,12 @@ public static void setupCluster() throws IOException CLUSTER = Cluster.build().withNodes(1).withConfig(c -> { c.with(Feature.NATIVE_PROTOCOL, Feature.NETWORK, Feature.GOSSIP); // need gossip to get hostid for java driver c.set("server_encryption_options", - ImmutableMap.builder().putAll(validKeystore) + ImmutableMap.builder().putAll(validFileBasedKeystores) .put("internode_encryption", "all") .put("optional", false) .build()); c.set("client_encryption_options", - ImmutableMap.builder().putAll(validKeystore) + ImmutableMap.builder().putAll(validFileBasedKeystores) .put("enabled", true) .put("optional", false) .put("accepted_protocols", Collections.singletonList("TLSv1.2")) diff --git a/test/distributed/org/apache/cassandra/distributed/test/WithPropertiesTest.java b/test/distributed/org/apache/cassandra/distributed/test/WithPropertiesTest.java new file mode 100644 index 000000000000..ad3c6dbd4d40 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/WithPropertiesTest.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import org.junit.Assert; +import org.junit.Test; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.distributed.shared.WithProperties; + +public class WithPropertiesTest +{ + @Test + public void testPreserveBeforeSet() + { + CassandraRelevantProperties booleanProperty = CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL; + boolean defaultPropertyValue = Boolean.parseBoolean(booleanProperty.getDefaultValue()); + boolean newPropertyValue = true; + + try (WithProperties properties = new WithProperties() + .preserve(booleanProperty) + .set(booleanProperty, newPropertyValue)) + { + Assert.assertEquals("Property value must match", newPropertyValue, booleanProperty.getBoolean()); + } + Assert.assertEquals("Property value must revert to the default value", defaultPropertyValue, booleanProperty.getBoolean()); + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXFeatureTest.java b/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXFeatureTest.java index 2ffddaed7e1d..99b90a2ec03b 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXFeatureTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXFeatureTest.java @@ -36,7 +36,7 @@ import org.apache.cassandra.distributed.shared.JMXUtil; import org.apache.cassandra.distributed.test.TestBaseImpl; -import static org.apache.cassandra.distributed.test.jmx.JMXGetterCheckTest.testAllValidGetters; +import static org.apache.cassandra.distributed.test.jmx.JMXTestsUtil.testAllValidGetters; import static org.hamcrest.Matchers.blankOrNullString; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.is; @@ -85,7 +85,7 @@ private void testJmxFeatures(INodeProvisionStrategy.Strategy provisionStrategy) allInstances.addAll(instancesContacted); // Make sure we actually exercise the mbeans by testing a bunch of getters. // Without this it's possible for the test to pass as we don't touch any mBeans that we register. - testAllValidGetters(cluster); + testAllValidGetters(cluster, null); } } Assert.assertEquals("Each instance from each cluster should have been unique", iterations * 2, allInstances.size()); @@ -122,7 +122,7 @@ public void testShutDownAndRestartInstances() throws Exception Assert.assertThat(statusResult.getStderr(), is(blankOrNullString())); Assert.assertThat(statusResult.getStdout(), containsString("UN 127.0.0.1")); testInstance(instances, cluster.get(1)); - testAllValidGetters(cluster); + testAllValidGetters(cluster, null); } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXGetterCheckTest.java b/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXGetterCheckTest.java index ea95003093dc..98691b944248 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXGetterCheckTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXGetterCheckTest.java @@ -17,53 +17,14 @@ */ package org.apache.cassandra.distributed.test.jmx; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Set; -import java.util.TreeSet; -import javax.management.JMRuntimeException; -import javax.management.MBeanAttributeInfo; -import javax.management.MBeanInfo; -import javax.management.MBeanOperationInfo; -import javax.management.MBeanServerConnection; -import javax.management.ObjectName; -import javax.management.remote.JMXConnector; - -import com.google.common.collect.ImmutableSet; import org.junit.Test; import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.api.Feature; -import org.apache.cassandra.distributed.api.IInstanceConfig; -import org.apache.cassandra.distributed.api.IInvokableInstance; -import org.apache.cassandra.distributed.shared.JMXUtil; import org.apache.cassandra.distributed.test.TestBaseImpl; public class JMXGetterCheckTest extends TestBaseImpl { - private static final Set<String> IGNORE_ATTRIBUTES = ImmutableSet.of( - "org.apache.cassandra.net:type=MessagingService:BackPressurePerHost", // throws unsupported saying the feature was removed... dropped in CASSANDRA-15375 - "org.apache.cassandra.db:type=DynamicEndpointSnitch:Scores", // when running in multiple-port-one-IP mode, this fails - "org.apache.cassandra.db:type=StorageService:OutstandingSchemaVersions", // deprecated (TCM) - "org.apache.cassandra.db:type=StorageService:OutstandingSchemaVersionsWithPort" // deprecated (TCM) - ); - private static final Set<String> IGNORE_OPERATIONS = ImmutableSet.of( - "org.apache.cassandra.db:type=StorageService:stopDaemon", // halts the instance, which then causes the JVM to exit - "org.apache.cassandra.db:type=StorageService:drain", // don't drain, it stops things which can cause other APIs to be unstable as we are in a stopped state - "org.apache.cassandra.db:type=StorageService:stopGossiping", // if we stop gossip this can cause other issues, so avoid - "org.apache.cassandra.db:type=StorageService:resetLocalSchema", // this will fail when there are no other nodes which can serve schema - "org.apache.cassandra.db:type=StorageService:joinRing", // Causes bootstrapping errors - "org.apache.cassandra.db:type=Tables,keyspace=system,table=local:loadNewSSTables", // Shouldn't attempt to load SSTables as sometimes the temp directories don't work - "org.apache.cassandra.db:type=CIDRGroupsMappingManager:loadCidrGroupsCache", // CIDR cache isn't enabled by default - "org.apache.cassandra.db:type=StorageService:clearConnectionHistory", // Throws a NullPointerException - "org.apache.cassandra.db:type=StorageService:startGossiping", // causes multiple loops to fail - "org.apache.cassandra.db:type=StorageService:startNativeTransport", // causes multiple loops to fail - "org.apache.cassandra.db:type=StorageService:resumeMove", // throws since there is no move in progress - "org.apache.cassandra.db:type=StorageService:abortMove", // throws since there is no move in progress - "org.apache.cassandra.db:type=CIDRGroupsMappingManager:loadCidrGroupsCache", // AllowAllCIDRAuthorizer doesn't support this operation, as feature is disabled by default - "org.apache.cassandra.db:type=StorageService:forceRemoveCompletion" // deprecated (TCM) - ); @Test public void testGetters() throws Exception @@ -72,95 +33,8 @@ public void testGetters() throws Exception { try (Cluster cluster = Cluster.build(1).withConfig(c -> c.with(Feature.values())).start()) { - testAllValidGetters(cluster); - } - } - } - - /** - * Tests JMX getters and operations. - * Useful for more than just testing getters, it also is used in JMXFeatureTest - * to make sure we've touched the complete JMX code path. - * @param cluster the cluster to test - * @throws Exception several kinds of exceptions can be thrown, mostly from JMX infrastructure issues. - */ - public static void testAllValidGetters(Cluster cluster) throws Exception - { - for (IInvokableInstance instance: cluster) - { - if (instance.isShutdown()) - { - continue; - } - IInstanceConfig config = instance.config(); - List<Named> errors = new ArrayList<>(); - try (JMXConnector jmxc = JMXUtil.getJmxConnector(config)) - { - MBeanServerConnection mbsc = jmxc.getMBeanServerConnection(); - Set<ObjectName> metricNames = new TreeSet<>(mbsc.queryNames(null, null)); - for (ObjectName name : metricNames) - { - if (!name.getDomain().startsWith("org.apache.cassandra")) - continue; - MBeanInfo info = mbsc.getMBeanInfo(name); - for (MBeanAttributeInfo a : info.getAttributes()) - { - String fqn = String.format("%s:%s", name, a.getName()); - if (!a.isReadable() || IGNORE_ATTRIBUTES.contains(fqn)) - continue; - try - { - mbsc.getAttribute(name, a.getName()); - } - catch (JMRuntimeException e) - { - errors.add(new Named(String.format("Attribute %s", fqn), e.getCause())); - } - } - - for (MBeanOperationInfo o : info.getOperations()) - { - String fqn = String.format("%s:%s", name, o.getName()); - if (o.getSignature().length != 0 || IGNORE_OPERATIONS.contains(fqn)) - continue; - try - { - mbsc.invoke(name, o.getName(), new Object[0], new String[0]); - } - catch (JMRuntimeException e) - { - errors.add(new Named(String.format("Operation %s", fqn), e.getCause())); - } - } - } - } - if (!errors.isEmpty()) - { - AssertionError root = new AssertionError(); - errors.forEach(root::addSuppressed); - throw root; - } - } - } - - /** - * This class is meant to make new errors easier to read, by adding the JMX endpoint, and cleaning up the unneeded JMX/Reflection logic cluttering the stacktrace - */ - private static class Named extends RuntimeException - { - public Named(String msg, Throwable cause) - { - super(msg + "\nCaused by: " + cause.getClass().getCanonicalName() + ": " + cause.getMessage(), cause.getCause()); - StackTraceElement[] stack = cause.getStackTrace(); - List<StackTraceElement> copy = new ArrayList<>(); - for (StackTraceElement s : stack) - { - if (!s.getClassName().startsWith("org.apache.cassandra")) - break; - copy.add(s); + JMXTestsUtil.testAllValidGetters(cluster, null); } - Collections.reverse(copy); - setStackTrace(copy.toArray(new StackTraceElement[0])); } } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXSslConfigDistributedTest.java b/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXSslConfigDistributedTest.java new file mode 100644 index 000000000000..146f548eaf95 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXSslConfigDistributedTest.java @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.jmx; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import javax.management.remote.rmi.RMIConnectorServer; +import javax.net.ssl.SSLException; +import javax.rmi.ssl.SslRMIClientSocketFactory; + +import com.google.common.collect.ImmutableMap; +import org.junit.Test; + +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.impl.JmxTestClientSslContextFactory; +import org.apache.cassandra.distributed.impl.JmxTestClientSslSocketFactory; +import org.apache.cassandra.distributed.shared.WithProperties; +import org.apache.cassandra.distributed.test.AbstractEncryptionOptionsImpl; +import org.apache.cassandra.utils.jmx.JMXSslPropertiesUtil; + +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_CIPHER_SUITES; + +/** + * Distributed tests for JMX SSL configuration via the system properties OR the encryption options in the cassandra.yaml. + */ +public class JMXSslConfigDistributedTest extends AbstractEncryptionOptionsImpl +{ + @Test + public void testDefaultEncryptionOptions() throws Throwable + { + // We must set the keystore in the system variable to make sure that the call to SSLContext.getDefault() + // uses it when Client SSL Socketfactory is initialized even if we don't need it here. + // The same default SSLContext.getDefault() will be used by other methods like testSystemSettings() in this test + // for the Server SSL Socketfactory and at that time we will need the keystore to be available + // All of the above is the issue because we run everything (JMX Server, Client) in the same JVM, multiple times + // and the SSLContext.getDefault() relies on static initialization that is reused + try (WithProperties withProperties = JMXSslPropertiesUtil.preserveAllProperties()) + { + setKeystoreProperties(withProperties); + ImmutableMap<String, Object> encryptionOptionsMap = ImmutableMap.<String, Object>builder().putAll(validFileBasedKeystores) + .put("enabled", true) + .put("accepted_protocols", Arrays.asList("TLSv1.2", "TLSv1.3", "TLSv1.1")) + .build(); + + try (Cluster cluster = builder().withNodes(1).withConfig(c -> { + c.with(Feature.JMX).set("jmx_encryption_options", encryptionOptionsMap); + }).start()) + { + Map<String, Object> jmxEnv = new HashMap<>(); + configureClientSocketFactory(jmxEnv, encryptionOptionsMap); + JMXTestsUtil.testAllValidGetters(cluster, jmxEnv); + } + } + } + + @Test + public void testClientAuth() throws Throwable + { + try (WithProperties withProperties = JMXSslPropertiesUtil.preserveAllProperties()) + { + setKeystoreProperties(withProperties); + ImmutableMap<String, Object> encryptionOptionsMap = ImmutableMap.<String, Object>builder().putAll(validFileBasedKeystores) + .put("enabled", true) + .put("require_client_auth", true) + .put("accepted_protocols", Arrays.asList("TLSv1.2", "TLSv1.3", "TLSv1.1")) + .build(); + + try (Cluster cluster = builder().withNodes(1).withConfig(c -> { + c.with(Feature.JMX).set("jmx_encryption_options", encryptionOptionsMap); + }).start()) + { + Map<String, Object> jmxEnv = new HashMap<>(); + configureClientSocketFactory(jmxEnv, encryptionOptionsMap); + JMXTestsUtil.testAllValidGetters(cluster, jmxEnv); + } + } + } + + @Test + public void testSystemSettings() throws Throwable + { + COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_CIPHER_SUITES.reset(); + try (WithProperties withProperties = JMXSslPropertiesUtil.use(true, false, + "TLSv1.2,TLSv1.3,TLSv1.1")) + { + setKeystoreProperties(withProperties); + try (Cluster cluster = builder().withNodes(1).withConfig(c -> { + c.with(Feature.JMX); + }).start()) + { + Map<String, Object> jmxEnv = new HashMap<>(); + SslRMIClientSocketFactory clientFactory = new SslRMIClientSocketFactory(); + jmxEnv.put(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE, clientFactory); + jmxEnv.put("com.sun.jndi.rmi.factory.socket", clientFactory); + JMXTestsUtil.testAllValidGetters(cluster, jmxEnv); + } + } + } + + @Test + public void testInvalidKeystorePath() throws Throwable + { + try (Cluster cluster = builder().withNodes(1).withConfig(c -> { + c.with(Feature.JMX).set("jmx_encryption_options", + ImmutableMap.<String, Object>builder() + .put("enabled", true) + .put("keystore", "/path/to/bad/keystore/that/should/not/exist") + .put("keystore_password", "cassandra") + .put("accepted_protocols", Arrays.asList("TLSv1.2", "TLSv1.3", "TLSv1.1")) + .build()); + }).createWithoutStarting()) + { + assertCannotStartDueToConfigurationException(cluster); + } + } + + /** + * Tests {@code disabled} jmx_encryption_options. Here even if the configured {@code keystore} is invalid, it will + * not matter and the JMX server/client should start. + */ + @Test + public void testDisabledEncryptionOptions() throws Throwable + { + try (Cluster cluster = builder().withNodes(1).withConfig(c -> { + c.with(Feature.JMX).set("jmx_encryption_options", + ImmutableMap.builder() + .put("enabled", false) + .put("keystore", "/path/to/bad/keystore/that/should/not/exist") + .put("keystore_password", "cassandra") + .build()); + }).start()) + { + JMXTestsUtil.testAllValidGetters(cluster, null); + } + } + + private void setKeystoreProperties(WithProperties properties) + { + properties.with("javax.net.ssl.trustStore", (String) validFileBasedKeystores.get("truststore"), + "javax.net.ssl.trustStorePassword", (String) validFileBasedKeystores.get("truststore_password"), + "javax.net.ssl.keyStore", (String) validFileBasedKeystores.get("keystore"), + "javax.net.ssl.keyStorePassword", (String) validFileBasedKeystores.get("keystore_password")); + } + + @SuppressWarnings("unchecked") + private void configureClientSocketFactory(Map<String, Object> jmxEnv, Map<String, Object> encryptionOptionsMap) throws SSLException + { + JmxTestClientSslContextFactory clientSslContextFactory = new JmxTestClientSslContextFactory(encryptionOptionsMap); + List<String> cipherSuitesList = (List<String>) encryptionOptionsMap.get("cipher_suites"); + String[] cipherSuites = cipherSuitesList == null ? null : cipherSuitesList.toArray(new String[0]); + List<String> acceptedProtocolList = (List<String>) encryptionOptionsMap.get("accepted_protocols"); + String[] acceptedProtocols = acceptedProtocolList == null ? null : acceptedProtocolList.toArray(new String[0]); + JmxTestClientSslSocketFactory clientFactory = new JmxTestClientSslSocketFactory(clientSslContextFactory.createSSLContext(), + cipherSuites, acceptedProtocols); + jmxEnv.put(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE, clientFactory); + jmxEnv.put("com.sun.jndi.rmi.factory.socket", clientFactory); + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXTestsUtil.java b/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXTestsUtil.java new file mode 100644 index 000000000000..5726c242d9e7 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXTestsUtil.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.jmx; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; +import javax.management.JMRuntimeException; +import javax.management.MBeanAttributeInfo; +import javax.management.MBeanInfo; +import javax.management.MBeanOperationInfo; +import javax.management.MBeanServerConnection; +import javax.management.ObjectName; +import javax.management.remote.JMXConnector; + +import com.google.common.collect.ImmutableSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.IInstanceConfig; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.shared.JMXUtil; + +public class JMXTestsUtil +{ + private static final Logger logger = LoggerFactory.getLogger(JMXTestsUtil.class); + private static final Set<String> IGNORE_ATTRIBUTES = ImmutableSet.of( + "org.apache.cassandra.net:type=MessagingService:BackPressurePerHost", // throws unsupported saying the feature was removed... dropped in CASSANDRA-15375 + "org.apache.cassandra.db:type=DynamicEndpointSnitch:Scores", // when running in multiple-port-one-IP mode, this fails + "org.apache.cassandra.db:type=StorageService:OutstandingSchemaVersions", // deprecated (TCM) + "org.apache.cassandra.db:type=StorageService:OutstandingSchemaVersionsWithPort" // deprecated (TCM) + ); + private static final Set<String> IGNORE_OPERATIONS = ImmutableSet.of( + "org.apache.cassandra.db:type=StorageService:stopDaemon", // halts the instance, which then causes the JVM to exit + "org.apache.cassandra.db:type=StorageService:drain", // don't drain, it stops things which can cause other APIs to be unstable as we are in a stopped state + "org.apache.cassandra.db:type=StorageService:stopGossiping", // if we stop gossip this can cause other issues, so avoid + "org.apache.cassandra.db:type=StorageService:resetLocalSchema", // this will fail when there are no other nodes which can serve schema + "org.apache.cassandra.db:type=StorageService:joinRing", // Causes bootstrapping errors + "org.apache.cassandra.db:type=Tables,keyspace=system,table=local:loadNewSSTables", // Shouldn't attempt to load SSTables as sometimes the temp directories don't work + "org.apache.cassandra.db:type=CIDRGroupsMappingManager:loadCidrGroupsCache", // CIDR cache isn't enabled by default + "org.apache.cassandra.db:type=StorageService:clearConnectionHistory", // Throws a NullPointerException + "org.apache.cassandra.db:type=StorageService:startGossiping", // causes multiple loops to fail + "org.apache.cassandra.db:type=StorageService:startNativeTransport", // causes multiple loops to fail + "org.apache.cassandra.db:type=StorageService:resumeMove", // throws since there is no move in progress + "org.apache.cassandra.db:type=StorageService:abortMove", // throws since there is no move in progress + "org.apache.cassandra.db:type=CIDRGroupsMappingManager:loadCidrGroupsCache", // AllowAllCIDRAuthorizer doesn't support this operation, as feature is disabled by default + "org.apache.cassandra.db:type=StorageService:forceRemoveCompletion" // deprecated (TCM) + ); + + /** + * Tests JMX getters and operations and allows passing JMX Env used for the client JMX connection. + * + * @param cluster the cluster to test + * @throws Exception several kinds of exceptions can be thrown, mostly from JMX infrastructure issues. + */ + public static void testAllValidGetters(Cluster cluster, Map<String, ?> jmxEnv) throws Exception + { + for (IInvokableInstance instance : cluster) + { + if (instance.isShutdown()) + { + continue; + } + IInstanceConfig config = instance.config(); + List<Named> errors = new ArrayList<>(); + try (JMXConnector jmxc = JMXUtil.getJmxConnector(config, jmxEnv)) + { + MBeanServerConnection mbsc = jmxc.getMBeanServerConnection(); + Set<ObjectName> metricNames = new TreeSet<>(mbsc.queryNames(null, null)); + for (ObjectName name : metricNames) + { + if (!name.getDomain().startsWith("org.apache.cassandra")) + continue; + MBeanInfo info = mbsc.getMBeanInfo(name); + for (MBeanAttributeInfo a : info.getAttributes()) + { + String fqn = String.format("%s:%s", name, a.getName()); + if (!a.isReadable() || IGNORE_ATTRIBUTES.contains(fqn)) + continue; + try + { + mbsc.getAttribute(name, a.getName()); + } + catch (JMRuntimeException e) + { + errors.add(new Named(String.format("Attribute %s", fqn), e.getCause())); + } + } + + for (MBeanOperationInfo o : info.getOperations()) + { + String fqn = String.format("%s:%s", name, o.getName()); + if (o.getSignature().length != 0 || IGNORE_OPERATIONS.contains(fqn)) + continue; + try + { + mbsc.invoke(name, o.getName(), new Object[0], new String[0]); + } + catch (JMRuntimeException e) + { + errors.add(new Named(String.format("Operation %s", fqn), e.getCause())); + } + } + } + } + if (!errors.isEmpty()) + { + AssertionError root = new AssertionError(); + for (Named error : errors) + { + // The Named object's message has the cause also so this only logs the message + logger.error("Error {}", error.getMessage()); + root.addSuppressed(error); + } + throw root; + } + } + } + + /** + * This class is meant to make new errors easier to read, by adding the JMX endpoint, and cleaning up the unneeded JMX/Reflection logic cluttering the stacktrace + */ + static class Named extends RuntimeException + { + public Named(String msg, Throwable cause) + { + super(msg + "\nCaused by: " + cause.getClass().getCanonicalName() + ": " + cause.getMessage(), cause.getCause()); + StackTraceElement[] stack = cause.getStackTrace(); + List<StackTraceElement> copy = new ArrayList<>(); + for (StackTraceElement s : stack) + { + if (!s.getClassName().startsWith("org.apache.cassandra")) + break; + copy.add(s); + } + Collections.reverse(copy); + setStackTrace(copy.toArray(new StackTraceElement[0])); + } + } +} diff --git a/test/unit/org/apache/cassandra/config/EncryptionOptionsTest.java b/test/unit/org/apache/cassandra/config/EncryptionOptionsTest.java index cbfea947fe6c..a295095dbbd1 100644 --- a/test/unit/org/apache/cassandra/config/EncryptionOptionsTest.java +++ b/test/unit/org/apache/cassandra/config/EncryptionOptionsTest.java @@ -250,7 +250,7 @@ public void testMisplacedConfigKey() for(EncryptionOptions.ConfigKey configKey: EncryptionOptions.ConfigKey.values()) { - customSslContextFactoryParams.put(configKey.getKeyName(), "my-custom-value"); + customSslContextFactoryParams.put(configKey.toString(), "my-custom-value"); } EncryptionOptionsTestCase.of(null, absentKeystore, true, customSslContextFactoryParams, ENCRYPTED); diff --git a/test/unit/org/apache/cassandra/config/PropertiesTest.java b/test/unit/org/apache/cassandra/config/PropertiesTest.java index 6fec2dfe68e2..075dcf1f0d2f 100644 --- a/test/unit/org/apache/cassandra/config/PropertiesTest.java +++ b/test/unit/org/apache/cassandra/config/PropertiesTest.java @@ -40,8 +40,10 @@ public void backAndForth() throws Exception Config config = new Config(); Set<String> keys = ImmutableSet.of("server_encryption_options.enabled", "client_encryption_options.enabled", + "jmx_encryption_options.enabled", "server_encryption_options.optional", - "client_encryption_options.optional"); + "client_encryption_options.optional", + "jmx_encryption_options.optional"); for (Property prop : ps.values()) { // skip these properties as they don't allow get/set within the context of this test diff --git a/test/unit/org/apache/cassandra/db/virtual/SettingsTableTest.java b/test/unit/org/apache/cassandra/db/virtual/SettingsTableTest.java index 6397afff95e4..1645e3150e59 100644 --- a/test/unit/org/apache/cassandra/db/virtual/SettingsTableTest.java +++ b/test/unit/org/apache/cassandra/db/virtual/SettingsTableTest.java @@ -52,6 +52,7 @@ public void config() config = new Config(); config.client_encryption_options.applyConfig(); config.server_encryption_options.applyConfig(); + config.jmx_encryption_options.applyConfig(); config.sstable_preemptive_open_interval = null; config.index_summary_resize_interval = null; config.cache_load_timeout = new DurationSpec.IntSecondsBound(0); diff --git a/test/unit/org/apache/cassandra/utils/jmx/JMXSslConfigTest.java b/test/unit/org/apache/cassandra/utils/jmx/JMXSslConfigTest.java new file mode 100644 index 000000000000..2e739c558ba5 --- /dev/null +++ b/test/unit/org/apache/cassandra/utils/jmx/JMXSslConfigTest.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils.jmx; + +import java.net.InetAddress; +import java.util.Map; +import javax.management.remote.rmi.RMIConnectorServer; +import javax.net.ssl.SSLException; +import javax.rmi.ssl.SslRMIServerSocketFactory; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.distributed.shared.WithProperties; +import org.apache.cassandra.utils.JMXServerUtils; + +import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_CONFIG; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS; + +/** + * Tests for Local JMX server, the remote JMX SSL configuration via System properties in absence of jmx_encryption_options + * in the cassandra.yaml. This is the behavior before CASSANDRA-18508. + */ +public class JMXSslConfigTest +{ + static WithProperties properties; + + @BeforeClass + public static void setupDatabaseDescriptor() + { + properties = new WithProperties().set(CASSANDRA_CONFIG, "cassandra.yaml"); + DatabaseDescriptor.daemonInitialization(); + } + + @AfterClass + public static void tearDownDatabaseDescriptor() + { + properties.close(); + } + + /** + * Tests for remote JMX SSL configuration specified via the System properties. + */ + @Test + public void testRemoteJmxSystemConfig() throws SSLException + { + InetAddress serverAddress = InetAddress.getLoopbackAddress(); + String enabledProtocols = "TLSv1.2,TLSv1.3,TLSv1.1"; + String cipherSuites = "TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256"; + + try (WithProperties ignored = JMXSslPropertiesUtil.use(true, true, enabledProtocols, + cipherSuites)) + { + Map<String, Object> env = JMXServerUtils.configureJmxSocketFactories(serverAddress, false); + Assert.assertNotNull("ServerSocketFactory must not be null", env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE)); + Assert.assertTrue("RMI_SERVER_SOCKET_FACTORY must be of SslRMIServerSocketFactory type", env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE) instanceof SslRMIServerSocketFactory); + Assert.assertNotNull("ClientSocketFactory must not be null", env.get(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE)); + Assert.assertNotNull("com.sun.jndi.rmi.factory.socket must be set in the env", env.get("com.sun.jndi.rmi.factory.socket")); + Assert.assertEquals("protocols must match", enabledProtocols, JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS.getString()); + Assert.assertEquals("cipher-suites must match", cipherSuites, JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES.getString()); + } + } + + /** + * Tests for {@code localOnly} JMX Server configuration. + */ + @Test + public void testLocalJmxServer() throws SSLException + { + InetAddress serverAddress = InetAddress.getLoopbackAddress(); + try (WithProperties ignored = JMXSslPropertiesUtil.use(false)) + { + Map<String, Object> env = JMXServerUtils.configureJmxSocketFactories(serverAddress, true); + + Assert.assertNull("ClientSocketFactory must be null", env.get(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE)); + Assert.assertNull("com.sun.jndi.rmi.factory.socket must not be set in the env", env.get("com.sun.jndi.rmi.factory.socket")); + Assert.assertNotNull("ServerSocketFactory must not be null", env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE)); + Assert.assertNull("protocols must be empty", JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS.getString()); + Assert.assertNull("cipher-suites must empty", JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES.getString()); + } + } +} diff --git a/test/unit/org/apache/cassandra/utils/jmx/JMXSslConfiguredWithYamlFileOptionsTest.java b/test/unit/org/apache/cassandra/utils/jmx/JMXSslConfiguredWithYamlFileOptionsTest.java new file mode 100644 index 000000000000..3855183a9259 --- /dev/null +++ b/test/unit/org/apache/cassandra/utils/jmx/JMXSslConfiguredWithYamlFileOptionsTest.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils.jmx; + +import java.net.InetAddress; +import java.util.Map; +import javax.management.remote.rmi.RMIConnectorServer; +import javax.net.ssl.SSLException; +import javax.rmi.ssl.SslRMIServerSocketFactory; + +import org.apache.commons.lang3.StringUtils; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.EncryptionOptions; +import org.apache.cassandra.distributed.shared.WithProperties; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.utils.JMXServerUtils; + +import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_CONFIG; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS; + +/** + * Tests for Default JMX SSL configuration specified in the cassandra.yaml using jmx_encryption_options. + * The default configurtion means keystore/truststore configured with file paths and using {@code DefaultSslContextFactory} + * to initialize the SSLContext. + */ +public class JMXSslConfiguredWithYamlFileOptionsTest +{ + static WithProperties properties; + + @BeforeClass + public static void setupDatabaseDescriptor() + { + properties = new WithProperties().set(CASSANDRA_CONFIG, "cassandra-jmx-sslconfig.yaml"); + DatabaseDescriptor.daemonInitialization(); + } + + @AfterClass + public static void tearDownDatabaseDescriptor() + { + properties.close(); + } + + @Test + public void testYamlFileJmxEncryptionOptions() throws SSLException + { + EncryptionOptions jmxEncryptionOptions = DatabaseDescriptor.getJmxEncryptionOptions(); + String expectedProtocols = StringUtils.join(jmxEncryptionOptions.getAcceptedProtocols(), ","); + String expectedCipherSuites = StringUtils.join(jmxEncryptionOptions.cipherSuitesArray(), ","); + + InetAddress serverAddress = InetAddress.getLoopbackAddress(); + + try (WithProperties ignored = JMXSslPropertiesUtil.use(false)) + { + Map<String, Object> env = JMXServerUtils.configureJmxSocketFactories(serverAddress, false); + Assert.assertTrue("com.sun.management.jmxremote.ssl must be true", COM_SUN_MANAGEMENT_JMXREMOTE_SSL.getBoolean()); + Assert.assertNotNull("ServerSocketFactory must not be null", env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE)); + Assert.assertTrue("RMI_SERVER_SOCKET_FACTORY must be of JMXSslRMIServerSocketFactory type", env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE) instanceof SslRMIServerSocketFactory); + Assert.assertNotNull("ClientSocketFactory must not be null", env.get(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE)); + Assert.assertNotNull("com.sun.jndi.rmi.factory.socket must be set in the env", env.get("com.sun.jndi.rmi.factory.socket")); + Assert.assertEquals("javax.rmi.ssl.client.enabledProtocols must match", expectedProtocols, JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS.getString()); + Assert.assertEquals("javax.rmi.ssl.client.enabledCipherSuites must match", expectedCipherSuites, JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES.getString()); + } + } + + /** + * Tests for the error scenario when the JMX SSL configuration is provided as + * system configuration as well as encryption_options. + * + * @throws SSLException + */ + @Test(expected = ConfigurationException.class) + public void testDuplicateConfig() throws SSLException + { + InetAddress serverAddress = InetAddress.getLoopbackAddress(); + + try (WithProperties ignored = JMXSslPropertiesUtil.use(true)) + { + JMXServerUtils.configureJmxSocketFactories(serverAddress, false); + } + } +} diff --git a/test/unit/org/apache/cassandra/utils/jmx/JMXSslDisabledEncryptionOptionsTest.java b/test/unit/org/apache/cassandra/utils/jmx/JMXSslDisabledEncryptionOptionsTest.java new file mode 100644 index 000000000000..ef2070f2aa20 --- /dev/null +++ b/test/unit/org/apache/cassandra/utils/jmx/JMXSslDisabledEncryptionOptionsTest.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils.jmx; + +import java.net.InetAddress; +import java.util.Map; +import javax.management.remote.rmi.RMIConnectorServer; +import javax.net.ssl.SSLException; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.distributed.shared.WithProperties; +import org.apache.cassandra.utils.JMXServerUtils; + +import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_CONFIG; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS; + +/** + * Tests for disabling jmx_encryption_options in the cassandra.yaml. + */ +public class JMXSslDisabledEncryptionOptionsTest +{ + static WithProperties properties; + + @BeforeClass + public static void setupDatabaseDescriptor() + { + properties = new WithProperties().set(CASSANDRA_CONFIG, "cassandra-jmx-disabled-sslconfig.yaml"); + DatabaseDescriptor.daemonInitialization(); + } + + @AfterClass + public static void tearDownDatabaseDescriptor() + { + properties.close(); + } + + /** + * Tests disabled JMX SSL configuration. This means, all below are either not configured OR configured with + * disabled mode. + * 1. local only JMX server + * 2. System properties set for remote JMX SSL + * 3. jmx_encryption_options in the cassandra.yaml + */ + @Test + public void testDisabledJmxSslConfigs() throws SSLException + { + InetAddress serverAddress = InetAddress.getLoopbackAddress(); + + try (WithProperties ignored = JMXSslPropertiesUtil.use(false)) + { + Map<String, Object> env = JMXServerUtils.configureJmxSocketFactories(serverAddress, false); + Assert.assertTrue("no properties must be set", env.isEmpty()); + Assert.assertFalse("com.sun.management.jmxremote.ssl must be false", COM_SUN_MANAGEMENT_JMXREMOTE_SSL.getBoolean()); + Assert.assertNull("javax.rmi.ssl.client.enabledProtocols must be null", JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS.getString()); + Assert.assertNull("javax.rmi.ssl.client.enabledCipherSuites must be null", JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES.getString()); + } + } + + /** + * Tests fallback to the `local only` JMX server when jmx_encryption_options are disabled in the cassandra.yaml + * and no System settings provided for the remote SSL config. + */ + @Test + public void testFallbackToLocalJmxServer() throws SSLException + { + InetAddress serverAddress = InetAddress.getLoopbackAddress(); + + try (WithProperties ignored = JMXSslPropertiesUtil.use(false)) + { + Map<String, Object> env = JMXServerUtils.configureJmxSocketFactories(serverAddress, true); + Assert.assertFalse("com.sun.management.jmxremote.ssl must be false", COM_SUN_MANAGEMENT_JMXREMOTE_SSL.getBoolean()); + Assert.assertNull("com.sun.jndi.rmi.factory.socket must be null", env.get("com.sun.jndi.rmi.factory.socket")); + Assert.assertNotNull("ServerSocketFactory must not be null", env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE)); + Assert.assertFalse("com.sun.management.jmxremote.ssl must be false", COM_SUN_MANAGEMENT_JMXREMOTE_SSL.getBoolean()); + Assert.assertNull("javax.rmi.ssl.client.enabledProtocols must be null", JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS.getString()); + Assert.assertNull("javax.rmi.ssl.client.enabledCipherSuites must be null", JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES.getString()); + } + } +} diff --git a/test/unit/org/apache/cassandra/utils/jmx/JMXSslPEMConfigTest.java b/test/unit/org/apache/cassandra/utils/jmx/JMXSslPEMConfigTest.java new file mode 100644 index 000000000000..c7db7b2c724b --- /dev/null +++ b/test/unit/org/apache/cassandra/utils/jmx/JMXSslPEMConfigTest.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils.jmx; + +import java.net.InetAddress; +import java.util.Map; +import javax.management.remote.rmi.RMIConnectorServer; +import javax.net.ssl.SSLException; +import javax.rmi.ssl.SslRMIServerSocketFactory; + +import org.apache.commons.lang3.StringUtils; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.EncryptionOptions; +import org.apache.cassandra.distributed.shared.WithProperties; +import org.apache.cassandra.utils.JMXServerUtils; + +import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_CONFIG; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS; + +/** + * Tests for JMX SSL config using keystore/truststore in the PEM format with jmx_encryption_options in the cassandra.yaml. + */ +public class JMXSslPEMConfigTest +{ + static WithProperties properties; + + @BeforeClass + public static void setupDatabaseDescriptor() + { + properties = new WithProperties().set(CASSANDRA_CONFIG, "cassandra-jmx-pem-sslconfig.yaml"); + DatabaseDescriptor.daemonInitialization(); + } + + @AfterClass + public static void tearDownDatabaseDescriptor() + { + properties.close(); + } + + @Test + public void testPEMBasedJmxSslConfig() throws SSLException + { + EncryptionOptions jmxEncryptionOptions = DatabaseDescriptor.getJmxEncryptionOptions(); + String expectedProtocols = StringUtils.join(jmxEncryptionOptions.getAcceptedProtocols(), ","); + String expectedCipherSuites = StringUtils.join(jmxEncryptionOptions.cipherSuitesArray(), ","); + + InetAddress serverAddress = InetAddress.getLoopbackAddress(); + + try (WithProperties ignored = JMXSslPropertiesUtil.use(false)) + { + Map<String, Object> env = JMXServerUtils.configureJmxSocketFactories(serverAddress, false); + Assert.assertTrue("com.sun.management.jmxremote.ssl must be true", COM_SUN_MANAGEMENT_JMXREMOTE_SSL.getBoolean()); + Assert.assertNotNull("ServerSocketFactory must not be null", env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE)); + Assert.assertTrue("RMI_SERVER_SOCKET_FACTORY must be of JMXSslRMIServerSocketFactory type", env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE) instanceof SslRMIServerSocketFactory); + Assert.assertNotNull("ClientSocketFactory must not be null", env.get(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE)); + Assert.assertNotNull("com.sun.jndi.rmi.factory.socket must be set in the env", env.get("com.sun.jndi.rmi.factory.socket")); + Assert.assertEquals("javax.rmi.ssl.client.enabledProtocols must match", expectedProtocols, JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS.getString()); + Assert.assertEquals("javax.rmi.ssl.client.enabledCipherSuites must match", expectedCipherSuites, JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES.getString()); + } + } +} diff --git a/test/unit/org/apache/cassandra/utils/jmx/JMXSslPropertiesUtil.java b/test/unit/org/apache/cassandra/utils/jmx/JMXSslPropertiesUtil.java new file mode 100644 index 000000000000..7675ce8fe20b --- /dev/null +++ b/test/unit/org/apache/cassandra/utils/jmx/JMXSslPropertiesUtil.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils.jmx; + +import org.apache.cassandra.distributed.shared.WithProperties; + +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_CIPHER_SUITES; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_PROTOCOLS; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_NEED_CLIENT_AUTH; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS; + +public final class JMXSslPropertiesUtil +{ + public static WithProperties use(boolean enableRemoteSsl, boolean needClientAuth, String enabledProtocols, String cipherSuites) + { + return preserveAllProperties() + .set(COM_SUN_MANAGEMENT_JMXREMOTE_SSL, enableRemoteSsl) + .set(COM_SUN_MANAGEMENT_JMXREMOTE_SSL_NEED_CLIENT_AUTH, needClientAuth) + .set(COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_PROTOCOLS, enabledProtocols) + .set(COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_CIPHER_SUITES, cipherSuites); + } + + public static WithProperties use(boolean enableRemoteSsl, boolean needClientAuth, String enabledProtocols) + { + return preserveAllProperties() + .set(COM_SUN_MANAGEMENT_JMXREMOTE_SSL, enableRemoteSsl) + .set(COM_SUN_MANAGEMENT_JMXREMOTE_SSL_NEED_CLIENT_AUTH, needClientAuth) + .set(COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_PROTOCOLS, enabledProtocols); + } + + public static WithProperties use(boolean enableRemoteSsl) + { + return preserveAllProperties() + .set(COM_SUN_MANAGEMENT_JMXREMOTE_SSL, enableRemoteSsl); + } + + public static WithProperties preserveAllProperties() + { + return new WithProperties() + .preserve(COM_SUN_MANAGEMENT_JMXREMOTE_SSL) + .preserve(COM_SUN_MANAGEMENT_JMXREMOTE_SSL_NEED_CLIENT_AUTH) + .preserve(COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_PROTOCOLS) + .preserve(COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_CIPHER_SUITES) + .preserve(JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS) + .preserve(JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES); + } +} From e476c48a1cd09ff2da58a226774e2920737e4186 Mon Sep 17 00:00:00 2001 From: Mick Semb Wever <mck@apache.org> Date: Thu, 21 Nov 2024 23:43:29 +0100 Subject: [PATCH 040/225] GossipShutdowns only apply if greater than or equal generation patch by Mick Semb Wever; reviewed by Brandon Williams for CASSANDRA-20033 --- CHANGES.txt | 1 + .../cassandra/gms/GossipShutdownVerbHandler.java | 11 +++++++++-- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 05046c9a954f..a8072f3e382a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.3 + * Fix delayed gossip shutdown messages clobbering startup states that leave restarted nodes appearing down (CASSANDRA-20033) * Streamline the serialized format for index status gossip messages (CASSANDRA-20058) * Batch clusterings into single SAI partition post-filtering reads (CASSANDRA-19497) * Ban the usage of "var" instead of full types in the production code (CASSANDRA-20038) diff --git a/src/java/org/apache/cassandra/gms/GossipShutdownVerbHandler.java b/src/java/org/apache/cassandra/gms/GossipShutdownVerbHandler.java index 9878520b813d..b5442a1b0b76 100644 --- a/src/java/org/apache/cassandra/gms/GossipShutdownVerbHandler.java +++ b/src/java/org/apache/cassandra/gms/GossipShutdownVerbHandler.java @@ -36,8 +36,15 @@ public void doVerb(Message<GossipShutdown> message) logger.debug("Ignoring shutdown message from {} because gossip is disabled", message.from()); return; } - if (message.payload == null) Gossiper.instance.markAsShutdown(message.from()); - else Gossiper.instance.markAsShutdown(message.from(), message.payload.state); + HeartBeatState previous = Gossiper.instance.getEndpointStateForEndpoint(message.from()).getHeartBeatState(); + + if (message.payload == null) + Gossiper.instance.markAsShutdown(message.from()); + else if (previous.getGeneration() <= message.payload.state.getHeartBeatState().getGeneration()) + Gossiper.instance.markAsShutdown(message.from(), message.payload.state); + else + logger.debug("Ignoring shutdown message from {} because generation {} older than local {}", + message.from(), message.payload.state.getHeartBeatState().getGeneration(), previous.getGeneration()); } } From 880fa6587b54db5b5d6cd64f5d935fd15da9a904 Mon Sep 17 00:00:00 2001 From: Sam Tunnicliffe <samt@apache.org> Date: Fri, 27 Oct 2023 13:03:56 +0100 Subject: [PATCH 041/225] Include column drop timestamp in ALTER TABLE transformation Patch by Sam Tunnicliffe and Marcus Eriksson; reviewed by Sam Tunnicliffe and Marcus Eriksson for CASSANDRA-18961 Co-authored-by: Marcus Eriksson <marcuse@apache.org> Co-authored-by: Sam Tunnicliffe <samt@apache.org> --- CHANGES.txt | 1 + .../schema/AlterSchemaStatement.java | 17 +++- .../schema/AlterTableStatement.java | 9 +- .../io/sstable/CQLSSTableWriter.java | 2 +- .../org/apache/cassandra/schema/Schema.java | 2 +- .../cassandra/tcm/membership/NodeVersion.java | 2 +- .../cassandra/tcm/serialization/Version.java | 4 + .../tcm/transformations/AlterSchema.java | 35 ++++--- .../distributed/test/DropColumnTest.java | 98 +++++++++++++++++++ .../distributed/test/log/CMSTestBase.java | 11 +-- .../test/log/ClusterMetadataTestHelper.java | 7 +- .../SchemaChangeDuringRangeMovementTest.java | 4 +- .../apache/cassandra/schema/SchemaTest.java | 6 +- 13 files changed, 163 insertions(+), 35 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/test/DropColumnTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 75dd48aa0246..49eb15619f38 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Include column drop timestamp in alter table transformation (CASSANDRA-18961) * Make JMX SSL configurable in cassandra.yaml (CASSANDRA-18508) * Fix cqlsh CAPTURE command to save query results without trace details when TRACING is ON (CASSANDRA-19105) * Optionally prevent tombstone purging during repair (CASSANDRA-20071) diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java index a3be1d0e06e5..590a55a3f926 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java @@ -17,6 +17,7 @@ */ package org.apache.cassandra.cql3.statements.schema; +import java.util.Optional; import java.util.Set; import com.google.common.collect.ImmutableSet; @@ -46,6 +47,8 @@ abstract public class AlterSchemaStatement implements CQLStatement.SingleKeyspac // TODO: not sure if this is going to stay the same, or will be replaced by more efficient serialization/sanitation means // or just `toString` for every statement private String cql; + public static final long NO_EXECUTION_TIMESTAMP = -1; + private long executionTimestamp = NO_EXECUTION_TIMESTAMP; protected AlterSchemaStatement(String keyspaceName) { @@ -57,6 +60,11 @@ public void setCql(String cql) this.cql = cql; } + public void setExecutionTimestamp(long executionTimestamp) + { + this.executionTimestamp = executionTimestamp; + } + @Override public String cql() { @@ -103,6 +111,12 @@ public String keyspace() return keyspaceName; } + @Override + public Optional<Long> fixedTimestampMicros() + { + return executionTimestamp == NO_EXECUTION_TIMESTAMP ? Optional.empty() : Optional.of(executionTimestamp); + } + public ResultMessage executeLocally(QueryState state, QueryOptions options) { return execute(state); @@ -147,6 +161,7 @@ public ResultMessage execute(QueryState state) throw ire("Virtual keyspace '%s' is not user-modifiable", keyspaceName); validateKeyspaceName(); + setExecutionTimestamp(state.getTimestamp()); // Perform a 'dry-run' attempt to apply the transformation locally before submitting to the CMS. This can save a // round trip to the CMS for things syntax errors, but also fail fast for things like configuration errors. // Such failures may be dependent on the specific node's config (for things like guardrails/memtable @@ -158,7 +173,6 @@ public ResultMessage execute(QueryState state) // cluster, as config can be heterogenous falling back to safe defaults may occur on some nodes. ClusterMetadata metadata = ClusterMetadata.current(); apply(metadata); - ClusterMetadata result = Schema.instance.submit(this); KeyspacesDiff diff = Keyspaces.diff(metadata.schema.getKeyspaces(), result.schema.getKeyspaces()); @@ -225,6 +239,7 @@ public String toString() return "AlterSchemaStatement{" + "keyspaceName='" + keyspaceName + '\'' + ", cql='" + cql() + '\'' + + ", executionTimestamp="+executionTimestamp + '}'; } } diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java index 3a5b9ae8fb8e..4302919ca99b 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java @@ -474,7 +474,14 @@ private void dropColumn(KeyspaceMetadata keyspace, TableMetadata table, ColumnId */ private long getTimestamp() { - return timestamp == null ? ClientState.getTimestamp() : timestamp; + // Prior to Metadata serialization V5, the execution timestamp was not included in AlterSchema + // serializations. Instead, the current time (from ClientState::getTimestamp) was used, making + // DROP COLUMN non-idempotent and causing potenial data loss as described in CASSANDRA-18961. + // This was fixed before release by serialization V5, but we include a dangerous backwards + // compatibility option here or so that we can still apply pre-V5 serialized transformations + // (which would only exist in clusters running pre-release versions of Cassandra). Once all peers + // are running a V5 compatible version, ClientState::getTimestamp will never be used. + return timestamp == null ? fixedTimestampMicros().orElseGet(ClientState::getTimestamp) : timestamp; } } diff --git a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java index 8bb882071e7f..dbe91cee106c 100644 --- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java @@ -801,7 +801,7 @@ private void applyIndexes(String keyspaceName) private void commitKeyspaceMetadata(KeyspaceMetadata keyspaceMetadata) { SchemaTransformation schemaTransformation = metadata -> metadata.schema.getKeyspaces().withAddedOrUpdated(keyspaceMetadata); - ClusterMetadataService.instance().commit(new AlterSchema(schemaTransformation, Schema.instance)); + ClusterMetadataService.instance().commit(new AlterSchema(schemaTransformation)); } /** diff --git a/src/java/org/apache/cassandra/schema/Schema.java b/src/java/org/apache/cassandra/schema/Schema.java index 147d24a80463..afb987a7a0c1 100644 --- a/src/java/org/apache/cassandra/schema/Schema.java +++ b/src/java/org/apache/cassandra/schema/Schema.java @@ -289,7 +289,7 @@ public ClusterMetadata submit(SchemaTransformation transformation) // result of this execution can be either a complete failure/timeout, or a success, but together with a log of // operations that have to be applied before we can do anything return ClusterMetadataService.instance() - .commit(new AlterSchema(transformation, this), + .commit(new AlterSchema(transformation), (metadata) -> metadata, (code, reason) -> { switch (code) diff --git a/src/java/org/apache/cassandra/tcm/membership/NodeVersion.java b/src/java/org/apache/cassandra/tcm/membership/NodeVersion.java index 9ce9f4602105..88d529635202 100644 --- a/src/java/org/apache/cassandra/tcm/membership/NodeVersion.java +++ b/src/java/org/apache/cassandra/tcm/membership/NodeVersion.java @@ -34,7 +34,7 @@ public class NodeVersion implements Comparable<NodeVersion> { public static final Serializer serializer = new Serializer(); - public static final Version CURRENT_METADATA_VERSION = Version.V4; + public static final Version CURRENT_METADATA_VERSION = Version.V5; public static final NodeVersion CURRENT = new NodeVersion(new CassandraVersion(FBUtilities.getReleaseVersionString()), CURRENT_METADATA_VERSION); private static final CassandraVersion SINCE_VERSION = CassandraVersion.CASSANDRA_5_0; diff --git a/src/java/org/apache/cassandra/tcm/serialization/Version.java b/src/java/org/apache/cassandra/tcm/serialization/Version.java index 7c6177dcf643..ed601009f65f 100644 --- a/src/java/org/apache/cassandra/tcm/serialization/Version.java +++ b/src/java/org/apache/cassandra/tcm/serialization/Version.java @@ -46,6 +46,10 @@ public enum Version * - Serialize allowAutoSnapshot and incrementalBackups when serializing TableParams */ V4(4), + /** + * - AlterSchema includes execution timestamp + */ + V5(5), UNKNOWN(Integer.MAX_VALUE); diff --git a/src/java/org/apache/cassandra/tcm/transformations/AlterSchema.java b/src/java/org/apache/cassandra/tcm/transformations/AlterSchema.java index 375193422454..93692f55ba93 100644 --- a/src/java/org/apache/cassandra/tcm/transformations/AlterSchema.java +++ b/src/java/org/apache/cassandra/tcm/transformations/AlterSchema.java @@ -28,6 +28,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import org.apache.cassandra.cql3.statements.schema.AlterSchemaStatement; import org.apache.cassandra.exceptions.AlreadyExistsException; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.exceptions.InvalidRequestException; @@ -38,8 +40,6 @@ import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.Keyspaces; import org.apache.cassandra.schema.ReplicationParams; -import org.apache.cassandra.schema.Schema; -import org.apache.cassandra.schema.SchemaProvider; import org.apache.cassandra.schema.SchemaTransformation; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.Tables; @@ -53,7 +53,9 @@ import org.apache.cassandra.tcm.serialization.AsymmetricMetadataSerializer; import org.apache.cassandra.tcm.serialization.Version; import org.apache.cassandra.utils.JVMStabilityInspector; +import org.apache.cassandra.utils.vint.VIntCoding; +import static org.apache.cassandra.cql3.statements.schema.AlterSchemaStatement.NO_EXECUTION_TIMESTAMP; import static org.apache.cassandra.exceptions.ExceptionCode.ALREADY_EXISTS; import static org.apache.cassandra.exceptions.ExceptionCode.CONFIG_ERROR; import static org.apache.cassandra.exceptions.ExceptionCode.INVALID; @@ -65,14 +67,11 @@ public class AlterSchema implements Transformation private static final Logger logger = LoggerFactory.getLogger(AlterSchema.class); public static final Serializer serializer = new Serializer(); - public final SchemaTransformation schemaTransformation; - protected final SchemaProvider schemaProvider; + private final SchemaTransformation schemaTransformation; - public AlterSchema(SchemaTransformation schemaTransformation, - SchemaProvider schemaProvider) + public AlterSchema(SchemaTransformation schemaTransformation) { this.schemaTransformation = schemaTransformation; - this.schemaProvider = schemaProvider; } @Override @@ -95,7 +94,6 @@ public final Result execute(ClusterMetadata prev) // log. In this case, there is a connected client and associated ClientState, so to avoid duplicate warnings // pause capture and resume after in applying the schema change. schemaTransformation.enterExecution(); - // Guard against an invalid SchemaTransformation supplying a TableMetadata with a future epoch newKeyspaces = schemaTransformation.apply(prev); newKeyspaces.forEach(ksm -> { @@ -253,20 +251,33 @@ static class Serializer implements AsymmetricMetadataSerializer<Transformation, public void serialize(Transformation t, DataOutputPlus out, Version version) throws IOException { SchemaTransformation.serializer.serialize(((AlterSchema) t).schemaTransformation, out, version); + if (version.isAtLeast(Version.V5)) + { + long fixedTimestamp = ((AlterSchema)t).schemaTransformation.fixedTimestampMicros().orElse(NO_EXECUTION_TIMESTAMP); + out.writeVInt(fixedTimestamp); + } } @Override public AlterSchema deserialize(DataInputPlus in, Version version) throws IOException { - return new AlterSchema(SchemaTransformation.serializer.deserialize(in, version), - Schema.instance); - + SchemaTransformation transformation = SchemaTransformation.serializer.deserialize(in, version); + if (version.isAtLeast(Version.V5)) + { + long timestamp = in.readVInt(); + if (transformation instanceof AlterSchemaStatement) + ((AlterSchemaStatement) transformation).setExecutionTimestamp(timestamp); + } + return new AlterSchema(transformation); } @Override public long serializedSize(Transformation t, Version version) { - return SchemaTransformation.serializer.serializedSize(((AlterSchema) t).schemaTransformation, version); + long size = SchemaTransformation.serializer.serializedSize(((AlterSchema) t).schemaTransformation, version); + if (version.isAtLeast(Version.V5)) + size += VIntCoding.computeVIntSize(((AlterSchema)t).schemaTransformation.fixedTimestampMicros().orElse(NO_EXECUTION_TIMESTAMP)); + return size; } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/DropColumnTest.java b/test/distributed/org/apache/cassandra/distributed/test/DropColumnTest.java new file mode 100644 index 000000000000..e3b85a8c86c8 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/DropColumnTest.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +import com.google.common.util.concurrent.Uninterruptibles; +import org.junit.Test; + +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.schema.DroppedColumn; + +import static org.junit.Assert.assertEquals; + +public class DropColumnTest extends TestBaseImpl +{ + @Test + public void dropTest() throws IOException, ExecutionException, InterruptedException + { + try (Cluster cluster = init(Cluster.build(2).start())) + { + cluster.schemaChange(withKeyspace("create table %s.tbl (id int primary key, a int, b int, c int)")); + cluster.filters().allVerbs().messagesMatching((i, i1, iMessage) -> { + Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS); + return false; + }).drop(); + cluster.schemaChange(withKeyspace("alter table %s.tbl drop b")); + cluster.filters().reset(); + + long dropTimeNode1 = getDropTime(cluster.get(1)); + long dropTimeNode2 = getDropTime(cluster.get(2)); + + assertEquals(dropTimeNode1, dropTimeNode2); + + cluster.get(2).shutdown().get(); + cluster.get(2).startup(); + + assertEquals(dropTimeNode1, getDropTime(cluster.get(2))); + } + } + + @Test + public void dropUsingTimestampTest() throws IOException, ExecutionException, InterruptedException + { + try (Cluster cluster = init(Cluster.build(2).start())) + { + cluster.schemaChange(withKeyspace("create table %s.tbl (id int primary key, a int, b int, c int)")); + cluster.filters().allVerbs().messagesMatching((i, i1, iMessage) -> { + Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS); + return false; + }).drop(); + cluster.schemaChange(withKeyspace("alter table %s.tbl drop b using timestamp 99999")); + cluster.filters().reset(); + + long dropTimeNode1 = getDropTime(cluster.get(1)); + long dropTimeNode2 = getDropTime(cluster.get(2)); + + assertEquals(dropTimeNode1, dropTimeNode2); + assertEquals(99999, dropTimeNode1); + + cluster.get(2).shutdown().get(); + cluster.get(2).startup(); + + assertEquals(dropTimeNode1, getDropTime(cluster.get(2))); + } + } + + private long getDropTime(IInvokableInstance instance) + { + return instance.callOnInstance(() -> { + Map<ByteBuffer, DroppedColumn> droppedColumns = Keyspace.open(KEYSPACE).getColumnFamilyStore("tbl").metadata().droppedColumns; + assertEquals(1, droppedColumns.size()); + return droppedColumns.values().iterator().next().droppedTime; + }); + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/CMSTestBase.java b/test/distributed/org/apache/cassandra/distributed/test/log/CMSTestBase.java index e254798915e5..ffc7f96bbac5 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/CMSTestBase.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/CMSTestBase.java @@ -33,7 +33,6 @@ import org.apache.cassandra.schema.DistributedSchema; import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.Keyspaces; -import org.apache.cassandra.schema.SchemaProvider; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.Commit; @@ -45,7 +44,6 @@ import org.apache.cassandra.tcm.transformations.AlterSchema; import org.apache.cassandra.tcm.transformations.cms.Initialize; import org.apache.cassandra.utils.FBUtilities; -import org.mockito.Mockito; public class CMSTestBase { @@ -87,19 +85,12 @@ public static class CMSSut implements AutoCloseable public final Murmur3Partitioner partitioner; public final LocalLog log; public final ClusterMetadataService service; - public final SchemaProvider schemaProvider; public final TokenPlacementModel.ReplicationFactor rf; public CMSSut(IIsolatedExecutor.SerializableFunction<LocalLog, Processor> processorFactory, boolean addListeners, TokenPlacementModel.ReplicationFactor rf) - { - this(processorFactory, addListeners, Mockito.mock(SchemaProvider.class), rf); - } - - public CMSSut(IIsolatedExecutor.SerializableFunction<LocalLog, Processor> processorFactory, boolean addListeners, SchemaProvider schemaProvider, TokenPlacementModel.ReplicationFactor rf) { partitioner = Murmur3Partitioner.instance; this.rf = rf; - this.schemaProvider = schemaProvider; ClusterMetadata initial = new ClusterMetadata(partitioner); log = LocalLog.logSpec() .sync() @@ -129,7 +120,7 @@ public Result execute(ClusterMetadata prev) }); service.commit(new AlterSchema((cm) -> { return cm.schema.getKeyspaces().with(Keyspaces.of(KeyspaceMetadata.create("test", rf.asKeyspaceParams()))); - }, schemaProvider)); + })); } public void close() diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/ClusterMetadataTestHelper.java b/test/distributed/org/apache/cassandra/distributed/test/log/ClusterMetadataTestHelper.java index d06d79afd1f0..b27952d284a6 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/ClusterMetadataTestHelper.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/ClusterMetadataTestHelper.java @@ -52,7 +52,6 @@ import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.schema.Keyspaces; import org.apache.cassandra.schema.ReplicationParams; -import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.SchemaTransformation; import org.apache.cassandra.service.ClientState; import org.apache.cassandra.tcm.AtomicLongBackedProcessor; @@ -203,7 +202,7 @@ public static void createKeyspace(String name, KeyspaceParams params) CreateKeyspaceStatement createKeyspaceStatement = new CreateKeyspaceStatement(name, attributes, false); try { - commit(new AlterSchema(createKeyspaceStatement, Schema.instance)); + commit(new AlterSchema(createKeyspaceStatement)); } catch (Throwable e) { @@ -216,7 +215,7 @@ public static void createKeyspace(String statement) CreateKeyspaceStatement createKeyspaceStatement = (CreateKeyspaceStatement) QueryProcessor.parseStatement(statement).prepare(ClientState.forInternalCalls()); try { - commit(new AlterSchema(createKeyspaceStatement, Schema.instance)); + commit(new AlterSchema(createKeyspaceStatement)); } catch (Throwable e) { @@ -839,7 +838,7 @@ public static void addOrUpdateKeyspace(KeyspaceMetadata keyspace) try { SchemaTransformation transformation = (cm) -> cm.schema.getKeyspaces().withAddedOrUpdated(keyspace); - commit(new AlterSchema(transformation, Schema.instance)); + commit(new AlterSchema(transformation)); } catch (Exception e) { diff --git a/test/unit/org/apache/cassandra/schema/SchemaChangeDuringRangeMovementTest.java b/test/unit/org/apache/cassandra/schema/SchemaChangeDuringRangeMovementTest.java index 7c7eb8fb228e..e6cdb1be8248 100644 --- a/test/unit/org/apache/cassandra/schema/SchemaChangeDuringRangeMovementTest.java +++ b/test/unit/org/apache/cassandra/schema/SchemaChangeDuringRangeMovementTest.java @@ -157,14 +157,14 @@ public void testRestrictedChanges() throws Throwable "WITH REPLICATION = {'class':'SimpleStrategy','replication_factor':9}", RF9_KS4)); SchemaTransformation dropAllowed = (metadata_) -> metadata_.schema.getKeyspaces().without(RF9_KS4).without(RF9_KS3); - metadata = ClusterMetadataService.instance().commit(new AlterSchema(dropAllowed, Schema.instance)); + metadata = ClusterMetadataService.instance().commit(new AlterSchema(dropAllowed)); assertFalse(metadata.schema.getKeyspaces().containsKeyspace(RF9_KS4)); assertFalse(metadata.schema.getKeyspaces().containsKeyspace(RF9_KS3)); try { SchemaTransformation dropRejected = (metadata_) -> metadata_.schema.getKeyspaces().without(RF9_KS2).without(RF9_KS1); - ClusterMetadataService.instance().commit(new AlterSchema(dropRejected, Schema.instance)); + ClusterMetadataService.instance().commit(new AlterSchema(dropRejected)); fail("Expected exception"); } catch (IllegalStateException e) diff --git a/test/unit/org/apache/cassandra/schema/SchemaTest.java b/test/unit/org/apache/cassandra/schema/SchemaTest.java index d8f176cccd1d..fcf3c6997b4f 100644 --- a/test/unit/org/apache/cassandra/schema/SchemaTest.java +++ b/test/unit/org/apache/cassandra/schema/SchemaTest.java @@ -209,7 +209,9 @@ private void applyAndAssertTableMetadata(SchemaTransformation transformation, bo private static AlterSchemaStatement cql(String keyspace, String cql) { - return (AlterSchemaStatement) QueryProcessor.parseStatement(String.format(cql, keyspace)) - .prepare(ClientState.forInternalCalls()); + AlterSchemaStatement statement = (AlterSchemaStatement) QueryProcessor.parseStatement(String.format(cql, keyspace)) + .prepare(ClientState.forInternalCalls()); + statement.setExecutionTimestamp(System.currentTimeMillis() * 1000); + return statement; } } From 3078aea1cfc70092a185bab8ac5dc8a35627330f Mon Sep 17 00:00:00 2001 From: Branimir Lambov <branimir.lambov@datastax.com> Date: Tue, 19 Nov 2024 12:41:41 +0200 Subject: [PATCH 042/225] Introduce SSTableSimpleScanner for compaction This removes the usage of index files during compaction and simplifies and improves the performance of compaction. patch by Branimir Lambov; reviewed by Sylvain Lebresne for CASSANDRA-20092 --- CHANGES.txt | 1 + .../apache/cassandra/cache/KeyCacheKey.java | 12 +- .../db/compaction/CompactionIterator.java | 2 +- .../io/compress/CompressionMetadata.java | 8 + .../io/sstable/SSTableIdentityIterator.java | 62 ++++++ .../io/sstable/format/SSTableReader.java | 142 +++++++++++-- .../io/sstable/format/SSTableScanner.java | 16 -- .../sstable/format/SSTableSimpleScanner.java | 195 +++++++++++++++++ .../io/sstable/format/big/BigTableReader.java | 37 ---- .../sstable/format/big/BigTableScanner.java | 21 -- .../io/sstable/format/bti/BtiTableReader.java | 23 -- .../sstable/format/bti/BtiTableScanner.java | 21 -- .../cassandra/tools/SSTablePartitions.java | 4 +- .../SSTableCorruptionDetectionTest.java | 33 +++ .../io/sstable/SSTableReaderTest.java | 198 ++++++++++++++++++ .../io/sstable/SSTableScannerTest.java | 125 +++++++++-- 16 files changed, 740 insertions(+), 160 deletions(-) create mode 100644 src/java/org/apache/cassandra/io/sstable/format/SSTableSimpleScanner.java diff --git a/CHANGES.txt b/CHANGES.txt index 49eb15619f38..af48dfb1d339 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Introduce SSTableSimpleScanner for compaction (CASSANDRA-20092) * Include column drop timestamp in alter table transformation (CASSANDRA-18961) * Make JMX SSL configurable in cassandra.yaml (CASSANDRA-18508) * Fix cqlsh CAPTURE command to save query results without trace details when TRACING is ON (CASSANDRA-19105) diff --git a/src/java/org/apache/cassandra/cache/KeyCacheKey.java b/src/java/org/apache/cassandra/cache/KeyCacheKey.java index ac6f1f969311..1a722b05e6c4 100644 --- a/src/java/org/apache/cassandra/cache/KeyCacheKey.java +++ b/src/java/org/apache/cassandra/cache/KeyCacheKey.java @@ -21,7 +21,6 @@ import java.util.Arrays; import java.util.Objects; -import org.apache.cassandra.db.marshal.UTF8Type; import org.apache.cassandra.io.sstable.Descriptor; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.utils.ByteBufferUtil; @@ -31,9 +30,7 @@ public class KeyCacheKey extends CacheKey { public final Descriptor desc; - private static final long EMPTY_SIZE = ObjectSizes.measure(new KeyCacheKey(TableMetadata.builder("ks", "tab") - .addPartitionKeyColumn("pk", UTF8Type.instance) - .build(), null, ByteBufferUtil.EMPTY_BYTE_BUFFER)); + private static final long EMPTY_SIZE = ObjectSizes.measure(new KeyCacheKey()); // keeping an array instead of a ByteBuffer lowers the overhead of the key cache working set, // without extra copies on lookup since client-provided key ByteBuffers will be array-backed already @@ -47,6 +44,13 @@ public KeyCacheKey(TableMetadata tableMetadata, Descriptor desc, ByteBuffer key) assert this.key != null; } + private KeyCacheKey() // Only for EMPTY_SIZE + { + super(null, null); + this.desc = null; + this.key = null; + } + public String toString() { return String.format("KeyCacheKey(%s, %s)", desc, ByteBufferUtil.bytesToHex(ByteBuffer.wrap(key))); diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java b/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java index b27da6f0aecb..eb1e761493d0 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java @@ -290,7 +290,7 @@ private void updateBytesRead() { long n = 0; for (ISSTableScanner scanner : scanners) - n += scanner.getCurrentPosition(); + n += scanner.getBytesScanned(); bytesRead = n; } diff --git a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java index 96b4ce841825..d5f5f05655e9 100644 --- a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java +++ b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java @@ -539,6 +539,14 @@ public String toString() { return String.format("Chunk<offset: %d, length: %d>", offset, length); } + + /** + * @return the end of the chunk in the file, including the checksum + */ + public long chunkEnd() + { + return offset + length + 4; + } } static class ChunkSerializer implements IVersionedSerializer<Chunk> diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java b/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java index cc201b4125c5..072a364af32c 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java @@ -46,6 +46,7 @@ public class SSTableIdentityIterator implements Comparable<SSTableIdentityIterat protected final SSTableSimpleIterator iterator; private final Row staticRow; + boolean isClosed = false; public SSTableIdentityIterator(SSTableReader sstable, DecoratedKey key, DeletionTime partitionLevelDeletion, String filename, SSTableSimpleIterator iterator) throws IOException @@ -100,6 +101,28 @@ public static SSTableIdentityIterator create(SSTableReader sstable, FileDataInpu } } + public static SSTableIdentityIterator create(SSTableReader sstable, FileDataInput dfile, boolean tombstoneOnly) + { + try + { + DecoratedKey key = sstable.decorateKey(ByteBufferUtil.readWithShortLength(dfile)); + DeletionTime partitionLevelDeletion = DeletionTime.getSerializer(sstable.descriptor.version).deserialize(dfile); + if (!partitionLevelDeletion.validate()) + UnfilteredValidation.handleInvalid(sstable.metadata(), key, sstable, "partitionLevelDeletion="+partitionLevelDeletion.toString()); + + DeserializationHelper helper = new DeserializationHelper(sstable.metadata(), sstable.descriptor.version.correspondingMessagingVersion(), DeserializationHelper.Flag.LOCAL); + SSTableSimpleIterator iterator = tombstoneOnly + ? SSTableSimpleIterator.createTombstoneOnly(sstable.metadata(), dfile, sstable.header, helper, partitionLevelDeletion) + : SSTableSimpleIterator.create(sstable.metadata(), dfile, sstable.header, helper, partitionLevelDeletion); + return new SSTableIdentityIterator(sstable, key, partitionLevelDeletion, dfile.getPath(), iterator); + } + catch (IOException e) + { + sstable.markSuspect(); + throw new CorruptSSTableException(e, dfile.getPath()); + } + } + public TableMetadata metadata() { return iterator.metadata; @@ -159,6 +182,9 @@ public Unfiltered next() { try { + if (isClosed) + throw new IllegalStateException("Iterator used after closing."); + return doCompute(); } catch (IndexOutOfBoundsException | VIntOutOfRangeException | AssertionError e) @@ -190,6 +216,42 @@ protected Unfiltered doCompute() public void close() { // creator is responsible for closing file when finished + isClosed = true; + } + + public boolean isClosed() + { + return isClosed; + } + + /** + * Called to advance to the next partition and make sure that we process all outstanding rows if user did not + * do so. Unlike next() and hasNext(), this can and will be called after the iterator is closed. + */ + public void exhaust() + { + try + { + while (iterator.hasNext()) + iterator.next(); + } + catch (IndexOutOfBoundsException | VIntOutOfRangeException | AssertionError e) + { + sstable.markSuspect(); + throw new CorruptSSTableException(e, filename); + } + catch (IOError e) + { + if (e.getCause() instanceof IOException) + { + sstable.markSuspect(); + throw new CorruptSSTableException((Exception)e.getCause(), filename); + } + else + { + throw e; + } + } } public String getPath() diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java index 716d9a815793..fe99b8430d01 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java @@ -719,7 +719,7 @@ public long getCompressionMetadataOffHeapSize() /** * Determine the minimal set of sections that can be extracted from this SSTable to cover the given ranges. * - * @return A sorted list of (offset,end) pairs that cover the given ranges in the datafile for this SSTable. + * @return A sorted list of [offset,end) pairs that cover the given ranges in the datafile for this SSTable. */ public List<PartitionPositionBounds> getPositionsForRanges(Collection<Range<Token>> ranges) { @@ -728,27 +728,110 @@ public List<PartitionPositionBounds> getPositionsForRanges(Collection<Range<Toke for (Range<Token> range : Range.normalize(ranges)) { assert !range.isWrapAround() || range.right.isMinimum(); - // truncate the range so it at most covers the sstable AbstractBounds<PartitionPosition> bounds = Range.makeRowRange(range); - PartitionPosition leftBound = bounds.left.compareTo(first) > 0 ? bounds.left : first.getToken().minKeyBound(); - PartitionPosition rightBound = bounds.right.isMinimum() ? last.getToken().maxKeyBound() : bounds.right; + PartitionPositionBounds pb = getPositionsForBounds(bounds); + if (pb != null) + positions.add(pb); + } + return positions; + } - if (leftBound.compareTo(last) > 0 || rightBound.compareTo(first) < 0) - continue; + /** + * Get a list of data positions in this SSTable that correspond to the given list of bounds. This method will remove + * non-covered intervals, but will not correct order or overlap in the supplied list, e.g. if bounds overlap, the + * result will be sections of the data file that repeat the same positions. + * + * @return A sorted list of [offset,end) pairs corresponding to the given boundsList in the datafile for this + * SSTable. + */ + public List<PartitionPositionBounds> getPositionsForBoundsIterator(Iterator<AbstractBounds<PartitionPosition>> boundsList) + { + // use the index to determine a minimal section for each range + List<PartitionPositionBounds> positions = new ArrayList<>(); + while (boundsList.hasNext()) + { + AbstractBounds<PartitionPosition> bounds = boundsList.next(); + PartitionPositionBounds pb = getPositionsForBounds(bounds); + if (pb != null) + positions.add(pb); + } + return positions; + } - long left = getPosition(leftBound, Operator.GT); - long right = (rightBound.compareTo(last) > 0) - ? uncompressedLength() - : getPosition(rightBound, Operator.GT); + /** + * Determine the data positions in this SSTable that cover the given bounds. + * + * @return An [offset,end) pair that cover the given bounds in the datafile for this SSTable, or null if the range + * is not covered by the sstable or is empty. + */ + public PartitionPositionBounds getPositionsForBounds(AbstractBounds<PartitionPosition> bounds) + { + long left = getPosition(bounds.left, bounds.inclusiveLeft() ? Operator.GE : Operator.GT); + // Note: getPosition will apply a moved start if the sstable is in MOVED_START state. + if (left < 0) // empty range + return null; - if (left == right) - // empty range - continue; + long right = bounds.right.isMinimum() ? -1 + : getPosition(bounds.right, bounds.inclusiveRight() ? Operator.GT + : Operator.GE); + if (right < 0) // right is beyond end + right = uncompressedLength(); // this should also be correct for EARLY readers + + if (left >= right) // empty range + return null; + + return new PartitionPositionBounds(left, right); + } + + /** + * Return an [offset,end) pair that covers the whole file. This could be null if the sstable's moved start has + * made the sstable effectively empty. + */ + public PartitionPositionBounds getPositionsForFullRange() + { + if (openReason != OpenReason.MOVED_START) + return new PartitionPositionBounds(0, uncompressedLength()); + else + { + // query a full range, so that the required adjustments can be applied + PartitionPosition minToken = getPartitioner().getMinimumToken().minKeyBound(); + return getPositionsForBounds(new Range<>(minToken, minToken)); + } + } - assert left < right : String.format("Range=%s openReason=%s first=%s last=%s left=%d right=%d", range, openReason, first, last, left, right); - positions.add(new PartitionPositionBounds(left, right)); + /** + * Calculate a total on-disk (compressed) size for the given partition positions. For uncompressed files this is + * equal to the sum of the size of the covered ranges. For compressed files this is the sum of the size of the + * chunks that contain the requested ranges and may be significantly bigger than the size of the requested ranges. + * + * @param positionBounds a list of [offset,end) pairs that specify the relevant sections of the data file; this must + * be non-overlapping and in ascending order. + */ + public long onDiskSizeForPartitionPositions(Collection<PartitionPositionBounds> positionBounds) + { + long total = 0; + if (!compression) + { + for (PartitionPositionBounds position : positionBounds) + total += position.upperPosition - position.lowerPosition; } - return positions; + else + { + final CompressionMetadata compressionMetadata = getCompressionMetadata(); + long lastEnd = 0; + for (PartitionPositionBounds position : positionBounds) + { + // The end of the chunk that contains the last required byte from the range. + long upperChunkEnd = compressionMetadata.chunkFor(position.upperPosition - 1).chunkEnd(); + // The start of the chunk that contains the first required byte from the range. + long lowerChunkStart = compressionMetadata.chunkFor(position.lowerPosition).offset; + if (lowerChunkStart < lastEnd) // if regions include the same chunk, count it only once + lowerChunkStart = lastEnd; + total += upperChunkEnd - lowerChunkStart; + lastEnd = upperChunkEnd; + } + } + return total; } /** @@ -940,11 +1023,18 @@ public ISSTableScanner getScanner(Range<Token> range) } /** - * Direct I/O SSTableScanner over the entirety of the sstable.. + * Direct I/O SSTableScanner over the entirety of the sstable. * * @return A Scanner over the full content of the SSTable. */ - public abstract ISSTableScanner getScanner(); + public ISSTableScanner getScanner() + { + PartitionPositionBounds fullRange = getPositionsForFullRange(); + if (fullRange != null) + return new SSTableSimpleScanner(this, Collections.singletonList(fullRange)); + else + return new SSTableSimpleScanner(this, Collections.emptyList()); + } /** * Direct I/O SSTableScanner over a defined collection of ranges of tokens. @@ -952,15 +1042,25 @@ public ISSTableScanner getScanner(Range<Token> range) * @param ranges the range of keys to cover * @return A Scanner for seeking over the rows of the SSTable. */ - public abstract ISSTableScanner getScanner(Collection<Range<Token>> ranges); + public ISSTableScanner getScanner(Collection<Range<Token>> ranges) + { + if (ranges != null) + return new SSTableSimpleScanner(this, getPositionsForRanges(ranges)); + else + return getScanner(); + } /** * Direct I/O SSTableScanner over an iterator of bounds. * - * @param rangeIterator the keys to cover + * @param boundsIterator the keys to cover * @return A Scanner for seeking over the rows of the SSTable. */ - public abstract ISSTableScanner getScanner(Iterator<AbstractBounds<PartitionPosition>> rangeIterator); + public ISSTableScanner getScanner(Iterator<AbstractBounds<PartitionPosition>> boundsIterator) + { + return new SSTableSimpleScanner(this, getPositionsForBoundsIterator(boundsIterator)); + } + /** * Create a {@link FileDataInput} for the data file of the sstable represented by this reader. This method returns diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableScanner.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableScanner.java index 217c17720639..86c0f7ec2dc0 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableScanner.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableScanner.java @@ -19,7 +19,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Set; @@ -35,9 +34,7 @@ import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.dht.AbstractBounds; import org.apache.cassandra.dht.AbstractBounds.Boundary; -import org.apache.cassandra.dht.Bounds; import org.apache.cassandra.dht.Range; -import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.AbstractRowIndexEntry; import org.apache.cassandra.io.sstable.CorruptSSTableException; import org.apache.cassandra.io.sstable.ISSTableScanner; @@ -86,14 +83,6 @@ protected SSTableScanner(S sstable, this.listener = listener; } - protected static List<AbstractBounds<PartitionPosition>> makeBounds(SSTableReader sstable, Collection<Range<Token>> tokenRanges) - { - List<AbstractBounds<PartitionPosition>> boundsList = new ArrayList<>(tokenRanges.size()); - for (Range<Token> range : Range.normalize(tokenRanges)) - addRange(sstable, Range.makeRowRange(range), boundsList); - return boundsList; - } - protected static List<AbstractBounds<PartitionPosition>> makeBounds(SSTableReader sstable, DataRange dataRange) { List<AbstractBounds<PartitionPosition>> boundsList = new ArrayList<>(2); @@ -101,11 +90,6 @@ protected static List<AbstractBounds<PartitionPosition>> makeBounds(SSTableReade return boundsList; } - protected static AbstractBounds<PartitionPosition> fullRange(SSTableReader sstable) - { - return new Bounds<>(sstable.getFirst(), sstable.getLast()); - } - private static void addRange(SSTableReader sstable, AbstractBounds<PartitionPosition> requested, List<AbstractBounds<PartitionPosition>> boundsList) { if (requested instanceof Range && ((Range<?>) requested).isWrapAround()) diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableSimpleScanner.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableSimpleScanner.java new file mode 100644 index 000000000000..6015265ba045 --- /dev/null +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableSimpleScanner.java @@ -0,0 +1,195 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.io.sstable.format; + +import java.util.Collection; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; + +import com.google.common.collect.ImmutableSet; + +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.io.sstable.CorruptSSTableException; +import org.apache.cassandra.io.sstable.ISSTableScanner; +import org.apache.cassandra.io.sstable.SSTableIdentityIterator; +import org.apache.cassandra.io.util.RandomAccessReader; +import org.apache.cassandra.schema.TableMetadata; + +import static org.apache.cassandra.io.sstable.format.SSTableReader.PartitionPositionBounds; + +/// Simple SSTable scanner that reads sequentially through an SSTable without using the index. +/// +/// This is a significant improvement for the performance of compaction over using the full-blown DataRange-capable +/// [SSTableScanner] and enables correct calculation of data sizes to process. +public class SSTableSimpleScanner +implements ISSTableScanner +{ + private final AtomicBoolean isClosed = new AtomicBoolean(false); + private final RandomAccessReader dfile; + private final SSTableReader sstable; + + private final Iterator<PartitionPositionBounds> rangeIterator; + + private long bytesScannedInPreviousRanges; + + private final long sizeInBytes; + private final long compressedSizeInBytes; + + private long currentEndPosition; + private long currentStartPosition; + + private SSTableIdentityIterator currentIterator; + private DecoratedKey lastKey; + + /// Create a new simple scanner over the given sstables and the given ranges of uncompressed positions. + /// Each range must start and end on a partition boundary, and, to satisfy the contract of [ISSTableScanner], the + /// ranges must be non-overlapping and in ascending order. This scanner will throw an [IllegalArgumentException] if + /// the latter is not true. + /// + /// The ranges can be constructed by [SSTableReader#getPositionsForRanges] and similar methods as done by the + /// various [SSTableReader#getScanner] variations. + public SSTableSimpleScanner(SSTableReader sstable, + Collection<PartitionPositionBounds> boundsList) + { + assert sstable != null; + + this.dfile = sstable.openDataReader(); + this.sstable = sstable; + this.sizeInBytes = boundsList.stream().mapToLong(ppb -> ppb.upperPosition - ppb.lowerPosition).sum(); + this.compressedSizeInBytes = sstable.compression ? sstable.onDiskSizeForPartitionPositions(boundsList) : sizeInBytes; + this.rangeIterator = boundsList.iterator(); + this.currentEndPosition = 0; + this.currentStartPosition = 0; + this.bytesScannedInPreviousRanges = 0; + this.currentIterator = null; + this.lastKey = null; + } + + public void close() + { + if (isClosed.compareAndSet(false, true)) + { + // ensure we report what we have actually processed + bytesScannedInPreviousRanges += dfile.getFilePointer() - currentStartPosition; + dfile.close(); + // close() may change the file pointer, update so that the difference is 0 when reported by getBytesScanned() + currentStartPosition = dfile.getFilePointer(); + } + } + + @Override + public long getLengthInBytes() + { + return sizeInBytes; + } + + + public long getCompressedLengthInBytes() + { + return compressedSizeInBytes; + } + + @Override + public long getCurrentPosition() + { + return dfile.getFilePointer(); + } + + public long getBytesScanned() + { + return bytesScannedInPreviousRanges + dfile.getFilePointer() - currentStartPosition; + } + + @Override + public Set<SSTableReader> getBackingSSTables() + { + return ImmutableSet.of(sstable); + } + + public TableMetadata metadata() + { + return sstable.metadata(); + } + + public boolean hasNext() + { + if (currentIterator != null) + { + currentIterator.close(); // Ensure that the iterator cannot be used further. No op if already closed. + + // Row iterator must be exhausted to advance to next partition + currentIterator.exhaust(); + currentIterator = null; + } + + if (dfile.getFilePointer() < currentEndPosition) + return true; + + return advanceRange(); + } + + boolean advanceRange() + { + if (!rangeIterator.hasNext()) + return false; + + bytesScannedInPreviousRanges += currentEndPosition - currentStartPosition; + + PartitionPositionBounds nextRange = rangeIterator.next(); + if (currentEndPosition > nextRange.lowerPosition) + throw new IllegalArgumentException("Ranges supplied to SSTableSimpleScanner must be non-overlapping and in ascending order."); + + currentEndPosition = nextRange.upperPosition; + currentStartPosition = nextRange.lowerPosition; + dfile.seek(currentStartPosition); + return true; + } + + public UnfilteredRowIterator next() + { + if (!hasNext()) + throw new NoSuchElementException(); + + currentIterator = SSTableIdentityIterator.create(sstable, dfile, false); + DecoratedKey currentKey = currentIterator.partitionKey(); + if (lastKey != null && lastKey.compareTo(currentKey) >= 0) + { + sstable.markSuspect(); + throw new CorruptSSTableException(new IllegalStateException(String.format("Invalid key order: current %s <= previous %s", + currentKey, + lastKey)), + sstable.getFilename()); + } + lastKey = currentKey; + return currentIterator; + } + + public void remove() + { + throw new UnsupportedOperationException(); + } + + @Override + public String toString() + { + return String.format("%s(sstable=%s)", getClass().getSimpleName(), sstable); + } +} diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java index b58dbc532eaf..692cadf34df4 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java @@ -21,7 +21,6 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; -import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -46,7 +45,6 @@ import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.db.rows.UnfilteredRowIteratorWithLowerBound; import org.apache.cassandra.db.rows.UnfilteredRowIterators; -import org.apache.cassandra.dht.AbstractBounds; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.AbstractRowIndexEntry; @@ -157,41 +155,6 @@ public KeyReader keyReader() throws IOException return BigTableKeyReader.create(ifile, rowIndexEntrySerializer); } - /** - * Direct I/O SSTableScanner over an iterator of bounds. - * - * @param boundsIterator the keys to cover - * @return A Scanner for seeking over the rows of the SSTable. - */ - public ISSTableScanner getScanner(Iterator<AbstractBounds<PartitionPosition>> boundsIterator) - { - return BigTableScanner.getScanner(this, boundsIterator); - } - - /** - * Direct I/O SSTableScanner over the full sstable. - * - * @return A Scanner for reading the full SSTable. - */ - public ISSTableScanner getScanner() - { - return BigTableScanner.getScanner(this); - } - - /** - * Direct I/O SSTableScanner over a defined collection of ranges of tokens. - * - * @param ranges the range of keys to cover - * @return A Scanner for seeking over the rows of the SSTable. - */ - public ISSTableScanner getScanner(Collection<Range<Token>> ranges) - { - if (ranges != null) - return BigTableScanner.getScanner(this, ranges); - else - return getScanner(); - } - /** * Finds and returns the first key beyond a given token in this SSTable or null if no such key exists. */ diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableScanner.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableScanner.java index 887d99784665..83243529c4c9 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableScanner.java +++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableScanner.java @@ -18,11 +18,8 @@ package org.apache.cassandra.io.sstable.format.big; import java.io.IOException; -import java.util.Collection; import java.util.Iterator; -import com.google.common.collect.Iterators; - import org.apache.cassandra.db.DataRange; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.PartitionPosition; @@ -30,8 +27,6 @@ import org.apache.cassandra.db.filter.ColumnFilter; import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.dht.AbstractBounds; -import org.apache.cassandra.dht.Range; -import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.CorruptSSTableException; import org.apache.cassandra.io.sstable.ISSTableScanner; import org.apache.cassandra.io.sstable.SSTable; @@ -50,12 +45,6 @@ public class BigTableScanner extends SSTableScanner<BigTableReader, RowIndexEntr private final RowIndexEntry.IndexSerializer rowIndexEntrySerializer; - // Full scan of the sstables - public static ISSTableScanner getScanner(BigTableReader sstable) - { - return getScanner(sstable, Iterators.singletonIterator(fullRange(sstable))); - } - public static ISSTableScanner getScanner(BigTableReader sstable, ColumnFilter columns, DataRange dataRange, @@ -64,16 +53,6 @@ public static ISSTableScanner getScanner(BigTableReader sstable, return new BigTableScanner(sstable, columns, dataRange, makeBounds(sstable, dataRange).iterator(), listener); } - public static ISSTableScanner getScanner(BigTableReader sstable, Collection<Range<Token>> tokenRanges) - { - return getScanner(sstable, makeBounds(sstable, tokenRanges).iterator()); - } - - public static ISSTableScanner getScanner(BigTableReader sstable, Iterator<AbstractBounds<PartitionPosition>> rangeIterator) - { - return new BigTableScanner(sstable, ColumnFilter.all(sstable.metadata()), null, rangeIterator, SSTableReadsListener.NOOP_LISTENER); - } - private BigTableScanner(BigTableReader sstable, ColumnFilter columns, DataRange dataRange, diff --git a/src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java index c5571e7fbbe3..9a65be1137bb 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java @@ -22,7 +22,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.Iterator; import java.util.List; import com.google.common.annotations.VisibleForTesting; @@ -45,7 +44,6 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.CorruptSSTableException; import org.apache.cassandra.io.sstable.Descriptor; -import org.apache.cassandra.io.sstable.ISSTableScanner; import org.apache.cassandra.io.sstable.IVerifier; import org.apache.cassandra.io.sstable.SSTable; import org.apache.cassandra.io.sstable.SSTableReadsListener; @@ -382,27 +380,6 @@ public UnfilteredRowIterator rowIterator(FileDataInput dataFileInput, return new SSTableIterator(this, dataFileInput, key, indexEntry, slices, selectedColumns, rowIndexFile); } - @Override - public ISSTableScanner getScanner() - { - return BtiTableScanner.getScanner(this); - } - - @Override - public ISSTableScanner getScanner(Collection<Range<Token>> ranges) - { - if (ranges != null) - return BtiTableScanner.getScanner(this, ranges); - else - return getScanner(); - } - - @Override - public ISSTableScanner getScanner(Iterator<AbstractBounds<PartitionPosition>> rangeIterator) - { - return BtiTableScanner.getScanner(this, rangeIterator); - } - @VisibleForTesting @Override public BtiTableReader cloneAndReplace(IFilter filter) diff --git a/src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScanner.java b/src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScanner.java index a9f862c68b50..4507ccf7f5e4 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScanner.java +++ b/src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScanner.java @@ -19,11 +19,8 @@ import java.io.Closeable; import java.io.IOException; -import java.util.Collection; import java.util.Iterator; -import com.google.common.collect.Iterators; - import org.apache.cassandra.db.DataRange; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.PartitionPosition; @@ -31,20 +28,12 @@ import org.apache.cassandra.db.filter.ColumnFilter; import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.dht.AbstractBounds; -import org.apache.cassandra.dht.Range; -import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.SSTableReadsListener; import org.apache.cassandra.io.sstable.format.SSTableScanner; import org.apache.cassandra.io.util.FileUtils; public class BtiTableScanner extends SSTableScanner<BtiTableReader, TrieIndexEntry, BtiTableScanner.BtiScanningIterator> { - // Full scan of the sstables - public static BtiTableScanner getScanner(BtiTableReader sstable) - { - return getScanner(sstable, Iterators.singletonIterator(fullRange(sstable))); - } - public static BtiTableScanner getScanner(BtiTableReader sstable, ColumnFilter columns, DataRange dataRange, @@ -53,16 +42,6 @@ public static BtiTableScanner getScanner(BtiTableReader sstable, return new BtiTableScanner(sstable, columns, dataRange, makeBounds(sstable, dataRange).iterator(), listener); } - public static BtiTableScanner getScanner(BtiTableReader sstable, Collection<Range<Token>> tokenRanges) - { - return getScanner(sstable, makeBounds(sstable, tokenRanges).iterator()); - } - - public static BtiTableScanner getScanner(BtiTableReader sstable, Iterator<AbstractBounds<PartitionPosition>> rangeIterator) - { - return new BtiTableScanner(sstable, ColumnFilter.all(sstable.metadata()), null, rangeIterator, SSTableReadsListener.NOOP_LISTENER); - } - private BtiTableScanner(BtiTableReader sstable, ColumnFilter columns, DataRange dataRange, diff --git a/src/java/org/apache/cassandra/tools/SSTablePartitions.java b/src/java/org/apache/cassandra/tools/SSTablePartitions.java index 2181346271bb..b435994bbaff 100644 --- a/src/java/org/apache/cassandra/tools/SSTablePartitions.java +++ b/src/java/org/apache/cassandra/tools/SSTablePartitions.java @@ -369,13 +369,15 @@ private static void processSSTable(String[] keys, { while (scanner.hasNext()) { + // hasNext() positions us on the next partition, next() has to advance to read its header. + long startOfPartition = scanner.getCurrentPosition(); try (UnfilteredRowIterator partition = scanner.next()) { ByteBuffer key = partition.partitionKey().getKey(); boolean isExcluded = excludedKeys.contains(metadata.partitionKeyType.getString(key)); PartitionStats partitionStats = new PartitionStats(key, - scanner.getCurrentPosition(), + startOfPartition, partition.partitionLevelDeletion().isLive()); // Consume the partition to populate the stats. diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableCorruptionDetectionTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableCorruptionDetectionTest.java index 21ac51ee865d..c631d5d09979 100644 --- a/test/unit/org/apache/cassandra/io/sstable/SSTableCorruptionDetectionTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/SSTableCorruptionDetectionTest.java @@ -37,6 +37,7 @@ import org.apache.cassandra.config.Config; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DataRange; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.Slices; @@ -156,6 +157,12 @@ public void testSSTableScanner() throws Throwable bruteForceCorruptionTest(ssTableReader, sstableScanner()); } + @Test + public void testSSTableSimpleScanner() throws Throwable + { + bruteForceCorruptionTest(ssTableReader, sstableSimpleScanner()); + } + private void bruteForceCorruptionTest(SSTableReader ssTableReader, Consumer<SSTableReader> walker) throws Throwable { FileChannel fc = new File(ssTableReader.getFilename()).newReadWriteChannel(); @@ -193,6 +200,32 @@ private void bruteForceCorruptionTest(SSTableReader ssTableReader, Consumer<SSTa } private Consumer<SSTableReader> sstableScanner() + { + return (SSTableReader sstable) -> { + try (var scanner = sstable.partitionIterator(ColumnFilter.NONE, DataRange.allData(sstable.getPartitioner()), SSTableReadsListener.NOOP_LISTENER)) + { + while (scanner.hasNext()) + { + try (UnfilteredRowIterator rowIter = scanner.next()) + { + if (rowIter.hasNext()) + { + Unfiltered unfiltered = rowIter.next(); + if (unfiltered.isRow()) + { + Row row = (Row) unfiltered; + assertEquals(2, row.clustering().size()); + // no-op read + } + } + } + + } + } + }; + } + + private Consumer<SSTableReader> sstableSimpleScanner() { return (SSTableReader sstable) -> { try (ISSTableScanner scanner = sstable.getScanner()) diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java index 9cb7ca3d14f5..a87ec024b58f 100644 --- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java @@ -30,8 +30,11 @@ import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import java.util.stream.Stream; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; import org.junit.Assume; import org.junit.BeforeClass; @@ -89,6 +92,7 @@ import static java.lang.String.format; import static org.apache.cassandra.cql3.QueryProcessor.executeInternal; +import static org.apache.cassandra.db.ColumnFamilyStore.FlushReason.UNIT_TESTS; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -178,6 +182,200 @@ public void testGetPositionsForRanges() } } + @Test + public void testOnDiskSizeForRanges() + { + ColumnFamilyStore store = discardSSTables(KEYSPACE1, CF_STANDARD2); + partitioner = store.getPartitioner(); + int count = 1000; + + // insert data and compact to a single sstable + for (int j = 0; j < count; j++) + { + new RowUpdateBuilder(store.metadata(), 15000, k0(j)) + .clustering("0") + .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER) + .build() + .applyUnsafe(); + } + store.forceBlockingFlush(UNIT_TESTS); + store.forceMajorCompaction(); + + SSTableReader sstable = store.getLiveSSTables().iterator().next(); + + // Non-compression-dependent checks + // Check several ways of going through the whole file + assertEquals(sstable.onDiskLength(), + onDiskSizeForRanges(sstable, Collections.singleton(new Range<>(t(cut(k0(0), 1)), t0(count - 1))))); + assertEquals(sstable.onDiskLength(), + onDiskSizeForRanges(sstable, Collections.singleton(new Range<>(sstable.getPartitioner().getMinimumToken(), + sstable.getPartitioner().getMinimumToken())))); + assertEquals(sstable.onDiskLength(), + onDiskSizeForRanges(sstable, Collections.singleton(new Range<>(sstable.getPartitioner().getMinimumToken(), + sstable.getLast().getToken())))); + + // Split at exact match + assertEquals(sstable.onDiskLength(), + onDiskSizeForRanges(sstable, ImmutableList.of(new Range<>(t(cut(k0(0), 1)), t0(347)), + new Range<>(t0(347), t0(count - 1))))); + + // Split at different prefixes pointing to the same position + assertEquals(sstable.onDiskLength(), + onDiskSizeForRanges(sstable, ImmutableList.of(new Range<>(t(cut(k0(0), 1)), t(cut(k0(600), 2))), + new Range<>(t(cut(k0(600), 1)), t0(count - 1))))); + + // Size one row + double oneRowSize = sstable.uncompressedLength() * 1.0 / count; + System.out.println("One row size: " + oneRowSize); + + if (!sstable.compression) + { + double delta = 0.9; + + // Ranges are end-inclusive, indexes are adjusted by one here to account for that. + assertEquals((52 - 38), + onDiskSizeForRanges(sstable, Collections.singleton(new Range<>(t0(37), t0(51)))) / oneRowSize, + delta); + + // Try non-matching positions (inexact indexes are not adjusted for the count). + assertEquals((34 - 30), + onDiskSizeForRanges(sstable, Collections.singleton(new Range<>(t(cut(k0(30), 1)), + t0(33)))) / oneRowSize, + delta); + + assertEquals((700 - 554), + onDiskSizeForRanges(sstable, Collections.singleton(new Range<>(t0(553), + t(cut(k0(700), 2))))) / oneRowSize, + delta); + + assertEquals((500 - 30), + onDiskSizeForRanges(sstable, Collections.singleton(new Range<>(t(cut(k0(30), 1)), + t(cut(k0(500), 2))))) / oneRowSize, + delta); + + // Try a list + List<Range<Token>> ranges = ImmutableList.of(new Range<>(t0(37), t0(51)), + new Range<>(t0(71), t(cut(k0(100), 2))), + new Range<>(t(cut(k0(230), 1)), t0(243)), + new Range<>(t(cut(k0(260), 1)), t(cut(k0(300), 2))), + new Range<>(t0(373), t0(382)), + new Range<>(t0(382), t0(385)), + new Range<>(t(cut(k0(400), 2)), t(cut(k0(400), 1))), // empty range + new Range<>(t0(563), t(cut(k0(600), 2))), // touching ranges + new Range<>(t(cut(k0(600), 1)), t0(621)) + ); + assertEquals((52 - 38 + 100 - 72 + 244 - 230 + 300 - 260 + 383 - 374 + 386 - 383 + 400 - 400 + 622 - 564), + onDiskSizeForRanges(sstable, ranges) / oneRowSize, + delta); + + // Check going through the whole file + assertEquals(sstable.onDiskLength(), + onDiskSizeForRanges(sstable, Collections.singleton(new Range<>(t(cut(k0(0), 1)), t0(count - 1))))); + + assertEquals(sstable.onDiskLength(), + onDiskSizeForRanges(sstable, ImmutableList.of(new Range<>(t(cut(k0(0), 1)), t0(347)), + new Range<>(t0(347), t0(count - 1))))); + + assertEquals(sstable.onDiskLength(), + onDiskSizeForRanges(sstable, ImmutableList.of(new Range<>(t(cut(k0(0), 1)), t(cut(k0(600), 2))), + new Range<>(t(cut(k0(600), 1)), t0(count - 1))))); + } + else + { + // It's much harder to test with compression. + + // Check first three rows have the same size (they must be in the same chunk) + final long row0size = onDiskSizeForRanges(sstable, Collections.singleton(new Range<>(t(cut(k0(0), 1)), t0(0)))); + assertEquals(row0size, onDiskSizeForRanges(sstable, Collections.singleton(new Range<>(t0(0), t0(1))))); + assertEquals(row0size, onDiskSizeForRanges(sstable, Collections.singleton(new Range<>(t0(1), t0(2))))); + + // As well as the first three rows together + assertEquals(row0size, onDiskSizeForRanges(sstable, Collections.singleton(new Range<>(t(cut(k0(0), 1)), t0(2))))); + + // And also when we query for them in separate ranges + assertEquals(row0size, onDiskSizeForRanges(sstable, ImmutableList.of(new Range<>(t(cut(k0(0), 1)), t0(0)), + new Range<>(t0(0), t0(1))))); + assertEquals(row0size, onDiskSizeForRanges(sstable, ImmutableList.of(new Range<>(t(cut(k0(0), 1)), t0(0)), + new Range<>(t0(1), t0(2))))); + assertEquals(row0size, onDiskSizeForRanges(sstable, ImmutableList.of(new Range<>(t(cut(k0(0), 1)), t0(0)), + new Range<>(t0(0), t0(1)), + new Range<>(t0(1), t0(2))))); + + // Finally, check that if we query for every second row we get the total size of the file. + assertEquals(sstable.onDiskLength(), + onDiskSizeForRanges(sstable, IntStream.range(0, count) + .filter(i -> i % 2 != 0) + .mapToObj(i -> new Range<>(t0(i), t0(i + 1))) + .collect(Collectors.toList()))); + } + } + + + @Test + public void testOnDiskSizeCompressedBoundaries() + { + ColumnFamilyStore store = discardSSTables(KEYSPACE1, CF_COMPRESSED); + partitioner = store.getPartitioner(); + int count = 1000; + // Use a longish string to let a key align with a chunk boundary + ByteBuffer dataBuf = ByteBufferUtil.bytes(String.format("%43d", 123)); + + // insert data and compact to a single sstable + for (int j = 0; j < count; j++) + { + new RowUpdateBuilder(store.metadata(), 15000, k0(j)) + .clustering("0") + .add("val", dataBuf) + .build() + .applyUnsafe(); + } + store.forceBlockingFlush(UNIT_TESTS); + store.forceMajorCompaction(); + + SSTableReader sstable = store.getLiveSSTables().iterator().next(); + + int chunkLength = sstable.getCompressionMetadata().chunkLength(); + System.out.println("Chunk length: " + chunkLength); + int[] alignedKeys = IntStream.range(0, count).filter(i -> (sstable.getPosition(dk0(i), SSTableReader.Operator.EQ) & (chunkLength - 1)) == 0).toArray(); + assertTrue("Test needs an aligned key, try changing the length of dataBuf", alignedKeys.length > 1); + for (int k : alignedKeys) + assertEquals("Coverage must not include chunk starting at end position", + sstable.getCompressionMetadata().chunkFor(sstable.getPosition(dk0(k), SSTableReader.Operator.EQ)).offset, + onDiskSizeForRanges(sstable, Collections.singleton(new Range<>(partitioner.getMinimumToken(), t0(k - 1))))); // inclusive end + } + + + long onDiskSizeForRanges(SSTableReader sstable, Collection<Range<Token>> ranges) + { + return sstable.onDiskSizeForPartitionPositions(sstable.getPositionsForRanges(ranges)); + } + + private Token t(String key) + { + return partitioner.getToken(ByteBufferUtil.bytes(key)); + } + + private String k0(int k) + { + return String.format("%08d", k); + } + + private Token t0(int k) + { + return t(k0(k)); + } + + private DecoratedKey dk0(int k) + { + return partitioner.decorateKey(ByteBufferUtil.bytes(k0(k))); + } + + private String cut(String s, int n) + { + return s.substring(0, s.length() - n); + } + + @Test public void testSpannedIndexPositions() throws IOException { diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java index 17b8a6cbb2af..73195b0617ab 100644 --- a/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java @@ -21,10 +21,13 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.function.Consumer; +import java.util.function.Function; import com.google.common.collect.Iterables; +import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; @@ -40,6 +43,7 @@ import org.apache.cassandra.db.filter.ClusteringIndexSliceFilter; import org.apache.cassandra.db.filter.ColumnFilter; import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; +import org.apache.cassandra.db.rows.Row; import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.dht.AbstractBounds; import org.apache.cassandra.dht.ByteOrderedPartitioner; @@ -49,6 +53,7 @@ import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.utils.ByteBufferUtil; +import org.hamcrest.Matchers; import static org.apache.cassandra.dht.AbstractBounds.isEmpty; import static org.junit.Assert.assertEquals; @@ -180,6 +185,12 @@ private static void insertRowWithKey(TableMetadata metadata, int key) } private static void assertScanMatches(SSTableReader sstable, int scanStart, int scanEnd, int ... boundaries) + { + assertScanMatchesUsingScanner(sstable, scanStart, scanEnd, boundaries); + assertScanMatchesUsingSimple(sstable, scanStart, scanEnd, boundaries); + } + + private static void assertScanMatchesUsingScanner(SSTableReader sstable, int scanStart, int scanEnd, int ... boundaries) { assert boundaries.length % 2 == 0; for (DataRange range : dataRanges(sstable.metadata(), scanStart, scanEnd)) @@ -200,6 +211,28 @@ private static void assertScanMatches(SSTableReader sstable, int scanStart, int } } + private static void assertScanMatchesUsingSimple(SSTableReader sstable, int scanStart, int scanEnd, int ... boundaries) + { + assert boundaries.length % 2 == 0; + for (DataRange range : dataRanges(sstable.metadata(), scanStart, scanEnd)) + { + if (range.isWrapAround() && !range.keyRange().right.isMinimum()) // getScanner on AbstractBounds<PartitionPosition> does not handle wraparounds + continue; + + try(UnfilteredPartitionIterator scanner = sstable.getScanner(Collections.singleton(range.keyRange()).iterator())) + { + for (int b = 0; b < boundaries.length; b += 2) + for (int i = boundaries[b]; i <= boundaries[b + 1]; i++) + assertEquals(toKey(i), new String(scanner.next().partitionKey().getKey().array())); + assertFalse(scanner.hasNext()); + } + catch (Exception e) + { + throw new RuntimeException(e); + } + } + } + private static void assertScanEmpty(SSTableReader sstable, int scanStart, int scanEnd) { assertScanMatches(sstable, scanStart, scanEnd); @@ -547,7 +580,30 @@ public void testSingleKeyMultipleRanges() throws IOException assertScanContainsRanges(scanner, 205, 205); } - private static void testRequestNextRowIteratorWithoutConsumingPrevious(Consumer<ISSTableScanner> consumer) + private static void testRequestNextRowIteratorWithoutConsumingPrevious(Function<SSTableReader, UnfilteredPartitionIterator> makeScanner, + Consumer<UnfilteredPartitionIterator> requestNext, + String messagePattern) + { + final SSTableReader sstable = prepareSmallSSTable(); + + try (UnfilteredPartitionIterator scanner = makeScanner.apply(sstable); + UnfilteredRowIterator currentRowIterator = scanner.next()) + { + assertTrue(currentRowIterator.hasNext()); + try + { + requestNext.accept(scanner); + currentRowIterator.next(); + fail("Should have thrown IllegalStateException"); + } + catch (IllegalStateException e) + { + Assert.assertThat(e.getMessage(), Matchers.matchesPattern(messagePattern)); + } + } + } + + private static SSTableReader prepareSmallSSTable() { Keyspace keyspace = Keyspace.open(KEYSPACE); ColumnFamilyStore store = keyspace.getColumnFamilyStore(TABLE); @@ -557,38 +613,77 @@ private static void testRequestNextRowIteratorWithoutConsumingPrevious(Consumer< store.disableAutoCompaction(); insertRowWithKey(store.metadata(), 0); + insertRowWithKey(store.metadata(), 3); store.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); assertEquals(1, store.getLiveSSTables().size()); SSTableReader sstable = store.getLiveSSTables().iterator().next(); + return sstable; + } - try (ISSTableScanner scanner = sstable.getScanner(); - UnfilteredRowIterator currentRowIterator = scanner.next()) + @Test + public void testSimpleHasNextRowIteratorWithoutConsumingPrevious() + { + testRequestNextRowIteratorWithoutConsumingPrevious(SSTableReader::getScanner, + UnfilteredPartitionIterator::hasNext, + "Iterator used after closing."); + } + + @Test + public void testSimpleNextRowIteratorWithoutConsumingPrevious() + { + testRequestNextRowIteratorWithoutConsumingPrevious(SSTableReader::getScanner, + UnfilteredPartitionIterator::next, + "Iterator used after closing."); + } + + @Test + public void testHasNextRowIteratorWithoutConsumingPrevious() + { + testRequestNextRowIteratorWithoutConsumingPrevious(r -> r.partitionIterator(ColumnFilter.NONE, DataRange.allData(r.getPartitioner()), SSTableReadsListener.NOOP_LISTENER), + UnfilteredPartitionIterator::hasNext, + ".*UnfilteredRowIterator.*must be closed.*"); + } + + @Test + public void testNextRowIteratorWithoutConsumingPrevious() + { + testRequestNextRowIteratorWithoutConsumingPrevious(r -> r.partitionIterator(ColumnFilter.NONE, DataRange.allData(r.getPartitioner()), SSTableReadsListener.NOOP_LISTENER), + UnfilteredPartitionIterator::next, + ".*UnfilteredRowIterator.*must be closed.*"); + } + + private static void testRequestNextRowIteratorAfterClosingPrevious(Function<SSTableReader, UnfilteredPartitionIterator> makeScanner) + { + final SSTableReader sstable = prepareSmallSSTable(); + + try (UnfilteredPartitionIterator scanner = makeScanner.apply(sstable)) { - assertTrue(currentRowIterator.hasNext()); - try + try (UnfilteredRowIterator p = scanner.next()) { - consumer.accept(scanner); - fail("Should have thrown IllegalStateException"); + assertEquals(toKey(0), new String(p.partitionKey().getKey().array())); + // do not read it, but close it } - catch (IllegalStateException e) + + try (UnfilteredRowIterator p = scanner.next()) { - assertEquals("The UnfilteredRowIterator returned by the last call to next() was initialized: " + - "it must be closed before calling hasNext() or next() again.", - e.getMessage()); + assertEquals(toKey(3), new String(p.partitionKey().getKey().array())); + assertTrue(p.hasNext()); + assertTrue(p.next() instanceof Row); } } } + @Test - public void testHasNextRowIteratorWithoutConsumingPrevious() + public void testSimpleRequestNextRowIteratorAfterClosingPreviouss() { - testRequestNextRowIteratorWithoutConsumingPrevious(ISSTableScanner::hasNext); + testRequestNextRowIteratorAfterClosingPrevious(SSTableReader::getScanner); } @Test - public void testNextRowIteratorWithoutConsumingPrevious() + public void testRequestNextRowIteratorAfterClosingPrevious() { - testRequestNextRowIteratorWithoutConsumingPrevious(ISSTableScanner::next); + testRequestNextRowIteratorAfterClosingPrevious(r -> r.partitionIterator(ColumnFilter.NONE, DataRange.allData(r.getPartitioner()), SSTableReadsListener.NOOP_LISTENER)); } } From bc40d3bb27cac7979867def92fe84e5fa5011155 Mon Sep 17 00:00:00 2001 From: Ekaterina Dimitrova <ekaterina.dimitrova@datastax.com> Date: Wed, 4 Dec 2024 15:46:30 -0500 Subject: [PATCH 043/225] Fix RequestFailureReason constants codes patch by Ekaterina Dimitrova; reviewed by Caleb Rackliffe and Stefan Miklosovic for CASSANDRA-20126 --- CHANGES.txt | 1 + .../exceptions/RequestFailureReason.java | 8 +-- .../exceptions/RequestFailureReasonTest.java | 57 +++++++++++++++++++ 3 files changed, 62 insertions(+), 4 deletions(-) create mode 100644 test/unit/org/apache/cassandra/exceptions/RequestFailureReasonTest.java diff --git a/CHANGES.txt b/CHANGES.txt index af48dfb1d339..08ff72184a36 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Fix RequestFailureReason constants codes (CASSANDRA-20126) * Introduce SSTableSimpleScanner for compaction (CASSANDRA-20092) * Include column drop timestamp in alter table transformation (CASSANDRA-18961) * Make JMX SSL configurable in cassandra.yaml (CASSANDRA-18508) diff --git a/src/java/org/apache/cassandra/exceptions/RequestFailureReason.java b/src/java/org/apache/cassandra/exceptions/RequestFailureReason.java index 041aa5e9a33a..1bc86ff061ab 100644 --- a/src/java/org/apache/cassandra/exceptions/RequestFailureReason.java +++ b/src/java/org/apache/cassandra/exceptions/RequestFailureReason.java @@ -38,10 +38,10 @@ public enum RequestFailureReason READ_SIZE (4), NODE_DOWN (5), INDEX_NOT_AVAILABLE (6), - NOT_CMS (7), - INVALID_ROUTING (8), - COORDINATOR_BEHIND (9), - READ_TOO_MANY_INDEXES (10), + READ_TOO_MANY_INDEXES (7), + NOT_CMS (8), + INVALID_ROUTING (9), + COORDINATOR_BEHIND (10), ; public static final Serializer serializer = new Serializer(); diff --git a/test/unit/org/apache/cassandra/exceptions/RequestFailureReasonTest.java b/test/unit/org/apache/cassandra/exceptions/RequestFailureReasonTest.java new file mode 100644 index 000000000000..b2fdcd365d73 --- /dev/null +++ b/test/unit/org/apache/cassandra/exceptions/RequestFailureReasonTest.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.exceptions; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class RequestFailureReasonTest +{ + private static final RequestFailureReason[] REASONS = RequestFailureReason.values(); + private static final Object[][] EXPECTED_VALUES = + { + { 0, "UNKNOWN" }, + { 1, "READ_TOO_MANY_TOMBSTONES" }, + { 2, "TIMEOUT" }, + { 3, "INCOMPATIBLE_SCHEMA" }, + { 4, "READ_SIZE" }, + { 5, "NODE_DOWN" }, + { 6, "INDEX_NOT_AVAILABLE" }, + { 7, "READ_TOO_MANY_INDEXES" }, + { 8, "NOT_CMS" }, + { 9, "INVALID_ROUTING" }, + { 10, "COORDINATOR_BEHIND" } + }; + + @Test + public void testEnumCodesAndNames() + { + for (int i = 0; i < REASONS.length; i++) + { + assertEquals("RequestFailureReason code mismatch for " + + REASONS[i].name(), EXPECTED_VALUES[i][0], REASONS[i].code); + assertEquals("RequestFailureReason name mismatch for code " + + REASONS[i].code, EXPECTED_VALUES[i][1], REASONS[i].name()); + } + + assertEquals("Number of RequestFailureReason enum constants has changed. Update the test.", + EXPECTED_VALUES.length, REASONS.length); + } +} From 70ec86c5762aee300739e40e1ea23dc9d8b1b819 Mon Sep 17 00:00:00 2001 From: Bereng <berenguerblasi@gmail.com> Date: Thu, 19 Oct 2023 09:24:23 +0200 Subject: [PATCH 044/225] Repeatable test runs, and named command line options, for test scripts Each test type has a -repeat suffixed companion, used with the `-e REPEATED_TESTS_COUNT` option. patch by Berenguer Blasi; reviewed by Mick Semb Wever, Brandon Williams for CASSANDRA-18942 --- .build/README.md | 62 +++++---- .build/docker/run-tests.sh | 84 +++++++++--- .build/run-python-dtests.sh | 33 ++++- .build/run-tests.sh | 247 ++++++++++++++++++++++++++++-------- .jenkins/Jenkinsfile | 6 +- 5 files changed, 326 insertions(+), 106 deletions(-) diff --git a/.build/README.md b/.build/README.md index 46ab9dddcb75..77c5d8f67dd0 100644 --- a/.build/README.md +++ b/.build/README.md @@ -75,54 +75,60 @@ Running Tests Running unit tests with docker: - .build/docker/run-tests.sh test + .build/docker/run-tests.sh -a test Running unittests without docker: - .build/run-tests.sh test + .build/run-tests.sh -a test Running only a split of unittests, with docker: - .build/docker/run-tests.sh test 1/64 + .build/docker/run-tests.sh -a test -c 1/64 Running unittests with a specific jdk with docker: - .build/docker/run-tests.sh test 1/64 11 + .build/docker/run-tests.sh -a test -c 1/64 -j 11 Running only unit tests matching a regexp, with docker: - .build/docker/run-tests.sh test VerifyTest 11 - .build/docker/run-tests.sh test "Compaction*Test$" 11 + .build/docker/run-tests.sh -a test -t VerifyTest -j 11 + .build/docker/run-tests.sh -a test -t "Compaction*Test$" -j 11 Running other types of tests with docker: - .build/docker/run-tests.sh test - .build/docker/run-tests.sh stress-test - .build/docker/run-tests.sh fqltool-test - .build/docker/run-tests.sh microbench - .build/docker/run-tests.sh test-cdc - .build/docker/run-tests.sh test-compression - .build/docker/run-tests.sh test-oa - .build/docker/run-tests.sh test-system-keyspace-directory - .build/docker/run-tests.sh test-latest - .build/docker/run-tests.sh test-burn - .build/docker/run-tests.sh long-test - .build/docker/run-tests.sh cqlsh-test - .build/docker/run-tests.sh jvm-dtest - .build/docker/run-tests.sh jvm-dtest-upgrade - .build/docker/run-tests.sh dtest - .build/docker/run-tests.sh dtest-novnode - .build/docker/run-tests.sh dtest-latest - .build/docker/run-tests.sh dtest-large - .build/docker/run-tests.sh dtest-large-novnode - .build/docker/run-tests.sh dtest-upgrade - .build/docker/run-tests.sh dtest-upgrade-large - + .build/docker/run-tests.sh -a test + .build/docker/run-tests.sh -a stress-test + .build/docker/run-tests.sh -a fqltool-test + .build/docker/run-tests.sh -a microbench + .build/docker/run-tests.sh -a test-cdc + .build/docker/run-tests.sh -a test-compression + .build/docker/run-tests.sh -a test-oa + .build/docker/run-tests.sh -a test-system-keyspace-directory + .build/docker/run-tests.sh -a test-tries + .build/docker/run-tests.sh -a test-burn + .build/docker/run-tests.sh -a long-test + .build/docker/run-tests.sh -a cqlsh-test + .build/docker/run-tests.sh -a jvm-dtest + .build/docker/run-tests.sh -a jvm-dtest-upgrade + .build/docker/run-tests.sh -a dtest + .build/docker/run-tests.sh -a dtest-novnode + .build/docker/run-tests.sh -a dtest-offheap + .build/docker/run-tests.sh -a dtest-large + .build/docker/run-tests.sh -a dtest-large-novnode + .build/docker/run-tests.sh -a dtest-upgrade + .build/docker/run-tests.sh -a dtest-upgrade-large + +Repeating tests + +Just add the '-repeat' suffix to the test type and pass in the repeat arguments + + .build/run-tests.sh -a jvm-dtest-repeat -e REPEATED_TESTS_COUNT=2 -e REPEATED_TESTS=BooleanTest + .build/docker/run-tests.sh -a jvm-dtest-repeat -e REPEATED_TESTS_COUNT=2 -e REPEATED_TESTS=BooleanTest -j 11 Running python dtests without docker: diff --git a/.build/docker/run-tests.sh b/.build/docker/run-tests.sh index 1063db1d5613..7bb8fc9d1ec5 100755 --- a/.build/docker/run-tests.sh +++ b/.build/docker/run-tests.sh @@ -15,27 +15,70 @@ # See the License for the specific language governing permissions and # limitations under the License. +# # A wrapper script to run-tests.sh (or dtest-python.sh) in docker. -# Can split (or grep) the test list into multiple docker runs, collecting results. +# [ $DEBUG ] && set -x -# help -if [ "$#" -lt 1 ] || [ "$#" -gt 3 ] || [ "$1" == "-h" ]; then - echo "" - echo "Usage: run-tests.sh test_type [split_chunk|test_regexp] [java_version]" - echo "" - echo " default split_chunk is 1/1" - echo " default java_version is what 'java.default' specifies in build.xml" - exit 1 -fi +print_help() { + echo "" + echo "Usage: $0 [-a|-t|-c|-j|-h] [extra arguments]" + echo " -a Test target type: test, test-compression, test-cdc, ..." + echo " -t Test name regexp to run." + echo " -c Chunk to run in the form X/Y: Run chunk X from a total of Y chunks." + echo " -j Java version. Default java_version is what 'java.default' specifies in build.xml." + echo " [extra arguments] will be passed to downstream scripts." + exit 1 +} error() { - echo >&2 $2; - set -x - exit $1 + echo >&2 $2; + set -x + exit $1 } +# legacy argument handling +case ${1} in + "build_dtest_jars" | "stress-test" | "fqltool-test" | "microbench" | "test-burn" | "long-test" | "cqlsh-test" | "simulator-dtest" | "dtest" | "dtest-novnode" | "dtest-latest" | "dtest-large" | "dtest-large-novnode" | "dtest-upgrade" | "dtest-upgrade-novnode"| "dtest-upgrade-large" | "dtest-upgrade-novnode-large" | "test" | "test-cdc" | "test-compression" | "test-oa" | "test-system-keyspace-directory" | "test-latest" | "jvm-dtest" | "jvm-dtest-upgrade" | "jvm-dtest-novnode" | "jvm-dtest-upgrade-novnode") + test_type="-a ${1}" + if [[ -z ${2} ]]; then + test_list="" + elif [[ -n ${2} && "${2}" =~ ^[0-9]+/[0-9]+$ ]]; then + test_list="-c ${2}"; + else + test_list="-t ${2}"; + fi + if [[ -n ${3} ]]; then java_version="-j ${3}"; else java_version=""; fi + echo "Using deprecated legacy arguments. Please update to new parameter format: ${test_type} ${test_list} ${java_version}" + $0 ${test_type} ${test_list} ${java_version} + exit $? +esac + +env_vars="" +while getopts ":a:t:c:e:hj:" opt; do + # shellcheck disable=SC2220 + # Invalid flags check disabled as we'll pass them to other scripts + case $opt in + a ) test_target="$OPTARG" + ;; + t ) test_name_regexp="$OPTARG" + ;; + c ) chunk="$OPTARG" + ;; + j ) java_version="$OPTARG" + ;; + e ) env_vars="${env_vars} -e $OPTARG" + ;; + h ) print_help + exit 0 + ;; + e) ;; # Repeat vars are just passed to downstream run-tests-enhaced.sh + \?) die "Invalid option: -$OPTARG" + ;; + esac +done + # variables, with defaults [ "x${cassandra_dir}" != "x" ] || cassandra_dir="$(readlink -f $(dirname "$0")/../..)" [ "x${cassandra_dtest_dir}" != "x" ] || cassandra_dtest_dir="${cassandra_dir}/../cassandra-dtest" @@ -53,10 +96,11 @@ command -v timeout >/dev/null 2>&1 || { error 1 "timeout needs to be installed"; [ -f "${cassandra_dir}/.build/run-tests.sh" ] || { error 1 "${cassandra_dir}/.build/run-tests.sh must exist"; } # arguments -target=$1 +target=${test_target} split_chunk="1/1" -[ "$#" -gt 1 ] && split_chunk=$2 -java_version=$3 +split_chunk=${chunk-'1/1'} +test_name_regexp=${test_name_regexp} +java_version=${java_version} test_script="run-tests.sh" java_version_default=`grep 'property\s*name="java.default"' ${cassandra_dir}/build.xml |sed -ne 's/.*value="\([^"]*\)".*/\1/p'` @@ -135,7 +179,7 @@ esac # figure out resource limits, scripts, and mounts for the test type docker_flags="-m 5g --memory-swap 5g" -case ${target} in +case ${test_target/-repeat/} in "build_dtest_jars") ;; "stress-test" | "fqltool-test" ) @@ -156,7 +200,7 @@ case ${target} in [[ ${mem} -gt $((15 * 1024 * 1024 * 1024 * ${jenkins_executors})) ]] || { error 1 "${target} require minimum docker memory 16g (per jenkins executor (${jenkins_executors})), found ${mem}"; } docker_flags="-m 15g --memory-swap 15g" ;; - "test"| "test-cdc" | "test-compression" | "test-oa" | "test-system-keyspace-directory" | "test-latest" | "jvm-dtest" | "jvm-dtest-upgrade" | "jvm-dtest-novnode" | "jvm-dtest-upgrade-novnode") + "test" | "test-cdc" | "test-compression" | "test-oa" | "test-system-keyspace-directory" | "test-latest" | "jvm-dtest" | "jvm-dtest-upgrade" | "jvm-dtest-novnode" | "jvm-dtest-upgrade-novnode") [[ ${mem} -gt $((5 * 1024 * 1024 * 1024 * ${jenkins_executors})) ]] || { error 1 "${target} require minimum docker memory 6g (per jenkins executor (${jenkins_executors})), found ${mem}"; } ;; *) @@ -236,8 +280,10 @@ logfile="${build_dir}/test/logs/docker_attach_${container_name}.log" # Docker commands: # set java to java_version # execute the run_script +[ -n "${test_name_regexp}" ] && test_name_regexp_arg="-t ${test_name_regexp}" || split_chunk_arg="-c ${split_chunk}" + docker_command="source \${CASSANDRA_DIR}/.build/docker/_set_java.sh ${java_version} ; \ - \${CASSANDRA_DIR}/.build/docker/_docker_init_tests.sh ${target} ${split_chunk} ; exit \$?" + \${CASSANDRA_DIR}/.build/docker/_docker_init_tests.sh -a ${target} ${split_chunk_arg} ${test_name_regexp_arg} ${env_vars} ; exit \$?" # start the container, timeout after 4 hours docker_id=$(docker run --name ${container_name} ${docker_flags} ${docker_envs} ${docker_mounts} ${docker_volume_opt} ${image_name} sleep 4h) diff --git a/.build/run-python-dtests.sh b/.build/run-python-dtests.sh index 5b1307e1546d..86652db0b83d 100755 --- a/.build/run-python-dtests.sh +++ b/.build/run-python-dtests.sh @@ -28,18 +28,41 @@ [ $DEBUG ] && set -x # help -if [ "$#" -lt 1 ] || [ "$#" -gt 2 ] || [ "$1" == "-h" ]; then +if [ "$#" -lt 1 ] || [ "$1" == "-h" ]; then echo "" - echo "Usage: run-python-dtest.sh test_type [split_chunk|test_regexp]" + echo "Usage: $0 [-a|-t|-c|-j|-h]" + echo " -a Test target type: dtest, dtest-latest, ..." + echo " -t Test name regexp to run." + echo " -c Chunk to run in the form X/Y: Run chunk X from a total of Y chunks." echo "" echo " default split_chunk is 1/1" exit 1 fi # Pass in target to run, defaults to dtest -DTEST_TARGET="${1:-dtest}" -# Optional: pass in chunk to test, formatted as "K/N" for the Kth chunk of N chunks -DTEST_SPLIT_CHUNK="$2" +DTEST_TARGET="dtest" + +# TODO implement repeated runs, eg CASSANDRA-18942 +while getopts "a:t:c:hj:" opt; do + case $opt in + a ) DTEST_TARGET="$OPTARG" + ;; + t ) DTEST_SPLIT_CHUNK="$OPTARG" + ;; + c ) DTEST_SPLIT_CHUNK="$OPTARG" + ;; + h ) print_help + exit 0 + ;; + j ) ;; # To avoid failing on java_version param from docker/run_tests.sh + \?) error 1 "Invalid option: -$OPTARG" + ;; + esac +done +shift $((OPTIND-1)) +if [ "$#" -ne 0 ]; then + error 1 "Unexpected arguments" +fi # variables, with defaults [ "x${CASSANDRA_DIR}" != "x" ] || CASSANDRA_DIR="$(readlink -f $(dirname "$0")/..)" diff --git a/.build/run-tests.sh b/.build/run-tests.sh index 7529af933a56..9ace42cbb571 100755 --- a/.build/run-tests.sh +++ b/.build/run-tests.sh @@ -24,25 +24,115 @@ set -o errexit set -o pipefail +[ $DEBUG ] && set -x + # variables, with defaults [ "x${CASSANDRA_DIR}" != "x" ] || CASSANDRA_DIR="$(readlink -f $(dirname "$0")/..)" [ "x${DIST_DIR}" != "x" ] || DIST_DIR="${CASSANDRA_DIR}/build" # pre-conditions -command -v ant >/dev/null 2>&1 || { echo >&2 "ant needs to be installed"; exit 1; } -command -v git >/dev/null 2>&1 || { echo >&2 "git needs to be installed"; exit 1; } -[ -d "${CASSANDRA_DIR}" ] || { echo >&2 "Directory ${CASSANDRA_DIR} must exist"; exit 1; } -[ -f "${CASSANDRA_DIR}/build.xml" ] || { echo >&2 "${CASSANDRA_DIR}/build.xml must exist"; exit 1; } +command -v ant >/dev/null 2>&1 || { error 1 "ant needs to be installed"; } +command -v git >/dev/null 2>&1 || { error 1 "git needs to be installed"; } +command -v uuidgen >/dev/null 2>&1 || test -f /proc/sys/kernel/random/uuid || { error 1 "uuidgen needs to be installed"; } +[ -d "${CASSANDRA_DIR}" ] || { error 1 "Directory ${CASSANDRA_DIR} must exist"; } +[ -f "${CASSANDRA_DIR}/build.xml" ] || { error 1 "${CASSANDRA_DIR}/build.xml must exist"; } [ -d "${DIST_DIR}" ] || { mkdir -p "${DIST_DIR}" ; } -# help -if [ "$#" -lt 1 ] || [ "$#" -gt 2 ] || [ "$1" == "-h" ]; then - echo "" - echo "Usage: run-tests.sh test_type [split_chunk|test_regexp]" - echo "" - echo " default split_chunk is 1/1" - exit 1 +error() { + echo >&2 $2; + set -x + exit $1 +} + +print_help() { + echo "Usage: $0 [-a|-t|-c|-e|-i|-b|-s|-h]" + echo " -a Test target type: test, test-compression, test-cdc, ..." + echo " -t Test name regexp to run." + echo " -c Chunk to run in the form X/Y: Run chunk X from a total of Y chunks." + echo " -b Specify the base git branch for comparison when determining changed tests to" + echo " repeat. Defaults to ${BASE_BRANCH}. Note that this option is not used when" + echo " the '-a' option is specified." + echo " -s Skip automatic detection of changed tests. Useful when you need to repeat a few ones," + echo " or when there are too many changed tests the CI env to handle." + echo " -e <key=value> Environment variables to be used in the repeated runs:" + echo " -e REPEATED_TESTS_STOP_ON_FAILURE=false" + echo " -e REPEATED_TESTS=org.apache.cassandra.cql3.ViewTest,ForceCompactionTest" + echo " -e REPEATED_TESTS_COUNT=500" + echo " If you want to specify multiple environment variables simply add multiple -e options." + echo " -i Ignore unknown environment variables" + echo " -h Print help" +} + + +# legacy argument handling +case ${1} in + "build_dtest_jars" | "stress-test" | "fqltool-test" | "microbench" | "test-burn" | "long-test" | "cqlsh-test" | "simulator-dtest" | "test" | "test-cdc" | "test-compression" | "test-oa" | "test-system-keyspace-directory" | "test-latest" | "jvm-dtest" | "jvm-dtest-upgrade" | "jvm-dtest-novnode" | "jvm-dtest-upgrade-novnode") + test_type="-a ${1}" + if [[ -z ${2} ]]; then + test_list="" + elif [[ -n ${2} && "${2}" =~ ^[0-9]+/[0-9]+$ ]]; then + test_list="-c ${2}"; + else + test_list="-t ${2}"; + fi + echo "Using deprecated legacy arguments. Please update to new parameter format: ${test_type} ${test_list}" + $0 ${test_type} ${test_list} + exit $? +esac + + +env_vars="" +has_env_vars=false +check_env_vars=true +detect_changed_tests=true +while getopts "a:t:c:e:ib:shj:" opt; do + case $opt in + a ) test_target="$OPTARG" + ;; + t ) test_name_regexp="$OPTARG" + ;; + c ) chunk="$OPTARG" + ;; + e ) if (! ($has_env_vars)); then + env_vars="$OPTARG" + else + env_vars="$env_vars|$OPTARG" + fi + has_env_vars=true + ;; + b ) BASE_BRANCH="$OPTARG" + ;; + i ) check_env_vars=false + ;; + s ) detect_changed_tests=false + ;; + h ) print_help + exit 0 + ;; + j ) ;; # To avoid failing on java_version param from docker/run_tests.sh + \?) error 1 "Invalid option: -$OPTARG" + ;; + esac +done +shift $((OPTIND-1)) +if [ "$#" -ne 0 ]; then + error 1 "Unexpected arguments" +fi + +# validate environment variables +if $has_env_vars && $check_env_vars; then + for entry in $(echo $env_vars | tr "|" "\n"); do + key=$(echo $entry | tr "=" "\n" | sed -n 1p) + case $key in + "REPEATED_TESTS_STOP_ON_FAILURE" | "REPEATED_TESTS" | "REPEATED_TESTS_COUNT" ) + [[ ${test_target} == *"-repeat" ]] || { error 1 "'-e REPEATED_*' variables only valid against *-repeat target types"; } + ;; + *) + error 1 "unrecognized environment variable name: $key" + ;; + esac + done fi # print debug information on versions @@ -65,7 +155,7 @@ _split_tests() { split_cmd=split if [[ "${_split_chunk}" =~ ^[0-9]+/[0-9]+$ ]]; then ( split --help 2>&1 ) | grep -q "r/K/N" || split_cmd=gsplit - command -v ${split_cmd} >/dev/null 2>&1 || { echo >&2 "${split_cmd} needs to be installed"; exit 1; } + command -v ${split_cmd} >/dev/null 2>&1 || { error 1 "${split_cmd} needs to be installed"; } ${split_cmd} -n r/${_split_chunk} elif [[ "x" != "x${_split_chunk}" ]] ; then grep -e "${_split_chunk}" @@ -78,6 +168,11 @@ _timeout_for() { grep "name=\"${1}\"" build.xml | awk -F'"' '{print $4}' } +_get_env_var() { + [[ ${env_vars} =~ ${1}=([^|]+) ]] + echo "${BASH_REMATCH[1]}" +} + _build_all_dtest_jars() { # build the dtest-jar for the branch under test. remember to `ant clean` if you want a new dtest jar built dtest_jar_version=$(grep 'property\s*name=\"base.version\"' build.xml |sed -ne 's/.*value=\"\([^"]*\)\".*/\1/p') @@ -126,41 +221,89 @@ _build_all_dtest_jars() { _run_testlist() { local _target_prefix=$1 local _testlist_target=$2 - local _split_chunk=$3 - local _test_timeout=$4 - testlist="$( _list_tests "${_target_prefix}" | _split_tests "${_split_chunk}")" - if [[ "${_split_chunk}" =~ ^[0-9]+/[0-9]+$ ]]; then + local _test_name_regexp=$3 + local _split_chunk=$4 + local _test_timeout=$5 + local _test_iterations=${6:-1} + + # are we running ${_test_name_regexp} or ${_split_chunk} + if [ -n "${_test_name_regexp}" ]; then + echo "Running tests: ${_test_name_regexp} (${_test_iterations} times)" + # test regexp can come in csv + for i in ${_test_name_regexp//,/ }; do + [ -n "${testlist}" ] && testlist="${testlist}"$'\n' + testlist="${testlist}$( _list_tests "${_target_prefix}" | _split_tests "${i}")" + done + [[ -z "${testlist}" ]] && error 1 "No tests found in test name regexp: ${_test_name_regexp}" + else + [ -n "${_split_chunk}" ] || { error 1 "Neither name regexp or split chunk defined"; } + echo "Running split: ${_split_chunk}" + testlist="$( _list_tests "${_target_prefix}" | _split_tests "${_split_chunk}")" if [[ -z "${testlist}" ]]; then # something has to run in the split to generate a junit xml result echo "Hacking ${_target_prefix} ${_testlist_target} to run only first test found as no tests in split ${_split_chunk} were found" - testlist="$( _list_tests "${_target_prefix}" | head -n1)" - fi - else - if [[ -z "${testlist}" ]]; then - echo "No tests match ${_split_chunk}" - exit 1 + testlist="$( _list_tests "${_target_prefix}" | sed -n 1p)" fi fi - ant $_testlist_target -Dtest.classlistprefix="${_target_prefix}" -Dtest.classlistfile=<(echo "${testlist}") -Dtest.timeout="${_test_timeout}" ${ANT_TEST_OPTS} || echo "failed ${_target_prefix} ${_testlist_target} ${split_chunk}" + + local -r _results_uuid="$(command -v uuidgen >/dev/null 2>&1 && uuidgen || cat /proc/sys/kernel/random/uuid)" + local failures=0 + for ((i=0; i < _test_iterations; i++)); do + [ "${_test_iterations}" -eq 1 ] || printf "–––– run ${i}\n" + set +o errexit + ant "$_testlist_target" -Dtest.classlistprefix="${_target_prefix}" -Dtest.classlistfile=<(echo "${testlist}") -Dtest.timeout="${_test_timeout}" ${ANT_TEST_OPTS} + ant_status=$? + set -o errexit + if [[ $ant_status -ne 0 ]]; then + echo "failed ${_target_prefix} ${_testlist_target} ${split_chunk} ${_test_name_regexp}" + + # Only store logs for failed tests on repeats to save up space + if [ "${_test_iterations}" -gt 1 ]; then + # Get this test results and rename file with iteration and 'fail' + find "${DIST_DIR}"/test/output/ -type f -name "*.xml" -not -name "*fail.xml" -print0 | while read -r -d $'\0' file; do + mv "${file}" "${file%.xml}-${_results_uuid}-${i}-fail.xml" + done + find "${DIST_DIR}"/test/logs/ -type f -name "*.log" -not -name "*fail.log" -print0 | while read -r -d $'\0' file; do + mv "${file}" "${file%.log}-${_results_uuid}-${i}-fail.log" + done + + if [ "$(_get_env_var 'REPEATED_TESTS_STOP_ON_FAILURE')" == true ]; then + error 0 "fail fast, after ${i} successful runs" + fi + let failures+=1 + fi + fi + done + [ "${_test_iterations}" -eq 1 ] || printf "––––\nfailure rate: ${failures}/${_test_iterations}\n" } _main() { # parameters - local -r target="${1:-}" + local -r target="${test_target/-repeat/}" + local -r split_chunk="${chunk:-'1/1'}" # Chunks formatted as "K/N" for the Kth chunk of N chunks - local -r split_chunk="${2:-'1/1'}" # Optional: pass in chunk or regexp to test. Chunks formatted as "K/N" for the Kth chunk of N chunks # check split_chunk is compatible with target (if not a regexp) if [[ "${_split_chunk}" =~ ^\d+/\d+$ ]] && [[ "1/1" != "${split_chunk}" ]] ; then case ${target} in "stress-test" | "fqltool-test" | "microbench" | "cqlsh-test" | "simulator-dtest") - echo "Target ${target} does not suport splits." - exit 1 + error 1 "Target ${target} does not suport splits." ;; *) ;; esac fi + # "-repeat" is a reserved suffix on target types + if [[ ${test_target} == *"-repeat" ]] ; then + [[ "${split_chunk}" =~ ^[0-9]+/[0-9]+$ ]] && { error 1 "Repeated tests not valid with splits"; } + if [[ -z "${test_name_regexp}" ]] ; then + test_name_regexp="$(_get_env_var 'REPEATED_TESTS')" + fi + local -r repeat_count="$(_get_env_var 'REPEATED_TESTS_COUNT')" + else + test_name_regexp="${test_name_regexp:-}" + fi + pushd ${CASSANDRA_DIR}/ >/dev/null # jdk check @@ -169,15 +312,14 @@ _main() { local -r java_version_default=`grep 'property\s*name="java.default"' build.xml |sed -ne 's/.*value="\([^"]*\)".*/\1/p'` if [ "${java_version}" -eq 17 ] && [[ "${target}" == "jvm-dtest-upgrade" ]] ; then - echo "Invalid JDK${java_version}. Only overlapping supported JDKs can be used when upgrading, as the same jdk must be used over the upgrade path." - exit 1 + error 1 "Invalid JDK${java_version}. Only overlapping supported JDKs can be used when upgrading, as the same jdk must be used over the upgrade path." fi # check project is already built. no cleaning is done, so jenkins unstash works, beware. - [[ -f "${DIST_DIR}/apache-cassandra-${version}.jar" ]] || [[ -f "${DIST_DIR}/apache-cassandra-${version}-SNAPSHOT.jar" ]] || { echo "Project must be built first. Use \`ant jar\`. Build directory is ${DIST_DIR} with: $(ls ${DIST_DIR} | xargs)"; exit 1; } + [[ -f "${DIST_DIR}/apache-cassandra-${version}.jar" ]] || [[ -f "${DIST_DIR}/apache-cassandra-${version}-SNAPSHOT.jar" ]] || { error 1 "Project must be built first. Use \`ant jar\`. Build directory is ${DIST_DIR} with: $(ls ${DIST_DIR} | xargs)"; } # check if dist artifacts exist, this breaks the dtests - [[ -d "${DIST_DIR}/dist" ]] && { echo "tests don't work when build/dist ("${DIST_DIR}/dist") exists (from \`ant artifacts\`)"; exit 1; } + [[ -d "${DIST_DIR}/dist" ]] && { error 1 "tests don't work when build/dist ("${DIST_DIR}/dist") exists (from \`ant artifacts\`)"; } # ant test setup export TMP_DIR="${DIST_DIR}/tmp" @@ -206,42 +348,44 @@ _main() { ant $target ${ANT_TEST_OPTS} -Dmaven.test.failure.ignore=true ;; "test") - _run_testlist "unit" "testclasslist" "${split_chunk}" "$(_timeout_for 'test.timeout')" + _run_testlist "unit" "testclasslist" "${test_name_regexp}" "${split_chunk}" "$(_timeout_for 'test.timeout')" "${repeat_count}" ;; "test-cdc") - _run_testlist "unit" "testclasslist-cdc" "${split_chunk}" "$(_timeout_for 'test.timeout')" + _run_testlist "unit" "testclasslist-cdc" "${test_name_regexp}" "${split_chunk}" "$(_timeout_for 'test.timeout')" "${repeat_count}" ;; "test-compression") - _run_testlist "unit" "testclasslist-compression" "${split_chunk}" "$(_timeout_for 'test.timeout')" + _run_testlist "unit" "testclasslist-compression" "${test_name_regexp}" "${split_chunk}" "$(_timeout_for 'test.timeout')" "${repeat_count}" ;; "test-oa") - _run_testlist "unit" "testclasslist-oa" "${split_chunk}" "$(_timeout_for 'test.timeout')" + _run_testlist "unit" "testclasslist-oa" "${test_name_regexp}" "${split_chunk}" "$(_timeout_for 'test.timeout')" "${repeat_count}" ;; "test-system-keyspace-directory") - _run_testlist "unit" "testclasslist-system-keyspace-directory" "${split_chunk}" "$(_timeout_for 'test.timeout')" + _run_testlist "unit" "testclasslist-system-keyspace-directory" "${test_name_regexp}" "${split_chunk}" "$(_timeout_for 'test.timeout')" "${repeat_count}" ;; "test-latest") - _run_testlist "unit" "testclasslist-latest" "${split_chunk}" "$(_timeout_for 'test.timeout')" + _run_testlist "unit" "testclasslist-latest" "${test_name_regexp}" "${split_chunk}" "$(_timeout_for 'test.timeout')" "${repeat_count}" ;; "test-burn") - _run_testlist "burn" "testclasslist" "${split_chunk}" "$(_timeout_for 'test.burn.timeout')" + _run_testlist "burn" "testclasslist" "${test_name_regexp}" "${split_chunk}" "$(_timeout_for 'test.burn.timeout')" "${repeat_count}" ;; "long-test") - _run_testlist "long" "testclasslist" "${split_chunk}" "$(_timeout_for 'test.long.timeout')" + _run_testlist "long" "testclasslist" "${test_name_regexp}" "${split_chunk}" "$(_timeout_for 'test.long.timeout')" "${repeat_count}" ;; "simulator-dtest") ant test-simulator-dtest ${ANT_TEST_OPTS} || echo "failed ${target}" ;; "jvm-dtest" | "jvm-dtest-novnode") [ "jvm-dtest-novnode" == "${target}" ] || ANT_TEST_OPTS="${ANT_TEST_OPTS} -Dcassandra.dtest.num_tokens=16" - testlist=$( _list_tests "distributed" | grep -v "upgrade" | _split_tests "${split_chunk}") - if [[ -z "$testlist" ]]; then - [[ "${split_chunk}" =~ ^[0-9]+/[0-9]+$ ]] || { echo "No tests match ${split_chunk}"; exit 1; } + if [[ -z "${test_name_regexp}" ]] ; then + test_name_regexp=$( _list_tests "distributed" | grep -v "upgrade" | _split_tests "${split_chunk}") + if [[ -z "${test_name_regexp}" ]]; then + [[ "${split_chunk}" =~ ^[0-9]+/[0-9]+$ ]] || { error 1 "No tests match ${test_name_regexp}"; } # something has to run in the split to generate a junit xml result echo "Hacking jvm-dtest to run only first test found as no tests in split ${split_chunk} were found" - testlist="$( _list_tests "distributed" | grep -v "upgrade" | head -n1)" + test_name_regexp="$( _list_tests "distributed" | grep -v "upgrade" | sed -n 1p)" + fi fi - ant testclasslist -Dtest.classlistprefix=distributed -Dtest.timeout=$(_timeout_for "test.distributed.timeout") -Dtest.classlistfile=<(echo "${testlist}") ${ANT_TEST_OPTS} || echo "failed ${target} ${split_chunk}" + _run_testlist "distributed" "testclasslist" "${test_name_regexp}" "" "$(_timeout_for 'test.distributed.timeout')" "${repeat_count}" ;; "build_dtest_jars") _build_all_dtest_jars @@ -249,21 +393,22 @@ _main() { "jvm-dtest-upgrade" | "jvm-dtest-upgrade-novnode") _build_all_dtest_jars [ "jvm-dtest-upgrade-novnode" == "${target}" ] || ANT_TEST_OPTS="${ANT_TEST_OPTS} -Dcassandra.dtest.num_tokens=16" - testlist=$( _list_tests "distributed" | grep "upgrade" | _split_tests "${split_chunk}") - if [[ -z "${testlist}" ]]; then - [[ "${split_chunk}" =~ ^[0-9]+/[0-9]+$ ]] || { echo "No tests match ${split_chunk}"; exit 1; } + if [[ -z "${test_name_regexp}" ]] ; then + test_name_regexp=$( _list_tests "distributed" | grep "upgrade" | _split_tests "${split_chunk}") + if [[ -z "${test_name_regexp}" ]]; then + [[ "${split_chunk}" =~ ^[0-9]+/[0-9]+$ ]] || { error 1 "No tests match ${test_name_regexp}"; } # something has to run in the split to generate a junit xml result echo "Hacking jvm-dtest-upgrade to run only first test found as no tests in split ${split_chunk} were found" - testlist="$( _list_tests "distributed" | grep "upgrade" | head -n1)" + test_name_regexp="$( _list_tests "distributed" | grep "upgrade" | sed -n 1p)" + fi fi - ant testclasslist -Dtest.classlistprefix=distributed -Dtest.timeout=$(_timeout_for "test.distributed.timeout") -Dtest.classlistfile=<(echo "${testlist}") ${ANT_TEST_OPTS} || echo "failed ${target} ${split_chunk}" + _run_testlist "distributed" "testclasslist" "${test_name_regexp}" "" "$(_timeout_for 'test.distributed.timeout')" "${repeat_count}" ;; "cqlsh-test") ./pylib/cassandra-cqlsh-tests.sh $(pwd) ;; *) - echo "unrecognized test type \"${target}\"" - exit 1 + error 1 "unrecognized test type \"${target}\"" ;; esac diff --git a/.jenkins/Jenkinsfile b/.jenkins/Jenkinsfile index c1c7e431d294..4f8ad6c26899 100644 --- a/.jenkins/Jenkinsfile +++ b/.jenkins/Jenkinsfile @@ -184,7 +184,6 @@ def tasks() { ] testSteps.each() { it.value.put('type', 'test') - it.value.put('script', '.build/docker/run-tests.sh') if (!it.value['size']) { it.value.put('size', 'medium') } @@ -360,6 +359,7 @@ def build(command, cell) { } def test(command, cell) { + if (command.containsKey('script')) { error("test commands all use `.build/docker/run-tests.sh`") } def splits = command.splits ? command.splits : 1 def maxAttempts = 2 def attempt = 0 @@ -384,7 +384,7 @@ def test(command, cell) { script_vars = fetchDTestsSource(command, script_vars) timeout(time: 1, unit: 'HOURS') { // best throughput with each cell at ~10 minutes buildJVMDTestJars(cell, script_vars, logfile) - def status = sh label: "RUNNING TESTS ${cell.step}...", script: "${script_vars} .build/docker/run-tests.sh ${cell.step} '${cell.split}/${splits}' ${cell.jdk} 2>&1 | tee >( xz -c > build/${logfile} )", returnStatus: true + def status = sh label: "RUNNING TESTS ${cell.step}...", script: "${script_vars} .build/docker/run-tests.sh -a ${cell.step} -c '${cell.split}/${splits}' -j ${cell.jdk} 2>&1 | tee >( xz -c > build/${logfile} )", returnStatus: true dir("build") { archiveArtifacts artifacts: "${logfile}", fingerprint: true copyToNightlies("${logfile}", "${cell.step}/${cell.arch}/jdk${cell.jdk}/python${cell.python}/cython_${cell.cython}/" + "split_${cell.split}_${splits}".replace("/", "_")) @@ -434,7 +434,7 @@ def buildJVMDTestJars(cell, script_vars, logfile) { try { unstash name: "jvm_dtests_${cell.arch}_${cell.jdk}" } catch (error) { - sh label: "RUNNING build_dtest_jars...", script: "${script_vars} .build/docker/run-tests.sh build_dtest_jars ${cell.jdk} 2>&1 | tee >( xz -c > build/${logfile} )" + sh label: "RUNNING build_dtest_jars...", script: "${script_vars} .build/docker/run-tests.sh -a build_dtest_jars -j ${cell.jdk} 2>&1 | tee >( xz -c > build/${logfile} )" stash name: "jvm_dtests_${cell.arch}_${cell.jdk}", includes: '**/dtest*.jar' } } From 2651623af6bb3da5f820d9e09abfbdd0683a1322 Mon Sep 17 00:00:00 2001 From: Caleb Rackliffe <calebrackliffe@gmail.com> Date: Wed, 20 Nov 2024 16:13:01 -0600 Subject: [PATCH 045/225] Interpret inet, bigint, varint, and decimal as non-reversed types for query construction and post-filtering patch by Caleb Rackliffe; reviewed by David Capwell for CASSANDRA-20100 --- CHANGES.txt | 1 + .../index/sai/utils/IndexTermType.java | 49 ++++-- .../sai/cql/DescClusteringRangeQueryTest.java | 145 ++++++++++++++++++ 3 files changed, 183 insertions(+), 12 deletions(-) create mode 100644 test/unit/org/apache/cassandra/index/sai/cql/DescClusteringRangeQueryTest.java diff --git a/CHANGES.txt b/CHANGES.txt index a8072f3e382a..6fbc22f8fbd8 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.3 + * Interpret inet, bigint, varint, and decimal as non-reversed types for query construction and post-filtering (CASSANDRA-20100) * Fix delayed gossip shutdown messages clobbering startup states that leave restarted nodes appearing down (CASSANDRA-20033) * Streamline the serialized format for index status gossip messages (CASSANDRA-20058) * Batch clusterings into single SAI partition post-filtering reads (CASSANDRA-19497) diff --git a/src/java/org/apache/cassandra/index/sai/utils/IndexTermType.java b/src/java/org/apache/cassandra/index/sai/utils/IndexTermType.java index a558d5ee82aa..51600f9d791b 100644 --- a/src/java/org/apache/cassandra/index/sai/utils/IndexTermType.java +++ b/src/java/org/apache/cassandra/index/sai/utils/IndexTermType.java @@ -141,20 +141,24 @@ private IndexTermType(ColumnMetadata columnMetadata, List<ColumnMetadata> partit this.indexTargetType = indexTargetType; this.capabilities = calculateCapabilities(columnMetadata, partitionColumns, indexTargetType); this.indexType = calculateIndexType(columnMetadata.type, capabilities, indexTargetType); - if (indexType.subTypes().isEmpty()) + + AbstractType<?> baseType = indexType.unwrap(); + + if (baseType.subTypes().isEmpty()) { this.subTypes = Collections.emptyList(); } else { - List<IndexTermType> subTypes = new ArrayList<>(indexType.subTypes().size()); - for (AbstractType<?> subType : indexType.subTypes()) + List<IndexTermType> subTypes = new ArrayList<>(baseType.subTypes().size()); + for (AbstractType<?> subType : baseType.subTypes()) subTypes.add(new IndexTermType(columnMetadata.withNewType(subType), partitionColumns, indexTargetType)); this.subTypes = Collections.unmodifiableList(subTypes); } + if (isVector()) { - VectorType<?> vectorType = (VectorType<?>) indexType; + VectorType<?> vectorType = (VectorType<?>) baseType; vectorElementType = vectorType.elementType; vectorDimension = vectorType.dimension; } @@ -454,12 +458,14 @@ public int compare(ByteBuffer b1, ByteBuffer b2) { if (isInetAddress()) return compareInet(b1, b2); - // BigInteger values, frozen types and composite types (map entries) use compareUnsigned to maintain - // a consistent order between the in-memory index and the on-disk index. + else if (isLong()) + return indexType.unwrap().compare(b1, b2); + // BigInteger values, frozen types and composite types (map entries) use compareUnsigned to maintain + // a consistent order between the in-memory index and the on-disk index. else if (isBigInteger() || isBigDecimal() || isComposite() || isFrozen()) return FastByteOperations.compareUnsigned(b1, b2); - return indexType.compare(b1, b2 ); + return indexType.compare(b1, b2); } /** @@ -491,10 +497,17 @@ public int comparePostFilter(Expression.Value requestedValue, Expression.Value c { if (isInetAddress()) return compareInet(requestedValue.encoded, columnValue.encoded); - // Override comparisons for frozen collections and composite types (map entries) + // bigint, decimal, and varint are not indexed in reversed byte-comparable form or treated as reversed types by + // Expression, so it is correct to compare with the base/unwrapped type + else if (isLong() || isBigDecimal() || isBigInteger()) + return indexType.unwrap().compare(requestedValue.raw, columnValue.raw); + // Override comparisons for frozen collections and composite types (map entries) else if (isComposite() || isFrozen()) return FastByteOperations.compareUnsigned(requestedValue.raw, columnValue.raw); + // Reversed types are treated as such by Expression here, so we cannot blindly compare with the unwrapped type. + // In the future, we might consider simplifying things to have SAI ignore reversed types altogether, but this + // will require a change to the on-disk formats. return indexType.compare(requestedValue.raw, columnValue.raw); } @@ -628,10 +641,7 @@ private EnumSet<Capability> calculateCapabilities(ColumnMetadata columnMetadata, capabilities.add(Capability.COMPOSITE_PARTITION); AbstractType<?> type = columnMetadata.type; - - if (type.isReversed()) - capabilities.add(Capability.REVERSED); - + boolean reversed = type.isReversed(); AbstractType<?> baseType = type.unwrap(); if (baseType.isCollection()) @@ -666,16 +676,31 @@ else if (!indexType.subTypes().isEmpty() && !indexType.isMultiCell()) capabilities.add(Capability.VECTOR); if (indexType instanceof InetAddressType) + { capabilities.add(Capability.INET_ADDRESS); + reversed = false; + } if (indexType instanceof IntegerType) + { capabilities.add(Capability.BIG_INTEGER); + reversed = false; + } if (indexType instanceof DecimalType) + { capabilities.add(Capability.BIG_DECIMAL); + reversed = false; + } if (indexType instanceof LongType) + { capabilities.add(Capability.LONG); + reversed = false; + } + + if (reversed) + capabilities.add(Capability.REVERSED); return capabilities; } diff --git a/test/unit/org/apache/cassandra/index/sai/cql/DescClusteringRangeQueryTest.java b/test/unit/org/apache/cassandra/index/sai/cql/DescClusteringRangeQueryTest.java new file mode 100644 index 000000000000..7a93947cfd3d --- /dev/null +++ b/test/unit/org/apache/cassandra/index/sai/cql/DescClusteringRangeQueryTest.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.index.sai.cql; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.net.InetAddress; + +import org.junit.Test; + +import com.datastax.driver.core.ResultSet; +import org.apache.cassandra.index.sai.SAITester; + +public class DescClusteringRangeQueryTest extends SAITester +{ + @Test + public void testReversedIntBetween() throws Throwable + { + createTable("CREATE TABLE %s(p int, c int, abbreviation ascii, PRIMARY KEY (p, c)) WITH CLUSTERING ORDER BY (c DESC)"); + createIndex("CREATE INDEX clustering_test_index ON %s(c) USING 'sai'"); + createIndex("CREATE INDEX abbreviation_test_index ON %s(abbreviation) USING 'sai'"); + + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, 1, 'CA')"); + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, 2, 'MA')"); + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, 3, 'MA')"); + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, 4, 'TX')"); + + beforeAndAfterFlush(() -> + { + ResultSet rangeRowsNet = executeNet("SELECT * FROM %s WHERE c >= 2 AND c <= 3 AND abbreviation = 'MA'"); + assertRowsNet(rangeRowsNet, row (0, 3, "MA"), row (0, 2, "MA")); + }); + } + + @Test + public void testReversedLongBetween() throws Throwable + { + createTable("CREATE TABLE %s(p int, c bigint, abbreviation ascii, PRIMARY KEY (p, c)) WITH CLUSTERING ORDER BY (c DESC)"); + createIndex("CREATE INDEX clustering_test_index ON %s(c) USING 'sai'"); + createIndex("CREATE INDEX abbreviation_test_index ON %s(abbreviation) USING 'sai'"); + + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, 1, 'CA')"); + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, 2, 'MA')"); + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, 3, 'MA')"); + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, 4, 'TX')"); + + beforeAndAfterFlush(() -> + { + ResultSet rangeRowsNet = executeNet("SELECT * FROM %s WHERE c >= 2 AND c <= 3 AND abbreviation = 'MA'"); + assertRowsNet(rangeRowsNet, row (0, 3L, "MA"), row (0, 2L, "MA")); + }); + } + + @Test + public void testReversedBigIntegerBetween() throws Throwable + { + createTable("CREATE TABLE %s(p int, c varint, abbreviation ascii, PRIMARY KEY (p, c)) WITH CLUSTERING ORDER BY (c DESC)"); + createIndex("CREATE INDEX clustering_test_index ON %s(c) USING 'sai'"); + createIndex("CREATE INDEX abbreviation_test_index ON %s(abbreviation) USING 'sai'"); + + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, 1, 'CA')"); + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, 2, 'MA')"); + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, 3, 'MA')"); + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, 4, 'TX')"); + + beforeAndAfterFlush(() -> + { + ResultSet rangeRowsNet = executeNet("SELECT * FROM %s WHERE c >= 2 AND c <= 3 AND abbreviation = 'MA'"); + assertRowsNet(rangeRowsNet, row (0, new BigInteger("3"), "MA"), row (0, new BigInteger("2"), "MA")); + }); + } + + @Test + public void testReversedBigDecimalBetween() throws Throwable + { + createTable("CREATE TABLE %s(p int, c decimal, abbreviation ascii, PRIMARY KEY (p, c)) WITH CLUSTERING ORDER BY (c DESC)"); + createIndex("CREATE INDEX clustering_test_index ON %s(c) USING 'sai'"); + createIndex("CREATE INDEX abbreviation_test_index ON %s(abbreviation) USING 'sai'"); + + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, 1.1, 'CA')"); + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, 2.1, 'MA')"); + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, 2.9, 'MA')"); + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, 4.0, 'TX')"); + + beforeAndAfterFlush(() -> + { + ResultSet rangeRowsNet = executeNet("SELECT * FROM %s WHERE c > 1.9 AND c < 3.0 AND abbreviation = 'MA'"); + assertRowsNet(rangeRowsNet, row (0, new BigDecimal("2.9"), "MA"), row (0, new BigDecimal("2.1"), "MA")); + }); + } + + @Test + public void testReversedInetBetween() throws Throwable + { + createTable("CREATE TABLE %s(p int, c inet, abbreviation ascii, PRIMARY KEY (p, c)) WITH CLUSTERING ORDER BY (c DESC)"); + createIndex("CREATE INDEX clustering_test_index ON %s(c) USING 'sai'"); + createIndex("CREATE INDEX abbreviation_test_index ON %s(abbreviation) USING 'sai'"); + + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, '127.0.0.1', 'CA')"); + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, '127.0.0.2', 'MA')"); + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, '127.0.0.3', 'MA')"); + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, '127.0.0.4', 'TX')"); + + beforeAndAfterFlush(() -> + { + ResultSet rangeRowsNet = executeNet("SELECT * FROM %s WHERE c >= '127.0.0.2' AND c <= '127.0.0.3' AND abbreviation = 'MA'"); + assertRowsNet(rangeRowsNet, row (0, InetAddress.getByName("127.0.0.3"), "MA"), row (0, InetAddress.getByName("127.0.0.2"), "MA")); + }); + } + + @Test + public void testReversedIntBetweenWithAnalyzer() throws Throwable + { + createTable("CREATE TABLE %s(p int, c int, abbreviation ascii, PRIMARY KEY (p, c)) WITH CLUSTERING ORDER BY (c DESC)"); + createIndex("CREATE INDEX clustering_test_index ON %s(c) USING 'sai'"); + createIndex("CREATE INDEX abbreviation_test_index ON %s(abbreviation) USING 'sai' WITH OPTIONS = {'case_sensitive': 'false'}"); + + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, 1, 'CA')"); + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, 2, 'MA')"); + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, 3, 'MA')"); + execute("INSERT INTO %s(p, c, abbreviation) VALUES (0, 4, 'TX')"); + + beforeAndAfterFlush(() -> + { + ResultSet rangeRowsNet = executeNet("SELECT * FROM %s WHERE c >= 2 AND c <= 3 AND abbreviation = 'MA'"); + assertRowsNet(rangeRowsNet, row (0, 3, "MA"), row (0, 2, "MA")); + }); + } +} From 4f49ca5e29d9c7207654a1f3c4eac9c9f0b84e5e Mon Sep 17 00:00:00 2001 From: David Capwell <dcapwell@apache.org> Date: Fri, 8 Nov 2024 13:43:50 -0800 Subject: [PATCH 046/225] TCM's Retry.Deadline#retryIndefinitely is dangerous if used with RemoteProcessor as the deadline does not impact message retries patch by David Capwell; reviewed by Alex Petrov, Sam Tunnicliffe for CASSANDRA-20059 --- .../org/apache/cassandra/tcm/Processor.java | 37 +++++++++++++++++-- src/java/org/apache/cassandra/tcm/Retry.java | 32 +--------------- 2 files changed, 35 insertions(+), 34 deletions(-) diff --git a/src/java/org/apache/cassandra/tcm/Processor.java b/src/java/org/apache/cassandra/tcm/Processor.java index e3f12852c2f8..fdb4cf23bb4f 100644 --- a/src/java/org/apache/cassandra/tcm/Processor.java +++ b/src/java/org/apache/cassandra/tcm/Processor.java @@ -20,9 +20,11 @@ import java.util.concurrent.TimeUnit; +import com.codahale.metrics.Meter; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.metrics.TCMMetrics; import org.apache.cassandra.tcm.log.Entry; +import org.apache.cassandra.utils.Clock; public interface Processor { @@ -37,9 +39,7 @@ default Commit.Result commit(Entry.Id entryId, Transformation transform, Epoch l // submit the STARTUP message. This allows the bounces affecting majority of CMS nodes to finish successfully. if (transform.kind() == Transformation.Kind.STARTUP) { - return commit(entryId, transform, lastKnown, - Retry.Deadline.retryIndefinitely(DatabaseDescriptor.getCmsAwaitTimeout().to(TimeUnit.NANOSECONDS), - TCMMetrics.instance.commitRetries)); + return commit(entryId, transform, lastKnown, unsafeRetryIndefinitely()); } return commit(entryId, transform, lastKnown, @@ -47,6 +47,37 @@ default Commit.Result commit(Entry.Id entryId, Transformation transform, Epoch l new Retry.Jitter(TCMMetrics.instance.commitRetries))); } + /** + * Since we are using message expiration for communicating timeouts to CMS nodes, we have to be careful not + * to overflow the long, since messaging is using only 32 bits for deadlines. To achieve that, we are + * giving `timeoutNanos` every time we retry, but will retry indefinitely. + */ + private static Retry.Deadline unsafeRetryIndefinitely() + { + long timeoutNanos = DatabaseDescriptor.getCmsAwaitTimeout().to(TimeUnit.NANOSECONDS); + Meter retryMeter = TCMMetrics.instance.commitRetries; + return new Retry.Deadline(Clock.Global.nanoTime() + timeoutNanos, + new Retry.Jitter(retryMeter)) + { + @Override + public boolean reachedMax() + { + return false; + } + + @Override + public long remainingNanos() + { + return timeoutNanos; + } + + public String toString() + { + return String.format("RetryIndefinitely{tries=%d}", currentTries()); + } + }; + } + Commit.Result commit(Entry.Id entryId, Transformation transform, Epoch lastKnown, Retry.Deadline retryPolicy); /** diff --git a/src/java/org/apache/cassandra/tcm/Retry.java b/src/java/org/apache/cassandra/tcm/Retry.java index 703e5904662b..3277531444a6 100644 --- a/src/java/org/apache/cassandra/tcm/Retry.java +++ b/src/java/org/apache/cassandra/tcm/Retry.java @@ -27,7 +27,6 @@ import org.apache.cassandra.utils.Clock; import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly; -import static org.apache.cassandra.tcm.Retry.Jitter.MAX_JITTER_MS; public abstract class Retry { @@ -143,7 +142,7 @@ public static class Deadline extends Retry public final long deadlineNanos; protected final Retry delegate; - private Deadline(long deadlineNanos, Retry delegate) + public Deadline(long deadlineNanos, Retry delegate) { super(delegate.maxTries, delegate.retryMeter); assert deadlineNanos > 0 : String.format("Deadline should be strictly positive but was %d.", deadlineNanos); @@ -161,35 +160,6 @@ public static Deadline after(long timeoutNanos, Retry delegate) return new Deadline(Clock.Global.nanoTime() + timeoutNanos, delegate); } - /** - * Since we are using message expiration for communicating timeouts to CMS nodes, we have to be careful not - * to overflow the long, since messaging is using only 32 bits for deadlines. To achieve that, we are - * giving `timeoutNanos` every time we retry, but will retry indefinitely. - */ - public static Deadline retryIndefinitely(long timeoutNanos, Meter retryMeter) - { - return new Deadline(Clock.Global.nanoTime() + timeoutNanos, - new Retry.Jitter(Integer.MAX_VALUE, MAX_JITTER_MS, new Random(), retryMeter)) - { - @Override - public boolean reachedMax() - { - return false; - } - - @Override - public long remainingNanos() - { - return timeoutNanos; - } - - public String toString() - { - return String.format("RetryIndefinitely{tries=%d}", currentTries()); - } - }; - } - @Override public boolean reachedMax() { From f410b0fa0bc5adbb674654a0e27b02282971cfec Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Thu, 20 Jun 2024 10:29:51 +0200 Subject: [PATCH 047/225] Consolidate all snapshot management to SnapshotManager patch by Stefan Miklosovic; reviewed by Francisco Guerrero for CASSANDRA-18111 --- CHANGES.txt | 1 + NEWS.txt | 3 + .../cassandra/db/ColumnFamilyStore.java | 316 +---- .../org/apache/cassandra/db/Directories.java | 184 +-- .../org/apache/cassandra/db/Keyspace.java | 89 +- .../apache/cassandra/db/SystemKeyspace.java | 25 +- .../db/compaction/CompactionTask.java | 9 +- .../cassandra/db/lifecycle/Tracker.java | 34 +- .../repair/CassandraTableRepairManager.java | 23 +- .../repair/CassandraValidationIterator.java | 10 +- .../cassandra/db/virtual/SnapshotsTable.java | 4 +- .../TableDroppedNotification.java | 34 + .../TablePreScrubNotification.java | 31 + .../notifications/TruncationNotification.java | 14 +- .../cassandra/repair/PreviewRepairTask.java | 3 +- .../service/ActiveRepairService.java | 7 +- .../cassandra/service/CassandraDaemon.java | 13 +- .../service/SnapshotVerbHandler.java | 16 +- .../cassandra/service/StorageService.java | 368 +---- .../service/StorageServiceMBean.java | 25 +- .../snapshot/AbstractSnapshotTask.java | 42 + .../service/snapshot/ClearSnapshotTask.java | 196 +++ .../service/snapshot/GetSnapshotsTask.java | 99 ++ .../service/snapshot/ListSnapshotsTask.java | 148 ++ .../snapshot}/SnapshotDetailsTabularData.java | 39 +- .../service/snapshot/SnapshotException.java | 32 + .../service/snapshot/SnapshotLoader.java | 9 +- .../service/snapshot/SnapshotManager.java | 550 +++++++- .../snapshot/SnapshotManagerMBean.java | 109 ++ .../service/snapshot/SnapshotManifest.java | 7 +- .../service/snapshot/SnapshotOptions.java | 235 ++++ .../service/snapshot/SnapshotType.java | 42 + .../service/snapshot/TableSnapshot.java | 364 +++-- .../service/snapshot/TakeSnapshotTask.java | 310 ++++ .../snapshot/TrueSnapshotSizeTask.java | 76 + .../org/apache/cassandra/tools/NodeProbe.java | 30 +- .../tools/nodetool/ListSnapshots.java | 2 +- .../cassandra/tools/nodetool/Snapshot.java | 21 +- .../utils/DiagnosticSnapshotService.java | 26 +- .../utils/DirectorySizeCalculator.java | 4 +- .../cassandra/distributed/impl/Instance.java | 11 +- .../mock/nodetool/InternalNodeProbe.java | 2 + .../test/AllowAutoSnapshotTest.java | 6 +- .../distributed/test/AutoSnapshotTtlTest.java | 24 +- .../test/EphemeralSnapshotTest.java | 9 + .../test/PreviewRepairSnapshotTest.java | 5 +- .../distributed/test/PreviewRepairTest.java | 5 +- .../test/RepairDigestTrackingTest.java | 11 +- .../test/SSTableIdGenerationTest.java | 27 +- .../distributed/test/SnapshotsTest.java | 109 +- .../fuzz/snapshots/SnapshotsTest.java | 1251 +++++++++++++++++ .../AbstractSnapshotManagerBase.java | 167 +++ .../test/microbench/CompactionBench.java | 4 +- .../test/microbench/SnapshotListingBench.java | 63 + .../test/microbench/SnapshotTakingBench.java | 64 + .../microbench/SnapshotTrueSizeBench.java | 65 + test/unit/org/apache/cassandra/Util.java | 34 +- .../org/apache/cassandra/cql3/CQLTester.java | 2 + .../operations/AutoSnapshotTest.java | 29 +- .../cassandra/db/ColumnFamilyStoreTest.java | 287 +++- .../apache/cassandra/db/DirectoriesTest.java | 96 +- .../org/apache/cassandra/db/KeyspaceTest.java | 34 +- .../cassandra/db/SchemaCQLHelperTest.java | 11 +- .../org/apache/cassandra/db/SnapshotTest.java | 4 +- .../cassandra/db/SystemKeyspaceTest.java | 13 +- .../db/virtual/SnapshotsTableTest.java | 59 +- .../apache/cassandra/index/sai/SAITester.java | 6 +- .../cassandra/index/sasi/SASIIndexTest.java | 8 +- .../cassandra/schema/SchemaKeyspaceTest.java | 5 +- .../service/ActiveRepairServiceTest.java | 3 +- .../service/StorageServiceServerTest.java | 14 +- .../snapshot/MetadataSnapshotsTest.java | 170 +-- .../service/snapshot/SnapshotLoaderTest.java | 8 + .../service/snapshot/SnapshotManagerTest.java | 274 ++++ .../service/snapshot/SnapshotOptionsTest.java | 64 + .../service/snapshot/TableSnapshotTest.java | 118 +- .../StandaloneUpgraderOnSStablesTest.java | 7 +- .../tools/nodetool/ClearSnapshotTest.java | 12 + .../cassandra/stress/CompactionStress.java | 4 +- 79 files changed, 5127 insertions(+), 1508 deletions(-) create mode 100644 src/java/org/apache/cassandra/notifications/TableDroppedNotification.java create mode 100644 src/java/org/apache/cassandra/notifications/TablePreScrubNotification.java create mode 100644 src/java/org/apache/cassandra/service/snapshot/AbstractSnapshotTask.java create mode 100644 src/java/org/apache/cassandra/service/snapshot/ClearSnapshotTask.java create mode 100644 src/java/org/apache/cassandra/service/snapshot/GetSnapshotsTask.java create mode 100644 src/java/org/apache/cassandra/service/snapshot/ListSnapshotsTask.java rename src/java/org/apache/cassandra/{db => service/snapshot}/SnapshotDetailsTabularData.java (65%) create mode 100644 src/java/org/apache/cassandra/service/snapshot/SnapshotException.java create mode 100644 src/java/org/apache/cassandra/service/snapshot/SnapshotManagerMBean.java create mode 100644 src/java/org/apache/cassandra/service/snapshot/SnapshotOptions.java create mode 100644 src/java/org/apache/cassandra/service/snapshot/SnapshotType.java create mode 100644 src/java/org/apache/cassandra/service/snapshot/TakeSnapshotTask.java create mode 100644 src/java/org/apache/cassandra/service/snapshot/TrueSnapshotSizeTask.java create mode 100644 test/distributed/org/apache/cassandra/fuzz/snapshots/SnapshotsTest.java create mode 100644 test/microbench/org/apache/cassandra/test/microbench/AbstractSnapshotManagerBase.java create mode 100644 test/microbench/org/apache/cassandra/test/microbench/SnapshotListingBench.java create mode 100644 test/microbench/org/apache/cassandra/test/microbench/SnapshotTakingBench.java create mode 100644 test/microbench/org/apache/cassandra/test/microbench/SnapshotTrueSizeBench.java create mode 100644 test/unit/org/apache/cassandra/service/snapshot/SnapshotManagerTest.java create mode 100644 test/unit/org/apache/cassandra/service/snapshot/SnapshotOptionsTest.java diff --git a/CHANGES.txt b/CHANGES.txt index c9d9cb17e0a2..88443f88c32d 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Consolidate all snapshot management to SnapshotManager and introduce SnapshotManagerMBean (CASSANDRA-18111) * Fix RequestFailureReason constants codes (CASSANDRA-20126) * Introduce SSTableSimpleScanner for compaction (CASSANDRA-20092) * Include column drop timestamp in alter table transformation (CASSANDRA-18961) diff --git a/NEWS.txt b/NEWS.txt index b4511c3bba9f..740be2eb9647 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -89,6 +89,9 @@ New features when 'GENERATED PASSWORD' clause is used. Character sets supported are: English, Cyrillic, modern Cyrillic, German, Polish and Czech. - JMX SSL configuration can be now done in cassandra.yaml via jmx_encryption_options section instead of cassandra-env.sh + - There is new MBean of name org.apache.cassandra.service.snapshot:type=SnapshotManager which exposes user-facing + snapshot operations. Snapshot-related methods on StorageServiceMBean are still present and functional + but marked as deprecated. Upgrading diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java index d56d690f4a22..cbd3d97c6e10 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -18,11 +18,9 @@ package org.apache.cassandra.db; import java.io.IOException; -import java.io.PrintStream; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; import java.nio.ByteBuffer; -import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -32,7 +30,6 @@ import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashMap; -import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Objects; @@ -68,7 +65,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.google.common.util.concurrent.RateLimiter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -79,7 +75,6 @@ import org.apache.cassandra.concurrent.ExecutorPlus; import org.apache.cassandra.concurrent.FutureTask; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.db.commitlog.CommitLog; import org.apache.cassandra.db.commitlog.CommitLogPosition; import org.apache.cassandra.db.commitlog.IntervalSet; @@ -116,8 +111,6 @@ import org.apache.cassandra.index.SecondaryIndexManager; import org.apache.cassandra.index.internal.CassandraIndex; import org.apache.cassandra.index.transactions.UpdateTransaction; -import org.apache.cassandra.io.FSReadError; -import org.apache.cassandra.io.FSWriteError; import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.Descriptor; import org.apache.cassandra.io.sstable.IScrubber; @@ -131,7 +124,6 @@ import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.format.Version; import org.apache.cassandra.io.util.File; -import org.apache.cassandra.io.util.FileOutputStreamPlus; import org.apache.cassandra.metrics.Sampler; import org.apache.cassandra.metrics.Sampler.Sample; import org.apache.cassandra.metrics.Sampler.SamplerType; @@ -157,9 +149,7 @@ import org.apache.cassandra.service.paxos.Ballot; import org.apache.cassandra.service.paxos.PaxosRepairHistory; import org.apache.cassandra.service.paxos.TablePaxosRepairHistory; -import org.apache.cassandra.service.snapshot.SnapshotLoader; -import org.apache.cassandra.service.snapshot.SnapshotManifest; -import org.apache.cassandra.service.snapshot.TableSnapshot; +import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.streaming.TableStreamManager; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.Epoch; @@ -185,7 +175,6 @@ import static org.apache.cassandra.db.commitlog.CommitLogPosition.NONE; import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; import static org.apache.cassandra.utils.Clock.Global.nanoTime; -import static org.apache.cassandra.utils.FBUtilities.now; import static org.apache.cassandra.utils.Throwables.maybeFail; import static org.apache.cassandra.utils.Throwables.merge; import static org.apache.cassandra.utils.concurrent.CountDownLatch.newCountDownLatch; @@ -260,8 +249,6 @@ public enum FlushReason private static final String SAMPLING_RESULTS_NAME = "SAMPLING_RESULTS"; - public static final String SNAPSHOT_TRUNCATE_PREFIX = "truncated"; - public static final String SNAPSHOT_DROP_PREFIX = "dropped"; static final String TOKEN_DELIMITER = ":"; /** Special values used when the local ranges are not changed with ring changes (e.g. local tables). */ @@ -517,6 +504,7 @@ public ColumnFamilyStore(Keyspace keyspace, // Note that this needs to happen before we load the first sstables, or the global sstable tracker will not // be notified on the initial loading. data.subscribe(StorageService.instance.sstablesTracker); + data.subscribe(SnapshotManager.instance); Collection<SSTableReader> sstables = null; // scan for sstables corresponding to this cf and load them @@ -785,14 +773,11 @@ public static synchronized ColumnFamilyStore createColumnFamilyStore(Keyspace ke * Removes unnecessary files from the cf directory at startup: these include temp files, orphans, zero-length files * and compacted sstables. Files that cannot be recognized will be ignored. */ - public static void scrubDataDirectories(TableMetadata metadata) throws StartupException + public static void scrubDataDirectories(TableMetadata metadata) throws StartupException { Directories directories = new Directories(metadata); Set<File> cleanedDirectories = new HashSet<>(); - // clear ephemeral snapshots that were not properly cleared last session (CASSANDRA-7357) - clearEphemeralSnapshots(directories); - directories.removeTemporaryDirectories(); logger.trace("Removing temporary or obsoleted files from unfinished operations for table {}", metadata.name); @@ -977,6 +962,11 @@ public String getKeyspaceName() return keyspace.getName(); } + public String getKeyspaceTableName() + { + return getKeyspaceName() + '.' + getTableName(); + } + public Descriptor newSSTableDescriptor(File directory) { return newSSTableDescriptor(directory, DatabaseDescriptor.getSelectedSSTableFormat().getLatestVersion()); @@ -1757,12 +1747,8 @@ public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, IScru public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean alwaysFail, IScrubber.Options options, int jobs) throws ExecutionException, InterruptedException { // skip snapshot creation during scrub, SEE JIRA 5891 - if(!disableSnapshot) - { - Instant creationTime = now(); - String snapshotName = "pre-scrub-" + creationTime.toEpochMilli(); - snapshotWithoutMemtable(snapshotName, creationTime); - } + if (!disableSnapshot) + data.notifyPreScrubbed(); try { @@ -2127,262 +2113,6 @@ public ClusteringComparator getComparator() return metadata().comparator; } - public TableSnapshot snapshotWithoutMemtable(String snapshotName) - { - return snapshotWithoutMemtable(snapshotName, now()); - } - - public TableSnapshot snapshotWithoutMemtable(String snapshotName, Instant creationTime) - { - return snapshotWithoutMemtable(snapshotName, null, false, null, null, creationTime); - } - - /** - * @param ephemeral If this flag is set to true, the snapshot will be cleaned during next startup - */ - public TableSnapshot snapshotWithoutMemtable(String snapshotName, Predicate<SSTableReader> predicate, boolean ephemeral, DurationSpec.IntSecondsBound ttl, RateLimiter rateLimiter, Instant creationTime) - { - if (ephemeral && ttl != null) - { - throw new IllegalStateException(String.format("can not take ephemeral snapshot (%s) while ttl is specified too", snapshotName)); - } - - if (rateLimiter == null) - rateLimiter = DatabaseDescriptor.getSnapshotRateLimiter(); - - Set<SSTableReader> snapshottedSSTables = new LinkedHashSet<>(); - for (ColumnFamilyStore cfs : concatWithIndexes()) - { - try (RefViewFragment currentView = cfs.selectAndReference(View.select(SSTableSet.CANONICAL, (x) -> predicate == null || predicate.apply(x)))) - { - for (SSTableReader ssTable : currentView.sstables) - { - File snapshotDirectory = Directories.getSnapshotDirectory(ssTable.descriptor, snapshotName); - ssTable.createLinks(snapshotDirectory.path(), rateLimiter); // hard links - if (logger.isTraceEnabled()) - logger.trace("Snapshot for {} keyspace data file {} created in {}", keyspace, ssTable.getFilename(), snapshotDirectory); - snapshottedSSTables.add(ssTable); - } - } - } - - return createSnapshot(snapshotName, ephemeral, ttl, snapshottedSSTables, creationTime); - } - - protected TableSnapshot createSnapshot(String tag, boolean ephemeral, DurationSpec.IntSecondsBound ttl, Set<SSTableReader> sstables, Instant creationTime) { - Set<File> snapshotDirs = sstables.stream() - .map(s -> Directories.getSnapshotDirectory(s.descriptor, tag).toAbsolute()) - .filter(dir -> !Directories.isSecondaryIndexFolder(dir)) // Remove secondary index subdirectory - .collect(Collectors.toCollection(HashSet::new)); - - // Create and write snapshot manifest - SnapshotManifest manifest = new SnapshotManifest(mapToDataFilenames(sstables), ttl, creationTime, ephemeral); - File manifestFile = getDirectories().getSnapshotManifestFile(tag); - writeSnapshotManifest(manifest, manifestFile); - snapshotDirs.add(manifestFile.parent().toAbsolute()); // manifest may create empty snapshot dir - - // Write snapshot schema - if (!SchemaConstants.isLocalSystemKeyspace(metadata.keyspace) && !SchemaConstants.isReplicatedSystemKeyspace(metadata.keyspace)) - { - File schemaFile = getDirectories().getSnapshotSchemaFile(tag); - writeSnapshotSchema(schemaFile); - snapshotDirs.add(schemaFile.parent().toAbsolute()); // schema may create empty snapshot dir - } - - TableSnapshot snapshot = new TableSnapshot(metadata.keyspace, metadata.name, metadata.id.asUUID(), - tag, manifest.createdAt, manifest.expiresAt, snapshotDirs, - manifest.ephemeral); - - StorageService.instance.addSnapshot(snapshot); - return snapshot; - } - - private SnapshotManifest writeSnapshotManifest(SnapshotManifest manifest, File manifestFile) - { - try - { - manifestFile.parent().tryCreateDirectories(); - manifest.serializeToJsonFile(manifestFile); - return manifest; - } - catch (IOException e) - { - throw new FSWriteError(e, manifestFile); - } - } - - private List<String> mapToDataFilenames(Collection<SSTableReader> sstables) - { - return sstables.stream().map(s -> s.descriptor.relativeFilenameFor(Components.DATA)).collect(Collectors.toList()); - } - - private void writeSnapshotSchema(File schemaFile) - { - try - { - if (!schemaFile.parent().exists()) - schemaFile.parent().tryCreateDirectories(); - - try (PrintStream out = new PrintStream(new FileOutputStreamPlus(schemaFile))) - { - SchemaCQLHelper.reCreateStatementsForSchemaCql(metadata(), - keyspace.getMetadata()) - .forEach(out::println); - } - } - catch (IOException e) - { - throw new FSWriteError(e, schemaFile); - } - } - - protected static void clearEphemeralSnapshots(Directories directories) - { - RateLimiter clearSnapshotRateLimiter = DatabaseDescriptor.getSnapshotRateLimiter(); - - List<TableSnapshot> ephemeralSnapshots = new SnapshotLoader(directories).loadSnapshots() - .stream() - .filter(TableSnapshot::isEphemeral) - .collect(Collectors.toList()); - - for (TableSnapshot ephemeralSnapshot : ephemeralSnapshots) - { - logger.trace("Clearing ephemeral snapshot {} leftover from previous session.", ephemeralSnapshot.getId()); - Directories.clearSnapshot(ephemeralSnapshot.getTag(), directories.getCFDirectories(), clearSnapshotRateLimiter); - } - } - - public Refs<SSTableReader> getSnapshotSSTableReaders(String tag) throws IOException - { - Map<SSTableId, SSTableReader> active = new HashMap<>(); - for (SSTableReader sstable : getSSTables(SSTableSet.CANONICAL)) - active.put(sstable.descriptor.id, sstable); - Map<Descriptor, Set<Component>> snapshots = getDirectories().sstableLister(Directories.OnTxnErr.IGNORE).snapshots(tag).list(); - Refs<SSTableReader> refs = new Refs<>(); - try - { - for (Map.Entry<Descriptor, Set<Component>> entries : snapshots.entrySet()) - { - // Try acquire reference to an active sstable instead of snapshot if it exists, - // to avoid opening new sstables. If it fails, use the snapshot reference instead. - SSTableReader sstable = active.get(entries.getKey().id); - if (sstable == null || !refs.tryRef(sstable)) - { - if (logger.isTraceEnabled()) - logger.trace("using snapshot sstable {}", entries.getKey()); - // open offline so we don't modify components or track hotness. - sstable = SSTableReader.open(this, entries.getKey(), entries.getValue(), metadata, true, true); - refs.tryRef(sstable); - // release the self ref as we never add the snapshot sstable to DataTracker where it is otherwise released - sstable.selfRef().release(); - } - else if (logger.isTraceEnabled()) - { - logger.trace("using active sstable {}", entries.getKey()); - } - } - } - catch (FSReadError | RuntimeException e) - { - // In case one of the snapshot sstables fails to open, - // we must release the references to the ones we opened so far - refs.release(); - throw e; - } - return refs; - } - - /** - * Take a snap shot of this columnfamily store. - * - * @param snapshotName the name of the associated with the snapshot - */ - public TableSnapshot snapshot(String snapshotName) - { - return snapshot(snapshotName, null); - } - - public TableSnapshot snapshot(String snapshotName, DurationSpec.IntSecondsBound ttl) - { - return snapshot(snapshotName, false, ttl, null, now()); - } - - /** - * Take a snap shot of this columnfamily store. - * - * @param snapshotName the name of the associated with the snapshot - * @param skipMemtable Skip flushing the memtable - * @param ttl duration after which the taken snapshot is removed automatically, if supplied with null, it will never be automatically removed - * @param rateLimiter Rate limiter for hardlinks-per-second - * @param creationTime time when this snapshot was taken - */ - public TableSnapshot snapshot(String snapshotName, boolean skipMemtable, DurationSpec.IntSecondsBound ttl, RateLimiter rateLimiter, Instant creationTime) - { - return snapshot(snapshotName, null, false, skipMemtable, ttl, rateLimiter, creationTime); - } - - - /** - * @param ephemeral If this flag is set to true, the snapshot will be cleaned up during next startup - * @param skipMemtable Skip flushing the memtable - */ - public TableSnapshot snapshot(String snapshotName, Predicate<SSTableReader> predicate, boolean ephemeral, boolean skipMemtable) - { - return snapshot(snapshotName, predicate, ephemeral, skipMemtable, null, null, now()); - } - - /** - * @param ephemeral If this flag is set to true, the snapshot will be cleaned up during next startup - * @param skipMemtable Skip flushing the memtable - * @param ttl duration after which the taken snapshot is removed automatically, if supplied with null, it will never be automatically removed - * @param rateLimiter Rate limiter for hardlinks-per-second - * @param creationTime time when this snapshot was taken - */ - public TableSnapshot snapshot(String snapshotName, Predicate<SSTableReader> predicate, boolean ephemeral, boolean skipMemtable, DurationSpec.IntSecondsBound ttl, RateLimiter rateLimiter, Instant creationTime) - { - if (!skipMemtable) - { - Memtable current = getTracker().getView().getCurrentMemtable(); - if (!current.isClean()) - { - if (current.shouldSwitch(FlushReason.SNAPSHOT)) - FBUtilities.waitOnFuture(switchMemtableIfCurrent(current, FlushReason.SNAPSHOT)); - else - current.performSnapshot(snapshotName); - } - } - return snapshotWithoutMemtable(snapshotName, predicate, ephemeral, ttl, rateLimiter, creationTime); - } - - public boolean snapshotExists(String snapshotName) - { - return getDirectories().snapshotExists(snapshotName); - } - - - /** - * Clear all the snapshots for a given column family. - * - * @param snapshotName the user supplied snapshot name. If left empty, - * all the snapshots will be cleaned. - */ - public void clearSnapshot(String snapshotName) - { - RateLimiter clearSnapshotRateLimiter = DatabaseDescriptor.getSnapshotRateLimiter(); - - List<File> snapshotDirs = getDirectories().getCFDirectories(); - Directories.clearSnapshot(snapshotName, snapshotDirs, clearSnapshotRateLimiter); - } - /** - * - * @return Return a map of all snapshots to space being used - * The pair for a snapshot has true size and size on disk. - */ - public Map<String, TableSnapshot> listSnapshots() - { - return getDirectories().listSnapshots(); - } - /** * @return the cached partition for @param key if it is already present in the cache. * Not that this will not readAndCache the parition if it is not present, nor @@ -2771,15 +2501,12 @@ public void run() // stream in data that is actually supposed to have been deleted ActiveRepairService.instance().abort((prs) -> prs.getTableIds().contains(metadata.id), "Stopping parent sessions {} due to truncation of tableId="+metadata.id); - data.notifyTruncated(truncatedAt); - - if (!noSnapshot && isAutoSnapshotEnabled()) - snapshot(Keyspace.getTimestampedSnapshotNameWithPrefix(name, SNAPSHOT_TRUNCATE_PREFIX), DatabaseDescriptor.getAutoSnapshotTtl()); + data.notifyTruncated(noSnapshot, truncatedAt, DatabaseDescriptor.getAutoSnapshotTtl()); - discardSSTables(truncatedAt); + discardSSTables(truncatedAt); - indexManager.truncateAllIndexesBlocking(truncatedAt); - viewManager.truncateBlocking(replayAfter, truncatedAt); + indexManager.truncateAllIndexesBlocking(truncatedAt); + viewManager.truncateBlocking(replayAfter, truncatedAt); SystemKeyspace.saveTruncationRecord(ColumnFamilyStore.this, truncatedAt, replayAfter); logger.trace("cleaning out row cache"); @@ -3303,10 +3030,20 @@ public double getDroppableTombstoneRatio() return allColumns > 0 ? allDroppable / allColumns : 0; } + public Set<String> getFilesOfCfs() + { + Set<String> files = new HashSet<>(); + + for (ColumnFamilyStore cfs : concatWithIndexes()) + cfs.getTracker().getView().liveSSTables().forEach(s -> files.addAll(s.getAllFilePaths())); + + return files; + } + @Override public long trueSnapshotsSize() { - return getDirectories().trueSnapshotsSize(); + return SnapshotManager.instance.getTrueSnapshotsSize(getKeyspaceName(), getTableName()); } /** @@ -3426,8 +3163,7 @@ void onTableDropped() CompactionManager.instance.interruptCompactionForCFs(concatWithIndexes(), (sstable) -> true, true); - if (isAutoSnapshotEnabled()) - snapshot(Keyspace.getTimestampedSnapshotNameWithPrefix(name, ColumnFamilyStore.SNAPSHOT_DROP_PREFIX), DatabaseDescriptor.getAutoSnapshotTtl()); + data.notifyDropped(DatabaseDescriptor.getAutoSnapshotTtl()); CommitLog.instance.forceRecycleAllSegments(Collections.singleton(metadata.id)); diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java index c6e30c961d79..12a64090f853 100644 --- a/src/java/org/apache/cassandra/db/Directories.java +++ b/src/java/org/apache/cassandra/db/Directories.java @@ -22,7 +22,6 @@ import java.nio.file.FileStore; import java.nio.file.Files; import java.nio.file.Path; -import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -49,8 +48,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; -import com.google.common.collect.Maps; -import com.google.common.util.concurrent.RateLimiter; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -74,8 +71,6 @@ import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.snapshot.SnapshotManifest; -import org.apache.cassandra.service.snapshot.TableSnapshot; -import org.apache.cassandra.utils.DirectorySizeCalculator; import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.Pair; @@ -306,6 +301,11 @@ public Directories(final TableMetadata metadata, DataDirectory[] paths) canonicalPathToDD = canonicalPathsBuilder.build(); } + public File[] getDataPaths() + { + return dataPaths; + } + /** * Returns SSTable location which is inside given data directory. * @@ -673,6 +673,16 @@ public static Optional<File> getSnapshotDirectoryIfExists(File location, String } } + public Set<File> getSnapshotDirs(String tag) + { + Set<File> snapshotDirs = new HashSet<>(); + + for (File cfDir : getCFDirectories()) + snapshotDirs.add(Directories.getSnapshotDirectory(cfDir, tag).toAbsolute()); + + return snapshotDirs; + } + public File getSnapshotManifestFile(String snapshotName) { File snapshotDir = getSnapshotDirectory(getDirectoryForNewSSTables(), snapshotName); @@ -1171,37 +1181,6 @@ private BiPredicate<File, FileType> getFilter(boolean includeForeignTables) } } - public Map<String, TableSnapshot> listSnapshots() - { - Map<String, Set<File>> snapshotDirsByTag = listSnapshotDirsByTag(); - - Map<String, TableSnapshot> snapshots = Maps.newHashMapWithExpectedSize(snapshotDirsByTag.size()); - - for (Map.Entry<String, Set<File>> entry : snapshotDirsByTag.entrySet()) - { - String tag = entry.getKey(); - Set<File> snapshotDirs = entry.getValue(); - SnapshotManifest manifest = maybeLoadManifest(metadata.keyspace, metadata.name, tag, snapshotDirs); - snapshots.put(tag, buildSnapshot(tag, manifest, snapshotDirs)); - } - - return snapshots; - } - - private TableSnapshot buildSnapshot(String tag, SnapshotManifest manifest, Set<File> snapshotDirs) - { - boolean ephemeral = manifest != null ? manifest.isEphemeral() : isLegacyEphemeralSnapshot(snapshotDirs); - Instant createdAt = manifest == null ? null : manifest.createdAt; - Instant expiresAt = manifest == null ? null : manifest.expiresAt; - return new TableSnapshot(metadata.keyspace, metadata.name, metadata.id.asUUID(), tag, createdAt, expiresAt, - snapshotDirs, ephemeral); - } - - private static boolean isLegacyEphemeralSnapshot(Set<File> snapshotDirs) - { - return snapshotDirs.stream().map(d -> new File(d, "ephemeral.snapshot")).anyMatch(File::exists); - } - @VisibleForTesting protected static SnapshotManifest maybeLoadManifest(String keyspace, String table, String tag, Set<File> snapshotDirs) { @@ -1230,100 +1209,10 @@ protected static SnapshotManifest maybeLoadManifest(String keyspace, String tabl return null; } - @VisibleForTesting - protected Map<String, Set<File>> listSnapshotDirsByTag() - { - Map<String, Set<File>> snapshotDirsByTag = new HashMap<>(); - for (final File dir : dataPaths) - { - File snapshotDir = isSecondaryIndexFolder(dir) - ? new File(dir.parentPath(), SNAPSHOT_SUBDIR) - : new File(dir, SNAPSHOT_SUBDIR); - if (snapshotDir.exists() && snapshotDir.isDirectory()) - { - final File[] snapshotDirs = snapshotDir.tryList(); - if (snapshotDirs != null) - { - for (final File snapshot : snapshotDirs) - { - if (snapshot.isDirectory()) { - snapshotDirsByTag.computeIfAbsent(snapshot.name(), k -> new LinkedHashSet<>()).add(snapshot.toAbsolute()); - } - } - } - } - } - return snapshotDirsByTag; - } - - public boolean snapshotExists(String snapshotName) - { - for (File dir : dataPaths) - { - File snapshotDir; - if (isSecondaryIndexFolder(dir)) - { - snapshotDir = new File(dir.parent(), join(SNAPSHOT_SUBDIR, snapshotName, dir.name())); - } - else - { - snapshotDir = new File(dir, join(SNAPSHOT_SUBDIR, snapshotName)); - } - if (snapshotDir.exists()) - return true; - } - return false; - } - - public static void clearSnapshot(String snapshotName, List<File> tableDirectories, RateLimiter snapshotRateLimiter) - { - // If snapshotName is empty or null, we will delete the entire snapshot directory - String tag = snapshotName == null ? "" : snapshotName; - for (File tableDir : tableDirectories) - { - File snapshotDir = new File(tableDir, join(SNAPSHOT_SUBDIR, tag)); - removeSnapshotDirectory(snapshotRateLimiter, snapshotDir); - } - } - - public static void removeSnapshotDirectory(RateLimiter snapshotRateLimiter, File snapshotDir) - { - if (snapshotDir.exists()) - { - logger.trace("Removing snapshot directory {}", snapshotDir); - try - { - FileUtils.deleteRecursiveWithThrottle(snapshotDir, snapshotRateLimiter); - } - catch (RuntimeException ex) - { - if (!snapshotDir.exists()) - return; // ignore - throw ex; - } - } - } - - /** - * @return total snapshot size in byte for all snapshots. - */ - public long trueSnapshotsSize() - { - long result = 0L; - for (File dir : dataPaths) - { - File snapshotDir = isSecondaryIndexFolder(dir) - ? new File(dir.parentPath(), SNAPSHOT_SUBDIR) - : new File(dir, SNAPSHOT_SUBDIR); - result += getTrueAllocatedSizeIn(snapshotDir); - } - return result; - } - /** * @return Raw size on disk for all directories */ - public long getRawDiretoriesSize() + public long getRawDirectoriesSize() { long totalAllocatedSize = 0L; @@ -1333,25 +1222,7 @@ public long getRawDiretoriesSize() return totalAllocatedSize; } - public long getTrueAllocatedSizeIn(File snapshotDir) - { - if (!snapshotDir.isDirectory()) - return 0; - - SSTableSizeSummer visitor = new SSTableSizeSummer(sstableLister(OnTxnErr.THROW).listFiles()); - try - { - Files.walkFileTree(snapshotDir.toPath(), visitor); - } - catch (IOException e) - { - logger.error("Could not calculate the size of {}. {}", snapshotDir, e.getMessage()); - } - - return visitor.getAllocatedSize(); - } - - // Recursively finds all the sub directories in the KS directory. + // Recursively finds all the subdirectories in the KS directory. public static List<File> getKSChildDirectories(String ksName) { List<File> result = new ArrayList<>(); @@ -1434,25 +1305,4 @@ private static String join(String... s) { return StringUtils.join(s, File.pathSeparator()); } - - private class SSTableSizeSummer extends DirectorySizeCalculator - { - private final Set<String> toSkip; - SSTableSizeSummer(List<File> files) - { - toSkip = files.stream().map(File::name).collect(Collectors.toSet()); - } - - @Override - public boolean isAcceptable(Path path) - { - File file = new File(path); - Descriptor desc = SSTable.tryDescriptorFromFile(file); - return desc != null - && desc.ksname.equals(metadata.keyspace) - && desc.cfname.equals(metadata.name) - && !toSkip.contains(file.name()); - } - } - } diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java index 7a18f112ba55..af651570bb8f 100644 --- a/src/java/org/apache/cassandra/db/Keyspace.java +++ b/src/java/org/apache/cassandra/db/Keyspace.java @@ -17,8 +17,6 @@ */ package org.apache.cassandra.db; -import java.io.IOException; -import java.time.Instant; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -35,18 +33,17 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Iterables; -import com.google.common.util.concurrent.RateLimiter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.cassandra.concurrent.Stage; import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.db.lifecycle.SSTableSet; import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.db.repair.CassandraKeyspaceRepairManager; import org.apache.cassandra.db.view.ViewManager; +import org.apache.cassandra.db.virtual.VirtualKeyspaceRegistry; import org.apache.cassandra.exceptions.WriteTimeoutException; import org.apache.cassandra.index.Index; import org.apache.cassandra.index.SecondaryIndexManager; @@ -61,7 +58,6 @@ import org.apache.cassandra.schema.SchemaProvider; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.service.snapshot.TableSnapshot; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.JVMStabilityInspector; @@ -74,7 +70,6 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.NANOSECONDS; import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; -import static org.apache.cassandra.utils.FBUtilities.now; import static org.apache.cassandra.utils.MonotonicClock.Global.approxTime; /** @@ -226,80 +221,19 @@ public boolean hasColumnFamilyStore(TableId id) return columnFamilyStores.containsKey(id); } - /** - * Take a snapshot of the specific column family, or the entire set of column families - * if columnFamily is null with a given timestamp - * - * @param snapshotName the tag associated with the name of the snapshot. This value may not be null - * @param columnFamilyName the column family to snapshot or all on null - * @param skipFlush Skip blocking flush of memtable - * @param rateLimiter Rate limiter for hardlinks-per-second - * @throws IOException if the column family doesn't exist - */ - public void snapshot(String snapshotName, String columnFamilyName, boolean skipFlush, DurationSpec.IntSecondsBound ttl, RateLimiter rateLimiter, Instant creationTime) throws IOException + public static void verifyKeyspaceIsValid(String keyspaceName) { - assert snapshotName != null; - boolean tookSnapShot = false; - for (ColumnFamilyStore cfStore : columnFamilyStores.values()) - { - if (columnFamilyName == null || cfStore.name.equals(columnFamilyName)) - { - tookSnapShot = true; - cfStore.snapshot(snapshotName, skipFlush, ttl, rateLimiter, creationTime); - } - } + if (null != VirtualKeyspaceRegistry.instance.getKeyspaceNullable(keyspaceName)) + throw new IllegalArgumentException("Cannot perform any operations against virtual keyspace " + keyspaceName); - if ((columnFamilyName != null) && !tookSnapShot) - throw new IOException("Failed taking snapshot. Table " + columnFamilyName + " does not exist."); + if (!Schema.instance.getKeyspaces().contains(keyspaceName)) + throw new IllegalArgumentException("Keyspace " + keyspaceName + " does not exist"); } - /** - * Take a snapshot of the specific column family, or the entire set of column families - * if columnFamily is null with a given timestamp - * - * @param snapshotName the tag associated with the name of the snapshot. This value may not be null - * @param columnFamilyName the column family to snapshot or all on null - * @throws IOException if the column family doesn't exist - */ - public void snapshot(String snapshotName, String columnFamilyName) throws IOException + public static Keyspace getValidKeyspace(String keyspaceName) { - snapshot(snapshotName, columnFamilyName, false, null, null, now()); - } - - /** - * @param clientSuppliedName may be null. - * @return the name of the snapshot - */ - public static String getTimestampedSnapshotName(String clientSuppliedName) - { - String snapshotName = Long.toString(currentTimeMillis()); - if (clientSuppliedName != null && !clientSuppliedName.equals("")) - { - snapshotName = snapshotName + "-" + clientSuppliedName; - } - return snapshotName; - } - - public static String getTimestampedSnapshotNameWithPrefix(String clientSuppliedName, String prefix) - { - return prefix + "-" + getTimestampedSnapshotName(clientSuppliedName); - } - - /** - * Check whether snapshots already exists for a given name. - * - * @param snapshotName the user supplied snapshot name - * @return true if the snapshot exists - */ - public boolean snapshotExists(String snapshotName) - { - assert snapshotName != null; - for (ColumnFamilyStore cfStore : columnFamilyStores.values()) - { - if (cfStore.snapshotExists(snapshotName)) - return true; - } - return false; + verifyKeyspaceIsValid(keyspaceName); + return Keyspace.open(keyspaceName); } /** @@ -313,11 +247,6 @@ public List<SSTableReader> getAllSSTables(SSTableSet sstableSet) return list; } - public Stream<TableSnapshot> getAllSnapshots() - { - return getColumnFamilyStores().stream().flatMap(cfs -> cfs.listSnapshots().values().stream()); - } - public static Keyspace forSchema(String keyspaceName, SchemaProvider schema) { return new Keyspace(keyspaceName, schema, true); diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java index 05a24375466c..2130d568d2a7 100644 --- a/src/java/org/apache/cassandra/db/SystemKeyspace.java +++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java @@ -109,6 +109,9 @@ import org.apache.cassandra.service.paxos.PaxosState; import org.apache.cassandra.service.paxos.uncommitted.PaxosRows; import org.apache.cassandra.service.paxos.uncommitted.PaxosUncommittedIndex; +import org.apache.cassandra.service.snapshot.SnapshotOptions; +import org.apache.cassandra.service.snapshot.SnapshotManager; +import org.apache.cassandra.service.snapshot.SnapshotType; import org.apache.cassandra.streaming.StreamOperation; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.Epoch; @@ -141,10 +144,10 @@ import static org.apache.cassandra.gms.ApplicationState.STATUS_WITH_PORT; import static org.apache.cassandra.gms.ApplicationState.TOKENS; import static org.apache.cassandra.service.paxos.Commit.latest; +import static org.apache.cassandra.service.snapshot.SnapshotOptions.systemSnapshot; import static org.apache.cassandra.utils.CassandraVersion.NULL_VERSION; import static org.apache.cassandra.utils.CassandraVersion.UNREADABLE_VERSION; import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; -import static org.apache.cassandra.utils.FBUtilities.now; public final class SystemKeyspace { @@ -1808,10 +1811,8 @@ public static synchronized Map<InetAddressAndPort, Set<Range<Token>>> getTransfe * Compare the release version in the system.local table with the one included in the distro. * If they don't match, snapshot all tables in the system and schema keyspaces. This is intended * to be called at startup to create a backup of the system tables during an upgrade - * - * @throws IOException */ - public static void snapshotOnVersionChange() throws IOException + public static void snapshotOnVersionChange() { String previous = getPreviousVersionString(); String next = FBUtilities.getReleaseVersionString(); @@ -1820,16 +1821,18 @@ public static void snapshotOnVersionChange() throws IOException // if we're restarting after an upgrade, snapshot the system and schema keyspaces if (!previous.equals(NULL_VERSION.toString()) && !previous.equals(next)) - { + List<String> entities = new ArrayList<>(); + for (String keyspace : SchemaConstants.LOCAL_SYSTEM_KEYSPACE_NAMES) + { + for (ColumnFamilyStore cfs : Keyspace.open(keyspace).getColumnFamilyStores()) + entities.add(cfs.getKeyspaceTableName()); + } + logger.info("Detected version upgrade from {} to {}, snapshotting system keyspaces", previous, next); - String snapshotName = Keyspace.getTimestampedSnapshotName(format("upgrade-%s-%s", - previous, - next)); - Instant creationTime = now(); - for (String keyspace : SchemaConstants.LOCAL_SYSTEM_KEYSPACE_NAMES) - Keyspace.open(keyspace).snapshot(snapshotName, null, false, null, null, creationTime); + SnapshotOptions options = systemSnapshot(format("%s-%s", previous, next), SnapshotType.UPGRADE, entities.toArray(new String[0])).build(); + SnapshotManager.instance.takeSnapshot(options); } } diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java index 79368f68d3e7..4f27736f266d 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java @@ -17,7 +17,6 @@ */ package org.apache.cassandra.db.compaction; -import java.time.Instant; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; @@ -46,6 +45,9 @@ import org.apache.cassandra.io.sstable.metadata.MetadataCollector; import org.apache.cassandra.io.util.File; import org.apache.cassandra.service.ActiveRepairService; +import org.apache.cassandra.service.snapshot.SnapshotManager; +import org.apache.cassandra.service.snapshot.SnapshotOptions; +import org.apache.cassandra.service.snapshot.SnapshotType; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.TimeUUID; import org.apache.cassandra.utils.concurrent.Refs; @@ -53,7 +55,6 @@ import static org.apache.cassandra.db.compaction.CompactionHistoryTabularData.COMPACTION_TYPE_PROPERTY; import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; import static org.apache.cassandra.utils.Clock.Global.nanoTime; -import static org.apache.cassandra.utils.FBUtilities.now; public class CompactionTask extends AbstractCompactionTask { @@ -126,8 +127,8 @@ protected void runMayThrow() throws Exception if (DatabaseDescriptor.isSnapshotBeforeCompaction()) { - Instant creationTime = now(); - cfs.snapshotWithoutMemtable(creationTime.toEpochMilli() + "-compact-" + cfs.name, creationTime); + SnapshotOptions options = SnapshotOptions.systemSnapshot(cfs.name, SnapshotType.COMPACT, cfs.getKeyspaceTableName()).skipFlush().build(); + SnapshotManager.instance.takeSnapshot(options); } try (CompactionController controller = getCompactionController(transaction.originals())) diff --git a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java index 58ffc1df3f38..adc18cd3b510 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java +++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java @@ -35,6 +35,7 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Directories; import org.apache.cassandra.db.commitlog.CommitLogPosition; @@ -56,6 +57,8 @@ import org.apache.cassandra.notifications.SSTableListChangedNotification; import org.apache.cassandra.notifications.SSTableMetadataChanged; import org.apache.cassandra.notifications.SSTableRepairStatusChanged; +import org.apache.cassandra.notifications.TableDroppedNotification; +import org.apache.cassandra.notifications.TablePreScrubNotification; import org.apache.cassandra.notifications.TruncationNotification; import org.apache.cassandra.utils.Pair; import org.apache.cassandra.utils.Throwables; @@ -510,31 +513,34 @@ public void notifySSTableRepairedStatusChanged(Collection<SSTableReader> repairS { if (repairStatusesChanged.isEmpty()) return; - INotification notification = new SSTableRepairStatusChanged(repairStatusesChanged); - for (INotificationConsumer subscriber : subscribers) - subscriber.handleNotification(notification, this); + notify(new SSTableRepairStatusChanged(repairStatusesChanged)); } public void notifySSTableMetadataChanged(SSTableReader levelChanged, StatsMetadata oldMetadata) { - INotification notification = new SSTableMetadataChanged(levelChanged, oldMetadata); - for (INotificationConsumer subscriber : subscribers) - subscriber.handleNotification(notification, this); - + notify(new SSTableMetadataChanged(levelChanged, oldMetadata)); } public void notifyDeleting(SSTableReader deleting) { - INotification notification = new SSTableDeletingNotification(deleting); - for (INotificationConsumer subscriber : subscribers) - subscriber.handleNotification(notification, this); + notify(new SSTableDeletingNotification(deleting)); } - public void notifyTruncated(long truncatedAt) + public void notifyTruncated(boolean disableSnapshot, + long truncatedAt, + DurationSpec.IntSecondsBound ttl) { - INotification notification = new TruncationNotification(truncatedAt); - for (INotificationConsumer subscriber : subscribers) - subscriber.handleNotification(notification, this); + notify(new TruncationNotification(cfstore, disableSnapshot, truncatedAt, ttl)); + } + + public void notifyDropped(DurationSpec.IntSecondsBound ttl) + { + notify(new TableDroppedNotification(cfstore, ttl)); + } + + public void notifyPreScrubbed() + { + notify(new TablePreScrubNotification(cfstore)); } public void notifyRenewed(Memtable renewed) diff --git a/src/java/org/apache/cassandra/db/repair/CassandraTableRepairManager.java b/src/java/org/apache/cassandra/db/repair/CassandraTableRepairManager.java index 24e79d24544b..3b6535752e6f 100644 --- a/src/java/org/apache/cassandra/db/repair/CassandraTableRepairManager.java +++ b/src/java/org/apache/cassandra/db/repair/CassandraTableRepairManager.java @@ -22,8 +22,7 @@ import java.util.Collection; import java.util.concurrent.Callable; import java.util.concurrent.Future; - -import com.google.common.base.Predicate; +import java.util.function.Predicate; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.compaction.CompactionManager; @@ -36,6 +35,9 @@ import org.apache.cassandra.repair.TableRepairManager; import org.apache.cassandra.repair.ValidationPartitionIterator; import org.apache.cassandra.repair.NoSuchRepairSessionException; +import org.apache.cassandra.service.snapshot.SnapshotOptions; +import org.apache.cassandra.service.snapshot.SnapshotManager; +import org.apache.cassandra.service.snapshot.SnapshotType; import org.apache.cassandra.utils.TimeUUID; import org.apache.cassandra.service.ActiveRepairService; @@ -79,17 +81,14 @@ public synchronized void snapshot(String name, Collection<Range<Token>> ranges, try { ActiveRepairService.instance().snapshotExecutor.submit(() -> { - if (force || !cfs.snapshotExists(name)) + if (force || !SnapshotManager.instance.exists(cfs.getKeyspaceName(), cfs.getTableName(), name)) { - cfs.snapshot(name, new Predicate<SSTableReader>() - { - public boolean apply(SSTableReader sstable) - { - return sstable != null && - !sstable.metadata().isIndex() && // exclude SSTables from 2i - new Bounds<>(sstable.getFirst().getToken(), sstable.getLast().getToken()).intersects(ranges); - } - }, true, false); //ephemeral snapshot, if repair fails, it will be cleaned next startup + Predicate<SSTableReader> predicate = sstable -> sstable != null && + !sstable.metadata().isIndex() && // exclude SSTables from 2i + new Bounds<>(sstable.getFirst().getToken(), sstable.getLast().getToken()).intersects(ranges); + + SnapshotOptions options = SnapshotOptions.systemSnapshot(name, SnapshotType.REPAIR, predicate, cfs.getKeyspaceTableName()).ephemeral().build(); + SnapshotManager.instance.takeSnapshot(options); } }).get(); } diff --git a/src/java/org/apache/cassandra/db/repair/CassandraValidationIterator.java b/src/java/org/apache/cassandra/db/repair/CassandraValidationIterator.java index 05408e91752d..5d4d88ed9efc 100644 --- a/src/java/org/apache/cassandra/db/repair/CassandraValidationIterator.java +++ b/src/java/org/apache/cassandra/db/repair/CassandraValidationIterator.java @@ -57,6 +57,8 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.ActiveRepairService; import org.apache.cassandra.repair.NoSuchRepairSessionException; +import org.apache.cassandra.service.snapshot.SnapshotManager; +import org.apache.cassandra.service.snapshot.TableSnapshot; import org.apache.cassandra.utils.TimeUUID; import org.apache.cassandra.utils.concurrent.Refs; @@ -179,19 +181,19 @@ public CassandraValidationIterator(ColumnFamilyStore cfs, SharedContext ctx, Col this.cfs = cfs; this.ctx = ctx; - isGlobalSnapshotValidation = cfs.snapshotExists(parentId.toString()); + isGlobalSnapshotValidation = SnapshotManager.instance.exists(cfs.getKeyspaceName(), cfs.getTableName(), parentId.toString()); if (isGlobalSnapshotValidation) snapshotName = parentId.toString(); else snapshotName = sessionID.toString(); - isSnapshotValidation = cfs.snapshotExists(snapshotName); + isSnapshotValidation = SnapshotManager.instance.exists(cfs.getKeyspaceName(), cfs.getTableName(), snapshotName); if (isSnapshotValidation) { // If there is a snapshot created for the session then read from there. // note that we populate the parent repair session when creating the snapshot, meaning the sstables in the snapshot are the ones we // are supposed to validate. - sstables = cfs.getSnapshotSSTableReaders(snapshotName); + sstables = TableSnapshot.getSnapshotSSTableReaders(cfs, snapshotName); } else { @@ -270,7 +272,7 @@ public void close() { // we can only clear the snapshot if we are not doing a global snapshot validation (we then clear it once anticompaction // is done). - cfs.clearSnapshot(snapshotName); + SnapshotManager.instance.clearSnapshot(cfs.getKeyspaceName(), cfs.getTableName(), snapshotName); } if (sstables != null) diff --git a/src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java b/src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java index d3df293903b9..c757ce7c54f5 100644 --- a/src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java +++ b/src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java @@ -26,7 +26,7 @@ import org.apache.cassandra.db.marshal.UTF8Type; import org.apache.cassandra.dht.LocalPartitioner; import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.service.snapshot.TableSnapshot; public class SnapshotsTable extends AbstractVirtualTable @@ -62,7 +62,7 @@ public DataSet data() { SimpleDataSet result = new SimpleDataSet(metadata()); - for (TableSnapshot tableSnapshot : StorageService.instance.snapshotManager.loadSnapshots()) + for (TableSnapshot tableSnapshot : SnapshotManager.instance.getSnapshots(false, true)) { SimpleDataSet row = result.row(tableSnapshot.getTag(), tableSnapshot.getKeyspaceName(), diff --git a/src/java/org/apache/cassandra/notifications/TableDroppedNotification.java b/src/java/org/apache/cassandra/notifications/TableDroppedNotification.java new file mode 100644 index 000000000000..aea1e1c9ba17 --- /dev/null +++ b/src/java/org/apache/cassandra/notifications/TableDroppedNotification.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.notifications; + +import org.apache.cassandra.config.DurationSpec; +import org.apache.cassandra.db.ColumnFamilyStore; + +public class TableDroppedNotification implements INotification +{ + public final ColumnFamilyStore cfs; + public final DurationSpec.IntSecondsBound ttl; + + public TableDroppedNotification(ColumnFamilyStore cfs, DurationSpec.IntSecondsBound ttl) + { + this.cfs = cfs; + this.ttl = ttl; + } +} diff --git a/src/java/org/apache/cassandra/notifications/TablePreScrubNotification.java b/src/java/org/apache/cassandra/notifications/TablePreScrubNotification.java new file mode 100644 index 000000000000..c8fad45b8e2c --- /dev/null +++ b/src/java/org/apache/cassandra/notifications/TablePreScrubNotification.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.notifications; + +import org.apache.cassandra.db.ColumnFamilyStore; + +public class TablePreScrubNotification implements INotification +{ + public final ColumnFamilyStore cfs; + + public TablePreScrubNotification(ColumnFamilyStore cfs) + { + this.cfs = cfs; + } +} diff --git a/src/java/org/apache/cassandra/notifications/TruncationNotification.java b/src/java/org/apache/cassandra/notifications/TruncationNotification.java index 345dd17e290c..cdf1ba515a94 100644 --- a/src/java/org/apache/cassandra/notifications/TruncationNotification.java +++ b/src/java/org/apache/cassandra/notifications/TruncationNotification.java @@ -17,16 +17,28 @@ */ package org.apache.cassandra.notifications; +import org.apache.cassandra.config.DurationSpec; +import org.apache.cassandra.db.ColumnFamilyStore; + /** * Fired during truncate, after the memtable has been flushed but before any * snapshot is taken and SSTables are discarded */ public class TruncationNotification implements INotification { + public final ColumnFamilyStore cfs; + public final boolean disableSnapshot; public final long truncatedAt; + public final DurationSpec.IntSecondsBound ttl; - public TruncationNotification(long truncatedAt) + public TruncationNotification(ColumnFamilyStore cfs, + boolean disableSnapshot, + long truncatedAt, + DurationSpec.IntSecondsBound ttl) { + this.cfs = cfs; + this.disableSnapshot = disableSnapshot; this.truncatedAt = truncatedAt; + this.ttl = ttl; } } diff --git a/src/java/org/apache/cassandra/repair/PreviewRepairTask.java b/src/java/org/apache/cassandra/repair/PreviewRepairTask.java index edee11cf2007..95c7a63f9466 100644 --- a/src/java/org/apache/cassandra/repair/PreviewRepairTask.java +++ b/src/java/org/apache/cassandra/repair/PreviewRepairTask.java @@ -32,6 +32,7 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.metrics.RepairMetrics; import org.apache.cassandra.repair.consistent.SyncStatSummary; +import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.utils.DiagnosticSnapshotService; import org.apache.cassandra.utils.TimeUUID; @@ -128,7 +129,7 @@ private void maybeSnapshotReplicas(TimeUUID parentSession, String keyspace, List for (String table : mismatchingTables) { // we can just check snapshot existence locally since the repair coordinator is always a replica (unlike in the read case) - if (!Keyspace.open(keyspace).getColumnFamilyStore(table).snapshotExists(snapshotName)) + if (!SnapshotManager.instance.exists(keyspace, table, snapshotName)) { List<Range<Token>> normalizedRanges = Range.normalize(ranges); logger.info("{} Snapshotting {}.{} for preview repair mismatch for ranges {} with tag {} on instances {}", diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java index b5fbc48d4374..1a60bd306745 100644 --- a/src/java/org/apache/cassandra/service/ActiveRepairService.java +++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java @@ -106,6 +106,7 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.paxos.PaxosRepair; import org.apache.cassandra.service.paxos.cleanup.PaxosCleanup; +import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.utils.ExecutorUtils; @@ -884,10 +885,8 @@ public synchronized ParentRepairSession removeParentRepairSession(TimeUUID paren .map(cfs -> cfs.metadata().toString()).collect(Collectors.joining(", "))); long startNanos = ctx.clock().nanoTime(); for (ColumnFamilyStore cfs : session.columnFamilyStores.values()) - { - if (cfs.snapshotExists(snapshotName)) - cfs.clearSnapshot(snapshotName); - } + SnapshotManager.instance.clearSnapshot(cfs.keyspace.getName(), cfs.name, snapshotName); + logger.info("[repair #{}] Cleared snapshots in {}ms", parentSessionId, TimeUnit.NANOSECONDS.toMillis(ctx.clock().nanoTime() - startNanos)); }); } diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java index ec66cf2fe9de..ef6c689edf47 100644 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@ -71,6 +71,7 @@ import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.security.ThreadAwareSecurityManager; import org.apache.cassandra.service.paxos.PaxosState; +import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.streaming.StreamManager; import org.apache.cassandra.tcm.CMSOperations; import org.apache.cassandra.tcm.ClusterMetadata; @@ -266,8 +267,16 @@ protected void setup() DatabaseDescriptor.createAllDirectories(); Keyspace.setInitialized(); CommitLog.instance.start(); - runStartupChecks(); + SnapshotManager.instance.start(false); + SnapshotManager.instance.clearExpiredSnapshots(); + SnapshotManager.instance.clearEphemeralSnapshots(); + SnapshotManager.instance.resumeSnapshotCleanup(); + SnapshotManager.instance.registerMBean(); + + // clearing of snapshots above here will in fact clear all ephemeral snapshots + // which were cleared as part of startup checks before CASSANDRA-18111 + runStartupChecks(); try { @@ -291,7 +300,7 @@ protected void setup() { SystemKeyspace.snapshotOnVersionChange(); } - catch (IOException e) + catch (Throwable e) { exitOrFail(StartupException.ERR_WRONG_DISK_STATE, e.getMessage(), e.getCause()); } diff --git a/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java b/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java index db0957898151..fc80089d37ee 100644 --- a/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java +++ b/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java @@ -20,11 +20,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.SnapshotCommand; import org.apache.cassandra.net.IVerbHandler; import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.service.snapshot.SnapshotManager; +import org.apache.cassandra.service.snapshot.SnapshotOptions; +import org.apache.cassandra.service.snapshot.SnapshotType; import org.apache.cassandra.utils.DiagnosticSnapshotService; @@ -38,7 +40,7 @@ public void doVerb(Message<SnapshotCommand> message) SnapshotCommand command = message.payload; if (command.clear_snapshot) { - StorageService.instance.clearSnapshot(command.snapshot_name, command.keyspace); + SnapshotManager.instance.clearSnapshots(command.snapshot_name, command.keyspace); } else if (DiagnosticSnapshotService.isDiagnosticSnapshotRequest(command)) { @@ -46,7 +48,15 @@ else if (DiagnosticSnapshotService.isDiagnosticSnapshotRequest(command)) } else { - Keyspace.open(command.keyspace).getColumnFamilyStore(command.column_family).snapshot(command.snapshot_name); + try + { + SnapshotOptions options = SnapshotOptions.systemSnapshot(command.snapshot_name, SnapshotType.DIAGNOSTICS, command.keyspace + '.' + command.column_family).build(); + SnapshotManager.instance.takeSnapshot(options); + } + catch (Exception ex) + { + throw new RuntimeException(ex); + } } logger.debug("Enqueuing response to snapshot request {} to {}", command.snapshot_name, message.from()); diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index ab8ddfe9bbb1..aa21f59ef2d6 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -22,8 +22,6 @@ import java.net.InetAddress; import java.net.UnknownHostException; import java.nio.ByteBuffer; -import java.time.Instant; -import java.time.format.DateTimeParseException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -59,7 +57,6 @@ import javax.management.openmbean.CompositeData; import javax.management.openmbean.OpenDataException; import javax.management.openmbean.TabularData; -import javax.management.openmbean.TabularDataSupport; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; @@ -72,7 +69,6 @@ import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; -import com.google.common.util.concurrent.RateLimiter; import com.google.common.util.concurrent.Uninterruptibles; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; @@ -96,21 +92,18 @@ import org.apache.cassandra.config.Converters; import org.apache.cassandra.config.DataStorageSpec; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.cql3.QueryHandler; import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.SizeEstimatesRecorder; -import org.apache.cassandra.db.SnapshotDetailsTabularData; import org.apache.cassandra.db.SystemKeyspace; import org.apache.cassandra.db.commitlog.CommitLog; import org.apache.cassandra.db.compaction.CompactionManager; import org.apache.cassandra.db.compaction.OperationType; import org.apache.cassandra.db.guardrails.Guardrails; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; -import org.apache.cassandra.db.virtual.VirtualKeyspaceRegistry; import org.apache.cassandra.dht.BootStrapper; import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.OwnedRanges; @@ -179,7 +172,6 @@ import org.apache.cassandra.service.paxos.cleanup.PaxosCleanupLocalCoordinator; import org.apache.cassandra.service.paxos.cleanup.PaxosRepairState; import org.apache.cassandra.service.snapshot.SnapshotManager; -import org.apache.cassandra.service.snapshot.TableSnapshot; import org.apache.cassandra.streaming.StreamManager; import org.apache.cassandra.streaming.StreamResultFuture; import org.apache.cassandra.streaming.StreamState; @@ -208,7 +200,6 @@ import org.apache.cassandra.tcm.transformations.Unregister; import org.apache.cassandra.transport.ClientResourceLimits; import org.apache.cassandra.transport.ProtocolVersion; -import org.apache.cassandra.utils.Clock; import org.apache.cassandra.utils.ExecutorUtils; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.JVMStabilityInspector; @@ -263,7 +254,6 @@ import static org.apache.cassandra.tcm.membership.NodeState.REGISTERED; import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; import static org.apache.cassandra.utils.FBUtilities.getBroadcastAddressAndPort; -import static org.apache.cassandra.utils.FBUtilities.now; /** * This abstraction contains the token/identifier of this node @@ -312,8 +302,6 @@ private static int getRingDelay() private final List<Runnable> preShutdownHooks = new ArrayList<>(); private final List<Runnable> postShutdownHooks = new ArrayList<>(); - public final SnapshotManager snapshotManager = new SnapshotManager(); - public static final StorageService instance = new StorageService(); private final SamplingManager samplingManager = new SamplingManager(); @@ -884,7 +872,6 @@ public void maybeInitializeServices() DiskUsageBroadcaster.instance.startBroadcasting(); HintsService.instance.startDispatch(); BatchlogManager.instance.start(); - startSnapshotManager(); servicesInitialized = true; } @@ -925,12 +912,6 @@ public static boolean isSeed() return DatabaseDescriptor.getSeeds().contains(getBroadcastAddressAndPort()); } - @VisibleForTesting - public void startSnapshotManager() - { - snapshotManager.start(); - } - public static boolean isReplacingSameAddress() { InetAddressAndPort replaceAddress = DatabaseDescriptor.getReplaceAddress(); @@ -2742,55 +2723,6 @@ public int garbageCollect(String tombstoneOptionString, int jobs, String keyspac return status.statusCode; } - /** - * Takes the snapshot of a multiple column family from different keyspaces. A snapshot name must be specified. - * - * @param tag - * the tag given to the snapshot; may not be null or empty - * @param options - * Map of options (skipFlush is the only supported option for now) - * @param entities - * list of keyspaces / tables in the form of empty | ks1 ks2 ... | ks1.cf1,ks2.cf2,... - */ - @Override - public void takeSnapshot(String tag, Map<String, String> options, String... entities) throws IOException - { - DurationSpec.IntSecondsBound ttl = options.containsKey("ttl") ? new DurationSpec.IntSecondsBound(options.get("ttl")) : null; - if (ttl != null) - { - int minAllowedTtlSecs = CassandraRelevantProperties.SNAPSHOT_MIN_ALLOWED_TTL_SECONDS.getInt(); - if (ttl.toSeconds() < minAllowedTtlSecs) - throw new IllegalArgumentException(String.format("ttl for snapshot must be at least %d seconds", minAllowedTtlSecs)); - } - - boolean skipFlush = Boolean.parseBoolean(options.getOrDefault("skipFlush", "false")); - if (entities != null && entities.length > 0 && entities[0].contains(".")) - { - takeMultipleTableSnapshot(tag, skipFlush, ttl, entities); - } - else - { - takeSnapshot(tag, skipFlush, ttl, entities); - } - } - - /** - * Takes the snapshot of a specific table. A snapshot name must be - * specified. - * - * @param keyspaceName - * the keyspace which holds the specified table - * @param tableName - * the table to snapshot - * @param tag - * the tag given to the snapshot; may not be null or empty - */ - public void takeTableSnapshot(String keyspaceName, String tableName, String tag) - throws IOException - { - takeMultipleTableSnapshot(tag, false, null, keyspaceName + "." + tableName); - } - @Override public void forceKeyspaceCompactionForTokenRange(String keyspaceName, String startToken, String endToken, String... tableNames) throws IOException, ExecutionException, InterruptedException { @@ -2840,296 +2772,126 @@ public void forceKeyspaceCompactionForPartitionKey(String keyspaceName, String p public void forceCompactionKeysIgnoringGcGrace(String keyspaceName, String tableName, String... partitionKeysIgnoreGcGrace) throws IOException, ExecutionException, InterruptedException { - ColumnFamilyStore cfStore = getValidKeyspace(keyspaceName).getColumnFamilyStore(tableName); + ColumnFamilyStore cfStore = Keyspace.getValidKeyspace(keyspaceName).getColumnFamilyStore(tableName); cfStore.forceCompactionKeysIgnoringGcGrace(partitionKeysIgnoreGcGrace); } /** - * Takes the snapshot for the given keyspaces. A snapshot name must be specified. + * Takes the snapshot of a multiple column family from different keyspaces. A snapshot name must be specified. * - * @param tag the tag given to the snapshot; may not be null or empty - * @param keyspaceNames the names of the keyspaces to snapshot; empty means "all." + * @param tag the tag given to the snapshot; may not be null or empty + * @param options Map of options (skipFlush is the only supported option for now) + * @param entities list of keyspaces / tables in the form of empty | ks1 ks2 ... | ks1.cf1,ks2.cf2,... + * + * @deprecated See CASSANDRA-18111 */ - public void takeSnapshot(String tag, String... keyspaceNames) throws IOException + @Override + @Deprecated(since = "5.1") + public void takeSnapshot(String tag, Map<String, String> options, String... entities) throws IOException { - takeSnapshot(tag, false, null, keyspaceNames); + SnapshotManager.instance.takeSnapshot(tag, options, entities); } /** - * Takes the snapshot of a multiple column family from different keyspaces. A snapshot name must be specified. + * Takes the snapshot of a specific table. A snapshot name must be + * specified. * - * @param tag - * the tag given to the snapshot; may not be null or empty - * @param tableList - * list of tables from different keyspace in the form of ks1.cf1 ks2.cf2 + * @param keyspaceName the keyspace which holds the specified table + * @param tableName the table to snapshot + * @param tag the tag given to the snapshot; may not be null or empty + * + * @deprecated use {@link #takeSnapshot(String tag, Map options, String... entities)} instead. See CASSANDRA-10907 */ - public void takeMultipleTableSnapshot(String tag, String... tableList) - throws IOException + @Override + @Deprecated(since = "3.4") + public void takeTableSnapshot(String keyspaceName, String tableName, String tag) throws IOException { - takeMultipleTableSnapshot(tag, false, null, tableList); + SnapshotManager.instance.takeSnapshot(tag, Collections.emptyMap(), keyspaceName + '.' + tableName); } /** * Takes the snapshot for the given keyspaces. A snapshot name must be specified. * * @param tag the tag given to the snapshot; may not be null or empty - * @param skipFlush Skip blocking flush of memtable * @param keyspaceNames the names of the keyspaces to snapshot; empty means "all." + * + * @deprecated use {@link #takeSnapshot(String tag, Map options, String... entities)} instead. See CASSANDRA-10907 */ - private void takeSnapshot(String tag, boolean skipFlush, DurationSpec.IntSecondsBound ttl, String... keyspaceNames) throws IOException + @Override + @Deprecated(since = "3.4") + public void takeSnapshot(String tag, String... keyspaceNames) throws IOException { - if (operationMode() == Mode.JOINING) - throw new IOException("Cannot snapshot until bootstrap completes"); - if (tag == null || tag.equals("")) - throw new IOException("You must supply a snapshot name."); - - Iterable<Keyspace> keyspaces; - if (keyspaceNames.length == 0) - { - keyspaces = Keyspace.all(); - } - else - { - ArrayList<Keyspace> t = new ArrayList<>(keyspaceNames.length); - for (String keyspaceName : keyspaceNames) - t.add(getValidKeyspace(keyspaceName)); - keyspaces = t; - } - - // Do a check to see if this snapshot exists before we actually snapshot - for (Keyspace keyspace : keyspaces) - if (keyspace.snapshotExists(tag)) - throw new IOException("Snapshot " + tag + " already exists."); - - - RateLimiter snapshotRateLimiter = DatabaseDescriptor.getSnapshotRateLimiter(); - Instant creationTime = now(); - - for (Keyspace keyspace : keyspaces) - { - keyspace.snapshot(tag, null, skipFlush, ttl, snapshotRateLimiter, creationTime); - } + SnapshotManager.instance.takeSnapshot(tag, Collections.emptyMap(), keyspaceNames); } /** * Takes the snapshot of a multiple column family from different keyspaces. A snapshot name must be specified. * + * @param tag the tag given to the snapshot; may not be null or empty + * @param tableList list of tables from different keyspace in the form of ks1.cf1 ks2.cf2 * - * @param tag - * the tag given to the snapshot; may not be null or empty - * @param skipFlush - * Skip blocking flush of memtable - * @param tableList - * list of tables from different keyspace in the form of ks1.cf1 ks2.cf2 + * @deprecated See CASSANDRA-10907 */ - private void takeMultipleTableSnapshot(String tag, boolean skipFlush, DurationSpec.IntSecondsBound ttl, String... tableList) - throws IOException - { - Map<Keyspace, List<String>> keyspaceColumnfamily = new HashMap<Keyspace, List<String>>(); - for (String table : tableList) - { - String splittedString[] = StringUtils.split(table, '.'); - if (splittedString.length == 2) - { - String keyspaceName = splittedString[0]; - String tableName = splittedString[1]; - - if (keyspaceName == null) - throw new IOException("You must supply a keyspace name"); - if (operationMode() == Mode.JOINING) - throw new IOException("Cannot snapshot until bootstrap completes"); - - if (tableName == null) - throw new IOException("You must supply a table name"); - if (tag == null || tag.equals("")) - throw new IOException("You must supply a snapshot name."); - - Keyspace keyspace = getValidKeyspace(keyspaceName); - ColumnFamilyStore columnFamilyStore = keyspace.getColumnFamilyStore(tableName); - // As there can be multiple column family from same keyspace check if snapshot exist for that specific - // columnfamily and not for whole keyspace - - if (columnFamilyStore.snapshotExists(tag)) - throw new IOException("Snapshot " + tag + " already exists."); - if (!keyspaceColumnfamily.containsKey(keyspace)) - { - keyspaceColumnfamily.put(keyspace, new ArrayList<String>()); - } - - // Add Keyspace columnfamily to map in order to support atomicity for snapshot process. - // So no snapshot should happen if any one of the above conditions fail for any keyspace or columnfamily - keyspaceColumnfamily.get(keyspace).add(tableName); - - } - else - { - throw new IllegalArgumentException( - "Cannot take a snapshot on secondary index or invalid column family name. You must supply a column family name in the form of keyspace.columnfamily"); - } - } - - RateLimiter snapshotRateLimiter = DatabaseDescriptor.getSnapshotRateLimiter(); - Instant creationTime = now(); - - for (Entry<Keyspace, List<String>> entry : keyspaceColumnfamily.entrySet()) - { - for (String table : entry.getValue()) - entry.getKey().snapshot(tag, table, skipFlush, ttl, snapshotRateLimiter, creationTime); - } - } - - private void verifyKeyspaceIsValid(String keyspaceName) - { - if (null != VirtualKeyspaceRegistry.instance.getKeyspaceNullable(keyspaceName)) - throw new IllegalArgumentException("Cannot perform any operations against virtual keyspace " + keyspaceName); - - if (!Schema.instance.getKeyspaces().contains(keyspaceName)) - throw new IllegalArgumentException("Keyspace " + keyspaceName + " does not exist"); - } - - private Keyspace getValidKeyspace(String keyspaceName) + @Override + @Deprecated(since = "3.4") + public void takeMultipleTableSnapshot(String tag, String... tableList) throws IOException { - verifyKeyspaceIsValid(keyspaceName); - return Keyspace.open(keyspaceName); + SnapshotManager.instance.takeSnapshot(tag, Collections.emptyMap(), tableList); } /** * Remove the snapshot with the given name from the given keyspaces. * If no tag is specified we will remove all snapshots. + * + * @deprecated See CASSANDRA-16860 */ - public void clearSnapshot(String tag, String... keyspaceNames) + @Override + @Deprecated(since = "5.0") + public void clearSnapshot(String tag, String... keyspaceNames) throws IOException { - clearSnapshot(Collections.emptyMap(), tag, keyspaceNames); + SnapshotManager.instance.clearSnapshot(tag, Collections.emptyMap(), keyspaceNames); } + @Override + @Deprecated(since = "5.1") public void clearSnapshot(Map<String, Object> options, String tag, String... keyspaceNames) { - if (tag == null) - tag = ""; - - if (options == null) - options = Collections.emptyMap(); - - Set<String> keyspaces = new HashSet<>(); - for (String dataDir : DatabaseDescriptor.getAllDataFileLocations()) - { - for (String keyspaceDir : new File(dataDir).tryListNames()) - { - // Only add a ks if it has been specified as a param, assuming params were actually provided. - if (keyspaceNames.length > 0 && !Arrays.asList(keyspaceNames).contains(keyspaceDir)) - continue; - keyspaces.add(keyspaceDir); - } - } - - Object olderThan = options.get("older_than"); - Object olderThanTimestamp = options.get("older_than_timestamp"); - - final long clearOlderThanTimestamp; - if (olderThan != null) - { - assert olderThan instanceof String : "it is expected that older_than is an instance of java.lang.String"; - clearOlderThanTimestamp = Clock.Global.currentTimeMillis() - new DurationSpec.LongSecondsBound((String) olderThan).toMilliseconds(); - } - else if (olderThanTimestamp != null) - { - assert olderThanTimestamp instanceof String : "it is expected that older_than_timestamp is an instance of java.lang.String"; - try - { - clearOlderThanTimestamp = Instant.parse((String) olderThanTimestamp).toEpochMilli(); - } - catch (DateTimeParseException ex) - { - throw new RuntimeException("Parameter older_than_timestamp has to be a valid instant in ISO format."); - } - } - else - clearOlderThanTimestamp = 0L; - - for (String keyspace : keyspaces) - clearKeyspaceSnapshot(keyspace, tag, clearOlderThanTimestamp); - - if (logger.isDebugEnabled()) - logger.debug("Cleared out snapshot directories"); - } - - /** - * Clear snapshots for a given keyspace. - * @param keyspace keyspace to remove snapshots for - * @param tag the user supplied snapshot name. If empty or null, all the snapshots will be cleaned - * @param olderThanTimestamp if a snapshot was created before this timestamp, it will be cleared, - * if its value is 0, this parameter is effectively ignored. - */ - private void clearKeyspaceSnapshot(String keyspace, String tag, long olderThanTimestamp) - { - Set<TableSnapshot> snapshotsToClear = snapshotManager.loadSnapshots(keyspace) - .stream() - .filter(TableSnapshot.shouldClearSnapshot(tag, olderThanTimestamp)) - .collect(Collectors.toSet()); - for (TableSnapshot snapshot : snapshotsToClear) - snapshotManager.clearSnapshot(snapshot); + SnapshotManager.instance.clearSnapshot(tag, options, keyspaceNames); } + @Override + @Deprecated(since = "5.1") public Map<String, TabularData> getSnapshotDetails(Map<String, String> options) { - boolean skipExpiring = options != null && Boolean.parseBoolean(options.getOrDefault("no_ttl", "false")); - boolean includeEphemeral = options != null && Boolean.parseBoolean(options.getOrDefault("include_ephemeral", "false")); - - Map<String, TabularData> snapshotMap = new HashMap<>(); - - for (TableSnapshot snapshot : snapshotManager.loadSnapshots()) - { - if (skipExpiring && snapshot.isExpiring()) - continue; - if (!includeEphemeral && snapshot.isEphemeral()) - continue; - - TabularDataSupport data = (TabularDataSupport) snapshotMap.get(snapshot.getTag()); - if (data == null) - { - data = new TabularDataSupport(SnapshotDetailsTabularData.TABULAR_TYPE); - snapshotMap.put(snapshot.getTag(), data); - } - - SnapshotDetailsTabularData.from(snapshot, data); - } - - return snapshotMap; + return SnapshotManager.instance.listSnapshots(options); } - /** @deprecated See CASSANDRA-16789 */ + @Override @Deprecated(since = "4.1") public Map<String, TabularData> getSnapshotDetails() { - return getSnapshotDetails(ImmutableMap.of()); + return SnapshotManager.instance.listSnapshots(ImmutableMap.of()); } + @Override + @Deprecated(since = "5.1") public long trueSnapshotsSize() { - long total = 0; - for (Keyspace keyspace : Keyspace.all()) - { - if (isLocalSystemKeyspace(keyspace.getName())) - continue; - - for (ColumnFamilyStore cfStore : keyspace.getColumnFamilyStores()) - { - total += cfStore.trueSnapshotsSize(); - } - } - - return total; + return SnapshotManager.instance.getTrueSnapshotSize(); } + @Override + @Deprecated(since = "5.1") public void setSnapshotLinksPerSecond(long throttle) { - logger.info("Setting snapshot throttle to {}", throttle); - DatabaseDescriptor.setSnapshotLinksPerSecond(throttle); + SnapshotManager.instance.setSnapshotLinksPerSecond(throttle); } + @Override + @Deprecated(since = "5.1") public long getSnapshotLinksPerSecond() { - return DatabaseDescriptor.getSnapshotLinksPerSecond(); + return SnapshotManager.instance.getSnapshotLinksPerSecond(); } public void refreshSizeEstimates() throws ExecutionException @@ -3152,7 +2914,7 @@ public void cleanupSizeEstimates() */ public Iterable<ColumnFamilyStore> getValidColumnFamilies(boolean allowIndexes, boolean autoAddIndexes, String keyspaceName, String... cfNames) { - Keyspace keyspace = getValidKeyspace(keyspaceName); + Keyspace keyspace = Keyspace.getValidKeyspace(keyspaceName); return keyspace.getValidColumnFamilies(allowIndexes, autoAddIndexes, cfNames); } @@ -3944,7 +3706,7 @@ protected synchronized void drain(boolean isFinalShutdown) throws IOException, I logger.error("Batchlog manager timed out shutting down", t); } - snapshotManager.stop(); + SnapshotManager.instance.close(); HintsService.instance.pauseDispatch(); if (daemon != null) @@ -4038,7 +3800,7 @@ protected synchronized void drain(boolean isFinalShutdown) throws IOException, I } FBUtilities.waitOnFutures(flushes); - SnapshotManager.shutdownAndWait(1L, MINUTES); + SnapshotManager.instance.shutdownAndWait(1L, MINUTES); HintsService.instance.shutdownBlocking(); // Interrupt ongoing compactions and shutdown CM to prevent further compactions. @@ -4167,7 +3929,7 @@ public IPartitioner setPartitionerUnsafe(IPartitioner newPartitioner) public void truncate(String keyspace, String table) throws TimeoutException, IOException { - verifyKeyspaceIsValid(keyspace); + Keyspace.verifyKeyspaceIsValid(keyspace); try { @@ -4537,7 +4299,7 @@ public void loadNewSSTables(String ksName, String cfName) { if (!isInitialized()) throw new RuntimeException("Not yet initialized, can't load new sstables"); - verifyKeyspaceIsValid(ksName); + Keyspace.verifyKeyspaceIsValid(ksName); ColumnFamilyStore.loadNewSSTables(ksName, cfName); } @@ -5240,10 +5002,6 @@ public int getCompactionTombstoneWarningThreshold() return Math.toIntExact(Guardrails.instance.getPartitionTombstonesWarnThreshold()); } - public void addSnapshot(TableSnapshot snapshot) { - snapshotManager.addSnapshot(snapshot); - } - @Override public boolean getReadThresholdsEnabled() { diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java index f23b3b454382..2a9096762a33 100644 --- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java +++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java @@ -273,7 +273,9 @@ public interface StorageServiceMBean extends NotificationEmitter public void takeTableSnapshot(String keyspaceName, String tableName, String tag) throws IOException; /** - * @deprecated use {@link #takeSnapshot(String tag, Map options, String... entities)} instead. See CASSANDRA-10907 + * Use {@link #takeSnapshot(String tag, Map options, String... entities)} instead. + * + * @deprecated See CASSANDRA-10907 */ @Deprecated(since = "3.4") public void takeMultipleTableSnapshot(String tag, String... tableList) throws IOException; @@ -281,13 +283,12 @@ public interface StorageServiceMBean extends NotificationEmitter /** * Takes the snapshot of a multiple column family from different keyspaces. A snapshot name must be specified. * - * @param tag - * the tag given to the snapshot; may not be null or empty - * @param options - * Map of options (skipFlush is the only supported option for now) - * @param entities - * list of keyspaces / tables in the form of empty | ks1 ks2 ... | ks1.cf1,ks2.cf2,... + * @param tag the tag given to the snapshot; may not be null or empty + * @param options Map of options (skipFlush is the only supported option for now) + * @param entities list of keyspaces / tables in the form of empty | ks1 ks2 ... | ks1.cf1,ks2.cf2,... + * @deprecated See CASSANDRA-18111 */ + @Deprecated(since = "5.1") public void takeSnapshot(String tag, Map<String, String> options, String... entities) throws IOException; /** @@ -308,7 +309,9 @@ public interface StorageServiceMBean extends NotificationEmitter * @param options map of options for cleanup operation, consult nodetool's ClearSnapshot * @param tag name of snapshot to clear, if null or empty string, all snapshots of given keyspace will be cleared * @param keyspaceNames name of keyspaces to clear snapshots for + * @deprecated See CASSANDRA-18111 */ + @Deprecated(since = "5.1") public void clearSnapshot(Map<String, Object> options, String tag, String... keyspaceNames) throws IOException; /** @@ -324,13 +327,17 @@ public interface StorageServiceMBean extends NotificationEmitter * * @param options map of options used for filtering of snapshots * @return A map of snapshotName to all its details in Tabular form. + * @deprecated See CASSANDRA-18111 */ + @Deprecated(since = "5.1") public Map<String, TabularData> getSnapshotDetails(Map<String, String> options); /** * Get the true size taken by all snapshots across all keyspaces. * @return True size taken by all the snapshots. + * @deprecated See CASSANDRA-18111 */ + @Deprecated(since = "5.1") public long trueSnapshotsSize(); /** @@ -338,7 +345,9 @@ public interface StorageServiceMBean extends NotificationEmitter * A setting of zero indicates no throttling * * @param throttle + * @deprecated See CASSANDRA-18111 */ + @Deprecated(since = "5.1") public void setSnapshotLinksPerSecond(long throttle); /** @@ -346,7 +355,9 @@ public interface StorageServiceMBean extends NotificationEmitter * A setting of zero indicates no throttling. * * @return snapshot links-per-second throttle + * @deprecated See CASSANDRA-18111 */ + @Deprecated(since = "5.1") public long getSnapshotLinksPerSecond(); /** diff --git a/src/java/org/apache/cassandra/service/snapshot/AbstractSnapshotTask.java b/src/java/org/apache/cassandra/service/snapshot/AbstractSnapshotTask.java new file mode 100644 index 000000000000..318a0042d588 --- /dev/null +++ b/src/java/org/apache/cassandra/service/snapshot/AbstractSnapshotTask.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.snapshot; + +import java.util.concurrent.Callable; + +public abstract class AbstractSnapshotTask<T> implements Callable<T> +{ + public enum SnapshotTaskType + { + SNAPSHOT, + CLEAR, + RELOAD, + LIST, + SIZE + } + + protected final SnapshotOptions options; + + public AbstractSnapshotTask(SnapshotOptions options) + { + this.options = options; + } + + public abstract SnapshotTaskType getTaskType(); +} diff --git a/src/java/org/apache/cassandra/service/snapshot/ClearSnapshotTask.java b/src/java/org/apache/cassandra/service/snapshot/ClearSnapshotTask.java new file mode 100644 index 000000000000..a2e2e828bcf2 --- /dev/null +++ b/src/java/org/apache/cassandra/service/snapshot/ClearSnapshotTask.java @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.snapshot; + +import java.time.Instant; +import java.time.format.DateTimeParseException; +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.function.Predicate; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.DurationSpec; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.utils.Clock; + +public class ClearSnapshotTask extends AbstractSnapshotTask<Void> +{ + private static final Logger logger = LoggerFactory.getLogger(ClearSnapshotTask.class); + + private final SnapshotManager manager; + private final Predicate<TableSnapshot> predicateForToBeCleanedSnapshots; + private final boolean deleteData; + + public ClearSnapshotTask(SnapshotManager manager, + Predicate<TableSnapshot> predicateForToBeCleanedSnapshots, + boolean deleteData) + { + super(null); + this.manager = manager; + this.predicateForToBeCleanedSnapshots = predicateForToBeCleanedSnapshots; + this.deleteData = deleteData; + } + + @Override + public SnapshotTaskType getTaskType() + { + return SnapshotTaskType.CLEAR; + } + + @Override + public Void call() + { + Set<TableSnapshot> toRemove = new HashSet<>(); + + for (TableSnapshot snapshot : new GetSnapshotsTask(manager, predicateForToBeCleanedSnapshots, false).call()) + { + logger.debug("Removing snapshot {}{}", snapshot, deleteData ? ", deleting data" : ""); + + toRemove.add(snapshot); + + if (deleteData) + { + for (File snapshotDir : snapshot.getDirectories()) + { + try + { + removeSnapshotDirectory(snapshotDir); + } + catch (Throwable ex) + { + logger.warn("Unable to remove snapshot directory {}", snapshotDir, ex); + } + } + } + } + + manager.getSnapshots().removeAll(toRemove); + + return null; + } + + /** + * Returns predicate which will pass the test when arguments match. + * + * @param tag name of snapshot + * @param options options for filtering + * @param keyspaceNames names of keyspaces a snapshot is supposed to be from + * @return predicate which will pass the test when arguments match. + */ + static Predicate<TableSnapshot> getPredicateForCleanedSnapshots(String tag, Map<String, Object> options, String... keyspaceNames) + { + if (options == null) + options = Collections.emptyMap(); + + Object olderThan = options.get("older_than"); + Object olderThanTimestamp = options.get("older_than_timestamp"); + + long maxCreatedAt = Clock.Global.currentTimeMillis(); + if (olderThan != null) + { + assert olderThan instanceof String : "it is expected that older_than is an instance of java.lang.String"; + maxCreatedAt -= new DurationSpec.LongSecondsBound((String) olderThan).toMilliseconds(); + } + else if (olderThanTimestamp != null) + { + assert olderThanTimestamp instanceof String : "it is expected that older_than_timestamp is an instance of java.lang.String"; + try + { + maxCreatedAt = Instant.parse((String) olderThanTimestamp).toEpochMilli(); + } + catch (DateTimeParseException ex) + { + throw new RuntimeException("Parameter older_than_timestamp has to be a valid instant in ISO format."); + } + } + + return getClearSnapshotPredicate(tag, Set.of(keyspaceNames), maxCreatedAt, false); + } + + /** + * Returns a predicate based on which a snapshot will be included for deletion or not. + * + * @param tag name of snapshot to remove + * @param keyspaces keyspaces this snapshot belongs to + * @param olderThanTimestamp clear the snapshot if it is older than given timestamp + * @param includeEphemeral whether to include ephemeral snapshots as well + * @return predicate which filters snapshots on given parameters + */ + static Predicate<TableSnapshot> getClearSnapshotPredicate(String tag, + Set<String> keyspaces, + long olderThanTimestamp, + boolean includeEphemeral) + { + return ts -> + { + // When no tag is supplied, all snapshots must be cleared + boolean clearAll = tag == null || tag.isEmpty(); + if (!clearAll && ts.isEphemeral() && !includeEphemeral) + logger.info("Skipping deletion of ephemeral snapshot '{}' in keyspace {}. " + + "Ephemeral snapshots are not removable by a user.", + tag, ts.getKeyspaceName()); + boolean passedEphemeralTest = !ts.isEphemeral() || (ts.isEphemeral() && includeEphemeral); + boolean shouldClearTag = clearAll || ts.getTag().equals(tag); + boolean byTimestamp = true; + + if (olderThanTimestamp > 0L) + { + Instant createdAt = ts.getCreatedAt(); + if (createdAt != null) + byTimestamp = createdAt.isBefore(Instant.ofEpochMilli(olderThanTimestamp)); + } + + boolean byKeyspace = (keyspaces.isEmpty() || keyspaces.contains(ts.getKeyspaceName())); + + return passedEphemeralTest && shouldClearTag && byTimestamp && byKeyspace; + }; + } + + private void removeSnapshotDirectory(File snapshotDir) + { + if (snapshotDir.exists()) + { + logger.trace("Removing snapshot directory {}", snapshotDir); + try + { + + snapshotDir.deleteRecursive(DatabaseDescriptor.getSnapshotRateLimiter()); + } + catch (RuntimeException ex) + { + if (!snapshotDir.exists()) + return; // ignore + throw ex; + } + } + } + + @Override + public String toString() + { + return "ClearSnapshotTask{" + + "deleteData=" + deleteData + + '}'; + } +} diff --git a/src/java/org/apache/cassandra/service/snapshot/GetSnapshotsTask.java b/src/java/org/apache/cassandra/service/snapshot/GetSnapshotsTask.java new file mode 100644 index 000000000000..6180425d1a28 --- /dev/null +++ b/src/java/org/apache/cassandra/service/snapshot/GetSnapshotsTask.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.snapshot; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.function.Predicate; + +public class GetSnapshotsTask implements Callable<List<TableSnapshot>> +{ + private final SnapshotManager snapshotManager; + private final Predicate<TableSnapshot> predicate; + private final boolean shouldRemoveIfNotExists; + + public GetSnapshotsTask(SnapshotManager snapshotManager, + Predicate<TableSnapshot> predicate, + boolean shouldRemoveIfNotExists) + { + this.snapshotManager = snapshotManager; + this.predicate = predicate; + this.shouldRemoveIfNotExists = shouldRemoveIfNotExists; + } + + @Override + public List<TableSnapshot> call() + { + if (shouldRemoveIfNotExists) + return getWithRemoval(); + else + return getWithoutRemoval(); + } + + private List<TableSnapshot> getWithRemoval() + { + List<TableSnapshot> notExistingAnymore = new ArrayList<>(); + List<TableSnapshot> snapshots = new ArrayList<>(); + for (TableSnapshot snapshot : snapshotManager.getSnapshots()) + { + if (snapshot.isCompleted() && predicate.test(snapshot)) + { + if (!snapshot.hasManifest()) + notExistingAnymore.add(snapshot); + else + snapshots.add(snapshot); + } + } + + // we do not want to clear snapshots which do not exist periodically + // because that would beat the purpose of caching (we would need to go to the disk + // to see if manifests still exists every time), hence, we will clean on listing, + // we do not need to have cache clean of non-existing snapshots when nobody is looking + if (!notExistingAnymore.isEmpty()) + { + snapshotManager.getSnapshots().removeAll(notExistingAnymore); + } + + return snapshots; + } + + private List<TableSnapshot> getWithoutRemoval() + { + List<TableSnapshot> snapshots = new ArrayList<>(); + for (TableSnapshot snapshot : snapshotManager.getSnapshots()) + { + if (snapshot.isCompleted() && predicate.test(snapshot)) + { + if (snapshot.hasManifest()) + snapshots.add(snapshot); + } + } + + return snapshots; + } + + @Override + public String toString() + { + return "GetSnapshotsTask{" + + "shouldRemoveIfNotExists=" + shouldRemoveIfNotExists + + '}'; + } +} diff --git a/src/java/org/apache/cassandra/service/snapshot/ListSnapshotsTask.java b/src/java/org/apache/cassandra/service/snapshot/ListSnapshotsTask.java new file mode 100644 index 000000000000..1351c12d0b0b --- /dev/null +++ b/src/java/org/apache/cassandra/service/snapshot/ListSnapshotsTask.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.snapshot; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.function.Predicate; +import javax.management.openmbean.TabularData; +import javax.management.openmbean.TabularDataSupport; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.ColumnFamilyStore; + +public class ListSnapshotsTask implements Callable<Map<String, TabularData>> +{ + private static final Logger logger = LoggerFactory.getLogger(ListSnapshotsTask.class); + + private final SnapshotManager snapshotManager; + private final Predicate<TableSnapshot> predicate; + private final Map<String, String> options; + private final boolean shouldRemoveIfNotExists; + + public ListSnapshotsTask(SnapshotManager snapshotManager, + Map<String, String> options, + boolean shouldRemoveIfNotExists) + { + this.snapshotManager = snapshotManager; + this.options = options; + this.predicate = getListingSnapshotsPredicate(options); + this.shouldRemoveIfNotExists = shouldRemoveIfNotExists; + } + + private static Predicate<TableSnapshot> getListingSnapshotsPredicate(Map<String, String> options) + { + boolean skipExpiring = options != null && Boolean.parseBoolean(options.getOrDefault("no_ttl", "false")); + boolean includeEphemeral = options != null && Boolean.parseBoolean(options.getOrDefault("include_ephemeral", "false")); + String selectedKeyspace = options != null ? options.get("keyspace") : null; + String selectedTable = options != null ? options.get("table") : null; + String selectedSnapshotName = options != null ? options.get("snapshot") : null; + + return s -> { + if (selectedSnapshotName != null && !s.getTag().equals(selectedSnapshotName)) + return false; + + if (skipExpiring && s.isExpiring()) + return false; + + if (!includeEphemeral && s.isEphemeral()) + return false; + + if (selectedKeyspace != null && !s.getKeyspaceName().equals(selectedKeyspace)) + return false; + + return selectedTable == null || s.getTableName().equals(selectedTable); + }; + } + + @Override + public Map<String, TabularData> call() + { + List<TableSnapshot> filteredSnapshots; + + try + { + filteredSnapshots = new GetSnapshotsTask(snapshotManager, predicate, shouldRemoveIfNotExists).call(); + } + catch (Exception e) + { + logger.trace("Unable to get snapshots for listing purposes", e); + return Map.of(); + } + + Map<String, TabularData> snapshotMap = new HashMap<>(); + Set<String> tags = new HashSet<>(); + + for (TableSnapshot t : filteredSnapshots) + tags.add(t.getTag()); + + for (String tag : tags) + snapshotMap.put(tag, new TabularDataSupport(SnapshotDetailsTabularData.TABULAR_TYPE)); + + Map<String, Set<String>> keyspaceTables = new HashMap<>(); + for (TableSnapshot s : filteredSnapshots) + { + keyspaceTables.computeIfAbsent(s.getKeyspaceName(), ignore -> new HashSet<>()); + keyspaceTables.get(s.getKeyspaceName()).add(s.getTableName()); + } + + Map<String, Set<String>> cfsFiles = new HashMap<>(); + + for (Map.Entry<String, Set<String>> entry : keyspaceTables.entrySet()) + { + for (String table : entry.getValue()) + { + try + { + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(entry.getKey(), table); + if (cfs == null) + continue; + + cfsFiles.put(cfs.getKeyspaceName() + '.' + cfs.name, cfs.getFilesOfCfs()); + } + catch (Throwable t) + { + logger.debug("Unable to get all files of live SSTables for {}.{}", entry.getKey(), entry.getValue()); + } + } + } + + for (TableSnapshot snapshot : filteredSnapshots) + { + TabularDataSupport data = (TabularDataSupport) snapshotMap.get(snapshot.getTag()); + SnapshotDetailsTabularData.from(snapshot, data, cfsFiles.get(snapshot.getKeyspaceTable())); + } + + return snapshotMap; + } + + @Override + public String toString() + { + return "ListSnapshotsTask{" + + "options=" + options + + '}'; + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/SnapshotDetailsTabularData.java b/src/java/org/apache/cassandra/service/snapshot/SnapshotDetailsTabularData.java similarity index 65% rename from src/java/org/apache/cassandra/db/SnapshotDetailsTabularData.java rename to src/java/org/apache/cassandra/service/snapshot/SnapshotDetailsTabularData.java index c5debe43f4b9..23c84e505678 100644 --- a/src/java/org/apache/cassandra/db/SnapshotDetailsTabularData.java +++ b/src/java/org/apache/cassandra/service/snapshot/SnapshotDetailsTabularData.java @@ -15,33 +15,33 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.cassandra.db; +package org.apache.cassandra.service.snapshot; +import java.util.Set; import javax.management.openmbean.*; import org.apache.cassandra.io.util.FileUtils; -import org.apache.cassandra.service.snapshot.TableSnapshot; public class SnapshotDetailsTabularData { private static final String[] ITEM_NAMES = new String[]{"Snapshot name", - "Keyspace name", - "Column family name", - "True size", - "Size on disk", - "Creation time", - "Expiration time", - "Ephemeral"}; + "Keyspace name", + "Column family name", + "True size", + "Size on disk", + "Creation time", + "Expiration time", + "Ephemeral"}; private static final String[] ITEM_DESCS = new String[]{"snapshot_name", - "keyspace_name", - "columnfamily_name", - "TrueDiskSpaceUsed", - "TotalDiskSpaceUsed", - "created_at", - "expires_at", - "ephemeral"}; + "keyspace_name", + "columnfamily_name", + "TrueDiskSpaceUsed", + "TotalDiskSpaceUsed", + "created_at", + "expires_at", + "ephemeral"}; private static final String TYPE_NAME = "SnapshotDetails"; @@ -70,17 +70,18 @@ public class SnapshotDetailsTabularData } - public static void from(TableSnapshot details, TabularDataSupport result) + public static void from(TableSnapshot details, TabularDataSupport result, Set<String> files) { try { final String totalSize = FileUtils.stringifyFileSize(details.computeSizeOnDiskBytes()); + long trueSizeBytes = details.computeTrueSizeBytes(files); final String liveSize = FileUtils.stringifyFileSize(details.computeTrueSizeBytes()); String createdAt = safeToString(details.getCreatedAt()); String expiresAt = safeToString(details.getExpiresAt()); String ephemeral = Boolean.toString(details.isEphemeral()); result.put(new CompositeDataSupport(COMPOSITE_TYPE, ITEM_NAMES, - new Object[]{ details.getTag(), details.getKeyspaceName(), details.getTableName(), liveSize, totalSize, createdAt, expiresAt, ephemeral })); + new Object[]{ details.getTag(), details.getKeyspaceName(), details.getTableName(), liveSize, totalSize, createdAt, expiresAt, ephemeral })); } catch (OpenDataException e) { @@ -92,4 +93,4 @@ private static String safeToString(Object object) { return object == null ? null : object.toString(); } -} +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/service/snapshot/SnapshotException.java b/src/java/org/apache/cassandra/service/snapshot/SnapshotException.java new file mode 100644 index 000000000000..39cdf7ef77dd --- /dev/null +++ b/src/java/org/apache/cassandra/service/snapshot/SnapshotException.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.snapshot; + +public class SnapshotException extends RuntimeException +{ + public SnapshotException(String message) + { + super(message); + } + + public SnapshotException(String message, Throwable cause) + { + super(message, cause); + } +} diff --git a/src/java/org/apache/cassandra/service/snapshot/SnapshotLoader.java b/src/java/org/apache/cassandra/service/snapshot/SnapshotLoader.java index be1d6621bc9d..0e0ecc11c0f9 100644 --- a/src/java/org/apache/cassandra/service/snapshot/SnapshotLoader.java +++ b/src/java/org/apache/cassandra/service/snapshot/SnapshotLoader.java @@ -29,6 +29,7 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; import java.util.Set; import java.util.UUID; @@ -153,7 +154,7 @@ private void loadSnapshotFromDir(Matcher snapshotDirMatcher, Path snapshotDir) String tag = snapshotDirMatcher.group("tag"); String snapshotId = buildSnapshotId(keyspaceName, tableName, tableId, tag); TableSnapshot.Builder builder = snapshots.computeIfAbsent(snapshotId, k -> new TableSnapshot.Builder(keyspaceName, tableName, tableId, tag)); - builder.addSnapshotDir(new File(snapshotDir)); + builder.addSnapshotDir(new File(snapshotDir).toAbsolute()); } } @@ -184,7 +185,11 @@ public Set<TableSnapshot> loadSnapshots(String keyspace) } } - return snapshots.values().stream().map(TableSnapshot.Builder::build).collect(Collectors.toSet()); + Set<TableSnapshot> tableSnapshots = new HashSet<>(); + for (TableSnapshot.Builder snapshotBuilder : snapshots.values()) + tableSnapshots.add(snapshotBuilder.build()); + + return tableSnapshots; } public Set<TableSnapshot> loadSnapshots() diff --git a/src/java/org/apache/cassandra/service/snapshot/SnapshotManager.java b/src/java/org/apache/cassandra/service/snapshot/SnapshotManager.java index 3925f3f9dc7b..bea0c04669e6 100644 --- a/src/java/org/apache/cassandra/service/snapshot/SnapshotManager.java +++ b/src/java/org/apache/cassandra/service/snapshot/SnapshotManager.java @@ -17,81 +17,213 @@ */ package org.apache.cassandra.service.snapshot; - -import java.util.Collection; -import java.util.PriorityQueue; +import java.io.IOException; +import java.time.Instant; +import java.util.List; +import java.util.Map; +import java.util.Optional; import java.util.Set; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.function.Predicate; +import javax.management.openmbean.TabularData; +import com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.cassandra.concurrent.ScheduledExecutorPlus; import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.db.Directories; - -import java.util.concurrent.TimeoutException; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Joiner; - -import org.apache.cassandra.io.util.File; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.notifications.INotification; +import org.apache.cassandra.notifications.INotificationConsumer; +import org.apache.cassandra.notifications.TableDroppedNotification; +import org.apache.cassandra.notifications.TablePreScrubNotification; +import org.apache.cassandra.notifications.TruncationNotification; +import org.apache.cassandra.utils.Clock; import org.apache.cassandra.utils.ExecutorUtils; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.MBeanWrapper; -import static java.util.Comparator.comparing; -import static java.util.stream.Collectors.toList; +import static java.lang.String.format; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory; -import static org.apache.cassandra.utils.FBUtilities.now; +import static org.apache.cassandra.service.snapshot.ClearSnapshotTask.getClearSnapshotPredicate; +import static org.apache.cassandra.service.snapshot.ClearSnapshotTask.getPredicateForCleanedSnapshots; -public class SnapshotManager { +public class SnapshotManager implements SnapshotManagerMBean, INotificationConsumer, AutoCloseable +{ + private static final Logger logger = LoggerFactory.getLogger(SnapshotManager.class); - private static final ScheduledExecutorPlus executor = executorFactory().scheduled(false, "SnapshotCleanup"); + private ScheduledExecutorPlus snapshotCleanupExecutor; - private static final Logger logger = LoggerFactory.getLogger(SnapshotManager.class); + public static final SnapshotManager instance = new SnapshotManager(); private final long initialDelaySeconds; private final long cleanupPeriodSeconds; - private final SnapshotLoader snapshotLoader; - @VisibleForTesting - protected volatile ScheduledFuture<?> cleanupTaskFuture; + private volatile ScheduledFuture<?> cleanupTaskFuture; + + private final String[] dataDirs; + + private volatile boolean started = false; /** - * Expiring snapshots ordered by expiration date, to allow only iterating over snapshots - * that need to be removed on {@link this#clearExpiredSnapshots()} + * We read / list snapshots way more often than write / create them so COW is ideal to use here. + * This enables us to not submit listing tasks or tasks computing snapshot sizes to any executor's queue as they + * can be just run concurrently which gives way better throughput in case + * of excessive listing from clients (dashboards and similar) where snapshot metrics are gathered. */ - private final PriorityQueue<TableSnapshot> expiringSnapshots = new PriorityQueue<>(comparing(TableSnapshot::getExpiresAt)); + private final List<TableSnapshot> snapshots = new CopyOnWriteArrayList<>(); - public SnapshotManager() + private SnapshotManager() { this(CassandraRelevantProperties.SNAPSHOT_CLEANUP_INITIAL_DELAY_SECONDS.getInt(), - CassandraRelevantProperties.SNAPSHOT_CLEANUP_PERIOD_SECONDS.getInt()); + CassandraRelevantProperties.SNAPSHOT_CLEANUP_PERIOD_SECONDS.getInt(), + DatabaseDescriptor.getAllDataFileLocations()); } @VisibleForTesting - protected SnapshotManager(long initialDelaySeconds, long cleanupPeriodSeconds) + SnapshotManager(long initialDelaySeconds, long cleanupPeriodSeconds, String[] dataDirs) { this.initialDelaySeconds = initialDelaySeconds; this.cleanupPeriodSeconds = cleanupPeriodSeconds; - snapshotLoader = new SnapshotLoader(DatabaseDescriptor.getAllDataFileLocations()); + this.dataDirs = dataDirs; + this.snapshotCleanupExecutor = createSnapshotCleanupExecutor(); + } + + public void registerMBean() + { + MBeanWrapper.instance.registerMBean(this, MBEAN_NAME); + } + + public void unregisterMBean() + { + MBeanWrapper.instance.unregisterMBean(MBEAN_NAME); + } + + public synchronized SnapshotManager start(boolean runPeriodicSnapshotCleaner) + { + if (started) + return this; + + if (snapshotCleanupExecutor == null) + snapshotCleanupExecutor = createSnapshotCleanupExecutor(); + + executeTask(new ReloadSnapshotsTask(dataDirs)); + + if (runPeriodicSnapshotCleaner) + resumeSnapshotCleanup(); + + started = true; + return this; + } + + @Override + public synchronized void close() + { + if (!started) + return; + + pauseSnapshotCleanup(); + + shutdownAndWait(1, TimeUnit.MINUTES); + snapshots.clear(); + + started = false; + } + + public synchronized void shutdownAndWait(long timeout, TimeUnit unit) + { + try + { + ExecutorUtils.shutdownNowAndWait(timeout, unit, snapshotCleanupExecutor); + } + catch (InterruptedException | TimeoutException ex) + { + throw new RuntimeException(ex); + } + finally + { + snapshotCleanupExecutor = null; + } + } + + public synchronized void restart(boolean runPeriodicSnapshotCleaner) + { + if (!started) + return; + + logger.debug("Restarting SnapshotManager"); + close(); + start(runPeriodicSnapshotCleaner); + logger.debug("SnapshotManager restarted"); + } + + + public synchronized void restart() + { + restart(true); + } + + private static class ReloadSnapshotsTask extends AbstractSnapshotTask<Set<TableSnapshot>> + { + private final String[] dataDirs; + + public ReloadSnapshotsTask(String[] dataDirs) + { + super(null); + this.dataDirs = dataDirs; + } + + @Override + public Set<TableSnapshot> call() + { + Set<TableSnapshot> tableSnapshots = new SnapshotLoader(dataDirs).loadSnapshots(); + new ClearSnapshotTask(SnapshotManager.instance, snapshot -> true, false).call(); + for (TableSnapshot snapshot : tableSnapshots) + SnapshotManager.instance.addSnapshot(snapshot); + + return tableSnapshots; + } + + @Override + public SnapshotTaskType getTaskType() + { + return SnapshotTaskType.RELOAD; + } + } + + void addSnapshot(TableSnapshot snapshot) + { + logger.debug("Adding snapshot {}", snapshot); + snapshots.add(snapshot); } - public Collection<TableSnapshot> getExpiringSnapshots() + List<TableSnapshot> getSnapshots() { - return expiringSnapshots; + return snapshots; } - public synchronized void start() + public void resumeSnapshotCleanup() { - addSnapshots(loadSnapshots()); - resumeSnapshotCleanup(); + if (cleanupTaskFuture == null) + { + logger.info("Scheduling expired snapshots cleanup with initialDelaySeconds={} and cleanupPeriodSeconds={}", + initialDelaySeconds, cleanupPeriodSeconds); + + cleanupTaskFuture = snapshotCleanupExecutor.scheduleWithFixedDelay(SnapshotManager.instance::clearExpiredSnapshots, + initialDelaySeconds, + cleanupPeriodSeconds, + SECONDS); + } } - public synchronized void stop() throws InterruptedException, TimeoutException + private void pauseSnapshotCleanup() { - expiringSnapshots.clear(); if (cleanupTaskFuture != null) { cleanupTaskFuture.cancel(false); @@ -99,74 +231,350 @@ public synchronized void stop() throws InterruptedException, TimeoutException } } - public synchronized void addSnapshot(TableSnapshot snapshot) + /** + * Deletes snapshot and removes it from manager. + * + * @param snapshot snapshot to clear + */ + void clearSnapshot(TableSnapshot snapshot) + { + executeTask(new ClearSnapshotTask(this, s -> s.equals(snapshot), true)); + } + + /** + * Returns list of snapshots of given keyspace + * + * @param keyspace keyspace of a snapshot + * @return list of snapshots of given keyspace. + */ + public List<TableSnapshot> getSnapshots(String keyspace) { - // We currently only care about expiring snapshots - if (snapshot.isExpiring()) + return getSnapshots(snapshot -> snapshot.getKeyspaceName().equals(keyspace)); + } + + /** + * Return snapshots based on given parameters. + * + * @param skipExpiring if expiring snapshots should be skipped + * @param includeEphemeral if ephemeral snapshots should be included + * @return snapshots based on given parameters + */ + public List<TableSnapshot> getSnapshots(boolean skipExpiring, boolean includeEphemeral) + { + return getSnapshots(s -> (!skipExpiring || !s.isExpiring()) && (includeEphemeral || !s.isEphemeral())); + } + + /** + * Returns all snapshots passing the given predicate. + * + * @param predicate predicate to filter all snapshots of + * @return list of snapshots passing the predicate + */ + public List<TableSnapshot> getSnapshots(Predicate<TableSnapshot> predicate) + { + return new GetSnapshotsTask(this, predicate, true).call(); + } + + /** + * Returns a snapshot or empty optional based on the given parameters. + * + * @param keyspace keyspace of a snapshot + * @param table table of a snapshot + * @param tag name of a snapshot + * @return empty optional if there is not such snapshot, non-empty otherwise + */ + public Optional<TableSnapshot> getSnapshot(String keyspace, String table, String tag) + { + List<TableSnapshot> foundSnapshots = new GetSnapshotsTask(this, + snapshot -> snapshot.getKeyspaceName().equals(keyspace) && + snapshot.getTableName().equals(table) && + snapshot.getTag().equals(tag) || (tag != null && tag.isEmpty()), + true).call(); + + if (foundSnapshots.isEmpty()) + return Optional.empty(); + else + return Optional.of(foundSnapshots.get(0)); + } + + /** + * Checks whether a snapshot for given keyspace and table exists of a given name exists. + * + * @param keyspace keyspace to get a snapshot of + * @param table table to get a snapshot of + * @param tag name of a snapshot + * @return true of a snapshot of given properties exist, false otherwise + */ + public boolean exists(String keyspace, String table, String tag) + { + return getSnapshot(keyspace, table, tag).isPresent(); + } + + /** + * Checks whether a snapshot which satisfies given predicate exists. + * + * @param predicate predicate to check the existence of a snapshot + * @return true if a snapshot which satisfies a predicate exists, false otherwise + */ + public boolean exists(Predicate<TableSnapshot> predicate) + { + return !getSnapshots(predicate).isEmpty(); + } + + /** + * Clear snapshots of given tag from given keyspace. Does not remove ephemeral snapshots. + * <p> + * + * @param tag snapshot name + * @param keyspace keyspace to clear all snapshots of a given tag of + */ + public void clearSnapshots(String tag, String keyspace) + { + clearSnapshots(tag, Set.of(keyspace), Clock.Global.currentTimeMillis()); + } + + /** + * Removes a snapshot. + * <p> + * + * @param keyspace keyspace of a snapshot to remove + * @param table table of a snapshot to remove + * @param tag name of a snapshot to remove. + */ + public void clearSnapshot(String keyspace, String table, String tag) + { + executeTask(new ClearSnapshotTask(this, + snapshot -> snapshot.getKeyspaceName().equals(keyspace) + && snapshot.getTableName().equals(table) + && snapshot.getTag().equals(tag), + true)); + } + + /** + * Removes all snapshots for given keyspace and table. + * + * @param keyspace keyspace to remove snapshots for + * @param table table in a given keyspace to remove snapshots for + */ + public void clearAllSnapshots(String keyspace, String table) + { + executeTask(new ClearSnapshotTask(this, + snapshot -> snapshot.getKeyspaceName().equals(keyspace) + && snapshot.getTableName().equals(table), + true)); + } + + /** + * Clears all snapshots, expiring and ephemeral as well. + */ + public void clearAllSnapshots() + { + executeTask(new ClearSnapshotTask(this, snapshot -> true, true)); + } + + /** + * Clear snapshots based on a given predicate + * + * @param predicate predicate to filter snapshots on + */ + public void clearSnapshot(Predicate<TableSnapshot> predicate) + { + executeTask(new ClearSnapshotTask(this, predicate, true)); + } + + /** + * Clears all ephemeral snapshots in a node. + */ + public void clearEphemeralSnapshots() + { + executeTask(new ClearSnapshotTask(this, TableSnapshot::isEphemeral, true)); + } + + /** + * Clears all expired snapshots in a node. + */ + public void clearExpiredSnapshots() + { + Instant now = FBUtilities.now(); + executeTask(new ClearSnapshotTask(this, s -> s.isExpired(now), true)); + } + + /** + * Clear snapshots of given tag from given keyspaces. + * <p> + * If tag is not present / is empty, all snapshots are considered to be cleared. + * If keyspaces are empty, all snapshots of given tag and older than maxCreatedAt are removed. + * + * @param tag optional tag of snapshot to clear + * @param keyspaces keyspaces to remove snapshots for + * @param maxCreatedAt clear all such snapshots which were created before this timestamp + */ + private void clearSnapshots(String tag, Set<String> keyspaces, long maxCreatedAt) + { + executeTask(new ClearSnapshotTask(this, getClearSnapshotPredicate(tag, keyspaces, maxCreatedAt, false), true)); + } + + public List<TableSnapshot> takeSnapshot(SnapshotOptions options) + { + return executeTask(new TakeSnapshotTask(this, options)); + } + + // Super methods + + @Override + public void takeSnapshot(String tag, String... entities) + { + takeSnapshot(SnapshotOptions.userSnapshot(tag, Map.of(), entities)); + } + + @Override + public void takeSnapshot(String tag, Map<String, String> optMap, String... entities) throws IOException + { + try { - logger.debug("Adding expiring snapshot {}", snapshot); - expiringSnapshots.add(snapshot); + takeSnapshot(SnapshotOptions.userSnapshot(tag, optMap, entities)); + } + catch (SnapshotException ex) + { + // to be compatible with deprecated methods in StorageService + throw new IOException(ex); } } - public synchronized Set<TableSnapshot> loadSnapshots(String keyspace) + @Override + public void clearSnapshot(String tag, Map<String, Object> options, String... keyspaceNames) { - return snapshotLoader.loadSnapshots(keyspace); + executeTask(new ClearSnapshotTask(this, getPredicateForCleanedSnapshots(tag, options, keyspaceNames), true)); } - public synchronized Set<TableSnapshot> loadSnapshots() + @Override + public Map<String, TabularData> listSnapshots(Map<String, String> options) { - return snapshotLoader.loadSnapshots(); + return new ListSnapshotsTask(this, options, true).call(); } - @VisibleForTesting - protected synchronized void addSnapshots(Collection<TableSnapshot> snapshots) + @Override + public long getTrueSnapshotSize() { - logger.debug("Adding snapshots: {}.", Joiner.on(", ").join(snapshots.stream().map(TableSnapshot::getId).collect(toList()))); - snapshots.forEach(this::addSnapshot); + return new TrueSnapshotSizeTask(this, s -> true).call(); } - // TODO: Support pausing snapshot cleanup - @VisibleForTesting - synchronized void resumeSnapshotCleanup() + @Override + public long getTrueSnapshotsSize(String keyspace) { - if (cleanupTaskFuture == null) + return new TrueSnapshotSizeTask(this, s -> s.getKeyspaceName().equals(keyspace)).call(); + } + + @Override + public long getTrueSnapshotsSize(String keyspace, String table) + { + return new TrueSnapshotSizeTask(this, s -> s.getKeyspaceName().equals(keyspace) && s.getTableName().equals(table)).call(); + } + + @Override + public void setSnapshotLinksPerSecond(long throttle) + { + logger.info("Setting snapshot throttle to {}", throttle); + DatabaseDescriptor.setSnapshotLinksPerSecond(throttle); + } + + @Override + public long getSnapshotLinksPerSecond() + { + return DatabaseDescriptor.getSnapshotLinksPerSecond(); + } + + @Override + public void handleNotification(INotification notification, Object sender) + { + if (notification instanceof TruncationNotification) { - logger.info("Scheduling expired snapshot cleanup with initialDelaySeconds={} and cleanupPeriodSeconds={}", - initialDelaySeconds, cleanupPeriodSeconds); - cleanupTaskFuture = executor.scheduleWithFixedDelay(this::clearExpiredSnapshots, initialDelaySeconds, - cleanupPeriodSeconds, TimeUnit.SECONDS); + TruncationNotification truncationNotification = (TruncationNotification) notification; + ColumnFamilyStore cfs = truncationNotification.cfs; + if (!truncationNotification.disableSnapshot && cfs.isAutoSnapshotEnabled()) + { + SnapshotOptions opts = SnapshotOptions.systemSnapshot(cfs.name, SnapshotType.TRUNCATE, cfs.getKeyspaceTableName()) + .ttl(truncationNotification.ttl).build(); + takeSnapshot(opts); + } + } + else if (notification instanceof TableDroppedNotification) + { + TableDroppedNotification tableDroppedNotification = (TableDroppedNotification) notification; + ColumnFamilyStore cfs = tableDroppedNotification.cfs; + if (cfs.isAutoSnapshotEnabled()) + { + SnapshotOptions opts = SnapshotOptions.systemSnapshot(cfs.name, SnapshotType.DROP, cfs.getKeyspaceTableName()) + .cfs(cfs).ttl(tableDroppedNotification.ttl).build(); + takeSnapshot(opts); + } + } + else if (notification instanceof TablePreScrubNotification) + { + ColumnFamilyStore cfs = ((TablePreScrubNotification) notification).cfs; + SnapshotOptions opts = SnapshotOptions.systemSnapshot(cfs.name, SnapshotType.PRE_SCRUB, cfs.getKeyspaceTableName()).build(); + takeSnapshot(opts); } } @VisibleForTesting - protected synchronized void clearExpiredSnapshots() + List<TableSnapshot> executeTask(TakeSnapshotTask task) { - TableSnapshot expiredSnapshot; - while ((expiredSnapshot = expiringSnapshots.peek()) != null) + try { - if (!expiredSnapshot.isExpired(now())) - break; // the earliest expiring snapshot is not expired yet, so there is no more expired snapshots to remove + prePopulateSnapshots(task); + return task.call(); + } + catch (Throwable t) + { + throw new SnapshotException(String.format("Exception occured while executing %s: %s", task.toString(), t.getMessage()), t); + } + } - logger.debug("Removing expired snapshot {}.", expiredSnapshot); - clearSnapshot(expiredSnapshot); + @VisibleForTesting + <T> T executeTask(AbstractSnapshotTask<T> task) + { + try + { + return task.call(); + } + catch (Throwable t) + { + throw new SnapshotException(String.format("Exception occured while executing %s", task.toString()), t); } } /** - * Deletes snapshot and remove it from manager + * Add table snapshots to snaphots cow list in advance in order to be sure that the snapshots to be created + * are not existing already. "executeTask" method can be invoked by multiple threads, and they hit + * this synchronized method which populates snapshots so next thread will fail to advance (exception is thrown) + * if table snapshots of some other tasks are already present in snapshots cow list. + * + * Added snapshots to snapshot list are at this point "incomplete", they will not appear in listing output + * until they are indeed taken and exist on disk, otherwise we would see "phantom" snapshots in listings, + * they would be present but in fact they are just going through the process of being taken. + * + * @param task task to process */ - public synchronized void clearSnapshot(TableSnapshot snapshot) + private synchronized void prePopulateSnapshots(TakeSnapshotTask task) { - for (File snapshotDir : snapshot.getDirectories()) - Directories.removeSnapshotDirectory(DatabaseDescriptor.getSnapshotRateLimiter(), snapshotDir); + Map<ColumnFamilyStore, TableSnapshot> snapshotsToCreate = task.getSnapshotsToCreate(); + for (Map.Entry<ColumnFamilyStore, TableSnapshot> toCreateEntry : snapshotsToCreate.entrySet()) + { + if (snapshots.contains(toCreateEntry.getValue())) + { + throw new RuntimeException(format("Snapshot %s for %s.%s already exists.", + toCreateEntry.getValue().getTag(), + toCreateEntry.getValue().getKeyspaceName(), + toCreateEntry.getValue().getTableName())); + } + } - expiringSnapshots.remove(snapshot); + snapshots.addAll(snapshotsToCreate.values()); } - @VisibleForTesting - public static void shutdownAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException + private static ScheduledExecutorPlus createSnapshotCleanupExecutor() { - ExecutorUtils.shutdownNowAndWait(timeout, unit, executor); + return executorFactory().scheduled(false, "SnapshotCleanup"); } -} +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/service/snapshot/SnapshotManagerMBean.java b/src/java/org/apache/cassandra/service/snapshot/SnapshotManagerMBean.java new file mode 100644 index 000000000000..2d69a7ca97aa --- /dev/null +++ b/src/java/org/apache/cassandra/service/snapshot/SnapshotManagerMBean.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.snapshot; + +import java.io.IOException; +import java.util.Map; +import javax.management.openmbean.TabularData; + +public interface SnapshotManagerMBean +{ + String MBEAN_NAME = "org.apache.cassandra.service.snapshot:type=SnapshotManager"; + + /** + * Takes the snapshot of a multiple column family from different keyspaces. A snapshot name must be specified. + * + * @param tag the tag given to the snapshot; may not be null or empty + * @param options Map of options (skipFlush is the only supported option for now) + * @param entities list of keyspaces / tables in the form of empty | ks1 ks2 ... | ks1.cf1,ks2.cf2,... + */ + void takeSnapshot(String tag, Map<String, String> options, String... entities) throws IOException; + + /** + * Takes the snapshot of a multiple column family from different keyspaces. A snapshot name must be specified. + * + * @param tag the tag given to the snapshot; may not be null or empty + * @param entities list of keyspaces / tables in the form of empty | ks1 ks2 ... | ks1.cf1,ks2.cf2,... + */ + void takeSnapshot(String tag, String... entities) throws IOException; + + /** + * Remove the snapshot with the given name from the given keyspaces. + * If no tag is specified we will remove all snapshots. + * + * @param tag name of snapshot to clear, if null or empty string, + * all snapshots of given keyspace will be cleared + * @param options map of options for cleanup operation, consult nodetool's ClearSnapshot + * @param keyspaceNames name of keyspaces to clear snapshots for + */ + void clearSnapshot(String tag, Map<String, Object> options, String... keyspaceNames) throws IOException; + + /** + * Get the details of all the snapshots + * + * @param options map of options used for filtering of snapshots + * @return A map of snapshotName to all its details in Tabular form. + */ + Map<String, TabularData> listSnapshots(Map<String, String> options); + + /** + * Get the true size taken by all snapshots across all keyspaces. + * + * @return True size taken by all the snapshots. + */ + long getTrueSnapshotSize(); + + /** + * Get the true size take by all snapshots in given keyspace. + * + * @param keyspace keyspace to get true size of all snapshots of + * @return true size of all snapshots in given keyspace + */ + long getTrueSnapshotsSize(String keyspace); + + /** + * Get the true size take by all snapshots in given keyspace and table. + * + * @param keyspace keyspace to get true size of all snapshots of + * @param table table in a keyspace to get true size of all snapshots of + * @return true size of all snapshots in given keyspace and table + */ + long getTrueSnapshotsSize(String keyspace, String table); + + /** + * Set the current hardlink-per-second throttle for snapshots + * A setting of zero indicates no throttling + * + * @param throttle hard-links-per-second + */ + void setSnapshotLinksPerSecond(long throttle); + + /** + * Get the current hardlink-per-second throttle for snapshots + * A setting of zero indicates no throttling. + * + * @return snapshot links-per-second throttle + */ + long getSnapshotLinksPerSecond(); + + /** + * Restarting means that snapshots will be reloaded from disk. + */ + void restart(); +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/service/snapshot/SnapshotManifest.java b/src/java/org/apache/cassandra/service/snapshot/SnapshotManifest.java index 8ee737cf82e0..6f23eba84d34 100644 --- a/src/java/org/apache/cassandra/service/snapshot/SnapshotManifest.java +++ b/src/java/org/apache/cassandra/service/snapshot/SnapshotManifest.java @@ -63,7 +63,7 @@ public SnapshotManifest(List<String> files, DurationSpec.IntSecondsBound ttl, In { this.files = files; this.createdAt = creationTime; - this.expiresAt = ttl == null ? null : createdAt.plusSeconds(ttl.toSeconds()); + this.expiresAt = computeExpiration(ttl, creationTime); this.ephemeral = ephemeral; } @@ -97,6 +97,11 @@ public static SnapshotManifest deserializeFromJsonFile(File file) throws IOExcep return JsonUtils.deserializeFromJsonFile(SnapshotManifest.class, file); } + public static Instant computeExpiration(DurationSpec.IntSecondsBound ttl, Instant createdAt) + { + return ttl == null ? null : createdAt.plusSeconds(ttl.toSeconds()); + } + @Override public boolean equals(Object o) { diff --git a/src/java/org/apache/cassandra/service/snapshot/SnapshotOptions.java b/src/java/org/apache/cassandra/service/snapshot/SnapshotOptions.java new file mode 100644 index 000000000000..409288f8bf17 --- /dev/null +++ b/src/java/org/apache/cassandra/service/snapshot/SnapshotOptions.java @@ -0,0 +1,235 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.snapshot; + +import java.time.Instant; +import java.util.Arrays; +import java.util.Collections; +import java.util.Map; +import java.util.function.Predicate; + +import com.google.common.util.concurrent.RateLimiter; +import org.apache.commons.lang3.StringUtils; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.DurationSpec; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.io.sstable.format.SSTableReader; + +import static java.lang.String.format; + +public class SnapshotOptions +{ + public static final String SKIP_FLUSH = "skipFlush"; + public static final String TTL = "ttl"; + public final SnapshotType type; + public final String tag; + public final DurationSpec.IntSecondsBound ttl; + public final Instant creationTime; + public final boolean skipFlush; + public final boolean ephemeral; + public final String[] entities; + public final RateLimiter rateLimiter; + public final Predicate<SSTableReader> sstableFilter; + public final ColumnFamilyStore cfs; + + private SnapshotOptions(SnapshotType type, + String tag, + DurationSpec.IntSecondsBound ttl, + Instant creationTime, + boolean skipFlush, + boolean ephemeral, + String[] entities, + RateLimiter rateLimiter, + Predicate<SSTableReader> sstableFilter, + ColumnFamilyStore cfs) + { + this.type = type; + this.tag = tag; + this.ttl = ttl; + this.creationTime = creationTime; + this.skipFlush = skipFlush; + this.ephemeral = ephemeral; + this.entities = entities; + this.rateLimiter = rateLimiter; + this.sstableFilter = sstableFilter; + this.cfs = cfs; + } + + public static Builder systemSnapshot(String tag, SnapshotType type, String... entities) + { + return new Builder(tag, type, ssTableReader -> true, entities); + } + + public static Builder systemSnapshot(String tag, SnapshotType type, Predicate<SSTableReader> sstableFilter, String... entities) + { + return new Builder(tag, type, sstableFilter, entities); + } + + public static SnapshotOptions userSnapshot(String tag, String... entities) + { + return userSnapshot(tag, Collections.emptyMap(), entities); + } + + public static SnapshotOptions userSnapshot(String tag, Map<String, String> options, String... entities) + { + Builder builder = new Builder(tag, SnapshotType.USER, ssTableReader -> true, entities).ttl(options.get(TTL)); + if (Boolean.parseBoolean(options.getOrDefault(SKIP_FLUSH, Boolean.FALSE.toString()))) + builder.skipFlush(); + return builder.build(); + } + + public String getSnapshotName(Instant creationTime) + { + // Diagnostic snapshots have very specific naming convention hence we are keeping it. + // Repair snapshots rely on snapshots having name of their repair session ids + if (type == SnapshotType.USER || type == SnapshotType.DIAGNOSTICS || type == SnapshotType.REPAIR) + return tag; + // System snapshots have the creation timestamp on the name + String snapshotName = format("%d-%s", creationTime.toEpochMilli(), type.label); + if (StringUtils.isNotBlank(tag)) + snapshotName = snapshotName + '-' + tag; + return snapshotName; + } + + public static class Builder + { + private final String tag; + private final String[] entities; + private DurationSpec.IntSecondsBound ttl; + private Instant creationTime; + private boolean skipFlush = false; + private boolean ephemeral = false; + private ColumnFamilyStore cfs; + private final Predicate<SSTableReader> sstableFilter; + private final SnapshotType type; + private RateLimiter rateLimiter; + + public Builder(String tag, SnapshotType type, Predicate<SSTableReader> sstableFilter, String... entities) + { + this.tag = tag; + this.type = type; + this.entities = entities; + this.sstableFilter = sstableFilter; + } + + public Builder ttl(String ttl) + { + if (ttl != null) + this.ttl = new DurationSpec.IntSecondsBound(ttl); + + return this; + } + + public Builder ttl(DurationSpec.IntSecondsBound ttl) + { + this.ttl = ttl; + return this; + } + + public Builder creationTime(String creationTime) + { + if (creationTime != null) + { + try + { + return creationTime(Long.parseLong(creationTime)); + } + catch (Exception ex) + { + throw new RuntimeException("Unable to parse creation time from " + creationTime); + } + } + + return this; + } + + public Builder creationTime(Instant creationTime) + { + this.creationTime = creationTime; + return this; + } + + public Builder creationTime(long creationTime) + { + return creationTime(Instant.ofEpochMilli(creationTime)); + } + + public Builder skipFlush() + { + skipFlush = true; + return this; + } + + public Builder ephemeral() + { + ephemeral = true; + return this; + } + + public Builder cfs(ColumnFamilyStore cfs) + { + this.cfs = cfs; + return this; + } + + public Builder rateLimiter(RateLimiter rateLimiter) + { + this.rateLimiter = rateLimiter; + return this; + } + + public SnapshotOptions build() + { + if (tag == null || tag.isEmpty()) + throw new RuntimeException("You must supply a snapshot name."); + + if (ttl != null) + { + int minAllowedTtlSecs = CassandraRelevantProperties.SNAPSHOT_MIN_ALLOWED_TTL_SECONDS.getInt(); + if (ttl.toSeconds() < minAllowedTtlSecs) + throw new IllegalArgumentException(format("ttl for snapshot must be at least %d seconds", minAllowedTtlSecs)); + } + + if (ephemeral && ttl != null) + throw new IllegalStateException(format("can not take ephemeral snapshot (%s) while ttl is specified too", tag)); + + if (rateLimiter == null) + rateLimiter = DatabaseDescriptor.getSnapshotRateLimiter(); + + return new SnapshotOptions(type, tag, ttl, creationTime, skipFlush, ephemeral, entities, rateLimiter, + sstableFilter, cfs); + } + } + + @Override + public String toString() + { + return "CreateSnapshotOptions{" + + "type=" + type + + ", tag='" + tag + '\'' + + ", ttl=" + ttl + + ", creationTime=" + creationTime + + ", skipFlush=" + skipFlush + + ", ephemeral=" + ephemeral + + ", entities=" + Arrays.toString(entities) + + '}'; + } +} diff --git a/src/java/org/apache/cassandra/service/snapshot/SnapshotType.java b/src/java/org/apache/cassandra/service/snapshot/SnapshotType.java new file mode 100644 index 000000000000..67beb091c176 --- /dev/null +++ b/src/java/org/apache/cassandra/service/snapshot/SnapshotType.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.snapshot; + +public enum SnapshotType +{ + USER("user"), + + TRUNCATE("truncated"), + DROP("dropped"), + PRE_SCRUB("pre-scrub"), + COMPACT("compact"), + UPGRADE("upgrade"), + DIAGNOSTICS("diagnostics"), + + REPAIR("repair"), + + MISC("misc"); + + public final String label; + + SnapshotType(String label) + { + this.label = label; + } +} diff --git a/src/java/org/apache/cassandra/service/snapshot/TableSnapshot.java b/src/java/org/apache/cassandra/service/snapshot/TableSnapshot.java index d10092a41af8..94bcd07d9a87 100644 --- a/src/java/org/apache/cassandra/service/snapshot/TableSnapshot.java +++ b/src/java/org/apache/cassandra/service/snapshot/TableSnapshot.java @@ -21,21 +21,33 @@ import java.nio.file.Files; import java.nio.file.Path; import java.time.Instant; +import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; import java.util.HashSet; +import java.util.List; +import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.Set; import java.util.UUID; -import java.util.function.Predicate; +import java.util.stream.Stream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Directories; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.lifecycle.SSTableSet; +import org.apache.cassandra.io.FSReadError; +import org.apache.cassandra.io.sstable.Component; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.SSTableId; +import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.FileUtils; -import org.apache.cassandra.utils.DirectorySizeCalculator; +import org.apache.cassandra.utils.Throwables; +import org.apache.cassandra.utils.concurrent.Refs; public class TableSnapshot { @@ -43,6 +55,7 @@ public class TableSnapshot private final String keyspaceName; private final String tableName; + private final String keyspaceTable; private final UUID tableId; private final String tag; private final boolean ephemeral; @@ -52,25 +65,51 @@ public class TableSnapshot private final Set<File> snapshotDirs; + private volatile long sizeOnDisk = 0; + + private volatile long manifestsSize; + private volatile long schemasSize; + + private volatile boolean inProgress = false; + public TableSnapshot(String keyspaceName, String tableName, UUID tableId, String tag, Instant createdAt, Instant expiresAt, Set<File> snapshotDirs, boolean ephemeral) { this.keyspaceName = keyspaceName; this.tableName = tableName; + this.keyspaceTable = keyspaceName + '.' + tableName; this.tableId = tableId; this.tag = tag; this.createdAt = createdAt; this.expiresAt = expiresAt; this.snapshotDirs = snapshotDirs; this.ephemeral = ephemeral; + + manifestsSize = getManifestsSize(); + schemasSize = getSchemasSize(); + } + + public boolean isCompleted() + { + return !inProgress; + } + + public void incomplete() + { + inProgress = true; + } + + public void complete() + { + inProgress = false; } /** * Unique identifier of a snapshot. Used * only to deduplicate snapshots internally, * not exposed externally. - * + * <p> * Format: "$ks:$table_name:$table_id:$tag" */ public String getId() @@ -88,6 +127,11 @@ public String getTableName() return tableName; } + public String getKeyspaceTable() + { + return keyspaceTable; + } + public String getTag() { return tag; @@ -95,13 +139,22 @@ public String getTag() public Instant getCreatedAt() { + if (createdAt == null) { - long minCreation = snapshotDirs.stream().mapToLong(File::lastModified).min().orElse(0); - if (minCreation != 0) + long minCreation = 0; + for (File snapshotDir : snapshotDirs) { - return Instant.ofEpochMilli(minCreation); + long lastModified = snapshotDir.lastModified(); + if (lastModified == 0) + continue; + + if (minCreation == 0 || minCreation > lastModified) + minCreation = lastModified; } + + if (minCreation != 0) + return Instant.ofEpochMilli(minCreation); } return createdAt; } @@ -123,7 +176,11 @@ public boolean isExpired(Instant now) public boolean exists() { - return snapshotDirs.stream().anyMatch(File::exists); + for (File snapshotDir : snapshotDirs) + if (snapshotDir.exists()) + return true; + + return false; } public boolean isEphemeral() @@ -138,26 +195,106 @@ public boolean isExpiring() public long computeSizeOnDiskBytes() { - return snapshotDirs.stream().mapToLong(FileUtils::folderSize).sum(); + long sum = sizeOnDisk; + if (sum == 0) + { + for (File snapshotDir : snapshotDirs) + sum += FileUtils.folderSize(snapshotDir); + + sizeOnDisk = sum; + } + + return sum; } public long computeTrueSizeBytes() { - DirectorySizeCalculator visitor = new SnapshotTrueSizeCalculator(); + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(keyspaceName, tableName); + if (cfs == null) + return 0; - for (File snapshotDir : snapshotDirs) + return computeTrueSizeBytes(cfs.getFilesOfCfs()); + } + + public long computeTrueSizeBytes(Set<String> files) + { + long size = manifestsSize + schemasSize; + + for (File dataPath : getDirectories()) { - try - { - Files.walkFileTree(snapshotDir.toPath(), visitor); - } - catch (IOException e) + List<Path> snapshotFiles = listDir(dataPath.toPath()); + for (Path snapshotFile : snapshotFiles) { - logger.error("Could not calculate the size of {}.", snapshotDir, e); + if (!snapshotFile.endsWith("manifest.json") && !snapshotFile.endsWith("schema.cql")) + { + // files == null means that the underlying table was most probably dropped + // so in that case we indeed go to count snapshots file in for true size + if (files == null || (!files.contains(getLiveFileFromSnapshotFile(snapshotFile)))) + size += getFileSize(snapshotFile); + } } } - return visitor.getAllocatedSize(); + return size; + } + + private List<Path> listDir(Path dir) + { + List<Path> paths = new ArrayList<>(); + try (Stream<Path> stream = Files.list(dir)) + { + stream.forEach(p -> { + if (p.getFileName().toString().startsWith(".")) + { + paths.addAll(listDir(p)); + } + else + { + paths.add(p); + } + }); + } + catch (Throwable t) + { + logger.error("Could not list directory content {}", dir, t); + } + + return paths; + } + + private long getFileSize(Path file) + { + try + { + return Files.size(file); + } + catch (Throwable t) + { + return 0; + } + } + + /** + * Returns the corresponding live file for a given snapshot file. + * <p> + * Example: + * - Base table: + * - Snapshot file: ~/.ccm/test/node1/data0/test_ks/tbl-e03faca0813211eca100c705ea09b5ef/snapshots/1643481737850/me-1-big-Data.db + * - Live file: ~/.ccm/test/node1/data0/test_ks/tbl-e03faca0813211eca100c705ea09b5ef/me-1-big-Data.db + * - Secondary index: + * - Snapshot file: ~/.ccm/test/node1/data0/test_ks/tbl-e03faca0813211eca100c705ea09b5ef/snapshots/1643481737850/.tbl_val_idx/me-1-big-Summary.db + * - Live file: ~/.ccm/test/node1/data0/test_ks/tbl-e03faca0813211eca100c705ea09b5ef/.tbl_val_idx/me-1-big-Summary.db + */ + static String getLiveFileFromSnapshotFile(Path snapshotFilePath) + { + // Snapshot directory structure format is {data_dir}/snapshots/{snapshot_name}/{snapshot_file} + Path liveDir = snapshotFilePath.getParent().getParent().getParent(); + if (Directories.isSecondaryIndexFolder(snapshotFilePath.getParent())) + { + // Snapshot file structure format is {data_dir}/snapshots/{snapshot_name}/.{index}/{sstable-component}.db + liveDir = File.getPath(liveDir.getParent().toString(), snapshotFilePath.getParent().getFileName().toString()); + } + return liveDir.resolve(snapshotFilePath.getFileName().toString()).toAbsolutePath().toString(); } public Collection<File> getDirectories() @@ -165,48 +302,88 @@ public Collection<File> getDirectories() return snapshotDirs; } - public Optional<File> getManifestFile() + /** + * Returns all manifest files of a snapshot. + * <p> + * In practice, there might be multiple manifest files, as many as we have snapshot dirs. + * Each snapshot dir will hold its view of a snapshot, containing only sstables located in such snapshot dir. + * + * @return all manifest files + */ + public Set<File> getManifestFiles() { + Set<File> manifestFiles = new HashSet<>(); for (File snapshotDir : snapshotDirs) { File manifestFile = Directories.getSnapshotManifestFile(snapshotDir); if (manifestFile.exists()) - { - return Optional.of(manifestFile); - } + manifestFiles.add(manifestFile); } - return Optional.empty(); + return manifestFiles; } - public Optional<File> getSchemaFile() + public boolean hasManifest() { + for (File snapshotDir : snapshotDirs) + { + if (Directories.getSnapshotManifestFile(snapshotDir).exists()) + return true; + } + return false; + } + + /** + * Returns all schemas files of a snapshot. + * + * @return all schema files + */ + public Set<File> getSchemaFiles() + { + Set<File> schemaFiles = new HashSet<>(); for (File snapshotDir : snapshotDirs) { File schemaFile = Directories.getSnapshotSchemaFile(snapshotDir); if (schemaFile.exists()) - { - return Optional.of(schemaFile); - } + schemaFiles.add(schemaFile); } - return Optional.empty(); + return schemaFiles; } + public long getManifestsSize() + { + long size = 0; + for (File manifestFile : getManifestFiles()) + size += manifestFile.length(); + + return size; + } + + public long getSchemasSize() + { + long size = 0; + for (File schemaFile : getSchemaFiles()) + size += schemaFile.length(); + + return size; + } + + @Override public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; TableSnapshot snapshot = (TableSnapshot) o; - return Objects.equals(keyspaceName, snapshot.keyspaceName) && Objects.equals(tableName, snapshot.tableName) && - Objects.equals(tableId, snapshot.tableId) && Objects.equals(tag, snapshot.tag) && - Objects.equals(createdAt, snapshot.createdAt) && Objects.equals(expiresAt, snapshot.expiresAt) && - Objects.equals(snapshotDirs, snapshot.snapshotDirs) && Objects.equals(ephemeral, snapshot.ephemeral); + return Objects.equals(keyspaceName, snapshot.keyspaceName) && + Objects.equals(tableName, snapshot.tableName) && + Objects.equals(tableId, snapshot.tableId) && + Objects.equals(tag, snapshot.tag); } @Override public int hashCode() { - return Objects.hash(keyspaceName, tableName, tableId, tag, createdAt, expiresAt, snapshotDirs, ephemeral); + return Objects.hash(keyspaceName, tableName, tableId, tag); } @Override @@ -224,7 +401,16 @@ public String toString() '}'; } - static class Builder { + public void updateMetadataSize() + { + if (manifestsSize == 0) + manifestsSize = getManifestsSize(); + if (schemasSize == 0) + schemasSize = getSchemasSize(); + } + + static class Builder + { private final String keyspaceName; private final String tableName; private final UUID tableId; @@ -287,66 +473,82 @@ protected static String buildSnapshotId(String keyspaceName, String tableName, U return String.format("%s:%s:%s:%s", keyspaceName, tableName, tableId, tag); } - public static class SnapshotTrueSizeCalculator extends DirectorySizeCalculator + public static Set<Descriptor> getSnapshotDescriptors(String keyspace, String table, String tag) { - /** - * Snapshots are composed of hard-linked sstables. The true snapshot size should only include - * snapshot files which do not contain a corresponding "live" sstable file. - */ - @Override - public boolean isAcceptable(Path snapshotFilePath) + try + { + Refs<SSTableReader> snapshotSSTableReaders = getSnapshotSSTableReaders(keyspace, table, tag); + + Set<Descriptor> descriptors = new HashSet<>(); + for (SSTableReader ssTableReader : snapshotSSTableReaders) + { + descriptors.add(ssTableReader.descriptor); + } + + return descriptors; + } + catch (IOException e) { - return !getLiveFileFromSnapshotFile(snapshotFilePath).exists(); + throw Throwables.unchecked(e); } } - /** - * Returns the corresponding live file for a given snapshot file. - * - * Example: - * - Base table: - * - Snapshot file: ~/.ccm/test/node1/data0/test_ks/tbl-e03faca0813211eca100c705ea09b5ef/snapshots/1643481737850/me-1-big-Data.db - * - Live file: ~/.ccm/test/node1/data0/test_ks/tbl-e03faca0813211eca100c705ea09b5ef/me-1-big-Data.db - * - Secondary index: - * - Snapshot file: ~/.ccm/test/node1/data0/test_ks/tbl-e03faca0813211eca100c705ea09b5ef/snapshots/1643481737850/.tbl_val_idx/me-1-big-Summary.db - * - Live file: ~/.ccm/test/node1/data0/test_ks/tbl-e03faca0813211eca100c705ea09b5ef/.tbl_val_idx/me-1-big-Summary.db - * - */ - static File getLiveFileFromSnapshotFile(Path snapshotFilePath) + public static Refs<SSTableReader> getSnapshotSSTableReaders(String keyspace, String table, String tag) throws IOException { - // Snapshot directory structure format is {data_dir}/snapshots/{snapshot_name}/{snapshot_file} - Path liveDir = snapshotFilePath.getParent().getParent().getParent(); - if (Directories.isSecondaryIndexFolder(snapshotFilePath.getParent())) - { - // Snapshot file structure format is {data_dir}/snapshots/{snapshot_name}/.{index}/{sstable-component}.db - liveDir = File.getPath(liveDir.getParent().toString(), snapshotFilePath.getParent().getFileName().toString()); - } - return new File(liveDir.toString(), snapshotFilePath.getFileName().toString()); + return getSnapshotSSTableReaders(Keyspace.open(keyspace).getColumnFamilyStore(table), tag); } - public static Predicate<TableSnapshot> shouldClearSnapshot(String tag, long olderThanTimestamp) + public static Refs<SSTableReader> getSnapshotSSTableReaders(ColumnFamilyStore cfs, String tag) { - return ts -> + Map<SSTableId, SSTableReader> active = new HashMap<>(); + for (SSTableReader sstable : cfs.getSSTables(SSTableSet.CANONICAL)) + active.put(sstable.descriptor.id, sstable); + Map<Descriptor, Set<Component>> snapshots = cfs.getDirectories().sstableLister(Directories.OnTxnErr.IGNORE).snapshots(tag).list(); + Refs<SSTableReader> refs = new Refs<>(); + try { - // When no tag is supplied, all snapshots must be cleared - boolean clearAll = tag == null || tag.isEmpty(); - if (!clearAll && ts.isEphemeral()) - logger.info("Skipping deletion of ephemeral snapshot '{}' in keyspace {}. " + - "Ephemeral snapshots are not removable by a user.", - tag, ts.keyspaceName); - boolean notEphemeral = !ts.isEphemeral(); - boolean shouldClearTag = clearAll || ts.tag.equals(tag); - boolean byTimestamp = true; - - if (olderThanTimestamp > 0L) + for (Map.Entry<Descriptor, Set<Component>> entries : snapshots.entrySet()) { - Instant createdAt = ts.getCreatedAt(); - if (createdAt != null) - byTimestamp = createdAt.isBefore(Instant.ofEpochMilli(olderThanTimestamp)); + // Try to acquire reference to an active sstable instead of snapshot if it exists, + // to avoid opening new sstables. If it fails, use the snapshot reference instead. + SSTableReader sstable = active.get(entries.getKey().id); + if (sstable == null || !refs.tryRef(sstable)) + { + if (logger.isTraceEnabled()) + logger.trace("using snapshot sstable {}", entries.getKey()); + // open offline so we don't modify components or track hotness. + sstable = SSTableReader.open(cfs, entries.getKey(), entries.getValue(), cfs.metadata, true, true); + refs.tryRef(sstable); + // release the self ref as we never add the snapshot sstable to DataTracker where it is otherwise released + sstable.selfRef().release(); + } + else if (logger.isTraceEnabled()) + { + logger.trace("using active sstable {}", entries.getKey()); + } } + } + catch (FSReadError | RuntimeException e) + { + // In case one of the snapshot sstables fails to open, + // we must release the references to the ones we opened so far + refs.release(); + throw e; + } + return refs; + } - return notEphemeral && shouldClearTag && byTimestamp; - }; + public static String getTimestampedSnapshotName(String clientSuppliedName, long timestamp) + { + String snapshotName = Long.toString(timestamp); + if (clientSuppliedName != null && !clientSuppliedName.isEmpty()) + snapshotName = snapshotName + '-' + clientSuppliedName; + + return snapshotName; } + public static String getTimestampedSnapshotNameWithPrefix(String clientSuppliedName, long timestamp, String prefix) + { + return prefix + '-' + getTimestampedSnapshotName(clientSuppliedName, timestamp); + } } diff --git a/src/java/org/apache/cassandra/service/snapshot/TakeSnapshotTask.java b/src/java/org/apache/cassandra/service/snapshot/TakeSnapshotTask.java new file mode 100644 index 000000000000..2b1345976598 --- /dev/null +++ b/src/java/org/apache/cassandra/service/snapshot/TakeSnapshotTask.java @@ -0,0 +1,310 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.snapshot; + +import java.io.IOException; +import java.io.PrintStream; +import java.time.Instant; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Predicate; + +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Directories; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.SchemaCQLHelper; +import org.apache.cassandra.db.lifecycle.SSTableSet; +import org.apache.cassandra.db.lifecycle.View; +import org.apache.cassandra.db.memtable.Memtable; +import org.apache.cassandra.index.Index; +import org.apache.cassandra.index.internal.CassandraIndex; +import org.apache.cassandra.io.FSWriteError; +import org.apache.cassandra.io.sstable.format.SSTableFormat; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.io.util.FileOutputStreamPlus; +import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.utils.Clock; +import org.apache.cassandra.utils.FBUtilities; + +public class TakeSnapshotTask extends AbstractSnapshotTask<List<TableSnapshot>> +{ + private static final Logger logger = LoggerFactory.getLogger(TakeSnapshotTask.class); + private final SnapshotManager manager; + + private Instant creationTime; + private String snapshotName; + private final Map<ColumnFamilyStore, TableSnapshot> snapshotsToCreate = new HashMap<>(); + + public TakeSnapshotTask(SnapshotManager manager, SnapshotOptions options) + { + super(options); + this.manager = manager; + } + + @Override + public SnapshotTaskType getTaskType() + { + return SnapshotTaskType.SNAPSHOT; + } + + public Map<ColumnFamilyStore, TableSnapshot> getSnapshotsToCreate() + { + if (StorageService.instance.operationMode() == StorageService.Mode.JOINING) + throw new RuntimeException("Cannot snapshot until bootstrap completes"); + + // This is not in builder's build method on purpose in order to postpone the timestamp for as long as possible + // until the actual snapshot is taken. If we constructed a task and have not done anything with it for 5 minutes + // then by the time a snapshot would be taken the creation time would be quite off + Instant creationTimeInOptions = options.creationTime; + if (creationTimeInOptions == null) + creationTime = Instant.ofEpochMilli(Clock.Global.currentTimeMillis()); + else + creationTime = options.creationTime; + + snapshotName = options.getSnapshotName(creationTime); + + Set<ColumnFamilyStore> entitiesForSnapshot = options.cfs == null ? parseEntitiesForSnapshot(options.entities) : Set.of(options.cfs); + + for (ColumnFamilyStore cfs : entitiesForSnapshot) + { + Set<File> snapshotDirs = cfs.getDirectories().getSnapshotDirs(snapshotName); + + TableSnapshot tableSnapshot = new TableSnapshot(cfs.metadata.keyspace, + cfs.metadata.name, + cfs.metadata.id.asUUID(), + snapshotName, + creationTime, + SnapshotManifest.computeExpiration(options.ttl, creationTime), + snapshotDirs, + options.ephemeral); + // this snapshot does not have any actual representation on disk + // because that snapshot was technically not taken yet + tableSnapshot.incomplete(); + snapshotsToCreate.put(cfs, tableSnapshot); + } + + return snapshotsToCreate; + } + + @Override + public List<TableSnapshot> call() + { + assert snapshotName != null : "You need to call getSnapshotsToCreate() first"; + + List<TableSnapshot> createdSnapshots = new ArrayList<>(); + + for (Map.Entry<ColumnFamilyStore, TableSnapshot> entry : snapshotsToCreate.entrySet()) + { + try + { + ColumnFamilyStore cfs = entry.getKey(); + if (!options.skipFlush) + { + Memtable current = cfs.getTracker().getView().getCurrentMemtable(); + if (!current.isClean()) + { + if (current.shouldSwitch(ColumnFamilyStore.FlushReason.SNAPSHOT)) + FBUtilities.waitOnFuture(cfs.switchMemtableIfCurrent(current, ColumnFamilyStore.FlushReason.SNAPSHOT)); + else + current.performSnapshot(snapshotName); + } + } + + createSnapshot(cfs, entry.getValue(), snapshotName, creationTime); + createdSnapshots.add(entry.getValue()); + } + catch (Throwable t) + { + logger.warn(String.format("Unable to create snapshot %s for %s", entry.getValue().getTag(), entry.getKey().getKeyspaceTableName()), t); + // if we fail to take a snapshot, there is its phantom / in-progress representation among manager's taken snapshots, + // so we need to remove it from there to not appear in the outputs / not leak + manager.getSnapshots().remove(entry.getValue()); + } + } + + return createdSnapshots; + } + + private void createSnapshot(ColumnFamilyStore cfs, TableSnapshot snapshotToCreate, String snapshotName, Instant creationTime) + { + Predicate<SSTableReader> predicate = options.sstableFilter; + Set<SSTableReader> sstables = new LinkedHashSet<>(); + for (ColumnFamilyStore aCfs : cfs.concatWithIndexes()) + { + try (ColumnFamilyStore.RefViewFragment currentView = aCfs.selectAndReference(View.select(SSTableSet.CANONICAL, (x) -> predicate == null || predicate.test(x)))) + { + for (SSTableReader ssTable : currentView.sstables) + { + File snapshotDirectory = Directories.getSnapshotDirectory(ssTable.descriptor, snapshotName); + ssTable.createLinks(snapshotDirectory.path(), options.rateLimiter); // hard links + if (logger.isTraceEnabled()) + logger.trace("Snapshot for {} keyspace data file {} created in {}", cfs.keyspace, ssTable.getFilename(), snapshotDirectory); + sstables.add(ssTable); + } + } + } + + List<String> dataComponents = new ArrayList<>(); + for (SSTableReader sstable : sstables) + dataComponents.add(sstable.descriptor.relativeFilenameFor(SSTableFormat.Components.DATA)); + + SnapshotManifest manifest = new SnapshotManifest(dataComponents, options.ttl, creationTime, options.ephemeral); + Set<File> snapshotDirs = cfs.getDirectories().getSnapshotDirs(snapshotName); + for (File snapshotDir : snapshotDirs) + { + writeSnapshotManifest(manifest, Directories.getSnapshotManifestFile(snapshotDir)); + + if (!SchemaConstants.isLocalSystemKeyspace(cfs.metadata.keyspace) + && !SchemaConstants.isReplicatedSystemKeyspace(cfs.metadata.keyspace)) + { + writeSnapshotSchema(Directories.getSnapshotSchemaFile(snapshotDir), cfs); + } + } + + snapshotToCreate.updateMetadataSize(); + snapshotToCreate.complete(); + } + + private Set<ColumnFamilyStore> parseEntitiesForSnapshot(String... entities) + { + Set<ColumnFamilyStore> entitiesForSnapshot = new HashSet<>(); + + if (entities != null && entities.length > 0 && entities[0].contains(".")) + { + for (String entity : entities) + { + String[] splitted = StringUtils.split(entity, '.'); + if (splitted.length == 2) + { + String keyspaceName = splitted[0]; + String tableName = splitted[1]; + + if (keyspaceName == null) + throw new RuntimeException("You must supply a keyspace name"); + if (tableName == null) + throw new RuntimeException("You must supply a table name"); + + Keyspace validKeyspace = Keyspace.getValidKeyspace(keyspaceName); + ColumnFamilyStore existingTable = validKeyspace.getColumnFamilyStore(tableName); + + entitiesForSnapshot.add(existingTable); + } + // special case for index which we can not normally create a snapshot for + // but a snapshot is apparently taken before a secondary index is scrubbed, + // so we preserve this behavior + else if (splitted.length == 3) + { + String keyspaceName = splitted[0]; + String tableName = splitted[1]; + + Keyspace validKeyspace = Keyspace.getValidKeyspace(keyspaceName); + ColumnFamilyStore existingTable = validKeyspace.getColumnFamilyStore(tableName); + Index indexByName = existingTable.indexManager.getIndexByName(splitted[2]); + if (indexByName instanceof CassandraIndex) + { + ColumnFamilyStore indexCfs = ((CassandraIndex) indexByName).getIndexCfs(); + entitiesForSnapshot.add(indexCfs); + } + else + { + throw new IllegalArgumentException("Unknown index " + entity); + } + } + else + { + throw new IllegalArgumentException("Cannot take a snapshot on secondary index or invalid column " + + "family name. You must supply a column family name in the " + + "form of keyspace.columnfamily"); + } + } + } + else + { + if (entities != null && entities.length == 0) + { + for (Keyspace keyspace : Keyspace.all()) + { + entitiesForSnapshot.addAll(keyspace.getColumnFamilyStores()); + } + } + else if (entities != null) + { + for (String keyspace : entities) + { + Keyspace validKeyspace = Keyspace.getValidKeyspace(keyspace); + entitiesForSnapshot.addAll(validKeyspace.getColumnFamilyStores()); + } + } + } + + return entitiesForSnapshot; + } + + + private void writeSnapshotManifest(SnapshotManifest manifest, File manifestFile) + { + try + { + manifestFile.parent().tryCreateDirectories(); + manifest.serializeToJsonFile(manifestFile); + } + catch (IOException e) + { + throw new FSWriteError(e, manifestFile); + } + } + + private void writeSnapshotSchema(File schemaFile, ColumnFamilyStore cfs) + { + try + { + if (!schemaFile.parent().exists()) + schemaFile.parent().tryCreateDirectories(); + + try (PrintStream out = new PrintStream(new FileOutputStreamPlus(schemaFile))) + { + SchemaCQLHelper.reCreateStatementsForSchemaCql(cfs.metadata(), cfs.keyspace.getMetadata()) + .forEach(out::println); + } + } + catch (IOException e) + { + throw new FSWriteError(e, schemaFile); + } + } + + @Override + public String toString() + { + return "TakeSnapshotTask{" + + "options=" + options + + '}'; + } +} diff --git a/src/java/org/apache/cassandra/service/snapshot/TrueSnapshotSizeTask.java b/src/java/org/apache/cassandra/service/snapshot/TrueSnapshotSizeTask.java new file mode 100644 index 000000000000..bcffa1fdacbe --- /dev/null +++ b/src/java/org/apache/cassandra/service/snapshot/TrueSnapshotSizeTask.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.snapshot; + +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.function.Predicate; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; + +public class TrueSnapshotSizeTask implements Callable<Long> +{ + private final SnapshotManager snapshotManager; + private final Predicate<TableSnapshot> predicate; + + public TrueSnapshotSizeTask(SnapshotManager snapshotManager, Predicate<TableSnapshot> predicate) + { + this.snapshotManager = snapshotManager; + this.predicate = predicate; + } + + @Override + public Long call() + { + long size = 0; + for (TableSnapshot snapshot : snapshotManager.getSnapshots()) + { + if (predicate.test(snapshot)) + try + { + size += snapshot.computeTrueSizeBytes(getTablesFiles(snapshot.getKeyspaceName(), snapshot.getTableName())); + } + catch (Throwable ex) + { + // if any error happens while computing size, we don't include + return 0L; + } + } + + return size; + } + + private Set<String> getTablesFiles(String keyspaceName, String tableName) + { + try + { + Keyspace keyspace = Keyspace.getValidKeyspace(keyspaceName); + ColumnFamilyStore table = keyspace.getColumnFamilyStore(tableName); + + return table.getFilesOfCfs(); + } + catch (IllegalArgumentException ex) + { + // keyspace / table not found + return null; + } + } + +} diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index 5c75985bda2e..b2951a49b055 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -102,6 +102,7 @@ import org.apache.cassandra.service.ActiveRepairServiceMBean; import org.apache.cassandra.service.CacheService; import org.apache.cassandra.service.CacheServiceMBean; +import org.apache.cassandra.service.snapshot.SnapshotManagerMBean; import org.apache.cassandra.tcm.CMSOperationsMBean; import org.apache.cassandra.service.GCInspector; import org.apache.cassandra.service.GCInspectorMXBean; @@ -150,6 +151,7 @@ public class NodeProbe implements AutoCloseable protected MBeanServerConnection mbeanServerConn; protected CompactionManagerMBean compactionProxy; protected StorageServiceMBean ssProxy; + protected SnapshotManagerMBean snapshotProxy; protected CMSOperationsMBean cmsProxy; protected GossiperMBean gossProxy; protected MemoryMXBean memProxy; @@ -263,6 +265,8 @@ protected void connect() throws IOException { ObjectName name = new ObjectName(ssObjName); ssProxy = JMX.newMBeanProxy(mbeanServerConn, name, StorageServiceMBean.class); + name = new ObjectName(SnapshotManagerMBean.MBEAN_NAME); + snapshotProxy = JMX.newMBeanProxy(mbeanServerConn, name, SnapshotManagerMBean.class); name = new ObjectName(CMSOperations.MBEAN_OBJECT_NAME); cmsProxy = JMX.newMBeanProxy(mbeanServerConn, name, CMSOperationsMBean.class); name = new ObjectName(MessagingService.MBEAN_NAME); @@ -875,12 +879,12 @@ public MemoryUsage getHeapMemoryUsage() public long getSnapshotLinksPerSecond() { - return ssProxy.getSnapshotLinksPerSecond(); + return snapshotProxy.getSnapshotLinksPerSecond(); } public void setSnapshotLinksPerSecond(long throttle) { - ssProxy.setSnapshotLinksPerSecond(throttle); + snapshotProxy.setSnapshotLinksPerSecond(throttle); } /** @@ -900,10 +904,10 @@ public void takeSnapshot(String snapshotName, String table, Map<String, String> throw new IOException("When specifying the table for a snapshot, you must specify one and only one keyspace"); } - ssProxy.takeSnapshot(snapshotName, options, keyspaces[0] + "." + table); + snapshotProxy.takeSnapshot(snapshotName, options, keyspaces[0] + "." + table); } else - ssProxy.takeSnapshot(snapshotName, options, keyspaces); + snapshotProxy.takeSnapshot(snapshotName, options, keyspaces); } /** @@ -921,11 +925,11 @@ public void takeMultipleTableSnapshot(String snapshotName, Map<String, String> o { if (null != tableList && tableList.length != 0) { - ssProxy.takeSnapshot(snapshotName, options, tableList); + snapshotProxy.takeSnapshot(snapshotName, options, tableList); } else { - throw new IOException("The column family List for a snapshot should not be empty or null"); + throw new IOException("The column family list for a snapshot should not be empty or null"); } } @@ -955,12 +959,20 @@ public void clearSnapshot(String tag, String... keyspaces) throws IOException */ public void clearSnapshot(Map<String, Object> options, String tag, String... keyspaces) throws IOException { - ssProxy.clearSnapshot(options, tag, keyspaces); + snapshotProxy.clearSnapshot(tag, options, keyspaces); } + /** + * Gets all snapshots' details. + * + * @param options options to use upon listing of snapshots + * @return details of snapshots + */ + /** @deprecated See CASSANDRA-18111 */ + @Deprecated(since = "5.1") public Map<String, TabularData> getSnapshotDetails(Map<String, String> options) { - return ssProxy.getSnapshotDetails(options); + return snapshotProxy.listSnapshots(options); } /** @deprecated See CASSANDRA-16789 */ @@ -972,7 +984,7 @@ public Map<String, TabularData> getSnapshotDetails() public long trueSnapshotsSize() { - return ssProxy.trueSnapshotsSize(); + return snapshotProxy.getTrueSnapshotSize(); } public boolean isJoined() diff --git a/src/java/org/apache/cassandra/tools/nodetool/ListSnapshots.java b/src/java/org/apache/cassandra/tools/nodetool/ListSnapshots.java index 803fe5a4f0db..6a1c71d38605 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/ListSnapshots.java +++ b/src/java/org/apache/cassandra/tools/nodetool/ListSnapshots.java @@ -95,4 +95,4 @@ public void execute(NodeProbe probe) throw new RuntimeException("Error during list snapshot", e); } } -} +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/tools/nodetool/Snapshot.java b/src/java/org/apache/cassandra/tools/nodetool/Snapshot.java index 52cc5df27cc8..c3392bbcf5a5 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/Snapshot.java +++ b/src/java/org/apache/cassandra/tools/nodetool/Snapshot.java @@ -29,6 +29,7 @@ import io.airlift.airline.Option; import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.io.util.File; +import org.apache.cassandra.service.snapshot.SnapshotOptions; import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.tools.NodeTool.NodeToolCmd; @@ -68,10 +69,10 @@ public void execute(NodeProbe probe) sb.append("Requested creating snapshot(s) for "); Map<String, String> options = new HashMap<String,String>(); - options.put("skipFlush", Boolean.toString(skipFlush)); + options.put(SnapshotOptions.SKIP_FLUSH, Boolean.toString(skipFlush)); if (null != ttl) { DurationSpec.LongNanosecondsBound d = new DurationSpec.LongNanosecondsBound(ttl); - options.put("ttl", d.toString()); + options.put(SnapshotOptions.TTL, d.toString()); } if (!snapshotName.isEmpty() && snapshotName.contains(File.pathSeparator())) @@ -83,16 +84,16 @@ public void execute(NodeProbe probe) { ktList = ktList.replace(" ", ""); if (keyspaces.isEmpty() && null == table) - sb.append("[").append(ktList).append("]"); + sb.append('[').append(ktList).append(']'); else { throw new IOException( "When specifying the Keyspace table list (using -kt,--kt-list,-kc,--kc.list), you must not also specify keyspaces to snapshot"); } if (!snapshotName.isEmpty()) - sb.append(" with snapshot name [").append(snapshotName).append("]"); - sb.append(" and options ").append(options.toString()); - out.println(sb.toString()); + sb.append(" with snapshot name [").append(snapshotName).append(']'); + sb.append(" and options ").append(options); + out.println(sb); probe.takeMultipleTableSnapshot(snapshotName, options, ktList.split(",")); out.println("Snapshot directory: " + snapshotName); } @@ -101,12 +102,12 @@ public void execute(NodeProbe probe) if (keyspaces.isEmpty()) sb.append("[all keyspaces]"); else - sb.append("[").append(join(keyspaces, ", ")).append("]"); + sb.append('[').append(join(keyspaces, ", ")).append(']'); if (!snapshotName.isEmpty()) - sb.append(" with snapshot name [").append(snapshotName).append("]"); - sb.append(" and options ").append(options.toString()); - out.println(sb.toString()); + sb.append(" with snapshot name [").append(snapshotName).append(']'); + sb.append(" and options ").append(options); + out.println(sb); probe.takeSnapshot(snapshotName, table, options, toArray(keyspaces, String.class)); out.println("Snapshot directory: " + snapshotName); diff --git a/src/java/org/apache/cassandra/utils/DiagnosticSnapshotService.java b/src/java/org/apache/cassandra/utils/DiagnosticSnapshotService.java index dcae5759406e..caf2082bdaa4 100644 --- a/src/java/org/apache/cassandra/utils/DiagnosticSnapshotService.java +++ b/src/java/org/apache/cassandra/utils/DiagnosticSnapshotService.java @@ -27,6 +27,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Predicate; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; @@ -39,12 +40,16 @@ import org.apache.cassandra.dht.Bounds; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.net.Verb; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.snapshot.SnapshotManager; +import org.apache.cassandra.service.snapshot.SnapshotOptions; +import org.apache.cassandra.service.snapshot.SnapshotType; import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory; import static org.apache.cassandra.config.CassandraRelevantProperties.DIAGNOSTIC_SNAPSHOT_INTERVAL_NANOS; @@ -194,7 +199,8 @@ public void run() } ColumnFamilyStore cfs = ks.getColumnFamilyStore(command.column_family); - if (cfs.snapshotExists(command.snapshot_name)) + + if (SnapshotManager.instance.exists(command.keyspace, command.column_family, command.snapshot_name)) { logger.info("Received diagnostic snapshot request from {} for {}.{}, " + "but snapshot with tag {} already exists", @@ -210,16 +216,14 @@ public void run() command.column_family, command.snapshot_name); - if (ranges.isEmpty()) - cfs.snapshot(command.snapshot_name); - else - { - cfs.snapshot(command.snapshot_name, - (sstable) -> checkIntersection(ranges, - sstable.getFirst().getToken(), - sstable.getLast().getToken()), - false, false); - } + Predicate<SSTableReader> predicate = null; + if (!ranges.isEmpty()) + predicate = (sstable) -> checkIntersection(ranges, + sstable.getFirst().getToken(), + sstable.getLast().getToken()); + + SnapshotOptions options = SnapshotOptions.systemSnapshot(command.snapshot_name, SnapshotType.DIAGNOSTICS, predicate, cfs.getKeyspaceTableName()).build(); + SnapshotManager.instance.takeSnapshot(options); } catch (IllegalArgumentException e) { diff --git a/src/java/org/apache/cassandra/utils/DirectorySizeCalculator.java b/src/java/org/apache/cassandra/utils/DirectorySizeCalculator.java index 94f9b229b413..22da5c06cfb2 100644 --- a/src/java/org/apache/cassandra/utils/DirectorySizeCalculator.java +++ b/src/java/org/apache/cassandra/utils/DirectorySizeCalculator.java @@ -42,7 +42,7 @@ public boolean isAcceptable(Path file) } @Override - public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException + public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) { if (isAcceptable(file)) size += attrs.size(); @@ -63,7 +63,7 @@ public long getAllocatedSize() /** * Reset the size to 0 in case that the size calculator is used multiple times */ - protected void resetSize() + public void resetSize() { size = 0; } diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java index 7b5501fbad5c..1d2272ad4d2a 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java @@ -632,7 +632,6 @@ public void startup(ICluster cluster) { partialStartup(cluster); } - StorageService.instance.startSnapshotManager(); } catch (Throwable t) { @@ -719,6 +718,12 @@ protected void partialStartup(ICluster<?> cluster) throws IOException, NoSuchFie CassandraDaemon.getInstanceForTesting().migrateSystemDataIfNeeded(); CommitLog.instance.start(); + + SnapshotManager.instance.start(false); + SnapshotManager.instance.clearExpiredSnapshots(); + SnapshotManager.instance.clearEphemeralSnapshots(); + SnapshotManager.instance.resumeSnapshotCleanup(); + CassandraDaemon.getInstanceForTesting().runStartupChecks(); Keyspace.setInitialized(); // TODO: this seems to be superfluous by now @@ -891,6 +896,8 @@ public Future<Void> shutdown(boolean runOnExitThreads, boolean shutdownMessaging Future<?> future = async((ExecutorService executor) -> { Throwable error = null; + error = parallelRun(error, executor, SnapshotManager.instance::close); + CompactionManager.instance.forceShutdown(); error = parallelRun(error, executor, @@ -945,7 +952,7 @@ public Future<Void> shutdown(boolean runOnExitThreads, boolean shutdownMessaging () -> shutdownAndWait(Collections.singletonList(ActiveRepairService.repairCommandExecutor())), () -> ActiveRepairService.instance().shutdownNowAndWait(1L, MINUTES), () -> EpochAwareDebounce.instance.close(), - () -> SnapshotManager.shutdownAndWait(1L, MINUTES) + SnapshotManager.instance::close ); internodeMessagingStarted = false; diff --git a/test/distributed/org/apache/cassandra/distributed/mock/nodetool/InternalNodeProbe.java b/test/distributed/org/apache/cassandra/distributed/mock/nodetool/InternalNodeProbe.java index a1a29f09cbcf..2f9d628d5480 100644 --- a/test/distributed/org/apache/cassandra/distributed/mock/nodetool/InternalNodeProbe.java +++ b/test/distributed/org/apache/cassandra/distributed/mock/nodetool/InternalNodeProbe.java @@ -44,6 +44,7 @@ import org.apache.cassandra.service.GCInspector; import org.apache.cassandra.service.StorageProxy; import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.streaming.StreamManager; import org.apache.cassandra.tcm.CMSOperations; import org.apache.cassandra.tools.NodeProbe; @@ -69,6 +70,7 @@ protected void connect() StorageService.instance.skipNotificationListeners = !withNotifications; ssProxy = StorageService.instance; + snapshotProxy = SnapshotManager.instance; cmsProxy = CMSOperations.instance; msProxy = MessagingService.instance(); streamProxy = StreamManager.instance; diff --git a/test/distributed/org/apache/cassandra/distributed/test/AllowAutoSnapshotTest.java b/test/distributed/org/apache/cassandra/distributed/test/AllowAutoSnapshotTest.java index 7a94905abb15..aa7da3445b8d 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/AllowAutoSnapshotTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/AllowAutoSnapshotTest.java @@ -27,7 +27,7 @@ import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.api.IIsolatedExecutor; -import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.service.snapshot.SnapshotManager; import static org.apache.cassandra.distributed.Cluster.build; import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL; @@ -147,8 +147,8 @@ private void checkSnapshots(Cluster cluster, boolean shouldContain, String table { final int node = i; // has to be effectively final for the usage in "until" method await().until(() -> cluster.get(node).appliesOnInstance((IIsolatedExecutor.SerializableTriFunction<Boolean, String, String, Boolean>) (shouldContainSnapshot, tableName, prefix) -> { - Stream<String> stream = StorageService.instance.getSnapshotDetails(Collections.emptyMap()).keySet().stream(); - Predicate<String> predicate = tag -> tag.startsWith(prefix + '-') && tag.endsWith('-' + tableName); + Stream<String> stream = SnapshotManager.instance.listSnapshots(Collections.emptyMap()).keySet().stream(); + Predicate<String> predicate = tag -> tag.contains(prefix + '-' + table); return shouldContainSnapshot ? stream.anyMatch(predicate) : stream.noneMatch(predicate); }).apply(shouldContain, table, snapshotPrefix)); } diff --git a/test/distributed/org/apache/cassandra/distributed/test/AutoSnapshotTtlTest.java b/test/distributed/org/apache/cassandra/distributed/test/AutoSnapshotTtlTest.java index 550cbdc6e471..e08c16adcc04 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/AutoSnapshotTtlTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/AutoSnapshotTtlTest.java @@ -25,16 +25,14 @@ import org.junit.Test; import org.apache.cassandra.config.CassandraRelevantProperties; -import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.api.ConsistencyLevel; import org.apache.cassandra.distributed.api.Feature; import org.apache.cassandra.distributed.api.IInvokableInstance; import org.apache.cassandra.distributed.shared.WithProperties; +import org.apache.cassandra.service.snapshot.SnapshotType; import static java.util.concurrent.TimeUnit.SECONDS; -import static org.apache.cassandra.db.ColumnFamilyStore.SNAPSHOT_DROP_PREFIX; -import static org.apache.cassandra.db.ColumnFamilyStore.SNAPSHOT_TRUNCATE_PREFIX; import static org.apache.cassandra.distributed.Cluster.build; import static org.apache.cassandra.distributed.shared.ClusterUtils.stopUnchecked; import static org.awaitility.Awaitility.await; @@ -81,12 +79,12 @@ public void testAutoSnapshotTTlOnTruncate() throws IOException cluster.schemaChange(withKeyspace("TRUNCATE %s.tbl;")); // Check snapshot is listed after table is truncated - instance.nodetoolResult("listsnapshots").asserts().success().stdoutContains(SNAPSHOT_TRUNCATE_PREFIX); + instance.nodetoolResult("listsnapshots").asserts().success().stdoutContains(SnapshotType.TRUNCATE.label); // Check snapshot is removed after 10s await().timeout(10, SECONDS) .pollInterval(1, SECONDS) - .until(() -> !instance.nodetoolResult("listsnapshots").getStdout().contains(SNAPSHOT_DROP_PREFIX)); + .until(() -> !instance.nodetoolResult("listsnapshots").getStdout().contains(SnapshotType.DROP.label)); } } @@ -111,12 +109,12 @@ public void testAutoSnapshotTTlOnDrop() throws IOException cluster.schemaChange(withKeyspace("DROP TABLE %s.tbl;")); // Check snapshot is listed after table is dropped - instance.nodetoolResult("listsnapshots").asserts().success().stdoutContains(SNAPSHOT_DROP_PREFIX); + instance.nodetoolResult("listsnapshots").asserts().success().stdoutContains(SnapshotType.DROP.label); // Check snapshot is removed after 10s await().timeout(10, SECONDS) .pollInterval(1, SECONDS) - .until(() -> !instance.nodetoolResult("listsnapshots").getStdout().contains(SNAPSHOT_DROP_PREFIX)); + .until(() -> !instance.nodetoolResult("listsnapshots").getStdout().contains(SnapshotType.DROP.label)); } } @@ -146,12 +144,12 @@ public void testAutoSnapshotTTlOnDropAfterRestart() throws IOException instance.startup(); // Check snapshot is listed after restart - instance.nodetoolResult("listsnapshots").asserts().success().stdoutContains(SNAPSHOT_DROP_PREFIX); + instance.nodetoolResult("listsnapshots").asserts().success().stdoutContains(SnapshotType.DROP.label); // Check snapshot is removed after at most auto_snapshot_ttl + 1s await().timeout(ONE_MINUTE + 1, SECONDS) .pollInterval(1, SECONDS) - .until(() -> !instance.nodetoolResult("listsnapshots").getStdout().contains(SNAPSHOT_DROP_PREFIX)); + .until(() -> !instance.nodetoolResult("listsnapshots").getStdout().contains(SnapshotType.DROP.label)); } } @@ -178,13 +176,13 @@ public void testAutoSnapshotTtlDisabled() throws IOException, InterruptedExcepti cluster.schemaChange(withKeyspace("DROP TABLE %s.tbl;")); // Check snapshots are created after table is truncated and dropped - instance.nodetoolResult("listsnapshots").asserts().success().stdoutContains(SNAPSHOT_TRUNCATE_PREFIX); - instance.nodetoolResult("listsnapshots").asserts().success().stdoutContains(SNAPSHOT_DROP_PREFIX); + instance.nodetoolResult("listsnapshots").asserts().success().stdoutContains(SnapshotType.TRUNCATE.label); + instance.nodetoolResult("listsnapshots").asserts().success().stdoutContains(SnapshotType.DROP.label); // Check snapshot are *NOT* expired after 10s Thread.sleep(2 * FIVE_SECONDS * 1000L); - instance.nodetoolResult("listsnapshots").asserts().success().stdoutContains(ColumnFamilyStore.SNAPSHOT_TRUNCATE_PREFIX); - instance.nodetoolResult("listsnapshots").asserts().success().stdoutContains(ColumnFamilyStore.SNAPSHOT_DROP_PREFIX); + instance.nodetoolResult("listsnapshots").asserts().success().stdoutContains(SnapshotType.TRUNCATE.label); + instance.nodetoolResult("listsnapshots").asserts().success().stdoutContains(SnapshotType.DROP.label); } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/EphemeralSnapshotTest.java b/test/distributed/org/apache/cassandra/distributed/test/EphemeralSnapshotTest.java index 2de8f54694bc..f4d423cf23f8 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/EphemeralSnapshotTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/EphemeralSnapshotTest.java @@ -30,6 +30,7 @@ import org.apache.cassandra.distributed.api.IInvokableInstance; import org.apache.cassandra.distributed.api.IIsolatedExecutor; import org.apache.cassandra.io.util.File; +import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.service.snapshot.SnapshotManifest; import org.apache.cassandra.utils.Pair; @@ -61,6 +62,8 @@ public void testStartupRemovesEphemeralSnapshotOnEphemeralFlagInManifest() throw rewriteManifestToEphemeral(initialisationData.left, initialisationData.right); + c.get(1).runOnInstance((IIsolatedExecutor.SerializableRunnable) () -> SnapshotManager.instance.restart(true)); + verify(c.get(1)); } } @@ -88,6 +91,8 @@ public void testStartupRemovesEphemeralSnapshotOnMarkerFile() throws Exception Files.createFile(ephemeralMarkerFile); + c.get(1).runOnInstance((IIsolatedExecutor.SerializableRunnable) () -> SnapshotManager.instance.restart(true)); + verify(c.get(1)); } } @@ -104,6 +109,8 @@ public void testEphemeralSnapshotIsNotClearableFromNodetool() throws Exception Pair<String, String[]> initialisationData = initialise(c); rewriteManifestToEphemeral(initialisationData.left, initialisationData.right); + c.get(1).runOnInstance((IIsolatedExecutor.SerializableRunnable) () -> SnapshotManager.instance.restart(true)); + assertTrue(instance.nodetoolResult("listsnapshots", "-e").getStdout().contains(snapshotName)); instance.nodetoolResult("clearsnapshot", "-t", snapshotName).asserts().success(); // ephemeral snapshot was not removed as it can not be (from nodetool / user operation) @@ -127,6 +134,8 @@ public void testClearingAllSnapshotsFromNodetoolWillKeepEphemeralSnaphotsIntact( rewriteManifestToEphemeral(initialisationData.left, initialisationData.right); + c.get(1).runOnInstance((IIsolatedExecutor.SerializableRunnable) () -> SnapshotManager.instance.restart(true)); + instance.nodetoolResult("clearsnapshot", "--all").asserts().success(); assertTrue(instance.nodetoolResult("listsnapshots", "-e").getStdout().contains(snapshotName)); assertFalse(instance.nodetoolResult("listsnapshots", "-e").getStdout().contains(snapshotName2)); diff --git a/test/distributed/org/apache/cassandra/distributed/test/PreviewRepairSnapshotTest.java b/test/distributed/org/apache/cassandra/distributed/test/PreviewRepairSnapshotTest.java index 529e7a3e5aa4..10c7dc187a66 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/PreviewRepairSnapshotTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/PreviewRepairSnapshotTest.java @@ -36,6 +36,7 @@ import org.apache.cassandra.distributed.api.ConsistencyLevel; import org.apache.cassandra.distributed.api.IIsolatedExecutor; import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.service.snapshot.TableSnapshot; import org.apache.cassandra.utils.concurrent.Refs; import static java.util.concurrent.TimeUnit.MILLISECONDS; @@ -126,7 +127,7 @@ private IIsolatedExecutor.SerializableRunnable checkSnapshot(Set<Token> mismatch String snapshotTag = await().atMost(1, MINUTES) .pollInterval(100, MILLISECONDS) .until(() -> { - for (String tag : cfs.listSnapshots().keySet()) + for (String tag : Util.listSnapshots(cfs).keySet()) { // we create the snapshot schema file last, so when this exists we know the snapshot is complete; if (cfs.getDirectories().getSnapshotSchemaFile(tag).exists()) @@ -138,7 +139,7 @@ private IIsolatedExecutor.SerializableRunnable checkSnapshot(Set<Token> mismatch Set<SSTableReader> inSnapshot = new HashSet<>(); - try (Refs<SSTableReader> sstables = cfs.getSnapshotSSTableReaders(snapshotTag)) + try (Refs<SSTableReader> sstables = TableSnapshot.getSnapshotSSTableReaders(cfs.getKeyspaceName(), cfs.name, snapshotTag)) { inSnapshot.addAll(sstables); } diff --git a/test/distributed/org/apache/cassandra/distributed/test/PreviewRepairTest.java b/test/distributed/org/apache/cassandra/distributed/test/PreviewRepairTest.java index a0b643f0d309..82c81f221dcd 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/PreviewRepairTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/PreviewRepairTest.java @@ -35,6 +35,7 @@ import com.google.common.util.concurrent.Uninterruptibles; +import org.apache.cassandra.Util; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.concurrent.Condition; import org.junit.BeforeClass; @@ -434,11 +435,11 @@ private void verifySnapshots(Cluster cluster, String table, boolean shouldBeEmpt ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(table); if(shouldBeEmpty) { - assertTrue(cfs.listSnapshots().isEmpty()); + assertTrue(Util.listSnapshots(cfs).isEmpty()); } else { - while (cfs.listSnapshots().isEmpty()) + while (Util.listSnapshots(cfs).isEmpty()) Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS); } })); diff --git a/test/distributed/org/apache/cassandra/distributed/test/RepairDigestTrackingTest.java b/test/distributed/org/apache/cassandra/distributed/test/RepairDigestTrackingTest.java index 1c50f0ee3b98..1e5773c67a05 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/RepairDigestTrackingTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/RepairDigestTrackingTest.java @@ -30,6 +30,8 @@ import com.google.common.util.concurrent.Uninterruptibles; import org.junit.Assert; + +import org.apache.cassandra.Util; import org.apache.cassandra.concurrent.SEPExecutor; import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.dht.Token; @@ -39,6 +41,7 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.ReplicaLayout; import org.apache.cassandra.locator.ReplicaUtils; +import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.utils.Throwables; import org.junit.Test; @@ -591,7 +594,7 @@ private IInvokableInstance.SerializableRunnable assertSnapshotPresent(String sna int attempts = 100; ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE); - while (cfs.listSnapshots().isEmpty()) + while (Util.listSnapshots(cfs).isEmpty()) { Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS); if (attempts-- < 0) @@ -602,11 +605,7 @@ private IInvokableInstance.SerializableRunnable assertSnapshotPresent(String sna private IInvokableInstance.SerializableRunnable assertSnapshotNotPresent(String snapshotName) { - return () -> - { - ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE); - Assert.assertFalse(cfs.snapshotExists(snapshotName)); - }; + return () -> Assert.assertFalse(SnapshotManager.instance.exists(KEYSPACE, TABLE, snapshotName)); } private long getConfirmedInconsistencies(IInvokableInstance instance) diff --git a/test/distributed/org/apache/cassandra/distributed/test/SSTableIdGenerationTest.java b/test/distributed/org/apache/cassandra/distributed/test/SSTableIdGenerationTest.java index a44532ee156f..c45108eea3a8 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/SSTableIdGenerationTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/SSTableIdGenerationTest.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.nio.file.Files; import java.util.Arrays; +import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.function.Function; @@ -51,6 +52,9 @@ import org.apache.cassandra.io.sstable.UUIDBasedSSTableId; import org.apache.cassandra.io.util.File; import org.apache.cassandra.metrics.RestorableMeter; +import org.apache.cassandra.service.snapshot.SnapshotManager; +import org.apache.cassandra.service.snapshot.SnapshotOptions; +import org.apache.cassandra.service.snapshot.TableSnapshot; import org.apache.cassandra.tools.SystemExitException; import org.apache.cassandra.utils.TimeUUID; import org.assertj.core.api.Assertions; @@ -60,7 +64,6 @@ import static org.apache.cassandra.Util.bulkLoadSSTables; import static org.apache.cassandra.Util.getBackups; import static org.apache.cassandra.Util.getSSTables; -import static org.apache.cassandra.Util.getSnapshots; import static org.apache.cassandra.Util.relativizePath; import static org.apache.cassandra.cql3.QueryProcessor.executeInternal; import static org.apache.cassandra.db.SystemKeyspace.LEGACY_SSTABLE_ACTIVITY; @@ -404,12 +407,20 @@ private static void truncateAndAssertEmpty(IInvokableInstance instance, String k private static Set<String> snapshot(IInvokableInstance instance, String ks, String tableName) { - Set<String> snapshotDirs = instance.callOnInstance(() -> ColumnFamilyStore.getIfExists(ks, tableName) - .snapshot(SNAPSHOT_TAG) - .getDirectories() - .stream() - .map(File::toString) - .collect(Collectors.toSet())); + Set<String> snapshotDirs = instance.callOnInstance(() -> { + + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(ks, tableName); + + if (cfs == null) + return Set.of(); + + TableSnapshot tableSnapshot = SnapshotManager.instance.takeSnapshot(SnapshotOptions.userSnapshot(SNAPSHOT_TAG, cfs.getKeyspaceTableName())).iterator().next(); + Set<String> dirs = new HashSet<>(); + for (File dir : tableSnapshot.getDirectories()) + dirs.add(dir.toString()); + + return dirs; + }); assertThat(snapshotDirs).isNotEmpty(); return snapshotDirs; } @@ -458,7 +469,7 @@ private static void assertSSTablesCount(IInvokableInstance instance, int expecte private static void assertSnapshotSSTablesCount(IInvokableInstance instance, int expectedSeqGenIds, int expectedUUIDGenIds, String ks, String... tableNames) { - instance.runOnInstance(rethrow(() -> Arrays.stream(tableNames).forEach(tableName -> assertSSTablesCount(getSnapshots(ks, tableName, SNAPSHOT_TAG), tableName, expectedSeqGenIds, expectedUUIDGenIds)))); + instance.runOnInstance(rethrow(() -> Arrays.stream(tableNames).forEach(tableName -> assertSSTablesCount(TableSnapshot.getSnapshotDescriptors(ks, tableName, SNAPSHOT_TAG), tableName, expectedSeqGenIds, expectedUUIDGenIds)))); } private static void assertBackupSSTablesCount(IInvokableInstance instance, int expectedSeqGenIds, int expectedUUIDGenIds, String ks, String... tableNames) diff --git a/test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java b/test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java index 8ba3a905600f..9ca5e052338b 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java @@ -19,6 +19,9 @@ package org.apache.cassandra.distributed.test; import java.io.IOException; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.regex.Pattern; @@ -29,9 +32,11 @@ import org.junit.Test; import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.api.ConsistencyLevel; import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableCallable; import org.apache.cassandra.distributed.api.NodeToolResult; import org.apache.cassandra.distributed.shared.WithProperties; import org.apache.cassandra.utils.Clock; @@ -40,8 +45,10 @@ import static java.util.concurrent.TimeUnit.SECONDS; import static java.util.stream.Collectors.toList; import static org.apache.cassandra.distributed.shared.ClusterUtils.stopUnchecked; +import static org.assertj.core.api.Assertions.assertThat; import static org.awaitility.Awaitility.await; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; public class SnapshotsTest extends TestBaseImpl { @@ -51,9 +58,9 @@ public class SnapshotsTest extends TestBaseImpl private static final WithProperties properties = new WithProperties(); private static Cluster cluster; - private final String[] exoticSnapshotNames = new String[] { "snapshot", "snapshots", "backup", "backups", - "Snapshot", "Snapshots", "Backups", "Backup", - "snapshot.with.dots-and-dashes"}; + private final String[] exoticSnapshotNames = new String[]{ "snapshot", "snapshots", "backup", "backups", + "Snapshot", "Snapshots", "Backups", "Backup", + "snapshot.with.dots-and-dashes" }; @BeforeClass public static void before() throws IOException @@ -61,15 +68,18 @@ public static void before() throws IOException properties.set(CassandraRelevantProperties.SNAPSHOT_CLEANUP_INITIAL_DELAY_SECONDS, 0); properties.set(CassandraRelevantProperties.SNAPSHOT_CLEANUP_PERIOD_SECONDS, SNAPSHOT_CLEANUP_PERIOD_SECONDS); properties.set(CassandraRelevantProperties.SNAPSHOT_MIN_ALLOWED_TTL_SECONDS, FIVE_SECONDS); - cluster = init(Cluster.build(1).start()); + cluster = init(Cluster.build(1) + .withDataDirCount(3) + .start()); } @After public void clearAllSnapshots() { cluster.schemaChange(withKeyspace("DROP TABLE IF EXISTS %s.tbl;")); + cluster.schemaChange(withKeyspace("DROP TABLE IF EXISTS %s.tbl2;")); cluster.get(1).nodetoolResult("clearsnapshot", "--all").asserts().success(); - for (String tag : new String[] {"basic", "first", "second", "tag1"}) + for (String tag : new String[]{ "basic", "first", "second", "tag1" }) waitForSnapshotCleared(tag); for (String tag : exoticSnapshotNames) waitForSnapshot(tag, false, true); @@ -83,6 +93,34 @@ public static void after() cluster.close(); } + @Test + public void testEverySnapshotDirHasManifestAndSchema() + { + cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (key int, value text, PRIMARY KEY (key))")); + String[] dataDirs = (String[]) cluster.get(1).config().get("data_file_directories"); + String tableId = cluster.get(1).callOnInstance((SerializableCallable<String>) () -> { + return ColumnFamilyStore.getIfExists("distributed_test_keyspace", "tbl").metadata().id.toHexString(); + }); + + cluster.get(1) + .nodetoolResult("snapshot", "-t", "mysnapshot", "-kt", format("%s.tbl", KEYSPACE)) + .asserts() + .success(); + + for (String dataDir : dataDirs) + { + Path snapshotDir = Paths.get(dataDir) + .resolve(KEYSPACE) + .resolve("tbl-" + tableId) + .resolve("snapshots") + .resolve("mysnapshot"); + + assertTrue(snapshotDir.toFile().exists()); + assertTrue(snapshotDir.resolve("manifest.json").toFile().exists()); + assertTrue(snapshotDir.resolve("schema.cql").toFile().exists()); + } + } + @Test public void testSnapshotsCleanupByTTL() { @@ -161,7 +199,8 @@ public void testListingSnapshotsWithoutTTL() } @Test - public void testManualSnapshotCleanup() { + public void testManualSnapshotCleanup() + { // take snapshots with ttl cluster.get(1).nodetoolResult("snapshot", "--ttl", format("%ds", TEN_SECONDS), @@ -210,8 +249,8 @@ public void testListSnapshotOfDroppedTable() populate(cluster); instance.nodetoolResult("snapshot", - "-t", "tag1", - "-kt", withKeyspace("%s.tbl")).asserts().success(); + "-t", "tag1", + "-kt", withKeyspace("%s.tbl")).asserts().success(); // Check snapshot is listed when table is not dropped waitForSnapshotPresent("tag1"); @@ -314,21 +353,47 @@ public void testSameTimestampOnEachTableOfSnaphot() Pattern COMPILE = Pattern.compile(" +"); long distinctTimestamps = Arrays.stream(result.getStdout().split("\n")) - .filter(line -> line.startsWith("sametimestamp")) - .map(line -> COMPILE.matcher(line).replaceAll(" ").split(" ")[7]) - .distinct() - .count(); + .filter(line -> line.startsWith("sametimestamp")) + .map(line -> COMPILE.matcher(line).replaceAll(" ").split(" ")[7]) + .distinct() + .count(); // assert all dates are same so there is just one value accross all individual tables assertEquals(1, distinctTimestamps); } + @Test + public void testFailureToSnapshotTwiceOnSameEntityWithSameSnapshotName() + { + cluster.get(1).nodetoolResult("snapshot", "-t", "somename").asserts().success(); + + NodeToolResult failedSnapshotResult = cluster.get(1).nodetoolResult("snapshot", "-t", "somename"); + failedSnapshotResult.asserts().failure(); + Throwable error = failedSnapshotResult.getError(); + assertThat(error.getMessage()).contains("already exists"); + } + + @Test + public void testTakingSnapshoWithSameNameOnDifferentTablesDoesNotFail() + { + cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (key int, value text, PRIMARY KEY (key))")); + cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl2 (key int, value text, PRIMARY KEY (key))")); + cluster.get(1).nodetoolResult("snapshot", "-t", "somename", "-kt", String.format("%s.tbl", KEYSPACE)).asserts().success(); + cluster.get(1).nodetoolResult("snapshot", "-t", "somename", "-kt", String.format("%s.tbl2", KEYSPACE)).asserts().success(); + } + private void populate(Cluster cluster) { for (int i = 0; i < 100; i++) cluster.coordinator(1).execute(withKeyspace("INSERT INTO %s.tbl (key, value) VALUES (?, 'txt')"), ConsistencyLevel.ONE, i); } + private void populate(Cluster cluster, String keyspace, String table) + { + for (int i = 0; i < 100; i++) + cluster.coordinator(1).execute(format("INSERT INTO %s.%s (key, value) VALUES (?, 'txt')", keyspace, table), ConsistencyLevel.ONE, i); + } + private void waitForSnapshotPresent(String snapshotName) { waitForSnapshot(snapshotName, true, false); @@ -339,20 +404,28 @@ private void waitForSnapshotCleared(String snapshotName) waitForSnapshot(snapshotName, false, false); } - private void waitForSnapshot(String snapshotName, boolean expectPresent, boolean noTTL) + private void waitForSnapshot(String keyspaceName, String tableName, String snapshotName, boolean expectPresent, boolean noTTL) { await().timeout(20, SECONDS) .pollDelay(0, SECONDS) .pollInterval(1, SECONDS) - .until(() -> waitForSnapshotInternal(snapshotName, expectPresent, noTTL)); + .until(() -> waitForSnapshotInternal(keyspaceName, tableName, snapshotName, expectPresent, noTTL)); + } + + private void waitForSnapshot(String snapshotName, boolean expectPresent, boolean noTTL) + { + waitForSnapshot(null, null, snapshotName, expectPresent, noTTL); } - private boolean waitForSnapshotInternal(String snapshotName, boolean expectPresent, boolean noTTL) { + private boolean waitForSnapshotInternal(String keyspaceName, String tableName, String snapshotName, boolean expectPresent, boolean noTTL) + { + List<String> args = new ArrayList<>(); + args.add("listsnapshots"); NodeToolResult listsnapshots; if (noTTL) - listsnapshots = cluster.get(1).nodetoolResult("listsnapshots", "-nt"); - else - listsnapshots = cluster.get(1).nodetoolResult("listsnapshots"); + args.add("-nt"); + + listsnapshots = cluster.get(1).nodetoolResult(args.toArray(new String[0])); List<String> lines = Arrays.stream(listsnapshots.getStdout().split("\n")) .filter(line -> !line.isEmpty()) diff --git a/test/distributed/org/apache/cassandra/fuzz/snapshots/SnapshotsTest.java b/test/distributed/org/apache/cassandra/fuzz/snapshots/SnapshotsTest.java new file mode 100644 index 000000000000..8c82b1cb11b6 --- /dev/null +++ b/test/distributed/org/apache/cassandra/fuzz/snapshots/SnapshotsTest.java @@ -0,0 +1,1251 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.fuzz.snapshots; + +import java.io.IOException; +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import javax.management.openmbean.TabularData; + +import com.google.common.collect.MapDifference; +import com.google.common.collect.MapDifference.ValueDifference; +import com.google.common.collect.Maps; +import com.google.common.util.concurrent.Uninterruptibles; +import org.junit.Test; + +import accord.utils.Property.StateOnlyCommand; +import accord.utils.RandomSource; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.api.IIsolatedExecutor; +import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableFunction; +import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableTriConsumer; +import org.apache.cassandra.distributed.api.NodeToolResult; +import org.apache.cassandra.fuzz.snapshots.SnapshotsTest.State.TestSnapshot; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.schema.Keyspaces; +import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.snapshot.SnapshotManager; +import org.apache.cassandra.service.snapshot.TableSnapshot; +import org.apache.cassandra.utils.Generators; +import org.apache.cassandra.utils.LocalizeString; +import org.apache.cassandra.utils.Pair; +import org.quicktheories.core.RandomnessSource; +import org.quicktheories.generators.SourceDSL; +import org.quicktheories.impl.JavaRandom; + +import static accord.utils.Property.commands; +import static accord.utils.Property.stateful; +import static com.google.common.collect.Sets.difference; +import static java.lang.String.format; +import static java.util.UUID.randomUUID; +import static org.apache.cassandra.fuzz.snapshots.SnapshotsTest.CreateKeyspace.createKeypace; +import static org.apache.cassandra.fuzz.snapshots.SnapshotsTest.CreateTable.createTable; +import static org.junit.Assert.assertEquals; + +public class SnapshotsTest +{ + @Test + public void fuzzySnapshotsTest() + { + stateful() + .withExamples(1) + .withSteps(500) + .withStepTimeout(Duration.ofMinutes(1)) + .check(commands(() -> State::new) + .add(CreateKeyspace::new) + .add(DropKeyspace::new) + .add(CreateTable::new) + .add(DropTable::new) + .add(TruncateTable::new) + .add(TakeSnapshot::new) + .add(ClearSnapshot::new) + .add(ListSnapshots::new) + .destroyState(State::destroy) + .build()); + } + + public static class ListSnapshots extends AbstractCommand + { + private final Pair<SnapshotsHolder, Map<String, String>> listingParams; + + public ListSnapshots(RandomSource rs, State state) + { + super(rs, state); + listingParams = generateParams(); + } + + @Override + public void doWork(State state) + { + assertEquals(listingParams.left, categorize(list(listingParams.right))); + } + + @Override + public String toString() + { + return "List snapshots with parameters: " + listingParams; + } + + private Pair<SnapshotsHolder, Map<String, String>> generateParams() + { + Map<String, String> listingParams = new HashMap<>(); + List<Integer> toShuffle = state.getShuffledListOfInts(4); + + String keyspace = null; + for (int i = 0; i < 4; i++) + { + boolean picked = false; + switch (toShuffle.get(i)) + { + case 1: + if (!state.truncatedSnapshots.isEmpty()) + { + keyspace = state.rs.pick(state.truncatedSnapshots.keySet()).split("\\.")[0]; + picked = true; + } + break; + case 2: + if (!state.droppedSnapshots.isEmpty()) + { + keyspace = state.rs.pick(state.droppedSnapshots).split("\\.")[0]; + picked = true; + } + break; + case 3: + if (!state.snapshots.isEmpty()) + { + keyspace = state.rs.pick(state.snapshots).getKeyspaceName(); + picked = true; + } + break; + default: + // keyspace will be null so all snapshost will be listed + picked = true; + break; + } + if (picked) + break; + } + + // we need to populate expected snapshots after listing + SnapshotsHolder holder = new SnapshotsHolder(); + + if (keyspace == null) + { + holder.normal.addAll(state.snapshots); + holder.dropped.addAll(state.droppedSnapshots); + holder.truncated.putAll(state.truncatedSnapshots); + } + else + { + listingParams.put("keyspace", keyspace); + for (String s : state.droppedSnapshots) + if (s.startsWith(keyspace + '.')) + holder.dropped.add(s); + + for (Map.Entry<String, Integer> entry : state.truncatedSnapshots.entrySet()) + if (entry.getKey().startsWith(keyspace + '.')) + holder.truncated.put(entry.getKey(), entry.getValue()); + + for (TestSnapshot s : state.snapshots) + if (s.getKeyspaceName().equals(keyspace)) + holder.normal.add(s); + } + + return Pair.create(holder, listingParams); + } + + private List<String> list(Map<String, String> parameters) + { + return getNode().applyOnInstance((SerializableFunction<Map<String, String>, List<String>>) (params) -> + { + Map<String, TabularData> listingResult = SnapshotManager.instance.listSnapshots(params); + + List<String> snapshots = new ArrayList<>(); + for (final Map.Entry<String, TabularData> snapshotDetail : listingResult.entrySet()) + { + Set<?> values = snapshotDetail.getValue().keySet(); + for (Object eachValue : values) + { + final List<?> value = (List<?>) eachValue; + String tag = (String) value.get(0); + String keyspace = (String) value.get(1); + String table = (String) value.get(2); + snapshots.add(format("%s.%s.%s", keyspace, table, tag)); + } + } + + return snapshots; + }, parameters); + } + + private SnapshotsHolder categorize(List<String> listedSnapshots) + { + SnapshotsHolder holder = new SnapshotsHolder(); + for (String snapshot : listedSnapshots) + { + if (snapshot.contains("dropped-")) + { + String[] split = snapshot.split("\\."); + holder.dropped.add(format("%s.%s", split[0], split[1])); + } + else if (snapshot.contains("truncated-")) + { + String[] split = snapshot.split("\\."); + String ksTb = format("%s.%s", split[0], split[1]); + holder.truncated.merge(ksTb, 1, Integer::sum); + } + else + { + String[] split = snapshot.split("\\."); + holder.normal.add(new TestSnapshot(split[0], split[1], split[2])); + } + } + + return holder; + } + + private static class SnapshotsHolder + { + Set<TestSnapshot> normal = new HashSet<>(); + Set<String> dropped = new HashSet<>(); + Map<String, Integer> truncated = new HashMap<>(); + + @Override + public String toString() + { + return "SnapshotsHolder{" + + "normal=" + normal + + ", dropped=" + dropped + + ", truncated=" + truncated + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + SnapshotsHolder holder = (SnapshotsHolder) o; + return Objects.equals(normal, holder.normal) && + Objects.equals(dropped, holder.dropped) && + Objects.equals(truncated, holder.truncated); + } + + @Override + public int hashCode() + { + return Objects.hash(normal, dropped, truncated); + } + } + } + + public static class CreateKeyspace extends AbstractCommand + { + private static final String CREATE_KEYSPACE_QUERY = + "CREATE KEYSPACE %s WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : '1' }"; + + private final Set<String> keyspaceToCreate = new HashSet<>(); + + public CreateKeyspace(RandomSource rs, State state) + { + super(rs, state); + keyspaceToCreate.add(state.addRandomKeyspace()); + } + + @Override + public void doWork(State state) + { + createKeypace(state); + assertEquals(state.schema.keySet(), state.getKeyspaces()); + state.populate(); + } + + public static void createKeypace(State state) + { + Set<String> existingKeyspaces = state.getKeyspaces(); + Set<String> keyspacesToBe = state.schema.keySet(); + + Set<String> difference = difference(keyspacesToBe, existingKeyspaces); + if (!difference.isEmpty()) + state.getNode().executeInternal(format(CREATE_KEYSPACE_QUERY, difference.iterator().next())); + } + + @Override + public String toString() + { + return "Create keyspace " + keyspaceToCreate; + } + } + + public static class DropKeyspace extends AbstractCommand + { + private static final String DROP_KEYSPACE_QUERY = "DROP KEYSPACE %s"; + + private String keyspaceToDrop; + + private final boolean shouldApply; + + public DropKeyspace(RandomSource rs, State state) + { + super(rs, state); + + Optional<Pair<String, List<String>>> maybeDroppedKeyspaceWithTables = state.removeRandomKeyspace(); + shouldApply = maybeDroppedKeyspaceWithTables.isPresent(); + + if (!shouldApply) + return; + + // dropping of a keyspace will create dropped tables which create "dropped" snapshots + String keyspace = maybeDroppedKeyspaceWithTables.get().left; + List<String> tables = maybeDroppedKeyspaceWithTables.get().right; + for (String table : tables) + state.addDroppedSnapshot(keyspace, table); + + Set<String> existingKeyspaces = state.getKeyspaces(); + Set<String> keyspacesToBe = state.schema.keySet(); + Set<String> diff = difference(existingKeyspaces, keyspacesToBe); + assert diff.size() == 1; + keyspaceToDrop = diff.iterator().next(); + } + + @Override + public void doWork(State state) + { + if (!shouldApply) + return; + + executeQuery(format(DROP_KEYSPACE_QUERY, keyspaceToDrop)); + assertEquals(state.schema.keySet(), state.getKeyspaces()); + state.populate(); + } + + @Override + public String toString() + { + if (keyspaceToDrop != null) + return "Drop keyspace " + keyspaceToDrop; + else + return "Drop keyspace skipping"; + } + } + + public static class CreateTable extends AbstractCommand + { + private static final String CREATE_TABLE_QUERY = "CREATE TABLE %s.%s (id uuid primary key, val uuid)"; + private static final String INSERT_QUERY = "INSERT INTO %s.%s (id, val) VALUES (%s, %s)"; + + private final Pair<String, String> table; + + public CreateTable(RandomSource rs, State state) + { + super(rs, state); + table = state.addRandomTable(); + } + + @Override + public void doWork(State state) + { + Map<String, List<String>> expected = state.schema; + createTable(state); + assertEquals(expected, state.getSchema()); + state.populate(); + } + + public static Map<String, List<String>> createTable(State state) + { + Map<String, List<String>> existingSchema = state.getSchema(); + Map<String, List<String>> schemaToBe = state.schema; + + MapDifference<String, List<String>> difference = Maps.difference(schemaToBe, existingSchema); + + Map<String, List<String>> tablesToCreate = new HashMap<>(); + + for (Entry<String, ValueDifference<List<String>>> entry : difference.entriesDiffering().entrySet()) + { + String keyspaceName = entry.getKey(); + tablesToCreate.put(keyspaceName, new ArrayList<>()); + ValueDifference<List<String>> tableNames = entry.getValue(); + for (String table : difference(new HashSet<>(tableNames.leftValue()), + new HashSet<>(tableNames.rightValue()))) + { + tablesToCreate.get(keyspaceName).add(table); + } + } + + for (Map.Entry<String, List<String>> entry : tablesToCreate.entrySet()) + { + String keyspace = entry.getKey(); + for (String table : entry.getValue()) + { + state.executeQuery(format(CREATE_TABLE_QUERY, keyspace, table)); + populateTable(state, keyspace, table); + } + } + + return tablesToCreate; + } + + public static void populateTable(State state, String keypace, String table) + { + // create between 1 and 10 sstables per table + for (int i = 0; i < state.rs.nextInt(1, 10); i++) + { + state.executeQuery(format(INSERT_QUERY, keypace, table, randomUUID(), randomUUID())); + state.nodetool("flush", keypace, table); + } + } + + @Override + public String toString() + { + return "Create table " + format("%s.%s", table.left, table.right); + } + } + + public static class CreateMoreData extends AbstractCommand + { + private Pair<String, String> tableToPopulate; + private final boolean shouldApply; + + public CreateMoreData(RandomSource rs, State state) + { + super(rs, state); + + Optional<Pair<String, String>> randomTable = state.pickRandomTable(false); + shouldApply = randomTable.isPresent(); + + if (!shouldApply) + return; + + tableToPopulate = randomTable.get(); + } + + @Override + public void doWork(State state) + { + if (!shouldApply) + return; + + CreateTable.populateTable(state, tableToPopulate.left, tableToPopulate.right); + } + + @Override + public String toString() + { + if (tableToPopulate != null) + return "Create more data " + format("%s.%s", tableToPopulate.left, tableToPopulate.right); + else + return "Create more data skipped"; + } + } + + public static class DropTable extends AbstractCommand + { + private static final String DROP_TABLE_QUERY = "DROP TABLE %s.%s"; + + private Pair<String, String> tableToDrop; + private final boolean shouldApply; + + public DropTable(RandomSource rs, State state) + { + super(rs, state); + Optional<Pair<String, String>> randomTable = state.pickRandomTable(false); + shouldApply = randomTable.isPresent(); + + if (!shouldApply) + return; + + tableToDrop = randomTable.get(); + state.removeTable(tableToDrop.left, tableToDrop.right); + + // if we drop a table, it will make a snapshot with "dropped-" prefix + // hence it will be among snapshot as well, however we do not know its name + // in advance because it contains timestamp in its name produced by Cassandra + // hence we can not add it among "normal" snapshots, hence special "droppedSnapshots" set. + state.addDroppedSnapshot(tableToDrop.left, tableToDrop.right); + } + + @Override + public void doWork(State state) + { + if (!shouldApply) + return; + + Map<String, List<String>> existingSchema = state.getSchema(); + Map<String, List<String>> schemaToBe = state.schema; + + MapDifference<String, List<String>> difference = Maps.difference(schemaToBe, existingSchema); + + for (Entry<String, ValueDifference<List<String>>> entry : difference.entriesDiffering().entrySet()) + { + String keyspaceName = entry.getKey(); + ValueDifference<List<String>> tableNames = entry.getValue(); + + Set<String> left = new HashSet<>(tableNames.leftValue()); + Set<String> right = new HashSet<>(tableNames.rightValue()); + + for (String table : difference(right, left)) + executeQuery(format(DROP_TABLE_QUERY, keyspaceName, table)); + } + + assertEquals(schemaToBe, state.getSchema()); + + state.populate(); + } + + @Override + public String toString() + { + if (tableToDrop != null) + return "Drop table " + format("%s.%s", tableToDrop.left, tableToDrop.right); + else + return "Drop table skipped"; + } + } + + public static class TruncateTable extends AbstractCommand + { + private final Pair<String, String> toTruncate; + + public TruncateTable(RandomSource rs, State state) + { + super(rs, state); + toTruncate = state.pickRandomTable(true).get(); // there is always in-built table to truncate + state.addTruncatedSnapshot(toTruncate.left, toTruncate.right); + } + + @Override + public void doWork(State state) + { + // create missing tables if any + createKeypace(state); + createTable(state); + + executeQuery(format("TRUNCATE TABLE %s.%s", toTruncate.left, toTruncate.right)); + assertEquals(state.truncatedSnapshots, state.getSnapshotsOfTruncatedTables()); + + state.populate(); + } + + @Override + public String toString() + { + return "Truncate table " + format("%s.%s", toTruncate.left, toTruncate.right); + } + } + + public static class TakeSnapshot extends AbstractCommand + { + private final String snapshotToTake; + + public TakeSnapshot(RandomSource rs, State state) + { + super(rs, state); + // there is always in-built table to take snapshot on + Pair<String, String> randomTable = state.pickRandomTable(true).get(); + String tag = state.addSnapshot(randomTable.left, randomTable.right); + snapshotToTake = format("%s.%s.%s", randomTable.left, randomTable.right, tag); + } + + @Override + public void doWork(State state) + { + // see what snapshots are to be taken + Set<TestSnapshot> existing = state.getSnapshots(); + Set<TestSnapshot> toBe = state.snapshots; + + Set<TestSnapshot> diff = difference(toBe, existing); + + assert diff.size() == 1 : "expecting one snapshot to take!"; + + state.takeSnapshots(diff); + assertEquals(toBe, state.getSnapshots()); + state.populate(); + } + + @Override + public String toString() + { + return "Snapshot " + snapshotToTake; + } + } + + public static class ClearSnapshot extends AbstractCommand + { + private static final String DROPPED_SNAPSHOT_PREFIX = "dropped-"; + private static final String TRUNCATED_SNAPSHOT_PREFIX = "truncated-"; + + private final Set<TestSnapshot> normalDiff = new HashSet<>(); + private final Set<String> truncatedDiff = new HashSet<>(); + private final Set<String> droppedDiff = new HashSet<>(); + + public ClearSnapshot(RandomSource rs, State state) + { + super(rs, state); + prepare(); + } + + private void prepare() + { + List<Integer> toShuffle = state.getShuffledListOfInts(3); + + for (int i = 0; i < 3; i++) + { + boolean picked = false; + switch (toShuffle.get(i)) + { + case 1: + if (!state.truncatedSnapshots.isEmpty()) + { + String randomKsTb = state.rs.pick(state.truncatedSnapshots.keySet()); + Integer numberOfTruncatedSnapshots = state.truncatedSnapshots.get(randomKsTb); + if (numberOfTruncatedSnapshots == 1) + state.truncatedSnapshots.remove(randomKsTb); + else + { + int newNumberOfTruncatedTables = state.truncatedSnapshots.get(randomKsTb) - 1; + state.truncatedSnapshots.put(randomKsTb, newNumberOfTruncatedTables); + } + picked = true; + } + break; + case 2: + if (!state.droppedSnapshots.isEmpty()) + { + state.droppedSnapshots.remove(state.rs.pick(state.droppedSnapshots)); + picked = true; + } + break; + case 3: + if (!state.snapshots.isEmpty()) + { + TestSnapshot pickedSnapshot = state.rs.pick(state.snapshots); + state.snapshots.remove(pickedSnapshot); + picked = true; + } + break; + } + if (picked) + break; + } + + Set<TestSnapshot> existingNormal = state.getSnapshots(); + Set<String> existingDropped = state.getSnapshotsOfDroppedTables(); + Map<String, Integer> existingTruncated = state.getSnapshotsOfTruncatedTables(); + + Set<TestSnapshot> normalToBe = state.snapshots; + Set<String> droppedToBe = state.droppedSnapshots; + Map<String, Integer> truncatedToBe = state.truncatedSnapshots; + + normalDiff.addAll(difference(existingNormal, normalToBe)); + truncatedDiff.addAll(difference(existingTruncated.keySet(), truncatedToBe.keySet())); + + Set<String> diff = new HashSet<>(); + + for (Map.Entry<String, Integer> existingTruncatedEntry : existingTruncated.entrySet()) + { + if (truncatedToBe.containsKey(existingTruncatedEntry.getKey())) + { + int toBeCount = truncatedToBe.get(existingTruncatedEntry.getKey()); + int existingCount = existingTruncatedEntry.getValue(); + if (toBeCount < existingCount) + { + diff.add(existingTruncatedEntry.getKey()); + } + } + } + + truncatedDiff.addAll(diff); + + droppedDiff.addAll(difference(existingDropped, droppedToBe)); + } + + @Override + public void doWork(State state) + { + clearSnapshot(state, normalDiff); + clearSnapshot(state, truncatedDiff, TRUNCATED_SNAPSHOT_PREFIX); + clearSnapshot(state, droppedDiff, DROPPED_SNAPSHOT_PREFIX); + + assertEquals(state.snapshots, state.getSnapshots()); + assertEquals(state.droppedSnapshots, state.getSnapshotsOfDroppedTables()); + + assertEquals(state.truncatedSnapshots, state.getSnapshotsOfTruncatedTables()); + + state.populate(); + } + + @Override + public String toString() + { + if (!normalDiff.isEmpty()) + return "Clear snapshot(s) " + normalDiff; + else if (!truncatedDiff.isEmpty()) + return "Clear snapshot(s) " + truncatedDiff; + else if (!droppedDiff.isEmpty()) + return "Clear snapshots(s) " + droppedDiff; + else + return "Clear snapshots"; + } + + private void clearSnapshot(State state, Set<String> toRemove, String prefix) + { + for (String snapshot : toRemove) + { + String[] split = snapshot.split("\\."); + state.getNode().acceptsOnInstance((SerializableTriConsumer<String, String, String>) (ks, tb, pref) -> + { + List<TableSnapshot> selectedSnapshots = SnapshotManager.instance.getSnapshots(s -> s.getKeyspaceName().equals(ks) && + s.getTableName().equals(tb) && + s.getTag().contains(pref)); + if (!selectedSnapshots.isEmpty()) + { + TableSnapshot selectedSnapshotForRemoval = selectedSnapshots.get(0); + SnapshotManager.instance.clearSnapshot(s -> s.equals(selectedSnapshotForRemoval)); + } + }).accept(split[0], split[1], prefix); + } + } + + private void clearSnapshot(State state, Set<TestSnapshot> snapshotsToClear) + { + for (TestSnapshot snapshot : snapshotsToClear) + { + state.getNode() + .acceptsOnInstance((SerializableTriConsumer<String, String, String>) + (ks, tb, tag) -> SnapshotManager.instance.clearSnapshot(ks, tb, tag)) + .accept(snapshot.getKeyspaceName(), snapshot.getTableName(), snapshot.getTag()); + } + } + } + + public static class State + { + private final Cluster.Builder builder = Cluster.build(1).withConfig(c -> c.with(Feature.NATIVE_PROTOCOL)); + private Cluster cluster; + + public Map<String, List<String>> schema = new HashMap<>(); + public Set<TestSnapshot> snapshots = new HashSet<>(); + public Set<String> droppedSnapshots = new HashSet<>(); + public Map<String, Integer> truncatedSnapshots = new HashMap<>(); + + public final RandomSource rs; + public final RandomnessSource randomnessSource; + + public String inBuiltKeyspace; + public String inBuiltTable; + + public State(RandomSource rs) + { + this.rs = rs; + this.randomnessSource = new JavaRandom(rs.asJdkRandom()); + start(); + populateSchema(); + populate(); + } + + public State populate() + { + this.schema = getSchema(); + this.snapshots = getSnapshots(); + this.droppedSnapshots = getSnapshotsOfDroppedTables(); + this.truncatedSnapshots = getSnapshotsOfTruncatedTables(); + return this; + } + + private void populateSchema() + { + new CreateKeyspace(rs, this).applyUnit(this); + new CreateTable(rs, this).applyUnit(this); + + this.inBuiltKeyspace = this.schema.keySet().iterator().next(); + this.inBuiltTable = this.schema.values().iterator().next().get(0); + } + + @Override + public String toString() + { + return "State{" + + "schema=" + schema + + ", snapshots=" + snapshots + + ", droppedSnapshots=" + droppedSnapshots + + ", truncatedSnapshots=" + truncatedSnapshots + + '}'; + } + + public Optional<String> pickRandomKeyspace(boolean inBuiltIncluded) + { + if (inBuiltIncluded) + return pickRandomKeyspace(); + + Set<String> withoutInBuilt = new HashSet<>(schema.keySet()); + withoutInBuilt.remove(inBuiltKeyspace); + + if (withoutInBuilt.isEmpty()) + return Optional.empty(); + else + return Optional.of(rs.pick(withoutInBuilt)); + } + + public Optional<String> pickRandomKeyspace() + { + if (schema.keySet().isEmpty()) + return Optional.empty(); + else + return Optional.of(rs.pick(schema.keySet())); + } + + public Optional<Pair<String, String>> pickRandomTable(boolean inBuiltIncluded) + { + Map<String, List<String>> keyspacesWithTables = new HashMap<>(); + for (Map.Entry<String, List<String>> entry : schema.entrySet()) + { + if (!entry.getValue().isEmpty()) + { + if (inBuiltIncluded) + { + keyspacesWithTables.put(entry.getKey(), entry.getValue()); + } + else + { + if (entry.getKey().equals(inBuiltKeyspace)) + { + List<String> tablesInInBuiltKeyspace = new ArrayList<>(); + for (String tableInInBuiltKeyspace : entry.getValue()) + { + if (!tableInInBuiltKeyspace.equals(inBuiltTable)) + tablesInInBuiltKeyspace.add(tableInInBuiltKeyspace); + } + if (!tablesInInBuiltKeyspace.isEmpty()) + keyspacesWithTables.put(entry.getKey(), tablesInInBuiltKeyspace); + } + else + { + if (!entry.getValue().isEmpty()) + keyspacesWithTables.put(entry.getKey(), entry.getValue()); + } + } + } + } + + if (keyspacesWithTables.isEmpty()) + return Optional.empty(); + + String randomKeyspaceWithTables = rs.pick(keyspacesWithTables.keySet()); + List<String> tables = keyspacesWithTables.get(randomKeyspaceWithTables); + String randomTable = rs.pick(tables); + + return Optional.of(Pair.create(randomKeyspaceWithTables, randomTable)); + } + + public String addRandomKeyspace() + { + String keyspace = createRandomKeyspaceIdentifier(); + addKeyspace(keyspace); + return keyspace; + } + + public Pair<String, String> addRandomTable() + { + Optional<String> randomKeyspace = pickRandomKeyspace(); + + assert randomKeyspace.isPresent(); + + String randomTableName = createRandomTableIdentifier(); + addTable(randomKeyspace.get(), randomTableName); + return Pair.create(randomKeyspace.get(), randomTableName); + } + + public Optional<Pair<String, List<String>>> removeRandomKeyspace() + { + Optional<String> randomKeyspace = pickRandomKeyspace(false); + + if (randomKeyspace.isEmpty()) + return Optional.empty(); + + return randomKeyspace.map(this::removeKeyspace); + } + + public void addKeyspace(String keyspaceName) + { + schema.put(keyspaceName, new ArrayList<>()); + } + + public Pair<String, List<String>> removeKeyspace(String keyspaceName) + { + List<String> removedTables = schema.remove(keyspaceName); + return Pair.create(keyspaceName, removedTables); + } + + public void addTable(String keyspaceName, String tableName) + { + List<String> tables = schema.get(keyspaceName); + if (tables == null) + tables = new ArrayList<>(); + + tables.add(tableName); + schema.put(keyspaceName, tables); + } + + public void removeTable(String keyspace, String table) + { + List<String> tables = schema.get(keyspace); + if (tables == null) + return; + + tables.remove(table); + } + + public String addSnapshot(String keyspace, String table) + { + String randomSnapshotIdentifier = createRandomSnapshotIdentifier(); + addSnapshot(new TestSnapshot(keyspace, table, randomSnapshotIdentifier)); + + return randomSnapshotIdentifier; + } + + public void addDroppedSnapshot(String keyspace, String table) + { + droppedSnapshots.add(keyspace + '.' + table); + } + + public void addTruncatedSnapshot(String keyspace, String table) + { + truncatedSnapshots.merge(keyspace + '.' + table, 1, Integer::sum); + } + + public void addSnapshot(TableSnapshot snapshot) + { + snapshots.add(new TestSnapshot(snapshot.getKeyspaceName(), snapshot.getTableName(), snapshot.getTag())); + } + + public void destroy() + { + snapshots.clear(); + droppedSnapshots.clear(); + truncatedSnapshots.clear(); + schema.clear(); + + stop(); + } + + public List<Integer> getShuffledListOfInts(int number) + { + List<Integer> toShuffle = new ArrayList<>(); + + for (int i = 0; i < number; i++) + toShuffle.add(i); + + Collections.shuffle(toShuffle, rs.asJdkRandom()); + + return toShuffle; + } + + // we just need something to store in the state and whole thing is too much and not necessary + public static class TestSnapshot extends TableSnapshot implements Comparable<TestSnapshot> + { + public TestSnapshot(String keyspaceName, String tableName, String tag) + { + this(keyspaceName, tableName, randomUUID(), tag, null, null, null, false); + } + + public TestSnapshot(String keyspaceName, String tableName, UUID tableId, String tag, + Instant createdAt, Instant expiresAt, Set<File> snapshotDirs, boolean ephemeral) + { + super(keyspaceName, tableName, tableId, tag, createdAt, expiresAt, snapshotDirs, ephemeral); + } + + @Override + public long getManifestsSize() + { + return 0; + } + + @Override + public long getSchemasSize() + { + return 0; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + TableSnapshot snapshot = (TableSnapshot) o; + return Objects.equals(getKeyspaceName(), snapshot.getKeyspaceName()) && + Objects.equals(getTableName(), snapshot.getTableName()) && + Objects.equals(getTag(), snapshot.getTag()); + } + + @Override + public int hashCode() + { + return Objects.hash(getKeyspaceName(), getTableName(), getTag()); + } + + @Override + public String toString() + { + return format("%s.%s.%s", getKeyspaceName(), getTableName(), getTag()); + } + + @Override + public int compareTo(TestSnapshot o) + { + return toString().compareTo(o.toString()); + } + } + + + private String createRandomKeyspaceIdentifier() + { + return trimIfNecessary("keyspace_" + getRandomString()); + } + + private String createRandomSnapshotIdentifier() + { + return trimIfNecessary("snapshot_" + getRandomString()); + } + + private String createRandomTableIdentifier() + { + return trimIfNecessary("table_" + getRandomString()); + } + + private String getRandomString() + { + return LocalizeString.toLowerCaseLocalized(Generators.regexWord(SourceDSL.integers().between(10, 50)).generate(randomnessSource)); + } + + private String trimIfNecessary(String maybeTooLongString) + { + if (maybeTooLongString.length() <= 48) + return maybeTooLongString; + + return maybeTooLongString.substring(0, 48); + } + + // taken from SUT + + public Map<String, List<String>> getSchema() + { + IIsolatedExecutor.SerializableCallable<Map<String, List<String>>> callable = () -> { + Keyspaces keyspaces = Schema.instance.distributedKeyspaces(); + + Map<String, List<String>> keyspacesWithTables = new HashMap<>(); + + for (KeyspaceMetadata ksm : keyspaces) + { + if (ksm.name.startsWith("system_")) + continue; + + List<String> tables = new ArrayList<>(); + for (TableMetadata tmd : ksm.tables) + tables.add(tmd.name); + + keyspacesWithTables.put(ksm.name, tables); + } + + return keyspacesWithTables; + }; + + return getNode().callOnInstance(callable); + } + + public Set<String> getKeyspaces() + { + return getSchema().keySet(); + } + + public Set<String> getSnapshotsOfDroppedTables() + { + SerializableFunction<String, Set<String>> callable = (p) -> { + Set<String> snapshots = new HashSet<>(); + for (TableSnapshot snapshot : SnapshotManager.instance.getSnapshots(snapshot -> snapshot.getTag().contains(p))) + snapshots.add(format("%s.%s", snapshot.getKeyspaceName(), snapshot.getTableName())); + + return snapshots; + }; + + return getNode().applyOnInstance(callable, "dropped-"); + } + + public Map<String, Integer> getSnapshotsOfTruncatedTables() + { + SerializableFunction<String, Map<String, Integer>> callable = (p) -> { + Map<String, Integer> snapshots = new HashMap<>(); + for (TableSnapshot snapshot : SnapshotManager.instance.getSnapshots(snapshot -> snapshot.getTag().contains(p))) + { + String ksTb = format("%s.%s", snapshot.getKeyspaceName(), snapshot.getTableName()); + snapshots.merge(ksTb, 1, Integer::sum); + } + + return snapshots; + }; + + return getNode().applyOnInstance(callable, "truncated-"); + } + + // these are not "dropped" nor "truncated" + public Set<TestSnapshot> getSnapshots() + { + Set<TestSnapshot> existingSnapshots = new HashSet<>(); + + IIsolatedExecutor.SerializableCallable<List<String>> callable = () -> { + List<String> snapshots = new ArrayList<>(); + for (TableSnapshot snapshot : SnapshotManager.instance.getSnapshots(p -> { + String tag = p.getTag(); + return !tag.contains("dropped-") && !tag.contains("truncated-"); + })) + { + snapshots.add(format("%s.%s.%s", snapshot.getKeyspaceName(), snapshot.getTableName(), snapshot.getTag())); + } + + return snapshots; + }; + + for (String tableSnapshot : getNode().callOnInstance(callable)) + { + String[] components = tableSnapshot.split("\\."); + existingSnapshots.add(new TestSnapshot(components[0], + components[1], + components[2])); + } + + return existingSnapshots; + } + + public void takeSnapshots(Set<TestSnapshot> snapshotsToTake) + { + for (TestSnapshot toTake : snapshotsToTake) + getNode().acceptsOnInstance((SerializableTriConsumer<String, String, String>) (tag, ks, tb) -> { + try + { + SnapshotManager.instance.takeSnapshot(tag, Map.of(), ks + '.' + tb); + } + catch (IOException ex) + { + throw new RuntimeException(ex); + } + }) + .accept(toTake.getTag(), toTake.getKeyspaceName(), toTake.getTableName()); + } + + public IInvokableInstance getNode() + { + if (cluster == null) + throw new RuntimeException("not started yet"); + + return cluster.get(1); + } + + public void executeQuery(String query) + { + getNode().executeInternal(query); + } + + public State start() + { + if (cluster != null) + return null; + + try + { + cluster = builder.start(); + } + catch (IOException e) + { + throw new RuntimeException(e); + } + + schema = getSchema(); + + return this; + } + + public void stop() + { + if (cluster != null) + { + try + { + cluster.close(); + } + catch (Throwable t) + { + // ignore + } + } + } + + public NodeToolResult nodetool(String... nodetoolArgs) + { + return getNode().nodetoolResult(nodetoolArgs); + } + } + + public static abstract class AbstractCommand implements StateOnlyCommand<State> + { + protected final RandomSource rs; + protected final State state; + + public AbstractCommand(RandomSource rs, State state) + { + this.rs = rs; + this.state = state; + } + + public void executeQuery(String query) + { + state.getNode().executeInternal(query); + } + + public IInvokableInstance getNode() + { + return state.getNode(); + } + + @Override + public void applyUnit(State state) + { + Uninterruptibles.sleepUninterruptibly(Generators.TINY_TIME_SPAN_NANOS.generate(state.randomnessSource), TimeUnit.NANOSECONDS); + doWork(state); + } + + public abstract void doWork(State state); + } +} diff --git a/test/microbench/org/apache/cassandra/test/microbench/AbstractSnapshotManagerBase.java b/test/microbench/org/apache/cassandra/test/microbench/AbstractSnapshotManagerBase.java new file mode 100644 index 000000000000..0f5bb8808d6c --- /dev/null +++ b/test/microbench/org/apache/cassandra/test/microbench/AbstractSnapshotManagerBase.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.test.microbench; + +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.UUID; + +import org.apache.commons.lang3.tuple.Pair; + +import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.Util; +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.config.Config; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.RowUpdateBuilder; +import org.apache.cassandra.db.commitlog.CommitLog; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.snapshot.SnapshotManager; +import org.apache.cassandra.service.snapshot.TableSnapshot; +import org.apache.cassandra.tcm.ClusterMetadataService; + +import static org.apache.cassandra.service.snapshot.SnapshotOptions.userSnapshot; +import static org.junit.Assert.assertEquals; + +public abstract class AbstractSnapshotManagerBase +{ + static final String KEYSPACE = "KEYSPACE"; + + private final Random random = new Random(); + + static int NUM_SSTABLES = 10; + static int NUM_KEYSPACES = 5; + static int NUM_TABLES_PER_KEYSPACE = 10; + static int NUM_SNAPSHOTS_PER_TABLE = 10; + + public void teardown() + { + SnapshotManager.instance.clearAllSnapshots(); + CQLTester.tearDownClass(); + CommitLog.instance.stopUnsafe(true); + // TODO CASSANDRA-20119 + ClusterMetadataService.instance().log().close(); + CQLTester.cleanup(); + } + + public void setup(boolean takeSnapshotAfterTablePopulation) + { + CassandraRelevantProperties.SNAPSHOT_CLEANUP_INITIAL_DELAY_SECONDS.getInt(1000); + CassandraRelevantProperties.SNAPSHOT_CLEANUP_PERIOD_SECONDS.setInt(60); + + DatabaseDescriptor.daemonInitialization(() -> { + Config config = DatabaseDescriptor.loadConfig(); + config.dump_heap_on_uncaught_exception = false; + return config; + }); + + SchemaLoader.prepareServer(); + SnapshotManager.instance.start(true); + + populateTables(takeSnapshotAfterTablePopulation); + } + + public void populateTables(boolean takeSnapshotAfterTablePopulation) + { + for (int i = 0; i < NUM_KEYSPACES; i++) + { + String keyspaceName = KEYSPACE + '_' + i; + + // Create Schema + TableMetadata[] tables = new TableMetadata[NUM_TABLES_PER_KEYSPACE]; + for (int j = 0; j < NUM_TABLES_PER_KEYSPACE; j++) + { + tables[j] = SchemaLoader.standardCFMD(keyspaceName, String.format("table%d", i + '_' + j)).build(); + } + + SchemaLoader.createKeyspace(keyspaceName, + KeyspaceParams.simple(1), + tables); + + for (int j = 0; j < NUM_TABLES_PER_KEYSPACE; j++) + { + String tableName = String.format("table%d", i + '_' + j); + + ColumnFamilyStore cfs = Keyspace.open(keyspaceName).getColumnFamilyStore(tableName); + cfs.disableAutoCompaction(); + for (int k = 0; k < NUM_SSTABLES; k++) + { + new RowUpdateBuilder(cfs.metadata(), 0, "key1") + .clustering("Column1") + .add("val", "asdf") + .build() + .applyUnsafe(); + Util.flush(cfs); + } + if (takeSnapshotAfterTablePopulation) + { + for (int snapId = 0; snapId < NUM_SNAPSHOTS_PER_TABLE; snapId++) + SnapshotManager.instance.takeSnapshot(userSnapshot(String.format("snap_%d_%d_%d", i, j, snapId), cfs.getKeyspaceTableName())); + } + } + } + + if (takeSnapshotAfterTablePopulation) + { + List<TableSnapshot> snapshots = SnapshotManager.instance.getSnapshots(t -> true); + assertEquals(NUM_KEYSPACES * NUM_TABLES_PER_KEYSPACE * NUM_SNAPSHOTS_PER_TABLE, snapshots.size()); + } + } + + public String getRandomKeyspaceTable() + { + List<Keyspace> keyspaces = new ArrayList<>(); + Keyspace.nonSystem().forEach(keyspaces::add); + Keyspace randomKeyspace = keyspaces.get(random.nextInt(keyspaces.size())); + List<ColumnFamilyStore> cfss = new ArrayList<>(randomKeyspace.getColumnFamilyStores()); + ColumnFamilyStore cfs = cfss.get(random.nextInt(cfss.size())); + + return cfs.getKeyspaceTableName(); + } + + public String getRandomKeyspace() + { + List<Keyspace> keyspaces = new ArrayList<>(); + Keyspace.nonSystem().forEach(keyspaces::add); + Keyspace randomKeyspace = keyspaces.get(random.nextInt(keyspaces.size())); + return randomKeyspace.getName(); + } + + public Pair<String, String> getRandomKeyspaceTablePair() + { + List<Keyspace> keyspaces = new ArrayList<>(); + Keyspace.nonSystem().forEach(keyspaces::add); + Keyspace randomKeyspace = keyspaces.get(random.nextInt(keyspaces.size())); + List<ColumnFamilyStore> cfss = new ArrayList<>(randomKeyspace.getColumnFamilyStores()); + ColumnFamilyStore cfs = cfss.get(random.nextInt(cfss.size())); + + return Pair.of(cfs.getKeyspaceName(), cfs.getTableName()); + } + + + public void takeSnapshotOnRandomTable() + { + SnapshotManager.instance.takeSnapshot(userSnapshot("snap_" + UUID.randomUUID(), getRandomKeyspaceTable())); + } +} diff --git a/test/microbench/org/apache/cassandra/test/microbench/CompactionBench.java b/test/microbench/org/apache/cassandra/test/microbench/CompactionBench.java index 8d7e800755cc..edbe249b2eca 100644 --- a/test/microbench/org/apache/cassandra/test/microbench/CompactionBench.java +++ b/test/microbench/org/apache/cassandra/test/microbench/CompactionBench.java @@ -30,6 +30,7 @@ import org.apache.cassandra.io.sstable.Descriptor; import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.service.snapshot.SnapshotManager; import org.openjdk.jmh.annotations.*; @BenchmarkMode(Mode.AverageTime) @@ -78,8 +79,7 @@ public void setup() throws Throwable cfs.forceBlockingFlush(ColumnFamilyStore.FlushReason.USER_FORCED); - cfs.snapshot("originals"); - + SnapshotManager.instance.takeSnapshot("originals", cfs.getKeyspaceTableName()); snapshotFiles = cfs.getDirectories().sstableLister(Directories.OnTxnErr.IGNORE).snapshots("originals").listFiles(); } diff --git a/test/microbench/org/apache/cassandra/test/microbench/SnapshotListingBench.java b/test/microbench/org/apache/cassandra/test/microbench/SnapshotListingBench.java new file mode 100644 index 000000000000..92d687159a96 --- /dev/null +++ b/test/microbench/org/apache/cassandra/test/microbench/SnapshotListingBench.java @@ -0,0 +1,63 @@ +package org.apache.cassandra.test.microbench; +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.util.concurrent.TimeUnit; + +import org.apache.cassandra.service.snapshot.SnapshotManager; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; + +@OutputTimeUnit(TimeUnit.MILLISECONDS) +@Fork(value = 1) +@State(Scope.Benchmark) +public class SnapshotListingBench extends AbstractSnapshotManagerBase +{ + @Setup(Level.Trial) + public void setup() + { + setup(true); + } + + @TearDown(Level.Trial) + public void teardown() + { + super.teardown(); + } + + @Benchmark + @BenchmarkMode(Mode.All) + @Warmup(iterations = 0) + @Threads(4) + @Measurement(iterations = 1, time = 60) + public void listingTest() + { + SnapshotManager.instance.listSnapshots(null); + } +} diff --git a/test/microbench/org/apache/cassandra/test/microbench/SnapshotTakingBench.java b/test/microbench/org/apache/cassandra/test/microbench/SnapshotTakingBench.java new file mode 100644 index 000000000000..4631905eb05b --- /dev/null +++ b/test/microbench/org/apache/cassandra/test/microbench/SnapshotTakingBench.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.test.microbench; + +import java.util.concurrent.TimeUnit; + +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; + +@OutputTimeUnit(TimeUnit.MILLISECONDS) +@Fork(value = 1) +@State(Scope.Benchmark) +public class SnapshotTakingBench extends AbstractSnapshotManagerBase +{ + + @Setup(Level.Trial) + public void setup() + { + setup(false); + } + + @TearDown(Level.Trial) + public void teardown() + { + super.teardown(); + } + + @Benchmark + @BenchmarkMode(Mode.All) + @Warmup(iterations = 0) + @Threads(4) + @Measurement(iterations = 1, time = 60) + public void takingSnapshotTest() + { + takeSnapshotOnRandomTable(); + } +} diff --git a/test/microbench/org/apache/cassandra/test/microbench/SnapshotTrueSizeBench.java b/test/microbench/org/apache/cassandra/test/microbench/SnapshotTrueSizeBench.java new file mode 100644 index 000000000000..b8d9ffe23cce --- /dev/null +++ b/test/microbench/org/apache/cassandra/test/microbench/SnapshotTrueSizeBench.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.test.microbench; + +import java.util.concurrent.TimeUnit; + +import org.apache.cassandra.service.snapshot.SnapshotManager; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; + +@OutputTimeUnit(TimeUnit.MILLISECONDS) +@Fork(value = 1) +@State(Scope.Benchmark) +public class SnapshotTrueSizeBench extends AbstractSnapshotManagerBase +{ + + @Setup(Level.Trial) + public void setup() + { + setup(true); + } + + @TearDown(Level.Trial) + public void teardown() + { + super.teardown(); + } + + @Benchmark + @BenchmarkMode({Mode.Throughput, Mode.AverageTime, Mode.SampleTime}) + @Warmup(iterations = 0) + @Threads(16) + @Measurement(iterations = 1, time = 60) + public void getTrueSizeOfSnapshots() + { + SnapshotManager.instance.getTrueSnapshotSize(); + } +} diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java index be883ab04769..590f7e356f12 100644 --- a/test/unit/org/apache/cassandra/Util.java +++ b/test/unit/org/apache/cassandra/Util.java @@ -35,8 +35,10 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.UUID; @@ -133,6 +135,8 @@ import org.apache.cassandra.schema.TableMetadataRef; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.service.pager.PagingState; +import org.apache.cassandra.service.snapshot.SnapshotLoader; +import org.apache.cassandra.service.snapshot.TableSnapshot; import org.apache.cassandra.streaming.StreamResultFuture; import org.apache.cassandra.streaming.StreamState; import org.apache.cassandra.tcm.ClusterMetadata; @@ -147,7 +151,6 @@ import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.FilterFactory; import org.apache.cassandra.utils.OutputHandler; -import org.apache.cassandra.utils.Throwables; import org.awaitility.Awaitility; import org.hamcrest.Matcher; import org.mockito.Mockito; @@ -1170,23 +1173,6 @@ public static Set<Descriptor> getSSTables(String ks, String tableName) .collect(Collectors.toSet()); } - public static Set<Descriptor> getSnapshots(String ks, String tableName, String snapshotTag) - { - try - { - return Keyspace.open(ks) - .getColumnFamilyStore(tableName) - .getSnapshotSSTableReaders(snapshotTag) - .stream() - .map(sstr -> sstr.descriptor) - .collect(Collectors.toSet()); - } - catch (IOException e) - { - throw Throwables.unchecked(e); - } - } - public static Set<Descriptor> getBackups(String ks, String tableName) { return Keyspace.open(ks) @@ -1283,4 +1269,16 @@ public static RuntimeException testMustBeImplementedForSSTableFormat() { return new UnsupportedOperationException("Test must be implemented for sstable format " + DatabaseDescriptor.getSelectedSSTableFormat().getClass().getName()); } + + public static Map<String, TableSnapshot> listSnapshots(ColumnFamilyStore cfs) + { + Set<TableSnapshot> snapshots = new SnapshotLoader(cfs.getDirectories()).loadSnapshots(); + Map<String, TableSnapshot> tagSnapshotsMap = new HashMap<>(); + + for (TableSnapshot snapshot : snapshots) + tagSnapshotsMap.put(snapshot.getTag(), snapshot); + + return tagSnapshotsMap; + } + } diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index 54171f9a0885..2f5857ae9792 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -172,6 +172,7 @@ import org.apache.cassandra.service.ClientState; import org.apache.cassandra.service.QueryState; import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.transport.Event; import org.apache.cassandra.transport.Message; @@ -445,6 +446,7 @@ protected static void prePrepareServer() DatabaseDescriptor.setRowCacheSizeInMiB(ROW_CACHE_SIZE_IN_MIB); StorageService.instance.registerMBeans(); StorageService.instance.setPartitionerUnsafe(Murmur3Partitioner.instance); + SnapshotManager.instance.registerMBean(); } @AfterClass diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AutoSnapshotTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AutoSnapshotTest.java index eedcdb0a9a3d..bcb0be6440ea 100644 --- a/test/unit/org/apache/cassandra/cql3/validation/operations/AutoSnapshotTest.java +++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AutoSnapshotTest.java @@ -30,21 +30,28 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import org.apache.cassandra.Util; +import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.service.snapshot.SnapshotType; import org.apache.cassandra.service.snapshot.TableSnapshot; import org.assertj.core.api.Condition; import static java.util.concurrent.TimeUnit.SECONDS; import static java.lang.String.format; -import static org.apache.cassandra.db.ColumnFamilyStore.SNAPSHOT_DROP_PREFIX; import static org.assertj.core.api.Assertions.assertThat; @RunWith(Parameterized.class) public class AutoSnapshotTest extends CQLTester { + static + { + CassandraRelevantProperties.SNAPSHOT_MIN_ALLOWED_TTL_SECONDS.setInt(1); + } + static int TTL_SECS = 1; public static Boolean enabledBefore; @@ -97,7 +104,7 @@ public void testAutoSnapshotOnTrucate() throws Throwable createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY(a, b))"); // Check there are no snapshots ColumnFamilyStore tableDir = getCurrentColumnFamilyStore(); - assertThat(tableDir.listSnapshots()).isEmpty(); + assertThat(Util.listSnapshots(tableDir)).isEmpty(); execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0); execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 1); @@ -106,7 +113,7 @@ public void testAutoSnapshotOnTrucate() throws Throwable execute("DROP TABLE %s"); - verifyAutoSnapshot(SNAPSHOT_DROP_PREFIX, tableDir, currentTable()); + verifyAutoSnapshot(SnapshotType.DROP.label, tableDir, currentTable()); } @Test @@ -115,7 +122,7 @@ public void testAutoSnapshotOnDrop() throws Throwable createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY(a, b))"); // Check there are no snapshots ColumnFamilyStore tableDir = getCurrentColumnFamilyStore(); - assertThat(tableDir.listSnapshots()).isEmpty(); + assertThat(Util.listSnapshots(tableDir)).isEmpty(); execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0); execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 1); @@ -124,7 +131,7 @@ public void testAutoSnapshotOnDrop() throws Throwable execute("DROP TABLE %s"); - verifyAutoSnapshot(SNAPSHOT_DROP_PREFIX, tableDir, currentTable()); + verifyAutoSnapshot(SnapshotType.DROP.label, tableDir, currentTable()); } @Test @@ -136,13 +143,13 @@ public void testAutoSnapshotOnDropKeyspace() throws Throwable flush(); // Check no snapshots - assertThat(tableA.listSnapshots()).isEmpty(); - assertThat(tableB.listSnapshots()).isEmpty(); + assertThat(Util.listSnapshots(tableA)).isEmpty(); + assertThat(Util.listSnapshots(tableB)).isEmpty(); // Drop keyspace, should have snapshot for table A and B execute(format("DROP KEYSPACE %s", keyspace())); - verifyAutoSnapshot(SNAPSHOT_DROP_PREFIX, tableA, tableA.name); - verifyAutoSnapshot(SNAPSHOT_DROP_PREFIX, tableB, tableB.name); + verifyAutoSnapshot(SnapshotType.DROP.label, tableA, tableA.name); + verifyAutoSnapshot(SnapshotType.DROP.label, tableB, tableB.name); } private ColumnFamilyStore createAndPopulateTable() throws Throwable @@ -163,11 +170,11 @@ private ColumnFamilyStore createAndPopulateTable() throws Throwable */ private void verifyAutoSnapshot(String snapshotPrefix, ColumnFamilyStore tableDir, String expectedTableName) { - Map<String, TableSnapshot> snapshots = tableDir.listSnapshots(); + Map<String, TableSnapshot> snapshots = Util.listSnapshots(tableDir); if (autoSnapshotEnabled) { assertThat(snapshots).hasSize(1); - assertThat(snapshots).hasKeySatisfying(new Condition<>(k -> k.startsWith(snapshotPrefix), "is dropped snapshot")); + assertThat(snapshots).hasKeySatisfying(new Condition<>(k -> k.contains(snapshotPrefix), "is dropped snapshot")); TableSnapshot snapshot = snapshots.values().iterator().next(); assertThat(snapshot.getTableName()).isEqualTo(expectedTableName); if (autoSnapshotTTl == null) diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java index 4de0186feac9..ba08437cd658 100644 --- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java +++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java @@ -1,21 +1,21 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one -* or more contributor license agreements. See the NOTICE file -* distributed with this work for additional information -* regarding copyright ownership. The ASF licenses this file -* to you under the Apache License, Version 2.0 (the -* "License"); you may not use this file except in compliance -* with the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, -* software distributed under the License is distributed on an -* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -* KIND, either express or implied. See the License for the -* specific language governing permissions and limitations -* under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.cassandra.db; import java.io.IOException; @@ -23,18 +23,19 @@ import java.nio.file.Path; import java.nio.file.Paths; import java.util.Collection; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Iterators; +import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; @@ -44,7 +45,9 @@ import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.UpdateBuilder; import org.apache.cassandra.Util; +import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.cql3.statements.schema.IndexTarget; import org.apache.cassandra.db.ColumnFamilyStore.FlushReason; import org.apache.cassandra.db.commitlog.CommitLogPosition; import org.apache.cassandra.db.filter.ColumnFilter; @@ -59,7 +62,9 @@ import org.apache.cassandra.db.rows.EncodingStats; import org.apache.cassandra.db.rows.Row; import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.index.Index; import org.apache.cassandra.index.transactions.UpdateTransaction; import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.Descriptor; @@ -71,11 +76,17 @@ import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.metrics.ClearableHistogram; import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.IndexMetadata; +import org.apache.cassandra.schema.Indexes; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.schema.SchemaTestUtil; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.reads.SpeculativeRetryPolicy; +import org.apache.cassandra.service.snapshot.SnapshotOptions; +import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.service.snapshot.SnapshotManifest; +import org.apache.cassandra.service.snapshot.SnapshotType; import org.apache.cassandra.service.snapshot.TableSnapshot; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.FBUtilities; @@ -83,6 +94,10 @@ import org.apache.cassandra.utils.concurrent.OpOrder.Barrier; import org.apache.cassandra.utils.concurrent.OpOrder.Group; +import static org.apache.cassandra.cql3.statements.schema.IndexTarget.Type.VALUES; +import static org.apache.cassandra.schema.IndexMetadata.Kind.COMPOSITES; +import static org.apache.cassandra.schema.IndexMetadata.Kind.CUSTOM; +import static org.apache.cassandra.schema.IndexMetadata.fromIndexTargets; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -121,14 +136,27 @@ public static void defineSchema() throws ConfigurationException @Before public void truncateCFS() { - Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1).truncateBlocking(); - Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD2).truncateBlocking(); - Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_INDEX1).truncateBlocking(); - Keyspace.open(KEYSPACE2).getColumnFamilyStore(CF_STANDARD1).truncateBlocking(); + Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1).truncateBlockingWithoutSnapshot(); + Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD2).truncateBlockingWithoutSnapshot(); + Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_INDEX1).truncateBlockingWithoutSnapshot(); + Keyspace.open(KEYSPACE2).getColumnFamilyStore(CF_STANDARD1).truncateBlockingWithoutSnapshot(); + Keyspace.open(KEYSPACE3).getColumnFamilyStore(CF_SPEC_RETRY1).truncateBlockingWithoutSnapshot(); + + SnapshotManager.instance.clearAllSnapshots(KEYSPACE1, CF_STANDARD1); + SnapshotManager.instance.clearAllSnapshots(KEYSPACE1, CF_STANDARD2); + SnapshotManager.instance.clearAllSnapshots(KEYSPACE1, CF_INDEX1); + SnapshotManager.instance.clearAllSnapshots(KEYSPACE2, CF_STANDARD1); + SnapshotManager.instance.clearAllSnapshots(KEYSPACE3, CF_SPEC_RETRY1); + } + + @After + public void afterTest() + { + SnapshotManager.instance.clearAllSnapshots(); } @Test - public void testMemtableTimestamp() throws Throwable + public void testMemtableTimestamp() { ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1); assertEquals(Memtable.NO_MIN_TIMESTAMP, fakeMemTableWithMinTS(cfs, EncodingStats.NO_STATS.minTimestamp).getMinTimestamp()); @@ -142,20 +170,20 @@ public void testTimeSortedQuery() ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1); new RowUpdateBuilder(cfs.metadata(), 0, "key1") - .clustering("Column1") - .add("val", "asdf") - .build() - .applyUnsafe(); + .clustering("Column1") + .add("val", "asdf") + .build() + .applyUnsafe(); Util.flush(cfs); new RowUpdateBuilder(cfs.metadata(), 1, "key1") - .clustering("Column1") - .add("val", "asdf") - .build() - .applyUnsafe(); + .clustering("Column1") + .add("val", "asdf") + .build() + .applyUnsafe(); Util.flush(cfs); - ((ClearableHistogram)cfs.metric.sstablesPerReadHistogram.cf).clear(); // resets counts + ((ClearableHistogram) cfs.metric.sstablesPerReadHistogram.cf).clear(); // resets counts Util.getAll(Util.cmd(cfs, "key1").includeRow("c1").build()); assertEquals(1, cfs.metric.sstablesPerReadHistogram.cf.getCount()); } @@ -165,7 +193,7 @@ public void testGetColumnWithWrongBF() { Keyspace keyspace = Keyspace.open(KEYSPACE1); ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1); - keyspace.getColumnFamilyStores().forEach(ColumnFamilyStore::truncateBlocking); + keyspace.getColumnFamilyStores().forEach(ColumnFamilyStore::truncateBlockingWithoutSnapshot); List<Mutation> rms = new LinkedList<>(); rms.add(new RowUpdateBuilder(cfs.metadata(), 0, "key1") @@ -195,7 +223,7 @@ public void runMayThrow() throws IOException { Row toCheck = Util.getOnlyRowUnfiltered(Util.cmd(cfs, "key1").build()); Iterator<Cell<?>> iter = toCheck.cells().iterator(); - assert(Iterators.size(iter) == 0); + assert (Iterators.size(iter) == 0); } }; @@ -265,42 +293,34 @@ public void testClearEphemeralSnapshots() { ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_INDEX1); - //cleanup any previous test gargbage - cfs.clearSnapshot(""); - int numRows = 1000; - long[] colValues = new long [numRows * 2]; // each row has two columns - for (int i = 0; i < colValues.length; i+=2) + long[] colValues = new long[numRows * 2]; // each row has two columns + for (int i = 0; i < colValues.length; i += 2) { colValues[i] = (i % 4 == 0 ? 1L : 2L); // index column - colValues[i+1] = 3L; //other column + colValues[i + 1] = 3L; //other column } ScrubTest.fillIndexCF(cfs, false, colValues); - cfs.snapshot("nonEphemeralSnapshot", null, false, false); - cfs.snapshot("ephemeralSnapshot", null, true, false); + SnapshotManager.instance.takeSnapshot(SnapshotOptions.systemSnapshot("nonEphemeralSnapshot", SnapshotType.MISC, cfs.getKeyspaceTableName()).build()); + SnapshotManager.instance.takeSnapshot(SnapshotOptions.systemSnapshot("ephemeralSnapshot", SnapshotType.REPAIR, cfs.getKeyspaceTableName()).ephemeral().build()); + + assertTrue(SnapshotManager.instance.exists(p -> p.getTag().endsWith("ephemeralSnapshot"))); + assertTrue(SnapshotManager.instance.exists(p -> p.getTag().endsWith("nonEphemeralSnapshot"))); - Map<String, TableSnapshot> snapshotDetails = cfs.listSnapshots(); - assertEquals(2, snapshotDetails.size()); - assertTrue(snapshotDetails.containsKey("ephemeralSnapshot")); - assertTrue(snapshotDetails.containsKey("nonEphemeralSnapshot")); + SnapshotManager.instance.clearEphemeralSnapshots(); - ColumnFamilyStore.clearEphemeralSnapshots(cfs.getDirectories()); + List<TableSnapshot> snapshots = SnapshotManager.instance.getSnapshots(p -> true); + assertEquals(1, snapshots.size()); - snapshotDetails = cfs.listSnapshots(); - assertEquals(1, snapshotDetails.size()); - assertTrue(snapshotDetails.containsKey("nonEphemeralSnapshot")); - //test cleanup - cfs.clearSnapshot(""); + assertTrue(snapshots.get(0).getTag().endsWith("nonEphemeralSnapshot")); } @Test public void testSnapshotSize() throws IOException { - // cleanup any previous test gargbage ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1); - cfs.clearSnapshot(""); // Add row new RowUpdateBuilder(cfs.metadata(), 0, "key1") @@ -311,10 +331,10 @@ public void testSnapshotSize() throws IOException Util.flush(cfs); // snapshot - cfs.snapshot("basic", null, false, false); + SnapshotManager.instance.takeSnapshot("basic", cfs.getKeyspaceTableName()); // check snapshot was created - Map<String, TableSnapshot> snapshotDetails = cfs.listSnapshots(); + Map<String, TableSnapshot> snapshotDetails = Util.listSnapshots(cfs); assertThat(snapshotDetails).hasSize(1); assertThat(snapshotDetails).containsKey("basic"); @@ -329,13 +349,121 @@ public void testSnapshotSize() throws IOException // sizeOnDisk > trueSize because trueSize does not include manifest.json // Check that truesize now is > 0 - snapshotDetails = cfs.listSnapshots(); + snapshotDetails = Util.listSnapshots(cfs); details = snapshotDetails.get("basic"); assertThat(details.computeSizeOnDiskBytes()).isEqualTo(details.computeTrueSizeBytes()); } @Test - public void testBackupAfterFlush() throws Throwable + public void testTrueSnapshotSizeWithLegacyIndex() throws Throwable + { + testTrueSnapshotSizeInternal("snapshot_sizes_legacy", "true_snapshot_size_with_legacy_index", false); + } + + @Test + public void testTrueSnapshotSizeWithSaiIndex() throws Throwable + { + testTrueSnapshotSizeInternal("snapshot_sizes_sai","true_snapshot_size_with_sai_index", true); + } + + private void testTrueSnapshotSizeInternal(String keyspace, String table, boolean forSai) throws Throwable + { + TableMetadata tableMetadata = SchemaLoader.standardCFMD(keyspace, table).partitioner(Murmur3Partitioner.instance).build(); + SchemaLoader.createKeyspace(keyspace, KeyspaceParams.simple(1), tableMetadata); + + ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(table); + + new RowUpdateBuilder(cfs.metadata(), 0, "key1") + .clustering("Column1") + .add("val", "value1") + .build() + .applyUnsafe(); + Util.flush(cfs); + + assertThat(cfs.trueSnapshotsSize()).isZero(); + + SnapshotManager.instance.takeSnapshot("snapshot_without_index", cfs.getKeyspaceTableName()); + + long firstSnapshotsSize = cfs.trueSnapshotsSize(); + Map<String, TableSnapshot> listedSnapshots = Util.listSnapshots(cfs); + assertThat(firstSnapshotsSize).isPositive(); + assertThat(listedSnapshots.size()).isEqualTo(1); + assertThat(listedSnapshots.get("snapshot_without_index")).isNotNull(); + long withoutIndexSize = listedSnapshots.get("snapshot_without_index").computeSizeOnDiskBytes(); + long withoutIndexTrueSize = listedSnapshots.get("snapshot_without_index").computeTrueSizeBytes(); + + assertThat(withoutIndexSize).isGreaterThan(withoutIndexTrueSize); + assertEquals(firstSnapshotsSize, withoutIndexTrueSize); + + // add index, trueSnapshotSize should reflect that + ColumnIdentifier col = ColumnIdentifier.getInterned("val", true); + + IndexMetadata indexMetadata; + if (forSai) + indexMetadata = fromIndexTargets(List.of(new IndexTarget(col, VALUES)), "idx", CUSTOM, Map.of("class_name", "sai")); + else + indexMetadata = fromIndexTargets(List.of(new IndexTarget(col, VALUES)), "idx", COMPOSITES, Map.of()); + + TableMetadata tableMetadataWithIndex = SchemaLoader.standardCFMD(keyspace, table) + .partitioner(Murmur3Partitioner.instance) + .id(tableMetadata.id) + .indexes(Indexes.of(indexMetadata)) + .build(); + + SchemaTestUtil.announceTableUpdate(tableMetadataWithIndex); + + rebuildIndices(cfs); + + SnapshotManager.instance.takeSnapshot("snapshot_with_index", new HashMap<>(), cfs.getKeyspaceTableName()); + + long secondSnapshotSize = cfs.trueSnapshotsSize(); + Map<String, TableSnapshot> secondListedSnapshots = Util.listSnapshots(cfs); + assertThat(secondSnapshotSize).isPositive(); + assertThat(secondSnapshotSize).isGreaterThan(firstSnapshotsSize); + + assertThat(secondListedSnapshots.size()).isEqualTo(2); + assertThat(secondListedSnapshots.get("snapshot_with_index")).isNotNull(); + assertThat(secondListedSnapshots.get("snapshot_without_index")).isNotNull(); + + long withIndexSize = secondListedSnapshots.get("snapshot_with_index").computeSizeOnDiskBytes(); + long withIndexTrueSize = secondListedSnapshots.get("snapshot_with_index").computeTrueSizeBytes(); + + assertThat(withIndexSize).isGreaterThan(withIndexTrueSize); + assertEquals(secondSnapshotSize, withIndexTrueSize + withoutIndexTrueSize); + + // taking another one is basically a copy of the previous + SnapshotManager.instance.takeSnapshot("another_snapshot_with_index", new HashMap<>(), cfs.getKeyspaceTableName()); + + long thirdSnapshotSize = cfs.trueSnapshotsSize(); + Map<String, TableSnapshot> thirdListedSnapshots = Util.listSnapshots(cfs); + assertThat(thirdSnapshotSize).isPositive(); + assertThat(thirdSnapshotSize).isGreaterThan(secondSnapshotSize); + + long anotherWithIndexSize = thirdListedSnapshots.get("another_snapshot_with_index").computeSizeOnDiskBytes(); + long anotherWithIndexTrueSize = thirdListedSnapshots.get("another_snapshot_with_index").computeTrueSizeBytes(); + + assertEquals(withIndexSize, anotherWithIndexSize); + assertEquals(withIndexTrueSize, anotherWithIndexTrueSize); + } + + private void rebuildIndices(ColumnFamilyStore cfs) + { + // be sure we build that index + for (Index index : cfs.indexManager.listIndexes()) + { + try + { + cfs.indexManager.buildIndex(index).get(); + } + catch (Throwable t) + { + throw new RuntimeException("Unable to build index", t); + } + } + } + + @Test + public void testBackupAfterFlush() { ColumnFamilyStore cfs = Keyspace.open(KEYSPACE2).getColumnFamilyStore(CF_STANDARD1); new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes("key1")).clustering("Column1").add("val", "asdf").build().applyUnsafe(); @@ -381,8 +509,8 @@ public void speculationThreshold() // remember, latencies are only an estimate - off by up to 20% by the 1.2 factor between buckets. assertThat(cfs.metric.coordinatorReadLatency.getCount()).isEqualTo(count); assertThat(cfs.metric.coordinatorReadLatency.getSnapshot().getValue(0.5)) - .isBetween((double) TimeUnit.MILLISECONDS.toMicros(5839), - (double) TimeUnit.MILLISECONDS.toMicros(5840)); + .isBetween((double) TimeUnit.MILLISECONDS.toMicros(5839), + (double) TimeUnit.MILLISECONDS.toMicros(5840)); // Sanity check the metrics - 75th percentileof linear 0-10000ms assertThat(cfs.metric.coordinatorWriteLatency.getCount()).isEqualTo(count); assertThat(cfs.metric.coordinatorWriteLatency.getSnapshot().getValue(0.75)) @@ -555,7 +683,7 @@ public void testSnapshotWithoutFlushWithSecondaryIndexes() throws Exception { Keyspace keyspace = Keyspace.open(KEYSPACE1); ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_INDEX1); - cfs.truncateBlocking(); + cfs.truncateBlockingWithoutSnapshot(); UpdateBuilder builder = UpdateBuilder.create(cfs.metadata.get(), "key") .newRow() @@ -565,7 +693,7 @@ public void testSnapshotWithoutFlushWithSecondaryIndexes() throws Exception Util.flush(cfs); String snapshotName = "newSnapshot"; - cfs.snapshotWithoutMemtable(snapshotName); + SnapshotManager.instance.takeSnapshot(snapshotName, Map.of(SnapshotOptions.SKIP_FLUSH, "true"), cfs.getKeyspaceTableName()); File snapshotManifestFile = cfs.getDirectories().getSnapshotManifestFile(snapshotName); SnapshotManifest manifest = SnapshotManifest.deserializeFromJsonFile(snapshotManifestFile); @@ -595,17 +723,18 @@ private void createSnapshotAndDelete(String ks, String table, boolean writeData) writeData(cfs); } - TableSnapshot snapshot = cfs.snapshot("basic"); - + SnapshotManager.instance.takeSnapshot("basic", cfs.getKeyspaceTableName()); + TableSnapshot snapshot = SnapshotManager.instance.getSnapshot(cfs.metadata.keyspace, cfs.metadata.name, "basic").get(); + assertNotNull(snapshot); assertThat(snapshot.exists()).isTrue(); - assertThat(cfs.listSnapshots().containsKey("basic")).isTrue(); - assertThat(cfs.listSnapshots().get("basic")).isEqualTo(snapshot); + assertThat(Util.listSnapshots(cfs).containsKey("basic")).isTrue(); + assertThat(Util.listSnapshots(cfs).get("basic")).isEqualTo(snapshot); snapshot.getDirectories().forEach(FileUtils::deleteRecursive); assertThat(snapshot.exists()).isFalse(); - assertFalse(cfs.listSnapshots().containsKey("basic")); + assertFalse(Util.listSnapshots(cfs).containsKey("basic")); } private void writeData(ColumnFamilyStore cfs) @@ -623,7 +752,8 @@ private void writeData(ColumnFamilyStore cfs) } @Test - public void testSnapshotCreationAndDeleteEmptyTable() { + public void testSnapshotCreationAndDeleteEmptyTable() + { createSnapshotAndDelete(KEYSPACE1, CF_INDEX1, false); createSnapshotAndDelete(KEYSPACE1, CF_STANDARD1, false); createSnapshotAndDelete(KEYSPACE1, CF_STANDARD2, false); @@ -632,7 +762,8 @@ public void testSnapshotCreationAndDeleteEmptyTable() { } @Test - public void testSnapshotCreationAndDeletePopulatedTable() { + public void testSnapshotCreationAndDeletePopulatedTable() + { createSnapshotAndDelete(KEYSPACE1, CF_INDEX1, true); createSnapshotAndDelete(KEYSPACE1, CF_STANDARD1, true); createSnapshotAndDelete(KEYSPACE1, CF_STANDARD2, true); @@ -651,8 +782,8 @@ public void testDataDirectoriesOfColumnFamily() throws Exception String keyspace = path.getParent().getFileName().toString(); String table = path.getFileName().toString().split("-")[0]; - Assert.assertEquals(cfs.getTableName(), table); - Assert.assertEquals(KEYSPACE1, keyspace); + assertEquals(cfs.getTableName(), table); + assertEquals(KEYSPACE1, keyspace); } @Test @@ -685,15 +816,15 @@ public void testScrubDataDirectories() throws Throwable } @VisibleForTesting - public static long getSnapshotManifestAndSchemaFileSizes(TableSnapshot snapshot) throws IOException + public static long getSnapshotManifestAndSchemaFileSizes(TableSnapshot snapshot) { - Optional<File> schemaFile = snapshot.getSchemaFile(); - Optional<File> manifestFile = snapshot.getManifestFile(); - long schemaAndManifestFileSizes = 0; - schemaAndManifestFileSizes += schemaFile.isPresent() ? schemaFile.get().length() : 0; - schemaAndManifestFileSizes += manifestFile.isPresent() ? manifestFile.get().length() : 0; + for (File schemaFile : snapshot.getSchemaFiles()) + schemaAndManifestFileSizes += schemaFile.length(); + + for (File manifestFile : snapshot.getManifestFiles()) + schemaAndManifestFileSizes += manifestFile.length(); return schemaAndManifestFileSizes; } @@ -827,7 +958,7 @@ public UnfilteredRowIterator rowIterator(DecoratedKey key, @Override public UnfilteredPartitionIterator - partitionIterator(ColumnFilter columnFilter, DataRange dataRange, SSTableReadsListener listener) + partitionIterator(ColumnFilter columnFilter, DataRange dataRange, SSTableReadsListener listener) { return null; } diff --git a/test/unit/org/apache/cassandra/db/DirectoriesTest.java b/test/unit/org/apache/cassandra/db/DirectoriesTest.java index 1794ae8843e3..b6d4126fc9b7 100644 --- a/test/unit/org/apache/cassandra/db/DirectoriesTest.java +++ b/test/unit/org/apache/cassandra/db/DirectoriesTest.java @@ -19,8 +19,11 @@ import java.io.IOException; import java.nio.file.FileStore; +import java.nio.file.FileVisitResult; import java.nio.file.Files; import java.nio.file.Path; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.attribute.BasicFileAttributes; import java.nio.file.attribute.FileAttributeView; import java.nio.file.attribute.FileStoreAttributeView; import java.time.Instant; @@ -93,6 +96,7 @@ import org.apache.cassandra.schema.SchemaKeyspaceTables; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.DefaultFSErrorHandler; +import org.apache.cassandra.service.snapshot.SnapshotLoader; import org.apache.cassandra.service.snapshot.SnapshotManifest; import org.apache.cassandra.service.snapshot.TableSnapshot; import org.apache.cassandra.utils.JVMStabilityInspector; @@ -326,12 +330,24 @@ public void testStandardDirs() } } + private Map<String, TableSnapshot> listSnapshots(Directories directories) + { + Set<TableSnapshot> snapshots = new SnapshotLoader(directories).loadSnapshots(); + Map<String, TableSnapshot> tagSnapshotsMap = new HashMap<>(); + + for (TableSnapshot snapshot : snapshots) + tagSnapshotsMap.put(snapshot.getTag(), snapshot); + + return tagSnapshotsMap; + } + @Test public void testListSnapshots() throws Exception { + // Initial state TableMetadata fakeTable = createFakeTable(TABLE_NAME); Directories directories = new Directories(fakeTable, toDataDirectories(tempDataDir)); - assertThat(directories.listSnapshots()).isEmpty(); + assertThat(listSnapshots(directories)).isEmpty(); // Create snapshot with and without manifest FakeSnapshot snapshot1 = createFakeSnapshot(fakeTable, SNAPSHOT1, true, false); @@ -340,7 +356,7 @@ public void testListSnapshots() throws Exception { FakeSnapshot snapshot3 = createFakeSnapshot(fakeTable, SNAPSHOT3, false, true); // Both snapshots should be present - Map<String, TableSnapshot> snapshots = directories.listSnapshots(); + Map<String, TableSnapshot> snapshots = listSnapshots(directories); assertThat(snapshots.keySet()).isEqualTo(Sets.newHashSet(SNAPSHOT1, SNAPSHOT2, SNAPSHOT3)); assertThat(snapshots.get(SNAPSHOT1)).isEqualTo(snapshot1.asTableSnapshot()); assertThat(snapshots.get(SNAPSHOT2)).isEqualTo(snapshot2.asTableSnapshot()); @@ -350,40 +366,13 @@ public void testListSnapshots() throws Exception { snapshot1.snapshotDir.deleteRecursive(); // Only snapshot 2 and 3 should be present - snapshots = directories.listSnapshots(); + snapshots = listSnapshots(directories); assertThat(snapshots.keySet()).isEqualTo(Sets.newHashSet(SNAPSHOT2, SNAPSHOT3)); assertThat(snapshots.get(SNAPSHOT2)).isEqualTo(snapshot2.asTableSnapshot()); assertThat(snapshots.get(SNAPSHOT3)).isEqualTo(snapshot3.asTableSnapshot()); assertThat(snapshots.get(SNAPSHOT3).isEphemeral()).isTrue(); } - @Test - public void testListSnapshotDirsByTag() throws Exception { - // Initial state - TableMetadata fakeTable = createFakeTable("FakeTable"); - Directories directories = new Directories(fakeTable, toDataDirectories(tempDataDir)); - assertThat(directories.listSnapshotDirsByTag()).isEmpty(); - - // Create snapshot with and without manifest - FakeSnapshot snapshot1 = createFakeSnapshot(fakeTable, SNAPSHOT1, true, false); - FakeSnapshot snapshot2 = createFakeSnapshot(fakeTable, SNAPSHOT2, false, false); - FakeSnapshot snapshot3 = createFakeSnapshot(fakeTable, SNAPSHOT3, false, true); - - // Both snapshots should be present - Map<String, Set<File>> snapshotDirs = directories.listSnapshotDirsByTag(); - assertThat(snapshotDirs.keySet()).isEqualTo(Sets.newHashSet(SNAPSHOT1, SNAPSHOT2, SNAPSHOT3)); - assertThat(snapshotDirs.get(SNAPSHOT1)).allMatch(snapshotDir -> snapshotDir.equals(snapshot1.snapshotDir)); - assertThat(snapshotDirs.get(SNAPSHOT2)).allMatch(snapshotDir -> snapshotDir.equals(snapshot2.snapshotDir)); - assertThat(snapshotDirs.get(SNAPSHOT3)).allMatch(snapshotDir -> snapshotDir.equals(snapshot3.snapshotDir)); - - // Now remove snapshot1 - snapshot1.snapshotDir.deleteRecursive(); - - // Only snapshot 2 and 3 should be present - snapshotDirs = directories.listSnapshotDirsByTag(); - assertThat(snapshotDirs.keySet()).isEqualTo(Sets.newHashSet(SNAPSHOT2, SNAPSHOT3)); - } - @Test public void testMaybeManifestLoading() throws Exception { for (TableMetadata cfm : CFM) @@ -412,7 +401,7 @@ public void testMaybeManifestLoading() throws Exception { } @Test - public void testSecondaryIndexDirectories() + public void testSecondaryIndexDirectories() throws IOException { TableMetadata.Builder builder = TableMetadata.builder(KS, "cf") @@ -447,8 +436,8 @@ public void testSecondaryIndexDirectories() // check if snapshot directory exists parentSnapshotDirectory.tryCreateDirectories(); - assertTrue(parentDirectories.snapshotExists("test")); - assertTrue(indexDirectories.snapshotExists("test")); + assertTrue(snapshotExists(parentDirectories, "test")); + assertTrue(snapshotExists(indexDirectories, "test")); // check true snapshot size Descriptor parentSnapshot = new Descriptor(parentSnapshotDirectory, KS, PARENT_CFM.name, sstableId(0), DatabaseDescriptor.getSelectedSSTableFormat()); @@ -456,14 +445,21 @@ public void testSecondaryIndexDirectories() Descriptor indexSnapshot = new Descriptor(indexSnapshotDirectory, KS, INDEX_CFM.name, sstableId(0), DatabaseDescriptor.getSelectedSSTableFormat()); createFile(indexSnapshot.fileFor(Components.DATA), 40); - assertEquals(30, parentDirectories.trueSnapshotsSize()); - assertEquals(40, indexDirectories.trueSnapshotsSize()); - // check snapshot details - Map<String, TableSnapshot> parentSnapshotDetail = parentDirectories.listSnapshots(); + Map<String, TableSnapshot> parentSnapshotDetail = listSnapshots(parentDirectories); assertTrue(parentSnapshotDetail.containsKey("test")); + + Set<String> files = new HashSet<>(); + Files.walkFileTree(parentSnapshotDirectory.toPath(), new SimpleFileVisitor<>() { + @Override + public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) + { + files.add(file.toAbsolutePath().toString()); + return FileVisitResult.CONTINUE; + } + }); // CASSANDRA-17357: include indexes when computing true size of parent table - assertEquals(70L, parentSnapshotDetail.get("test").computeTrueSizeBytes()); + assertEquals(70L, parentSnapshotDetail.get("test").computeTrueSizeBytes(files)); // check backup directory File parentBackupDirectory = Directories.getBackupsDirectory(parentDesc); @@ -471,6 +467,30 @@ public void testSecondaryIndexDirectories() assertEquals(parentBackupDirectory, indexBackupDirectory.parent()); } + private boolean snapshotExists(Directories directories, String snapshotName) + { + for (File dir : directories.getDataPaths()) + { + File snapshotDir; + if (Directories.isSecondaryIndexFolder(dir)) + { + snapshotDir = new File(dir.parent(), join(Directories.SNAPSHOT_SUBDIR, snapshotName, dir.name())); + } + else + { + snapshotDir = new File(dir, join(Directories.SNAPSHOT_SUBDIR, snapshotName)); + } + if (snapshotDir.exists()) + return true; + } + return false; + } + + private static String join(String... s) + { + return StringUtils.join(s, File.pathSeparator()); + } + private File createFile(File file, int size) { try (FileOutputStreamPlus writer = new FileOutputStreamPlus(file);) diff --git a/test/unit/org/apache/cassandra/db/KeyspaceTest.java b/test/unit/org/apache/cassandra/db/KeyspaceTest.java index 84c5b3ecfd9b..f78d081dcee6 100644 --- a/test/unit/org/apache/cassandra/db/KeyspaceTest.java +++ b/test/unit/org/apache/cassandra/db/KeyspaceTest.java @@ -19,34 +19,49 @@ package org.apache.cassandra.db; import java.nio.ByteBuffer; -import java.util.*; +import java.util.Collection; +import java.util.List; -import org.assertj.core.api.Assertions; +import org.junit.Before; import org.junit.Test; import org.apache.cassandra.Util; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.UntypedResultSet; +import org.apache.cassandra.db.compaction.CompactionManager; +import org.apache.cassandra.db.filter.ClusteringIndexSliceFilter; +import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.filter.DataLimits; +import org.apache.cassandra.db.filter.RowFilter; +import org.apache.cassandra.db.partitions.PartitionIterator; import org.apache.cassandra.db.rows.Cell; import org.apache.cassandra.db.rows.Row; import org.apache.cassandra.db.rows.RowIterator; import org.apache.cassandra.io.sstable.AbstractRowIndexEntry; -import org.apache.cassandra.db.compaction.CompactionManager; -import org.apache.cassandra.db.filter.*; -import org.apache.cassandra.db.partitions.PartitionIterator; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.format.big.BigTableReader; import org.apache.cassandra.metrics.ClearableHistogram; +import org.apache.cassandra.service.snapshot.SnapshotManager; +import org.apache.cassandra.service.snapshot.TableSnapshot; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.FBUtilities; +import org.assertj.core.api.Assertions; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.fail; public class KeyspaceTest extends CQLTester { // Test needs synchronous table drop to avoid flushes causing flaky failures of testLimitSSTables + @Before + public void cleanupSnapshots() + { + SnapshotManager.instance.clearAllSnapshots(); + } + @Override protected String createTable(String query) { @@ -425,10 +440,11 @@ public void testSnapshotCreation() throws Throwable { Keyspace ks = Keyspace.open(KEYSPACE_PER_TEST); String table = getCurrentColumnFamilyStore().name; - ks.snapshot("test", table); + SnapshotManager.instance.takeSnapshot("test", ks.getName() + '.' + table); - assertTrue(ks.snapshotExists("test")); - assertEquals(1, ks.getAllSnapshots().count()); + List<TableSnapshot> snapshots = SnapshotManager.instance.getSnapshots(ks.getName()); + assertEquals(1, snapshots.size()); + assertEquals(snapshots.get(0).getTag(), "test"); } @Test diff --git a/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java b/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java index 3343e6d7f663..d045584412c6 100644 --- a/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java +++ b/test/unit/org/apache/cassandra/db/SchemaCQLHelperTest.java @@ -35,6 +35,8 @@ import org.apache.cassandra.index.sasi.SASIIndex; import org.apache.cassandra.schema.*; import org.apache.cassandra.service.reads.SpeculativeRetryPolicy; +import org.apache.cassandra.service.snapshot.SnapshotManager; +import org.apache.cassandra.service.snapshot.SnapshotOptions; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.JsonUtils; @@ -431,7 +433,7 @@ public void testSnapshot() throws Throwable execute("INSERT INTO %s (pk1, pk2, ck1, ck2, reg1, reg2) VALUES (?, ?, ?, ?, ?, ?)", i, i + 1, i + 2, i + 3, null, i + 5); ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(tableName); - cfs.snapshot(SNAPSHOT); + SnapshotManager.instance.takeSnapshot(SnapshotOptions.userSnapshot(SNAPSHOT, cfs.getKeyspaceTableName())); String schema = Files.toString(cfs.getDirectories().getSnapshotSchemaFile(SNAPSHOT).toJavaIOFile(), Charset.defaultCharset()); assertThat(schema, @@ -480,6 +482,7 @@ public void testSnapshot() throws Throwable "INDEX IF NOT EXISTS " + tableName + "_reg2_idx ON " + keyspace() + '.' + tableName + " (reg2)" + (" USING '" + (isIndexLegacy ? CassandraIndex.NAME : DatabaseDescriptor.getDefaultSecondaryIndex()) + "'") + ";")); + // TODO: construct manifest from SnapshotManager JsonNode manifest = JsonUtils.JSON_OBJECT_MAPPER.readTree(cfs.getDirectories().getSnapshotManifestFile(SNAPSHOT).toJavaIOFile()); JsonNode files = manifest.get("files"); // two files, the second is index @@ -508,7 +511,7 @@ public void testSnapshotWithDroppedColumnsWithoutReAdding() throws Throwable execute("INSERT INTO %s (pk1, pk2, ck1, ck2, reg1) VALUES (?, ?, ?, ?, ?)", i, i + 1, i + 2, i + 3, null); ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(tableName); - cfs.snapshot(SNAPSHOT); + SnapshotManager.instance.takeSnapshot(SNAPSHOT, cfs.getKeyspaceTableName()); String schema = Files.toString(cfs.getDirectories().getSnapshotSchemaFile(SNAPSHOT).toJavaIOFile(), Charset.defaultCharset()); schema = schema.substring(schema.indexOf("CREATE TABLE")); // trim to ensure order @@ -551,7 +554,7 @@ public void testSnapshotWithDroppedColumnsWithoutReAddingOnSingleKeyTable() thro execute("INSERT INTO %s (pk1, reg1) VALUES (?, ?)", i, i + 1); ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(tableName); - cfs.snapshot(SNAPSHOT); + SnapshotManager.instance.takeSnapshot(SNAPSHOT, cfs.getKeyspaceTableName()); String schema = Files.toString(cfs.getDirectories().getSnapshotSchemaFile(SNAPSHOT).toJavaIOFile(), Charset.defaultCharset()); schema = schema.substring(schema.indexOf("CREATE TABLE")); // trim to ensure order @@ -577,7 +580,7 @@ public void testSnapshotWithDroppedColumnsWithoutReAddingOnSingleKeyTable() thro public void testSystemKsSnapshot() { ColumnFamilyStore cfs = Keyspace.open("system").getColumnFamilyStore("peers"); - cfs.snapshot(SNAPSHOT); + SnapshotManager.instance.takeSnapshot(SNAPSHOT, cfs.getKeyspaceTableName()); Assert.assertTrue(cfs.getDirectories().getSnapshotManifestFile(SNAPSHOT).exists()); Assert.assertFalse(cfs.getDirectories().getSnapshotSchemaFile(SNAPSHOT).exists()); diff --git a/test/unit/org/apache/cassandra/db/SnapshotTest.java b/test/unit/org/apache/cassandra/db/SnapshotTest.java index fdfdd6b3aa92..4db09417366b 100644 --- a/test/unit/org/apache/cassandra/db/SnapshotTest.java +++ b/test/unit/org/apache/cassandra/db/SnapshotTest.java @@ -27,6 +27,7 @@ import org.apache.cassandra.io.sstable.format.SSTableFormat.Components; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.File; +import org.apache.cassandra.service.snapshot.SnapshotManager; public class SnapshotTest extends CQLTester { @@ -41,6 +42,7 @@ public void testEmptyTOC() throws Throwable File toc = sstable.descriptor.fileFor(Components.TOC); Files.write(toc.toPath(), new byte[0], StandardOpenOption.TRUNCATE_EXISTING); } - getCurrentColumnFamilyStore().snapshot("hello"); + + SnapshotManager.instance.takeSnapshot("hello", getCurrentColumnFamilyStore().getKeyspaceTableName()); } } diff --git a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java index bcaefc23d9d7..2047210323cc 100644 --- a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java +++ b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java @@ -25,6 +25,7 @@ import org.junit.Test; import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.Util; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.cql3.UntypedResultSet; @@ -34,7 +35,7 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.SchemaKeyspace; -import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.CassandraVersion; @@ -100,14 +101,14 @@ public void snapshotSystemKeyspaceIfUpgrading() throws IOException // First, check that in the absence of any previous installed version, we don't create snapshots for (ColumnFamilyStore cfs : Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStores()) cfs.clearUnsafe(); - StorageService.instance.clearSnapshot(Collections.emptyMap(), null, SchemaConstants.SYSTEM_KEYSPACE_NAME); + SnapshotManager.instance.clearSnapshot(null, Collections.emptyMap(), SchemaConstants.SYSTEM_KEYSPACE_NAME); SystemKeyspace.snapshotOnVersionChange(); assertDeleted(); // now setup system.local as if we're upgrading from a previous version setupReleaseVersion(getOlderVersionString()); - StorageService.instance.clearSnapshot(Collections.emptyMap(), null, SchemaConstants.SYSTEM_KEYSPACE_NAME); + SnapshotManager.instance.clearSnapshot(null, Collections.emptyMap(), SchemaConstants.SYSTEM_KEYSPACE_NAME); assertDeleted(); // Compare versions again & verify that snapshots were created for all tables in the system ks @@ -120,7 +121,7 @@ public void snapshotSystemKeyspaceIfUpgrading() throws IOException // clear out the snapshots & set the previous recorded version equal to the latest, we shouldn't // see any new snapshots created this time. - StorageService.instance.clearSnapshot(Collections.emptyMap(), null, SchemaConstants.SYSTEM_KEYSPACE_NAME); + SnapshotManager.instance.clearSnapshot(null, Collections.emptyMap(), SchemaConstants.SYSTEM_KEYSPACE_NAME); setupReleaseVersion(FBUtilities.getReleaseVersionString()); SystemKeyspace.snapshotOnVersionChange(); @@ -129,7 +130,7 @@ public void snapshotSystemKeyspaceIfUpgrading() throws IOException // 10 files expected. assertDeleted(); - StorageService.instance.clearSnapshot(Collections.emptyMap(), null, SchemaConstants.SYSTEM_KEYSPACE_NAME); + SnapshotManager.instance.clearSnapshot(null, Collections.emptyMap(), SchemaConstants.SYSTEM_KEYSPACE_NAME); } @Test @@ -170,7 +171,7 @@ private Set<String> getSystemSnapshotFiles(String keyspace) Set<String> snapshottedTableNames = new HashSet<>(); for (ColumnFamilyStore cfs : Keyspace.open(keyspace).getColumnFamilyStores()) { - if (!cfs.listSnapshots().isEmpty()) + if (!Util.listSnapshots(cfs).isEmpty()) snapshottedTableNames.add(cfs.getTableName()); } return snapshottedTableNames; diff --git a/test/unit/org/apache/cassandra/db/virtual/SnapshotsTableTest.java b/test/unit/org/apache/cassandra/db/virtual/SnapshotsTableTest.java index 09832309c157..0d4ed611c400 100644 --- a/test/unit/org/apache/cassandra/db/virtual/SnapshotsTableTest.java +++ b/test/unit/org/apache/cassandra/db/virtual/SnapshotsTableTest.java @@ -19,27 +19,30 @@ package org.apache.cassandra.db.virtual; import java.time.Instant; -import java.time.temporal.ChronoUnit; import java.util.Collections; import java.util.Date; +import java.util.List; +import java.util.Map; import com.google.common.collect.ImmutableList; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.cql3.UntypedResultSet; -import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.utils.Clock; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.service.snapshot.SnapshotOptions; +import org.apache.cassandra.service.snapshot.SnapshotManager; +import org.apache.cassandra.service.snapshot.TableSnapshot; public class SnapshotsTableTest extends CQLTester { private static final String KS_NAME = "vts"; private static final String SNAPSHOT_TTL = "snapshotTtl"; private static final String SNAPSHOT_NO_TTL = "snapshotNoTtl"; - private static final DurationSpec.IntSecondsBound ttl = new DurationSpec.IntSecondsBound("4h"); + private static final String TTL = "4h"; @Before public void before() throws Throwable @@ -58,48 +61,56 @@ public void before() throws Throwable @After public void after() { - StorageService.instance.clearSnapshot(Collections.emptyMap(), SNAPSHOT_NO_TTL, KEYSPACE); - StorageService.instance.clearSnapshot(Collections.emptyMap(), SNAPSHOT_TTL, KEYSPACE); + SnapshotManager.instance.clearSnapshot(SNAPSHOT_NO_TTL, Collections.emptyMap(), KEYSPACE); + SnapshotManager.instance.clearSnapshot(SNAPSHOT_TTL, Collections.emptyMap(), KEYSPACE); + schemaChange(String.format("DROP TABLE %s", KEYSPACE + '.' + currentTable())); + } - schemaChange(String.format("DROP TABLE %s", KEYSPACE + "." + currentTable())); + private static Date toDate(Instant instant) + { + return new Date(instant.toEpochMilli()); } - @Test - public void testSnapshots() throws Throwable + private static TableSnapshot createSnapshot(String snapshotTtl, Map<String, String> options, ColumnFamilyStore cfs) { - Instant now = Instant.ofEpochMilli(Clock.Global.currentTimeMillis()).truncatedTo(ChronoUnit.MILLIS); - Date createdAt = new Date(now.toEpochMilli()); - Date expiresAt = new Date(now.plusSeconds(ttl.toSeconds()).toEpochMilli()); + List<TableSnapshot> snapshots = SnapshotManager.instance.takeSnapshot(SnapshotOptions.userSnapshot(snapshotTtl, options, cfs.getKeyspaceTableName())); + Assert.assertEquals(1, snapshots.size()); + return snapshots.iterator().next(); + } - getCurrentColumnFamilyStore(KEYSPACE).snapshot(SNAPSHOT_NO_TTL, null, false, false, null, null, now); - getCurrentColumnFamilyStore(KEYSPACE).snapshot(SNAPSHOT_TTL, null, false, false, ttl, null, now); + @Test + public void testSnapshots() + { + ColumnFamilyStore cfs = getCurrentColumnFamilyStore(KEYSPACE); + TableSnapshot snapshotWithTtl = createSnapshot(SNAPSHOT_TTL, Map.of(SnapshotOptions.TTL, TTL), cfs); + TableSnapshot snapshotWithoutTtl = createSnapshot(SNAPSHOT_NO_TTL, Collections.emptyMap(), cfs); // query all from snapshots virtual table UntypedResultSet result = execute("SELECT name, keyspace_name, table_name, created_at, expires_at, ephemeral FROM vts.snapshots"); assertRowsIgnoringOrder(result, - row(SNAPSHOT_NO_TTL, KEYSPACE, currentTable(), createdAt, null, false), - row(SNAPSHOT_TTL, KEYSPACE, currentTable(), createdAt, expiresAt, false)); + row(SNAPSHOT_NO_TTL, KEYSPACE, currentTable(), toDate(snapshotWithoutTtl.getCreatedAt()), null, false), + row(SNAPSHOT_TTL, KEYSPACE, currentTable(), toDate(snapshotWithTtl.getCreatedAt()), toDate(snapshotWithTtl.getExpiresAt()), false)); // query with conditions result = execute("SELECT name, keyspace_name, table_name, created_at, expires_at, ephemeral FROM vts.snapshots where ephemeral = false"); assertRows(result, - row(SNAPSHOT_NO_TTL, KEYSPACE, currentTable(), createdAt, null, false), - row(SNAPSHOT_TTL, KEYSPACE, currentTable(), createdAt, expiresAt, false)); + row(SNAPSHOT_NO_TTL, KEYSPACE, currentTable(), toDate(snapshotWithoutTtl.getCreatedAt()), null, false), + row(SNAPSHOT_TTL, KEYSPACE, currentTable(), toDate(snapshotWithTtl.getCreatedAt()), toDate(snapshotWithTtl.getExpiresAt()), false)); result = execute("SELECT name, keyspace_name, table_name, created_at, expires_at, ephemeral FROM vts.snapshots where size_on_disk > 1000"); assertRows(result, - row(SNAPSHOT_NO_TTL, KEYSPACE, currentTable(), createdAt, null, false), - row(SNAPSHOT_TTL, KEYSPACE, currentTable(), createdAt, expiresAt, false)); + row(SNAPSHOT_NO_TTL, KEYSPACE, currentTable(), toDate(snapshotWithoutTtl.getCreatedAt()), null, false), + row(SNAPSHOT_TTL, KEYSPACE, currentTable(), toDate(snapshotWithTtl.getCreatedAt()), toDate(snapshotWithTtl.getExpiresAt()), false)); result = execute("SELECT name, keyspace_name, table_name, created_at, expires_at, ephemeral FROM vts.snapshots where name = ?", SNAPSHOT_TTL); assertRows(result, - row(SNAPSHOT_TTL, KEYSPACE, currentTable(), createdAt, expiresAt, false)); + row(SNAPSHOT_TTL, KEYSPACE, currentTable(), toDate(snapshotWithTtl.getCreatedAt()), toDate(snapshotWithTtl.getExpiresAt()), false)); // clear some snapshots - StorageService.instance.clearSnapshot(Collections.emptyMap(), SNAPSHOT_NO_TTL, KEYSPACE); + SnapshotManager.instance.clearSnapshot(SNAPSHOT_NO_TTL, Collections.emptyMap(), KEYSPACE); result = execute("SELECT name, keyspace_name, table_name, created_at, expires_at, ephemeral FROM vts.snapshots"); assertRowsIgnoringOrder(result, - row(SNAPSHOT_TTL, KEYSPACE, currentTable(), createdAt, expiresAt, false)); + row(SNAPSHOT_TTL, KEYSPACE, currentTable(), toDate(snapshotWithTtl.getCreatedAt()), toDate(snapshotWithTtl.getExpiresAt()), false)); } } \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/index/sai/SAITester.java b/test/unit/org/apache/cassandra/index/sai/SAITester.java index e1e6af7f0b86..e004f0c0ee6c 100644 --- a/test/unit/org/apache/cassandra/index/sai/SAITester.java +++ b/test/unit/org/apache/cassandra/index/sai/SAITester.java @@ -102,7 +102,8 @@ import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.service.snapshot.TableSnapshot; +import org.apache.cassandra.service.snapshot.SnapshotManager; +import org.apache.cassandra.service.snapshot.SnapshotOptions; import org.apache.cassandra.utils.ConfigGenBuilder; import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.Throwables; @@ -683,8 +684,7 @@ protected int getCompactionTasks() protected int snapshot(String snapshotName) { ColumnFamilyStore cfs = getCurrentColumnFamilyStore(); - TableSnapshot snapshot = cfs.snapshot(snapshotName); - return snapshot.getDirectories().size(); + return SnapshotManager.instance.takeSnapshot(SnapshotOptions.userSnapshot(snapshotName, cfs.getKeyspaceTableName())).iterator().next().getDirectories().size(); } protected void restoreSnapshot(String snapshot) diff --git a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java index 174ebd3d56fb..70da2a0ab6d6 100644 --- a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java +++ b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java @@ -126,6 +126,7 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.serializers.TypeSerializer; +import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.service.snapshot.SnapshotManifest; import org.apache.cassandra.service.snapshot.TableSnapshot; import org.apache.cassandra.utils.ByteBufferUtil; @@ -199,7 +200,7 @@ public void testSASIComponentsAddedToSnapshot() throws Throwable try { - store.snapshot(snapshotName); + SnapshotManager.instance.takeSnapshot(snapshotName, store.getKeyspaceTableName()); // Compact to make true snapshot size != 0 store.forceMajorCompaction(); @@ -208,6 +209,7 @@ public void testSASIComponentsAddedToSnapshot() throws Throwable SnapshotManifest manifest = SnapshotManifest.deserializeFromJsonFile(store.getDirectories().getSnapshotManifestFile(snapshotName)); Assert.assertFalse(ssTableReaders.isEmpty()); + Assert.assertNotNull(manifest.files); Assert.assertFalse(manifest.files.isEmpty()); Assert.assertEquals(ssTableReaders.size(), manifest.files.size()); @@ -243,7 +245,7 @@ public void testSASIComponentsAddedToSnapshot() throws Throwable } } - TableSnapshot details = store.listSnapshots().get(snapshotName); + TableSnapshot details = Util.listSnapshots(store).get(snapshotName); // check that SASI components are included in the computation of snapshot size long snapshotSize = tableSize + indexSize + getSnapshotManifestAndSchemaFileSizes(details); @@ -251,7 +253,7 @@ public void testSASIComponentsAddedToSnapshot() throws Throwable } finally { - store.clearSnapshot(snapshotName); + SnapshotManager.instance.clearSnapshot(store.getKeyspaceName(), store.getTableName(), snapshotName); } } diff --git a/test/unit/org/apache/cassandra/schema/SchemaKeyspaceTest.java b/test/unit/org/apache/cassandra/schema/SchemaKeyspaceTest.java index 0a06a853edb6..7ceceaad0426 100644 --- a/test/unit/org/apache/cassandra/schema/SchemaKeyspaceTest.java +++ b/test/unit/org/apache/cassandra/schema/SchemaKeyspaceTest.java @@ -31,6 +31,7 @@ import org.junit.runner.RunWith; import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.Util; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.cql3.UntypedResultSet; @@ -129,7 +130,7 @@ public void testAutoSnapshotEnabledOnTable() SchemaTestUtil.announceTableDrop(keyspaceName, tableName); - assertFalse(cfs.listSnapshots().isEmpty()); + assertFalse(Util.listSnapshots(cfs).isEmpty()); } @Test @@ -147,7 +148,7 @@ public void testAutoSnapshotDisabledOnTable() SchemaTestUtil.announceTableDrop(keyspaceName, tableName); - assertTrue(cfs.listSnapshots().isEmpty()); + assertTrue(Util.listSnapshots(cfs).isEmpty()); } private static void updateTable(String keyspace, TableMetadata oldTable, TableMetadata newTable) diff --git a/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java b/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java index 23cfa80bfc24..c59163ae0bc3 100644 --- a/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java +++ b/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java @@ -61,6 +61,7 @@ import org.apache.cassandra.locator.Replica; import org.apache.cassandra.repair.messages.RepairOption; import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.service.snapshot.TableSnapshot; import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.membership.NodeAddresses; @@ -293,7 +294,7 @@ public void testSnapshotAddSSTables() throws Exception true, PreviewKind.NONE); createSSTables(store, 2); store.getRepairManager().snapshot(prsId.toString(), ranges, false); - try (Refs<SSTableReader> refs = store.getSnapshotSSTableReaders(prsId.toString())) + try (Refs<SSTableReader> refs = TableSnapshot.getSnapshotSSTableReaders(store, prsId.toString())) { assertEquals(original, Sets.newHashSet(refs.iterator())); } diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java index 87534135801a..42ed428c76ba 100644 --- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java +++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java @@ -55,6 +55,7 @@ import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.SchemaKeyspaceTables; import org.apache.cassandra.schema.SchemaTestUtil; +import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.membership.Location; @@ -144,6 +145,7 @@ private static void setupDefaultPlacements() @Before public void resetCMS() { + SnapshotManager.instance.clearAllSnapshots(); ServerTestUtils.resetCMS(); } @@ -158,21 +160,27 @@ public void testGetAllRangesEmpty() public void testSnapshotWithFlush() throws IOException { // no need to insert extra data, even an "empty" database will have a little information in the system keyspace - StorageService.instance.takeSnapshot(UUID.randomUUID().toString()); + String snapshotName = UUID.randomUUID().toString(); + StorageService.instance.takeSnapshot(snapshotName); + Assert.assertTrue(SnapshotManager.instance.exists(p -> p.getTag().equals(snapshotName))); } @Test public void testTableSnapshot() throws IOException { // no need to insert extra data, even an "empty" database will have a little information in the system keyspace - StorageService.instance.takeTableSnapshot(SchemaConstants.SCHEMA_KEYSPACE_NAME, SchemaKeyspaceTables.KEYSPACES, UUID.randomUUID().toString()); + String snapshotName = UUID.randomUUID().toString(); + StorageService.instance.takeTableSnapshot(SchemaConstants.SCHEMA_KEYSPACE_NAME, SchemaKeyspaceTables.KEYSPACES, snapshotName); + Assert.assertTrue(SnapshotManager.instance.exists(SchemaConstants.SCHEMA_KEYSPACE_NAME, SchemaKeyspaceTables.KEYSPACES, snapshotName)); } @Test public void testSnapshot() throws IOException { // no need to insert extra data, even an "empty" database will have a little information in the system keyspace - StorageService.instance.takeSnapshot(UUID.randomUUID().toString(), SchemaConstants.SCHEMA_KEYSPACE_NAME); + String snapshotName = UUID.randomUUID().toString(); + StorageService.instance.takeSnapshot(snapshotName, SchemaConstants.SCHEMA_KEYSPACE_NAME); + Assert.assertTrue(SnapshotManager.instance.exists(p -> p.getTag().equals(snapshotName) && p.getKeyspaceName().equals(SchemaConstants.SCHEMA_KEYSPACE_NAME))); } @Test diff --git a/test/unit/org/apache/cassandra/service/snapshot/MetadataSnapshotsTest.java b/test/unit/org/apache/cassandra/service/snapshot/MetadataSnapshotsTest.java index f2565ee3f54f..ee996e3a333e 100644 --- a/test/unit/org/apache/cassandra/service/snapshot/MetadataSnapshotsTest.java +++ b/test/unit/org/apache/cassandra/service/snapshot/MetadataSnapshotsTest.java @@ -22,26 +22,22 @@ import java.util.Arrays; import java.util.List; import java.util.UUID; -import java.util.concurrent.atomic.AtomicReference; -import com.google.common.util.concurrent.Uninterruptibles; +import org.junit.After; +import org.junit.Before; import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.service.DefaultFSErrorHandler; -import static java.util.concurrent.TimeUnit.MINUTES; -import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.cassandra.service.snapshot.TableSnapshotTest.createFolders; -import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; import static org.apache.cassandra.utils.FBUtilities.now; import static org.assertj.core.api.Assertions.assertThat; -import static org.awaitility.Awaitility.await; -import static org.junit.Assert.assertTrue; public class MetadataSnapshotsTest { @@ -50,6 +46,9 @@ public class MetadataSnapshotsTest @BeforeClass public static void beforeClass() { + CassandraRelevantProperties.SNAPSHOT_CLEANUP_INITIAL_DELAY_SECONDS.setInt(3); + CassandraRelevantProperties.SNAPSHOT_CLEANUP_PERIOD_SECONDS.setInt(3); + DatabaseDescriptor.daemonInitialization(); FileUtils.setFSErrorHandler(new DefaultFSErrorHandler()); } @@ -57,6 +56,21 @@ public static void beforeClass() @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + private SnapshotManager manager; + + @Before + public void beforeTest() + { + manager = SnapshotManager.instance; + } + + @After + public void afterTest() throws Exception + { + SnapshotManager.instance.clearAllSnapshots(); + SnapshotManager.instance.close(); + } + private TableSnapshot generateSnapshotDetails(String tag, Instant expiration, boolean ephemeral) { try @@ -77,25 +91,28 @@ private TableSnapshot generateSnapshotDetails(String tag, Instant expiration, bo } @Test - public void testLoadSnapshots() throws Exception { + public void testExpiringSnapshots() + { TableSnapshot expired = generateSnapshotDetails("expired", Instant.EPOCH, false); TableSnapshot nonExpired = generateSnapshotDetails("non-expired", now().plusSeconds(ONE_DAY_SECS), false); TableSnapshot nonExpiring = generateSnapshotDetails("non-expiring", null, false); List<TableSnapshot> snapshots = Arrays.asList(expired, nonExpired, nonExpiring); // Create SnapshotManager with 3 snapshots: expired, non-expired and non-expiring - SnapshotManager manager = new SnapshotManager(3, 3); - manager.addSnapshots(snapshots); + manager.start(false); + for (TableSnapshot snapshot : snapshots) + manager.addSnapshot(snapshot); // Only expiring snapshots should be loaded - assertThat(manager.getExpiringSnapshots()).hasSize(2); - assertThat(manager.getExpiringSnapshots()).contains(expired); - assertThat(manager.getExpiringSnapshots()).contains(nonExpired); + assertThat(manager.getSnapshots(TableSnapshot::isExpiring)).hasSize(2); + assertThat(manager.getSnapshots(TableSnapshot::isExpiring)).contains(expired); + assertThat(manager.getSnapshots(TableSnapshot::isExpiring)).contains(nonExpired); } @Test - public void testClearExpiredSnapshots() throws Exception { - SnapshotManager manager = new SnapshotManager(3, 3); + public void testClearExpiredSnapshots() + { + manager.start(false); // Add 3 snapshots: expired, non-expired and non-expiring TableSnapshot expired = generateSnapshotDetails("expired", Instant.EPOCH, false); @@ -106,119 +123,66 @@ public void testClearExpiredSnapshots() throws Exception { manager.addSnapshot(nonExpiring); // Only expiring snapshot should be indexed and all should exist - assertThat(manager.getExpiringSnapshots()).hasSize(2); - assertThat(manager.getExpiringSnapshots()).contains(expired); - assertThat(manager.getExpiringSnapshots()).contains(nonExpired); + assertThat(manager.getSnapshots(TableSnapshot::isExpiring)).hasSize(2); + assertThat(manager.getSnapshots(TableSnapshot::isExpiring)).contains(expired); + assertThat(manager.getSnapshots(TableSnapshot::isExpiring)).contains(nonExpired); assertThat(expired.exists()).isTrue(); assertThat(nonExpired.exists()).isTrue(); assertThat(nonExpiring.exists()).isTrue(); // After clearing expired snapshots, expired snapshot should be removed while the others should remain manager.clearExpiredSnapshots(); - assertThat(manager.getExpiringSnapshots()).hasSize(1); - assertThat(manager.getExpiringSnapshots()).contains(nonExpired); + assertThat(manager.getSnapshots(TableSnapshot::isExpiring)).hasSize(1); + assertThat(manager.getSnapshots(TableSnapshot::isExpiring)).contains(nonExpired); assertThat(expired.exists()).isFalse(); assertThat(nonExpired.exists()).isTrue(); assertThat(nonExpiring.exists()).isTrue(); } @Test - public void testScheduledCleanup() throws Exception { - SnapshotManager manager = new SnapshotManager(0, 1); - try - { - // Start snapshot manager which should start expired snapshot cleanup thread - manager.start(); - - // Add 2 expiring snapshots: one to expire in 2 seconds, another in 1 day - int TTL_SECS = 2; - TableSnapshot toExpire = generateSnapshotDetails("to-expire", now().plusSeconds(TTL_SECS), false); - TableSnapshot nonExpired = generateSnapshotDetails("non-expired", now().plusMillis(ONE_DAY_SECS), false); - manager.addSnapshot(toExpire); - manager.addSnapshot(nonExpired); - - // Check both snapshots still exist - assertThat(toExpire.exists()).isTrue(); - assertThat(nonExpired.exists()).isTrue(); - assertThat(manager.getExpiringSnapshots()).hasSize(2); - assertThat(manager.getExpiringSnapshots()).contains(toExpire); - assertThat(manager.getExpiringSnapshots()).contains(nonExpired); - - // Sleep 4 seconds - Thread.sleep((TTL_SECS + 2) * 1000L); - - // Snapshot with ttl=2s should be gone, while other should remain - assertThat(manager.getExpiringSnapshots()).hasSize(1); - assertThat(manager.getExpiringSnapshots()).contains(nonExpired); - assertThat(toExpire.exists()).isFalse(); - assertThat(nonExpired.exists()).isTrue(); - } - finally - { - manager.stop(); - } + public void testScheduledCleanup() throws Exception + { + manager.start(true); + + // Add 2 expiring snapshots: one to expire in 6 seconds, another in 1 day + int TTL_SECS = 6; + TableSnapshot toExpire = generateSnapshotDetails("to-expire", now().plusSeconds(TTL_SECS), false); + TableSnapshot nonExpired = generateSnapshotDetails("non-expired", now().plusMillis(ONE_DAY_SECS), false); + manager.addSnapshot(toExpire); + manager.addSnapshot(nonExpired); + + // Check both snapshots still exist + assertThat(toExpire.exists()).isTrue(); + assertThat(nonExpired.exists()).isTrue(); + assertThat(manager.getSnapshots(TableSnapshot::isExpiring)).hasSize(2); + assertThat(manager.getSnapshots(TableSnapshot::isExpiring)).contains(toExpire); + assertThat(manager.getSnapshots(TableSnapshot::isExpiring)).contains(nonExpired); + + // Sleep 10 seconds + Thread.sleep((TTL_SECS + 4) * 1000L); + + // Snapshot with ttl=6s should be gone, while other should remain + assertThat(manager.getSnapshots(TableSnapshot::isExpiring)).hasSize(1); + assertThat(manager.getSnapshots(TableSnapshot::isExpiring)).contains(nonExpired); + assertThat(toExpire.exists()).isFalse(); + assertThat(nonExpired.exists()).isTrue(); } @Test - public void testClearSnapshot() throws Exception + public void testClearSnapshot() { // Given - SnapshotManager manager = new SnapshotManager(1, 3); + manager.start(false); TableSnapshot expiringSnapshot = generateSnapshotDetails("snapshot", now().plusMillis(50000), false); manager.addSnapshot(expiringSnapshot); - assertThat(manager.getExpiringSnapshots()).contains(expiringSnapshot); + assertThat(manager.getSnapshots(TableSnapshot::isExpiring)).contains(expiringSnapshot); assertThat(expiringSnapshot.exists()).isTrue(); // When manager.clearSnapshot(expiringSnapshot); // Then - assertThat(manager.getExpiringSnapshots()).doesNotContain(expiringSnapshot); + assertThat(manager.getSnapshots(TableSnapshot::isExpiring)).doesNotContain(expiringSnapshot); assertThat(expiringSnapshot.exists()).isFalse(); } - - @Test // see CASSANDRA-18211 - public void testConcurrentClearingOfSnapshots() throws Exception - { - - AtomicReference<Long> firstInvocationTime = new AtomicReference<>(0L); - AtomicReference<Long> secondInvocationTime = new AtomicReference<>(0L); - - SnapshotManager manager = new SnapshotManager(0, 5) - { - @Override - public synchronized void clearSnapshot(TableSnapshot snapshot) - { - if (snapshot.getTag().equals("mysnapshot")) - { - firstInvocationTime.set(currentTimeMillis()); - Uninterruptibles.sleepUninterruptibly(10, SECONDS); - } - else if (snapshot.getTag().equals("mysnapshot2")) - { - secondInvocationTime.set(currentTimeMillis()); - } - super.clearSnapshot(snapshot); - } - }; - - TableSnapshot expiringSnapshot = generateSnapshotDetails("mysnapshot", Instant.now().plusSeconds(15), false); - manager.addSnapshot(expiringSnapshot); - - manager.resumeSnapshotCleanup(); - - Thread nonExpiringSnapshotCleanupThred = new Thread(() -> manager.clearSnapshot(generateSnapshotDetails("mysnapshot2", null, false))); - - // wait until the first snapshot expires - await().pollInterval(1, SECONDS) - .pollDelay(0, SECONDS) - .timeout(1, MINUTES) - .until(() -> firstInvocationTime.get() > 0); - - // this will block until the first snapshot is cleaned up - nonExpiringSnapshotCleanupThred.start(); - nonExpiringSnapshotCleanupThred.join(); - - assertTrue(secondInvocationTime.get() - firstInvocationTime.get() > 10_000); - } } \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/service/snapshot/SnapshotLoaderTest.java b/test/unit/org/apache/cassandra/service/snapshot/SnapshotLoaderTest.java index 9c2303f84293..54ecf19d98f9 100644 --- a/test/unit/org/apache/cassandra/service/snapshot/SnapshotLoaderTest.java +++ b/test/unit/org/apache/cassandra/service/snapshot/SnapshotLoaderTest.java @@ -28,10 +28,12 @@ import java.util.concurrent.ThreadLocalRandom; import org.junit.Assert; +import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.db.Directories; import org.apache.cassandra.io.util.File; @@ -67,6 +69,12 @@ public class SnapshotLoaderTest @ClassRule public static TemporaryFolder tmpDir = new TemporaryFolder(); + @BeforeClass + public static void setup() + { + DatabaseDescriptor.daemonInitialization(); + } + @Test public void testMatcher() { diff --git a/test/unit/org/apache/cassandra/service/snapshot/SnapshotManagerTest.java b/test/unit/org/apache/cassandra/service/snapshot/SnapshotManagerTest.java new file mode 100644 index 000000000000..703e7996add2 --- /dev/null +++ b/test/unit/org/apache/cassandra/service/snapshot/SnapshotManagerTest.java @@ -0,0 +1,274 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.snapshot; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.time.Instant; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.function.Consumer; + +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.io.FSWriteError; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.io.util.PathUtils; +import org.mockito.MockedStatic; +import org.mockito.Mockito; + +import static java.lang.String.format; +import static org.apache.cassandra.config.CassandraRelevantProperties.SNAPSHOT_CLEANUP_INITIAL_DELAY_SECONDS; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class SnapshotManagerTest +{ + @ClassRule + public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @ClassRule + public static TemporaryFolder temporaryFolder2 = new TemporaryFolder(); + + private static File rootDir1; + private static File rootDir2; + + private static String[] dataDirs; + + @BeforeClass + public static void beforeClass() throws Exception + { + CassandraRelevantProperties.SNAPSHOT_CLEANUP_PERIOD_SECONDS.setInt(10); + SNAPSHOT_CLEANUP_INITIAL_DELAY_SECONDS.setInt(5); + + rootDir1 = new File(temporaryFolder.getRoot()); + rootDir2 = new File(temporaryFolder2.getRoot()); + + dataDirs = new String[]{ + rootDir1.toPath().toAbsolutePath().toString(), + rootDir2.toPath().toAbsolutePath().toString() + }; + } + + private static class ThrowingTask extends AbstractSnapshotTask<Void> + { + public ThrowingTask() + { + super(null); + } + + @Override + public Void call() + { + throw new RuntimeException("an exception"); + } + + @Override + public SnapshotTaskType getTaskType() + { + return SnapshotTaskType.SNAPSHOT; + } + } + + @Test + public void testTaskThrowingException() + { + doWithManager(manager -> assertThatThrownBy(() -> manager.executeTask(new ThrowingTask())).isInstanceOf(RuntimeException.class) + .hasRootCauseMessage("an exception") + .hasMessageContaining("Exception occured while executing")); + + doWithManager(manager -> { + + manager.close(); + + assertThatThrownBy(() -> manager.executeTask(new ThrowingTask())).isInstanceOf(RuntimeException.class) + .hasRootCauseMessage("an exception") + .hasMessageContaining("Exception occured while executing"); + + manager.start(false); + + assertThatThrownBy(() -> manager.executeTask(new ThrowingTask())).isInstanceOf(RuntimeException.class) + .hasRootCauseMessage("an exception") + .hasMessageContaining("Exception occured while executing"); + }); + } + + /** + * Tests that if we remove all manifests files, that equals to stopping manager to track + * such snapshot, however no data will be removed. + */ + @Test + public void testRemovingManifestsLogicallyRemovesSnapshot() + { + doWithManager(manager -> { + List<TableSnapshot> tableSnapshots = generateTableSnapshots(10, 100); + + for (TableSnapshot snapshot : tableSnapshots) + manager.addSnapshot(snapshot); + + // we still have 1000 snapshots because we removed just one manifest + removeManifestOfSnapshot(tableSnapshots.get(0)); + assertEquals(1000, manager.getSnapshots((t) -> true).size()); + + // remove the second manifest, that will render snapshot to be logically removed + removeManifestOfSnapshot(tableSnapshots.get(0)); + assertEquals(999, manager.getSnapshots((t) -> true).size()); + + // check that data are still there + assertFalse(tableSnapshots.get(0).hasManifest()); + assertTrue(tableSnapshots.get(0).exists()); + for (File snapshotDir : tableSnapshots.get(0).getDirectories()) + assertTrue(snapshotDir.exists()); + }); + } + + private void doWithManager(Consumer<SnapshotManager> action) + { + try (MockedStatic<DatabaseDescriptor> mockedDD = Mockito.mockStatic(DatabaseDescriptor.class)) + { + mockedDD.when(() -> DatabaseDescriptor.getNonLocalSystemKeyspacesDataFileLocations()).thenReturn(dataDirs); + mockedDD.when(() -> DatabaseDescriptor.getLocalSystemKeyspacesDataFileLocations()).thenReturn(new String[]{}); + mockedDD.when(() -> DatabaseDescriptor.getAllDataFileLocations()).thenReturn(dataDirs); + mockedDD.when(() -> DatabaseDescriptor.getDumpHeapOnUncaughtException()).thenReturn(false); + + try (SnapshotManager snapshotManager = new SnapshotManager(SNAPSHOT_CLEANUP_INITIAL_DELAY_SECONDS.getInt(10), + SNAPSHOT_CLEANUP_INITIAL_DELAY_SECONDS.getInt(5), + dataDirs).start(true)) + { + action.accept(snapshotManager); + } + finally + { + clearDirectory(rootDir1.toPath()); + clearDirectory(rootDir2.toPath()); + } + } + } + + private List<TableSnapshot> generateTableSnapshots(int keyspaces, int tables) throws RuntimeException + { + List<TableSnapshot> tableSnapshots = new ArrayList<>(); + for (int i = 0; i < keyspaces; i++) + { + for (int j = 0; j < tables; j++) + { + String snapshotName = format("mysnapshot_%s_%s", i, j); + File dir1 = new File(Paths.get(rootDir1.absolutePath(), "ks", "tb-1b255f4def2540a60000000000000005", "snapshots", snapshotName)); + File dir2 = new File(Paths.get(rootDir2.absolutePath(), "ks", "tb-1b255f4def2540a60000000000000005", "snapshots", snapshotName)); + dir1.tryCreateDirectories(); + dir2.tryCreateDirectories(); + TableSnapshot snapshot = generateSnapshotDetails(Set.of(dir1, dir2), snapshotName, "ks", "tb-1b255f4def2540a60000000000000005", null, false); + SnapshotManifest manifest = new SnapshotManifest(List.of(), null, snapshot.getCreatedAt(), snapshot.isEphemeral()); + try + { + manifest.serializeToJsonFile(new File(dir1.toPath().resolve("manifest.json"))); + manifest.serializeToJsonFile(new File(dir2.toPath().resolve("manifest.json"))); + generateFileInSnapshot(snapshot); + } + catch (Exception ex) + { + throw new RuntimeException(ex); + } + + tableSnapshots.add(snapshot); + } + } + + return tableSnapshots; + } + + private void generateFileInSnapshot(TableSnapshot tableSnapshot) throws IOException + { + for (File snapshotDir : tableSnapshot.getDirectories()) + Files.createFile(snapshotDir.toPath().resolve("schema.cql")); + } + + private void removeManifestOfSnapshot(TableSnapshot tableSnapshot) + { + for (File snapshotDir : tableSnapshot.getDirectories()) + { + if (snapshotDir.exists()) + { + File manifest = new File(snapshotDir, "manifest.json"); + if (!manifest.exists()) + continue; + + manifest.delete(); + return; + } + } + } + + private TableSnapshot generateSnapshotDetails(Set<File> roots, + String tag, + String keyspace, + String table, + Instant expiration, + boolean ephemeral) + { + try + { + Set<File> snapshotDirs = new HashSet<>(); + for (File root : roots) + { + root.tryCreateDirectories(); + snapshotDirs.add(root); + } + + return new TableSnapshot(keyspace, + table, + UUID.randomUUID(), + tag, + Instant.EPOCH, + expiration, + snapshotDirs, + ephemeral); + } + catch (Exception ex) + { + throw new RuntimeException(ex); + } + } + + + /** + * Empties everything in directory of "path" but keeps the directory itself. + * + * @param path directory to be emptied + * @throws FSWriteError if any part of the tree cannot be deleted + */ + public static void clearDirectory(Path path) + { + if (PathUtils.isDirectory(path)) + PathUtils.forEach(path, PathUtils::deleteRecursive); + } + +} diff --git a/test/unit/org/apache/cassandra/service/snapshot/SnapshotOptionsTest.java b/test/unit/org/apache/cassandra/service/snapshot/SnapshotOptionsTest.java new file mode 100644 index 000000000000..e61d283962ea --- /dev/null +++ b/test/unit/org/apache/cassandra/service/snapshot/SnapshotOptionsTest.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.snapshot; + +import java.time.Instant; +import java.util.EnumSet; +import java.util.List; + +import com.google.common.util.concurrent.RateLimiter; +import org.junit.Test; + +import static java.lang.String.format; +import static org.junit.Assert.assertEquals; + +public class SnapshotOptionsTest +{ + @Test + public void testSnapshotName() + { + List<SnapshotType> sameNameTypes = List.of(SnapshotType.DIAGNOSTICS, SnapshotType.REPAIR, SnapshotType.USER); + + for (SnapshotType type : sameNameTypes) + { + SnapshotOptions options = SnapshotOptions.systemSnapshot("a_name", type, "ks.tb") + .rateLimiter(RateLimiter.create(5)) + .build(); + + String snapshotName = options.getSnapshotName(Instant.now()); + assertEquals("a_name", snapshotName); + } + + EnumSet<SnapshotType> snapshotTypes = EnumSet.allOf(SnapshotType.class); + snapshotTypes.removeAll(sameNameTypes); + + for (SnapshotType type : snapshotTypes) + { + SnapshotOptions options = SnapshotOptions.systemSnapshot("a_name", type, "ks.tb") + .rateLimiter(RateLimiter.create(5)) + .build(); + + Instant now = Instant.now(); + + String snapshotName = options.getSnapshotName(now); + + assertEquals(format("%d-%s-%s", now.toEpochMilli(), type.label, "a_name"), snapshotName); + } + } +} diff --git a/test/unit/org/apache/cassandra/service/snapshot/TableSnapshotTest.java b/test/unit/org/apache/cassandra/service/snapshot/TableSnapshotTest.java index d592dd91d284..3ec0064c732a 100644 --- a/test/unit/org/apache/cassandra/service/snapshot/TableSnapshotTest.java +++ b/test/unit/org/apache/cassandra/service/snapshot/TableSnapshotTest.java @@ -19,7 +19,7 @@ package org.apache.cassandra.service.snapshot; import java.io.IOException; -import java.nio.file.Paths; +import java.nio.file.Files; import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; @@ -38,10 +38,16 @@ import org.apache.cassandra.io.util.FileOutputStreamPlus; import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.utils.Pair; +import org.mockito.MockedStatic; import static org.apache.cassandra.utils.FBUtilities.now; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.times; public class TableSnapshotTest { @@ -62,13 +68,68 @@ public static Set<File> createFolders(TemporaryFolder temp) throws IOException { File subfolder = new File(folder, folderName); subfolder.tryCreateDirectories(); - assertThat(subfolder.exists()); + File manifest = new File(subfolder.toPath().resolve("manifest.json")); + File schema = new File(subfolder.toPath().resolve("schema.cql")); + manifest.createFileIfNotExists(); + schema.createFileIfNotExists(); + Files.write(manifest.toPath(), "{}".getBytes()); + Files.write(schema.toPath(), "cql schema".getBytes()); folders.add(subfolder); } - ; + return folders; } + @Test + public void testTableSnapshotEquality() throws IOException + { + TableSnapshot snapshot1 = new TableSnapshot("ks", "tbl", UUID.randomUUID(), "some", null, null, createFolders(tempFolder), false); + TableSnapshot snapshot2 = new TableSnapshot("ks", "tbl", UUID.randomUUID(), "some", null, null, createFolders(tempFolder), false); + + // they are not equal, because table id is random for each + assertNotEquals(snapshot1, snapshot2); + + UUID tableId = UUID.randomUUID(); + + snapshot1 = new TableSnapshot("ks", "tbl", tableId, "some", null, null, createFolders(tempFolder), false); + snapshot2 = new TableSnapshot("ks", "tbl", tableId, "some", null, null, createFolders(tempFolder), false); + + // they are equal, even their directories differ + assertEquals(snapshot1, snapshot2); + + Set<File> folders = createFolders(tempFolder); + + snapshot1 = new TableSnapshot("ks", "tbl", tableId, "some", Instant.now(), null, folders, false); + snapshot2 = new TableSnapshot("ks", "tbl", tableId, "some", Instant.now().plusSeconds(1), null, folders, false); + + // they are equal, even their creation times differ + assertEquals(snapshot1, snapshot2); + + snapshot1 = new TableSnapshot("ks", "tbl", tableId, "some", null, Instant.now(), folders, false); + snapshot2 = new TableSnapshot("ks", "tbl", tableId, "some", null, Instant.now().plusSeconds(1), folders, false); + + // they are equal, even their expiration times differ + assertEquals(snapshot1, snapshot2); + + snapshot1 = new TableSnapshot("ks", "tbl", tableId, "some", null, null, folders, false); + snapshot2 = new TableSnapshot("ks", "tbl", UUID.randomUUID(), "some", null, null, folders, false); + + // they are not equal, because their tableId differs + assertNotEquals(snapshot1, snapshot2); + + snapshot1 = new TableSnapshot("ks", "tbl", tableId, "some1", null, null, folders, false); + snapshot2 = new TableSnapshot("ks", "tbl", tableId, "some2", null, null, folders, false); + + // they are not equal, because their tag differs + assertNotEquals(snapshot1, snapshot2); + + snapshot1 = new TableSnapshot("ks", "tbl", tableId, "some1", null, null, folders, false); + snapshot2 = new TableSnapshot("ks", "tbl2", tableId, "some1", null, null, folders, false); + + // they are not equal, because their table differs + assertNotEquals(snapshot1, snapshot2); + } + @Test public void testSnapshotExists() throws IOException { @@ -186,8 +247,18 @@ public void testComputeSizeOnDisk() throws IOException res += FileUtils.folderSize(dir); } - assertThat(tableDetails.computeSizeOnDiskBytes()).isGreaterThan(0L); - assertThat(tableDetails.computeSizeOnDiskBytes()).isEqualTo(res); + try (MockedStatic<FileUtils> fileUtilsMock = mockStatic(FileUtils.class)) + { + fileUtilsMock.when(() -> FileUtils.folderSize(any())).thenCallRealMethod(); + + assertThat(tableDetails.computeSizeOnDiskBytes()).isGreaterThan(0L); + assertThat(tableDetails.computeSizeOnDiskBytes()).isEqualTo(res); + + // when we invoke computeSizeOnDiskBytes for the second time, it will use cached value + // 3 invocations for folderSize are from the first invocation of computeSizeOnDiskBytes because + // we have 3 data dirs, if we have not cached it, the number of invocations would be 6. + fileUtilsMock.verify(() -> FileUtils.folderSize(any()), times(3)); + } } @Test @@ -208,15 +279,27 @@ public void testComputeTrueSize() throws IOException Long res = 0L; + Set<String> files = new HashSet<>(); for (File dir : folders) { File file = new File(dir, "tmp"); + files.add(file.toAbsolute().toString()); writeBatchToFile(file); res += file.length(); + res += new File(dir, "manifest.json").length(); + res += new File(dir, "schema.cql").length(); } - assertThat(tableDetails.computeTrueSizeBytes()).isGreaterThan(0L); - assertThat(tableDetails.computeTrueSizeBytes()).isEqualTo(res); + try (MockedStatic<TableSnapshot> tableSnapshotMock = mockStatic(TableSnapshot.class)) + { + tableSnapshotMock.when(() -> TableSnapshot.getLiveFileFromSnapshotFile(any())).thenCallRealMethod(); + + assertThat(tableDetails.computeTrueSizeBytes(files)).isGreaterThan(0L); + assertThat(tableDetails.computeTrueSizeBytes(files)).isEqualTo(res); + + // 6 because we avoided to call it for manifest and schema because they are cached + tableSnapshotMock.verify(() -> TableSnapshot.getLiveFileFromSnapshotFile(any()), times(6)); + } } @Test @@ -298,7 +381,7 @@ public void testShouldClearSnapshot() throws Exception // 1. snapshot to clear is not ephemeral // 2. tag to clear is null, empty, or it is equal to snapshot tag // 3. byTimestamp is true - if (TableSnapshot.shouldClearSnapshot(testingTag, olderThanTimestamp).test(snapshot)) + if (ClearSnapshotTask.getClearSnapshotPredicate(testingTag, Set.of(keyspace), olderThanTimestamp, false).test(snapshot)) { // shouldClearTag = true boolean shouldClearTag = (testingTag == null || testingTag.isEmpty()) || snapshot.getTag().equals(testingTag); @@ -321,23 +404,4 @@ public void testShouldClearSnapshot() throws Exception } } } - - @Test - public void testGetLiveFileFromSnapshotFile() - { - testGetLiveFileFromSnapshotFile("~/.ccm/test/node1/data0/test_ks/tbl-e03faca0813211eca100c705ea09b5ef/snapshots/1643481737850/me-1-big-Data.db", - "~/.ccm/test/node1/data0/test_ks/tbl-e03faca0813211eca100c705ea09b5ef/me-1-big-Data.db"); - } - - @Test - public void testGetLiveFileFromSnapshotIndexFile() - { - testGetLiveFileFromSnapshotFile("~/.ccm/test/node1/data0/test_ks/tbl-e03faca0813211eca100c705ea09b5ef/snapshots/1643481737850/.tbl_val_idx/me-1-big-Summary.db", - "~/.ccm/test/node1/data0/test_ks/tbl-e03faca0813211eca100c705ea09b5ef/.tbl_val_idx/me-1-big-Summary.db"); - } - - public void testGetLiveFileFromSnapshotFile(String snapshotFile, String expectedLiveFile) - { - assertThat(TableSnapshot.getLiveFileFromSnapshotFile(Paths.get(snapshotFile)).toString()).isEqualTo(expectedLiveFile); - } } diff --git a/test/unit/org/apache/cassandra/tools/StandaloneUpgraderOnSStablesTest.java b/test/unit/org/apache/cassandra/tools/StandaloneUpgraderOnSStablesTest.java index 13705f9b34b9..99655ed32e53 100644 --- a/test/unit/org/apache/cassandra/tools/StandaloneUpgraderOnSStablesTest.java +++ b/test/unit/org/apache/cassandra/tools/StandaloneUpgraderOnSStablesTest.java @@ -18,7 +18,6 @@ package org.apache.cassandra.tools; -import java.util.Collections; import java.util.List; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -34,7 +33,7 @@ import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.exceptions.StartupException; import org.apache.cassandra.io.sstable.LegacySSTableTest; -import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.tools.ToolRunner.ToolResult; import org.assertj.core.api.Assertions; @@ -93,9 +92,7 @@ public void testUpgradeSnapshot() throws Throwable { LegacySSTableTest.truncateLegacyTables(legacyId); LegacySSTableTest.loadLegacyTables(legacyId); - StorageService.instance.takeSnapshot("testsnapshot", - Collections.emptyMap(), - "legacy_tables.legacy_" + legacyId + "_simple"); + SnapshotManager.instance.takeSnapshot("testsnapshot", "legacy_tables.legacy_" + legacyId + "_simple"); ToolResult tool = ToolRunner.invokeClass(StandaloneUpgrader.class, "-k", diff --git a/test/unit/org/apache/cassandra/tools/nodetool/ClearSnapshotTest.java b/test/unit/org/apache/cassandra/tools/nodetool/ClearSnapshotTest.java index 379d02a66e9f..b0de9d32cde6 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/ClearSnapshotTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/ClearSnapshotTest.java @@ -30,6 +30,7 @@ import javax.management.openmbean.TabularData; import org.junit.AfterClass; +import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -37,6 +38,7 @@ import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.io.util.File; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.service.snapshot.SnapshotManifest; import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.tools.ToolRunner.ToolResult; @@ -65,6 +67,12 @@ public static void setup() throws Exception probe = new NodeProbe(jmxHost, jmxPort); } + @Before + public void clearAllSnapshots() + { + SnapshotManager.instance.clearAllSnapshots(); + } + @AfterClass public static void teardown() throws IOException { @@ -315,11 +323,15 @@ private void prepareData(Instant start) throws Throwable String tableId2 = DASH_PATTERN.matcher(tableMetadata2.orElseThrow(() -> new IllegalStateException(format("no metadata found for %s.%s", keyspace2, tableName2))) .id.asUUID().toString()).replaceAll(""); + SnapshotManager.instance.close(); + rewriteManifest(tableId, getAllDataFileLocations(), KEYSPACE, tableName, "snapshot-to-clear-ks1-tb1", start.minus(5, HOURS)); rewriteManifest(tableId, getAllDataFileLocations(), KEYSPACE, tableName, "some-other-snapshot-ks1-tb1", start.minus(2, HOURS)); rewriteManifest(tableId, getAllDataFileLocations(), KEYSPACE, tableName, "last-snapshot-ks1-tb1", start.minus(1, SECONDS)); rewriteManifest(tableId2, getAllDataFileLocations(), keyspace2, tableName2, "snapshot-to-clear-ks2-tb2", start.minus(5, HOURS)); rewriteManifest(tableId2, getAllDataFileLocations(), keyspace2, tableName2, "some-other-snapshot-ks2-tb2", start.minus(2, HOURS)); rewriteManifest(tableId2, getAllDataFileLocations(), keyspace2, tableName2, "last-snapshot-ks2-tb2", start.minus(1, SECONDS)); + + SnapshotManager.instance.start(true); } } diff --git a/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java b/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java index 426ccae58fd3..eb741c6915c7 100644 --- a/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java +++ b/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java @@ -340,7 +340,7 @@ public void run() } double currentSizeGiB; - while ((currentSizeGiB = directories.getRawDiretoriesSize() / BYTES_IN_GIB) < totalSizeGiB) + while ((currentSizeGiB = directories.getRawDirectoriesSize() / BYTES_IN_GIB) < totalSizeGiB) { if (finished.getCount() == 0) break; @@ -353,7 +353,7 @@ public void run() workManager.stop(); Uninterruptibles.awaitUninterruptibly(finished); - currentSizeGiB = directories.getRawDiretoriesSize() / BYTES_IN_GIB; + currentSizeGiB = directories.getRawDirectoriesSize() / BYTES_IN_GIB; System.out.println(String.format("Finished writing %.2fGB", currentSizeGiB)); } } From ce035b021542717a61972a00fe341d9ac9f829d3 Mon Sep 17 00:00:00 2001 From: Caleb Rackliffe <calebrackliffe@gmail.com> Date: Mon, 9 Dec 2024 17:21:21 -0600 Subject: [PATCH 048/225] Restore optimization in MultiCBuilder around building one clustering patch by Caleb Rackliffe; reviewed by Benjamin Lerer, Abe Ratnofsky, and Bernardo Botella for CASSANDRA-20129 --- CHANGES.txt | 1 + src/java/org/apache/cassandra/db/MultiCBuilder.java | 6 ++++++ 2 files changed, 7 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index 88443f88c32d..b3c4f13bec1a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Restore optimization in MultiCBuilder around building one clustering (CASSANDRA-20129) * Consolidate all snapshot management to SnapshotManager and introduce SnapshotManagerMBean (CASSANDRA-18111) * Fix RequestFailureReason constants codes (CASSANDRA-20126) * Introduce SSTableSimpleScanner for compaction (CASSANDRA-20092) diff --git a/src/java/org/apache/cassandra/db/MultiCBuilder.java b/src/java/org/apache/cassandra/db/MultiCBuilder.java index 99bbc64a65e2..6e9cf5cd5bb6 100644 --- a/src/java/org/apache/cassandra/db/MultiCBuilder.java +++ b/src/java/org/apache/cassandra/db/MultiCBuilder.java @@ -17,6 +17,7 @@ */ package org.apache.cassandra.db; +import java.nio.ByteBuffer; import java.util.List; import java.util.NavigableSet; import java.util.Set; @@ -193,6 +194,11 @@ public NavigableSet<Clustering<?>> build() if (clusterings.isEmpty()) return BTreeSet.of(comparator, Clustering.EMPTY); + // If we have one clustering, avoid unnecessary builder and iterator allocations that would otherwise follow. + // This is a very hot path, touched by both reads and writes. + if (clusterings.size() == 1) + return BTreeSet.of(comparator, Clustering.make(clusterings.get(0).toArray(new ByteBuffer[comparator.size()]))); + CBuilder builder = CBuilder.create(comparator); BTreeSet.Builder<Clustering<?>> set = BTreeSet.builder(builder.comparator()); From 9bf1680b1f1c016717381aa59367950fa770f57f Mon Sep 17 00:00:00 2001 From: Marcus Eriksson <marcuse@apache.org> Date: Wed, 27 Nov 2024 08:47:42 +0100 Subject: [PATCH 049/225] Avoid prepared statement invalidation race when committing schema changes Patch by marcuse; reviewed by Sam Tunnicliffe for CASSANDRA-20116 --- CHANGES.txt | 1 + .../cassandra/schema/DistributedSchema.java | 69 ++++++--- .../tcm/listeners/SchemaListener.java | 1 + ...PreparedStatementInvalidationRaceTest.java | 139 ++++++++++++++++++ 4 files changed, 188 insertions(+), 22 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/test/PreparedStatementInvalidationRaceTest.java diff --git a/CHANGES.txt b/CHANGES.txt index b3c4f13bec1a..4d2ef6c3d42f 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Avoid prepared statement invalidation race when committing schema changes (CASSANDRA-20116) * Restore optimization in MultiCBuilder around building one clustering (CASSANDRA-20129) * Consolidate all snapshot management to SnapshotManager and introduce SnapshotManagerMBean (CASSANDRA-18111) * Fix RequestFailureReason constants codes (CASSANDRA-20126) diff --git a/src/java/org/apache/cassandra/schema/DistributedSchema.java b/src/java/org/apache/cassandra/schema/DistributedSchema.java index e10c1e00495c..ec0c65eb3a2e 100644 --- a/src/java/org/apache/cassandra/schema/DistributedSchema.java +++ b/src/java/org/apache/cassandra/schema/DistributedSchema.java @@ -187,16 +187,8 @@ public void initializeKeyspaceInstances(DistributedSchema prev, boolean loadSSTa SchemaChangeNotifier schemaChangeNotifier = Schema.instance.schemaChangeNotifier(); schemaChangeNotifier.notifyPreChanges(new SchemaTransformation.SchemaTransformationResult(prev, this, ksDiff)); - ksDiff.dropped.forEach(metadata -> { - schemaChangeNotifier.notifyKeyspaceDropped(metadata, loadSSTables); - dropKeyspace(metadata, true); - }); - - ksDiff.created.forEach(metadata -> { - schemaChangeNotifier.notifyKeyspaceCreated(metadata); - keyspaceInstances.put(metadata.name, new Keyspace(Schema.instance, metadata, loadSSTables)); - }); - + ksDiff.dropped.forEach(metadata -> dropKeyspace(metadata, true)); + ksDiff.created.forEach(metadata -> keyspaceInstances.put(metadata.name, new Keyspace(Schema.instance, metadata, loadSSTables))); ksDiff.altered.forEach(delta -> { boolean initialized = Keyspace.isInitialized(); @@ -218,12 +210,9 @@ public void initializeKeyspaceInstances(DistributedSchema prev, boolean loadSSTa delta.tables.altered.forEach(diff -> alterTable(keyspace, diff.after)); delta.views.altered.forEach(diff -> alterView(keyspace, diff.after)); - schemaChangeNotifier.notifyKeyspaceAltered(delta, loadSSTables); // deal with all added, and altered views keyspace.viewManager.reload(keyspaces.get(keyspace.getName()).get()); } - - SchemaDiagnostics.keyspaceAltered(Schema.instance, delta); }); // Avoid system table side effects during initialization @@ -234,6 +223,51 @@ public void initializeKeyspaceInstances(DistributedSchema prev, boolean loadSSTa } } + public void notifyPostCommit(DistributedSchema prevSchema, boolean loadSSTables) + { + if (!prevSchema.isEmpty() && prevSchema.keyspaceInstances.isEmpty()) + prevSchema = DistributedSchema.empty(); + + Keyspaces.KeyspacesDiff ksDiff = Keyspaces.diff(prevSchema.getKeyspaces(), getKeyspaces()); + + SchemaChangeNotifier schemaChangeNotifier = Schema.instance.schemaChangeNotifier(); + + ksDiff.dropped.forEach(metadata -> { + schemaChangeNotifier.notifyKeyspaceDropped(metadata, loadSSTables); + if (Keyspace.isInitialized()) + { + metadata.views.forEach((tableMetadata) -> SchemaDiagnostics.tableDropped(Schema.instance, tableMetadata.metadata)); + metadata.tables.forEach((tableMetadata) -> SchemaDiagnostics.tableDropped(Schema.instance, tableMetadata)); + SchemaDiagnostics.metadataRemoved(Schema.instance, metadata); + SchemaDiagnostics.keyspaceDropped(Schema.instance, metadata); + } + }); + ksDiff.created.forEach(schemaChangeNotifier::notifyKeyspaceCreated); + ksDiff.altered.forEach(delta -> { + boolean initialized = Keyspace.isInitialized(); + Keyspace keyspace = initialized ? keyspaceInstances.get(delta.before.name) : null; + if (initialized) + { + assert keyspace != null : String.format("Keyspace %s is not initialized. Initialized keyspaces: %s.", delta.before.name, keyspaceInstances.keySet()); + assert delta.before.name.equals(delta.after.name); + schemaChangeNotifier.notifyKeyspaceAltered(delta, loadSSTables); + + // drop tables and views + delta.views.dropped.forEach(v -> SchemaDiagnostics.tableDropped(Schema.instance, v.metadata)); + delta.tables.dropped.forEach(t -> SchemaDiagnostics.tableDropped(Schema.instance, t)); + + // add tables and views + delta.tables.created.forEach(t -> SchemaDiagnostics.tableCreated(Schema.instance, t)); + delta.views.created.forEach(v -> SchemaDiagnostics.tableCreated(Schema.instance, v.metadata)); + + // update tables and views + delta.tables.altered.forEach(diff -> SchemaDiagnostics.tableAltered(Schema.instance, diff.after)); + delta.views.altered.forEach(diff -> SchemaDiagnostics.tableAltered(Schema.instance, diff.after.metadata)); + } + SchemaDiagnostics.keyspaceAltered(Schema.instance, delta); + }); + } + public static void maybeRebuildViews(DistributedSchema prev, DistributedSchema current) { Keyspaces.KeyspacesDiff ksDiff = Keyspaces.diff(prev.getKeyspaces(), current.getKeyspaces()); @@ -274,7 +308,6 @@ private void dropKeyspace(KeyspaceMetadata keyspaceMetadata, boolean dropData) // remove the keyspace from the static instances Keyspace unloadedKeyspace = keyspaceInstances.remove(keyspaceMetadata.name); unloadedKeyspace.unload(true); - SchemaDiagnostics.metadataRemoved(Schema.instance, keyspaceMetadata); assert unloadedKeyspace == keyspace; Keyspace.writeOrder.awaitNewBarrier(); @@ -282,10 +315,7 @@ private void dropKeyspace(KeyspaceMetadata keyspaceMetadata, boolean dropData) else { keyspace.unload(true); - SchemaDiagnostics.metadataRemoved(Schema.instance, keyspaceMetadata); } - - SchemaDiagnostics.keyspaceDropped(Schema.instance, keyspaceMetadata); } /** * @@ -296,35 +326,30 @@ private void dropTable(Keyspace keyspace, TableMetadata metadata, boolean dropDa { SchemaDiagnostics.tableDropping(Schema.instance, metadata); keyspace.dropCf(metadata.id, dropData); - SchemaDiagnostics.tableDropped(Schema.instance, metadata); } private void createTable(Keyspace keyspace, TableMetadata table, boolean loadSSTables) { SchemaDiagnostics.tableCreating(Schema.instance, table); keyspace.initCf(table, loadSSTables); - SchemaDiagnostics.tableCreated(Schema.instance, table); } private void createView(Keyspace keyspace, ViewMetadata view) { SchemaDiagnostics.tableCreating(Schema.instance, view.metadata); keyspace.initCf(view.metadata, true); - SchemaDiagnostics.tableCreated(Schema.instance, view.metadata); } private void alterTable(Keyspace keyspace, TableMetadata updated) { SchemaDiagnostics.tableAltering(Schema.instance, updated); keyspace.getColumnFamilyStore(updated.name).reload(updated); - SchemaDiagnostics.tableAltered(Schema.instance, updated); } private void alterView(Keyspace keyspace, ViewMetadata updated) { SchemaDiagnostics.tableAltering(Schema.instance, updated.metadata); keyspace.getColumnFamilyStore(updated.name()).reload(updated.metadata); - SchemaDiagnostics.tableAltered(Schema.instance, updated.metadata); } public Keyspaces getKeyspaces() diff --git a/src/java/org/apache/cassandra/tcm/listeners/SchemaListener.java b/src/java/org/apache/cassandra/tcm/listeners/SchemaListener.java index e3507bf2bb76..2dadbb5007d9 100644 --- a/src/java/org/apache/cassandra/tcm/listeners/SchemaListener.java +++ b/src/java/org/apache/cassandra/tcm/listeners/SchemaListener.java @@ -55,6 +55,7 @@ public void notifyPostCommit(ClusterMetadata prev, ClusterMetadata next, boolean if (!fromSnapshot && next.schema.lastModified().equals(prev.schema.lastModified())) return; + next.schema.notifyPostCommit(prev.schema, loadSSTables); DistributedSchema.maybeRebuildViews(prev.schema, next.schema); SchemaDiagnostics.versionUpdated(Schema.instance); Gossiper.instance.addLocalApplicationState(ApplicationState.SCHEMA, StorageService.instance.valueFactory.schema(next.schema.getVersion())); diff --git a/test/distributed/org/apache/cassandra/distributed/test/PreparedStatementInvalidationRaceTest.java b/test/distributed/org/apache/cassandra/distributed/test/PreparedStatementInvalidationRaceTest.java new file mode 100644 index 000000000000..0a12855753db --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/PreparedStatementInvalidationRaceTest.java @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import java.util.Collection; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.junit.Test; + +import com.datastax.driver.core.PreparedStatement; +import com.datastax.driver.core.Session; +import net.bytebuddy.ByteBuddy; +import net.bytebuddy.dynamic.loading.ClassLoadingStrategy; +import net.bytebuddy.implementation.MethodDelegation; +import net.bytebuddy.implementation.bind.annotation.SuperCall; +import org.apache.cassandra.cql3.QueryHandler; +import org.apache.cassandra.cql3.QueryProcessor; +import org.apache.cassandra.cql3.statements.SelectStatement; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.distributed.api.ICluster; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.schema.DistributedSchema; +import org.apache.cassandra.schema.TableMetadata; + +import static net.bytebuddy.matcher.ElementMatchers.named; +import static net.bytebuddy.matcher.ElementMatchers.takesArguments; +import static org.apache.cassandra.distributed.api.Feature.GOSSIP; +import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL; +import static org.apache.cassandra.distributed.api.Feature.NETWORK; +import static org.junit.Assert.assertEquals; + +public class PreparedStatementInvalidationRaceTest extends TestBaseImpl +{ + @Test + public void testInvalidationRace() throws Exception + { + try (ICluster<IInvokableInstance> c = init(builder().withNodes(1) + .withConfig(config -> config.with(GOSSIP, NETWORK, NATIVE_PROTOCOL)) + .withInstanceInitializer(BBHelper::install) + .start())) + { + try (com.datastax.driver.core.Cluster cluster = com.datastax.driver.core.Cluster.builder() + .addContactPoint("127.0.0.1") + .build(); + Session s = cluster.connect()) + { + s.execute(withKeyspace("CREATE TABLE %s.tbl (pk int primary key)")); + PreparedStatement prepared = s.prepare(withKeyspace("select pk from %s.tbl where pk = ?")); + s.execute(prepared.bind(1)); + c.get(1).runOnInstance(() -> BBHelper.enabled.set(true)); + Thread t = new Thread(() -> s.execute(withKeyspace("alter table %s.tbl add x int"))); + t.start(); + c.get(1).runOnInstance(() -> await(BBHelper.initializeKeyspaceInstancesDone)); + // This is where the race existed before - we used to invalidate the statement in + // initializeKeyspaceInstances, but the schema change is not yet committed so the + // next execute would reprepare with the wrong tablemetadata (and keep it forever). + // Now we invalidate after committing - so the last execute below will reprepare the + // query, with the correct tablemetadata + s.execute(prepared.bind(1)); + c.get(1).runOnInstance(() -> { + BBHelper.enabled.set(false); + BBHelper.delayCommittingSchemaChange.countDown(); + }); + t.join(); + s.execute(prepared.bind(1)); + c.get(1).runOnInstance(() -> { + TableMetadata tableMetadata = Keyspace.open(KEYSPACE).getColumnFamilyStore("tbl").metadata(); + Collection<QueryHandler.Prepared> serverSidePrepared = QueryProcessor.instance.getPreparedStatements().values(); + assertEquals(1, serverSidePrepared.size()); + for (QueryHandler.Prepared ssp : QueryProcessor.instance.getPreparedStatements().values()) + assertEquals(tableMetadata.epoch, ((SelectStatement)ssp.statement).table.epoch); + }); + } + } + } + + public static class BBHelper + { + static AtomicBoolean enabled = new AtomicBoolean(); + static CountDownLatch delayCommittingSchemaChange = new CountDownLatch(1); + static CountDownLatch initializeKeyspaceInstancesDone = new CountDownLatch(1); + + public static void install(ClassLoader cl, int i) + { + new ByteBuddy().rebase(DistributedSchema.class) + .method(named("initializeKeyspaceInstances").and(takesArguments(2))) + .intercept(MethodDelegation.to(BBHelper.class)) + .make() + .load(cl, ClassLoadingStrategy.Default.INJECTION); + } + + public static void initializeKeyspaceInstances(DistributedSchema prev, boolean loadSSTables, @SuperCall Callable<Void> zuper) + { + try + { + zuper.call(); + if (enabled.get()) + { + initializeKeyspaceInstancesDone.countDown(); + delayCommittingSchemaChange.await(); + } + } + catch (Exception e) + { + throw new RuntimeException(e); + } + } + } + + private static void await(CountDownLatch cdl) + { + try + { + cdl.await(); + } + catch (InterruptedException e) + { + throw new RuntimeException(e); + } + } +} \ No newline at end of file From 0ed989e5544f02455a3f293db84658db41940d0c Mon Sep 17 00:00:00 2001 From: Marcus Eriksson <marcuse@apache.org> Date: Wed, 4 Dec 2024 15:20:50 +0100 Subject: [PATCH 050/225] Don't skip paxos repair on topology change Patch by marcuse; reviewed by Sam Tunnicliffe for CASSANDRA-20136 --- src/java/org/apache/cassandra/service/StorageService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index aa21f59ef2d6..9c20aa262d0b 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -5111,7 +5111,7 @@ public int getDefaultKeyspaceReplicationFactor() public boolean getSkipPaxosRepairOnTopologyChange() { - return true;//TODO //DatabaseDescriptor.skipPaxosRepairOnTopologyChange(); + return DatabaseDescriptor.skipPaxosRepairOnTopologyChange(); } public void setSkipPaxosRepairOnTopologyChange(boolean v) From f9ab5cf797013211c51c4e7d2dbd401300eb44e6 Mon Sep 17 00:00:00 2001 From: Marcus Eriksson <marcuse@apache.org> Date: Fri, 8 Nov 2024 13:39:51 +0100 Subject: [PATCH 051/225] Add flag to avoid invalidating key cache on sstable deletions Patch by marcuse; reviewed by Caleb Rackliffe for CASSANDRA-20068 --- CHANGES.txt | 1 + src/java/org/apache/cassandra/config/Config.java | 1 + .../cassandra/config/DatabaseDescriptor.java | 10 ++++++++++ .../io/sstable/format/big/BigFormat.java | 15 +++++++++------ .../apache/cassandra/service/StorageService.java | 10 ++++++++++ .../cassandra/service/StorageServiceMBean.java | 2 ++ .../io/sstable/keycache/KeyCacheTest.java | 6 +++--- 7 files changed, 36 insertions(+), 9 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 6fbc22f8fbd8..b038387817d9 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.3 + * Add flag to avoid invalidating key cache on sstable deletions (CASSANDRA-20068) * Interpret inet, bigint, varint, and decimal as non-reversed types for query construction and post-filtering (CASSANDRA-20100) * Fix delayed gossip shutdown messages clobbering startup states that leave restarted nodes appearing down (CASSANDRA-20033) * Streamline the serialized format for index status gossip messages (CASSANDRA-20058) diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 103d8a7cf7ea..8f6283ac1d4c 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -455,6 +455,7 @@ public static class SSTableConfig public volatile DataStorageSpec.IntMebibytesBound sstable_preemptive_open_interval = new DataStorageSpec.IntMebibytesBound("50MiB"); public volatile boolean key_cache_migrate_during_compaction = true; + public volatile boolean key_cache_invalidate_after_sstable_deletion = false; public volatile int key_cache_keys_to_save = Integer.MAX_VALUE; @Replaces(oldName = "key_cache_size_in_mb", converter = Converters.MEBIBYTES_DATA_STORAGE_LONG, deprecated = true) public DataStorageSpec.LongMebibytesBound key_cache_size = null; diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index bf44b53e2b0d..8fe910aad717 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -3760,6 +3760,16 @@ public static void setMigrateKeycacheOnCompaction(boolean migrateCacheEntry) conf.key_cache_migrate_during_compaction = migrateCacheEntry; } + public static boolean shouldInvalidateKeycacheOnSSTableDeletion() + { + return conf.key_cache_invalidate_after_sstable_deletion; + } + + public static void setInvalidateKeycacheOnSSTableDeletion(boolean invalidateCacheEntry) + { + conf.key_cache_invalidate_after_sstable_deletion = invalidateCacheEntry; + } + /** This method can return negative number for disabled */ public static int getSSTablePreemptiveOpenIntervalInMiB() { diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java index 4abfc88b5a4f..d18863866930 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java +++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java @@ -247,13 +247,16 @@ public void delete(Descriptor desc) { try { - // remove key cache entries for the sstable being deleted - Iterator<KeyCacheKey> it = CacheService.instance.keyCache.keyIterator(); - while (it.hasNext()) + if (DatabaseDescriptor.shouldInvalidateKeycacheOnSSTableDeletion()) { - KeyCacheKey key = it.next(); - if (key.desc.equals(desc)) - it.remove(); + // remove key cache entries for the sstable being deleted + Iterator<KeyCacheKey> it = CacheService.instance.keyCache.keyIterator(); + while (it.hasNext()) + { + KeyCacheKey key = it.next(); + if (key.desc.equals(desc)) + it.remove(); + } } delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents()))); diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 7a86e89fd57f..ec771ab9bf3e 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -6646,6 +6646,16 @@ public void setMigrateKeycacheOnCompaction(boolean invalidateKeyCacheOnCompactio DatabaseDescriptor.setMigrateKeycacheOnCompaction(invalidateKeyCacheOnCompaction); } + public boolean getInvalidateKeycacheOnSSTableDeletion() + { + return DatabaseDescriptor.shouldInvalidateKeycacheOnSSTableDeletion(); + } + + public void setInvalidateKeycacheOnSSTableDeletion(boolean invalidate) + { + DatabaseDescriptor.setInvalidateKeycacheOnSSTableDeletion(invalidate); + } + public int getTombstoneWarnThreshold() { return DatabaseDescriptor.getTombstoneWarnThreshold(); diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java index 815209683191..bd0f80aba6c1 100644 --- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java +++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java @@ -833,6 +833,8 @@ default int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, public boolean getMigrateKeycacheOnCompaction(); public void setMigrateKeycacheOnCompaction(boolean invalidateKeyCacheOnCompaction); + public boolean getInvalidateKeycacheOnSSTableDeletion(); + public void setInvalidateKeycacheOnSSTableDeletion(boolean invalidate); public int getConcurrentViewBuilders(); public void setConcurrentViewBuilders(int value); diff --git a/test/unit/org/apache/cassandra/io/sstable/keycache/KeyCacheTest.java b/test/unit/org/apache/cassandra/io/sstable/keycache/KeyCacheTest.java index 15c37ea60ab5..abe83b59d573 100644 --- a/test/unit/org/apache/cassandra/io/sstable/keycache/KeyCacheTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/keycache/KeyCacheTest.java @@ -350,14 +350,14 @@ private void testKeyCache(String cf) throws ExecutionException, InterruptedExcep LifecycleTransaction.waitForDeletions(); - // after releasing the reference this should drop to 2 - assertKeyCacheSize(2, KEYSPACE1, cf); + // after releasing the reference this should drop to 2, unless we don't invalidate keycache on sstable deletion + assertKeyCacheSize(DatabaseDescriptor.shouldInvalidateKeycacheOnSSTableDeletion() ? 2 : 4, KEYSPACE1, cf); // re-read same keys to verify that key cache didn't grow further Util.getAll(Util.cmd(cfs, "key1").build()); Util.getAll(Util.cmd(cfs, "key2").build()); - assertKeyCacheSize( 2, KEYSPACE1, cf); + assertKeyCacheSize( DatabaseDescriptor.shouldInvalidateKeycacheOnSSTableDeletion() ? 2 : 4, KEYSPACE1, cf); } @Test From 54e46880690bd5effb31116986292c1bdc9e891e Mon Sep 17 00:00:00 2001 From: Branimir Lambov <branimir.lambov@datastax.com> Date: Thu, 14 Nov 2024 18:10:01 +0200 Subject: [PATCH 052/225] Parallelized UCS compactions patch by Branimir Lambov, reviewed by Sylvain Lebresne for CASSANDRA-18802 --- CHANGES.txt | 1 + NEWS.txt | 10 + .../config/CassandraRelevantProperties.java | 1 + .../cassandra/db/ColumnFamilyStore.java | 14 +- .../cassandra/db/ColumnFamilyStoreMBean.java | 9 + .../AbstractCompactionStrategy.java | 21 +- .../db/compaction/AbstractCompactionTask.java | 51 ++-- .../db/compaction/AbstractStrategyHolder.java | 6 +- .../db/compaction/CompactionController.java | 20 +- .../db/compaction/CompactionManager.java | 171 ++++++++++-- .../compaction/CompactionStrategyHolder.java | 4 +- .../compaction/CompactionStrategyManager.java | 29 ++- .../db/compaction/CompactionTask.java | 135 +++++++--- .../compaction/CompositeCompactionTask.java | 124 +++++++++ .../compaction/LeveledCompactionStrategy.java | 10 +- .../db/compaction/LeveledCompactionTask.java | 9 +- .../db/compaction/PendingRepairHolder.java | 12 +- .../db/compaction/PendingRepairManager.java | 33 +-- .../db/compaction/SSTableSplitter.java | 8 +- .../cassandra/db/compaction/ShardManager.java | 44 +++- .../db/compaction/ShardManagerDiskAware.java | 4 +- .../db/compaction/ShardManagerTrivial.java | 13 +- .../db/compaction/SingleSSTableLCSTask.java | 18 +- .../SizeTieredCompactionStrategy.java | 22 +- .../TimeWindowCompactionStrategy.java | 14 +- .../compaction/UnifiedCompactionStrategy.java | 159 +++++++++--- .../compaction/UnifiedCompactionStrategy.md | 43 ++- .../db/compaction/unified/Controller.java | 46 +++- .../unified/ShardedCompactionWriter.java | 7 +- .../unified/UnifiedCompactionTask.java | 67 ++++- .../writers/CompactionAwareWriter.java | 18 +- .../writers/DefaultCompactionWriter.java | 6 +- .../writers/MajorLeveledCompactionWriter.java | 6 +- .../writers/MaxSSTableSizeWriter.java | 6 +- .../SplittingSizeTieredCompactionWriter.java | 6 +- .../CompositeLifecycleTransaction.java | 161 ++++++++++++ .../db/lifecycle/ILifecycleTransaction.java | 33 ++- .../db/lifecycle/LifecycleTransaction.java | 15 +- .../db/lifecycle/LogTransaction.java | 7 +- .../PartialLifecycleTransaction.java | 225 ++++++++++++++++ .../cassandra/db/lifecycle/Tracker.java | 13 +- .../WrappedLifecycleTransaction.java | 11 + .../cassandra/io/sstable/SSTableRewriter.java | 7 +- .../io/sstable/format/SSTableReader.java | 3 +- .../cassandra/service/StorageService.java | 8 + .../service/StorageServiceMBean.java | 5 + .../org/apache/cassandra/tools/NodeProbe.java | 5 + .../cassandra/tools/nodetool/Compact.java | 11 +- .../apache/cassandra/utils/Throwables.java | 7 + .../org/apache/cassandra/utils/TimeUUID.java | 19 ++ .../distributed/test/PaxosRepair2Test.java | 2 +- .../distributed/test/PaxosRepairTest.java | 2 +- .../distributed/test/UpgradeSSTablesTest.java | 4 +- .../db/compaction/LongCompactionsTest.java | 2 +- .../LongLeveledCompactionStrategyTest.java | 17 +- .../test/microbench/CachingBenchTest.java | 2 +- .../microbench/ZeroCopyStreamingBench.java | 2 +- .../cassandra/cql3/GcCompactionTest.java | 2 +- .../miscellaneous/CrcCheckChanceTest.java | 2 +- .../org/apache/cassandra/db/KeyspaceTest.java | 2 +- .../cassandra/db/RangeTombstoneTest.java | 6 +- .../AbstractCompactionStrategyTest.java | 9 +- .../db/compaction/CancelCompactionsTest.java | 3 +- .../compaction/CompactionControllerTest.java | 6 +- ...ctionStrategyManagerPendingRepairTest.java | 16 +- .../db/compaction/CompactionTaskTest.java | 8 +- .../db/compaction/CompactionsCQLTest.java | 53 ++-- .../db/compaction/CompactionsPurgeTest.java | 12 +- .../CorruptedSSTablesCompactionsTest.java | 118 ++++++--- .../LeveledCompactionStrategyTest.java | 7 +- .../db/compaction/PartialCompactionsTest.java | 2 +- .../compaction/PendingRepairManagerTest.java | 11 +- .../db/compaction/ShardManagerTest.java | 154 ++++++++++- .../compaction/SingleSSTableLCSTaskTest.java | 5 +- .../db/compaction/TTLExpiryTest.java | 2 +- .../TimeWindowCompactionStrategyTest.java | 15 +- .../UnifiedCompactionStrategyTest.java | 245 ++++++++++++++++-- .../BackgroundCompactionTrackingTest.java | 194 ++++++++++++++ .../unified/ParallelizedTasksTest.java | 128 +++++++++ .../unified/ShardedCompactionWriterTest.java | 133 +--------- .../compaction/unified/ShardingTestBase.java | 205 +++++++++++++++ ...assandraEntireSSTableStreamWriterTest.java | 2 +- .../streaming/CassandraOutgoingFileTest.java | 2 +- .../streaming/CassandraStreamHeaderTest.java | 2 +- ...StreamConcurrentComponentMutationTest.java | 2 +- .../io/sstable/LegacySSTableTest.java | 5 +- .../io/sstable/SSTableReaderTest.java | 16 +- .../io/sstable/SSTableWriterTestBase.java | 2 +- .../cassandra/io/sstable/VerifyTest.java | 4 +- ...SSTableStreamingCorrectFilesCountTest.java | 2 +- .../cassandra/stress/CompactionStress.java | 2 +- 91 files changed, 2507 insertions(+), 578 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/compaction/CompositeCompactionTask.java create mode 100644 src/java/org/apache/cassandra/db/lifecycle/CompositeLifecycleTransaction.java create mode 100644 src/java/org/apache/cassandra/db/lifecycle/PartialLifecycleTransaction.java create mode 100644 test/unit/org/apache/cassandra/db/compaction/unified/BackgroundCompactionTrackingTest.java create mode 100644 test/unit/org/apache/cassandra/db/compaction/unified/ParallelizedTasksTest.java create mode 100644 test/unit/org/apache/cassandra/db/compaction/unified/ShardingTestBase.java diff --git a/CHANGES.txt b/CHANGES.txt index c4fde4445ea0..ffe042e3f35a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Parallelized UCS compactions (CASSANDRA-18802) * Avoid prepared statement invalidation race when committing schema changes (CASSANDRA-20116) * Restore optimization in MultiCBuilder around building one clustering (CASSANDRA-20129) * Consolidate all snapshot management to SnapshotManager and introduce SnapshotManagerMBean (CASSANDRA-18111) diff --git a/NEWS.txt b/NEWS.txt index 740be2eb9647..ae1ba39d303c 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -92,6 +92,16 @@ New features - There is new MBean of name org.apache.cassandra.service.snapshot:type=SnapshotManager which exposes user-facing snapshot operations. Snapshot-related methods on StorageServiceMBean are still present and functional but marked as deprecated. + - The unified compaction strategy will now parallelize individual compactions by splitting operations into separate + tasks per output shard. This significantly shortens compaction durations but cannot take advantage of preemptive + SSTable opening and thus can be less efficient if the strategy is configured to create large sstables. + This functionality is controlled by the parallelize_output_shards compaction option (the default can also be set + using -Dunified_compaction.parallelize_output_shards=false/true). + - Compaction parallelism for the unified compaction strategy also applies to major compactions. Because the number + of shards for a major compaction is usually quite high, this can dramatically improve the duration of major + compactions. To avoid the possibility of starving background compactions from resources, the number of threads + used for major compactions is limited to half the available compaction threads by default, and can be controlled + by a new --jobs / -j option of nodetool compact. Upgrading diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java index 9862f227822d..e409e975f638 100644 --- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java +++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java @@ -610,6 +610,7 @@ public enum CassandraRelevantProperties UCS_BASE_SHARD_COUNT("unified_compaction.base_shard_count", "4"), UCS_MIN_SSTABLE_SIZE("unified_compaction.min_sstable_size", "100MiB"), UCS_OVERLAP_INCLUSION_METHOD("unified_compaction.overlap_inclusion_method"), + UCS_PARALLELIZE_OUTPUT_SHARDS("unified_compaction.parallelize_output_shards", "true"), UCS_SCALING_PARAMETER("unified_compaction.scaling_parameters", "T4"), UCS_SSTABLE_GROWTH("unified_compaction.sstable_growth", "0.333"), UCS_SURVIVAL_FACTOR("unified_compaction.survival_factor", "1"), diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java index cbd3d97c6e10..91a3a40152af 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -2208,12 +2208,22 @@ public void putCachedCounter(ByteBuffer partitionKey, Clustering<?> clustering, public void forceMajorCompaction() { - forceMajorCompaction(false); + forceMajorCompaction(false, -1); } public void forceMajorCompaction(boolean splitOutput) { - CompactionManager.instance.performMaximal(this, splitOutput); + forceMajorCompaction(splitOutput, -1); + } + + public void forceMajorCompaction(int permittedParallelism) + { + forceMajorCompaction(false, permittedParallelism); + } + + public void forceMajorCompaction(boolean splitOutput, int permittedParallelism) + { + CompactionManager.instance.performMaximal(this, splitOutput, permittedParallelism); } @Override diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java b/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java index 7d7b9e58eebc..33cd6ebbc25b 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java @@ -52,6 +52,15 @@ public interface ColumnFamilyStoreMBean */ public void forceMajorCompaction(boolean splitOutput) throws ExecutionException, InterruptedException; + /** + * force a major compaction of this column family + * + * @param permittedParallelism The maximum number of compaction threads that can be used by the operation. + * If 0, the operation can use all available threads. + * If <0, the default parallelism will be used. + */ + public void forceMajorCompaction(int permittedParallelism) throws ExecutionException, InterruptedException; + /** * Forces a major compaction of specified token ranges in this column family. * <p> diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java index fd210d6d028f..9d189b9975ce 100644 --- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java +++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java @@ -92,7 +92,7 @@ public abstract class AbstractCompactionStrategy private final Directories directories; /** - * pause/resume/getNextBackgroundTask must synchronize. This guarantees that after pause completes, + * pause/resume/getNextBackgroundTasks must synchronize. This guarantees that after pause completes, * no new tasks will be generated; or put another way, pause can't run until in-progress tasks are * done being created. * @@ -176,21 +176,22 @@ public void shutdown() /** * @param gcBefore throw away tombstones older than this * - * @return the next background/minor compaction task to run; null if nothing to do. + * @return the next background/minor compaction tasks to run; an empty collection if nothing to do. * * Is responsible for marking its sstables as compaction-pending. */ - public abstract AbstractCompactionTask getNextBackgroundTask(final long gcBefore); + public abstract Collection<AbstractCompactionTask> getNextBackgroundTasks(final long gcBefore); /** - * @param gcBefore throw away tombstones older than this - * - * @return a compaction task that should be run to compact this columnfamilystore - * as much as possible. Null if nothing to do. - * + * @param gcBefore throw away tombstones older than this + * @return A list of compaction tasks that should be run to compact this columnfamilystore + * as much as possible. Empty if nothing to do. + * Order matters if a parallelism limit is applied, as the tasks are run in way that parallelizes ones that + * are close together in the list. + * <p> * Is responsible for marking its sstables as compaction-pending. */ - public abstract Collection<AbstractCompactionTask> getMaximalTask(final long gcBefore, boolean splitOutput); + public abstract List<AbstractCompactionTask> getMaximalTasks(final long gcBefore, boolean splitOutput); /** * @param sstables SSTables to compact. Must be marked as compacting. @@ -418,7 +419,7 @@ protected boolean worthDroppingTombstones(SSTableReader sstable, long gcBefore) // there is no overlap, tombstones are safely droppable return true; } - else if (CompactionController.getFullyExpiredSSTables(cfs, Collections.singleton(sstable), overlaps, gcBefore).size() > 0) + else if (CompactionController.getFullyExpiredSSTables(cfs, Collections.singleton(sstable), s -> overlaps, gcBefore).size() > 0) { return true; } diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java index 40c4cb49e123..d0b3dc498f06 100644 --- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java +++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java @@ -23,18 +23,16 @@ import com.google.common.base.Preconditions; import org.apache.cassandra.db.ColumnFamilyStore; -import org.apache.cassandra.db.Directories; -import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter; import org.apache.cassandra.io.FSDiskFullWriteError; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.utils.TimeUUID; import org.apache.cassandra.utils.WrappedRunnable; -import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.lifecycle.ILifecycleTransaction; public abstract class AbstractCompactionTask extends WrappedRunnable { protected final ColumnFamilyStore cfs; - protected LifecycleTransaction transaction; + protected ILifecycleTransaction transaction; protected boolean isUserDefined; protected OperationType compactionType; @@ -42,18 +40,30 @@ public abstract class AbstractCompactionTask extends WrappedRunnable * @param cfs * @param transaction the modifying managing the status of the sstables we're replacing */ - public AbstractCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction transaction) + public AbstractCompactionTask(ColumnFamilyStore cfs, ILifecycleTransaction transaction) { this.cfs = cfs; this.transaction = transaction; this.isUserDefined = false; this.compactionType = OperationType.COMPACTION; - // enforce contract that caller should mark sstables compacting - Set<SSTableReader> compacting = transaction.tracker.getCompacting(); - for (SSTableReader sstable : transaction.originals()) - assert compacting.contains(sstable) : sstable.getFilename() + " is not correctly marked compacting"; - validateSSTables(transaction.originals()); + try + { + if (!transaction.isOffline()) + { + // enforce contract that caller should mark sstables compacting + Set<SSTableReader> compacting = cfs.getTracker().getCompacting(); + for (SSTableReader sstable : transaction.originals()) + assert compacting.contains(sstable) : sstable.getFilename() + " is not correctly marked compacting"; + } + + validateSSTables(transaction.originals()); + } + catch (Throwable err) + { + rejected(); + throw err; + } } /** @@ -61,7 +71,7 @@ public AbstractCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction transa */ private void validateSSTables(Set<SSTableReader> sstables) { - // do not allow to be compacted together + // do not allow sstables in different repair states to be compacted together if (!sstables.isEmpty()) { Iterator<SSTableReader> iter = sstables.iterator(); @@ -93,11 +103,11 @@ private void validateSSTables(Set<SSTableReader> sstables) /** * executes the task and unmarks sstables compacting */ - public int execute(ActiveCompactionsTracker activeCompactions) + public void execute(ActiveCompactionsTracker activeCompactions) { try { - return executeInternal(activeCompactions); + executeInternal(activeCompactions); } catch(FSDiskFullWriteError e) { @@ -107,12 +117,21 @@ public int execute(ActiveCompactionsTracker activeCompactions) } finally { - transaction.close(); + cleanup(); } } - public abstract CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Directories directories, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables); - protected abstract int executeInternal(ActiveCompactionsTracker activeCompactions); + protected void cleanup() + { + transaction.close(); + } + + public void rejected() + { + cleanup(); + } + + protected abstract void executeInternal(ActiveCompactionsTracker activeCompactions); public AbstractCompactionTask setUserDefined(boolean isUserDefined) { diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractStrategyHolder.java b/src/java/org/apache/cassandra/db/compaction/AbstractStrategyHolder.java index a1471c77468c..2db7e1db60d8 100644 --- a/src/java/org/apache/cassandra/db/compaction/AbstractStrategyHolder.java +++ b/src/java/org/apache/cassandra/db/compaction/AbstractStrategyHolder.java @@ -53,15 +53,15 @@ public abstract class AbstractStrategyHolder public static class TaskSupplier implements Comparable<TaskSupplier> { private final int numRemaining; - private final Supplier<AbstractCompactionTask> supplier; + private final Supplier<Collection<AbstractCompactionTask>> supplier; - TaskSupplier(int numRemaining, Supplier<AbstractCompactionTask> supplier) + TaskSupplier(int numRemaining, Supplier<Collection<AbstractCompactionTask>> supplier) { this.numRemaining = numRemaining; this.supplier = supplier; } - public AbstractCompactionTask getTask() + public Collection<AbstractCompactionTask> getTasks() { return supplier.get(); } diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionController.java b/src/java/org/apache/cassandra/db/compaction/CompactionController.java index b4096c81dddf..1753d7e66426 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionController.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionController.java @@ -26,6 +26,7 @@ import java.util.Map; import java.util.Set; import java.util.function.LongPredicate; +import java.util.function.UnaryOperator; import com.google.common.base.Predicates; import com.google.common.collect.Iterables; @@ -130,7 +131,12 @@ void refreshOverlaps() public Set<SSTableReader> getFullyExpiredSSTables() { - return getFullyExpiredSSTables(cfs, compacting, overlappingSSTables, gcBefore, ignoreOverlaps()); + return getFullyExpiredSSTables(cfs, + compacting, + x -> overlappingSSTables != null ? overlappingSSTables + : Collections.emptyList(), + gcBefore, + ignoreOverlaps()); } /** @@ -145,20 +151,20 @@ public Set<SSTableReader> getFullyExpiredSSTables() * * @param cfStore * @param compacting we take the drop-candidates from this set, it is usually the sstables included in the compaction - * @param overlapping the sstables that overlap the ones in compacting. + * @param overlappingSupplier called on the compacting sstables to compute the set of sstables that overlap with them if needed * @param gcBefore * @param ignoreOverlaps don't check if data shadows/overlaps any data in other sstables * @return */ public static Set<SSTableReader> getFullyExpiredSSTables(ColumnFamilyStore cfStore, Iterable<SSTableReader> compacting, - Iterable<SSTableReader> overlapping, + UnaryOperator<Iterable<SSTableReader>> overlappingSupplier, long gcBefore, boolean ignoreOverlaps) { logger.trace("Checking droppable sstables in {}", cfStore); - if (NEVER_PURGE_TOMBSTONES_PROPERTY_VALUE || compacting == null || cfStore.getNeverPurgeTombstones() || overlapping == null) + if (NEVER_PURGE_TOMBSTONES_PROPERTY_VALUE || compacting == null || cfStore.getNeverPurgeTombstones()) return Collections.emptySet(); if (cfStore.getCompactionStrategyManager().onlyPurgeRepairedTombstones() && !Iterables.all(compacting, SSTableReader::isRepaired)) @@ -183,7 +189,7 @@ public static Set<SSTableReader> getFullyExpiredSSTables(ColumnFamilyStore cfSto List<SSTableReader> candidates = new ArrayList<>(); long minTimestamp = Long.MAX_VALUE; - for (SSTableReader sstable : overlapping) + for (SSTableReader sstable : overlappingSupplier.apply(compacting)) { // Overlapping might include fully expired sstables. What we care about here is // the min timestamp of the overlapping sstables that actually contain live data. @@ -230,10 +236,10 @@ public static Set<SSTableReader> getFullyExpiredSSTables(ColumnFamilyStore cfSto public static Set<SSTableReader> getFullyExpiredSSTables(ColumnFamilyStore cfStore, Iterable<SSTableReader> compacting, - Iterable<SSTableReader> overlapping, + UnaryOperator<Iterable<SSTableReader>> overlappingSupplier, long gcBefore) { - return getFullyExpiredSSTables(cfStore, compacting, overlapping, gcBefore, false); + return getFullyExpiredSSTables(cfStore, compacting, overlappingSupplier, gcBefore, false); } /** diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java index 67b847e0efda..801b1ec06be7 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java @@ -34,6 +34,7 @@ import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.BooleanSupplier; import java.util.function.Predicate; import java.util.function.Supplier; @@ -57,6 +58,7 @@ import com.google.common.util.concurrent.Uninterruptibles; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import com.codahale.metrics.Meter; import net.openhft.chronicle.core.util.ThrowingSupplier; import org.apache.cassandra.cache.AutoSavingCache; @@ -108,6 +110,8 @@ import org.apache.cassandra.service.ActiveRepairService; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.streaming.PreviewKind; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.ownership.DataPlacement; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.MBeanWrapper; @@ -115,10 +119,10 @@ import org.apache.cassandra.utils.Throwables; import org.apache.cassandra.utils.TimeUUID; import org.apache.cassandra.utils.WrappedRunnable; -import org.apache.cassandra.tcm.ClusterMetadata; -import org.apache.cassandra.tcm.ownership.DataPlacement; +import org.apache.cassandra.utils.concurrent.AsyncPromise; import org.apache.cassandra.utils.concurrent.Future; import org.apache.cassandra.utils.concurrent.ImmediateFuture; +import org.apache.cassandra.utils.concurrent.Promise; import org.apache.cassandra.utils.concurrent.Refs; import static java.util.Collections.singleton; @@ -268,9 +272,9 @@ public List<Future<?>> submitBackground(final ColumnFamilyStore cfs) cfs.getCompactionStrategyManager().getName()); List<Future<?>> futures = new ArrayList<>(1); - Future<?> fut = executor.submitIfRunning(new BackgroundCompactionCandidate(cfs), "background task"); - if (!fut.isCancelled()) - futures.add(fut); + Promise<Void> promise = new AsyncPromise<>(); + if (!executor.submitIfRunning(new BackgroundCompactionCandidate(cfs, promise), "background task").isCancelled()) + futures.add(promise); else compactingCF.remove(cfs); return futures; @@ -356,16 +360,22 @@ public void finishCompactionsAndShutdown(long timeout, TimeUnit unit) throws Int class BackgroundCompactionCandidate implements Runnable { private final ColumnFamilyStore cfs; + private final Promise<Void> toSignalWhenDone; + private final AtomicReference<Throwable> asyncErrors = new AtomicReference<>(null); - BackgroundCompactionCandidate(ColumnFamilyStore cfs) + BackgroundCompactionCandidate(ColumnFamilyStore cfs, Promise<Void> toSignalWhenDone) { compactingCF.add(cfs); this.cfs = cfs; + this.toSignalWhenDone = toSignalWhenDone; } public void run() { boolean ranCompaction = false; + boolean async = false; + Throwable error = null; + Collection<AbstractCompactionTask> tasks = null; try { logger.trace("Checking {}.{}", cfs.getKeyspaceName(), cfs.name); @@ -376,24 +386,131 @@ public void run() } CompactionStrategyManager strategy = cfs.getCompactionStrategyManager(); - AbstractCompactionTask task = strategy.getNextBackgroundTask(getDefaultGcBefore(cfs, FBUtilities.nowInSeconds())); - if (task == null) + tasks = strategy.getNextBackgroundTasks(getDefaultGcBefore(cfs, FBUtilities.nowInSeconds())); + if (tasks == null || tasks.isEmpty()) { if (DatabaseDescriptor.automaticSSTableUpgrade()) ranCompaction = maybeRunUpgradeTask(strategy); } - else + else if (tasks.size() == 1) { - task.execute(active); + // If just one task, run it directly on this thread + for (AbstractCompactionTask task : tasks) + task.execute(active); ranCompaction = true; } + else + { + // more than 1 task: we need to do this outside the catch and complete block + async = true; + } + } + catch (Throwable t) + { + error = t; + } + + if (!async) + { + complete(ranCompaction, error); + Throwables.maybeFail(error); + } + else // async + processTasksAsync(tasks); + } + + private void processTasksAsync(Collection<AbstractCompactionTask> tasks) + { + assert tasks != null; + + // We should only signal overall completion when all tasks are done + AtomicInteger toComplete = new AtomicInteger(tasks.size()); + + AbstractCompactionTask lastTask = null; + // Submit all but the last task for execution, + for (AbstractCompactionTask task : tasks) + { + if (lastTask != null) + { + AbstractCompactionTask toRun = lastTask; + try + { + executor.submit(() -> runTask(toRun, toComplete)); + } + catch (RejectedExecutionException e) + { + logger.debug("Failed to submit background compaction task: {}", e.getMessage()); + rejectTask(toRun, toComplete); + } + } + lastTask = task; + } + + // and run the last task directly in this thread. + assert lastTask != null; + runTask(lastTask, toComplete); + } + + private void runTask(AbstractCompactionTask task, AtomicInteger toComplete) + { + try + { + task.execute(active); + } + catch (Throwable t) + { + addAsyncError(t); + } + finally + { + if (toComplete.decrementAndGet() == 0) + complete(true, asyncErrors.get()); + } + } + + private void rejectTask(AbstractCompactionTask task, AtomicInteger toComplete) + { + try + { + task.rejected(); + throw new RuntimeException("Failed to submit background compaction task"); + } + catch (Throwable t) // make sure we catch exceptions thrown by task.rejected() as well + { + addAsyncError(t); } finally { - compactingCF.remove(cfs); + if (toComplete.decrementAndGet() == 0) + complete(false, asyncErrors.get()); } - if (ranCompaction) // only submit background if we actually ran a compaction - otherwise we end up in an infinite loop submitting noop background tasks - submitBackground(cfs); + } + + private void complete(boolean submitNew, Throwable error) + { + compactingCF.remove(cfs); + if (error == null) + { + toSignalWhenDone.setSuccess(null); + if (submitNew) + submitBackground(cfs); + } + else + toSignalWhenDone.setFailure(error); + } + + private void addAsyncError(Throwable t) + { + asyncErrors.accumulateAndGet(t, (a, b) -> + { + if (a == null) + return b; + else + { + a.addSuppressed(b); + return a; + } + }); } boolean maybeRunUpgradeTask(CompactionStrategyManager strategy) @@ -423,7 +540,7 @@ boolean maybeRunUpgradeTask(CompactionStrategyManager strategy) @VisibleForTesting public BackgroundCompactionCandidate getBackgroundCompactionCandidate(ColumnFamilyStore cfs) { - return new BackgroundCompactionCandidate(cfs); + return new BackgroundCompactionCandidate(cfs, new AsyncPromise<>()); } /** @@ -721,7 +838,7 @@ public Iterable<SSTableReader> filterSSTables(LifecycleTransaction transaction) } catch (Throwable t) { - logger.warn(String.format("Unable to cancel %s from transaction %s", sstable, transaction.opId()), t); + logger.warn(String.format("Unable to cancel %s from transaction %s", sstable, transaction.opIdString()), t); } } else @@ -1004,22 +1121,32 @@ else if (r.intersects(sstableBounds)) return fullyContainedSSTables; } - public void performMaximal(final ColumnFamilyStore cfStore, boolean splitOutput) + public void performMaximal(final ColumnFamilyStore cfStore) + { + performMaximal(cfStore, false, -1); + } + + public void performMaximal(final ColumnFamilyStore cfStore, boolean splitOutput, int permittedParallelism) { - FBUtilities.waitOnFutures(submitMaximal(cfStore, getDefaultGcBefore(cfStore, FBUtilities.nowInSeconds()), splitOutput)); + FBUtilities.waitOnFutures(submitMaximal(cfStore, getDefaultGcBefore(cfStore, FBUtilities.nowInSeconds()), splitOutput, permittedParallelism)); } - public List<Future<?>> submitMaximal(final ColumnFamilyStore cfStore, final long gcBefore, boolean splitOutput) + public List<Future<?>> submitMaximal(final ColumnFamilyStore cfStore, final long gcBefore, boolean splitOutput, int permittedParallelism) { - return submitMaximal(cfStore, gcBefore, splitOutput, OperationType.MAJOR_COMPACTION); + return submitMaximal(cfStore, gcBefore, splitOutput, permittedParallelism, OperationType.MAJOR_COMPACTION); } - public List<Future<?>> submitMaximal(final ColumnFamilyStore cfStore, final long gcBefore, boolean splitOutput, OperationType operationType) + public List<Future<?>> submitMaximal(final ColumnFamilyStore cfStore, final long gcBefore, boolean splitOutput, int permittedParallelism, OperationType operationType) { + if (permittedParallelism < 0) + permittedParallelism = getCoreCompactorThreads() / 2; + else if (permittedParallelism == 0) + permittedParallelism = Integer.MAX_VALUE; + // here we compute the task off the compaction executor, so having that present doesn't // confuse runWithCompactionsDisabled -- i.e., we don't want to deadlock ourselves, waiting // for ourselves to finish/acknowledge cancellation before continuing. - CompactionTasks tasks = cfStore.getCompactionStrategyManager().getMaximalTasks(gcBefore, splitOutput, operationType); + CompactionTasks tasks = cfStore.getCompactionStrategyManager().getMaximalTasks(gcBefore, splitOutput, permittedParallelism, operationType); if (tasks.isEmpty()) return Collections.emptyList(); @@ -1043,6 +1170,8 @@ protected void runMayThrow() Future<?> fut = executor.submitIfRunning(runnable, "maximal task"); if (!fut.isCancelled()) futures.add(fut); + else + task.rejected(); } if (nonEmptyTasks > 1) logger.info("Major compaction will not result in a single sstable - repaired and unrepaired data is kept separate and compaction runs per data_file_directory."); diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyHolder.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyHolder.java index becd3b954af1..9bc4d6d7cbaf 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyHolder.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyHolder.java @@ -106,7 +106,7 @@ public Collection<TaskSupplier> getBackgroundTaskSuppliers(long gcBefore) { List<TaskSupplier> suppliers = new ArrayList<>(strategies.size()); for (AbstractCompactionStrategy strategy : strategies) - suppliers.add(new TaskSupplier(strategy.getEstimatedRemainingTasks(), () -> strategy.getNextBackgroundTask(gcBefore))); + suppliers.add(new TaskSupplier(strategy.getEstimatedRemainingTasks(), () -> strategy.getNextBackgroundTasks(gcBefore))); return suppliers; } @@ -117,7 +117,7 @@ public Collection<AbstractCompactionTask> getMaximalTasks(long gcBefore, boolean List<AbstractCompactionTask> tasks = new ArrayList<>(strategies.size()); for (AbstractCompactionStrategy strategy : strategies) { - Collection<AbstractCompactionTask> task = strategy.getMaximalTask(gcBefore, splitOutput); + Collection<AbstractCompactionTask> task = strategy.getMaximalTasks(gcBefore, splitOutput); if (task != null) tasks.addAll(task); } diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java index d022be45c479..dfb908425a96 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java @@ -195,26 +195,26 @@ public int getIndexForSSTableDirectory(Descriptor descriptor) * * Returns a task for the compaction strategy that needs it the most (most estimated remaining tasks) */ - public AbstractCompactionTask getNextBackgroundTask(long gcBefore) + public Collection<AbstractCompactionTask> getNextBackgroundTasks(long gcBefore) { maybeReloadDiskBoundaries(); readLock.lock(); try { if (!isEnabled()) - return null; + return Collections.emptyList(); int numPartitions = getNumTokenPartitions(); // first try to promote/demote sstables from completed repairs - AbstractCompactionTask repairFinishedTask; - repairFinishedTask = pendingRepairs.getNextRepairFinishedTask(); - if (repairFinishedTask != null) - return repairFinishedTask; + Collection<AbstractCompactionTask> repairFinishedTasks; + repairFinishedTasks = pendingRepairs.getNextRepairFinishedTasks(); + if (repairFinishedTasks != null && !repairFinishedTasks.isEmpty()) + return repairFinishedTasks; - repairFinishedTask = transientRepairs.getNextRepairFinishedTask(); - if (repairFinishedTask != null) - return repairFinishedTask; + repairFinishedTasks = transientRepairs.getNextRepairFinishedTasks(); + if (repairFinishedTasks != null && !repairFinishedTasks.isEmpty()) + return repairFinishedTasks; // sort compaction task suppliers by remaining tasks descending List<TaskSupplier> suppliers = new ArrayList<>(numPartitions * holders.size()); @@ -226,12 +226,12 @@ public AbstractCompactionTask getNextBackgroundTask(long gcBefore) // return the first non-null task for (TaskSupplier supplier : suppliers) { - AbstractCompactionTask task = supplier.getTask(); - if (task != null) - return task; + Collection<AbstractCompactionTask> tasks = supplier.getTasks(); + if (tasks != null && !tasks.isEmpty()) + return tasks; } - return null; + return Collections.emptyList(); } finally { @@ -1079,7 +1079,7 @@ private void validateForCompaction(Iterable<SSTableReader> input) } } - public CompactionTasks getMaximalTasks(final long gcBefore, final boolean splitOutput, OperationType operationType) + public CompactionTasks getMaximalTasks(final long gcBefore, final boolean splitOutput, int permittedParallelism, OperationType operationType) { maybeReloadDiskBoundaries(); // runWithCompactionsDisabled cancels active compactions and disables them, then we are able @@ -1097,6 +1097,7 @@ public CompactionTasks getMaximalTasks(final long gcBefore, final boolean splitO tasks.add(task.setCompactionType(operationType)); } } + tasks = CompositeCompactionTask.applyParallelismLimit(tasks, permittedParallelism); } finally { diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java index 4f27736f266d..0cbd9fe3a816 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java @@ -26,10 +26,13 @@ import java.util.concurrent.TimeUnit; import com.google.common.base.Predicate; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; -import com.google.common.collect.Sets; import com.google.common.util.concurrent.RateLimiter; +import org.apache.cassandra.db.compaction.unified.UnifiedCompactionTask; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,7 +43,7 @@ import org.apache.cassandra.db.SystemKeyspace; import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter; import org.apache.cassandra.db.compaction.writers.DefaultCompactionWriter; -import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.lifecycle.ILifecycleTransaction; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.metadata.MetadataCollector; import org.apache.cassandra.io.util.File; @@ -64,12 +67,12 @@ public class CompactionTask extends AbstractCompactionTask protected static long totalBytesCompacted = 0; private ActiveCompactionsTracker activeCompactions; - public CompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, long gcBefore) + public CompactionTask(ColumnFamilyStore cfs, ILifecycleTransaction txn, long gcBefore) { this(cfs, txn, gcBefore, false); } - public CompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, long gcBefore, boolean keepOriginals) + public CompactionTask(ColumnFamilyStore cfs, ILifecycleTransaction txn, long gcBefore, boolean keepOriginals) { super(cfs, txn); this.gcBefore = gcBefore; @@ -81,44 +84,73 @@ public static synchronized long addToTotalBytesCompacted(long bytesCompacted) return totalBytesCompacted += bytesCompacted; } - protected int executeInternal(ActiveCompactionsTracker activeCompactions) + protected void executeInternal(ActiveCompactionsTracker activeCompactions) { this.activeCompactions = activeCompactions == null ? ActiveCompactionsTracker.NOOP : activeCompactions; run(); - return transaction.originals().size(); } public boolean reduceScopeForLimitedSpace(Set<SSTableReader> nonExpiredSSTables, long expectedSize) { - if (partialCompactionsAcceptable() && transaction.originals().size() > 1) + if (partialCompactionsAcceptable() && nonExpiredSSTables.size() > 1) { // Try again w/o the largest one. SSTableReader removedSSTable = cfs.getMaxSizeFile(nonExpiredSSTables); logger.warn("insufficient space to compact all requested files. {}MiB required, {} for compaction {} - removing largest SSTable: {}", (float) expectedSize / 1024 / 1024, - StringUtils.join(transaction.originals(), ", "), - transaction.opId(), + StringUtils.join(nonExpiredSSTables, ", "), + transaction.opIdString(), removedSSTable); // Note that we have removed files that are still marked as compacting. // This suboptimal but ok since the caller will unmark all the sstables at the end. transaction.cancel(removedSSTable); + nonExpiredSSTables.remove(removedSSTable); return true; } return false; } + /** + * @return The token range that the operation should compact. This is usually null, but if we have a parallelizable + * multi-task operation (see {@link UnifiedCompactionStrategy#createCompactionTasks}), it will specify a subrange. + */ + protected Range<Token> tokenRange() + { + return null; + } + + /** + * @return The set of input sstables for this compaction. This must be a subset of the transaction originals and + * must reflect any removal of sstables from the originals set for correct overlap tracking. + * See {@link UnifiedCompactionTask} for an example. + */ + protected Set<SSTableReader> inputSSTables() + { + return transaction.originals(); + } + + /** + * @return True if the task should try to limit the operation size to the available space by removing sstables from + * the compacting set. This cannot be done if this is part of a multi-task operation with a shared transaction. + */ + protected boolean shouldReduceScopeForSpace() + { + return true; + } + /** * For internal use and testing only. The rest of the system should go through the submit* methods, * which are properly serialized. * Caller is in charge of marking/unmarking the sstables as compacting. */ + @Override protected void runMayThrow() throws Exception { // The collection of sstables passed may be empty (but not null); even if // it is not empty, it may compact down to nothing if all rows are deleted. assert transaction != null; - if (transaction.originals().isEmpty()) + if (inputSSTables().isEmpty()) return; // Note that the current compaction strategy, is not necessarily the one this task was created under. @@ -131,22 +163,35 @@ protected void runMayThrow() throws Exception SnapshotManager.instance.takeSnapshot(options); } - try (CompactionController controller = getCompactionController(transaction.originals())) + try (CompactionController controller = getCompactionController(inputSSTables())) { + // Note: the controller set-up above relies on using the transaction-provided sstable list, from which + // fully-expired sstables should not be removed (so that the overlap tracker does not include them), but + // sstables excluded for scope reduction should be removed. + Set<SSTableReader> actuallyCompact = new HashSet<>(inputSSTables()); final Set<SSTableReader> fullyExpiredSSTables = controller.getFullyExpiredSSTables(); + if (!fullyExpiredSSTables.isEmpty()) + { + logger.debug("Compaction {} dropping expired sstables: {}", transaction.opIdString(), fullyExpiredSSTables); + actuallyCompact.removeAll(fullyExpiredSSTables); + } TimeUUID taskId = transaction.opId(); // select SSTables to compact based on available disk space. - if (!buildCompactionCandidatesForAvailableDiskSpace(fullyExpiredSSTables, taskId)) + final boolean hasExpirations = !fullyExpiredSSTables.isEmpty(); + if ((shouldReduceScopeForSpace() && !buildCompactionCandidatesForAvailableDiskSpace(actuallyCompact, hasExpirations, taskId)) + || hasExpirations) { // The set of sstables has changed (one or more were excluded due to limited available disk space). - // We need to recompute the overlaps between sstables. + // We need to recompute the overlaps between sstables. The iterators used in the compaction controller + // and tracker will reflect the changed set of sstables made by LifecycleTransaction.cancel(), + // so refreshing the overlaps will be based on the updated set of sstables. controller.refreshOverlaps(); } // sanity check: all sstables must belong to the same cfs - assert !Iterables.any(transaction.originals(), new Predicate<SSTableReader>() + assert !Iterables.any(actuallyCompact, new Predicate<SSTableReader>() { @Override public boolean apply(SSTableReader sstable) @@ -159,13 +204,15 @@ public boolean apply(SSTableReader sstable) // so in our single-threaded compaction world this is a valid way of determining if we're compacting // all the sstables (that existed when we started) StringBuilder ssTableLoggerMsg = new StringBuilder("["); - for (SSTableReader sstr : transaction.originals()) + for (SSTableReader sstr : actuallyCompact) { - ssTableLoggerMsg.append(String.format("%s:level=%d, ", sstr.getFilename(), sstr.getSSTableLevel())); + ssTableLoggerMsg.append(sstr.getSSTableLevel() != 0 ? String.format("%s:level=%d", sstr.getFilename(), sstr.getSSTableLevel()) + : sstr.getFilename()); + ssTableLoggerMsg.append(", "); } ssTableLoggerMsg.append("]"); - logger.info("Compacting ({}) {}", taskId, ssTableLoggerMsg); + logger.info("Compacting ({}) {}", transaction.opIdString(), ssTableLoggerMsg); RateLimiter limiter = CompactionManager.instance.getRateLimiter(); long start = nanoTime(); @@ -175,15 +222,17 @@ public boolean apply(SSTableReader sstable) long inputSizeBytes; long timeSpentWritingKeys; - Set<SSTableReader> actuallyCompact = Sets.difference(transaction.originals(), fullyExpiredSSTables); Collection<SSTableReader> newSStables; long[] mergedRowCounts; long totalSourceCQLRows; + Range<Token> tokenRange = tokenRange(); + List<Range<Token>> rangeList = tokenRange != null ? ImmutableList.of(tokenRange) : null; + long nowInSec = FBUtilities.nowInSeconds(); try (Refs<SSTableReader> refs = Refs.ref(actuallyCompact); - AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(actuallyCompact); + AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(actuallyCompact, rangeList); CompactionIterator ci = new CompactionIterator(compactionType, scanners.scanners, controller, nowInSec, taskId)) { long lastCheckObsoletion = start; @@ -257,37 +306,36 @@ public boolean apply(SSTableReader sstable) ImmutableMap.of(COMPACTION_TYPE_PROPERTY, compactionType.type)); logger.info(String.format("Compacted (%s) %d sstables to [%s] to level=%d. %s to %s (~%d%% of original) in %,dms. Read Throughput = %s, Write Throughput = %s, Row Throughput = ~%,d/s. %,d total partitions merged to %,d. Partition merge counts were {%s}. Time spent writing keys = %,dms", - taskId, - transaction.originals().size(), - newSSTableNames.toString(), - getLevel(), - FBUtilities.prettyPrintMemory(startsize), - FBUtilities.prettyPrintMemory(endsize), - (int) (ratio * 100), - dTime, - FBUtilities.prettyPrintMemoryPerSecond(startsize, durationInNano), - FBUtilities.prettyPrintMemoryPerSecond(endsize, durationInNano), - (int) totalSourceCQLRows / (TimeUnit.NANOSECONDS.toSeconds(durationInNano) + 1), - totalSourceRows, - totalKeysWritten, - mergeSummary, - timeSpentWritingKeys)); + transaction.opIdString(), + actuallyCompact.size(), + newSSTableNames.toString(), + getLevel(), + FBUtilities.prettyPrintMemory(startsize), + FBUtilities.prettyPrintMemory(endsize), + (int) (ratio * 100), + dTime, + FBUtilities.prettyPrintMemoryPerSecond(startsize, durationInNano), + FBUtilities.prettyPrintMemoryPerSecond(endsize, durationInNano), + (int) totalSourceCQLRows / (TimeUnit.NANOSECONDS.toSeconds(durationInNano) + 1), + totalSourceRows, + totalKeysWritten, + mergeSummary, + timeSpentWritingKeys)); if (logger.isTraceEnabled()) { logger.trace("CF Total Bytes Compacted: {}", FBUtilities.prettyPrintMemory(CompactionTask.addToTotalBytesCompacted(endsize))); logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten)); } - cfs.getCompactionStrategyManager().compactionLogger.compaction(startTime, transaction.originals(), currentTimeMillis(), newSStables); + cfs.getCompactionStrategyManager().compactionLogger.compaction(startTime, inputSSTables(), currentTimeMillis(), newSStables); // update the metrics cfs.metric.compactionBytesWritten.inc(endsize); } } - @Override public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Directories directories, - LifecycleTransaction transaction, + ILifecycleTransaction transaction, Set<SSTableReader> nonExpiredSSTables) { return new DefaultCompactionWriter(cfs, directories, transaction, nonExpiredSSTables, keepOriginals, getLevel()); @@ -366,7 +414,7 @@ public static boolean getIsTransient(Set<SSTableReader> sstables) * * @return true if there is enough disk space to execute the complete compaction, false if some sstables are excluded. */ - protected boolean buildCompactionCandidatesForAvailableDiskSpace(final Set<SSTableReader> fullyExpiredSSTables, TimeUUID taskId) + protected boolean buildCompactionCandidatesForAvailableDiskSpace(final Set<SSTableReader> nonExpiredSSTables, boolean containsExpired, TimeUUID taskId) { if(!cfs.isCompactionDiskSpaceCheckEnabled() && compactionType == OperationType.COMPACTION) { @@ -374,7 +422,6 @@ protected boolean buildCompactionCandidatesForAvailableDiskSpace(final Set<SSTab return true; } - final Set<SSTableReader> nonExpiredSSTables = Sets.difference(transaction.originals(), fullyExpiredSSTables); CompactionStrategyManager strategy = cfs.getCompactionStrategyManager(); int sstablesRemoved = 0; @@ -408,11 +455,13 @@ protected boolean buildCompactionCandidatesForAvailableDiskSpace(final Set<SSTab // we end up here if we can't take any more sstables out of the compaction. // usually means we've run out of disk space - // but we can still compact expired SSTables - if(partialCompactionsAcceptable() && fullyExpiredSSTables.size() > 0 ) + // but we can still remove expired SSTables + if (partialCompactionsAcceptable() && containsExpired) { - // sanity check to make sure we compact only fully expired SSTables. - assert transaction.originals().equals(fullyExpiredSSTables); + for (SSTableReader rdr : nonExpiredSSTables) + transaction.cancel(rdr); + nonExpiredSSTables.clear(); + assert transaction.originals().size() > 0; break; } diff --git a/src/java/org/apache/cassandra/db/compaction/CompositeCompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompositeCompactionTask.java new file mode 100644 index 000000000000..7ff39fc7499b --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/CompositeCompactionTask.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.compaction; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.utils.Throwables; + +/// A composition of several compaction tasks into one. This object executes the given tasks sequentially and +/// is used to limit the parallelism of some compaction tasks that split into a large number of parallelizable ones +/// but should not be allowed to take all compaction executor threads. +public class CompositeCompactionTask extends AbstractCompactionTask +{ + @VisibleForTesting + final ArrayList<AbstractCompactionTask> tasks; + + public CompositeCompactionTask(AbstractCompactionTask first) + { + super(first.cfs, first.cfs.getTracker().tryModify(Collections.emptyList(), OperationType.COMPACTION)); + tasks = new ArrayList<>(); + addTask(first); + } + + /// Add a task to the composition. + public CompositeCompactionTask addTask(AbstractCompactionTask task) + { + tasks.add(task); + return this; + } + + @Override + protected void executeInternal(ActiveCompactionsTracker tracker) + { + // Run all tasks in sequence, regardless if any of them fail. + Throwables.perform(tasks.stream().map(x -> () -> x.execute(tracker))); + } + + @Override + protected void runMayThrow() throws Exception { + throw new IllegalStateException("CompositeCompactionTask should be run through execute"); + } + + @Override + public void rejected() + { + Throwables.perform(tasks.stream().map(t -> t::rejected), () -> super.rejected()); + } + + @Override + public AbstractCompactionTask setUserDefined(boolean isUserDefined) + { + for (AbstractCompactionTask task : tasks) + task.setUserDefined(isUserDefined); + return super.setUserDefined(isUserDefined); + } + + @Override + public AbstractCompactionTask setCompactionType(OperationType compactionType) + { + for (AbstractCompactionTask task : tasks) + task.setCompactionType(compactionType); + return super.setCompactionType(compactionType); + } + + @Override + public String toString() + { + return "Composite " + tasks; + } + + /// Limit the parallelism of a list of compaction tasks by combining them into a smaller number of composite tasks. + /// This method assumes that the caller has preference for the tasks to be executed in order close to the order of + /// the input list. See [UnifiedCompactionStrategy#getMaximalTasks] for an example of how to use this method. + public static List<AbstractCompactionTask> applyParallelismLimit(List<AbstractCompactionTask> tasks, int parallelismLimit) + { + if (tasks.size() <= parallelismLimit || parallelismLimit <= 0) + return tasks; + + List<AbstractCompactionTask> result = new ArrayList<>(parallelismLimit); + int taskIndex = 0; + for (AbstractCompactionTask task : tasks) { + if (result.size() < parallelismLimit) + result.add(task); + else { + result.set(taskIndex, combineTasks(result.get(taskIndex), task)); + if (++taskIndex == parallelismLimit) + taskIndex = 0; + } + } + return result; + } + + /// Make a composite tasks that combines two tasks. If the former is already a composite task, the latter is added + /// to it. Otherwise, a new composite task is created. + public static CompositeCompactionTask combineTasks(AbstractCompactionTask task1, AbstractCompactionTask task2) + { + CompositeCompactionTask composite; + if (task1 instanceof CompositeCompactionTask) + composite = (CompositeCompactionTask) task1; + else + composite = new CompositeCompactionTask(task1); + return composite.addTask(task2); + } +} diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java index 58166630bc26..ff90219224c5 100644 --- a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java +++ b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java @@ -126,7 +126,7 @@ public void startup() * the only difference between background and maximal in LCS is that maximal is still allowed * (by explicit user request) even when compaction is disabled. */ - public AbstractCompactionTask getNextBackgroundTask(long gcBefore) + public Collection<AbstractCompactionTask> getNextBackgroundTasks(long gcBefore) { Collection<SSTableReader> previousCandidate = null; while (true) @@ -140,7 +140,7 @@ public AbstractCompactionTask getNextBackgroundTask(long gcBefore) if (sstable == null) { logger.trace("No compaction necessary for {}", this); - return null; + return Collections.emptyList(); } candidate = new LeveledManifest.CompactionCandidate(Collections.singleton(sstable), sstable.getSSTableLevel(), @@ -159,7 +159,7 @@ public AbstractCompactionTask getNextBackgroundTask(long gcBefore) logger.warn("Could not acquire references for compacting SSTables {} which is not a problem per se," + "unless it happens frequently, in which case it must be reported. Will retry later.", candidate.sstables); - return null; + return Collections.emptyList(); } LifecycleTransaction txn = cfs.getTracker().tryModify(candidate.sstables, OperationType.COMPACTION); @@ -172,13 +172,13 @@ public AbstractCompactionTask getNextBackgroundTask(long gcBefore) newTask = new SingleSSTableLCSTask(cfs, txn, candidate.level); newTask.setCompactionType(op); - return newTask; + return Collections.singletonList(newTask); } previousCandidate = candidate.sstables; } } - public synchronized Collection<AbstractCompactionTask> getMaximalTask(long gcBefore, boolean splitOutput) + public synchronized List<AbstractCompactionTask> getMaximalTasks(long gcBefore, boolean splitOutput) { Iterable<SSTableReader> sstables = manifest.getSSTables(); diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java index 8f5a70a84d16..d5867410f1aa 100644 --- a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java +++ b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java @@ -26,7 +26,7 @@ import org.apache.cassandra.db.compaction.writers.MajorLeveledCompactionWriter; import org.apache.cassandra.db.compaction.writers.MaxSSTableSizeWriter; import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.lifecycle.ILifecycleTransaction; public class LeveledCompactionTask extends CompactionTask { @@ -34,7 +34,7 @@ public class LeveledCompactionTask extends CompactionTask private final long maxSSTableBytes; private final boolean majorCompaction; - public LeveledCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int level, long gcBefore, long maxSSTableBytes, boolean majorCompaction) + public LeveledCompactionTask(ColumnFamilyStore cfs, ILifecycleTransaction txn, int level, long gcBefore, long maxSSTableBytes, boolean majorCompaction) { super(cfs, txn, gcBefore); this.level = level; @@ -45,7 +45,7 @@ public LeveledCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, in @Override public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Directories directories, - LifecycleTransaction txn, + ILifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables) { if (majorCompaction) @@ -67,7 +67,7 @@ protected int getLevel() @Override public boolean reduceScopeForLimitedSpace(Set<SSTableReader> nonExpiredSSTables, long expectedSize) { - if (transaction.originals().size() > 1 && level <= 1) + if (nonExpiredSSTables.size() > 1 && level <= 1) { // Try again w/o the largest one. logger.warn("insufficient space to do L0 -> L{} compaction. {}MiB required, {} for compaction {}", @@ -100,6 +100,7 @@ public boolean reduceScopeForLimitedSpace(Set<SSTableReader> nonExpiredSSTables, largestL0SSTable.onDiskLength(), transaction.opId()); transaction.cancel(largestL0SSTable); + nonExpiredSSTables.remove(largestL0SSTable); return true; } } diff --git a/src/java/org/apache/cassandra/db/compaction/PendingRepairHolder.java b/src/java/org/apache/cassandra/db/compaction/PendingRepairHolder.java index 0c5d53c1d8a3..1f2ff4a1a184 100644 --- a/src/java/org/apache/cassandra/db/compaction/PendingRepairHolder.java +++ b/src/java/org/apache/cassandra/db/compaction/PendingRepairHolder.java @@ -116,7 +116,7 @@ public Collection<TaskSupplier> getBackgroundTaskSuppliers(long gcBefore) { List<TaskSupplier> suppliers = new ArrayList<>(managers.size()); for (PendingRepairManager manager : managers) - suppliers.add(new TaskSupplier(manager.getMaxEstimatedRemainingTasks(), () -> manager.getNextBackgroundTask(gcBefore))); + suppliers.add(new TaskSupplier(manager.getMaxEstimatedRemainingTasks(), () -> manager.getNextBackgroundTasks(gcBefore))); return suppliers; } @@ -156,7 +156,7 @@ public void addSSTable(SSTableReader sstable) managers.get(router.getIndexForSSTable(sstable)).addSSTable(sstable); } - AbstractCompactionTask getNextRepairFinishedTask() + Collection<AbstractCompactionTask> getNextRepairFinishedTasks() { List<TaskSupplier> repairFinishedSuppliers = getRepairFinishedTaskSuppliers(); if (!repairFinishedSuppliers.isEmpty()) @@ -164,9 +164,9 @@ AbstractCompactionTask getNextRepairFinishedTask() Collections.sort(repairFinishedSuppliers); for (TaskSupplier supplier : repairFinishedSuppliers) { - AbstractCompactionTask task = supplier.getTask(); - if (task != null) - return task; + Collection<AbstractCompactionTask> tasks = supplier.getTasks(); + if (tasks != null && !tasks.isEmpty()) + return tasks; } } return null; @@ -180,7 +180,7 @@ private ArrayList<TaskSupplier> getRepairFinishedTaskSuppliers() int numPending = manager.getNumPendingRepairFinishedTasks(); if (numPending > 0) { - suppliers.add(new TaskSupplier(numPending, manager::getNextRepairFinishedTask)); + suppliers.add(new TaskSupplier(numPending, manager::getNextRepairFinishedTasks)); } } diff --git a/src/java/org/apache/cassandra/db/compaction/PendingRepairManager.java b/src/java/org/apache/cassandra/db/compaction/PendingRepairManager.java index 7251c04dcf02..5adb829970b9 100644 --- a/src/java/org/apache/cassandra/db/compaction/PendingRepairManager.java +++ b/src/java/org/apache/cassandra/db/compaction/PendingRepairManager.java @@ -33,13 +33,10 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.cassandra.db.ColumnFamilyStore; -import org.apache.cassandra.db.Directories; -import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; @@ -352,22 +349,25 @@ synchronized int getNumPendingRepairFinishedTasks() return count; } - synchronized AbstractCompactionTask getNextRepairFinishedTask() + synchronized Collection<AbstractCompactionTask> getNextRepairFinishedTasks() { + List<AbstractCompactionTask> tasks = new ArrayList<>(); for (TimeUUID sessionID : strategies.keySet()) { if (canCleanup(sessionID)) { - return getRepairFinishedCompactionTask(sessionID); + RepairFinishedCompactionTask repairFinishedTask = getRepairFinishedCompactionTask(sessionID); + if (repairFinishedTask != null) + tasks.add(repairFinishedTask); } } - return null; + return tasks; } - synchronized AbstractCompactionTask getNextBackgroundTask(long gcBefore) + synchronized Collection<AbstractCompactionTask> getNextBackgroundTasks(long gcBefore) { if (strategies.isEmpty()) - return null; + return Collections.emptyList(); Map<TimeUUID, Integer> numTasks = new HashMap<>(strategies.size()); ArrayList<TimeUUID> sessions = new ArrayList<>(strategies.size()); @@ -382,13 +382,13 @@ synchronized AbstractCompactionTask getNextBackgroundTask(long gcBefore) } if (sessions.isEmpty()) - return null; + return Collections.emptyList(); // we want the session with the most compactions at the head of the list sessions.sort((o1, o2) -> numTasks.get(o2) - numTasks.get(o1)); TimeUUID sessionID = sessions.get(0); - return get(sessionID).getNextBackgroundTask(gcBefore); + return get(sessionID).getNextBackgroundTasks(gcBefore); } synchronized Collection<AbstractCompactionTask> getMaximalTasks(long gcBefore, boolean splitOutput) @@ -405,7 +405,7 @@ synchronized Collection<AbstractCompactionTask> getMaximalTasks(long gcBefore, b } else { - Collection<AbstractCompactionTask> tasks = entry.getValue().getMaximalTask(gcBefore, splitOutput); + Collection<AbstractCompactionTask> tasks = entry.getValue().getMaximalTasks(gcBefore, splitOutput); if (tasks != null) maximalTasks.addAll(tasks); } @@ -536,7 +536,8 @@ protected void runMayThrow() throws Exception { if (obsoleteSSTables) { - transaction.finish(); + transaction.prepareToCommit(); + transaction.commit(); } else { @@ -552,15 +553,9 @@ protected void runMayThrow() throws Exception } } - public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Directories directories, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables) - { - throw new UnsupportedOperationException(); - } - - protected int executeInternal(ActiveCompactionsTracker activeCompactions) + protected void executeInternal(ActiveCompactionsTracker activeCompactions) { run(); - return transaction.originals().size(); } } diff --git a/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java b/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java index 6f68c340f0e5..f35e1659beed 100644 --- a/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java +++ b/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java @@ -24,13 +24,13 @@ import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter; import org.apache.cassandra.db.compaction.writers.MaxSSTableSizeWriter; import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.lifecycle.ILifecycleTransaction; public class SSTableSplitter { private final SplittingCompactionTask task; - public SSTableSplitter(ColumnFamilyStore cfs, LifecycleTransaction transaction, int sstableSizeInMB) + public SSTableSplitter(ColumnFamilyStore cfs, ILifecycleTransaction transaction, int sstableSizeInMB) { this.task = new SplittingCompactionTask(cfs, transaction, sstableSizeInMB); } @@ -44,7 +44,7 @@ public static class SplittingCompactionTask extends CompactionTask { private final int sstableSizeInMiB; - public SplittingCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction transaction, int sstableSizeInMB) + public SplittingCompactionTask(ColumnFamilyStore cfs, ILifecycleTransaction transaction, int sstableSizeInMB) { super(cfs, transaction, CompactionManager.NO_GC, false); this.sstableSizeInMiB = sstableSizeInMB; @@ -62,7 +62,7 @@ protected CompactionController getCompactionController(Set<SSTableReader> toComp @Override public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Directories directories, - LifecycleTransaction txn, + ILifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables) { return new MaxSSTableSizeWriter(cfs, directories, txn, nonExpiredSSTables, sstableSizeInMiB * 1024L * 1024L, 0, false); diff --git a/src/java/org/apache/cassandra/db/compaction/ShardManager.java b/src/java/org/apache/cassandra/db/compaction/ShardManager.java index 6ea2cd72a84c..7a5c65b228ce 100644 --- a/src/java/org/apache/cassandra/db/compaction/ShardManager.java +++ b/src/java/org/apache/cassandra/db/compaction/ShardManager.java @@ -18,7 +18,8 @@ package org.apache.cassandra.db.compaction; -import java.util.Set; +import java.util.*; +import java.util.function.BiFunction; import java.util.stream.Collectors; import com.google.common.collect.ImmutableList; @@ -144,7 +145,7 @@ default int compareByDensity(SSTableReader a, SSTableReader b) /** * Estimate the density of the sstable that will be the result of compacting the given sources. */ - default double calculateCombinedDensity(Set<? extends SSTableReader> sstables) + default double calculateCombinedDensity(Collection<SSTableReader> sstables) { if (sstables.isEmpty()) return 0; @@ -163,4 +164,43 @@ default double calculateCombinedDensity(Set<? extends SSTableReader> sstables) else return onDiskLength; } + + /** + * Seggregate the given sstables into the shard ranges that intersect sstables from the collection, and call + * the given function on the combination of each shard range and the intersecting sstable set. + */ + default <T> List<T> splitSSTablesInShards(Collection<SSTableReader> sstables, + int numShardsForDensity, + BiFunction<Collection<SSTableReader>, Range<Token>, T> maker) + { + ShardTracker boundaries = boundaries(numShardsForDensity); + List<T> tasks = new ArrayList<>(); + SSTableReader[] items = sstables.toArray(SSTableReader[]::new); + Arrays.sort(items, SSTableReader.firstKeyComparator); + PriorityQueue<SSTableReader> active = new PriorityQueue<>(SSTableReader.lastKeyComparator); + int i = 0; + while (i < items.length || !active.isEmpty()) + { + if (active.isEmpty()) + { + boundaries.advanceTo(items[i].getFirst().getToken()); + active.add(items[i++]); + } + Token shardEnd = boundaries.shardEnd(); + + while (i < items.length && (shardEnd == null || items[i].getFirst().getToken().compareTo(shardEnd) <= 0)) + active.add(items[i++]); + + final T result = maker.apply(active, boundaries.shardSpan()); + if (result != null) + tasks.add(result); + + while (!active.isEmpty() && (shardEnd == null || active.peek().getLast().getToken().compareTo(shardEnd) <= 0)) + active.poll(); + + if (!active.isEmpty()) // shardEnd must be non-null (otherwise the line above exhausts all) + boundaries.advanceTo(shardEnd.nextValidToken()); + } + return tasks; + } } diff --git a/src/java/org/apache/cassandra/db/compaction/ShardManagerDiskAware.java b/src/java/org/apache/cassandra/db/compaction/ShardManagerDiskAware.java index 4f8aba283aba..41c5d6700277 100644 --- a/src/java/org/apache/cassandra/db/compaction/ShardManagerDiskAware.java +++ b/src/java/org/apache/cassandra/db/compaction/ShardManagerDiskAware.java @@ -204,7 +204,7 @@ private void setEndToken() public int count() { - return countPerDisk; + return countPerDisk * diskBoundaryPositions.length; } /** @@ -231,7 +231,7 @@ public double rangeSpanned(PartitionPosition first, PartitionPosition last) public int shardIndex() { - return nextShardIndex - 1; + return diskIndex * countPerDisk + nextShardIndex - 1; } } } diff --git a/src/java/org/apache/cassandra/db/compaction/ShardManagerTrivial.java b/src/java/org/apache/cassandra/db/compaction/ShardManagerTrivial.java index 407bff4f0d67..064b5033b5e9 100644 --- a/src/java/org/apache/cassandra/db/compaction/ShardManagerTrivial.java +++ b/src/java/org/apache/cassandra/db/compaction/ShardManagerTrivial.java @@ -18,7 +18,10 @@ package org.apache.cassandra.db.compaction; +import java.util.Collection; +import java.util.List; import java.util.Set; +import java.util.function.BiFunction; import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.dht.IPartitioner; @@ -54,7 +57,7 @@ public double rangeSpanned(SSTableReader rdr) } @Override - public double calculateCombinedDensity(Set<? extends SSTableReader> sstables) + public double calculateCombinedDensity(Collection<SSTableReader> sstables) { double totalSize = 0; for (SSTableReader sstable : sstables) @@ -62,6 +65,14 @@ public double calculateCombinedDensity(Set<? extends SSTableReader> sstables) return totalSize; } + @Override + public <T> List<T> splitSSTablesInShards(Collection<SSTableReader> sstables, + int numShardsForDensity, + BiFunction<Collection<SSTableReader>, Range<Token>, T> maker) + { + return List.of(maker.apply(sstables, new Range<>(partitioner.getMinimumToken(), partitioner.getMinimumToken()))); + } + @Override public double localSpaceCoverage() { diff --git a/src/java/org/apache/cassandra/db/compaction/SingleSSTableLCSTask.java b/src/java/org/apache/cassandra/db/compaction/SingleSSTableLCSTask.java index 1f73c4cd30de..2d9768924cfb 100644 --- a/src/java/org/apache/cassandra/db/compaction/SingleSSTableLCSTask.java +++ b/src/java/org/apache/cassandra/db/compaction/SingleSSTableLCSTask.java @@ -18,19 +18,14 @@ package org.apache.cassandra.db.compaction; -import java.util.Set; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import org.apache.cassandra.db.ColumnFamilyStore; -import org.apache.cassandra.db.Directories; -import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.io.sstable.CorruptSSTableException; import org.apache.cassandra.io.sstable.format.SSTableFormat.Components; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.metadata.StatsMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Special compaction task that does not do any compaction, instead it @@ -51,16 +46,9 @@ public SingleSSTableLCSTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int } @Override - public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Directories directories, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables) - { - throw new UnsupportedOperationException("This method should never be called on SingleSSTableLCSTask"); - } - - @Override - protected int executeInternal(ActiveCompactionsTracker activeCompactions) + protected void executeInternal(ActiveCompactionsTracker activeCompactions) { run(); - return 1; } @Override diff --git a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java index 74a96ca211ab..1ed39ea5626d 100644 --- a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java +++ b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java @@ -30,7 +30,7 @@ import org.apache.cassandra.db.Directories; import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter; import org.apache.cassandra.db.compaction.writers.SplittingSizeTieredCompactionWriter; -import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.lifecycle.ILifecycleTransaction; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.schema.CompactionParams; @@ -175,7 +175,7 @@ private static double hotness(SSTableReader sstr) return sstr.getReadMeter() == null ? 0.0 : sstr.getReadMeter().twoHourRate() / sstr.estimatedKeys(); } - public AbstractCompactionTask getNextBackgroundTask(long gcBefore) + public Collection<AbstractCompactionTask> getNextBackgroundTasks(long gcBefore) { List<SSTableReader> previousCandidate = null; while (true) @@ -183,7 +183,7 @@ public AbstractCompactionTask getNextBackgroundTask(long gcBefore) List<SSTableReader> hottestBucket = getNextBackgroundSSTables(gcBefore); if (hottestBucket.isEmpty()) - return null; + return Collections.emptyList(); // Already tried acquiring references without success. It means there is a race with // the tracker but candidate SSTables were not yet replaced in the compaction strategy manager @@ -192,22 +192,22 @@ public AbstractCompactionTask getNextBackgroundTask(long gcBefore) logger.warn("Could not acquire references for compacting SSTables {} which is not a problem per se," + "unless it happens frequently, in which case it must be reported. Will retry later.", hottestBucket); - return null; + return Collections.emptyList(); } - LifecycleTransaction transaction = cfs.getTracker().tryModify(hottestBucket, OperationType.COMPACTION); + ILifecycleTransaction transaction = cfs.getTracker().tryModify(hottestBucket, OperationType.COMPACTION); if (transaction != null) - return new CompactionTask(cfs, transaction, gcBefore); + return Collections.singletonList(new CompactionTask(cfs, transaction, gcBefore)); previousCandidate = hottestBucket; } } - public synchronized Collection<AbstractCompactionTask> getMaximalTask(final long gcBefore, boolean splitOutput) + public synchronized List<AbstractCompactionTask> getMaximalTasks(final long gcBefore, boolean splitOutput) { Iterable<SSTableReader> filteredSSTables = filterSuspectSSTables(sstables); if (Iterables.isEmpty(filteredSSTables)) return null; - LifecycleTransaction txn = cfs.getTracker().tryModify(filteredSSTables, OperationType.COMPACTION); + ILifecycleTransaction txn = cfs.getTracker().tryModify(filteredSSTables, OperationType.COMPACTION); if (txn == null) return null; if (splitOutput) @@ -219,7 +219,7 @@ public AbstractCompactionTask getUserDefinedTask(Collection<SSTableReader> sstab { assert !sstables.isEmpty(); // checked for by CM.submitUserDefined - LifecycleTransaction transaction = cfs.getTracker().tryModify(sstables, OperationType.COMPACTION); + ILifecycleTransaction transaction = cfs.getTracker().tryModify(sstables, OperationType.COMPACTION); if (transaction == null) { logger.trace("Unable to mark {} for compaction; probably a background compaction got to it first. You can disable background compactions temporarily if this is a problem", sstables); @@ -347,7 +347,7 @@ public String toString() private static class SplittingCompactionTask extends CompactionTask { - public SplittingCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, long gcBefore) + public SplittingCompactionTask(ColumnFamilyStore cfs, ILifecycleTransaction txn, long gcBefore) { super(cfs, txn, gcBefore); } @@ -355,7 +355,7 @@ public SplittingCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, @Override public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Directories directories, - LifecycleTransaction txn, + ILifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables) { return new SplittingSizeTieredCompactionWriter(cfs, directories, txn, nonExpiredSSTables); diff --git a/src/java/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategy.java index 2709d43ae56d..ef4d78a5428e 100644 --- a/src/java/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategy.java +++ b/src/java/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategy.java @@ -80,7 +80,7 @@ public TimeWindowCompactionStrategy(ColumnFamilyStore cfs, Map<String, String> o } @Override - public AbstractCompactionTask getNextBackgroundTask(long gcBefore) + public Collection<AbstractCompactionTask> getNextBackgroundTasks(long gcBefore) { List<SSTableReader> previousCandidate = null; while (true) @@ -88,7 +88,7 @@ public AbstractCompactionTask getNextBackgroundTask(long gcBefore) List<SSTableReader> latestBucket = getNextBackgroundSSTables(gcBefore); if (latestBucket.isEmpty()) - return null; + return Collections.emptyList(); // Already tried acquiring references without success. It means there is a race with // the tracker but candidate SSTables were not yet replaced in the compaction strategy manager @@ -97,12 +97,12 @@ public AbstractCompactionTask getNextBackgroundTask(long gcBefore) logger.warn("Could not acquire references for compacting SSTables {} which is not a problem per se," + "unless it happens frequently, in which case it must be reported. Will retry later.", latestBucket); - return null; + return Collections.emptyList(); } LifecycleTransaction modifier = cfs.getTracker().tryModify(latestBucket, OperationType.COMPACTION); if (modifier != null) - return new TimeWindowCompactionTask(cfs, modifier, gcBefore, options.ignoreOverlaps); + return Collections.singletonList(new TimeWindowCompactionTask(cfs, modifier, gcBefore, options.ignoreOverlaps)); previousCandidate = latestBucket; } } @@ -125,7 +125,7 @@ private synchronized List<SSTableReader> getNextBackgroundSSTables(final long gc if (currentTimeMillis() - lastExpiredCheck > options.expiredSSTableCheckFrequency) { logger.debug("TWCS expired check sufficiently far in the past, checking for fully expired SSTables"); - expired = CompactionController.getFullyExpiredSSTables(cfs, uncompacting, options.ignoreOverlaps ? Collections.emptySet() : cfs.getOverlappingLiveSSTables(uncompacting), + expired = CompactionController.getFullyExpiredSSTables(cfs, uncompacting, cfs::getOverlappingLiveSSTables, gcBefore, options.ignoreOverlaps); lastExpiredCheck = currentTimeMillis(); } @@ -379,7 +379,7 @@ static List<SSTableReader> trimToThreshold(Set<SSTableReader> bucket, int maxThr } @Override - public synchronized Collection<AbstractCompactionTask> getMaximalTask(long gcBefore, boolean splitOutput) + public synchronized List<AbstractCompactionTask> getMaximalTasks(long gcBefore, boolean splitOutput) { Iterable<SSTableReader> filteredSSTables = filterSuspectSSTables(sstables); if (Iterables.isEmpty(filteredSSTables)) @@ -387,7 +387,7 @@ public synchronized Collection<AbstractCompactionTask> getMaximalTask(long gcBef LifecycleTransaction txn = cfs.getTracker().tryModify(filteredSSTables, OperationType.COMPACTION); if (txn == null) return null; - return Collections.singleton(new TimeWindowCompactionTask(cfs, txn, gcBefore, options.ignoreOverlaps)); + return Collections.singletonList(new TimeWindowCompactionTask(cfs, txn, gcBefore, options.ignoreOverlaps)); } /** diff --git a/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionStrategy.java index 089045f28aad..c23e600e7b71 100644 --- a/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionStrategy.java +++ b/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionStrategy.java @@ -31,6 +31,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; @@ -44,8 +45,10 @@ import org.apache.cassandra.db.compaction.unified.Controller; import org.apache.cassandra.db.compaction.unified.ShardedMultiWriter; import org.apache.cassandra.db.compaction.unified.UnifiedCompactionTask; +import org.apache.cassandra.db.lifecycle.CompositeLifecycleTransaction; import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.lifecycle.PartialLifecycleTransaction; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.index.Index; import org.apache.cassandra.io.sstable.Descriptor; @@ -139,15 +142,25 @@ private static int atLeast2(int value, String str) public static String printScalingParameter(int w) { if (w < 0) - return "L" + Integer.toString(2 - w); + return 'L' + Integer.toString(2 - w); else if (w > 0) - return "T" + Integer.toString(w + 2); + return 'T' + Integer.toString(w + 2); else return "N"; } + private TimeUUID nextTimeUUID() + { + // Make a time-UUID with sequence 0. The reason to do this is to accommodate parallelized compactions: + // - Sequence 0 (visible as -8000- in the UUID string) denotes single-task (i.e. non-parallelized) compactions. + // - Sequence >0 (-800n-) denotes the individual task's index of a parallelized compaction. + // - Parallelized compactions use sequence 0 as the transaction id, and sequences from 1 to the number of tasks + // for the ids of individual tasks. + return TimeUUID.Generator.nextTimeUUID().withSequence(0); + } + @Override - public synchronized Collection<AbstractCompactionTask> getMaximalTask(long gcBefore, boolean splitOutput) + public synchronized List<AbstractCompactionTask> getMaximalTasks(long gcBefore, boolean splitOutput) { maybeUpdateShardManager(); // The tasks are split by repair status and disk, as well as in non-overlapping sections to enable some @@ -155,31 +168,55 @@ public synchronized Collection<AbstractCompactionTask> getMaximalTask(long gcBef // split across shards according to its density. Depending on the parallelism, the operation may require up to // 100% extra space to complete. List<AbstractCompactionTask> tasks = new ArrayList<>(); - List<Set<SSTableReader>> nonOverlapping = splitInNonOverlappingSets(filterSuspectSSTables(getSSTables())); - for (Set<SSTableReader> set : nonOverlapping) + + try { + List<SSTableReader> sstables = getSuitableSSTables(); + if (sstables.isEmpty()) + return Collections.emptyList(); + + // If possible, we want to issue separate compactions for non-overlapping sets of sstables, to allow + // for smaller extra space requirements. However, if the sharding configuration has changed, a major + // compaction should combine non-overlapping sets if they are split on a boundary that is no longer + // in effect. + final ShardManager shardManager = getShardManager(); + List<Set<SSTableReader>> groups = + shardManager.splitSSTablesInShards(sstables, + controller.getNumShards(shardManager.calculateCombinedDensity(sstables)), + (sstableShard, shardRange) -> Sets.newHashSet(sstableShard)); + + // Now combine all of these groups that share an sstable so that we have valid independent transactions. + groups = combineSetsWithCommonElement(groups); + + for (Collection<SSTableReader> set : groups) + { + LifecycleTransaction txn = cfs.getTracker().tryModify(set, OperationType.COMPACTION, nextTimeUUID()); + // The tasks may be further split by output shard to increase the parallelism. + if (txn != null) + tasks.addAll(createCompactionTasks(gcBefore, txn)); + // we ignore splitOutput (always split according to the strategy's sharding) and do not need isMaximal + } + + return tasks; + } + catch (Throwable t) { - LifecycleTransaction txn = cfs.getTracker().tryModify(set, OperationType.COMPACTION); - if (txn != null) - tasks.add(createCompactionTask(txn, gcBefore)); + for (AbstractCompactionTask task : tasks) + task.rejected(); + throw t; } - return tasks; } - private static List<Set<SSTableReader>> splitInNonOverlappingSets(Collection<SSTableReader> sstables) + /** + * Transform a list to transitively combine adjacent sets that have a common element, resulting in disjoint sets. + */ + private static <T> List<Set<T>> combineSetsWithCommonElement(List<? extends Set<T>> overlapSets) { - List<Set<SSTableReader>> overlapSets = Overlaps.constructOverlapSets(new ArrayList<>(sstables), - UnifiedCompactionStrategy::startsAfter, - SSTableReader.firstKeyComparator, - SSTableReader.lastKeyComparator); - if (overlapSets.isEmpty()) - return overlapSets; - - Set<SSTableReader> group = overlapSets.get(0); - List<Set<SSTableReader>> groups = new ArrayList<>(); + Set<T> group = overlapSets.get(0); + List<Set<T>> groups = new ArrayList<>(); for (int i = 1; i < overlapSets.size(); ++i) { - Set<SSTableReader> current = overlapSets.get(i); - if (Sets.intersection(current, group).isEmpty()) + Set<T> current = overlapSets.get(i); + if (Collections.disjoint(current, group)) { groups.add(group); group = current; @@ -198,7 +235,7 @@ public AbstractCompactionTask getUserDefinedTask(Collection<SSTableReader> sstab { assert !sstables.isEmpty(); // checked for by CM.submitUserDefined - LifecycleTransaction transaction = cfs.getTracker().tryModify(sstables, OperationType.COMPACTION); + LifecycleTransaction transaction = cfs.getTracker().tryModify(sstables, OperationType.COMPACTION, nextTimeUUID()); if (transaction == null) { logger.trace("Unable to mark {} for compaction; probably a background compaction got to it first. You can disable background compactions temporarily if this is a problem", sstables); @@ -209,37 +246,37 @@ public AbstractCompactionTask getUserDefinedTask(Collection<SSTableReader> sstab } /** - * Returns a compaction task to run next. - * - * This method is synchronized because task creation is significantly more expensive in UCS; the strategy is + * Returns a collections of compaction tasks. + * <p> + * This method is synchornized because task creation is significantly more expensive in UCS; the strategy is * stateless, therefore it has to compute the shard/bucket structure on each call. * * @param gcBefore throw away tombstones older than this + * @return collection of AbstractCompactionTask, which could be either a CompactionTask or an UnifiedCompactionTask */ @Override - public synchronized UnifiedCompactionTask getNextBackgroundTask(long gcBefore) + public synchronized Collection<AbstractCompactionTask> getNextBackgroundTasks(long gcBefore) { while (true) { CompactionPick pick = getNextCompactionPick(gcBefore); if (pick == null) - return null; - UnifiedCompactionTask task = createCompactionTask(pick, gcBefore); - if (task != null) - return task; + return Collections.emptyList(); + Collection<AbstractCompactionTask> tasks = createCompactionTasks(pick, gcBefore); + if (tasks != null) + return tasks; } } - private UnifiedCompactionTask createCompactionTask(CompactionPick pick, long gcBefore) + private Collection<AbstractCompactionTask> createCompactionTasks(CompactionPick pick, long gcBefore) { Preconditions.checkNotNull(pick); Preconditions.checkArgument(!pick.isEmpty()); - LifecycleTransaction transaction = cfs.getTracker().tryModify(pick, - OperationType.COMPACTION); + LifecycleTransaction transaction = cfs.getTracker().tryModify(pick, OperationType.COMPACTION, nextTimeUUID()); if (transaction != null) { - return createCompactionTask(transaction, gcBefore); + return createCompactionTasks(gcBefore, transaction); } else { @@ -284,6 +321,15 @@ public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, boundaries); } + @VisibleForTesting + List<AbstractCompactionTask> createCompactionTasks(long gcBefore, LifecycleTransaction transaction) + { + if (controller.parallelizeOutputShards()) + return createParallelCompactionTasks(transaction, gcBefore); + else + return ImmutableList.of(createCompactionTask(transaction, gcBefore)); + } + /** * Create the task that in turns creates the sstable writer used for compaction. * @@ -294,6 +340,42 @@ private UnifiedCompactionTask createCompactionTask(LifecycleTransaction transact return new UnifiedCompactionTask(cfs, this, transaction, gcBefore, getShardManager()); } + private List<AbstractCompactionTask> createParallelCompactionTasks(LifecycleTransaction transaction, long gcBefore) + { + Collection<SSTableReader> sstables = transaction.originals(); + ShardManager shardManager = getShardManager(); + CompositeLifecycleTransaction compositeTransaction = new CompositeLifecycleTransaction(transaction); + + double density = shardManager.calculateCombinedDensity(sstables); + int numShards = controller.getNumShards(density * shardManager.shardSetCoverage()); + if (numShards <= 1) + return Collections.singletonList(createCompactionTask(transaction, gcBefore)); + List<AbstractCompactionTask> tasks = shardManager.splitSSTablesInShards( + sstables, + numShards, + (rangeSSTables, range) -> + new UnifiedCompactionTask(cfs, + this, + new PartialLifecycleTransaction(compositeTransaction), + gcBefore, + shardManager, + range, + rangeSSTables) + ); + compositeTransaction.completeInitialization(); + + if (tasks.isEmpty()) + transaction.close(); // this should not be reachable normally, close the transaction for safety + + if (tasks.size() == 1) // if there's just one range, make it a non-ranged task (to apply early open etc.) + { + assert ((CompactionTask) tasks.get(0)).inputSSTables().equals(sstables); + return Collections.singletonList(createCompactionTask(transaction, gcBefore)); + } + else + return tasks; + } + private void maybeUpdateShardManager() { // TODO - modify ShardManager::isOutOfDate to take an Epoch @@ -358,7 +440,7 @@ private Set<SSTableReader> maybeGetExpiredSSTables(long gcBefore, List<SSTableRe lastExpiredCheck = ts; expired = CompactionController.getFullyExpiredSSTables(cfs, suitable, - cfs.getOverlappingLiveSSTables(suitable), + cfs::getOverlappingLiveSSTables, gcBefore, controller.getIgnoreOverlapsInExpirationCheck()); if (logger.isTraceEnabled() && !expired.isEmpty()) @@ -452,7 +534,7 @@ List<Level> getLevels() * new compactions, and by external tools to analyze the strategy decisions. * * @param sstables a collection of the sstables to be assigned to levels - * @param compactionFilter a filter to exclude CompactionSSTables, + * @param compactionFilter a filter to exclude SSTableReaders, * e.g., {@link #isSuitableForCompaction} * * @return a list of the levels in the compaction hierarchy @@ -512,6 +594,11 @@ private List<Level> formLevels(List<SSTableReader> suitable) return levels; } + List<SSTableReader> getSuitableSSTables() + { + return getCompactableSSTables(getSSTables(), UnifiedCompactionStrategy::isSuitableForCompaction); + } + private List<SSTableReader> getCompactableSSTables(Collection<SSTableReader> sstables, Predicate<SSTableReader> compactionFilter) { diff --git a/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionStrategy.md b/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionStrategy.md index 5f8d548af97c..23d309ea46f1 100644 --- a/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionStrategy.md +++ b/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionStrategy.md @@ -218,7 +218,7 @@ This sharding mechanism is independent of the compaction specification. This sharding scheme easily admits extensions. In particular, when the size of the data set is expected to grow very large, to avoid having to pre-specify a high enough target size to avoid problems with per-sstable overhead, we can -apply an "SSTtable growth" parameter, which determines what part of the density growth should be assigned to increased +apply an "SSTable growth" parameter, which determines what part of the density growth should be assigned to increased SSTable size, reducing the growth of the number of shards (and hence non-overlapping sstables). Additionally, to allow for a mode of operation with a fixed number of shards, and splitting conditional on reaching @@ -332,14 +332,38 @@ with legacy strategies (e.g. all resources consumed by L0 and sstables accumulat steady state where compactions always use more sstables than the assigned threshold and fan factor and maintain a tiered hierarchy based on the lowest overlap they are able to maintain for the load. -## Major compaction +## Output shard parallelization + +Because the sharding of the output of a compaction operation is known in advance, we can parallelize the compaction +process by starting a separate task for each shard. This can dramatically speed the throughput of compaction and is +especially helpful for the lower levels of the compaction heirarchy, where the number of input shards is very low +(often just one). To make sure that we correctly change the state of input and output sstables, such operations will +share a transaction and will complete only when all individual tasks complete (and, conversely, abort if any of the +individual tasks abort). Early opening of sstables is not supported in this mode, because we currently do not support +arbitraty filtering of the requests to an sstable; it is expected that the smaller size and quicker completion time of +compactions should make up for this. -Under the working principles of UCS, a major compaction is an operation which compacts together all sstables that have -(transitive) overlap, and where the output is split on shard boundaries appropriate for the expected result density. +This is controlled by the `parallelize_output_shards` parameter, which is `true` by default. + +## Major compaction -In other words, it is expected that a major compaction will result in $b$ concurrent compactions, each containing all -sstables covered in each of the base shards, and that the result will be split on shard boundaries whose number -depends on the total size of data contained in the shard. +Major compaction in UCS always splits the output into a shard number suitable for the expected result density. +If the input sstables can be split into non-overlapping sets that correspond to current shard boundaries, the compaction +will construct independent operations that work over these sets, to improve the space overhead of the operation as well +as the time needed to persistently complete individual steps. Because all levels will usually be split in $b$ shards, +it will very often be the case that major compactions split into $b$ individual jobs, reducing the space overhead by a +factor close to $b$. Note that this does not always apply; for example, if a topology change causes the sharding +boundaries to move, the mismatch between old and new sharding boundaries will cause the compaction to produce a single +operation and require 100% space overhead. + +Output shard parallelization also applies to major compactions: if the `parallelize_output_shards` option is enabled, +shards of individual compactions will be compacted concurrently, which can significantly reduce the time needed to +perform the compaction; if the option is not enabled, major compaction will only be parallelized up to the number of +individual non-overlapping sets the sstables can be split into. In either case, the number of parallel operations is +limited to a number specified as a parameter of the operation (e.g. `nodetool compact -j n`), which is set to half the +compaction thread count by default. Using a jobs of 0 will let the compaction use all available threads and run +as quickly as possible, but this will prevent other compaction operations from running until it completes and thus +should be used with caution, only while the database is known to not receive any writes. ## Differences with STCS and LCS @@ -441,6 +465,11 @@ UCS accepts these compaction strategy parameters: that are considered too small. If set, the strategy will split the space into fewer than the base count shards, to make the estimated sstables size at least as large as this value. A value of 0 disables this feature. The default value is 100MiB. +* **parallelize_output_shards**. Enables or disables parallelization of compaction tasks for the output shards of a + compaction. This can dramatically improve compaction throughput especially on the lowest levels of the hierarchy, + but disables early open and thus may be less efficient when compaction is configured to produce very large + sstables. + The default value is `true`. * **expired_sstable_check_frequency_seconds**. Determines how often to check for expired SSTables. The default value is 10 minutes. diff --git a/src/java/org/apache/cassandra/db/compaction/unified/Controller.java b/src/java/org/apache/cassandra/db/compaction/unified/Controller.java index 5faab2702787..df2a66590c30 100644 --- a/src/java/org/apache/cassandra/db/compaction/unified/Controller.java +++ b/src/java/org/apache/cassandra/db/compaction/unified/Controller.java @@ -31,6 +31,7 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.compaction.CompactionManager; import org.apache.cassandra.db.compaction.UnifiedCompactionStrategy; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.utils.FBUtilities; @@ -150,6 +151,14 @@ public class Controller static final Overlaps.InclusionMethod DEFAULT_OVERLAP_INCLUSION_METHOD = CassandraRelevantProperties.UCS_OVERLAP_INCLUSION_METHOD.getEnum(Overlaps.InclusionMethod.TRANSITIVE); + /** + * Whether to create subtask for the output shards of individual compactions and execute them in parallel. + * Defaults to true for improved parallelization and efficiency. + */ + static final String PARALLELIZE_OUTPUT_SHARDS_OPTION = "parallelize_output_shards"; + static final boolean DEFAULT_PARALLELIZE_OUTPUT_SHARDS = + CassandraRelevantProperties.UCS_PARALLELIZE_OUTPUT_SHARDS.getBoolean(true); + protected final ColumnFamilyStore cfs; protected final MonotonicClock clock; private final int[] scalingParameters; @@ -172,6 +181,7 @@ public class Controller private static final double INVERSE_LOG_2 = 1.0 / Math.log(2); protected final Overlaps.InclusionMethod overlapInclusionMethod; + protected final boolean parallelizeOutputShards; Controller(ColumnFamilyStore cfs, MonotonicClock clock, @@ -185,7 +195,8 @@ public class Controller int baseShardCount, double targetSStableSize, double sstableGrowthModifier, - Overlaps.InclusionMethod overlapInclusionMethod) + Overlaps.InclusionMethod overlapInclusionMethod, + boolean parallelizeOutputShards) { this.cfs = cfs; this.clock = clock; @@ -199,6 +210,7 @@ public class Controller this.targetSSTableSize = targetSStableSize; this.overlapInclusionMethod = overlapInclusionMethod; this.sstableGrowthModifier = sstableGrowthModifier; + this.parallelizeOutputShards = parallelizeOutputShards; if (maxSSTablesToCompact <= 0) maxSSTablesToCompact = Integer.MAX_VALUE; @@ -356,6 +368,11 @@ else if (pow >= 0) } } + public boolean parallelizeOutputShards() + { + return parallelizeOutputShards; + } + /** * @return the survival factor o * @param index @@ -446,6 +463,10 @@ public static Controller fromOptions(ColumnFamilyStore cfs, Map<String, String> ? Overlaps.InclusionMethod.valueOf(toUpperCaseLocalized(options.get(OVERLAP_INCLUSION_METHOD_OPTION))) : DEFAULT_OVERLAP_INCLUSION_METHOD; + boolean parallelizeOutputShards = options.containsKey(PARALLELIZE_OUTPUT_SHARDS_OPTION) + ? Boolean.parseBoolean(options.get(PARALLELIZE_OUTPUT_SHARDS_OPTION)) + : DEFAULT_PARALLELIZE_OUTPUT_SHARDS; + return new Controller(cfs, MonotonicClock.Global.preciseTime, Ws, @@ -458,7 +479,8 @@ public static Controller fromOptions(ColumnFamilyStore cfs, Map<String, String> baseShardCount, targetSStableSize, sstableGrowthModifier, - inclusionMethod); + inclusionMethod, + parallelizeOutputShards); } public static Map<String, String> validateOptions(Map<String, String> options) throws ConfigurationException @@ -572,12 +594,8 @@ public static Map<String, String> validateOptions(Map<String, String> options) t } } - s = options.remove(ALLOW_UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION_OPTION); - if (s != null && !s.equalsIgnoreCase("true") && !s.equalsIgnoreCase("false")) - { - throw new ConfigurationException(String.format("%s should either be 'true' or 'false', not %s", - ALLOW_UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION_OPTION, s)); - } + validateBoolean(options, ALLOW_UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION_OPTION); + validateBoolean(options, PARALLELIZE_OUTPUT_SHARDS_OPTION); s = options.remove(OVERLAP_INCLUSION_METHOD_OPTION); if (s != null) @@ -645,6 +663,16 @@ public static Map<String, String> validateOptions(Map<String, String> options) t return options; } + private static void validateBoolean(Map<String, String> options, String option) + { + String s; + s = options.remove(option); + if (s != null && !s.equalsIgnoreCase("true") && !s.equalsIgnoreCase("false")) { + throw new ConfigurationException(String.format("%s should either be 'true' or 'false', not %s", + option, s)); + } + } + // The methods below are implemented here (rather than directly in UCS) to aid testability. public double getBaseSstableSize(int F) @@ -675,7 +703,7 @@ public double maxThroughput() public int maxConcurrentCompactions() { - return DatabaseDescriptor.getConcurrentCompactors(); + return CompactionManager.instance.getMaximumCompactorThreads(); } public int maxSSTablesToCompact() diff --git a/src/java/org/apache/cassandra/db/compaction/unified/ShardedCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/unified/ShardedCompactionWriter.java index ca5e99749cca..c90afaa8d7d0 100644 --- a/src/java/org/apache/cassandra/db/compaction/unified/ShardedCompactionWriter.java +++ b/src/java/org/apache/cassandra/db/compaction/unified/ShardedCompactionWriter.java @@ -28,7 +28,7 @@ import org.apache.cassandra.db.Directories; import org.apache.cassandra.db.compaction.ShardTracker; import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter; -import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.lifecycle.ILifecycleTransaction; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.format.SSTableWriter; import org.apache.cassandra.utils.FBUtilities; @@ -47,12 +47,13 @@ public class ShardedCompactionWriter extends CompactionAwareWriter public ShardedCompactionWriter(ColumnFamilyStore cfs, Directories directories, - LifecycleTransaction txn, + ILifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, boolean keepOriginals, + boolean earlyOpenAllowed, ShardTracker boundaries) { - super(cfs, directories, txn, nonExpiredSSTables, keepOriginals); + super(cfs, directories, txn, nonExpiredSSTables, keepOriginals, earlyOpenAllowed); this.boundaries = boundaries; long totalKeyCount = nonExpiredSSTables.stream() diff --git a/src/java/org/apache/cassandra/db/compaction/unified/UnifiedCompactionTask.java b/src/java/org/apache/cassandra/db/compaction/unified/UnifiedCompactionTask.java index 5a729f643299..0b3e9a6f4a6a 100644 --- a/src/java/org/apache/cassandra/db/compaction/unified/UnifiedCompactionTask.java +++ b/src/java/org/apache/cassandra/db/compaction/unified/UnifiedCompactionTask.java @@ -18,15 +18,20 @@ package org.apache.cassandra.db.compaction.unified; +import java.util.Collection; import java.util.Set; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Directories; import org.apache.cassandra.db.compaction.CompactionTask; import org.apache.cassandra.db.compaction.ShardManager; import org.apache.cassandra.db.compaction.UnifiedCompactionStrategy; import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter; -import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.lifecycle.ILifecycleTransaction; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.format.SSTableReader; /** @@ -36,26 +41,80 @@ public class UnifiedCompactionTask extends CompactionTask { private final ShardManager shardManager; private final Controller controller; + private final Range<Token> operationRange; + private final Set<SSTableReader> actuallyCompact; public UnifiedCompactionTask(ColumnFamilyStore cfs, UnifiedCompactionStrategy strategy, - LifecycleTransaction txn, + ILifecycleTransaction txn, long gcBefore, ShardManager shardManager) + { + this(cfs, strategy, txn, gcBefore, shardManager, null, null); + } + + public UnifiedCompactionTask(ColumnFamilyStore cfs, + UnifiedCompactionStrategy strategy, + ILifecycleTransaction txn, + long gcBefore, + ShardManager shardManager, + Range<Token> operationRange, + Collection<SSTableReader> actuallyCompact) { super(cfs, txn, gcBefore); this.controller = strategy.getController(); this.shardManager = shardManager; + + if (operationRange != null) + { + assert actuallyCompact != null : "Ranged tasks should use a set of sstables to compact"; + } + this.operationRange = operationRange; + // To make sure actuallyCompact tracks any removals from txn.originals(), we intersect the given set with it. + // This should not be entirely necessary (as shouldReduceScopeForSpace() is false for ranged tasks), but it + // is cleaner to enforce inputSSTables()'s requirements. + this.actuallyCompact = actuallyCompact != null ? Sets.intersection(ImmutableSet.copyOf(actuallyCompact), + txn.originals()) + : txn.originals(); } @Override public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Directories directories, - LifecycleTransaction txn, + ILifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables) { double density = shardManager.calculateCombinedDensity(nonExpiredSSTables); int numShards = controller.getNumShards(density * shardManager.shardSetCoverage()); - return new ShardedCompactionWriter(cfs, directories, txn, nonExpiredSSTables, keepOriginals, shardManager.boundaries(numShards)); + // In multi-task operations we need to expire many ranges in a source sstable for early open. Not doable yet. + boolean earlyOpenAllowed = tokenRange() == null; + return new ShardedCompactionWriter(cfs, + directories, + txn, + nonExpiredSSTables, + keepOriginals, + earlyOpenAllowed, + shardManager.boundaries(numShards)); + } + + @Override + protected Range<Token> tokenRange() + { + return operationRange; + } + + @Override + protected boolean shouldReduceScopeForSpace() + { + // Because parallelized tasks share input sstables, we can't reduce the scope of individual tasks + // (as doing that will leave some part of an sstable out of the compaction but still drop the whole sstable + // when the task set completes). + return tokenRange() == null; + } + + @Override + protected Set<SSTableReader> inputSSTables() + { + return actuallyCompact; } } \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java index 145163a39cf6..ea21f7be57e0 100644 --- a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java +++ b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java @@ -33,7 +33,7 @@ import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.db.SerializationHeader; import org.apache.cassandra.db.compaction.CompactionTask; -import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.lifecycle.ILifecycleTransaction; import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.io.sstable.Descriptor; import org.apache.cassandra.io.sstable.SSTableRewriter; @@ -64,7 +64,7 @@ public abstract class CompactionAwareWriter extends Transactional.AbstractTransa protected final boolean isTransient; protected final SSTableRewriter sstableWriter; - protected final LifecycleTransaction txn; + protected final ILifecycleTransaction txn; private final List<Directories.DataDirectory> locations; private final List<PartitionPosition> diskBoundaries; private int locationIndex; @@ -72,9 +72,19 @@ public abstract class CompactionAwareWriter extends Transactional.AbstractTransa public CompactionAwareWriter(ColumnFamilyStore cfs, Directories directories, - LifecycleTransaction txn, + ILifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, boolean keepOriginals) + { + this(cfs, directories, txn, nonExpiredSSTables, keepOriginals, true); + } + + public CompactionAwareWriter(ColumnFamilyStore cfs, + Directories directories, + ILifecycleTransaction txn, + Set<SSTableReader> nonExpiredSSTables, + boolean keepOriginals, + boolean earlyOpenAllowed) { this.cfs = cfs; this.directories = directories; @@ -83,7 +93,7 @@ public CompactionAwareWriter(ColumnFamilyStore cfs, estimatedTotalKeys = SSTableReader.getApproximateKeyCount(nonExpiredSSTables); maxAge = CompactionTask.getMaxDataAge(nonExpiredSSTables); - sstableWriter = SSTableRewriter.construct(cfs, txn, keepOriginals, maxAge); + sstableWriter = SSTableRewriter.construct(cfs, txn, keepOriginals, maxAge, earlyOpenAllowed); minRepairedAt = CompactionTask.getMinRepairedAt(nonExpiredSSTables); pendingRepair = CompactionTask.getPendingRepair(nonExpiredSSTables); isTransient = CompactionTask.getIsTransient(nonExpiredSSTables); diff --git a/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java index fbb0e27a99b0..2b124f4417ab 100644 --- a/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java +++ b/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java @@ -26,7 +26,7 @@ import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.Directories; -import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.lifecycle.ILifecycleTransaction; import org.apache.cassandra.io.sstable.format.SSTableReader; /** @@ -37,12 +37,12 @@ public class DefaultCompactionWriter extends CompactionAwareWriter protected static final Logger logger = LoggerFactory.getLogger(DefaultCompactionWriter.class); private final int sstableLevel; - public DefaultCompactionWriter(ColumnFamilyStore cfs, Directories directories, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables) + public DefaultCompactionWriter(ColumnFamilyStore cfs, Directories directories, ILifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables) { this(cfs, directories, txn, nonExpiredSSTables, false, 0); } - public DefaultCompactionWriter(ColumnFamilyStore cfs, Directories directories, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, boolean keepOriginals, int sstableLevel) + public DefaultCompactionWriter(ColumnFamilyStore cfs, Directories directories, ILifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, boolean keepOriginals, int sstableLevel) { super(cfs, directories, txn, nonExpiredSSTables, keepOriginals); this.sstableLevel = sstableLevel; diff --git a/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java index 09263df8530b..d0fb70587ca1 100644 --- a/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java +++ b/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java @@ -23,7 +23,7 @@ import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.Directories; import org.apache.cassandra.db.compaction.LeveledManifest; -import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.lifecycle.ILifecycleTransaction; import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.io.sstable.format.SSTableReader; @@ -40,7 +40,7 @@ public class MajorLeveledCompactionWriter extends CompactionAwareWriter public MajorLeveledCompactionWriter(ColumnFamilyStore cfs, Directories directories, - LifecycleTransaction txn, + ILifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, long maxSSTableSize) { @@ -49,7 +49,7 @@ public MajorLeveledCompactionWriter(ColumnFamilyStore cfs, public MajorLeveledCompactionWriter(ColumnFamilyStore cfs, Directories directories, - LifecycleTransaction txn, + ILifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, long maxSSTableSize, boolean keepOriginals) diff --git a/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java index 1ded2128e77d..75f85b1e4da0 100644 --- a/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java +++ b/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java @@ -23,7 +23,7 @@ import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.Directories; import org.apache.cassandra.db.compaction.OperationType; -import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.lifecycle.ILifecycleTransaction; import org.apache.cassandra.io.sstable.format.SSTableReader; public class MaxSSTableSizeWriter extends CompactionAwareWriter @@ -34,7 +34,7 @@ public class MaxSSTableSizeWriter extends CompactionAwareWriter public MaxSSTableSizeWriter(ColumnFamilyStore cfs, Directories directories, - LifecycleTransaction txn, + ILifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, long maxSSTableSize, int level) @@ -44,7 +44,7 @@ public MaxSSTableSizeWriter(ColumnFamilyStore cfs, public MaxSSTableSizeWriter(ColumnFamilyStore cfs, Directories directories, - LifecycleTransaction txn, + ILifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, long maxSSTableSize, int level, diff --git a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java index 4cd0858e18a4..917fbb9cf05a 100644 --- a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java +++ b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java @@ -26,7 +26,7 @@ import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.Directories; -import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.lifecycle.ILifecycleTransaction; import org.apache.cassandra.io.sstable.format.SSTableReader; /** @@ -46,12 +46,12 @@ public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter private long currentBytesToWrite; private int currentRatioIndex = 0; - public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, Directories directories, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables) + public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, Directories directories, ILifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables) { this(cfs, directories, txn, nonExpiredSSTables, DEFAULT_SMALLEST_SSTABLE_BYTES); } - public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, Directories directories, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, long smallestSSTable) + public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, Directories directories, ILifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, long smallestSSTable) { super(cfs, directories, txn, nonExpiredSSTables, false); this.allSSTables = txn.originals(); diff --git a/src/java/org/apache/cassandra/db/lifecycle/CompositeLifecycleTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/CompositeLifecycleTransaction.java new file mode 100644 index 000000000000..1628baf3e9f2 --- /dev/null +++ b/src/java/org/apache/cassandra/db/lifecycle/CompositeLifecycleTransaction.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.lifecycle; + +import java.util.concurrent.atomic.AtomicInteger; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.utils.TimeUUID; + +/// Composite lifecycle transaction. This is a wrapper around a lifecycle transaction that allows for multiple partial +/// operations that comprise the whole transaction. This is used to parallelize compaction operations over individual +/// output shards where the compaction sources are shared among the operations; in this case we can only release the +/// shared sources once all operations are complete. +/// +/// A composite transaction is initialized with a main transaction that will be used to commit the transaction. Each +/// part of the composite transaction must be registered with the transaction before it is used. The transaction must +/// be initialized by calling [#completeInitialization()] before any of the processing is allowed to proceed. +/// +/// The transaction is considered complete when all parts have been committed or aborted. If any part is aborted, the +/// whole transaction is also aborted ([PartialLifecycleTransaction] will also throw an exception on other parts when +/// they access it if the composite is already aborted). +/// +/// When all parts are committed, the full transaction is applied by performing a checkpoint, obsoletion of the +/// originals if any of the parts requested it, preparation and commit. This may somewhat violate the rules of +/// transactions as a part that has been committed may actually have no effect if another part is aborted later. +/// There are also restrictions on the operations that this model can accept, e.g. replacement of sources and partial +/// checkpointing are not supported (as they are parts of early open which we don't aim to support at this time), +/// and we consider that all parts will have the same opinion about the obsoletion of the originals. +public class CompositeLifecycleTransaction +{ + protected static final Logger logger = LoggerFactory.getLogger(CompositeLifecycleTransaction.class); + + final LifecycleTransaction mainTransaction; + private final AtomicInteger partsToCommitOrAbort; + private volatile boolean obsoleteOriginalsRequested; + private volatile boolean wasAborted; + private volatile boolean initializationComplete; + private volatile int partsCount = 0; + + /// Create a composite transaction wrapper over the given transaction. After construction, the individual parts of + /// the operation must be registered using [#register] and the composite sealed by calling [#completeInitialization]. + /// The composite will then track the state of the parts and commit after all of them have committed (respectively + /// abort if one aborts but only after waiting for all the other tasks to complete, successfully or not). + /// + /// To make it easy to recognize the parts of a composite transaction, the given transaction should have an id with + /// sequence number 0, and partial transactions should use the id that [#register] returns. + public CompositeLifecycleTransaction(LifecycleTransaction mainTransaction) + { + this.mainTransaction = mainTransaction; + this.partsToCommitOrAbort = new AtomicInteger(0); + this.wasAborted = false; + this.obsoleteOriginalsRequested = false; + } + + /// Register one part of the composite transaction. Every part must register itself before the composite transaction + /// is initialized and the parts are allowed to proceed. + /// @param part the part to register + public TimeUUID register(PartialLifecycleTransaction part) + { + int index = partsToCommitOrAbort.incrementAndGet(); + return mainTransaction.opId().withSequence(index); + } + + /// Complete the initialization of the composite transaction. This must be called before any of the parts are + /// executed. + public void completeInitialization() + { + partsCount = partsToCommitOrAbort.get(); + initializationComplete = true; + if (logger.isTraceEnabled()) + logger.trace("Composite transaction {} initialized with {} parts.", mainTransaction.opIdString(), partsCount); + } + + /// Get the number of parts in the composite transaction. 0 if the transaction is not yet initialized. + public int partsCount() + { + return partsCount; + } + + /// Request that the original sstables are obsoleted when the transaction is committed. Note that this class has + /// an expectation that all parts will have the same opinion about this, and one request will be sufficient to + /// trigger obsoletion. + public void requestObsoleteOriginals() + { + obsoleteOriginalsRequested = true; + } + + /// Commit a part of the composite transaction. This will trigger the final commit of the whole transaction if it is + /// the last part to complete. A part has to commit or abort exactly once. + public void commitPart() + { + partCommittedOrAborted(); + } + + /// Signal an abort of one part of the transaction. If this is the last part to signal, the whole transaction will + /// now abort. Otherwise the composite transaction will wait for the other parts to complete and will abort the + /// composite when they all give their commit or abort signal. A part has to commit or abort exactly once. + /// + /// [PartialLifecycleTransaction] will attempt to abort other parts sooner by throwing an exception when any of its + /// methods are called when the composite transaction is already aborted. + public void abortPart() + { + wasAborted = true; + partCommittedOrAborted(); + } + + boolean wasAborted() + { + return wasAborted; + } + + private void partCommittedOrAborted() + { + if (!initializationComplete) + throw new IllegalStateException("Composite transaction used before initialization is complete."); + if (partsToCommitOrAbort.decrementAndGet() == 0) + { + if (wasAborted) + { + if (logger.isTraceEnabled()) + logger.trace("Composite transaction {} with {} parts aborted.", + mainTransaction.opIdString(), + partsCount); + + mainTransaction.abort(); + } + else + { + if (logger.isTraceEnabled()) + logger.trace("Composite transaction {} with {} parts completed{}.", + mainTransaction.opIdString(), + partsCount, + obsoleteOriginalsRequested ? " with obsoletion" : ""); + + mainTransaction.checkpoint(); + if (obsoleteOriginalsRequested) + mainTransaction.obsoleteOriginals(); + mainTransaction.prepareToCommit(); + mainTransaction.commit(); + } + } + } +} diff --git a/src/java/org/apache/cassandra/db/lifecycle/ILifecycleTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/ILifecycleTransaction.java index c014e3865f93..f3e9c9b7c5f1 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/ILifecycleTransaction.java +++ b/src/java/org/apache/cassandra/db/lifecycle/ILifecycleTransaction.java @@ -21,7 +21,11 @@ import java.util.Collection; import java.util.Set; +import com.google.common.collect.Iterables; + import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.utils.Throwables; +import org.apache.cassandra.utils.TimeUUID; import org.apache.cassandra.utils.concurrent.Transactional; public interface ILifecycleTransaction extends Transactional, LifecycleNewTracker @@ -29,10 +33,37 @@ public interface ILifecycleTransaction extends Transactional, LifecycleNewTracke void checkpoint(); void update(SSTableReader reader, boolean original); void update(Collection<SSTableReader> readers, boolean original); - public SSTableReader current(SSTableReader reader); + SSTableReader current(SSTableReader reader); void obsolete(SSTableReader reader); void obsoleteOriginals(); Set<SSTableReader> originals(); boolean isObsolete(SSTableReader reader); boolean isOffline(); + TimeUUID opId(); + + /// Op identifier as a string to use in debug prints. Usually just the opId, with added part information for partial + /// transactions. + default String opIdString() + { + return opId().toString(); + } + + void cancel(SSTableReader removedSSTable); + + default void abort() + { + Throwables.maybeFail(abort(null)); + } + + default void commit() + { + Throwables.maybeFail(commit(null)); + } + + default SSTableReader onlyOne() + { + final Set<SSTableReader> originals = originals(); + assert originals.size() == 1; + return Iterables.getFirst(originals, null); + } } diff --git a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java index f6e3bb8a1fdf..4c871cb8e485 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java +++ b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java @@ -125,7 +125,7 @@ public String toString() } } - public final Tracker tracker; + private final Tracker tracker; // The transaction logs keep track of new and old sstable files private final LogTransaction log; // the original readers this transaction was opened over, and that it guards @@ -184,6 +184,11 @@ public static LifecycleTransaction offline(OperationType operationType) this(tracker, new LogTransaction(operationType, tracker), readers); } + LifecycleTransaction(Tracker tracker, OperationType operationType, Iterable<? extends SSTableReader> readers, TimeUUID id) + { + this(tracker, new LogTransaction(operationType, tracker, id), readers); + } + LifecycleTransaction(Tracker tracker, LogTransaction log, Iterable<? extends SSTableReader> readers) { this.tracker = tracker; @@ -207,11 +212,18 @@ public OperationType opType() return log.type(); } + @Override public TimeUUID opId() { return log.id(); } + @VisibleForTesting + public Tracker tracker() + { + return tracker; + } + public void doPrepare() { // note for future: in anticompaction two different operations use the same Transaction, and both prepareToCommit() @@ -576,6 +588,7 @@ private Throwable unmarkCompacting(Set<SSTableReader> unmark, Throwable accumula } // convenience method for callers that know only one sstable is involved in the transaction + // overridden to avoid defensive copying public SSTableReader onlyOne() { assert originals.size() == 1; diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java index b1d13abf3d24..635ab38ec621 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java +++ b/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java @@ -133,9 +133,14 @@ public CorruptTransactionLogException(String message, LogFile txnFile) } LogTransaction(OperationType opType, Tracker tracker) + { + this(opType, tracker, nextTimeUUID()); + } + + LogTransaction(OperationType opType, Tracker tracker, TimeUUID id) { this.tracker = tracker; - this.txnFile = new LogFile(opType, nextTimeUUID()); + this.txnFile = new LogFile(opType, id); this.lock = new Object(); this.selfRef = new Ref<>(this, new TransactionTidier(txnFile, lock)); diff --git a/src/java/org/apache/cassandra/db/lifecycle/PartialLifecycleTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/PartialLifecycleTransaction.java new file mode 100644 index 000000000000..fccf79479c1f --- /dev/null +++ b/src/java/org/apache/cassandra/db/lifecycle/PartialLifecycleTransaction.java @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.lifecycle; + +import java.util.Collection; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.cassandra.db.compaction.OperationType; +import org.apache.cassandra.io.sstable.SSTable; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.utils.Throwables; +import org.apache.cassandra.utils.TimeUUID; + +/// Partial lifecycle transaction. This works together with a CompositeLifecycleTransaction to allow for multiple +/// tasks using a shared transaction to be committed or aborted together. This is used to parallelize compaction +/// operations over the same sources. See [CompositeLifecycleTransaction] for more details. +/// +/// This class takes care of synchronizing various operations on the shared transaction, making sure that an abort +/// or commit signal is given exactly once (provided that this partial transaction is closed), and throwing an exception +/// when progress is made when the transaction was already aborted by another part. +public class PartialLifecycleTransaction implements ILifecycleTransaction +{ + final CompositeLifecycleTransaction composite; + final ILifecycleTransaction mainTransaction; + final AtomicBoolean committedOrAborted = new AtomicBoolean(false); + final TimeUUID id; + + public PartialLifecycleTransaction(CompositeLifecycleTransaction composite) + { + this.composite = composite; + this.mainTransaction = composite.mainTransaction; + this.id = composite.register(this); + } + + public void checkpoint() + { + // don't do anything, composite will checkpoint at end + } + + private RuntimeException earlyOpenUnsupported() + { + throw new UnsupportedOperationException("PartialLifecycleTransaction does not support early opening of SSTables"); + } + + public void update(SSTableReader reader, boolean original) + { + throwIfCompositeAborted(); + if (original) + throw earlyOpenUnsupported(); + + synchronized (mainTransaction) + { + mainTransaction.update(reader, original); + } + } + + public void update(Collection<SSTableReader> readers, boolean original) + { + throwIfCompositeAborted(); + if (original) + throw earlyOpenUnsupported(); + + synchronized (mainTransaction) + { + mainTransaction.update(readers, original); + } + } + + public SSTableReader current(SSTableReader reader) + { + synchronized (mainTransaction) + { + return mainTransaction.current(reader); + } + } + + public void obsolete(SSTableReader reader) + { + earlyOpenUnsupported(); + } + + public void obsoleteOriginals() + { + composite.requestObsoleteOriginals(); + } + + public Set<SSTableReader> originals() + { + return mainTransaction.originals(); + } + + public boolean isObsolete(SSTableReader reader) + { + throw earlyOpenUnsupported(); + } + + private boolean markCommittedOrAborted() + { + return committedOrAborted.compareAndSet(false, true); + } + + /// Commit the transaction part. Because this is a part of a composite transaction, the actual commit will be + /// carried out only after all parts have committed. + /// + /// + public Throwable commit(Throwable accumulate) + { + Throwables.maybeFail(accumulate); // we must be called with a null accumulate + if (markCommittedOrAborted()) + composite.commitPart(); + else + throw new IllegalStateException("Partial transaction already committed or aborted."); + return null; + } + + public Throwable abort(Throwable accumulate) + { + Throwables.maybeFail(accumulate); // we must be called with a null accumulate + if (markCommittedOrAborted()) + composite.abortPart(); + else + throw new IllegalStateException("Partial transaction already committed or aborted."); + return null; + } + + private void throwIfCompositeAborted() + { + if (composite.wasAborted()) + throw new AbortedException("Transaction aborted, likely by another partial operation."); + } + + public void prepareToCommit() + { + if (committedOrAborted.get()) + throw new IllegalStateException("Partial transaction already committed or aborted."); + + throwIfCompositeAborted(); + // nothing else to do, the composite transaction will perform the preparation when all parts are done + } + + public void close() + { + if (markCommittedOrAborted()) // close should abort if not committed + composite.abortPart(); + } + + public void trackNew(SSTable table) + { + throwIfCompositeAborted(); + synchronized (mainTransaction) + { + mainTransaction.trackNew(table); + } + } + + public void untrackNew(SSTable table) + { + synchronized (mainTransaction) + { + mainTransaction.untrackNew(table); + } + } + + public OperationType opType() + { + return mainTransaction.opType(); + } + + public boolean isOffline() + { + return mainTransaction.isOffline(); + } + + @Override + public TimeUUID opId() + { + return id; + } + + @Override + public String opIdString() + { + return String.format("%s (%d/%d)", id, id.sequence(), composite.partsCount()); + } + + @Override + public void cancel(SSTableReader removedSSTable) + { + synchronized (mainTransaction) + { + mainTransaction.cancel(removedSSTable); + } + } + + @Override + public String toString() + { + return opIdString(); + } + + public static class AbortedException extends RuntimeException + { + public AbortedException(String message) + { + super(message); + } + } +} diff --git a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java index adc18cd3b510..ea8c17783f75 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java +++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java @@ -62,6 +62,7 @@ import org.apache.cassandra.notifications.TruncationNotification; import org.apache.cassandra.utils.Pair; import org.apache.cassandra.utils.Throwables; +import org.apache.cassandra.utils.TimeUUID; import org.apache.cassandra.utils.concurrent.OpOrder; import static com.google.common.base.Predicates.and; @@ -122,12 +123,20 @@ public LifecycleTransaction tryModify(SSTableReader sstable, OperationType opera * @return a Transaction over the provided sstables if we are able to mark the given @param sstables as compacted, before anyone else */ public LifecycleTransaction tryModify(Iterable<? extends SSTableReader> sstables, OperationType operationType) + { + return tryModify(sstables, operationType, TimeUUID.Generator.nextTimeUUID()); + } + + /** + * @return a Transaction over the provided sstables if we are able to mark the given @param sstables as compacted, before anyone else + */ + public LifecycleTransaction tryModify(Iterable<? extends SSTableReader> sstables, OperationType operationType, TimeUUID operationId) { if (Iterables.isEmpty(sstables)) - return new LifecycleTransaction(this, operationType, sstables); + return new LifecycleTransaction(this, operationType, sstables, operationId); if (null == apply(permitCompacting(sstables), updateCompacting(emptySet(), sstables))) return null; - return new LifecycleTransaction(this, operationType, sstables); + return new LifecycleTransaction(this, operationType, sstables, operationId); } diff --git a/src/java/org/apache/cassandra/db/lifecycle/WrappedLifecycleTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/WrappedLifecycleTransaction.java index 12c46a9573ca..74d0a75670cd 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/WrappedLifecycleTransaction.java +++ b/src/java/org/apache/cassandra/db/lifecycle/WrappedLifecycleTransaction.java @@ -24,6 +24,7 @@ import org.apache.cassandra.db.compaction.OperationType; import org.apache.cassandra.io.sstable.SSTable; import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.utils.TimeUUID; public class WrappedLifecycleTransaction implements ILifecycleTransaction { @@ -113,4 +114,14 @@ public boolean isOffline() { return delegate.isOffline(); } + + public TimeUUID opId() + { + return delegate.opId(); + } + + public void cancel(SSTableReader removedSSTable) + { + delegate.cancel(removedSSTable); + } } diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java index 31df8b112005..1b2c9c03a07d 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java @@ -99,7 +99,12 @@ public static SSTableRewriter constructWithoutEarlyOpening(ILifecycleTransaction public static SSTableRewriter construct(ColumnFamilyStore cfs, ILifecycleTransaction transaction, boolean keepOriginals, long maxAge) { - return new SSTableRewriter(transaction, maxAge, calculateOpenInterval(cfs.supportsEarlyOpen()), keepOriginals, true); + return construct(cfs, transaction, keepOriginals, maxAge, true); + } + + public static SSTableRewriter construct(ColumnFamilyStore cfs, ILifecycleTransaction transaction, boolean keepOriginals, long maxAge, boolean earlyOpenAllowed) + { + return new SSTableRewriter(transaction, maxAge, calculateOpenInterval(earlyOpenAllowed && cfs.supportsEarlyOpen()), keepOriginals, true); } private static long calculateOpenInterval(boolean shouldOpenEarly) diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java index fe99b8430d01..8e2db41f6cbf 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java @@ -1019,7 +1019,8 @@ public ISSTableScanner getScanner(Range<Token> range) { if (range == null) return getScanner(); - return getScanner(Collections.singletonList(range)); + else + return getScanner(Collections.singletonList(range)); } /** diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 47bb83a5ec00..28d9d124e92c 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -2689,6 +2689,14 @@ public void forceKeyspaceCompaction(boolean splitOutput, String keyspaceName, St } } + public void forceKeyspaceCompaction(boolean splitOutput, int parallelism, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException + { + for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, tableNames)) + { + cfStore.forceMajorCompaction(splitOutput, parallelism); + } + } + public int relocateSSTables(String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException { return relocateSSTables(0, keyspaceName, tableNames); diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java index 1529a59a56e1..46ecc0e05aac 100644 --- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java +++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java @@ -375,6 +375,11 @@ public interface StorageServiceMBean extends NotificationEmitter */ public void forceKeyspaceCompaction(boolean splitOutput, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException; + /** + * Forces major compaction of a single keyspace with the given parallelism limit + */ + public void forceKeyspaceCompaction(boolean splitOutput, int parallelism, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException; + /** @deprecated See CASSANDRA-11179 */ @Deprecated(since = "3.5") public int relocateSSTables(String keyspace, String ... cfnames) throws IOException, ExecutionException, InterruptedException; diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index b2951a49b055..8191bb73ecc3 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -469,6 +469,11 @@ public void forceUserDefinedCompaction(String datafiles) throws IOException, Exe compactionProxy.forceUserDefinedCompaction(datafiles); } + public void forceKeyspaceCompaction(boolean splitOutput, int parallelism, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException + { + ssProxy.forceKeyspaceCompaction(splitOutput, parallelism, keyspaceName, tableNames); + } + public void forceKeyspaceCompaction(boolean splitOutput, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException { ssProxy.forceKeyspaceCompaction(splitOutput, keyspaceName, tableNames); diff --git a/src/java/org/apache/cassandra/tools/nodetool/Compact.java b/src/java/org/apache/cassandra/tools/nodetool/Compact.java index f5a83ed90475..1f8b4c4b46e6 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/Compact.java +++ b/src/java/org/apache/cassandra/tools/nodetool/Compact.java @@ -50,6 +50,12 @@ public class Compact extends NodeToolCmd @Option(title = "partition_key", name = {"--partition"}, description = "String representation of the partition key") private String partitionKey = EMPTY; + @Option(title = "jobs", + name = {"-j", "--jobs"}, + description = "Use -j to specify the maximum number of threads to use for parallel compaction. " + + "If not set, up to half the compaction threads will be used. " + + "If set to 0, the major compaction will use all threads and will not permit other compactions to run until it completes (use with caution).") + private Integer parallelism = null; @Override public void execute(NodeProbe probe) @@ -95,7 +101,10 @@ else if (partitionKeyProvided) } else { - probe.forceKeyspaceCompaction(splitOutput, keyspace, tableNames); + if (parallelism != null) + probe.forceKeyspaceCompaction(splitOutput, parallelism, keyspace, tableNames); + else // avoid referring to the new method to work with older versions + probe.forceKeyspaceCompaction(splitOutput, keyspace, tableNames); } } catch (Exception e) { diff --git a/src/java/org/apache/cassandra/utils/Throwables.java b/src/java/org/apache/cassandra/utils/Throwables.java index 3665dfca82d0..242c7688089a 100644 --- a/src/java/org/apache/cassandra/utils/Throwables.java +++ b/src/java/org/apache/cassandra/utils/Throwables.java @@ -333,4 +333,11 @@ public static void assertAnyCause(Throwable err, Class<? extends Throwable> caus if (!anyCauseMatches(err, cause::isInstance)) throw new AssertionError("The exception is not caused by " + cause.getName(), err); } + + @VisibleForTesting + public static void assertAnyCause(Throwable err, Class<? extends Throwable>... causeClasses) + { + if (Arrays.stream(causeClasses).noneMatch(c -> anyCauseMatches(err, c::isInstance))) + throw new AssertionError("The exception is not caused by any of " + Arrays.toString(causeClasses), err); + } } diff --git a/src/java/org/apache/cassandra/utils/TimeUUID.java b/src/java/org/apache/cassandra/utils/TimeUUID.java index d993f171af32..0b17504763ba 100644 --- a/src/java/org/apache/cassandra/utils/TimeUUID.java +++ b/src/java/org/apache/cassandra/utils/TimeUUID.java @@ -293,6 +293,25 @@ public static String toString(TimeUUID ballot, String kind) return ballot == null ? "null" : String.format("%s(%d:%s)", kind, ballot.uuidTimestamp(), ballot); } + public int sequence() + { + return (int) ((lsb >> 48) & 0x0000000000003FFFL); + } + + /** + * Returns a new TimeUUID with the same data as this one, but with the provided sequence value. + * + * <b>Warning:</b> the uniqueness of the returned TimeUUID is not guaranteed by this method. Caller must ensure that + * the sequence numbers in use are distinct. + */ + public TimeUUID withSequence(long sequence) + { + long sequenceBits = 0x0000000000003FFFL; + long sequenceMask = ~(sequenceBits << 48); + final long bits = (sequence & sequenceBits) << 48; + return new TimeUUID(uuidTimestamp, lsb() & sequenceMask | bits); + } + @Override public int compareTo(TimeUUID that) { diff --git a/test/distributed/org/apache/cassandra/distributed/test/PaxosRepair2Test.java b/test/distributed/org/apache/cassandra/distributed/test/PaxosRepair2Test.java index ed066b2c3bf3..8b043ab6aa83 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/PaxosRepair2Test.java +++ b/test/distributed/org/apache/cassandra/distributed/test/PaxosRepair2Test.java @@ -287,7 +287,7 @@ private static void compactPaxos() { ColumnFamilyStore paxos = Keyspace.open(SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.PAXOS); FBUtilities.waitOnFuture(paxos.forceFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS)); - FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(paxos, 0, false)); + FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(paxos, 0, false, 0)); } private static Map<Integer, PaxosRow> getPaxosRows() diff --git a/test/distributed/org/apache/cassandra/distributed/test/PaxosRepairTest.java b/test/distributed/org/apache/cassandra/distributed/test/PaxosRepairTest.java index 8e3db2039f9e..e22a6be3ea1c 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/PaxosRepairTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/PaxosRepairTest.java @@ -597,7 +597,7 @@ private static void compactPaxos() { ColumnFamilyStore paxos = Keyspace.open(SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.PAXOS); FBUtilities.waitOnFuture(paxos.forceFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS)); - FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(paxos, 0, false)); + FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(paxos, 0, false, 0)); } private static Map<Integer, PaxosRow> getPaxosRows() diff --git a/test/distributed/org/apache/cassandra/distributed/test/UpgradeSSTablesTest.java b/test/distributed/org/apache/cassandra/distributed/test/UpgradeSSTablesTest.java index c2caac3e0e12..3ec895222138 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/UpgradeSSTablesTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/UpgradeSSTablesTest.java @@ -83,7 +83,7 @@ public void upgradeSSTablesInterruptsOngoingCompaction() throws Throwable Future<?> future = cluster.get(1).asyncAcceptsOnInstance((String ks) -> { ColumnFamilyStore cfs = Keyspace.open(ks).getColumnFamilyStore("tbl"); - CompactionManager.instance.submitMaximal(cfs, FBUtilities.nowInSeconds(), false, OperationType.COMPACTION); + CompactionManager.instance.submitMaximal(cfs, FBUtilities.nowInSeconds(), false, 1, OperationType.COMPACTION); }).apply(KEYSPACE); Assert.assertTrue(cluster.get(1).callOnInstance(() -> CompactionLatchByteman.starting.awaitUninterruptibly(1, TimeUnit.MINUTES))); @@ -129,7 +129,7 @@ public void compactionDoesNotCancelUpgradeSSTables() throws Throwable cluster.get(1).acceptsOnInstance((String ks) -> { ColumnFamilyStore cfs = Keyspace.open(ks).getColumnFamilyStore("tbl"); - FBUtilities.allOf(CompactionManager.instance.submitMaximal(cfs, FBUtilities.nowInSeconds(), false, OperationType.COMPACTION)) + FBUtilities.allOf(CompactionManager.instance.submitMaximal(cfs, FBUtilities.nowInSeconds(), false, 1, OperationType.COMPACTION)) .awaitUninterruptibly(1, TimeUnit.MINUTES); }).accept(KEYSPACE); diff --git a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java index caf9a359a317..e7043ba5ae8d 100644 --- a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java +++ b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java @@ -200,7 +200,7 @@ private void forceCompactions(ColumnFamilyStore cfs) if (cfs.getLiveSSTables().size() > 1) { - CompactionManager.instance.performMaximal(cfs, false); + CompactionManager.instance.performMaximal(cfs); } } } diff --git a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java index 6c4a8ddb9b71..51500d0bab9f 100644 --- a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java +++ b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java @@ -99,16 +99,19 @@ public void testParallelLeveledCompaction() throws Exception { while (true) { - final AbstractCompactionTask nextTask = lcs.getNextBackgroundTask(Integer.MIN_VALUE); - if (nextTask == null) + final var nextTasks = lcs.getNextBackgroundTasks(Integer.MIN_VALUE); + if (nextTasks == null || nextTasks.isEmpty()) break; - tasks.add(new Runnable() + for (var nextTask : nextTasks) { - public void run() + tasks.add(new Runnable() { - nextTask.execute(ActiveCompactionsTracker.NOOP); - } - }); + public void run() + { + nextTask.execute(ActiveCompactionsTracker.NOOP); + } + }); + } } if (tasks.isEmpty()) break; diff --git a/test/microbench/org/apache/cassandra/test/microbench/CachingBenchTest.java b/test/microbench/org/apache/cassandra/test/microbench/CachingBenchTest.java index 3a0f09fc2a4d..8ffd8150be54 100644 --- a/test/microbench/org/apache/cassandra/test/microbench/CachingBenchTest.java +++ b/test/microbench/org/apache/cassandra/test/microbench/CachingBenchTest.java @@ -262,7 +262,7 @@ public void testSetup(String compactionClass, String compressorClass, DiskAccess String hashesBefore = getHashes(); long startTime = currentTimeMillis(); - CompactionManager.instance.performMaximal(cfs, true); + CompactionManager.instance.performMaximal(cfs, true, 0); long endTime = currentTimeMillis(); int endRowCount = countRows(cfs); diff --git a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBench.java b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBench.java index 7b2bc3afaa29..e10b91d9b2e5 100644 --- a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBench.java +++ b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBench.java @@ -202,7 +202,7 @@ private void generateData() .applyUnsafe(); } store.forceBlockingFlush(ColumnFamilyStore.FlushReason.USER_FORCED); - CompactionManager.instance.performMaximal(store, false); + CompactionManager.instance.performMaximal(store); } @TearDown diff --git a/test/unit/org/apache/cassandra/cql3/GcCompactionTest.java b/test/unit/org/apache/cassandra/cql3/GcCompactionTest.java index 31b303471e9d..7e118e792848 100644 --- a/test/unit/org/apache/cassandra/cql3/GcCompactionTest.java +++ b/test/unit/org/apache/cassandra/cql3/GcCompactionTest.java @@ -232,7 +232,7 @@ public void testGarbageCollectRetainsLCSLevel() throws Throwable assertEquals(0, collected.getSSTableLevel()); // garbagecollect should leave the LCS level where it was - CompactionManager.instance.performMaximal(cfs, false); + CompactionManager.instance.performMaximal(cfs); assertEquals(1, cfs.getLiveSSTables().size()); SSTableReader compacted = cfs.getLiveSSTables().iterator().next(); diff --git a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java index 74a0a4b17f03..b7d142536a53 100644 --- a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java +++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java @@ -188,7 +188,7 @@ public void testDropDuringCompaction() } DatabaseDescriptor.setCompactionThroughputMebibytesPerSec(1); - List<? extends Future<?>> futures = CompactionManager.instance.submitMaximal(cfs, CompactionManager.getDefaultGcBefore(cfs, FBUtilities.nowInSeconds()), false); + List<? extends Future<?>> futures = CompactionManager.instance.submitMaximal(cfs, CompactionManager.getDefaultGcBefore(cfs, FBUtilities.nowInSeconds()), false, 1); execute("DROP TABLE %s"); try diff --git a/test/unit/org/apache/cassandra/db/KeyspaceTest.java b/test/unit/org/apache/cassandra/db/KeyspaceTest.java index f78d081dcee6..a42359de461c 100644 --- a/test/unit/org/apache/cassandra/db/KeyspaceTest.java +++ b/test/unit/org/apache/cassandra/db/KeyspaceTest.java @@ -419,7 +419,7 @@ public void testGetSliceFromLarge() throws Throwable // compact so we have a big row with more than the minimum index count if (cfs.getLiveSSTables().size() > 1) - CompactionManager.instance.performMaximal(cfs, false); + CompactionManager.instance.performMaximal(cfs); // verify that we do indeed have multiple index entries SSTableReader sstable = cfs.getLiveSSTables().iterator().next(); diff --git a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java index 2fee75877949..2eb521c68804 100644 --- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java +++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java @@ -421,7 +421,7 @@ public void overlappingRangeTest() throws Exception partition.getRow(Clustering.make(bb(i))).hasLiveData(nowInSec, enforceStrictLiveness)); // Compact everything and re-test - CompactionManager.instance.performMaximal(cfs, false); + CompactionManager.instance.performMaximal(cfs); partition = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).build()); for (int i = 0; i < 5; i++) @@ -515,7 +515,7 @@ public void testRowWithRangeTombstonesUpdatesSecondaryIndex() throws Exception assertEquals(10, index.rowsInserted.size()); - CompactionManager.instance.performMaximal(cfs, false); + CompactionManager.instance.performMaximal(cfs); // compacted down to single sstable assertEquals(1, cfs.getLiveSSTables().size()); @@ -547,7 +547,7 @@ public void testRangeTombstoneCompaction() throws Exception assertEquals(2, cfs.getLiveSSTables().size()); // compact down to single sstable - CompactionManager.instance.performMaximal(cfs, false); + CompactionManager.instance.performMaximal(cfs); assertEquals(1, cfs.getLiveSSTables().size()); // test the physical structure of the sstable i.e. rt & columns on disk diff --git a/test/unit/org/apache/cassandra/db/compaction/AbstractCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/AbstractCompactionStrategyTest.java index 1fc43bdcc3d1..043778092e6e 100644 --- a/test/unit/org/apache/cassandra/db/compaction/AbstractCompactionStrategyTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/AbstractCompactionStrategyTest.java @@ -20,6 +20,7 @@ import java.util.Collections; +import com.google.common.collect.Iterables; import org.junit.After; import org.junit.BeforeClass; import org.junit.Test; @@ -31,7 +32,7 @@ import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.RowUpdateBuilder; -import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.lifecycle.ILifecycleTransaction; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.schema.CompactionParams; import org.apache.cassandra.schema.KeyspaceParams; @@ -100,7 +101,8 @@ public void testGetNextBackgroundTaskDoesNotBlock(String table) } // Check they are returned on the next background task - try (LifecycleTransaction txn = strategy.getNextBackgroundTask(FBUtilities.nowInSeconds()).transaction) + long gcBefore1 = FBUtilities.nowInSeconds(); + try (ILifecycleTransaction txn = Iterables.getOnlyElement(strategy.getNextBackgroundTasks(gcBefore1), null).transaction) { Assert.assertEquals(cfs.getLiveSSTables(), txn.originals()); } @@ -109,7 +111,8 @@ public void testGetNextBackgroundTaskDoesNotBlock(String table) cfs.getTracker().removeUnsafe(cfs.getLiveSSTables()); // verify the compaction strategy will return null - Assert.assertNull(strategy.getNextBackgroundTask(FBUtilities.nowInSeconds())); + long gcBefore = FBUtilities.nowInSeconds(); + Assert.assertNull(Iterables.<AbstractCompactionTask>getOnlyElement(strategy.getNextBackgroundTasks(gcBefore), null)); } diff --git a/test/unit/org/apache/cassandra/db/compaction/CancelCompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/CancelCompactionsTest.java index b3a182f18634..757ad1c08670 100644 --- a/test/unit/org/apache/cassandra/db/compaction/CancelCompactionsTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/CancelCompactionsTest.java @@ -31,6 +31,7 @@ import java.util.stream.Collectors; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; import com.google.common.util.concurrent.Uninterruptibles; import org.junit.Assume; import org.junit.Test; @@ -485,7 +486,7 @@ public void testStandardCompactionTaskCancellation() throws Throwable { for (AbstractCompactionStrategy cs : css) { - ct = cs.getNextBackgroundTask(0); + ct = Iterables.getOnlyElement(cs.getNextBackgroundTasks(0), null); if (ct != null) break; } diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionControllerTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionControllerTest.java index 1f15529aa296..f46150cffced 100644 --- a/test/unit/org/apache/cassandra/db/compaction/CompactionControllerTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/CompactionControllerTest.java @@ -184,19 +184,19 @@ public void testGetFullyExpiredSSTables() // the first sstable should be expired because the overlapping sstable is newer and the gc period is later long gcBefore = (System.currentTimeMillis() / 1000) + 5; - Set<SSTableReader> expired = CompactionController.getFullyExpiredSSTables(cfs, compacting, overlapping, gcBefore); + Set<SSTableReader> expired = CompactionController.getFullyExpiredSSTables(cfs, compacting, x -> overlapping, gcBefore); assertNotNull(expired); assertEquals(1, expired.size()); assertEquals(compacting.iterator().next(), expired.iterator().next()); // however if we add an older mutation to the memtable then the sstable should not be expired applyMutation(cfs.metadata(), key, timestamp3); - expired = CompactionController.getFullyExpiredSSTables(cfs, compacting, overlapping, gcBefore); + expired = CompactionController.getFullyExpiredSSTables(cfs, compacting, x -> overlapping, gcBefore); assertNotNull(expired); assertEquals(0, expired.size()); // Now if we explicitly ask to ignore overlaped sstables, we should get back our expired sstable - expired = CompactionController.getFullyExpiredSSTables(cfs, compacting, overlapping, gcBefore, true); + expired = CompactionController.getFullyExpiredSSTables(cfs, compacting, x -> overlapping, gcBefore, true); assertNotNull(expired); assertEquals(1, expired.size()); } diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerPendingRepairTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerPendingRepairTest.java index 483b77a29a58..c8eca664bd20 100644 --- a/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerPendingRepairTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerPendingRepairTest.java @@ -253,7 +253,7 @@ public void cleanupCompactionFinalized() throws NoSuchRepairSessionException Assert.assertFalse(sstable.isRepaired()); cfs.getCompactionStrategyManager().enable(); // enable compaction to fetch next background task - AbstractCompactionTask compactionTask = csm.getNextBackgroundTask(FBUtilities.nowInSeconds()); + AbstractCompactionTask compactionTask = Iterables.getOnlyElement(csm.getNextBackgroundTasks(FBUtilities.nowInSeconds()), null); Assert.assertNotNull(compactionTask); Assert.assertSame(PendingRepairManager.RepairFinishedCompactionTask.class, compactionTask.getClass()); @@ -294,7 +294,7 @@ public void cleanupCompactionFailed() Assert.assertFalse(sstable.isRepaired()); cfs.getCompactionStrategyManager().enable(); // enable compaction to fetch next background task - AbstractCompactionTask compactionTask = csm.getNextBackgroundTask(FBUtilities.nowInSeconds()); + AbstractCompactionTask compactionTask = Iterables.getOnlyElement(csm.getNextBackgroundTasks(FBUtilities.nowInSeconds()), null); Assert.assertNotNull(compactionTask); Assert.assertSame(PendingRepairManager.RepairFinishedCompactionTask.class, compactionTask.getClass()); @@ -348,7 +348,7 @@ public void testFinalizedAndCompactionRace() throws NoSuchRepairSessionException cfs.getCompactionStrategyManager().enable(); for (SSTableReader sstable : sstables) pendingContains(sstable); - AbstractCompactionTask compactionTask = csm.getNextBackgroundTask(FBUtilities.nowInSeconds()); + AbstractCompactionTask compactionTask = Iterables.getOnlyElement(csm.getNextBackgroundTasks(FBUtilities.nowInSeconds()), null); // Finalize the repair session LocalSessionAccessor.finalizeUnsafe(repairID); @@ -374,13 +374,13 @@ public void testFinalizedAndCompactionRace() throws NoSuchRepairSessionException System.out.println("*********************************************************************************************"); // Run compaction again. It should pick up the pending repair sstable - compactionTask = csm.getNextBackgroundTask(FBUtilities.nowInSeconds()); + compactionTask = Iterables.getOnlyElement(csm.getNextBackgroundTasks(FBUtilities.nowInSeconds()), null); if (compactionTask != null) { Assert.assertSame(PendingRepairManager.RepairFinishedCompactionTask.class, compactionTask.getClass()); compactionTask.execute(ActiveCompactionsTracker.NOOP); - while ((compactionTask = csm.getNextBackgroundTask(FBUtilities.nowInSeconds())) != null) + while ((compactionTask = Iterables.getOnlyElement(csm.getNextBackgroundTasks(FBUtilities.nowInSeconds()), null)) != null) compactionTask.execute(ActiveCompactionsTracker.NOOP); } @@ -388,7 +388,7 @@ public void testFinalizedAndCompactionRace() throws NoSuchRepairSessionException Util.spinAssertEquals(Boolean.FALSE, () -> { AbstractCompactionTask ctask; - while ((ctask = csm.getNextBackgroundTask(FBUtilities.nowInSeconds())) != null) + while ((ctask = Iterables.getOnlyElement(csm.getNextBackgroundTasks(FBUtilities.nowInSeconds()), null)) != null) ctask.execute(ActiveCompactionsTracker.NOOP); return hasPendingStrategiesFor(repairID); @@ -434,7 +434,7 @@ public void finalizedSessionTransientCleanup() Assert.assertFalse(unrepairedContains(sstable)); cfs.getCompactionStrategyManager().enable(); // enable compaction to fetch next background task - AbstractCompactionTask compactionTask = csm.getNextBackgroundTask(FBUtilities.nowInSeconds()); + AbstractCompactionTask compactionTask = Iterables.getOnlyElement(csm.getNextBackgroundTasks(FBUtilities.nowInSeconds()), null); Assert.assertNotNull(compactionTask); Assert.assertSame(PendingRepairManager.RepairFinishedCompactionTask.class, compactionTask.getClass()); @@ -465,7 +465,7 @@ public void failedSessionTransientCleanup() Assert.assertFalse(unrepairedContains(sstable)); cfs.getCompactionStrategyManager().enable(); // enable compaction to fetch next background task - AbstractCompactionTask compactionTask = csm.getNextBackgroundTask(FBUtilities.nowInSeconds()); + AbstractCompactionTask compactionTask = Iterables.getOnlyElement(csm.getNextBackgroundTasks(FBUtilities.nowInSeconds()), null); Assert.assertNotNull(compactionTask); Assert.assertSame(PendingRepairManager.RepairFinishedCompactionTask.class, compactionTask.getClass()); diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java index 066aa92067f1..8109e40dfe75 100644 --- a/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java @@ -213,13 +213,13 @@ public void testOfflineCompaction() try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.COMPACTION, sstables)) { - Assert.assertEquals(4, txn.tracker.getView().liveSSTables().size()); + Assert.assertEquals(4, txn.tracker().getView().liveSSTables().size()); CompactionTask task = new CompactionTask(cfs, txn, 1000); task.execute(null); // Check that new SSTable was not released - Assert.assertEquals(1, txn.tracker.getView().liveSSTables().size()); - SSTableReader newSSTable = txn.tracker.getView().liveSSTables().iterator().next(); + Assert.assertEquals(1, txn.tracker().getView().liveSSTables().size()); + SSTableReader newSSTable = txn.tracker().getView().liveSSTables().iterator().next(); Assert.assertNotNull(newSSTable.tryRef()); } finally @@ -233,7 +233,7 @@ public void testOfflineCompaction() public void testMajorCompactTask() { //major compact without range/pk specified - CompactionTasks compactionTasks = cfs.getCompactionStrategyManager().getMaximalTasks(Integer.MAX_VALUE, false, OperationType.MAJOR_COMPACTION); + CompactionTasks compactionTasks = cfs.getCompactionStrategyManager().getMaximalTasks(Integer.MAX_VALUE, false, Integer.MAX_VALUE, OperationType.MAJOR_COMPACTION); Assert.assertTrue(compactionTasks.stream().allMatch(task -> task.compactionType.equals(OperationType.MAJOR_COMPACTION))); } } diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java index ca9f5bbcd16a..ce0422217281 100644 --- a/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java @@ -17,36 +17,17 @@ */ package org.apache.cassandra.db.compaction; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.file.FileStore; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.Set; - -import org.apache.commons.lang3.StringUtils; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - +import com.google.common.collect.Iterables; import org.apache.cassandra.Util; import org.apache.cassandra.config.Config; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.cql3.UntypedResultSet; -import org.apache.cassandra.db.ColumnFamilyStore; -import org.apache.cassandra.db.DecoratedKey; -import org.apache.cassandra.db.Directories; -import org.apache.cassandra.db.Keyspace; -import org.apache.cassandra.db.RowUpdateBuilder; +import org.apache.cassandra.db.*; import org.apache.cassandra.db.commitlog.CommitLog; import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter; import org.apache.cassandra.db.compaction.writers.MaxSSTableSizeWriter; +import org.apache.cassandra.db.lifecycle.ILifecycleTransaction; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.db.rows.Cell; import org.apache.cassandra.db.rows.Row; @@ -61,14 +42,20 @@ import org.apache.cassandra.io.util.PathUtils; import org.apache.cassandra.schema.CompactionParams; import org.apache.cassandra.serializers.MarshalException; +import org.apache.commons.lang3.StringUtils; import org.assertj.core.api.Assertions; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.file.FileStore; +import java.util.*; import static org.apache.cassandra.utils.TimeUUID.Generator.nextTimeUUID; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.Assert.*; public class CompactionsCQLTest extends CQLTester { @@ -491,7 +478,7 @@ public void testLCSThresholdParams() throws Throwable } assertEquals(50, cfs.getLiveSSTables().size()); LeveledCompactionStrategy lcs = (LeveledCompactionStrategy) cfs.getCompactionStrategyManager().getUnrepairedUnsafe().first(); - AbstractCompactionTask act = lcs.getNextBackgroundTask(0); + AbstractCompactionTask act = Iterables.getOnlyElement(lcs.getNextBackgroundTasks(0), null); // we should be compacting all 50 sstables: assertEquals(50, act.transaction.originals().size()); act.execute(ActiveCompactionsTracker.NOOP); @@ -525,7 +512,7 @@ public void testSTCSinL0() throws Throwable // mark the L1 sstable as compacting to make sure we trigger STCS in L0: LifecycleTransaction txn = cfs.getTracker().tryModify(l1sstable, OperationType.COMPACTION); LeveledCompactionStrategy lcs = (LeveledCompactionStrategy) cfs.getCompactionStrategyManager().getUnrepairedUnsafe().first(); - AbstractCompactionTask act = lcs.getNextBackgroundTask(0); + AbstractCompactionTask act = Iterables.getOnlyElement(lcs.getNextBackgroundTasks(0), null); // note that max_threshold is 60 (more than the amount of L0 sstables), but MAX_COMPACTING_L0 is 32, which means we will trigger STCS with at most max_threshold sstables assertEquals(50, act.transaction.originals().size()); assertEquals(0, ((LeveledCompactionTask)act).getLevel()); @@ -558,7 +545,7 @@ public void testAbortNotifications() throws Throwable LeveledCompactionTask lcsTask; while (true) { - lcsTask = (LeveledCompactionTask) lcs.getNextBackgroundTask(0); + lcsTask = (LeveledCompactionTask) Iterables.getOnlyElement(lcs.getNextBackgroundTasks(0), null); if (lcsTask != null) { lcsTask.execute(CompactionManager.instance.active); @@ -595,7 +582,7 @@ public void testAbortNotifications() throws Throwable // sstables have been removed. try { - AbstractCompactionTask task = new NotifyingCompactionTask((LeveledCompactionTask) lcs.getNextBackgroundTask(0)); + AbstractCompactionTask task = new NotifyingCompactionTask((LeveledCompactionTask) Iterables.getOnlyElement(lcs.getNextBackgroundTasks(0), null)); task.execute(CompactionManager.instance.active); fail("task should throw exception"); } @@ -604,7 +591,7 @@ public void testAbortNotifications() throws Throwable // ignored } - lcsTask = (LeveledCompactionTask) lcs.getNextBackgroundTask(0); + lcsTask = (LeveledCompactionTask) Iterables.getOnlyElement(lcs.getNextBackgroundTasks(0), null); try { assertNotNull(lcsTask); @@ -626,7 +613,7 @@ public NotifyingCompactionTask(LeveledCompactionTask task) @Override public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Directories directories, - LifecycleTransaction txn, + ILifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables) { return new MaxSSTableSizeWriter(cfs, directories, txn, nonExpiredSSTables, 1 << 20, 1) diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java index 4f1b639d7081..9916bffa7451 100644 --- a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java @@ -120,7 +120,7 @@ public void testMajorCompactionPurge() Util.flush(cfs); // major compact and test that all columns but the resurrected one is completely gone - FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Integer.MAX_VALUE, false)); + FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Long.MAX_VALUE, false, 0)); cfs.invalidateCachedPartition(dk(key)); ImmutableBTreePartition partition = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).build()); @@ -156,7 +156,7 @@ public void testMajorCompactionPurgeTombstonesWithMaxTimestamp() Util.flush(cfs); // major compact - tombstones should be purged - FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Integer.MAX_VALUE, false)); + FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Long.MAX_VALUE, false, 0)); // resurrect one column RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 2, key); @@ -200,7 +200,7 @@ public void testMajorCompactionPurgeTopLevelTombstoneWithMaxTimestamp() Util.flush(cfs); // major compact - tombstones should be purged - FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Integer.MAX_VALUE, false)); + FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Long.MAX_VALUE, false, 0)); // resurrect one column RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 2, key); @@ -242,7 +242,7 @@ public void testMajorCompactionPurgeRangeTombstoneWithMaxTimestamp() Util.flush(cfs); // major compact - tombstones should be purged - FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Integer.MAX_VALUE, false)); + FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Long.MAX_VALUE, false, 0)); // resurrect one column RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 2, key); @@ -519,7 +519,7 @@ public void testRowTombstoneObservedBeforePurging() assertEquals(0, result.size()); // compact the two sstables with a gcBefore that does *not* allow the row tombstone to be purged - FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, (int) (System.currentTimeMillis() / 1000) - 10000, false)); + FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, (int) (System.currentTimeMillis() / 1000) - 10000, false, 0)); // the data should be gone, but the tombstone should still exist assertEquals(1, cfs.getLiveSSTables().size()); @@ -539,7 +539,7 @@ public void testRowTombstoneObservedBeforePurging() Util.flush(cfs); // compact the two sstables with a gcBefore that *does* allow the row tombstone to be purged - FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, (int) (System.currentTimeMillis() / 1000) + 10000, false)); + FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, (int) (System.currentTimeMillis() / 1000) + 10000, false, 0)); // both the data and the tombstone should be gone this time assertEquals(0, cfs.getLiveSSTables().size()); diff --git a/test/unit/org/apache/cassandra/db/compaction/CorruptedSSTablesCompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/CorruptedSSTablesCompactionsTest.java index dc78276254f6..1705256c76a6 100644 --- a/test/unit/org/apache/cassandra/db/compaction/CorruptedSSTablesCompactionsTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/CorruptedSSTablesCompactionsTest.java @@ -23,34 +23,47 @@ import java.nio.ByteBuffer; import java.nio.channels.FileChannel; -import java.util.*; - +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Random; +import java.util.Set; + +import com.google.common.collect.ImmutableMap; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.cassandra.utils.Clock.Global.nanoTime; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; - import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.Util; import org.apache.cassandra.cache.ChunkCache; -import org.apache.cassandra.config.*; -import org.apache.cassandra.db.*; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.RowUpdateBuilder; +import org.apache.cassandra.db.lifecycle.PartialLifecycleTransaction; import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.io.sstable.CorruptSSTableException; +import org.apache.cassandra.io.sstable.ISSTableScanner; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.FileUtils; -import org.apache.cassandra.schema.*; +import org.apache.cassandra.schema.CompactionParams; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.utils.Throwables; +import static org.apache.cassandra.utils.Clock.Global.nanoTime; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; public class CorruptedSSTablesCompactionsTest @@ -63,6 +76,7 @@ public class CorruptedSSTablesCompactionsTest private static final String STANDARD_STCS = "Standard_STCS"; private static final String STANDARD_LCS = "Standard_LCS"; private static final String STANDARD_UCS = "Standard_UCS"; + private static final String STANDARD_UCS_PARALLEL = "Standard_UCS_Parallel"; private static int maxValueSize; @After @@ -77,6 +91,9 @@ public void leakDetect() throws InterruptedException @BeforeClass public static void defineSchema() throws ConfigurationException { + DatabaseDescriptor.daemonInitialization(); // because of all the static initialization in CFS + DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); + long seed = nanoTime(); //long seed = 754271160974509L; // CASSANDRA-9530: use this seed to reproduce compaction failures if reading empty rows @@ -91,7 +108,8 @@ public static void defineSchema() throws ConfigurationException KeyspaceParams.simple(1), makeTable(STANDARD_STCS).compaction(CompactionParams.stcs(Collections.emptyMap())), makeTable(STANDARD_LCS).compaction(CompactionParams.lcs(Collections.emptyMap())), - makeTable(STANDARD_UCS).compaction(CompactionParams.ucs(Collections.emptyMap()))); + makeTable(STANDARD_UCS).compaction(CompactionParams.ucs(Collections.emptyMap())), + makeTable(STANDARD_UCS_PARALLEL).compaction(CompactionParams.ucs(new HashMap<>(ImmutableMap.of("min_sstable_size", "1KiB"))))); maxValueSize = DatabaseDescriptor.getMaxValueSize(); DatabaseDescriptor.setMaxValueSize(1024 * 1024); @@ -140,6 +158,12 @@ public void testCorruptedSSTablesWithUnifiedCompactionStrategy() throws Exceptio testCorruptedSSTables(STANDARD_UCS); } + @Test + public void testCorruptedSSTablesWithUnifiedCompactionStrategyParallelized() throws Exception + { + testCorruptedSSTables(STANDARD_UCS_PARALLEL); + } + public void testCorruptedSSTables(String tableName) throws Exception { @@ -189,30 +213,32 @@ public void testCorruptedSSTables(String tableName) throws Exception if (currentSSTable + 1 > SSTABLES_TO_CORRUPT) break; - FileChannel fc = null; - - try + do { - int corruptionSize = 100; - fc = new File(sstable.getFilename()).newReadWriteChannel(); - assertNotNull(fc); - assertTrue(fc.size() > corruptionSize); - long pos = random.nextInt((int)(fc.size() - corruptionSize)); - logger.info("Corrupting sstable {} [{}] at pos {} / {}", currentSSTable, sstable.getFilename(), pos, fc.size()); - fc.position(pos); - // We want to write something large enough that the corruption cannot get undetected - // (even without compression) - byte[] corruption = new byte[corruptionSize]; - random.nextBytes(corruption); - fc.write(ByteBuffer.wrap(corruption)); + FileChannel fc = null; + try + { + int corruptionSize = 25; + fc = new File(sstable.getFilename()).newReadWriteChannel(); + assertNotNull(fc); + assertTrue(fc.size() > corruptionSize); + long pos = random.nextInt((int) (fc.size() - corruptionSize)); + logger.info("Corrupting sstable {} [{}] at pos {} / {}", currentSSTable, sstable.getFilename(), pos, fc.size()); + fc.position(pos); + // We want to write something large enough that the corruption cannot get undetected + // (even without compression) + byte[] corruption = new byte[corruptionSize]; + random.nextBytes(corruption); + fc.write(ByteBuffer.wrap(corruption)); + } + finally + { + FileUtils.closeQuietly(fc); + } if (ChunkCache.instance != null) ChunkCache.instance.invalidateFile(sstable.getFilename()); - - } - finally - { - FileUtils.closeQuietly(fc); } + while (readsWithoutError(sstable)); currentSSTable++; } @@ -231,12 +257,38 @@ public void testCorruptedSSTables(String tableName) throws Exception { // This is the expected path. The SSTable should be marked corrupted, and retrying the compaction // should move on to the next corruption. - Throwables.assertAnyCause(e, CorruptSSTableException.class); + Throwables.assertAnyCause(e, CorruptSSTableException.class, PartialLifecycleTransaction.AbortedException.class); failures++; } } cfs.truncateBlocking(); - assertEquals(SSTABLES_TO_CORRUPT, failures); + if (tableName != STANDARD_UCS_PARALLEL) + assertEquals(SSTABLES_TO_CORRUPT, failures); + else + { + // Since we proceed in parallel, we can mark more than one SSTable as corrupted in an iteration. + assertTrue(failures > 0 && failures <= SSTABLES_TO_CORRUPT); + } + } + + private boolean readsWithoutError(SSTableReader sstable) + { + try + { + ISSTableScanner scanner = sstable.getScanner(); + while (scanner.hasNext()) + { + UnfilteredRowIterator iter = scanner.next(); + while (iter.hasNext()) + iter.next(); + } + return true; + } + catch (Throwable t) + { + sstable.unmarkSuspect(); + return false; + } } } diff --git a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java index d56003ae2a4a..a7166bad5f47 100644 --- a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java @@ -593,7 +593,7 @@ private int getTaskLevel(ColumnFamilyStore cfs) { for (AbstractCompactionStrategy strategy : strategies) { - AbstractCompactionTask task = strategy.getNextBackgroundTask(0); + AbstractCompactionTask task = Iterables.getOnlyElement(strategy.getNextBackgroundTasks(0), null); if (task != null) { try @@ -922,7 +922,7 @@ public void testReduceScopeL0L1() throws IOException l0sstables.add(MockSchema.sstable(i, (i + 1) * 1024 * 1024, cfs)); try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.COMPACTION, Iterables.concat(l0sstables, l1sstables))) { - Set<SSTableReader> nonExpired = Sets.difference(txn.originals(), Collections.emptySet()); + Set<SSTableReader> nonExpired = new HashSet<>(Sets.difference(txn.originals(), Collections.emptySet())); CompactionTask task = new LeveledCompactionTask(cfs, txn, 1, 0, 1024*1024, false); SSTableReader lastRemoved = null; boolean removed = true; @@ -974,7 +974,8 @@ public void testReduceScopeL0() for (int i = 0; i < l0sstables.size(); i++) { Set<SSTableReader> before = new HashSet<>(txn.originals()); - removed = task.reduceScopeForLimitedSpace(before, 0); + Set<SSTableReader> sources = new HashSet<>(before); + removed = task.reduceScopeForLimitedSpace(sources, 0); SSTableReader removedSSTable = Sets.difference(before, txn.originals()).stream().findFirst().orElse(null); if (removed) { diff --git a/test/unit/org/apache/cassandra/db/compaction/PartialCompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/PartialCompactionsTest.java index d9fcccbaa7eb..1d877904a25e 100644 --- a/test/unit/org/apache/cassandra/db/compaction/PartialCompactionsTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/PartialCompactionsTest.java @@ -98,7 +98,7 @@ public void shouldNotResurrectDataFromSSTableExcludedDueToInsufficientSpace() LimitableDataDirectory.setAvailableSpace(cfs, enoughSpaceForAllButTheLargestSSTable(cfs)); // when - run a compaction where all tombstones have timed out - FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Integer.MAX_VALUE, false)); + FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Integer.MAX_VALUE, false, 0)); // then - the tombstones should not be removed assertEquals("live sstables after compaction", 2, cfs.getLiveSSTables().size()); diff --git a/test/unit/org/apache/cassandra/db/compaction/PendingRepairManagerTest.java b/test/unit/org/apache/cassandra/db/compaction/PendingRepairManagerTest.java index d2b3693032fc..8c5535c1eb6c 100644 --- a/test/unit/org/apache/cassandra/db/compaction/PendingRepairManagerTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/PendingRepairManagerTest.java @@ -21,6 +21,7 @@ import java.util.Collection; import java.util.Collections; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import org.junit.Assert; import org.junit.Test; @@ -152,8 +153,8 @@ public void getNextBackgroundTask() LocalSessionAccessor.finalizeUnsafe(repairID); Assert.assertEquals(2, prm.getSessions().size()); - Assert.assertNull(prm.getNextBackgroundTask(FBUtilities.nowInSeconds())); - AbstractCompactionTask compactionTask = prm.getNextRepairFinishedTask(); + Assert.assertEquals(0, prm.getNextBackgroundTasks(FBUtilities.nowInSeconds()).size()); + AbstractCompactionTask compactionTask = Iterables.getOnlyElement(prm.getNextRepairFinishedTasks()); try { Assert.assertNotNull(compactionTask); @@ -171,7 +172,7 @@ public void getNextBackgroundTask() public void getNextBackgroundTaskNoSessions() { PendingRepairManager prm = csm.getPendingRepairManagers().get(0); - Assert.assertNull(prm.getNextBackgroundTask(FBUtilities.nowInSeconds())); + Assert.assertEquals(0, prm.getNextBackgroundTasks(FBUtilities.nowInSeconds()).size()); } /** @@ -191,7 +192,7 @@ public void getNextBackgroundTaskAllCleanup() throws Exception Assert.assertNotNull(prm.get(repairID)); LocalSessionAccessor.finalizeUnsafe(repairID); - Assert.assertNull(prm.getNextBackgroundTask(FBUtilities.nowInSeconds())); + Assert.assertEquals(0, prm.getNextBackgroundTasks(FBUtilities.nowInSeconds()).size()); } @@ -303,7 +304,7 @@ public void noEmptyCompactionTask() prm.getOrCreate(sstable); cfs.truncateBlocking(); Assert.assertFalse(cfs.getSSTables(SSTableSet.LIVE).iterator().hasNext()); - Assert.assertNull(cfs.getCompactionStrategyManager().getNextBackgroundTask(0)); + Assert.assertEquals(0, cfs.getCompactionStrategyManager().getNextBackgroundTasks(0).size()); } } diff --git a/test/unit/org/apache/cassandra/db/compaction/ShardManagerTest.java b/test/unit/org/apache/cassandra/db/compaction/ShardManagerTest.java index 71e5d8a0641f..ced5d7882144 100644 --- a/test/unit/org/apache/cassandra/db/compaction/ShardManagerTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/ShardManagerTest.java @@ -21,10 +21,13 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashSet; import java.util.List; +import java.util.Set; import java.util.stream.Collectors; import com.google.common.collect.ImmutableList; +import org.apache.cassandra.dht.*; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -37,12 +40,8 @@ import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.DiskBoundaries; import org.apache.cassandra.db.PartitionPosition; -import org.apache.cassandra.dht.IPartitioner; -import org.apache.cassandra.dht.Murmur3Partitioner; -import org.apache.cassandra.dht.Range; -import org.apache.cassandra.dht.Splitter; -import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.utils.Pair; import org.mockito.Mockito; import static org.apache.cassandra.db.ColumnFamilyStore.RING_VERSION_IRRELEVANT; @@ -409,4 +408,149 @@ public void testRangeEnds() } } } + + + @Test + public void testSplitSSTablesInRanges() + { + testSplitSSTablesInRanges(8, ints(1, 2, 4)); + testSplitSSTablesInRanges(4, ints(1, 2, 4)); + testSplitSSTablesInRanges(2, ints(1, 2, 4)); + testSplitSSTablesInRanges(5, ints(1, 2, 4)); + testSplitSSTablesInRanges(5, ints(2, 4, 8)); + testSplitSSTablesInRanges(3, ints(1, 3, 5)); + testSplitSSTablesInRanges(3, ints(3, 3, 3)); + + testSplitSSTablesInRanges(1, ints(1, 2, 3)); + + testSplitSSTablesInRanges(3, ints()); + } + + @Test + public void testSplitSSTablesInRangesMissingParts() + { + // Drop some sstables without losing ranges + testSplitSSTablesInRanges(8, ints(2, 4, 8), + ints(1)); + + testSplitSSTablesInRanges(8, ints(2, 4, 8), + ints(1), ints(0), ints(2, 7)); + + testSplitSSTablesInRanges(5, ints(2, 4, 8), + ints(1), ints(0), ints(2, 7)); + } + + @Test + public void testSplitSSTablesInRangesOneRange() + { + // Drop second half + testSplitSSTablesInRanges(2, ints(2, 4, 8), + ints(1), ints(2, 3), ints(4, 5, 6, 7)); + // Drop all except center, within shard + testSplitSSTablesInRanges(3, ints(5, 7, 9), + ints(0, 1, 3, 4), ints(0, 1, 2, 4, 5, 6), ints(0, 1, 2, 6, 7, 8)); + } + + @Test + public void testSplitSSTablesInRangesSkippedRange() + { + // Drop all sstables containing the 4/8-5/8 range. + testSplitSSTablesInRanges(8, ints(2, 4, 8), + ints(1), ints(2), ints(4)); + // Drop all sstables containing the 4/8-6/8 range. + testSplitSSTablesInRanges(8, ints(2, 4, 8), + ints(1), ints(2), ints(4, 5)); + // Drop all sstables containing the 4/8-8/8 range. + testSplitSSTablesInRanges(8, ints(2, 4, 8), + ints(1), ints(2, 3), ints(4, 5, 6, 7)); + + // Drop all sstables containing the 0/8-2/8 range. + testSplitSSTablesInRanges(5, ints(2, 4, 8), + ints(0), ints(0), ints(0, 1)); + // Drop all sstables containing the 6/8-8/8 range. + testSplitSSTablesInRanges(5, ints(2, 4, 8), + ints(1), ints(3), ints(6, 7)); + // Drop sstables on both ends. + testSplitSSTablesInRanges(5, ints(3, 4, 8), + ints(0, 2), ints(0, 3), ints(0, 1, 6, 7)); + } + + public void testSplitSSTablesInRanges(int numShards, int[] perLevelCounts, int[]... dropsPerLevel) + { + weightedRanges.clear(); + weightedRanges.add(new Splitter.WeightedRange(1.0, new Range<>(minimumToken, minimumToken))); + ShardManager manager = new ShardManagerNoDisks(weightedRanges); + + Set<SSTableReader> allSSTables = new HashSet<>(); + int levelNum = 0; + for (int perLevelCount : perLevelCounts) + { + List<SSTableReader> ssTables = mockNonOverlappingSSTables(perLevelCount); + if (levelNum < dropsPerLevel.length) + { + for (int i = dropsPerLevel[levelNum].length - 1; i >= 0; i--) + ssTables.remove(dropsPerLevel[levelNum][i]); + } + allSSTables.addAll(ssTables); + ++levelNum; + } + + var results = new ArrayList<Pair<Range<Token>, Set<SSTableReader>>>(); + manager.splitSSTablesInShards(allSSTables, numShards, (sstables, range) -> results.add(Pair.create(range, Set.copyOf(sstables)))); + int i = 0; + int[] expectedSSTablesInTasks = new int[results.size()]; + int[] collectedSSTablesPerTask = new int[results.size()]; + for (var t : results) + { + collectedSSTablesPerTask[i] = t.right().size(); + expectedSSTablesInTasks[i] = (int) allSSTables.stream().filter(x -> intersects(x, t.left())).count(); + ++i; + } + Assert.assertEquals(Arrays.toString(expectedSSTablesInTasks), Arrays.toString(collectedSSTablesPerTask)); + System.out.println(Arrays.toString(expectedSSTablesInTasks)); + } + + private boolean intersects(SSTableReader r, Range<Token> range) + { + if (range == null) + return true; + return range.intersects(range(r)); + } + + + private Bounds<Token> range(SSTableReader x) + { + return new Bounds<>(x.getFirst().getToken(), x.getLast().getToken()); + } + + List<SSTableReader> mockNonOverlappingSSTables(int numSSTables) + { + if (!partitioner.splitter().isPresent()) + throw new IllegalStateException(String.format("Cannot split ranges with current partitioner %s", partitioner)); + + ByteBuffer emptyBuffer = ByteBuffer.allocate(0); + + List<SSTableReader> sstables = new ArrayList<>(numSSTables); + for (int i = 0; i < numSSTables; i++) + { + DecoratedKey first = new BufferDecoratedKey(boundary(numSSTables, i).nextValidToken(), emptyBuffer); + DecoratedKey last = new BufferDecoratedKey(boundary(numSSTables, i+1), emptyBuffer); + sstables.add(mockSSTable(first, last)); + } + + return sstables; + } + + private Token boundary(int numSSTables, int i) + { + return partitioner.split(partitioner.getMinimumToken(), partitioner.getMaximumToken(), i * 1.0 / numSSTables); + } + + private SSTableReader mockSSTable(DecoratedKey first, DecoratedKey last) + { + SSTableReader sstable = Mockito.mock(SSTableReader.class); + when(sstable.getFirst()).thenReturn(first); + when(sstable.getLast()).thenReturn(last); + return sstable; + } } diff --git a/test/unit/org/apache/cassandra/db/compaction/SingleSSTableLCSTaskTest.java b/test/unit/org/apache/cassandra/db/compaction/SingleSSTableLCSTaskTest.java index 9e03ec3a9a5e..e14d48eae00f 100644 --- a/test/unit/org/apache/cassandra/db/compaction/SingleSSTableLCSTaskTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/SingleSSTableLCSTaskTest.java @@ -22,6 +22,7 @@ import java.nio.channels.FileChannel; import java.util.Random; +import com.google.common.collect.Iterables; import org.apache.commons.lang3.StringUtils; import org.junit.Test; @@ -101,14 +102,14 @@ private void compactionTestHelper(boolean singleSSTUplevel) throws Throwable } // now we have a bunch of data in L0, first compaction will be a normal one, containing all sstables: LeveledCompactionStrategy lcs = (LeveledCompactionStrategy) cfs.getCompactionStrategyManager().getUnrepairedUnsafe().first(); - AbstractCompactionTask act = lcs.getNextBackgroundTask(0); + AbstractCompactionTask act = Iterables.getOnlyElement(lcs.getNextBackgroundTasks(0), null); act.execute(ActiveCompactionsTracker.NOOP); // now all sstables are laid out non-overlapping in L1, this means that the rest of the compactions // will be single sstable ones, make sure that we use SingleSSTableLCSTask if singleSSTUplevel is true: while (lcs.getEstimatedRemainingTasks() > 0) { - act = lcs.getNextBackgroundTask(0); + act = Iterables.getOnlyElement(lcs.getNextBackgroundTasks(0), null); assertEquals(singleSSTUplevel, act instanceof SingleSSTableLCSTask); act.execute(ActiveCompactionsTracker.NOOP); } diff --git a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java index e7e97bed9988..bf3c508af493 100644 --- a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java @@ -141,7 +141,7 @@ public void testAggressiveFullyExpired() Set<SSTableReader> expired = CompactionController.getFullyExpiredSSTables( cfs, sstables, - Collections.EMPTY_SET, + s -> Collections.EMPTY_SET, gcBefore); assertEquals(2, expired.size()); diff --git a/test/unit/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategyTest.java index 4576db78f7f1..5387732a7147 100644 --- a/test/unit/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategyTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategyTest.java @@ -297,11 +297,13 @@ public void testDropExpiredSSTables() throws InterruptedException twcs.addSSTable(sstable); twcs.startup(); - assertNull(twcs.getNextBackgroundTask(nowInSeconds())); + long gcBefore1 = nowInSeconds(); + assertNull(Iterables.<AbstractCompactionTask>getOnlyElement(twcs.getNextBackgroundTasks(gcBefore1), null)); // Wait for the expiration of the first sstable Thread.sleep(TimeUnit.SECONDS.toMillis(TTL_SECONDS + 1)); - AbstractCompactionTask t = twcs.getNextBackgroundTask(nowInSeconds()); + long gcBefore = nowInSeconds(); + AbstractCompactionTask t = Iterables.getOnlyElement(twcs.getNextBackgroundTasks(gcBefore), null); assertNotNull(t); assertEquals(1, Iterables.size(t.transaction.originals())); SSTableReader sstable = t.transaction.originals().iterator().next(); @@ -352,11 +354,13 @@ public void testDropOverlappingExpiredSSTables() throws InterruptedException twcs.addSSTable(sstable); twcs.startup(); - assertNull(twcs.getNextBackgroundTask(nowInSeconds())); + long gcBefore2 = nowInSeconds(); + assertNull(Iterables.<AbstractCompactionTask>getOnlyElement(twcs.getNextBackgroundTasks(gcBefore2), null)); // Wait for the expiration of the first sstable Thread.sleep(TimeUnit.SECONDS.toMillis(TTL_SECONDS + 1)); - assertNull(twcs.getNextBackgroundTask(nowInSeconds())); + long gcBefore1 = nowInSeconds(); + assertNull(Iterables.<AbstractCompactionTask>getOnlyElement(twcs.getNextBackgroundTasks(gcBefore1), null)); options.put(TimeWindowCompactionStrategyOptions.UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION_KEY, "true"); twcs = new TimeWindowCompactionStrategy(cfs, options); @@ -364,7 +368,8 @@ public void testDropOverlappingExpiredSSTables() throws InterruptedException twcs.addSSTable(sstable); twcs.startup(); - AbstractCompactionTask t = twcs.getNextBackgroundTask(nowInSeconds()); + long gcBefore = nowInSeconds(); + AbstractCompactionTask t = Iterables.getOnlyElement(twcs.getNextBackgroundTasks(gcBefore), null); assertNotNull(t); assertEquals(1, Iterables.size(t.transaction.originals())); SSTableReader sstable = t.transaction.originals().iterator().next(); diff --git a/test/unit/org/apache/cassandra/db/compaction/UnifiedCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/UnifiedCompactionStrategyTest.java index 7b3d85d0b7e2..8bca5e8f80e0 100644 --- a/test/unit/org/apache/cassandra/db/compaction/UnifiedCompactionStrategyTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/UnifiedCompactionStrategyTest.java @@ -20,7 +20,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Collection; +import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -29,9 +29,11 @@ import java.util.Set; import java.util.TreeMap; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import com.google.common.collect.Iterables; import org.apache.commons.math3.random.JDKRandomGenerator; +import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -45,11 +47,15 @@ import org.apache.cassandra.db.DiskBoundaries; import org.apache.cassandra.db.compaction.unified.Controller; import org.apache.cassandra.db.compaction.unified.UnifiedCompactionTask; +import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.lifecycle.PartialLifecycleTransaction; import org.apache.cassandra.db.lifecycle.SSTableSet; import org.apache.cassandra.db.lifecycle.Tracker; import org.apache.cassandra.db.marshal.AsciiType; +import org.apache.cassandra.dht.Bounds; import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Splitter; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.format.SSTableReader; @@ -58,12 +64,15 @@ import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.Overlaps; import org.apache.cassandra.utils.Pair; +import org.apache.cassandra.utils.concurrent.Transactional; import org.mockito.Answers; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.MockitoAnnotations; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertSame; @@ -175,7 +184,8 @@ public void testNoSSTables() UnifiedCompactionStrategy strategy = new UnifiedCompactionStrategy(cfs, new HashMap<>(), controller); - assertNull(strategy.getNextBackgroundTask(FBUtilities.nowInSeconds())); + long gcBefore = FBUtilities.nowInSeconds(); + assertNull(Iterables.getOnlyElement(strategy.getNextBackgroundTasks(gcBefore), null)); assertEquals(0, strategy.getEstimatedRemainingTasks()); } @@ -474,7 +484,7 @@ public void testGetNextBackgroundTasks() strategy.addSSTables(sstables); dataTracker.addInitialSSTables(sstables); - AbstractCompactionTask task = strategy.getNextBackgroundTask(0); + AbstractCompactionTask task = Iterables.getOnlyElement(strategy.getNextBackgroundTasks(0), null); assertSame(UnifiedCompactionTask.class, task.getClass()); task.transaction.abort(); } @@ -656,38 +666,228 @@ public void testPending() } @Test - public void testMaximalSelection() + public void testCreateParallelTasks() + { + testCreateParallelTasks(8, arr(1, 2, 4)); + testCreateParallelTasks(4, arr(1, 2, 4)); + testCreateParallelTasks(2, arr(1, 2, 4)); + testCreateParallelTasks(5, arr(1, 2, 4)); + testCreateParallelTasks(5, arr(2, 4, 8)); + testCreateParallelTasks(3, arr(1, 3, 5)); + testCreateParallelTasks(3, arr(3, 3, 3)); + + testCreateParallelTasks(1, arr(1, 2, 3)); + } + + @Test + public void testCreateParallelTasksMissingParts() + { + // Drop some sstables without losing ranges + testCreateParallelTasks(8, arr(2, 4, 8), + arr(1)); + + testCreateParallelTasks(8, arr(2, 4, 8), + arr(1), arr(0), arr(2, 7)); + + testCreateParallelTasks(5, arr(2, 4, 8), + arr(1), arr(0), arr(2, 7)); + } + + @Test + public void testCreateParallelTasksOneRange() + { + // Drop second half + testCreateParallelTasks(2, arr(2, 4, 8), + arr(1), arr(2, 3), arr(4, 5, 6, 7)); + // Drop all except center, within shard + testCreateParallelTasks(3, arr(5, 7, 9), + arr(0, 1, 3, 4), arr(0, 1, 2, 4, 5, 6), arr(0, 1, 2, 6, 7, 8)); + } + + @Test + public void testCreateParallelTasksSkippedRange() + { + // Drop all sstables containing the 4/8-5/8 range. + testCreateParallelTasks(8, arr(2, 4, 8), + arr(1), arr(2), arr(4)); + // Drop all sstables containing the 4/8-6/8 range. + testCreateParallelTasks(8, arr(2, 4, 8), + arr(1), arr(2), arr(4, 5)); + // Drop all sstables containing the 4/8-8/8 range. + testCreateParallelTasks(8, arr(2, 4, 8), + arr(1), arr(2, 3), arr(4, 5, 6, 7)); + + // Drop all sstables containing the 0/8-2/8 range. + testCreateParallelTasks(5, arr(2, 4, 8), + arr(0), arr(0), arr(0, 1)); + // Drop all sstables containing the 6/8-8/8 range. + testCreateParallelTasks(5, arr(2, 4, 8), + arr(1), arr(3), arr(6, 7)); + // Drop sstables on both ends. + testCreateParallelTasks(5, arr(3, 4, 8), + arr(0, 2), arr(0, 3), arr(0, 1, 6, 7)); + } + + public void testCreateParallelTasks(int numShards, int[] perLevelCounts, int[]... dropsPerLevel) { + // Note: This test has a counterpart in ShardManagerTest that exercises splitSSTablesInShards directly and more thoroughly. + // This one ensures the data is correctly passed to and presented in compaction tasks. + Set<SSTableReader> allSSTables = new HashSet<>(); + int levelNum = 0; + for (int perLevelCount : perLevelCounts) + { + List<SSTableReader> ssTables = mockNonOverlappingSSTables(perLevelCount, levelNum, 100 << (20 + levelNum)); + if (levelNum < dropsPerLevel.length) + { + for (int i = dropsPerLevel[levelNum].length - 1; i >= 0; i--) + ssTables.remove(dropsPerLevel[levelNum][i]); + } + allSSTables.addAll(ssTables); + ++levelNum; + } + dataTracker.addInitialSSTables(allSSTables); + + Controller controller = Mockito.mock(Controller.class); + when(controller.getNumShards(anyDouble())).thenReturn(numShards); + when(controller.parallelizeOutputShards()).thenReturn(true); + UnifiedCompactionStrategy strategy = new UnifiedCompactionStrategy(cfs, new HashMap<>(), controller); + strategy.startup(); + LifecycleTransaction txn = dataTracker.tryModify(allSSTables, OperationType.COMPACTION); + var tasks = strategy.createCompactionTasks(0, txn); + int i = 0; + int[] expectedSSTablesInTasks = new int[tasks.size()]; + int[] collectedSSTablesPerTask = new int[tasks.size()]; + for (AbstractCompactionTask act : tasks) + { + CompactionTask t = (CompactionTask) act; + assertTrue(t instanceof UnifiedCompactionTask); + assertFalse(t.inputSSTables().isEmpty()); + collectedSSTablesPerTask[i] = t.inputSSTables().size(); + expectedSSTablesInTasks[i] = (int) allSSTables.stream().filter(x -> intersects(x, t.tokenRange())).count(); + t.rejected(); // close transaction + ++i; + } + if (tasks.size() == 1) + assertNull(((CompactionTask) tasks.get(0)).tokenRange()); // make sure single-task compactions are not ranged + Assert.assertEquals(Arrays.toString(expectedSSTablesInTasks), Arrays.toString(collectedSSTablesPerTask)); + System.out.println(Arrays.toString(expectedSSTablesInTasks)); + assertThat(tasks.size()).isLessThanOrEqualTo(numShards); + assertEquals(allSSTables, tasks.stream().flatMap(t -> ((CompactionTask) t).inputSSTables().stream()).collect(Collectors.toSet())); + for (var t : tasks) + for (var q : tasks) + if (t != q) + assertFalse("Subranges " + ((CompactionTask)t).tokenRange() + " and " + ((CompactionTask)q).tokenRange() + "intersect", ((CompactionTask)t).tokenRange().intersects(((CompactionTask)q).tokenRange())); + + // make sure the composite transaction has the correct number of tasks + assertEquals(Transactional.AbstractTransactional.State.ABORTED, txn.state()); + } + + private boolean intersects(SSTableReader r, Range<Token> range) + { + if (range == null) + return true; + return range.intersects(range(r)); + } + + + private Bounds<Token> range(SSTableReader x) + { + return new Bounds<>(x.getFirst().getToken(), x.getLast().getToken()); + } + + @Test + public void testDontCreateParallelTasks() + { + int numShards = 5; Set<SSTableReader> allSSTables = new HashSet<>(); allSSTables.addAll(mockNonOverlappingSSTables(10, 0, 100 << 20)); allSSTables.addAll(mockNonOverlappingSSTables(15, 1, 200 << 20)); allSSTables.addAll(mockNonOverlappingSSTables(25, 2, 400 << 20)); + dataTracker.addInitialSSTables(allSSTables); + Controller controller = Mockito.mock(Controller.class); + when(controller.getNumShards(anyDouble())).thenReturn(numShards); + when(controller.parallelizeOutputShards()).thenReturn(false); + UnifiedCompactionStrategy strategy = new UnifiedCompactionStrategy(cfs, new HashMap<>(), controller); + strategy.startup(); + LifecycleTransaction txn = dataTracker.tryModify(allSSTables, OperationType.COMPACTION); + var tasks = strategy.createCompactionTasks(0, txn); + assertEquals(1, tasks.size()); + assertEquals(allSSTables, ((CompactionTask) tasks.get(0)).inputSSTables()); + } + @Test + public void testMaximalSelection() + { + // shared transaction, all tasks refer to the same input sstables + testMaximalSelection(1, 1, 0, false, 12 + 18 + 30, ((12 * 100L + 18 * 200 + 30 * 400) << 20)); + testMaximalSelection(5, 5, 0, true, 12 + 18 + 30, ((12 * 100L + 18 * 200 + 30 * 400) << 20)); + // when there's a common split point of existing and new sharding (i.e. gcd(num_shards,12,18,30) > 1), it should be used + testMaximalSelection(3, 3, 0, false, 4 + 6 + 10, ((4 * 100L + 6 * 200 + 10 * 400) << 20)); + testMaximalSelection(9, 3, 0, false, 4 + 6 + 10, ((4 * 100L + 6 * 200 + 10 * 400) << 20)); + testMaximalSelection(9, 9, 0, true, 4 + 6 + 10, ((4 * 100L + 6 * 200 + 10 * 400) << 20)); + testMaximalSelection(2, 2, 0, false, 6 + 9 + 15, ((6 * 100L + 9 * 200 + 15 * 400) << 20)); + testMaximalSelection(4, 2, 0, false, 6 + 9 + 15, ((6 * 100L + 9 * 200 + 15 * 400) << 20)); + testMaximalSelection(4, 4, 0, true, 6 + 9 + 15, ((6 * 100L + 9 * 200 + 15 * 400) << 20)); + testMaximalSelection(18, 6, 0, false, 2 + 3 + 5, ((2 * 100L + 3 * 200 + 5 * 400) << 20)); + testMaximalSelection(18, 18, 0, true, 2 + 3 + 5, ((2 * 100L + 3 * 200 + 5 * 400) << 20)); + } + + @Test + public void testMaximalSelectionWithLimit() + { + // shared transaction, all tasks refer to the same input sstables + testMaximalSelection(5, 5, 2, true, 12 + 18 + 30, ((12 * 100L + 18 * 200 + 30 * 400) << 20)); + // when there's a common split point of existing and new sharding (i.e. gcd(num_shards,12,18,30) > 1), it should be used + testMaximalSelection(3, 3, 2, false, 4 + 6 + 10, ((4 * 100L + 6 * 200 + 10 * 400) << 20)); + testMaximalSelection(9, 3, 1, false, 4 + 6 + 10, ((4 * 100L + 6 * 200 + 10 * 400) << 20)); + testMaximalSelection(9, 9, 3, true, 4 + 6 + 10, ((4 * 100L + 6 * 200 + 10 * 400) << 20)); + testMaximalSelection(18, 6, 4, false, 2 + 3 + 5, ((2 * 100L + 3 * 200 + 5 * 400) << 20)); + testMaximalSelection(18, 18, 5, true, 2 + 3 + 5, ((2 * 100L + 3 * 200 + 5 * 400) << 20)); + } + + private void testMaximalSelection(int numShards, int expectedTaskCount, int parallelismLimit, boolean parallelize, int originalsCount, long onDiskLength) + { + if (parallelismLimit == 0) + parallelismLimit = Integer.MAX_VALUE; + Set<SSTableReader> allSSTables = new HashSet<>(); + allSSTables.addAll(mockNonOverlappingSSTables(12, 0, 100 << 20)); + allSSTables.addAll(mockNonOverlappingSSTables(18, 1, 200 << 20)); + allSSTables.addAll(mockNonOverlappingSSTables(30, 2, 400 << 20)); + dataTracker.addInitialSSTables(allSSTables); Controller controller = Mockito.mock(Controller.class); + when(controller.getNumShards(anyDouble())).thenReturn(numShards); + when(controller.parallelizeOutputShards()).thenReturn(parallelize); + when(controller.maxConcurrentCompactions()).thenReturn(1000); UnifiedCompactionStrategy strategy = new UnifiedCompactionStrategy(cfs, new HashMap<>(), controller); strategy.addSSTables(allSSTables); - dataTracker.addInitialSSTables(allSSTables); - Collection<AbstractCompactionTask> tasks = strategy.getMaximalTask(0, false); - assertEquals(5, tasks.size()); // 5 (gcd of 10,15,25) common boundaries - for (AbstractCompactionTask task : tasks) + List<AbstractCompactionTask> allTasks = strategy.getMaximalTasks(0, false); + List<AbstractCompactionTask> limitedParallelismTasks = CompositeCompactionTask.applyParallelismLimit(allTasks, parallelismLimit); + assertEquals(expectedTaskCount, allTasks.size()); + for (AbstractCompactionTask task : allTasks) { Set<SSTableReader> compacting = task.transaction.originals(); - assertEquals(2 + 3 + 5, compacting.size()); // count / gcd sstables of each level - assertEquals((2 * 100L + 3 * 200 + 5 * 400) << 20, compacting.stream().mapToLong(SSTableReader::onDiskLength).sum()); + assertEquals(originalsCount, compacting.size()); // count / gcd sstables of each level + assertEquals(onDiskLength, compacting.stream().mapToLong(SSTableReader::onDiskLength).sum()); - // None of the selected sstables may intersect any in any other set. - for (AbstractCompactionTask task2 : tasks) + if (!(task.transaction instanceof PartialLifecycleTransaction)) { - if (task == task2) - continue; + // None of the selected sstables may intersect any in any other set. + for (AbstractCompactionTask task2 : allTasks) + { + if (task == task2) + continue; - Set<SSTableReader> compacting2 = task2.transaction.originals(); - for (SSTableReader r1 : compacting) - for (SSTableReader r2 : compacting2) - assertTrue(r1 + " intersects " + r2, r1.getFirst().compareTo(r2.getLast()) > 0 || r1.getLast().compareTo(r2.getFirst()) < 0); + Set<SSTableReader> compacting2 = task2.transaction.originals(); + for (SSTableReader r1 : compacting) + for (SSTableReader r2 : compacting2) + assertTrue(r1 + " intersects " + r2, r1.getFirst().compareTo(r2.getLast()) > 0 || r1.getLast().compareTo(r2.getFirst()) < 0); + } } } + + if (parallelismLimit > 0) + assertTrue(limitedParallelismTasks.size() <= parallelismLimit); } @Test @@ -899,8 +1099,8 @@ List<SSTableReader> mockNonOverlappingSSTables(int numSSTables, int level, long List<SSTableReader> sstables = new ArrayList<>(numSSTables); for (int i = 0; i < numSSTables; i++) { - DecoratedKey first = new BufferDecoratedKey(boundary(numSSTables, i).nextValidToken(), emptyBuffer); - DecoratedKey last = new BufferDecoratedKey(boundary(numSSTables, i+1), emptyBuffer); + DecoratedKey first = new BufferDecoratedKey(boundary(numSSTables, i + 0.0001), emptyBuffer); + DecoratedKey last = new BufferDecoratedKey(boundary(numSSTables, i + 0.9999), emptyBuffer); sstables.add(mockSSTable(level, bytesOnDisk, timestamp, 0., first, last)); timestamp+=10; @@ -909,8 +1109,9 @@ List<SSTableReader> mockNonOverlappingSSTables(int numSSTables, int level, long return sstables; } - private Token boundary(int numSSTables, int i) + private Token boundary(int numSSTables, double i) { - return partitioner.split(partitioner.getMinimumToken(), partitioner.getMaximumToken(), i * 1.0 / numSSTables); + return partitioner.split(partitioner.getMinimumToken(), partitioner.getMaximumToken(), i / numSSTables); } + } diff --git a/test/unit/org/apache/cassandra/db/compaction/unified/BackgroundCompactionTrackingTest.java b/test/unit/org/apache/cassandra/db/compaction/unified/BackgroundCompactionTrackingTest.java new file mode 100644 index 000000000000..212182e2b830 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/compaction/unified/BackgroundCompactionTrackingTest.java @@ -0,0 +1,194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.compaction.unified; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.BitSet; +import java.util.HashSet; +import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.stream.Collectors; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; + +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.compaction.CompactionInfo; +import org.apache.cassandra.db.compaction.CompactionManager; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.utils.TimeUUID; +import org.jboss.byteman.contrib.bmunit.BMRule; +import org.jboss.byteman.contrib.bmunit.BMUnitConfig; +import org.jboss.byteman.contrib.bmunit.BMUnitRunner; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +@RunWith(BMUnitRunner.class) +@BMUnitConfig(debug=true) +@BMRule( +name = "Get stats before task completion", +targetClass = "org.apache.cassandra.db.compaction.ActiveCompactions", +targetMethod = "finishCompaction", +targetLocation = "AT ENTRY", +action = "org.apache.cassandra.db.compaction.unified.BackgroundCompactionTrackingTest.getStats()" +) +public class BackgroundCompactionTrackingTest extends CQLTester +{ + // Get rid of commitlog noise + @Before + public void disableCommitlog() + { + schemaChange("ALTER KEYSPACE " + KEYSPACE + " WITH durable_writes = false"); + } + @After + public void enableCommitlog() + { + schemaChange("ALTER KEYSPACE " + KEYSPACE + " WITH durable_writes = true"); + } + + @Test + public void testBackgroundCompactionTracking() + { + testBackgroundCompactionTracking(false, 3); + } + + @Test + public void testBackgroundCompactionTrackingParallelized() + { + testBackgroundCompactionTracking(true, 3); + } + + public void testBackgroundCompactionTracking(boolean parallelize, int shards) + { + int bytes_per_row = 5000; + int partitions = 5000; + int rows_per_partition = 10; + long targetSize = 1L * partitions * rows_per_partition * bytes_per_row / shards; + CompactionManager.instance.setMaximumCompactorThreads(50); + CompactionManager.instance.setCoreCompactorThreads(50); + String table = createTable(String.format("CREATE TABLE %%s (k int, t int, v blob, PRIMARY KEY (k, t))" + + " with compression = {'enabled': false} " + + " and compaction = {" + + "'class': 'UnifiedCompactionStrategy', " + + "'parallelize_output_shards': '%s', " + + "'base_shard_count': %d, " + + "'min_sstable_size': '%dB', " + // to disable sharding on flush + "'scaling_parameters': 'T4, T7'" + + "}", + parallelize, shards, targetSize)); + ColumnFamilyStore cfs = getColumnFamilyStore(KEYSPACE, table); + cfs.disableAutoCompaction(); + + for (int iter = 1; iter <= 5; ++iter) + { + byte [] payload = new byte[bytes_per_row]; + new Random(42).nextBytes(payload); + ByteBuffer b = ByteBuffer.wrap(payload); + Set<SSTableReader> before = new HashSet<>(cfs.getLiveSSTables()); + + for (int i = 0; i < partitions; i++) + { + for (int j = 0; j < rows_per_partition; j++) + execute(String.format("INSERT INTO %s.%s(k, t, v) VALUES (?, ?, ?)", KEYSPACE, table), i, j, b); + + if ((i + 1) % ((partitions + 3) / 4) == 0) + cfs.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + } + cfs.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + operations = new ArrayList<>(); + Set<SSTableReader> newSSTables = new HashSet<>(cfs.getLiveSSTables()); + newSSTables.removeAll(before); + long uncompressedSize = newSSTables.stream().mapToLong(SSTableReader::uncompressedLength).sum(); + + cfs.enableAutoCompaction(true); // since the trigger is hit, this initiates an L0 compaction + cfs.disableAutoCompaction(); + + // Check that the background compactions state is correct during the compaction + Assert.assertTrue("Byteman rule did not fire", !operations.isEmpty()); + printStats(); + int tasks = parallelize ? shards : 1; + assertEquals(tasks, operations.size()); + TimeUUID mainOpId = null; + for (int i = 0; i < operations.size(); ++i) + { + BitSet seqs = new BitSet(shards); + int expectedSize = tasks - i; + final List<CompactionInfo> ops = getCompactionOps(i, cfs); + final int size = ops.size(); + int finished = tasks - size; + assertTrue(size >= expectedSize); // some task may have not managed to close + for (var op : ops) + { + final TimeUUID opIdSeq0 = op.getTaskId().withSequence(0); + if (mainOpId == null) + mainOpId = opIdSeq0; + else + assertEquals(mainOpId, opIdSeq0); + seqs.set(op.getTaskId().sequence()); + + if (i == 0) + Assert.assertEquals(uncompressedSize * 1.0 / tasks, op.getTotal(), uncompressedSize * 0.03); + assertTrue(op.getCompleted() <= op.getTotal() * 1.03); + if (op.getCompleted() >= op.getTotal() * 0.97) + ++finished; + } + assertTrue(finished > i); + assertEquals(size, seqs.cardinality()); + } + assertEquals(iter * shards, cfs.getLiveSSTables().size()); + + // Check that the background compactions state is correct after the compaction + operations.clear(); + getStats(); + printStats(); + final List<CompactionInfo> ops = getCompactionOps(0, cfs); + assertEquals(0, ops.size()); + } + } + + private static List<CompactionInfo> getCompactionOps(int i, ColumnFamilyStore cfs) + { + return operations.get(i) + .stream() + .filter(op -> op.getTableMetadata() == cfs.metadata()) + .collect(Collectors.toList()); + } + + private void printStats() + { + for (int i = 0; i < operations.size(); ++i) + { + System.out.println(operations.get(i).stream().map(Object::toString).collect(Collectors.joining("\n"))); + } + } + + public static synchronized void getStats() + { + operations.add(CompactionManager.instance.getSSTableTasks()); + } + + static List<List<CompactionInfo>> operations; +} diff --git a/test/unit/org/apache/cassandra/db/compaction/unified/ParallelizedTasksTest.java b/test/unit/org/apache/cassandra/db/compaction/unified/ParallelizedTasksTest.java new file mode 100644 index 000000000000..7e20e2d5a63f --- /dev/null +++ b/test/unit/org/apache/cassandra/db/compaction/unified/ParallelizedTasksTest.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.compaction.unified; + +import java.util.Collection; +import java.util.List; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.Future; +import java.util.stream.Collectors; + +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.Test; + +import org.apache.cassandra.cql3.QueryProcessor; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.compaction.AbstractCompactionTask; +import org.apache.cassandra.db.compaction.ActiveCompactionsTracker; +import org.apache.cassandra.db.compaction.OperationType; +import org.apache.cassandra.db.compaction.ShardManager; +import org.apache.cassandra.db.compaction.ShardManagerNoDisks; +import org.apache.cassandra.db.compaction.UnifiedCompactionStrategy; +import org.apache.cassandra.db.lifecycle.CompositeLifecycleTransaction; +import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.lifecycle.PartialLifecycleTransaction; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.concurrent.Transactional; +import org.mockito.Mockito; + +import static org.apache.cassandra.db.ColumnFamilyStore.RING_VERSION_IRRELEVANT; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class ParallelizedTasksTest extends ShardingTestBase +{ + @Before + public void before() + { + // Disabling durable write since we don't care + schemaChange("CREATE KEYSPACE IF NOT EXISTS " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'} AND durable_writes=false"); + schemaChange(String.format("CREATE TABLE IF NOT EXISTS %s.%s (k int, t int, v blob, PRIMARY KEY (k, t))", KEYSPACE, TABLE)); + } + + @AfterClass + public static void tearDownClass() + { + QueryProcessor.executeInternal("DROP KEYSPACE IF EXISTS " + KEYSPACE); + } + + private ColumnFamilyStore getColumnFamilyStore() + { + return Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE); + } + + @Test + public void testOneSSTablePerShard() throws Throwable + { + int numShards = 5; + testParallelized(numShards, PARTITIONS, numShards, true); + } + + @Test + public void testMultipleInputSSTables() throws Throwable + { + int numShards = 3; + testParallelized(numShards, PARTITIONS, numShards, false); + } + + private void testParallelized(int numShards, int rowCount, int numOutputSSTables, boolean compact) throws Throwable + { + ColumnFamilyStore cfs = getColumnFamilyStore(); + cfs.disableAutoCompaction(); + + populate(rowCount, compact); + + LifecycleTransaction transaction = cfs.getTracker().tryModify(cfs.getLiveSSTables(), OperationType.COMPACTION); + + ShardManager shardManager = new ShardManagerNoDisks(ColumnFamilyStore.fullWeightedRange(RING_VERSION_IRRELEVANT, cfs.getPartitioner())); + + Controller mockController = Mockito.mock(Controller.class); + UnifiedCompactionStrategy mockStrategy = Mockito.mock(UnifiedCompactionStrategy.class, Mockito.CALLS_REAL_METHODS); + Mockito.when(mockStrategy.getController()).thenReturn(mockController); + Mockito.when(mockController.getNumShards(Mockito.anyDouble())).thenReturn(numShards); + + Collection<SSTableReader> sstables = transaction.originals(); + CompositeLifecycleTransaction compositeTransaction = new CompositeLifecycleTransaction(transaction); + + List<AbstractCompactionTask> tasks = shardManager.splitSSTablesInShards( + sstables, + numShards, + (rangeSSTables, range) -> + new UnifiedCompactionTask(cfs, + mockStrategy, + new PartialLifecycleTransaction(compositeTransaction), + 0, + shardManager, + range, + rangeSSTables) + ); + compositeTransaction.completeInitialization(); + assertEquals(numOutputSSTables, tasks.size()); + + List<Future<?>> futures = tasks.stream().map(t -> ForkJoinPool.commonPool().submit(() -> t.execute(ActiveCompactionsTracker.NOOP))).collect(Collectors.toList()); + FBUtilities.waitOnFutures(futures); + assertTrue(transaction.state() == Transactional.AbstractTransactional.State.COMMITTED); + + verifySharding(numShards, rowCount, numOutputSSTables, cfs); + cfs.truncateBlocking(); + } +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/db/compaction/unified/ShardedCompactionWriterTest.java b/test/unit/org/apache/cassandra/db/compaction/unified/ShardedCompactionWriterTest.java index 2ebe1bdf835f..50f917974311 100644 --- a/test/unit/org/apache/cassandra/db/compaction/unified/ShardedCompactionWriterTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/unified/ShardedCompactionWriterTest.java @@ -18,10 +18,7 @@ package org.apache.cassandra.db.compaction.unified; -import java.nio.ByteBuffer; -import java.util.Collection; import java.util.List; -import java.util.Random; import java.util.Set; import java.util.stream.Collectors; @@ -29,35 +26,27 @@ import org.junit.Before; import org.junit.Test; -import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; -import org.apache.cassandra.db.compaction.AbstractCompactionStrategy; -import org.apache.cassandra.db.compaction.CompactionController; -import org.apache.cassandra.db.compaction.CompactionIterator; import org.apache.cassandra.db.compaction.OperationType; import org.apache.cassandra.db.compaction.ShardManager; import org.apache.cassandra.db.compaction.ShardManagerDiskAware; import org.apache.cassandra.db.compaction.ShardManagerNoDisks; -import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter; -import org.apache.cassandra.utils.FBUtilities; -import org.apache.cassandra.utils.TimeUUID; import static org.apache.cassandra.db.ColumnFamilyStore.RING_VERSION_IRRELEVANT; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -public class ShardedCompactionWriterTest extends CQLTester +public class ShardedCompactionWriterTest extends ShardingTestBase { private static final String KEYSPACE = "cawt_keyspace"; private static final String TABLE = "cawt_table"; private static final int ROW_PER_PARTITION = 10; + private static final int PARTITIONS = 50000; @Before public void before() @@ -84,8 +73,7 @@ public void testOneSSTablePerShard() throws Throwable // If we set the minSSTableSize ratio to 0.5, because this gets multiplied by the shard size to give the min sstable size, // assuming evenly distributed data, it should split at each boundary and so we should end up with numShards sstables int numShards = 5; - int rowCount = 5000; - testShardedCompactionWriter(numShards, rowCount, numShards, true); + testShardedCompactionWriter(numShards, PARTITIONS, numShards, true); } @@ -93,8 +81,7 @@ public void testOneSSTablePerShard() throws Throwable public void testMultipleInputSSTables() throws Throwable { int numShards = 3; - int rowCount = 5000; - testShardedCompactionWriter(numShards, rowCount, numShards, false); + testShardedCompactionWriter(numShards, PARTITIONS, numShards, false); } private void testShardedCompactionWriter(int numShards, int rowCount, int numOutputSSTables, boolean majorCompaction) throws Throwable @@ -107,33 +94,15 @@ private void testShardedCompactionWriter(int numShards, int rowCount, int numOut LifecycleTransaction txn = cfs.getTracker().tryModify(cfs.getLiveSSTables(), OperationType.COMPACTION); ShardManager boundaries = new ShardManagerNoDisks(ColumnFamilyStore.fullWeightedRange(RING_VERSION_IRRELEVANT, cfs.getPartitioner())); - ShardedCompactionWriter writer = new ShardedCompactionWriter(cfs, cfs.getDirectories(), txn, txn.originals(), false, boundaries.boundaries(numShards)); + ShardedCompactionWriter writer = new ShardedCompactionWriter(cfs, cfs.getDirectories(), txn, txn.originals(), false, true, boundaries.boundaries(numShards)); int rows = compact(cfs, txn, writer); - assertEquals(numOutputSSTables, cfs.getLiveSSTables().size()); assertEquals(rowCount, rows); - long totalOnDiskLength = cfs.getLiveSSTables().stream().mapToLong(SSTableReader::onDiskLength).sum(); - long totalBFSize = cfs.getLiveSSTables().stream().mapToLong(ShardedCompactionWriterTest::getFilterSize).sum(); - assert totalBFSize > 16 * numOutputSSTables : "Bloom Filter is empty"; // 16 is the size of empty bloom filter - for (SSTableReader rdr : cfs.getLiveSSTables()) - { - assertEquals((double) rdr.onDiskLength() / totalOnDiskLength, - (double) getFilterSize(rdr) / totalBFSize, 0.1); - assertEquals(1.0 / numOutputSSTables, rdr.tokenSpaceCoverage(), 0.05); - } - - validateData(cfs, rowCount); + verifySharding(numShards, rowCount, numOutputSSTables, cfs); cfs.truncateBlocking(); } - static long getFilterSize(SSTableReader rdr) - { - if (!(rdr instanceof SSTableReaderWithFilter)) - return 0; - return ((SSTableReaderWithFilter) rdr).getFilterSerializedSize(); - } - @Test public void testDiskAdvance() throws Throwable { @@ -200,94 +169,4 @@ public void testDiskAdvance() throws Throwable validateData(cfs, rowCount); cfs.truncateBlocking(); } - - private int compact(int numShards, ColumnFamilyStore cfs, ShardManager shardManager, Collection<SSTableReader> selection) - { - int rows; - LifecycleTransaction txn = cfs.getTracker().tryModify(selection, OperationType.COMPACTION); - ShardedCompactionWriter writer = new ShardedCompactionWriter(cfs, - cfs.getDirectories(), - txn, - txn.originals(), - false, - shardManager.boundaries(numShards)); - - rows = compact(cfs, txn, writer); - return rows; - } - - private static void verifyNoSpannedBoundaries(List<Token> diskBoundaries, SSTableReader rdr) - { - for (int i = 0; i < diskBoundaries.size(); ++i) - { - Token boundary = diskBoundaries.get(i); - // rdr cannot span a boundary. I.e. it must be either fully before (last <= boundary) or fully after - // (first > boundary). - assertTrue(rdr.getFirst().getToken().compareTo(boundary) > 0 || - rdr.getLast().getToken().compareTo(boundary) <= 0); - } - } - - private int compact(ColumnFamilyStore cfs, LifecycleTransaction txn, CompactionAwareWriter writer) - { - //assert txn.originals().size() == 1; - int rowsWritten = 0; - long nowInSec = FBUtilities.nowInSeconds(); - try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategyManager().getScanners(txn.originals()); - CompactionController controller = new CompactionController(cfs, txn.originals(), cfs.gcBefore(nowInSec)); - CompactionIterator ci = new CompactionIterator(OperationType.COMPACTION, scanners.scanners, controller, nowInSec, TimeUUID.minAtUnixMillis(System.currentTimeMillis()))) - { - while (ci.hasNext()) - { - if (writer.append(ci.next())) - rowsWritten++; - } - } - writer.finish(); - return rowsWritten; - } - - private void populate(int count, boolean compact) throws Throwable - { - byte [] payload = new byte[5000]; - new Random(42).nextBytes(payload); - ByteBuffer b = ByteBuffer.wrap(payload); - - ColumnFamilyStore cfs = getColumnFamilyStore(); - for (int i = 0; i < count; i++) - { - for (int j = 0; j < ROW_PER_PARTITION; j++) - execute(String.format("INSERT INTO %s.%s(k, t, v) VALUES (?, ?, ?)", KEYSPACE, TABLE), i, j, b); - - if (i % (count / 4) == 0) - cfs.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); - } - - cfs.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); - if (compact && cfs.getLiveSSTables().size() > 1) - { - // we want just one big sstable to avoid doing actual compaction in compact() above - try - { - cfs.forceMajorCompaction(); - } - catch (Throwable t) - { - throw new RuntimeException(t); - } - assert cfs.getLiveSSTables().size() == 1 : cfs.getLiveSSTables(); - } - } - - private void validateData(ColumnFamilyStore cfs, int rowCount) throws Throwable - { - for (int i = 0; i < rowCount; i++) - { - Object[][] expected = new Object[ROW_PER_PARTITION][]; - for (int j = 0; j < ROW_PER_PARTITION; j++) - expected[j] = row(i, j); - - assertRows(execute(String.format("SELECT k, t FROM %s.%s WHERE k = :i", KEYSPACE, TABLE), i), expected); - } - } } \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/db/compaction/unified/ShardingTestBase.java b/test/unit/org/apache/cassandra/db/compaction/unified/ShardingTestBase.java new file mode 100644 index 000000000000..fc2da63d006c --- /dev/null +++ b/test/unit/org/apache/cassandra/db/compaction/unified/ShardingTestBase.java @@ -0,0 +1,205 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.compaction.unified; + +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.List; +import java.util.Random; + +import org.junit.AfterClass; +import org.junit.Before; + +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.cql3.QueryProcessor; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.compaction.AbstractCompactionStrategy; +import org.apache.cassandra.db.compaction.CompactionController; +import org.apache.cassandra.db.compaction.CompactionIterator; +import org.apache.cassandra.db.compaction.OperationType; +import org.apache.cassandra.db.compaction.ShardManager; +import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter; +import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.FilterFactory; +import org.apache.cassandra.utils.TimeUUID; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class ShardingTestBase extends CQLTester +{ + static final String KEYSPACE = "cawt_keyspace"; + static final String TABLE = "cawt_table"; + + static final int ROW_PER_PARTITION = 10; + static final int PARTITIONS = 50000; + + @Before + public void before() + { + // Disabling durable write since we don't care + schemaChange("CREATE KEYSPACE IF NOT EXISTS " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'} AND durable_writes=false"); + schemaChange(String.format("CREATE TABLE IF NOT EXISTS %s.%s (k int, t int, v blob, PRIMARY KEY (k, t))", KEYSPACE, TABLE)); + } + + @AfterClass + public static void tearDownClass() + { + QueryProcessor.executeInternal("DROP KEYSPACE IF EXISTS " + KEYSPACE); + } + + private ColumnFamilyStore getColumnFamilyStore() + { + return Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE); + } + + void verifySharding(int numShards, int rowCount, int numOutputSSTables, ColumnFamilyStore cfs) throws Throwable + { + assertEquals(numOutputSSTables, cfs.getLiveSSTables().size()); + + long totalOnDiskLength = cfs.getLiveSSTables().stream().mapToLong(SSTableReader::onDiskLength).sum(); + long totalBFSize = cfs.getLiveSSTables().stream().mapToLong(ShardingTestBase::getFilterSize).sum(); + long totalKeyCount = cfs.getLiveSSTables().stream().mapToLong(SSTableReader::estimatedKeys).sum(); + assert totalBFSize > 16 * numOutputSSTables : "Bloom Filter is empty"; // 16 is the size of empty bloom filter + for (SSTableReader rdr : cfs.getLiveSSTables()) + { + assertEquals((double) rdr.onDiskLength() / totalOnDiskLength, + (double) getFilterSize(rdr) / totalBFSize, 0.1); + assertEquals(1.0 / numOutputSSTables, rdr.tokenSpaceCoverage(), 0.05); + } + + System.out.println("Total on disk length: " + FBUtilities.prettyPrintMemory(totalOnDiskLength)); + System.out.println("Total BF size: " + FBUtilities.prettyPrintMemory(totalBFSize)); + System.out.println("Total key count: " + FBUtilities.prettyPrintDecimal(totalKeyCount, "", "")); + try (var filter = FilterFactory.getFilter(totalKeyCount, 0.01)) + { + System.out.println("Optimal total BF size: " + FBUtilities.prettyPrintMemory(filter.serializedSize(false))); + } + try (var filter = FilterFactory.getFilter(totalKeyCount / numShards, 0.01)) + { + System.out.println("Sharded optimal total BF size: " + FBUtilities.prettyPrintMemory(filter.serializedSize(false) * numShards)); + } + + cfs.getLiveSSTables().forEach(s -> System.out.println("SSTable: " + s.toString() + " covers " + s.getFirst() + " to " + s.getLast())); + + validateData(cfs, rowCount); + } + + static long getFilterSize(SSTableReader rdr) + { + if (!(rdr instanceof SSTableReaderWithFilter)) + return 0; + return ((SSTableReaderWithFilter) rdr).getFilterSerializedSize(); + } + + int compact(int numShards, ColumnFamilyStore cfs, ShardManager shardManager, Collection<SSTableReader> selection) + { + int rows; + LifecycleTransaction txn = cfs.getTracker().tryModify(selection, OperationType.COMPACTION); + ShardedCompactionWriter writer = new ShardedCompactionWriter(cfs, + cfs.getDirectories(), + txn, + txn.originals(), + false, + true, + shardManager.boundaries(numShards)); + + rows = compact(cfs, txn, writer); + return rows; + } + + static void verifyNoSpannedBoundaries(List<Token> diskBoundaries, SSTableReader rdr) + { + for (int i = 0; i < diskBoundaries.size(); ++i) + { + Token boundary = diskBoundaries.get(i); + // rdr cannot span a boundary. I.e. it must be either fully before (last <= boundary) or fully after + // (first > boundary). + assertTrue(rdr.getFirst().getToken().compareTo(boundary) > 0 || + rdr.getLast().getToken().compareTo(boundary) <= 0); + } + } + + int compact(ColumnFamilyStore cfs, LifecycleTransaction txn, CompactionAwareWriter writer) + { + //assert txn.originals().size() == 1; + int rowsWritten = 0; + long nowInSec = FBUtilities.nowInSeconds(); + try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategyManager().getScanners(txn.originals()); + CompactionController controller = new CompactionController(cfs, txn.originals(), cfs.gcBefore(nowInSec)); + CompactionIterator ci = new CompactionIterator(OperationType.COMPACTION, scanners.scanners, controller, nowInSec, TimeUUID.minAtUnixMillis(System.currentTimeMillis()))) + { + while (ci.hasNext()) + { + if (writer.append(ci.next())) + rowsWritten++; + } + } + writer.finish(); + return rowsWritten; + } + + void populate(int count, boolean compact) throws Throwable + { + byte [] payload = new byte[5000]; + new Random(42).nextBytes(payload); + ByteBuffer b = ByteBuffer.wrap(payload); + + ColumnFamilyStore cfs = getColumnFamilyStore(); + for (int i = 0; i < count; i++) + { + for (int j = 0; j < ROW_PER_PARTITION; j++) + execute(String.format("INSERT INTO %s.%s(k, t, v) VALUES (?, ?, ?)", KEYSPACE, TABLE), i, j, b); + + if (i % (count / 4) == 0) + cfs.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + } + + cfs.forceBlockingFlush(ColumnFamilyStore.FlushReason.UNIT_TESTS); + if (compact && cfs.getLiveSSTables().size() > 1) + { + // we want no overlapping sstables to avoid doing actual compaction in compact() above + try + { + cfs.forceMajorCompaction(); + } + catch (Throwable t) + { + throw new RuntimeException(t); + } + } + } + + void validateData(ColumnFamilyStore cfs, int rowCount) throws Throwable + { + for (int i = 0; i < rowCount; i++) + { + Object[][] expected = new Object[ROW_PER_PARTITION][]; + for (int j = 0; j < ROW_PER_PARTITION; j++) + expected[j] = row(i, j); + + assertRows(execute(String.format("SELECT k, t FROM %s.%s WHERE k = :i", KEYSPACE, TABLE), i), expected); + } + } +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java index 89a3e9ed367c..c29f059000c7 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java @@ -104,7 +104,7 @@ public static void defineSchemaAndPrepareSSTable() .applyUnsafe(); } Util.flush(store); - CompactionManager.instance.performMaximal(store, false); + CompactionManager.instance.performMaximal(store); sstable = store.getLiveSSTables().iterator().next(); descriptor = sstable.descriptor; diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java index 2921b5ae1749..8663c5f5ebe4 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java @@ -81,7 +81,7 @@ public static void defineSchemaAndPrepareSSTable() .applyUnsafe(); } Util.flush(store); - CompactionManager.instance.performMaximal(store, false); + CompactionManager.instance.performMaximal(store); sstable = store.getLiveSSTables().iterator().next(); } diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java index de953734ca21..00e30938e2cf 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java @@ -82,7 +82,7 @@ public static void defineSchemaAndPrepareSSTable() .applyUnsafe(); } Util.flush(store); - CompactionManager.instance.performMaximal(store, false); + CompactionManager.instance.performMaximal(store); sstable = store.getLiveSSTables().iterator().next(); } diff --git a/test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java b/test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java index 8107fd9789b5..7f0fe2c9c6e4 100644 --- a/test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/EntireSSTableStreamConcurrentComponentMutationTest.java @@ -130,7 +130,7 @@ public static void defineSchemaAndPrepareSSTable() .applyUnsafe(); } Util.flush(store); - CompactionManager.instance.performMaximal(store, false); + CompactionManager.instance.performMaximal(store); Token start = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(0)); Token end = ByteOrderedPartitioner.instance.getTokenFactory().fromString(Long.toHexString(100)); diff --git a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java index 96ab73f626e7..7162da101853 100644 --- a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java @@ -44,7 +44,6 @@ import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.SinglePartitionSliceCommandTest; -import org.apache.cassandra.db.compaction.AbstractCompactionTask; import org.apache.cassandra.db.compaction.CompactionManager; import org.apache.cassandra.db.repair.PendingAntiCompaction; import org.apache.cassandra.db.rows.RangeTombstoneMarker; @@ -398,9 +397,9 @@ public void testAutomaticUpgrade() throws Exception truncateLegacyTables(legacyVersion); loadLegacyTables(legacyVersion); ColumnFamilyStore cfs = Keyspace.open(LEGACY_TABLES_KEYSPACE).getColumnFamilyStore(String.format("legacy_%s_simple", legacyVersion)); - AbstractCompactionTask act = cfs.getCompactionStrategyManager().getNextBackgroundTask(0); + var act = cfs.getCompactionStrategyManager().getNextBackgroundTasks(0); // there should be no compactions to run with auto upgrades disabled: - assertEquals(null, act); + assertEquals(0, act.size()); } DatabaseDescriptor.setAutomaticSSTableUpgradeEnabled(true); diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java index a87ec024b58f..4997420233e5 100644 --- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java @@ -159,7 +159,7 @@ public void testGetPositionsForRanges() .applyUnsafe(); } Util.flush(store); - CompactionManager.instance.performMaximal(store, false); + CompactionManager.instance.performMaximal(store); List<Range<Token>> ranges = new ArrayList<>(); // 1 key @@ -397,7 +397,7 @@ public void testSpannedIndexPositions() throws IOException .applyUnsafe(); } Util.flush(store); - CompactionManager.instance.performMaximal(store, false); + CompactionManager.instance.performMaximal(store); // check that all our keys are found correctly SSTableReader sstable = store.getLiveSSTables().iterator().next(); @@ -522,7 +522,7 @@ public void testGetPositionsForRangesWithKeyCache() } Util.flush(store); - CompactionManager.instance.performMaximal(store, false); + CompactionManager.instance.performMaximal(store); SSTableReader sstable = store.getLiveSSTables().iterator().next(); long p2 = sstable.getPosition(dk(2), SSTableReader.Operator.EQ); @@ -576,7 +576,7 @@ public void testGetPositionsKeyCacheStats() .applyUnsafe(); } Util.flush(store); - CompactionManager.instance.performMaximal(store, false); + CompactionManager.instance.performMaximal(store); SSTableReader sstable = store.getLiveSSTables().iterator().next(); KeyCache keyCache = ((KeyCacheSupport<?>) sstable).getKeyCache(); @@ -773,7 +773,7 @@ private SSTableReaderWithFilter prepareGetPositions() } } Util.flush(store); - CompactionManager.instance.performMaximal(store, false); + CompactionManager.instance.performMaximal(store); SSTableReaderWithFilter sstable = (SSTableReaderWithFilter) store.getLiveSSTables().iterator().next(); sstable = (SSTableReaderWithFilter) sstable.cloneWithNewStart(dk(3)); @@ -1114,7 +1114,7 @@ public void testGetPositionsForRangesFromTableOpenedForBulkLoading() } Util.flush(store); - CompactionManager.instance.performMaximal(store, false); + CompactionManager.instance.performMaximal(store); // construct a range which is present in the sstable, but whose // keys are not found in the first segment of the index. @@ -1151,7 +1151,7 @@ public void testIndexSummaryReplacement() throws IOException, ExecutionException .applyUnsafe(); } Util.flush(store); - CompactionManager.instance.performMaximal(store, false); + CompactionManager.instance.performMaximal(store); Collection<SSTableReader> sstables = store.getLiveSSTables(); assert sstables.size() == 1; @@ -1228,7 +1228,7 @@ private <R extends SSTableReader & IndexSummarySupport<R>> void testIndexSummary .applyUnsafe(); } Util.flush(store); - CompactionManager.instance.performMaximal(store, false); + CompactionManager.instance.performMaximal(store); Collection<R> sstables = ServerTestUtils.<R>getLiveIndexSummarySupportingReaders(store); assert sstables.size() == 1; diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java b/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java index 2e410da5cfea..e4c47c68b16d 100644 --- a/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java +++ b/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java @@ -152,7 +152,7 @@ public static void validateCFS(ColumnFamilyStore cfs) assertTrue(cfs.getTracker().getCompacting().isEmpty()); if(cfs.getLiveSSTables().size() > 0) - assertFalse(CompactionManager.instance.submitMaximal(cfs, cfs.gcBefore((int) (System.currentTimeMillis() / 1000)), false).isEmpty()); + assertFalse(CompactionManager.instance.submitMaximal(cfs, cfs.gcBefore((int) (System.currentTimeMillis() / 1000)), false, 0).isEmpty()); } public static SSTableWriter getWriter(ColumnFamilyStore cfs, File directory, LifecycleTransaction txn, long repairedAt, TimeUUID pendingRepair, boolean isTransient) diff --git a/test/unit/org/apache/cassandra/io/sstable/VerifyTest.java b/test/unit/org/apache/cassandra/io/sstable/VerifyTest.java index 5493356a2eb9..3b651fc36b0c 100644 --- a/test/unit/org/apache/cassandra/io/sstable/VerifyTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/VerifyTest.java @@ -108,6 +108,7 @@ public class VerifyTest public static final String COUNTER_CF4 = "Counter4"; public static final String CORRUPT_CF = "Corrupt1"; public static final String CORRUPT_CF2 = "Corrupt2"; + public static final String CORRUPT_CF3 = "Corrupt3"; public static final String CORRUPTCOUNTER_CF = "CounterCorrupt1"; public static final String CORRUPTCOUNTER_CF2 = "CounterCorrupt2"; @@ -130,6 +131,7 @@ public static void defineSchema() throws ConfigurationException standardCFMD(KEYSPACE, CF4), standardCFMD(KEYSPACE, CORRUPT_CF), standardCFMD(KEYSPACE, CORRUPT_CF2), + standardCFMD(KEYSPACE, CORRUPT_CF3), counterCFMD(KEYSPACE, COUNTER_CF).compression(compressionParameters), counterCFMD(KEYSPACE, COUNTER_CF2).compression(compressionParameters), counterCFMD(KEYSPACE, COUNTER_CF3), @@ -516,7 +518,7 @@ public void testMutateRepair() throws IOException { CompactionManager.instance.disableAutoCompaction(); Keyspace keyspace = Keyspace.open(KEYSPACE); - ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CORRUPT_CF2); + ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CORRUPT_CF3); fillCF(cfs, 2); diff --git a/test/unit/org/apache/cassandra/streaming/EntireSSTableStreamingCorrectFilesCountTest.java b/test/unit/org/apache/cassandra/streaming/EntireSSTableStreamingCorrectFilesCountTest.java index a4022cb34604..80dd71f2734c 100644 --- a/test/unit/org/apache/cassandra/streaming/EntireSSTableStreamingCorrectFilesCountTest.java +++ b/test/unit/org/apache/cassandra/streaming/EntireSSTableStreamingCorrectFilesCountTest.java @@ -104,7 +104,7 @@ public static void defineSchemaAndPrepareSSTable() } Util.flush(store); - CompactionManager.instance.performMaximal(store, false); + CompactionManager.instance.performMaximal(store); sstable = store.getLiveSSTables().iterator().next(); diff --git a/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java b/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java index eb741c6915c7..4fdb9c076f13 100644 --- a/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java +++ b/tools/stress/src/org/apache/cassandra/stress/CompactionStress.java @@ -239,7 +239,7 @@ public void run() List<Future<?>> futures = new ArrayList<>(threads); if (maximal) { - futures = CompactionManager.instance.submitMaximal(cfs, FBUtilities.nowInSeconds(), false); + futures = CompactionManager.instance.submitMaximal(cfs, FBUtilities.nowInSeconds(), false, Integer.MAX_VALUE); } else { From 2ff41551a6897705a1fad842f9d550b019f6eadb Mon Sep 17 00:00:00 2001 From: Zhongxiang Zheng <zzheng@yahoo-corp.jp> Date: Thu, 30 Jun 2016 18:17:49 +0900 Subject: [PATCH 053/225] Enable JMX server configuration to be in cassandra.yaml patch by Zhongxiang Zheng; reviewed by Stefan Miklosovic, Maulin Vasavada, Cheng Wang, Jordan West for CASSANDRA-11695 Co-authored-by: Stefan Miklosovic <smiklosovic@apache.org> Co-authored-by: Sam Tunnicliffe <samt@apache.org> --- CHANGES.txt | 1 + NEWS.txt | 7 +- bin/nodetool | 13 + conf/cassandra-env.sh | 107 ++++---- conf/cassandra.yaml | 61 ++++- conf/cassandra_latest.yaml | 61 ++++- doc/scripts/convert_yaml_to_adoc.py | 2 +- .../config/CassandraRelevantProperties.java | 6 + .../org/apache/cassandra/config/Config.java | 6 +- .../cassandra/config/DatabaseDescriptor.java | 21 +- .../cassandra/config/EncryptionOptions.java | 7 +- .../cassandra/config/JMXServerOptions.java | 240 ++++++++++++++++++ .../cassandra/service/CassandraDaemon.java | 33 +-- .../cassandra/service/StartupChecks.java | 20 +- .../cassandra/utils/JMXServerUtils.java | 79 +++--- .../utils/jmx/AbstractJmxSocketFactory.java | 111 ++------ .../utils/jmx/DefaultJmxSocketFactory.java | 9 - .../cassandra-jmx-disabled-sslconfig.yaml | 168 ++++++------ test/conf/cassandra-jmx-pem-sslconfig.yaml | 166 ++++++------ test/conf/cassandra-jmx-sslconfig.yaml | 16 +- ...llectingSslRMIServerSocketFactoryImpl.java | 10 - .../distributed/impl/IsolatedJmx.java | 28 +- .../impl/IsolatedJmxSocketFactory.java | 33 +-- .../test/jmx/JMXSslConfigDistributedTest.java | 48 ++-- test/resources/auth/cassandra-test-jaas.conf | 2 +- ...AuthTest.java => AbstractJMXAuthTest.java} | 113 ++++----- .../auth/jmx/JMXAuthJMXServerOptionsTest.java | 48 ++++ .../auth/jmx/JMXAuthSystemPropertiesTest.java | 55 ++++ .../config/DatabaseDescriptorRefTest.java | 1 + .../org/apache/cassandra/cql3/CQLTester.java | 6 +- .../db/virtual/SettingsTableTest.java | 4 +- .../cassandra/tools/ToolsEnvsConfigsTest.java | 3 +- .../cassandra/tools/nodetool/SjkTest.java | 2 + .../jmx/DuplicateJMXConfigurationTest.java | 67 +++++ .../cassandra/utils/jmx/JMXSslConfigTest.java | 19 +- ...XSslConfiguredWithYamlFileOptionsTest.java | 28 +- .../JMXSslDisabledEncryptionOptionsTest.java | 5 +- .../utils/jmx/JMXSslPEMConfigTest.java | 10 +- 38 files changed, 1042 insertions(+), 574 deletions(-) create mode 100644 src/java/org/apache/cassandra/config/JMXServerOptions.java rename test/unit/org/apache/cassandra/auth/jmx/{JMXAuthTest.java => AbstractJMXAuthTest.java} (87%) create mode 100644 test/unit/org/apache/cassandra/auth/jmx/JMXAuthJMXServerOptionsTest.java create mode 100644 test/unit/org/apache/cassandra/auth/jmx/JMXAuthSystemPropertiesTest.java create mode 100644 test/unit/org/apache/cassandra/utils/jmx/DuplicateJMXConfigurationTest.java diff --git a/CHANGES.txt b/CHANGES.txt index ffe042e3f35a..f3b51f4972eb 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Enable JMX server configuration to be in cassandra.yaml (CASSANDRA-11695) * Parallelized UCS compactions (CASSANDRA-18802) * Avoid prepared statement invalidation race when committing schema changes (CASSANDRA-20116) * Restore optimization in MultiCBuilder around building one clustering (CASSANDRA-20129) diff --git a/NEWS.txt b/NEWS.txt index ae1ba39d303c..b493e48d5187 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -88,7 +88,6 @@ New features generate a password of configured password strength policy upon role creation or alteration when 'GENERATED PASSWORD' clause is used. Character sets supported are: English, Cyrillic, modern Cyrillic, German, Polish and Czech. - - JMX SSL configuration can be now done in cassandra.yaml via jmx_encryption_options section instead of cassandra-env.sh - There is new MBean of name org.apache.cassandra.service.snapshot:type=SnapshotManager which exposes user-facing snapshot operations. Snapshot-related methods on StorageServiceMBean are still present and functional but marked as deprecated. @@ -102,6 +101,12 @@ New features compactions. To avoid the possibility of starving background compactions from resources, the number of threads used for major compactions is limited to half the available compaction threads by default, and can be controlled by a new --jobs / -j option of nodetool compact. + - It is possible to configure JMX server in cassandra.yaml in jmx_server_options configuration section. + JMX SSL configuration can be configured via jmx_encryption_options in jmx_server_options. The old way of + configuring JMX is still present and default, but new way is preferable as it will e.g. not leak credentials for + JMX to JVM parameters. You have to opt-in to use the configuration via cassandra.yaml by uncommenting + the respective configuration sections and by commenting out `configure_jmx` function call in cassandra-env.sh. + Enabling both ways of configuring JMX will result in a node failing to start. Upgrading diff --git a/bin/nodetool b/bin/nodetool index f78b02e34418..6ba19e16f061 100755 --- a/bin/nodetool +++ b/bin/nodetool @@ -52,6 +52,19 @@ if [ -f "$CASSANDRA_CONF/cassandra-env.sh" ]; then JVM_OPTS="$JVM_OPTS_SAVE" fi +# In case JMX_PORT is not set (when configure_jmx in cassandra-env.sh is commented out), +# try to parse it from cassandra.yaml. +if [ "x$JMX_PORT" = "x" ]; then + if [ -f "$CASSANDRA_CONF/cassandra.yaml" ]; then + JMX_PORT=`grep jmx_port $CASSANDRA_CONF/cassandra.yaml | cut -d ':' -f 2 | tr -d '[[:space:]]'` + fi +fi + +# If, by any chance, it is not there either, set it to default. +if [ "x$JMX_PORT" = "x" ]; then + JMX_PORT=7199 +fi + # JMX Port passed via cmd line args (-p 9999 / --port 9999 / --port=9999) # should override the value from cassandra-env.sh ARGS="" diff --git a/conf/cassandra-env.sh b/conf/cassandra-env.sh index 98b7ed8ccdf1..46085068dae2 100644 --- a/conf/cassandra-env.sh +++ b/conf/cassandra-env.sh @@ -218,55 +218,66 @@ if [ "x$LOCAL_JMX" = "x" ]; then LOCAL_JMX=yes fi -# Specifies the default port over which Cassandra will be available for -# JMX connections. -# For security reasons, you should not expose this port to the internet. Firewall it if needed. -JMX_PORT="7199" - -if [ "$LOCAL_JMX" = "yes" ]; then - JVM_OPTS="$JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT" - JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=false" -else - JVM_OPTS="$JVM_OPTS -Dcassandra.jmx.remote.port=$JMX_PORT" - # if ssl is enabled the same port cannot be used for both jmx and rmi so either - # pick another value for this property or comment out to use a random port (though see CASSANDRA-7087 for origins) - JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.rmi.port=$JMX_PORT" - - # turn on JMX authentication. See below for further options - JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true" - - # jmx ssl options - # Consider using the jmx_encryption_options section of cassandra.yaml instead - # to prevent sensitive information being exposed. - # In case jmx ssl options are configured in both the places - this file and cassandra.yaml, and - # if com.sun.management.jmxremote.ssl is configured to be true here and encryption_options are marked enabled in - # cassandra.yaml, then we will get exception at the startup - #JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl=true" - #JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.need.client.auth=true" - #JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.protocols=<enabled-protocols>" - #JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.cipher.suites=<enabled-cipher-suites>" - #JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.keyStore=/path/to/keystore" - #JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.keyStorePassword=<keystore-password>" - #JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.trustStore=/path/to/truststore" - #JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.trustStorePassword=<truststore-password>" -fi +configure_jmx() +{ + JMX_PORT=$1 + + if [ "$LOCAL_JMX" = "yes" ]; then + JVM_OPTS="$JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT" + JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=false" + else + JVM_OPTS="$JVM_OPTS -Dcassandra.jmx.remote.port=$JMX_PORT" + # if ssl is enabled the same port cannot be used for both jmx and rmi so either + # pick another value for this property or comment out to use a random port (though see CASSANDRA-7087 for origins) + JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.rmi.port=$JMX_PORT" + + # turn on JMX authentication. See below for further options + JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true" + + # jmx ssl options + # Consider using the jmx_encryption_options section of jmx_server_options in cassandra.yaml instead + # to prevent sensitive information being exposed. + # In case jmx ssl options are configured in both the places - this file and cassandra.yaml, and + # if com.sun.management.jmxremote.ssl is configured to be true here and encryption_options are marked enabled in + # cassandra.yaml, then we will get exception at the startup + #JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl=true" + #JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.need.client.auth=true" + #JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.protocols=<enabled-protocols>" + #JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.cipher.suites=<enabled-cipher-suites>" + #JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.keyStore=/path/to/keystore" + #JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.keyStorePassword=<keystore-password>" + #JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.trustStore=/path/to/truststore" + #JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.trustStorePassword=<truststore-password>" + fi + + # jmx authentication and authorization options. By default, auth is only + # activated for remote connections but they can also be enabled for local only JMX + ## Basic file based authn & authz + JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.password.file=/etc/cassandra/jmxremote.password" + #JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.access.file=/etc/cassandra/jmxremote.access" + ## Custom auth settings which can be used as alternatives to JMX's out of the box auth utilities. + ## JAAS login modules can be used for authentication by uncommenting these two properties. + ## Cassandra ships with a LoginModule implementation - org.apache.cassandra.auth.CassandraLoginModule - + ## which delegates to the IAuthenticator configured in cassandra.yaml. See the sample JAAS configuration + ## file cassandra-jaas.config + #JVM_OPTS="$JVM_OPTS -Dcassandra.jmx.remote.login.config=CassandraLogin" + #JVM_OPTS="$JVM_OPTS -Djava.security.auth.login.config=$CASSANDRA_CONF/cassandra-jaas.config" + + ## Cassandra also ships with a helper for delegating JMX authz calls to the configured IAuthorizer, + ## uncomment this to use it. Requires one of the two authentication options to be enabled + #JVM_OPTS="$JVM_OPTS -Dcassandra.jmx.authorizer=org.apache.cassandra.auth.jmx.AuthorizationProxy" +} -# jmx authentication and authorization options. By default, auth is only -# activated for remote connections but they can also be enabled for local only JMX -## Basic file based authn & authz -JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.password.file=/etc/cassandra/jmxremote.password" -#JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.access.file=/etc/cassandra/jmxremote.access" -## Custom auth settings which can be used as alternatives to JMX's out of the box auth utilities. -## JAAS login modules can be used for authentication by uncommenting these two properties. -## Cassandra ships with a LoginModule implementation - org.apache.cassandra.auth.CassandraLoginModule - -## which delegates to the IAuthenticator configured in cassandra.yaml. See the sample JAAS configuration -## file cassandra-jaas.config -#JVM_OPTS="$JVM_OPTS -Dcassandra.jmx.remote.login.config=CassandraLogin" -#JVM_OPTS="$JVM_OPTS -Djava.security.auth.login.config=$CASSANDRA_CONF/cassandra-jaas.config" - -## Cassandra also ships with a helper for delegating JMX authz calls to the configured IAuthorizer, -## uncomment this to use it. Requires one of the two authentication options to be enabled -#JVM_OPTS="$JVM_OPTS -Dcassandra.jmx.authorizer=org.apache.cassandra.auth.jmx.AuthorizationProxy" +# If this function call is commented out, then Cassandra will start with no system properties for JMX set whatsoever. +# We will be expecting the settings in jmx_server_options and jmx_encryption_options respectively instead. +# The argument specifies the default port over which Cassandra will be available for JMX connections. +# +# If you comment out configure_jmx method call, then JMX_PORT variable will not be set, which means +# nodetool which sources this file will not see it either and port from cassandra.yaml will be parsed instead, +# if not found there either, it defaults to 7199. +# +# For security reasons, you should not expose this port to the internet. Firewall it if needed. +configure_jmx 7199 # To use mx4j, an HTML interface for JMX, add mx4j-tools.jar to the lib/ # directory. diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index b93b41fa0bb7..f8d9f354cbf0 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -1721,14 +1721,59 @@ client_encryption_options: # JMX SSL. # Similar to `client/server_encryption_options`, you can specify PEM-based # key material or customize the SSL configuration using `ssl_context_factory` in `jmx_encryption_options`. -#jmx_encryption_options: -# enabled: true -# cipher_suites: [TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256] -# accepted_protocols: [TLSv1.2,TLSv1.3,TLSv1.1] -# keystore: conf/cassandra_ssl.keystore -# keystore_password: cassandra -# truststore: conf/cassandra_ssl.truststore -# truststore_password: cassandra +# If you uncomment this section, please be sure that you comment out configure_jmx function call in cassandra-env.sh +# as it is errorneous to have JMX set by two ways, both in cassandra-env.sh and in this yaml. +#jmx_server_options: + # enabled: true + # remote: false + # jmx_port: 7199 + # + # Port used by the RMI registry when remote connections are enabled. + # To simplify firewall configs, this can be set to the same as the JMX server port (port). See CASSANDRA-7087. + # However, if ssl is enabled the same port cannot be used for both jmx and rmi so either + # pick another value for this property. Alternatively, comment out or set to 0 to use a random + # port (pre-CASSANDRA-7087 behaviour) + # rmi_port: 7199 + # + # jmx ssl options - only apply when remote connections are enabled + # + # jmx_encryption_options: + # enabled: true + # cipher_suites: [TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256] + # accepted_protocols: [TLSv1.2,TLSv1.3,TLSv1.1] + # keystore: conf/cassandra_ssl.keystore + # keystore_password: cassandra + # truststore: conf/cassandra_ssl.truststore + # truststore_password: cassandra + # + # jmx authentication and authorization options. + # authenticate: false + # + # Options for basic file based authentication & authorization + # password_file: /etc/cassandra/jmxremote.password + # access_file: /etc/cassandra/jmxremote.access + # + # Custom auth settings which can be used as alternatives to JMX's out of the box auth utilities. + # JAAS login modules can be used for authentication using this property.Cassandra ships with a + # LoginModule implementation - org.apache.cassandra.auth.CassandraLoginModule - which delegates + # to the IAuthenticator configured in cassandra.yaml. + # + # login_config_name refers to the Application Name in the JAAS configuration under which the + # desired LoginModule(s) are configured. + # The location of the JAAS config file may be set using the standard JVM mechanism, by setting + # the system property "java.security.auth.login.config". If this property is set, it's value + # will be used to locate the config file. For convenience, if the property is not already set + # at startup, a value can be supplied here via the login_config_file setting. + # + # The Application Name specified must be present in the JAAS config or an error will be thrown + # when authentication is attempted. + # See the sample JAAS configuration file conf/cassandra-jaas.config + # login_config_name: CassandraLogin + # login_config_file: conf/cassandra-jaas.config + # + # Cassandra also ships with a helper for delegating JMX authz calls to the configured IAuthorizer, + # uncomment this to use it. Requires one of the two authentication options to be enabled + # authorizer: org.apache.cassandra.auth.jmx.AuthorizationProxy # internode_compression controls whether traffic between nodes is # compressed. diff --git a/conf/cassandra_latest.yaml b/conf/cassandra_latest.yaml index 2ad1a25f0779..109811956db4 100644 --- a/conf/cassandra_latest.yaml +++ b/conf/cassandra_latest.yaml @@ -1685,14 +1685,59 @@ client_encryption_options: # JMX SSL. # Similar to `client/server_encryption_options`, you can specify PEM-based # key material or customize the SSL configuration using `ssl_context_factory` in `jmx_encryption_options`. -#jmx_encryption_options: -# enabled: true -# cipher_suites: [TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256] -# accepted_protocols: [TLSv1.2,TLSv1.3,TLSv1.1] -# keystore: conf/cassandra_ssl.keystore -# keystore_password: cassandra -# truststore: conf/cassandra_ssl.truststore -# truststore_password: cassandra +# If you uncomment this section, please be sure that you comment out configure_jmx function call in cassandra-env.sh +# as it is errorneous to have JMX set by two ways, both in cassandra-env.sh and in this yaml. +#jmx_server_options: + # enabled: true + # remote: false + # jmx_port: 7199 + # + # Port used by the RMI registry when remote connections are enabled. + # To simplify firewall configs, this can be set to the same as the JMX server port (port). See CASSANDRA-7087. + # However, if ssl is enabled the same port cannot be used for both jmx and rmi so either + # pick another value for this property. Alternatively, comment out or set to 0 to use a random + # port (pre-CASSANDRA-7087 behaviour) + # rmi_port: 7199 + # + # jmx ssl options - only apply when remote connections are enabled + # + # jmx_encryption_options: + # enabled: true + # cipher_suites: [TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256] + # accepted_protocols: [TLSv1.2,TLSv1.3,TLSv1.1] + # keystore: conf/cassandra_ssl.keystore + # keystore_password: cassandra + # truststore: conf/cassandra_ssl.truststore + # truststore_password: cassandra + # + # jmx authentication and authorization options. + # authenticate: false + # + # Options for basic file based authentication & authorization + # password_file: /etc/cassandra/jmxremote.password + # access_file: /etc/cassandra/jmxremote.access + # + # Custom auth settings which can be used as alternatives to JMX's out of the box auth utilities. + # JAAS login modules can be used for authentication using this property.Cassandra ships with a + # LoginModule implementation - org.apache.cassandra.auth.CassandraLoginModule - which delegates + # to the IAuthenticator configured in cassandra.yaml. + # + # login_config_name refers to the Application Name in the JAAS configuration under which the + # desired LoginModule(s) are configured. + # The location of the JAAS config file may be set using the standard JVM mechanism, by setting + # the system property "java.security.auth.login.config". If this property is set, it's value + # will be used to locate the config file. For convenience, if the property is not already set + # at startup, a value can be supplied here via the login_config_file setting. + # + # The Application Name specified must be present in the JAAS config or an error will be thrown + # when authentication is attempted. + # See the sample JAAS configuration file conf/cassandra-jaas.config + # login_config_name: CassandraLogin + # login_config_file: conf/cassandra-jaas.config + # + # Cassandra also ships with a helper for delegating JMX authz calls to the configured IAuthorizer, + # uncomment this to use it. Requires one of the two authentication options to be enabled + # authorizer: org.apache.cassandra.auth.jmx.AuthorizationProxy # internode_compression controls whether traffic between nodes is # compressed. diff --git a/doc/scripts/convert_yaml_to_adoc.py b/doc/scripts/convert_yaml_to_adoc.py index 061706f83589..6062f9f79954 100644 --- a/doc/scripts/convert_yaml_to_adoc.py +++ b/doc/scripts/convert_yaml_to_adoc.py @@ -49,7 +49,7 @@ 'hints_compression', 'server_encryption_options', 'client_encryption_options', - 'jmx_encryption_options', + 'jmx_server_options', 'transparent_data_encryption_options', 'hinted_handoff_disabled_datacenters', 'startup_checks', diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java index e409e975f638..8e5b0c6dd9dc 100644 --- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java +++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java @@ -146,6 +146,8 @@ public enum CassandraRelevantProperties COM_SUN_MANAGEMENT_JMXREMOTE_PASSWORD_FILE("com.sun.management.jmxremote.password.file"), /** Port number to enable JMX RMI connections - com.sun.management.jmxremote.port */ COM_SUN_MANAGEMENT_JMXREMOTE_PORT("com.sun.management.jmxremote.port"), + /** Enables SSL sockets for the RMI registry from which clients obtain the JMX connector stub */ + COM_SUN_MANAGEMENT_JMXREMOTE_REGISTRY_SSL("com.sun.management.jmxremote.registry.ssl"), /** * The port number to which the RMI connector will be bound - com.sun.management.jmxremote.rmi.port. * An Integer object that represents the value of the second argument is returned @@ -287,6 +289,10 @@ public enum CassandraRelevantProperties IO_NETTY_EVENTLOOP_THREADS("io.netty.eventLoopThreads"), IO_NETTY_TRANSPORT_ESTIMATE_SIZE_ON_SUBMIT("io.netty.transport.estimateSizeOnSubmit"), IO_NETTY_TRANSPORT_NONATIVE("io.netty.transport.noNative"), + JAVAX_NET_SSL_KEYSTORE("javax.net.ssl.keyStore"), + JAVAX_NET_SSL_KEYSTOREPASSWORD("javax.net.ssl.keyStorePassword"), + JAVAX_NET_SSL_TRUSTSTORE("javax.net.ssl.trustStore"), + JAVAX_NET_SSL_TRUSTSTOREPASSWORD("javax.net.ssl.trustStorePassword"), JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES("javax.rmi.ssl.client.enabledCipherSuites"), JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS("javax.rmi.ssl.client.enabledProtocols"), /** Java class path. */ diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 9335953d20ee..aaf6167ebb27 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -434,7 +434,8 @@ public static class SSTableConfig public EncryptionOptions.ServerEncryptionOptions server_encryption_options = new EncryptionOptions.ServerEncryptionOptions(); public EncryptionOptions client_encryption_options = new EncryptionOptions(); - public EncryptionOptions jmx_encryption_options = new EncryptionOptions(); + + public JMXServerOptions jmx_server_options; public InternodeCompression internode_compression = InternodeCompression.none; @@ -1322,7 +1323,8 @@ public enum BatchlogEndpointStrategy private static final Set<String> SENSITIVE_KEYS = new HashSet<String>() {{ add("client_encryption_options"); add("server_encryption_options"); - add("jmx_encryption_options"); + // jmx_server_options output (JMXServerOptions.toString()) doesn't + // include sensitive encryption config so no need to blocklist here }}; public static void log(Config config) diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index f80dcfd5b5fb..169fc1380f17 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -960,8 +960,19 @@ else if (conf.commitlog_segment_size.toKibibytes() < 2 * conf.max_mutation_size. if (conf.client_encryption_options != null) conf.client_encryption_options.applyConfig(); - if (conf.jmx_encryption_options != null) - conf.jmx_encryption_options.applyConfig(); + if (conf.jmx_server_options == null) + { + conf.jmx_server_options = JMXServerOptions.createParsingSystemProperties(); + } + else if (JMXServerOptions.isEnabledBySystemProperties()) + { + throw new ConfigurationException("Configure either jmx_server_options in cassandra.yaml and comment out " + + "configure_jmx function call in cassandra-env.sh or keep cassandra-env.sh " + + "to call configure_jmx function but you have to keep jmx_server_options " + + "in cassandra.yaml commented out."); + } + + conf.jmx_server_options.jmx_encryption_options.applyConfig(); if (conf.snapshot_links_per_second < 0) throw new ConfigurationException("snapshot_links_per_second must be >= 0"); @@ -1310,7 +1321,7 @@ public static void applySslContext() SSLFactory.validateSslContext("Internode messaging", conf.server_encryption_options, REQUIRED, true); SSLFactory.validateSslContext("Native transport", conf.client_encryption_options, conf.client_encryption_options.getClientAuth(), true); // For JMX SSL the validation is pretty much the same as the Native transport - SSLFactory.validateSslContext("JMX transport", conf.jmx_encryption_options, conf.jmx_encryption_options.getClientAuth(), true); + SSLFactory.validateSslContext("JMX transport", conf.jmx_server_options.jmx_encryption_options, conf.jmx_server_options.jmx_encryption_options.getClientAuth(), true); SSLFactory.initHotReloading(conf.server_encryption_options, conf.client_encryption_options, false); /* For JMX SSL, the hot reloading of the SSLContext is out of scope for CASSANDRA-18508. @@ -3663,9 +3674,9 @@ public static EncryptionOptions getNativeProtocolEncryptionOptions() return conf.client_encryption_options; } - public static EncryptionOptions getJmxEncryptionOptions() + public static JMXServerOptions getJmxServerOptions() { - return conf.jmx_encryption_options; + return conf.jmx_server_options; } @VisibleForTesting diff --git a/src/java/org/apache/cassandra/config/EncryptionOptions.java b/src/java/org/apache/cassandra/config/EncryptionOptions.java index eeac859f317c..8d922aa96e2f 100644 --- a/src/java/org/apache/cassandra/config/EncryptionOptions.java +++ b/src/java/org/apache/cassandra/config/EncryptionOptions.java @@ -470,7 +470,12 @@ public ClientAuth getClientAuth() public String[] acceptedProtocolsArray() { List<String> ap = getAcceptedProtocols(); - return ap == null ? new String[0] : ap.toArray(new String[0]); + return ap == null ? null : ap.toArray(new String[0]); + } + + public List<String> getCipherSuites() + { + return sslContextFactoryInstance == null ? null : sslContextFactoryInstance.getCipherSuites(); } public String[] cipherSuitesArray() diff --git a/src/java/org/apache/cassandra/config/JMXServerOptions.java b/src/java/org/apache/cassandra/config/JMXServerOptions.java new file mode 100644 index 000000000000..8b51d3b75688 --- /dev/null +++ b/src/java/org/apache/cassandra/config/JMXServerOptions.java @@ -0,0 +1,240 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.config; + +import java.util.HashMap; +import java.util.List; + +import org.apache.commons.lang3.StringUtils; + +import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_AUTHORIZER; +import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_LOCAL_PORT; +import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_REMOTE_LOGIN_CONFIG; +import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_REMOTE_PORT; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_ACCESS_FILE; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_AUTHENTICATE; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_PASSWORD_FILE; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_RMI_PORT; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_CIPHER_SUITES; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_PROTOCOLS; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_NEED_CLIENT_AUTH; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_NET_SSL_KEYSTORE; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_NET_SSL_KEYSTOREPASSWORD; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_NET_SSL_TRUSTSTORE; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_NET_SSL_TRUSTSTOREPASSWORD; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_SECURITY_AUTH_LOGIN_CONFIG; + +public class JMXServerOptions +{ + //jmx server settings + public final Boolean enabled; + public final Boolean remote; + public final int jmx_port; + public final int rmi_port; + public final Boolean authenticate; + + // ssl options + public final EncryptionOptions jmx_encryption_options; + + // options for using Cassandra's own authentication mechanisms + public final String login_config_name; + public final String login_config_file; + + // location for credentials file if using JVM's file-based authentication + public final String password_file; + // location of standard access file, if using JVM's file-based access control + public final String access_file; + + // classname of authorizer if using a custom authz mechanism. Usually, this will + // refer to o.a.c.auth.jmx.AuthorizationProxy which delegates to the IAuthorizer + // configured in cassandra.yaml + public final String authorizer; + + public JMXServerOptions() + { + this(true, false, 7199, 0, false, + new EncryptionOptions(), null, null, null, + null, null); + } + + public static JMXServerOptions create(boolean enabled, boolean local, int jmxPort, EncryptionOptions options) + { + return new JMXServerOptions(enabled, !local, jmxPort, 0, false, + options, null, null, null, + null, null); + } + + public static JMXServerOptions fromDescriptor(boolean enabled, boolean local, int jmxPort) + { + JMXServerOptions from = DatabaseDescriptor.getJmxServerOptions(); + return new JMXServerOptions(enabled, !local, jmxPort, jmxPort, from.authenticate, + from.jmx_encryption_options, from.login_config_name, from.login_config_file, from.password_file, + from.access_file, from.authorizer); + } + + public JMXServerOptions(Boolean enabled, + Boolean remote, + int jmxPort, + int rmiPort, + Boolean authenticate, + EncryptionOptions jmx_encryption_options, + String loginConfigName, + String loginConfigFile, + String passwordFile, + String accessFile, + String authorizer) + { + this.enabled = enabled; + this.remote = remote; + this.jmx_port = jmxPort; + this.rmi_port = rmiPort; + this.authenticate = authenticate; + this.jmx_encryption_options = jmx_encryption_options; + this.login_config_name = loginConfigName; + this.login_config_file = loginConfigFile; + this.password_file = passwordFile; + this.access_file = accessFile; + this.authorizer = authorizer; + } + + @Override + public String toString() + { + // we are not including encryption options on purpose + // as that contains credentials etc. + String jmxOptionsString; + if (jmx_encryption_options == null) + jmxOptionsString = "unspecified"; + else + jmxOptionsString = jmx_encryption_options.enabled ? "enabled" : "disabled"; + + return "JMXServerOptions{" + + "enabled=" + enabled + + ", remote=" + remote + + ", jmx_port=" + jmx_port + + ", rmi_port=" + rmi_port + + ", authenticate=" + authenticate + + ", jmx_encryption_options=" + jmx_encryption_options + + ", login_config_name='" + login_config_name + '\'' + + ", login_config_file='" + login_config_file + '\'' + + ", password_file='" + password_file + '\'' + + ", access_file='" + access_file + '\'' + + ", authorizer='" + authorizer + '\'' + + '}'; + } + + public static boolean isEnabledBySystemProperties() + { + return CASSANDRA_JMX_REMOTE_PORT.isPresent() || CASSANDRA_JMX_LOCAL_PORT.isPresent(); + } + + public static JMXServerOptions createParsingSystemProperties() + { + int jmxPort; + boolean remote; + if (CASSANDRA_JMX_REMOTE_PORT.isPresent()) + { + jmxPort = CASSANDRA_JMX_REMOTE_PORT.getInt(); + remote = true; + } + else + { + jmxPort = CASSANDRA_JMX_LOCAL_PORT.getInt(7199); + remote = false; + } + + boolean enabled = isEnabledBySystemProperties(); + + int rmiPort = COM_SUN_MANAGEMENT_JMXREMOTE_RMI_PORT.getInt(); + + boolean authenticate = COM_SUN_MANAGEMENT_JMXREMOTE_AUTHENTICATE.getBoolean(); + + String loginConfigName = CASSANDRA_JMX_REMOTE_LOGIN_CONFIG.getString(); + String loginConfigFile = JAVA_SECURITY_AUTH_LOGIN_CONFIG.getString(); + String accessFile = COM_SUN_MANAGEMENT_JMXREMOTE_ACCESS_FILE.getString(); + String passwordFile = COM_SUN_MANAGEMENT_JMXREMOTE_PASSWORD_FILE.getString(); + String authorizer = CASSANDRA_JMX_AUTHORIZER.getString(); + + // encryption options + + String keystore = JAVAX_NET_SSL_KEYSTORE.getString(); + String keystorePassword = JAVAX_NET_SSL_KEYSTOREPASSWORD.getString(); + String truststore = JAVAX_NET_SSL_TRUSTSTORE.getString(); + String truststorePassword = JAVAX_NET_SSL_TRUSTSTOREPASSWORD.getString(); + + String rawCipherSuites = COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_CIPHER_SUITES.getString(); + List<String> cipherSuites = null; + if (rawCipherSuites != null) + cipherSuites = List.of(StringUtils.split(rawCipherSuites, ",")); + + String rawSslProtocols = COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_PROTOCOLS.getString(); + List<String> acceptedProtocols = null; + if (rawSslProtocols != null) + acceptedProtocols = List.of(StringUtils.split(rawSslProtocols, ",")); + + String requireClientAuth = COM_SUN_MANAGEMENT_JMXREMOTE_SSL_NEED_CLIENT_AUTH.getString("false"); + + boolean sslEnabled = COM_SUN_MANAGEMENT_JMXREMOTE_SSL.getBoolean(); + + EncryptionOptions encryptionOptions = new EncryptionOptions(new ParameterizedClass("org.apache.cassandra.security.DefaultSslContextFactory", new HashMap<>()), + keystore, + keystorePassword, + truststore, + truststorePassword, + cipherSuites, + null, // protocol + acceptedProtocols, + null, // algorithm + null, // store_type + requireClientAuth, + false, // require endpoint verification + sslEnabled, + false, // optional + null, // max_certificate_validity_period + null); // certificate_validity_warn_threshold + + return new JMXServerOptions(enabled, remote, jmxPort, rmiPort, authenticate, + encryptionOptions, loginConfigName, loginConfigFile, passwordFile, accessFile, + authorizer); + } + + + /** + * Sets the following JMX system properties. + * <pre> + * com.sun.management.jmxremote.ssl=true + * javax.rmi.ssl.client.enabledCipherSuites=<applicable cipher suites provided in the configuration> + * javax.rmi.ssl.client.enabledProtocols=<applicable protocols provided in the configuration> + * </pre> + * + * @param acceptedProtocols for the SSL communication + * @param cipherSuites for the SSL communication + */ + public static void setJmxSystemProperties(List<String> acceptedProtocols, List<String> cipherSuites) + { + COM_SUN_MANAGEMENT_JMXREMOTE_SSL.setBoolean(true); + if (acceptedProtocols != null) + JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS.setString(StringUtils.join(acceptedProtocols, ",")); + + if (cipherSuites != null) + JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES.setString(StringUtils.join(cipherSuites, ",")); + } +} diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java index ef6c689edf47..360c26bdcc33 100644 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@ -48,6 +48,7 @@ import org.apache.cassandra.concurrent.ScheduledExecutors; import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.JMXServerOptions; import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; @@ -91,8 +92,6 @@ import static java.util.concurrent.TimeUnit.NANOSECONDS; import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_FOREGROUND; -import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_LOCAL_PORT; -import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_REMOTE_PORT; import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_PID_FILE; import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_PORT; import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_CLASS_PATH; @@ -153,42 +152,24 @@ private void maybeInitJmx() // then the JVM agent will have already started up a default JMX connector // server. This behaviour is deprecated, but some clients may be relying // on it, so log a warning and skip setting up the server with the settings - // as configured in cassandra-env.(sh|ps1) + // as configured in cassandra.yaml or cassandra-env.sh. // See: CASSANDRA-11540 & CASSANDRA-11725 if (COM_SUN_MANAGEMENT_JMXREMOTE_PORT.isPresent()) { - logger.warn("JMX settings in cassandra-env.sh have been bypassed as the JMX connector server is " + - "already initialized. Please refer to cassandra-env.(sh|ps1) for JMX configuration info"); + logger.warn("JMX settings in cassandra.yaml or cassandra-env.sh have been bypassed as the JMX connector server is " + + "already initialized. Please refer to cassandra.yaml or cassandra-env.sh for JMX configuration info"); return; } JAVA_RMI_SERVER_RANDOM_ID.setBoolean(true); - // If a remote port has been specified then use that to set up a JMX - // connector server which can be accessed remotely. Otherwise, look - // for the local port property and create a server which is bound - // only to the loopback address. Auth options are applied to both - // remote and local-only servers, but currently SSL is only - // available for remote. - // If neither is remote nor local port is set in cassandra-env.(sh|ps) - // then JMX is effectively disabled. - boolean localOnly = false; - String jmxPort = CASSANDRA_JMX_REMOTE_PORT.getString(); - - if (jmxPort == null) - { - localOnly = true; - jmxPort = CASSANDRA_JMX_LOCAL_PORT.getString(); - } - - if (jmxPort == null) + JMXServerOptions jmxServerOptions = DatabaseDescriptor.getJmxServerOptions(); + if (!jmxServerOptions.enabled) return; try { - jmxServer = JMXServerUtils.createJMXServer(Integer.parseInt(jmxPort), localOnly); - if (jmxServer == null) - return; + jmxServer = JMXServerUtils.createJMXServer(jmxServerOptions); } catch (IOException e) { diff --git a/src/java/org/apache/cassandra/service/StartupChecks.java b/src/java/org/apache/cassandra/service/StartupChecks.java index 91a95a4e1069..a16864cc2ae9 100644 --- a/src/java/org/apache/cassandra/service/StartupChecks.java +++ b/src/java/org/apache/cassandra/service/StartupChecks.java @@ -54,6 +54,7 @@ import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.config.Config; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.JMXServerOptions; import org.apache.cassandra.config.StartupChecksOptions; import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.cql3.UntypedResultSet; @@ -74,7 +75,6 @@ import org.apache.cassandra.utils.JavaUtils; import org.apache.cassandra.utils.NativeLibrary; -import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_LOCAL_PORT; import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_PORT; import static org.apache.cassandra.config.CassandraRelevantProperties.IGNORE_KERNEL_BUG_1057843_CHECK; import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_VERSION; @@ -312,17 +312,21 @@ public void execute(StartupChecksOptions options) { if (options.isDisabled(getStartupCheckType())) return; - String jmxPort = CassandraRelevantProperties.CASSANDRA_JMX_REMOTE_PORT.getString(); - if (jmxPort == null) + + JMXServerOptions jmxServerOptions = DatabaseDescriptor.getJmxServerOptions(); + if (!jmxServerOptions.enabled) + { + logger.warn("JMX connection server is not enabled for either local or remote connections. " + + "Please see jmx_server_options in cassandra.yaml for more info"); + } + if (!jmxServerOptions.remote) { - logger.warn("JMX is not enabled to receive remote connections. Please see cassandra-env.sh for more info."); - jmxPort = CassandraRelevantProperties.CASSANDRA_JMX_LOCAL_PORT.toString(); - if (jmxPort == null) - logger.error(CASSANDRA_JMX_LOCAL_PORT.getKey() + " missing from cassandra-env.sh, unable to start local JMX service."); + logger.warn("JMX is not enabled to receive remote connections. " + + "Please see jmx_server_options in cassandra.yaml for more info."); } else { - logger.info("JMX is enabled to receive remote connections on port: {}", jmxPort); + logger.info("JMX is enabled to receive remote connections on port: {}", jmxServerOptions.jmx_port); } } }; diff --git a/src/java/org/apache/cassandra/utils/JMXServerUtils.java b/src/java/org/apache/cassandra/utils/JMXServerUtils.java index 43221f28bfb9..be8f943d2b45 100644 --- a/src/java/org/apache/cassandra/utils/JMXServerUtils.java +++ b/src/java/org/apache/cassandra/utils/JMXServerUtils.java @@ -33,7 +33,6 @@ import java.rmi.NotBoundException; import java.rmi.Remote; import java.rmi.RemoteException; -import java.rmi.registry.Registry; import java.rmi.server.RMIClientSocketFactory; import java.rmi.server.RMIServerSocketFactory; import java.rmi.server.UnicastRemoteObject; @@ -49,20 +48,17 @@ import javax.security.auth.Subject; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Strings; import com.google.common.collect.ImmutableMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.cassandra.auth.jmx.AuthenticationProxy; -import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.config.JMXServerOptions; +import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.utils.jmx.DefaultJmxSocketFactory; -import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_AUTHORIZER; -import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_REMOTE_LOGIN_CONFIG; -import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_ACCESS_FILE; -import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_AUTHENTICATE; -import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_PASSWORD_FILE; -import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_RMI_PORT; import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_RMI_SERVER_HOSTNAME; public class JMXServerUtils @@ -74,13 +70,12 @@ public class JMXServerUtils * inaccessable. */ @VisibleForTesting - public static JMXConnectorServer createJMXServer(int port, String hostname, boolean local) - throws IOException + public static JMXConnectorServer createJMXServer(JMXServerOptions options, String hostname) throws IOException { Map<String, Object> env = new HashMap<>(); InetAddress serverAddress = null; - if (local) + if (!options.remote) { serverAddress = InetAddress.getLoopbackAddress(); JAVA_RMI_SERVER_HOSTNAME.setString(serverAddress.getHostAddress()); @@ -88,18 +83,18 @@ public static JMXConnectorServer createJMXServer(int port, String hostname, bool // Configure the RMI client & server socket factories, including SSL config. // CASSANDRA-18508: Make JMX SSL to be configured in cassandra.yaml - env.putAll(configureJmxSocketFactories(serverAddress, local)); + env.putAll(configureJmxSocketFactories(serverAddress, options)); // configure the RMI registry - Registry registry = new JmxRegistry(port, - (RMIClientSocketFactory) env.get(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE), - (RMIServerSocketFactory) env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE), - "jmxrmi"); + JmxRegistry registry = new JmxRegistry(options.jmx_port, + (RMIClientSocketFactory) env.get(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE), + (RMIServerSocketFactory) env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE), + "jmxrmi"); // Configure authn, using a JMXAuthenticator which either wraps a set log LoginModules configured // via a JAAS configuration entry, or one which delegates to the standard file based authenticator. // Authn is disabled if com.sun.management.jmxremote.authenticate=false - env.putAll(configureJmxAuthentication()); + env.putAll(configureJmxAuthentication(options)); // Secure credential passing to avoid deserialization attacks env.putAll(configureSecureCredentials()); @@ -107,7 +102,7 @@ public static JMXConnectorServer createJMXServer(int port, String hostname, bool // If not, but a location for the standard access file is set in system properties, the // return value is null, and an entry is added to the env map detailing that location // If neither method is specified, no access control is applied - MBeanServerForwarder authzProxy = configureJmxAuthorization(env); + MBeanServerForwarder authzProxy = configureJmxAuthorization(options, env); // Mark the JMX server as a permanently exported object. This allows the JVM to exit with the // server running and also exempts it from the distributed GC scheduler which otherwise would @@ -121,7 +116,7 @@ public static JMXConnectorServer createJMXServer(int port, String hostname, bool // Set the port used to create subsequent connections to exported objects over RMI. This simplifies // configuration in firewalled environments, but it can't be used in conjuction with SSL sockets. // See: CASSANDRA-7087 - int rmiPort = COM_SUN_MANAGEMENT_JMXREMOTE_RMI_PORT.getInt(); + int rmiPort = options.rmi_port; // We create the underlying RMIJRMPServerImpl so that we can manually bind it to the registry, // rather then specifying a binding address in the JMXServiceURL and letting it be done automatically @@ -142,14 +137,14 @@ public static JMXConnectorServer createJMXServer(int port, String hostname, bool jmxServer.setMBeanServerForwarder(authzProxy); jmxServer.start(); - ((JmxRegistry)registry).setRemoteServerStub(server.toStub()); - logJmxServiceUrl(serverAddress, port); + registry.setRemoteServerStub(server.toStub()); + logJmxServiceUrl(serverAddress, options.jmx_port); return jmxServer; } - public static JMXConnectorServer createJMXServer(int port, boolean local) throws IOException + public static JMXConnectorServer createJMXServer(JMXServerOptions serverOptions) throws IOException { - return createJMXServer(port, null, local); + return createJMXServer(serverOptions, null); } private static Map<String, Object> configureSecureCredentials() @@ -159,10 +154,10 @@ private static Map<String, Object> configureSecureCredentials() return env; } - private static Map<String, Object> configureJmxAuthentication() + private static Map<String, Object> configureJmxAuthentication(JMXServerOptions options) { Map<String, Object> env = new HashMap<>(); - if (!COM_SUN_MANAGEMENT_JMXREMOTE_AUTHENTICATE.getBoolean()) + if (!options.authenticate) return env; // If authentication is enabled, initialize the appropriate JMXAuthenticator @@ -176,14 +171,30 @@ private static Map<String, Object> configureJmxAuthentication() // before creating the authenticator. If no password file has been // explicitly set, it's read from the default location // $JAVA_HOME/lib/management/jmxremote.password - String configEntry = CASSANDRA_JMX_REMOTE_LOGIN_CONFIG.getString(); + String configEntry = options.login_config_name; if (configEntry != null) { + if (Strings.isNullOrEmpty(CassandraRelevantProperties.JAVA_SECURITY_AUTH_LOGIN_CONFIG.getString())) + { + if (Strings.isNullOrEmpty(options.login_config_file)) + { + throw new ConfigurationException(String.format("Login config name %s specified for JMX auth, but no " + + "configuration is available. Please set config " + + "location in cassandra.yaml or with the " + + "'%s' system property", + configEntry, + CassandraRelevantProperties.JAVA_SECURITY_AUTH_LOGIN_CONFIG.getKey())); + } + else + { + CassandraRelevantProperties.JAVA_SECURITY_AUTH_LOGIN_CONFIG.setString(options.login_config_file); + } + } env.put(JMXConnectorServer.AUTHENTICATOR, new AuthenticationProxy(configEntry)); } else { - String passwordFile = COM_SUN_MANAGEMENT_JMXREMOTE_PASSWORD_FILE.getString(); + String passwordFile = options.password_file; if (passwordFile != null) { // stash the password file location where JMXPluggableAuthenticator expects it @@ -195,14 +206,14 @@ private static Map<String, Object> configureJmxAuthentication() return env; } - private static MBeanServerForwarder configureJmxAuthorization(Map<String, Object> env) + private static MBeanServerForwarder configureJmxAuthorization(JMXServerOptions options, Map<String, Object> env) { // If a custom authz proxy is supplied (Cassandra ships with AuthorizationProxy, which // delegates to its own role based IAuthorizer), then instantiate and return one which // can be set as the JMXConnectorServer's MBeanServerForwarder. // If no custom proxy is supplied, check system properties for the location of the // standard access file & stash it in env - String authzProxyClass = CASSANDRA_JMX_AUTHORIZER.getString(); + String authzProxyClass = options.authorizer; if (authzProxyClass != null) { final InvocationHandler handler = FBUtilities.construct(authzProxyClass, "JMX authz proxy"); @@ -213,7 +224,7 @@ private static MBeanServerForwarder configureJmxAuthorization(Map<String, Object } else { - String accessFile = COM_SUN_MANAGEMENT_JMXREMOTE_ACCESS_FILE.getString(); + String accessFile = options.access_file; if (accessFile != null) { env.put("jmx.remote.x.access.file", accessFile); @@ -227,18 +238,16 @@ private static MBeanServerForwarder configureJmxAuthorization(Map<String, Object * for configuring this. * * @param serverAddress the JMX server is bound to - * @param localOnly {@code true} if the JMX server only allows local connections; {@code false} if the JMX server - * allows the remote connections. + * @param serverOptions options for JMX server, either from {@code cassandra.yaml} or parsed as system properties from {@code cassandra-env.sh}. * @return Map<String, Object@gt; containing {@code jmx.remote.rmi.client.socket.factory}, {@code jmx.remote.rmi.server.socket.factory} * and {@code com.sun.jndi.rmi.factory.socket} properties for the client and server socket factories. * @throws SSLException if it fails to configure the socket factories with the given input - * * @see DefaultJmxSocketFactory */ @VisibleForTesting - public static Map<String, Object> configureJmxSocketFactories(InetAddress serverAddress, boolean localOnly) throws SSLException + public static Map<String, Object> configureJmxSocketFactories(InetAddress serverAddress, JMXServerOptions serverOptions) throws SSLException { - return new DefaultJmxSocketFactory().configure(serverAddress, localOnly, DatabaseDescriptor.getJmxEncryptionOptions()); + return new DefaultJmxSocketFactory().configure(serverAddress, serverOptions, serverOptions.jmx_encryption_options); } @VisibleForTesting diff --git a/src/java/org/apache/cassandra/utils/jmx/AbstractJmxSocketFactory.java b/src/java/org/apache/cassandra/utils/jmx/AbstractJmxSocketFactory.java index 6ee390ee93a7..3990c64ee7d4 100644 --- a/src/java/org/apache/cassandra/utils/jmx/AbstractJmxSocketFactory.java +++ b/src/java/org/apache/cassandra/utils/jmx/AbstractJmxSocketFactory.java @@ -24,19 +24,11 @@ import javax.net.ssl.SSLContext; import javax.net.ssl.SSLException; -import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.cassandra.config.EncryptionOptions; -import org.apache.cassandra.exceptions.ConfigurationException; - -import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL; -import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_CIPHER_SUITES; -import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_PROTOCOLS; -import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_NEED_CLIENT_AUTH; -import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES; -import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS; +import org.apache.cassandra.config.JMXServerOptions; /** * Abstracts out the most common workflow in setting up the SSL client and server socket factorires for JMX. @@ -66,8 +58,7 @@ abstract public class AbstractJmxSocketFactory * </pre> * * @param serverAddress the JMX server is bound to - * @param localOnly {@code true} if the JMX server only allows local connections; {@code false} if the JMX server - * allows the remote connections. + * @param serverOptions JMX server options * @param jmxEncryptionOptions {@link EncryptionOptions} used for the SSL configuration in case of the remote * connections. Could be {@code null} if system properties are * used instead as per <a href="https://docs.oracle.com/en/java/javase/17/management/monitoring-and-management-using-jmx-technology.html#GUID-F08985BB-629A-4FBF-A0CB-8762DF7590E0">Java Documentation</a> @@ -75,67 +66,30 @@ abstract public class AbstractJmxSocketFactory * and {@code com.sun.jndi.rmi.factory.socket} properties for the client and server socket factories. * @throws SSLException if it fails to configure the socket factories with the given input */ - public Map<String, Object> configure(InetAddress serverAddress, boolean localOnly, + public Map<String, Object> configure(InetAddress serverAddress, + JMXServerOptions serverOptions, EncryptionOptions jmxEncryptionOptions) throws SSLException { Map<String, Object> env = new HashMap<>(); - boolean jmxRemoteSslSystemConfigProvided = COM_SUN_MANAGEMENT_JMXREMOTE_SSL.getBoolean(); + // We check for the enabled jmx_encryption_options here because in case of no configuration provided in cassandra.yaml // it will default to empty/non-null encryption options. Hence, we consider it set only if 'enabled' flag is set to true - boolean jmxEncryptionOptionsProvided = jmxEncryptionOptions != null - && jmxEncryptionOptions.getEnabled() != null - && jmxEncryptionOptions.getEnabled(); + boolean jmxEncryptionOptionsProvided = jmxEncryptionOptions != null && jmxEncryptionOptions.getEnabled() != null && jmxEncryptionOptions.getEnabled(); - if (jmxRemoteSslSystemConfigProvided && jmxEncryptionOptionsProvided) + if (jmxEncryptionOptionsProvided) { - throw new ConfigurationException("Please specify JMX SSL configuration in either cassandra-env.sh or " + - "cassandra.yaml, not in both locations"); - } - - boolean requireClientAuth = false; - String[] ciphers = null; - String[] protocols = null; - SSLContext sslContext = null; - - if (jmxRemoteSslSystemConfigProvided) - { - logger.info("Enabling JMX SSL using environment file properties"); - logger.warn("Consider using the jmx_encryption_options section of cassandra.yaml instead to prevent " + - "sensitive information being exposed"); - requireClientAuth = COM_SUN_MANAGEMENT_JMXREMOTE_SSL_NEED_CLIENT_AUTH.getBoolean(); - String protocolList = COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_PROTOCOLS.getString(); - if (protocolList != null) - { - JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS.setString(protocolList); - protocols = StringUtils.split(protocolList, ','); - } - - String cipherList = COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_CIPHER_SUITES.getString(); - if (cipherList != null) - { - JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES.setString(cipherList); - ciphers = StringUtils.split(cipherList, ','); - } - configureSslClientSocketFactory(env, serverAddress); - configureSslServerSocketFactory(env, serverAddress, ciphers, protocols, requireClientAuth); - } - else if (jmxEncryptionOptionsProvided) - { - logger.info("Enabling JMX SSL using jmx_encryption_options from cassandra.yaml"); - // Here we can continue to use the SslRMIClientSocketFactory for client sockets. - // However, we should still set System properties for cipher_suites and enabled_protocols - // to have the same behavior as cassandra-env.sh based JMX SSL settings - setJmxSystemProperties(jmxEncryptionOptions); - - requireClientAuth = jmxEncryptionOptions.getClientAuth() == EncryptionOptions.ClientAuth.REQUIRED; - ciphers = jmxEncryptionOptions.cipherSuitesArray(); - protocols = jmxEncryptionOptions.acceptedProtocolsArray(); - sslContext = jmxEncryptionOptions.sslContextFactoryInstance - .createJSSESslContext(jmxEncryptionOptions.getClientAuth()); + if (jmxEncryptionOptions.getEnabled()) + JMXServerOptions.setJmxSystemProperties(jmxEncryptionOptions.getAcceptedProtocols(), jmxEncryptionOptions.getCipherSuites()); + + logger.info("Enabling JMX SSL using jmx_encryption_options"); + boolean requireClientAuth = jmxEncryptionOptions.getClientAuth() == EncryptionOptions.ClientAuth.REQUIRED; + String[] ciphers = jmxEncryptionOptions.cipherSuitesArray(); + String[] protocols = jmxEncryptionOptions.acceptedProtocolsArray(); + SSLContext sslContext = jmxEncryptionOptions.sslContextFactoryInstance.createJSSESslContext(jmxEncryptionOptions.getClientAuth()); configureSslClientSocketFactory(env, serverAddress); configureSslServerSocketFactory(env, serverAddress, ciphers, protocols, requireClientAuth, sslContext); } - else if (localOnly) + else if (!serverOptions.remote) { configureLocalSocketFactories(env, serverAddress); } @@ -159,19 +113,6 @@ else if (localOnly) */ abstract public void configureSslClientSocketFactory(Map<String, Object> env, InetAddress serverAddress); - /** - * Configures SSL based server socket factory based on system config for key/trust stores. - * - * @param env output param containing the configured socket factories - * @param serverAddress the JMX server is bound to - * @param enabledCipherSuites for the SSL communication - * @param enabledProtocols for the SSL communication - * @param needClientAuth {@code true} if it requires the client-auth; {@code false} otherwise - */ - abstract public void configureSslServerSocketFactory(Map<String, Object> env, InetAddress serverAddress, - String[] enabledCipherSuites, String[] enabledProtocols, - boolean needClientAuth); - /** * Configures SSL based server socket factory based on custom SSLContext. * @@ -185,24 +126,4 @@ abstract public void configureSslServerSocketFactory(Map<String, Object> env, In abstract public void configureSslServerSocketFactory(Map<String, Object> env, InetAddress serverAddress, String[] enabledCipherSuites, String[] enabledProtocols, boolean needClientAuth, SSLContext sslContext); - - /** - * Sets the following JMX system properties. - * <pre> - * com.sun.management.jmxremote.ssl=true - * javax.rmi.ssl.client.enabledCipherSuites=<applicable cipher suites provided in the configuration> - * javax.rmi.ssl.client.enabledProtocols=<applicable protocols provided in the configuration> - * </pre> - * - * @param jmxEncryptionOptions for the SSL communication - */ - private void setJmxSystemProperties(EncryptionOptions jmxEncryptionOptions) - { - COM_SUN_MANAGEMENT_JMXREMOTE_SSL.setBoolean(true); - if (jmxEncryptionOptions.getAcceptedProtocols() != null) - JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS.setString(StringUtils.join(jmxEncryptionOptions.getAcceptedProtocols(), ",")); - - if (jmxEncryptionOptions.cipherSuitesArray() != null) - JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES.setString(StringUtils.join(jmxEncryptionOptions.cipherSuitesArray(), ",")); - } } diff --git a/src/java/org/apache/cassandra/utils/jmx/DefaultJmxSocketFactory.java b/src/java/org/apache/cassandra/utils/jmx/DefaultJmxSocketFactory.java index 35cc92b731e3..45d4df72ddab 100644 --- a/src/java/org/apache/cassandra/utils/jmx/DefaultJmxSocketFactory.java +++ b/src/java/org/apache/cassandra/utils/jmx/DefaultJmxSocketFactory.java @@ -53,15 +53,6 @@ public void configureSslClientSocketFactory(Map<String, Object> env, InetAddress env.put("com.sun.jndi.rmi.factory.socket", clientFactory); } - @Override - public void configureSslServerSocketFactory(Map<String, Object> env, InetAddress serverAddress, String[] enabledCipherSuites, - String[] enabledProtocols, boolean needClientAuth) - { - SslRMIServerSocketFactory serverFactory = new SslRMIServerSocketFactory(enabledCipherSuites, enabledProtocols, needClientAuth); - env.put(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE, serverFactory); - logJmxSslConfig(serverFactory); - } - @Override public void configureSslServerSocketFactory(Map<String, Object> env, InetAddress serverAddress, String[] enabledCipherSuites, String[] enabledProtocols, boolean needClientAuth, SSLContext sslContext) diff --git a/test/conf/cassandra-jmx-disabled-sslconfig.yaml b/test/conf/cassandra-jmx-disabled-sslconfig.yaml index 2b6338992668..12baefb75f88 100644 --- a/test/conf/cassandra-jmx-disabled-sslconfig.yaml +++ b/test/conf/cassandra-jmx-disabled-sslconfig.yaml @@ -48,89 +48,91 @@ seed_provider: - seeds: "127.0.0.1:7012" endpoint_snitch: org.apache.cassandra.locator.SimpleSnitch dynamic_snitch: true -jmx_encryption_options: - enabled: false - cipher_suites: [TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256] - accepted_protocols: [TLSv1.2,TLSv1.3,TLSv1.1] - ssl_context_factory: - class_name: org.apache.cassandra.security.PEMBasedSslContextFactory - parameters: - private_key: | - -----BEGIN ENCRYPTED PRIVATE KEY----- - MIIE6jAcBgoqhkiG9w0BDAEDMA4ECOWqSzq5PBIdAgIFxQSCBMjXsCK30J0aT3J/ - g5kcbmevTOY1pIhJGbf5QYYrMUPiuDK2ydxIbiPzoTE4/S+OkCeHhlqwn/YydpBl - xgjZZ1Z5rLJHO27d2biuESqanDiBVXYuVmHmaifRnFy0uUTFkStB5mjVZEiJgO29 - L83hL60uWru71EVuVriC2WCfmZ/EXp6wyYszOqCFQ8Quk/rDO6XuaBl467MJbx5V - sucGT6E9XKNd9hB14/Izb2jtVM5kqKxoiHpz1na6yhEYJiE5D1uOonznWjBnjwB/ - f0x+acpDfVDoJKTlRdz+DEcbOF7mb9lBVVjP6P/AAsmQzz6JKwHjvCrjYfQmyyN8 - RI4KRQnWgm4L3dtByLqY8HFU4ogisCMCgI+hZQ+OKMz/hoRO540YGiPcTRY3EOUR - 0bd5JxU6tCJDMTqKP9aSL2KmLoiLowdMkSPz7TCzLsZ2bGJemuCfpAs4XT1vXCHs - evrUbOnh8et1IA8mZ9auThfqsZtNagJLEXA6hWIKp1FfVL3Q49wvMKZt4eTn/zwU - tLL0m5yPo6/HAaOA3hbm/oghZS0dseshXl7PZrmZQtvYnIvjyoxEL7ducYDQCDP6 - wZ7Nzyh1QZAauSS15hl3vLFRZCA9hWAVgwQAviTvhB342O0i9qI7TQkcHk+qcTPN - K+iGNbFZ8ma1izXNKSJ2PgI/QqFNIeJWvZrb9PhJRmaZVsTJ9fERm1ewpebZqkVv - zMqMhlKgx9ggAaSKgnGZkwXwB6GrSbbzUrwRCKm3FieD1QE4VVYevaadVUU75GG5 - mrFKorJEH7kFZlic8OTjDksYnHbcgU36XZrGEXa2+ldVeGKL3CsXWciaQRcJg8yo - WQDjZpcutGI0eMJWCqUkv8pYZC2/wZU4htCve5nVJUU4t9uuo9ex7lnwlLWPvheQ - jUBMgzSRsZ+zwaIusvufAAxiKK/cJm4ubZSZPIjBbfd4U7VPxtirP4Accydu7EK6 - eG/MZwtAMFNJxfxUR+/aYzJU/q1ePw7fWVHrpt58t/22CX2SJBEiUGmSmuyER4Ny - DPw6d6mhvPUS1jRhIZ9A81ht8MOX7VL5uVp307rt7o5vRpV1mo0iPiRHzGscMpJn - AP36klEAUNTf0uLTKZa7KHiwhn5iPmsCrENHkOKJjxhRrqHjD2wy3YHs3ow2voyY - Ua4Cids+c1hvRkNEDGNHm4+rKGFOGOsG/ZU7uj/6gflO4JXxNGiyTLflqMdWBvow - Zd7hk1zCaGAAn8nZ0hPweGxQ4Q30I9IBZrimGxB0vjiUqNio9+qMf33dCHFJEuut - ZGJMaUGVaPhXQcTy4uD5hzsPZV5xcsU4H3vBYyBcZgrusJ6OOgkuZQaU7p8rWQWr - bUEVbXuZdwEmxsCe7H/vEVv5+aA4sF4kWnMMFL7/LIYaiEzkTqdJlRv/KyJJgcAH - hg2BvR3XTAq8wiX0C98CdmTbsx2eyQdj5tCU606rEohFLKUxWkJYAKxCiUbxGGpI - RheVmxkef9ErxJiq7hsAsGrSJvMtJuDKIasnD14SOEwD/7jRAq6WdL9VLpxtzlOw - pWnIl8kUCO3WoaG9Jf+ZTIv2hnxJhaSzYrdXzGPNnaWKhBlwnXJRvQEdrIxZOimP - FujZhqbKUDbYAcqTkoQ= - -----END ENCRYPTED PRIVATE KEY----- - -----BEGIN CERTIFICATE----- - MIIDkTCCAnmgAwIBAgIETxH5JDANBgkqhkiG9w0BAQsFADB5MRAwDgYDVQQGEwdV - bmtub3duMRAwDgYDVQQIEwdVbmtub3duMRAwDgYDVQQHEwdVbmtub3duMRAwDgYD - VQQKEwdVbmtub3duMRQwEgYDVQQLDAtzc2xfdGVzdGluZzEZMBcGA1UEAxMQQXBh - Y2hlIENhc3NhbmRyYTAeFw0xNjAzMTgyMTI4MDJaFw0xNjA2MTYyMTI4MDJaMHkx - EDAOBgNVBAYTB1Vua25vd24xEDAOBgNVBAgTB1Vua25vd24xEDAOBgNVBAcTB1Vu - a25vd24xEDAOBgNVBAoTB1Vua25vd24xFDASBgNVBAsMC3NzbF90ZXN0aW5nMRkw - FwYDVQQDExBBcGFjaGUgQ2Fzc2FuZHJhMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8A - MIIBCgKCAQEAjkmVX/HS49cS8Hn6o26IGwMIcEV3d7ZhH0GNcx8rnSRd10dU9F6d - ugSjbwGFMcWUQzYNejN6az0Wb8JIQyXRPTWjfgaWTyVGr0bGTnxg6vwhzfI/9jzy - q59xv29OuSY1dxmY31f0pZ9OOw3mabWksjoO2TexfKoxqsRHJ8PrM1f8E84Z4xo2 - TJXGzpuIxRkAJ+sVDqKEAhrKAfRYMSgdJ7zRt8VXv9ngjX20uA2m092NcH0Kmeto - TmuWUtK8E/qcN7ULN8xRWNUn4hu6mG6mayk4XliGRqI1VZupqh+MgNqHznuTd0bA - YrQsFPw9HaZ2hvVnJffJ5l7njAekZNOL+wIDAQABoyEwHzAdBgNVHQ4EFgQUcdiD - N6aylI91kAd34Hl2AzWY51QwDQYJKoZIhvcNAQELBQADggEBAG9q29ilUgCWQP5v - iHkZHj10gXGEoMkdfrPBf8grC7dpUcaw1Qfku/DJ7kPvMALeEsmFDk/t78roeNbh - IYBLJlzI1HZN6VPtpWQGsqxltAy5XN9Xw9mQM/tu70ShgsodGmE1UoW6eE5+/GMv - 6Fg+zLuICPvs2cFNmWUvukN5LW146tJSYCv0Q/rCPB3m9dNQ9pBxrzPUHXw4glwG - qGnGddXmOC+tSW5lDLLG1BRbKv4zxv3UlrtIjqlJtZb/sQMT6WtG2ihAz7SKOBHa - HOWUwuPTetWIuJCKP7P4mWWtmSmjLy+BFX5seNEngn3RzJ2L8uuTJQ/88OsqgGru - n3MVF9w= - -----END CERTIFICATE----- - private_key_password: "cassandra" - trusted_certificates: | - -----BEGIN CERTIFICATE----- - MIIDkTCCAnmgAwIBAgIETxH5JDANBgkqhkiG9w0BAQsFADB5MRAwDgYDVQQGEwdV - bmtub3duMRAwDgYDVQQIEwdVbmtub3duMRAwDgYDVQQHEwdVbmtub3duMRAwDgYD - VQQKEwdVbmtub3duMRQwEgYDVQQLDAtzc2xfdGVzdGluZzEZMBcGA1UEAxMQQXBh - Y2hlIENhc3NhbmRyYTAeFw0xNjAzMTgyMTI4MDJaFw0xNjA2MTYyMTI4MDJaMHkx - EDAOBgNVBAYTB1Vua25vd24xEDAOBgNVBAgTB1Vua25vd24xEDAOBgNVBAcTB1Vu - a25vd24xEDAOBgNVBAoTB1Vua25vd24xFDASBgNVBAsMC3NzbF90ZXN0aW5nMRkw - FwYDVQQDExBBcGFjaGUgQ2Fzc2FuZHJhMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8A - MIIBCgKCAQEAjkmVX/HS49cS8Hn6o26IGwMIcEV3d7ZhH0GNcx8rnSRd10dU9F6d - ugSjbwGFMcWUQzYNejN6az0Wb8JIQyXRPTWjfgaWTyVGr0bGTnxg6vwhzfI/9jzy - q59xv29OuSY1dxmY31f0pZ9OOw3mabWksjoO2TexfKoxqsRHJ8PrM1f8E84Z4xo2 - TJXGzpuIxRkAJ+sVDqKEAhrKAfRYMSgdJ7zRt8VXv9ngjX20uA2m092NcH0Kmeto - TmuWUtK8E/qcN7ULN8xRWNUn4hu6mG6mayk4XliGRqI1VZupqh+MgNqHznuTd0bA - YrQsFPw9HaZ2hvVnJffJ5l7njAekZNOL+wIDAQABoyEwHzAdBgNVHQ4EFgQUcdiD - N6aylI91kAd34Hl2AzWY51QwDQYJKoZIhvcNAQELBQADggEBAG9q29ilUgCWQP5v - iHkZHj10gXGEoMkdfrPBf8grC7dpUcaw1Qfku/DJ7kPvMALeEsmFDk/t78roeNbh - IYBLJlzI1HZN6VPtpWQGsqxltAy5XN9Xw9mQM/tu70ShgsodGmE1UoW6eE5+/GMv - 6Fg+zLuICPvs2cFNmWUvukN5LW146tJSYCv0Q/rCPB3m9dNQ9pBxrzPUHXw4glwG - qGnGddXmOC+tSW5lDLLG1BRbKv4zxv3UlrtIjqlJtZb/sQMT6WtG2ihAz7SKOBHa - HOWUwuPTetWIuJCKP7P4mWWtmSmjLy+BFX5seNEngn3RzJ2L8uuTJQ/88OsqgGru - n3MVF9w= - -----END CERTIFICATE----- +jmx_server_options: + enabled: true + jmx_encryption_options: + enabled: false + cipher_suites: [TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256] + accepted_protocols: [TLSv1.2,TLSv1.3,TLSv1.1] + ssl_context_factory: + class_name: org.apache.cassandra.security.PEMBasedSslContextFactory + parameters: + private_key: | + -----BEGIN ENCRYPTED PRIVATE KEY----- + MIIE6jAcBgoqhkiG9w0BDAEDMA4ECOWqSzq5PBIdAgIFxQSCBMjXsCK30J0aT3J/ + g5kcbmevTOY1pIhJGbf5QYYrMUPiuDK2ydxIbiPzoTE4/S+OkCeHhlqwn/YydpBl + xgjZZ1Z5rLJHO27d2biuESqanDiBVXYuVmHmaifRnFy0uUTFkStB5mjVZEiJgO29 + L83hL60uWru71EVuVriC2WCfmZ/EXp6wyYszOqCFQ8Quk/rDO6XuaBl467MJbx5V + sucGT6E9XKNd9hB14/Izb2jtVM5kqKxoiHpz1na6yhEYJiE5D1uOonznWjBnjwB/ + f0x+acpDfVDoJKTlRdz+DEcbOF7mb9lBVVjP6P/AAsmQzz6JKwHjvCrjYfQmyyN8 + RI4KRQnWgm4L3dtByLqY8HFU4ogisCMCgI+hZQ+OKMz/hoRO540YGiPcTRY3EOUR + 0bd5JxU6tCJDMTqKP9aSL2KmLoiLowdMkSPz7TCzLsZ2bGJemuCfpAs4XT1vXCHs + evrUbOnh8et1IA8mZ9auThfqsZtNagJLEXA6hWIKp1FfVL3Q49wvMKZt4eTn/zwU + tLL0m5yPo6/HAaOA3hbm/oghZS0dseshXl7PZrmZQtvYnIvjyoxEL7ducYDQCDP6 + wZ7Nzyh1QZAauSS15hl3vLFRZCA9hWAVgwQAviTvhB342O0i9qI7TQkcHk+qcTPN + K+iGNbFZ8ma1izXNKSJ2PgI/QqFNIeJWvZrb9PhJRmaZVsTJ9fERm1ewpebZqkVv + zMqMhlKgx9ggAaSKgnGZkwXwB6GrSbbzUrwRCKm3FieD1QE4VVYevaadVUU75GG5 + mrFKorJEH7kFZlic8OTjDksYnHbcgU36XZrGEXa2+ldVeGKL3CsXWciaQRcJg8yo + WQDjZpcutGI0eMJWCqUkv8pYZC2/wZU4htCve5nVJUU4t9uuo9ex7lnwlLWPvheQ + jUBMgzSRsZ+zwaIusvufAAxiKK/cJm4ubZSZPIjBbfd4U7VPxtirP4Accydu7EK6 + eG/MZwtAMFNJxfxUR+/aYzJU/q1ePw7fWVHrpt58t/22CX2SJBEiUGmSmuyER4Ny + DPw6d6mhvPUS1jRhIZ9A81ht8MOX7VL5uVp307rt7o5vRpV1mo0iPiRHzGscMpJn + AP36klEAUNTf0uLTKZa7KHiwhn5iPmsCrENHkOKJjxhRrqHjD2wy3YHs3ow2voyY + Ua4Cids+c1hvRkNEDGNHm4+rKGFOGOsG/ZU7uj/6gflO4JXxNGiyTLflqMdWBvow + Zd7hk1zCaGAAn8nZ0hPweGxQ4Q30I9IBZrimGxB0vjiUqNio9+qMf33dCHFJEuut + ZGJMaUGVaPhXQcTy4uD5hzsPZV5xcsU4H3vBYyBcZgrusJ6OOgkuZQaU7p8rWQWr + bUEVbXuZdwEmxsCe7H/vEVv5+aA4sF4kWnMMFL7/LIYaiEzkTqdJlRv/KyJJgcAH + hg2BvR3XTAq8wiX0C98CdmTbsx2eyQdj5tCU606rEohFLKUxWkJYAKxCiUbxGGpI + RheVmxkef9ErxJiq7hsAsGrSJvMtJuDKIasnD14SOEwD/7jRAq6WdL9VLpxtzlOw + pWnIl8kUCO3WoaG9Jf+ZTIv2hnxJhaSzYrdXzGPNnaWKhBlwnXJRvQEdrIxZOimP + FujZhqbKUDbYAcqTkoQ= + -----END ENCRYPTED PRIVATE KEY----- + -----BEGIN CERTIFICATE----- + MIIDkTCCAnmgAwIBAgIETxH5JDANBgkqhkiG9w0BAQsFADB5MRAwDgYDVQQGEwdV + bmtub3duMRAwDgYDVQQIEwdVbmtub3duMRAwDgYDVQQHEwdVbmtub3duMRAwDgYD + VQQKEwdVbmtub3duMRQwEgYDVQQLDAtzc2xfdGVzdGluZzEZMBcGA1UEAxMQQXBh + Y2hlIENhc3NhbmRyYTAeFw0xNjAzMTgyMTI4MDJaFw0xNjA2MTYyMTI4MDJaMHkx + EDAOBgNVBAYTB1Vua25vd24xEDAOBgNVBAgTB1Vua25vd24xEDAOBgNVBAcTB1Vu + a25vd24xEDAOBgNVBAoTB1Vua25vd24xFDASBgNVBAsMC3NzbF90ZXN0aW5nMRkw + FwYDVQQDExBBcGFjaGUgQ2Fzc2FuZHJhMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8A + MIIBCgKCAQEAjkmVX/HS49cS8Hn6o26IGwMIcEV3d7ZhH0GNcx8rnSRd10dU9F6d + ugSjbwGFMcWUQzYNejN6az0Wb8JIQyXRPTWjfgaWTyVGr0bGTnxg6vwhzfI/9jzy + q59xv29OuSY1dxmY31f0pZ9OOw3mabWksjoO2TexfKoxqsRHJ8PrM1f8E84Z4xo2 + TJXGzpuIxRkAJ+sVDqKEAhrKAfRYMSgdJ7zRt8VXv9ngjX20uA2m092NcH0Kmeto + TmuWUtK8E/qcN7ULN8xRWNUn4hu6mG6mayk4XliGRqI1VZupqh+MgNqHznuTd0bA + YrQsFPw9HaZ2hvVnJffJ5l7njAekZNOL+wIDAQABoyEwHzAdBgNVHQ4EFgQUcdiD + N6aylI91kAd34Hl2AzWY51QwDQYJKoZIhvcNAQELBQADggEBAG9q29ilUgCWQP5v + iHkZHj10gXGEoMkdfrPBf8grC7dpUcaw1Qfku/DJ7kPvMALeEsmFDk/t78roeNbh + IYBLJlzI1HZN6VPtpWQGsqxltAy5XN9Xw9mQM/tu70ShgsodGmE1UoW6eE5+/GMv + 6Fg+zLuICPvs2cFNmWUvukN5LW146tJSYCv0Q/rCPB3m9dNQ9pBxrzPUHXw4glwG + qGnGddXmOC+tSW5lDLLG1BRbKv4zxv3UlrtIjqlJtZb/sQMT6WtG2ihAz7SKOBHa + HOWUwuPTetWIuJCKP7P4mWWtmSmjLy+BFX5seNEngn3RzJ2L8uuTJQ/88OsqgGru + n3MVF9w= + -----END CERTIFICATE----- + private_key_password: "cassandra" + trusted_certificates: | + -----BEGIN CERTIFICATE----- + MIIDkTCCAnmgAwIBAgIETxH5JDANBgkqhkiG9w0BAQsFADB5MRAwDgYDVQQGEwdV + bmtub3duMRAwDgYDVQQIEwdVbmtub3duMRAwDgYDVQQHEwdVbmtub3duMRAwDgYD + VQQKEwdVbmtub3duMRQwEgYDVQQLDAtzc2xfdGVzdGluZzEZMBcGA1UEAxMQQXBh + Y2hlIENhc3NhbmRyYTAeFw0xNjAzMTgyMTI4MDJaFw0xNjA2MTYyMTI4MDJaMHkx + EDAOBgNVBAYTB1Vua25vd24xEDAOBgNVBAgTB1Vua25vd24xEDAOBgNVBAcTB1Vu + a25vd24xEDAOBgNVBAoTB1Vua25vd24xFDASBgNVBAsMC3NzbF90ZXN0aW5nMRkw + FwYDVQQDExBBcGFjaGUgQ2Fzc2FuZHJhMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8A + MIIBCgKCAQEAjkmVX/HS49cS8Hn6o26IGwMIcEV3d7ZhH0GNcx8rnSRd10dU9F6d + ugSjbwGFMcWUQzYNejN6az0Wb8JIQyXRPTWjfgaWTyVGr0bGTnxg6vwhzfI/9jzy + q59xv29OuSY1dxmY31f0pZ9OOw3mabWksjoO2TexfKoxqsRHJ8PrM1f8E84Z4xo2 + TJXGzpuIxRkAJ+sVDqKEAhrKAfRYMSgdJ7zRt8VXv9ngjX20uA2m092NcH0Kmeto + TmuWUtK8E/qcN7ULN8xRWNUn4hu6mG6mayk4XliGRqI1VZupqh+MgNqHznuTd0bA + YrQsFPw9HaZ2hvVnJffJ5l7njAekZNOL+wIDAQABoyEwHzAdBgNVHQ4EFgQUcdiD + N6aylI91kAd34Hl2AzWY51QwDQYJKoZIhvcNAQELBQADggEBAG9q29ilUgCWQP5v + iHkZHj10gXGEoMkdfrPBf8grC7dpUcaw1Qfku/DJ7kPvMALeEsmFDk/t78roeNbh + IYBLJlzI1HZN6VPtpWQGsqxltAy5XN9Xw9mQM/tu70ShgsodGmE1UoW6eE5+/GMv + 6Fg+zLuICPvs2cFNmWUvukN5LW146tJSYCv0Q/rCPB3m9dNQ9pBxrzPUHXw4glwG + qGnGddXmOC+tSW5lDLLG1BRbKv4zxv3UlrtIjqlJtZb/sQMT6WtG2ihAz7SKOBHa + HOWUwuPTetWIuJCKP7P4mWWtmSmjLy+BFX5seNEngn3RzJ2L8uuTJQ/88OsqgGru + n3MVF9w= + -----END CERTIFICATE----- incremental_backups: true concurrent_compactors: 4 compaction_throughput: 0MiB/s diff --git a/test/conf/cassandra-jmx-pem-sslconfig.yaml b/test/conf/cassandra-jmx-pem-sslconfig.yaml index 45f58ffda272..55adfd87dd26 100644 --- a/test/conf/cassandra-jmx-pem-sslconfig.yaml +++ b/test/conf/cassandra-jmx-pem-sslconfig.yaml @@ -48,89 +48,91 @@ seed_provider: - seeds: "127.0.0.1:7012" endpoint_snitch: org.apache.cassandra.locator.SimpleSnitch dynamic_snitch: true -jmx_encryption_options: +jmx_server_options: enabled: true - cipher_suites: [TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256] - accepted_protocols: [TLSv1.2,TLSv1.3,TLSv1.1] - ssl_context_factory: - class_name: org.apache.cassandra.security.PEMBasedSslContextFactory - parameters: - private_key: | - -----BEGIN ENCRYPTED PRIVATE KEY----- - MIIE6jAcBgoqhkiG9w0BDAEDMA4ECOWqSzq5PBIdAgIFxQSCBMjXsCK30J0aT3J/ - g5kcbmevTOY1pIhJGbf5QYYrMUPiuDK2ydxIbiPzoTE4/S+OkCeHhlqwn/YydpBl - xgjZZ1Z5rLJHO27d2biuESqanDiBVXYuVmHmaifRnFy0uUTFkStB5mjVZEiJgO29 - L83hL60uWru71EVuVriC2WCfmZ/EXp6wyYszOqCFQ8Quk/rDO6XuaBl467MJbx5V - sucGT6E9XKNd9hB14/Izb2jtVM5kqKxoiHpz1na6yhEYJiE5D1uOonznWjBnjwB/ - f0x+acpDfVDoJKTlRdz+DEcbOF7mb9lBVVjP6P/AAsmQzz6JKwHjvCrjYfQmyyN8 - RI4KRQnWgm4L3dtByLqY8HFU4ogisCMCgI+hZQ+OKMz/hoRO540YGiPcTRY3EOUR - 0bd5JxU6tCJDMTqKP9aSL2KmLoiLowdMkSPz7TCzLsZ2bGJemuCfpAs4XT1vXCHs - evrUbOnh8et1IA8mZ9auThfqsZtNagJLEXA6hWIKp1FfVL3Q49wvMKZt4eTn/zwU - tLL0m5yPo6/HAaOA3hbm/oghZS0dseshXl7PZrmZQtvYnIvjyoxEL7ducYDQCDP6 - wZ7Nzyh1QZAauSS15hl3vLFRZCA9hWAVgwQAviTvhB342O0i9qI7TQkcHk+qcTPN - K+iGNbFZ8ma1izXNKSJ2PgI/QqFNIeJWvZrb9PhJRmaZVsTJ9fERm1ewpebZqkVv - zMqMhlKgx9ggAaSKgnGZkwXwB6GrSbbzUrwRCKm3FieD1QE4VVYevaadVUU75GG5 - mrFKorJEH7kFZlic8OTjDksYnHbcgU36XZrGEXa2+ldVeGKL3CsXWciaQRcJg8yo - WQDjZpcutGI0eMJWCqUkv8pYZC2/wZU4htCve5nVJUU4t9uuo9ex7lnwlLWPvheQ - jUBMgzSRsZ+zwaIusvufAAxiKK/cJm4ubZSZPIjBbfd4U7VPxtirP4Accydu7EK6 - eG/MZwtAMFNJxfxUR+/aYzJU/q1ePw7fWVHrpt58t/22CX2SJBEiUGmSmuyER4Ny - DPw6d6mhvPUS1jRhIZ9A81ht8MOX7VL5uVp307rt7o5vRpV1mo0iPiRHzGscMpJn - AP36klEAUNTf0uLTKZa7KHiwhn5iPmsCrENHkOKJjxhRrqHjD2wy3YHs3ow2voyY - Ua4Cids+c1hvRkNEDGNHm4+rKGFOGOsG/ZU7uj/6gflO4JXxNGiyTLflqMdWBvow - Zd7hk1zCaGAAn8nZ0hPweGxQ4Q30I9IBZrimGxB0vjiUqNio9+qMf33dCHFJEuut - ZGJMaUGVaPhXQcTy4uD5hzsPZV5xcsU4H3vBYyBcZgrusJ6OOgkuZQaU7p8rWQWr - bUEVbXuZdwEmxsCe7H/vEVv5+aA4sF4kWnMMFL7/LIYaiEzkTqdJlRv/KyJJgcAH - hg2BvR3XTAq8wiX0C98CdmTbsx2eyQdj5tCU606rEohFLKUxWkJYAKxCiUbxGGpI - RheVmxkef9ErxJiq7hsAsGrSJvMtJuDKIasnD14SOEwD/7jRAq6WdL9VLpxtzlOw - pWnIl8kUCO3WoaG9Jf+ZTIv2hnxJhaSzYrdXzGPNnaWKhBlwnXJRvQEdrIxZOimP - FujZhqbKUDbYAcqTkoQ= - -----END ENCRYPTED PRIVATE KEY----- - -----BEGIN CERTIFICATE----- - MIIDkTCCAnmgAwIBAgIETxH5JDANBgkqhkiG9w0BAQsFADB5MRAwDgYDVQQGEwdV - bmtub3duMRAwDgYDVQQIEwdVbmtub3duMRAwDgYDVQQHEwdVbmtub3duMRAwDgYD - VQQKEwdVbmtub3duMRQwEgYDVQQLDAtzc2xfdGVzdGluZzEZMBcGA1UEAxMQQXBh - Y2hlIENhc3NhbmRyYTAeFw0xNjAzMTgyMTI4MDJaFw0xNjA2MTYyMTI4MDJaMHkx - EDAOBgNVBAYTB1Vua25vd24xEDAOBgNVBAgTB1Vua25vd24xEDAOBgNVBAcTB1Vu - a25vd24xEDAOBgNVBAoTB1Vua25vd24xFDASBgNVBAsMC3NzbF90ZXN0aW5nMRkw - FwYDVQQDExBBcGFjaGUgQ2Fzc2FuZHJhMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8A - MIIBCgKCAQEAjkmVX/HS49cS8Hn6o26IGwMIcEV3d7ZhH0GNcx8rnSRd10dU9F6d - ugSjbwGFMcWUQzYNejN6az0Wb8JIQyXRPTWjfgaWTyVGr0bGTnxg6vwhzfI/9jzy - q59xv29OuSY1dxmY31f0pZ9OOw3mabWksjoO2TexfKoxqsRHJ8PrM1f8E84Z4xo2 - TJXGzpuIxRkAJ+sVDqKEAhrKAfRYMSgdJ7zRt8VXv9ngjX20uA2m092NcH0Kmeto - TmuWUtK8E/qcN7ULN8xRWNUn4hu6mG6mayk4XliGRqI1VZupqh+MgNqHznuTd0bA - YrQsFPw9HaZ2hvVnJffJ5l7njAekZNOL+wIDAQABoyEwHzAdBgNVHQ4EFgQUcdiD - N6aylI91kAd34Hl2AzWY51QwDQYJKoZIhvcNAQELBQADggEBAG9q29ilUgCWQP5v - iHkZHj10gXGEoMkdfrPBf8grC7dpUcaw1Qfku/DJ7kPvMALeEsmFDk/t78roeNbh - IYBLJlzI1HZN6VPtpWQGsqxltAy5XN9Xw9mQM/tu70ShgsodGmE1UoW6eE5+/GMv - 6Fg+zLuICPvs2cFNmWUvukN5LW146tJSYCv0Q/rCPB3m9dNQ9pBxrzPUHXw4glwG - qGnGddXmOC+tSW5lDLLG1BRbKv4zxv3UlrtIjqlJtZb/sQMT6WtG2ihAz7SKOBHa - HOWUwuPTetWIuJCKP7P4mWWtmSmjLy+BFX5seNEngn3RzJ2L8uuTJQ/88OsqgGru - n3MVF9w= - -----END CERTIFICATE----- - private_key_password: "cassandra" - trusted_certificates: | - -----BEGIN CERTIFICATE----- - MIIDkTCCAnmgAwIBAgIETxH5JDANBgkqhkiG9w0BAQsFADB5MRAwDgYDVQQGEwdV - bmtub3duMRAwDgYDVQQIEwdVbmtub3duMRAwDgYDVQQHEwdVbmtub3duMRAwDgYD - VQQKEwdVbmtub3duMRQwEgYDVQQLDAtzc2xfdGVzdGluZzEZMBcGA1UEAxMQQXBh - Y2hlIENhc3NhbmRyYTAeFw0xNjAzMTgyMTI4MDJaFw0xNjA2MTYyMTI4MDJaMHkx - EDAOBgNVBAYTB1Vua25vd24xEDAOBgNVBAgTB1Vua25vd24xEDAOBgNVBAcTB1Vu - a25vd24xEDAOBgNVBAoTB1Vua25vd24xFDASBgNVBAsMC3NzbF90ZXN0aW5nMRkw - FwYDVQQDExBBcGFjaGUgQ2Fzc2FuZHJhMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8A - MIIBCgKCAQEAjkmVX/HS49cS8Hn6o26IGwMIcEV3d7ZhH0GNcx8rnSRd10dU9F6d - ugSjbwGFMcWUQzYNejN6az0Wb8JIQyXRPTWjfgaWTyVGr0bGTnxg6vwhzfI/9jzy - q59xv29OuSY1dxmY31f0pZ9OOw3mabWksjoO2TexfKoxqsRHJ8PrM1f8E84Z4xo2 - TJXGzpuIxRkAJ+sVDqKEAhrKAfRYMSgdJ7zRt8VXv9ngjX20uA2m092NcH0Kmeto - TmuWUtK8E/qcN7ULN8xRWNUn4hu6mG6mayk4XliGRqI1VZupqh+MgNqHznuTd0bA - YrQsFPw9HaZ2hvVnJffJ5l7njAekZNOL+wIDAQABoyEwHzAdBgNVHQ4EFgQUcdiD - N6aylI91kAd34Hl2AzWY51QwDQYJKoZIhvcNAQELBQADggEBAG9q29ilUgCWQP5v - iHkZHj10gXGEoMkdfrPBf8grC7dpUcaw1Qfku/DJ7kPvMALeEsmFDk/t78roeNbh - IYBLJlzI1HZN6VPtpWQGsqxltAy5XN9Xw9mQM/tu70ShgsodGmE1UoW6eE5+/GMv - 6Fg+zLuICPvs2cFNmWUvukN5LW146tJSYCv0Q/rCPB3m9dNQ9pBxrzPUHXw4glwG - qGnGddXmOC+tSW5lDLLG1BRbKv4zxv3UlrtIjqlJtZb/sQMT6WtG2ihAz7SKOBHa - HOWUwuPTetWIuJCKP7P4mWWtmSmjLy+BFX5seNEngn3RzJ2L8uuTJQ/88OsqgGru - n3MVF9w= - -----END CERTIFICATE----- + jmx_encryption_options: + enabled: true + cipher_suites: [TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256] + accepted_protocols: [TLSv1.2,TLSv1.3,TLSv1.1] + ssl_context_factory: + class_name: org.apache.cassandra.security.PEMBasedSslContextFactory + parameters: + private_key: | + -----BEGIN ENCRYPTED PRIVATE KEY----- + MIIE6jAcBgoqhkiG9w0BDAEDMA4ECOWqSzq5PBIdAgIFxQSCBMjXsCK30J0aT3J/ + g5kcbmevTOY1pIhJGbf5QYYrMUPiuDK2ydxIbiPzoTE4/S+OkCeHhlqwn/YydpBl + xgjZZ1Z5rLJHO27d2biuESqanDiBVXYuVmHmaifRnFy0uUTFkStB5mjVZEiJgO29 + L83hL60uWru71EVuVriC2WCfmZ/EXp6wyYszOqCFQ8Quk/rDO6XuaBl467MJbx5V + sucGT6E9XKNd9hB14/Izb2jtVM5kqKxoiHpz1na6yhEYJiE5D1uOonznWjBnjwB/ + f0x+acpDfVDoJKTlRdz+DEcbOF7mb9lBVVjP6P/AAsmQzz6JKwHjvCrjYfQmyyN8 + RI4KRQnWgm4L3dtByLqY8HFU4ogisCMCgI+hZQ+OKMz/hoRO540YGiPcTRY3EOUR + 0bd5JxU6tCJDMTqKP9aSL2KmLoiLowdMkSPz7TCzLsZ2bGJemuCfpAs4XT1vXCHs + evrUbOnh8et1IA8mZ9auThfqsZtNagJLEXA6hWIKp1FfVL3Q49wvMKZt4eTn/zwU + tLL0m5yPo6/HAaOA3hbm/oghZS0dseshXl7PZrmZQtvYnIvjyoxEL7ducYDQCDP6 + wZ7Nzyh1QZAauSS15hl3vLFRZCA9hWAVgwQAviTvhB342O0i9qI7TQkcHk+qcTPN + K+iGNbFZ8ma1izXNKSJ2PgI/QqFNIeJWvZrb9PhJRmaZVsTJ9fERm1ewpebZqkVv + zMqMhlKgx9ggAaSKgnGZkwXwB6GrSbbzUrwRCKm3FieD1QE4VVYevaadVUU75GG5 + mrFKorJEH7kFZlic8OTjDksYnHbcgU36XZrGEXa2+ldVeGKL3CsXWciaQRcJg8yo + WQDjZpcutGI0eMJWCqUkv8pYZC2/wZU4htCve5nVJUU4t9uuo9ex7lnwlLWPvheQ + jUBMgzSRsZ+zwaIusvufAAxiKK/cJm4ubZSZPIjBbfd4U7VPxtirP4Accydu7EK6 + eG/MZwtAMFNJxfxUR+/aYzJU/q1ePw7fWVHrpt58t/22CX2SJBEiUGmSmuyER4Ny + DPw6d6mhvPUS1jRhIZ9A81ht8MOX7VL5uVp307rt7o5vRpV1mo0iPiRHzGscMpJn + AP36klEAUNTf0uLTKZa7KHiwhn5iPmsCrENHkOKJjxhRrqHjD2wy3YHs3ow2voyY + Ua4Cids+c1hvRkNEDGNHm4+rKGFOGOsG/ZU7uj/6gflO4JXxNGiyTLflqMdWBvow + Zd7hk1zCaGAAn8nZ0hPweGxQ4Q30I9IBZrimGxB0vjiUqNio9+qMf33dCHFJEuut + ZGJMaUGVaPhXQcTy4uD5hzsPZV5xcsU4H3vBYyBcZgrusJ6OOgkuZQaU7p8rWQWr + bUEVbXuZdwEmxsCe7H/vEVv5+aA4sF4kWnMMFL7/LIYaiEzkTqdJlRv/KyJJgcAH + hg2BvR3XTAq8wiX0C98CdmTbsx2eyQdj5tCU606rEohFLKUxWkJYAKxCiUbxGGpI + RheVmxkef9ErxJiq7hsAsGrSJvMtJuDKIasnD14SOEwD/7jRAq6WdL9VLpxtzlOw + pWnIl8kUCO3WoaG9Jf+ZTIv2hnxJhaSzYrdXzGPNnaWKhBlwnXJRvQEdrIxZOimP + FujZhqbKUDbYAcqTkoQ= + -----END ENCRYPTED PRIVATE KEY----- + -----BEGIN CERTIFICATE----- + MIIDkTCCAnmgAwIBAgIETxH5JDANBgkqhkiG9w0BAQsFADB5MRAwDgYDVQQGEwdV + bmtub3duMRAwDgYDVQQIEwdVbmtub3duMRAwDgYDVQQHEwdVbmtub3duMRAwDgYD + VQQKEwdVbmtub3duMRQwEgYDVQQLDAtzc2xfdGVzdGluZzEZMBcGA1UEAxMQQXBh + Y2hlIENhc3NhbmRyYTAeFw0xNjAzMTgyMTI4MDJaFw0xNjA2MTYyMTI4MDJaMHkx + EDAOBgNVBAYTB1Vua25vd24xEDAOBgNVBAgTB1Vua25vd24xEDAOBgNVBAcTB1Vu + a25vd24xEDAOBgNVBAoTB1Vua25vd24xFDASBgNVBAsMC3NzbF90ZXN0aW5nMRkw + FwYDVQQDExBBcGFjaGUgQ2Fzc2FuZHJhMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8A + MIIBCgKCAQEAjkmVX/HS49cS8Hn6o26IGwMIcEV3d7ZhH0GNcx8rnSRd10dU9F6d + ugSjbwGFMcWUQzYNejN6az0Wb8JIQyXRPTWjfgaWTyVGr0bGTnxg6vwhzfI/9jzy + q59xv29OuSY1dxmY31f0pZ9OOw3mabWksjoO2TexfKoxqsRHJ8PrM1f8E84Z4xo2 + TJXGzpuIxRkAJ+sVDqKEAhrKAfRYMSgdJ7zRt8VXv9ngjX20uA2m092NcH0Kmeto + TmuWUtK8E/qcN7ULN8xRWNUn4hu6mG6mayk4XliGRqI1VZupqh+MgNqHznuTd0bA + YrQsFPw9HaZ2hvVnJffJ5l7njAekZNOL+wIDAQABoyEwHzAdBgNVHQ4EFgQUcdiD + N6aylI91kAd34Hl2AzWY51QwDQYJKoZIhvcNAQELBQADggEBAG9q29ilUgCWQP5v + iHkZHj10gXGEoMkdfrPBf8grC7dpUcaw1Qfku/DJ7kPvMALeEsmFDk/t78roeNbh + IYBLJlzI1HZN6VPtpWQGsqxltAy5XN9Xw9mQM/tu70ShgsodGmE1UoW6eE5+/GMv + 6Fg+zLuICPvs2cFNmWUvukN5LW146tJSYCv0Q/rCPB3m9dNQ9pBxrzPUHXw4glwG + qGnGddXmOC+tSW5lDLLG1BRbKv4zxv3UlrtIjqlJtZb/sQMT6WtG2ihAz7SKOBHa + HOWUwuPTetWIuJCKP7P4mWWtmSmjLy+BFX5seNEngn3RzJ2L8uuTJQ/88OsqgGru + n3MVF9w= + -----END CERTIFICATE----- + private_key_password: "cassandra" + trusted_certificates: | + -----BEGIN CERTIFICATE----- + MIIDkTCCAnmgAwIBAgIETxH5JDANBgkqhkiG9w0BAQsFADB5MRAwDgYDVQQGEwdV + bmtub3duMRAwDgYDVQQIEwdVbmtub3duMRAwDgYDVQQHEwdVbmtub3duMRAwDgYD + VQQKEwdVbmtub3duMRQwEgYDVQQLDAtzc2xfdGVzdGluZzEZMBcGA1UEAxMQQXBh + Y2hlIENhc3NhbmRyYTAeFw0xNjAzMTgyMTI4MDJaFw0xNjA2MTYyMTI4MDJaMHkx + EDAOBgNVBAYTB1Vua25vd24xEDAOBgNVBAgTB1Vua25vd24xEDAOBgNVBAcTB1Vu + a25vd24xEDAOBgNVBAoTB1Vua25vd24xFDASBgNVBAsMC3NzbF90ZXN0aW5nMRkw + FwYDVQQDExBBcGFjaGUgQ2Fzc2FuZHJhMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8A + MIIBCgKCAQEAjkmVX/HS49cS8Hn6o26IGwMIcEV3d7ZhH0GNcx8rnSRd10dU9F6d + ugSjbwGFMcWUQzYNejN6az0Wb8JIQyXRPTWjfgaWTyVGr0bGTnxg6vwhzfI/9jzy + q59xv29OuSY1dxmY31f0pZ9OOw3mabWksjoO2TexfKoxqsRHJ8PrM1f8E84Z4xo2 + TJXGzpuIxRkAJ+sVDqKEAhrKAfRYMSgdJ7zRt8VXv9ngjX20uA2m092NcH0Kmeto + TmuWUtK8E/qcN7ULN8xRWNUn4hu6mG6mayk4XliGRqI1VZupqh+MgNqHznuTd0bA + YrQsFPw9HaZ2hvVnJffJ5l7njAekZNOL+wIDAQABoyEwHzAdBgNVHQ4EFgQUcdiD + N6aylI91kAd34Hl2AzWY51QwDQYJKoZIhvcNAQELBQADggEBAG9q29ilUgCWQP5v + iHkZHj10gXGEoMkdfrPBf8grC7dpUcaw1Qfku/DJ7kPvMALeEsmFDk/t78roeNbh + IYBLJlzI1HZN6VPtpWQGsqxltAy5XN9Xw9mQM/tu70ShgsodGmE1UoW6eE5+/GMv + 6Fg+zLuICPvs2cFNmWUvukN5LW146tJSYCv0Q/rCPB3m9dNQ9pBxrzPUHXw4glwG + qGnGddXmOC+tSW5lDLLG1BRbKv4zxv3UlrtIjqlJtZb/sQMT6WtG2ihAz7SKOBHa + HOWUwuPTetWIuJCKP7P4mWWtmSmjLy+BFX5seNEngn3RzJ2L8uuTJQ/88OsqgGru + n3MVF9w= + -----END CERTIFICATE----- incremental_backups: true concurrent_compactors: 4 compaction_throughput: 0MiB/s diff --git a/test/conf/cassandra-jmx-sslconfig.yaml b/test/conf/cassandra-jmx-sslconfig.yaml index 317d5236b62a..1a6ef9a9457f 100644 --- a/test/conf/cassandra-jmx-sslconfig.yaml +++ b/test/conf/cassandra-jmx-sslconfig.yaml @@ -48,14 +48,16 @@ seed_provider: - seeds: "127.0.0.1:7012" endpoint_snitch: org.apache.cassandra.locator.SimpleSnitch dynamic_snitch: true -jmx_encryption_options: +jmx_server_options: enabled: true - cipher_suites: [TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256] - accepted_protocols: [TLSv1.2,TLSv1.3,TLSv1.1] - keystore: test/conf/cassandra_ssl_test.keystore - keystore_password: cassandra - truststore: test/conf/cassandra_ssl_test.truststore - truststore_password: cassandra + jmx_encryption_options: + enabled: true + cipher_suites: [TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256] + accepted_protocols: [TLSv1.2,TLSv1.3,TLSv1.1] + keystore: test/conf/cassandra_ssl_test.keystore + keystore_password: cassandra + truststore: test/conf/cassandra_ssl_test.truststore + truststore_password: cassandra incremental_backups: true concurrent_compactors: 4 compaction_throughput: 0MiB/s diff --git a/test/distributed/org/apache/cassandra/distributed/impl/CollectingSslRMIServerSocketFactoryImpl.java b/test/distributed/org/apache/cassandra/distributed/impl/CollectingSslRMIServerSocketFactoryImpl.java index 01d051e4cf07..0d13aa6290cc 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/CollectingSslRMIServerSocketFactoryImpl.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/CollectingSslRMIServerSocketFactoryImpl.java @@ -57,16 +57,6 @@ public CollectingSslRMIServerSocketFactoryImpl(InetAddress bindAddress, String[] this.sslSocketFactory = sslContext.getSocketFactory(); } - public CollectingSslRMIServerSocketFactoryImpl(InetAddress bindAddress, String[] enabledCipherSuites, - String[] enabledProtocols, boolean needClientAuth) - { - this.bindAddress = bindAddress; - this.enabledCipherSuites = enabledCipherSuites; - this.enabledProtocols = enabledProtocols; - this.needClientAuth = needClientAuth; - this.sslSocketFactory = getDefaultSSLSocketFactory(); - } - public String[] getEnabledCipherSuites() { return enabledCipherSuites; diff --git a/test/distributed/org/apache/cassandra/distributed/impl/IsolatedJmx.java b/test/distributed/org/apache/cassandra/distributed/impl/IsolatedJmx.java index 6a575972eb35..18eaeb85c532 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/IsolatedJmx.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/IsolatedJmx.java @@ -35,6 +35,7 @@ import org.slf4j.Logger; import org.apache.cassandra.config.EncryptionOptions; +import org.apache.cassandra.config.JMXServerOptions; import org.apache.cassandra.distributed.api.IInstance; import org.apache.cassandra.distributed.api.IInstanceConfig; import org.apache.cassandra.distributed.shared.JMXUtil; @@ -91,12 +92,24 @@ public void startJmx() ((MBeanWrapper.DelegatingMbeanWrapper) MBeanWrapper.instance).setDelegate(wrapper); // CASSANDRA-18508: Sensitive JMX SSL configuration options can be easily exposed - Map<String, Object> encryptionOptionsMap = (Map<String, Object>) config.getParams().get("jmx_encryption_options"); - EncryptionOptions jmxEncryptionOptions = getJmxEncryptionOptions(encryptionOptionsMap); + Map<String, Object> jmxServerOptionsMap = (Map<String, Object>) config.getParams().get("jmx_server_options"); + EncryptionOptions jmxEncryptionOptions; + if (jmxServerOptionsMap == null) + { + JMXServerOptions parsingSystemProperties = JMXServerOptions.createParsingSystemProperties(); + jmxEncryptionOptions = parsingSystemProperties.jmx_encryption_options; + jmxEncryptionOptions.applyConfig(); + } + else + { + jmxEncryptionOptions = getJmxEncryptionOptions(jmxServerOptionsMap); + } + // Here the `localOnly` is always passed as true as it is for the local isolated JMX testing // However if the `jmxEncryptionOptions` are provided or JMX SSL configuration is set it will configure // the socket factories appropriately. - Map<String, Object> socketFactories = new IsolatedJmxSocketFactory().configure(addr, true, jmxEncryptionOptions); + JMXServerOptions jmxServerOptions = JMXServerOptions.create(true, true, jmxPort, jmxEncryptionOptions); + Map<String, Object> socketFactories = new IsolatedJmxSocketFactory().configure(addr, jmxServerOptions, jmxServerOptions.jmx_encryption_options); serverSocketFactory = (RMICloseableServerSocketFactory) socketFactories.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE); clientSocketFactory = (RMICloseableClientSocketFactory) socketFactories.get(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE); Map<String, Object> env = new HashMap<>(socketFactories); @@ -148,12 +161,17 @@ public void startJmx() /** * Builds {@code EncryptionOptions} from the map based SSL configuration properties. * - * @param encryptionOptionsMap of SSL configuration properties + * @param jmxServerOptionsMap of jmx server configuration properties * @return EncryptionOptions built object */ @SuppressWarnings("unchecked") - private EncryptionOptions getJmxEncryptionOptions(Map<String, Object> encryptionOptionsMap) + private EncryptionOptions getJmxEncryptionOptions(Map<String, Object> jmxServerOptionsMap) { + if (jmxServerOptionsMap == null) + return null; + + Map<String, Object> encryptionOptionsMap = (Map<String, Object>) jmxServerOptionsMap.get("jmx_encryption_options"); + if (encryptionOptionsMap == null) { return null; diff --git a/test/distributed/org/apache/cassandra/distributed/impl/IsolatedJmxSocketFactory.java b/test/distributed/org/apache/cassandra/distributed/impl/IsolatedJmxSocketFactory.java index 868cb8ae4b28..2aa4415af929 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/IsolatedJmxSocketFactory.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/IsolatedJmxSocketFactory.java @@ -22,16 +22,19 @@ import java.util.Arrays; import java.util.Map; import java.util.stream.Collectors; -import javax.management.remote.rmi.RMIConnectorServer; import javax.net.ssl.SSLContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.utils.RMIClientSocketFactoryImpl; import org.apache.cassandra.utils.jmx.AbstractJmxSocketFactory; +import static javax.management.remote.rmi.RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE; +import static javax.management.remote.rmi.RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS; + /** * JMX Socket factory used for the isolated JMX testing. */ @@ -43,35 +46,21 @@ public class IsolatedJmxSocketFactory extends AbstractJmxSocketFactory public void configureLocalSocketFactories(Map<String, Object> env, InetAddress serverAddress) { CollectingRMIServerSocketFactoryImpl serverSocketFactory = new CollectingRMIServerSocketFactoryImpl(serverAddress); - env.put(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE, - serverSocketFactory); RMIClientSocketFactoryImpl clientSocketFactory = new RMIClientSocketFactoryImpl(serverAddress); - env.put(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE, - clientSocketFactory); + env.put(RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE, serverSocketFactory); + env.put(RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE, clientSocketFactory); } @Override public void configureSslClientSocketFactory(Map<String, Object> env, InetAddress serverAddress) { RMISslClientSocketFactoryImpl clientFactory = new RMISslClientSocketFactoryImpl(serverAddress, - CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES.getString(), - CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS.getString()); - env.put(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE, clientFactory); + JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES.getString(), + JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS.getString()); + env.put(RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE, clientFactory); env.put("com.sun.jndi.rmi.factory.socket", clientFactory); } - @Override - public void configureSslServerSocketFactory(Map<String, Object> env, InetAddress serverAddress, String[] enabledCipherSuites, - String[] enabledProtocols, boolean needClientAuth) - { - CollectingSslRMIServerSocketFactoryImpl serverFactory = new CollectingSslRMIServerSocketFactoryImpl(serverAddress, - enabledCipherSuites, - enabledProtocols, - needClientAuth); - env.put(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE, serverFactory); - logJmxSslConfig(serverFactory); - } - @Override public void configureSslServerSocketFactory(Map<String, Object> env, InetAddress serverAddress, String[] enabledCipherSuites, String[] enabledProtocols, boolean needClientAuth, SSLContext sslContext) @@ -81,7 +70,7 @@ public void configureSslServerSocketFactory(Map<String, Object> env, InetAddress enabledProtocols, needClientAuth, sslContext); - env.put(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE, serverFactory); + env.put(RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE, serverFactory); logJmxSslConfig(serverFactory); } diff --git a/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXSslConfigDistributedTest.java b/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXSslConfigDistributedTest.java index 146f548eaf95..b923229ec043 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXSslConfigDistributedTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/jmx/JMXSslConfigDistributedTest.java @@ -37,7 +37,13 @@ import org.apache.cassandra.distributed.test.AbstractEncryptionOptionsImpl; import org.apache.cassandra.utils.jmx.JMXSslPropertiesUtil; +import static java.util.Map.of; +import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_LOCAL_PORT; import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_CIPHER_SUITES; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_NET_SSL_KEYSTORE; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_NET_SSL_KEYSTOREPASSWORD; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_NET_SSL_TRUSTSTORE; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_NET_SSL_TRUSTSTOREPASSWORD; /** * Distributed tests for JMX SSL configuration via the system properties OR the encryption options in the cassandra.yaml. @@ -62,7 +68,9 @@ public void testDefaultEncryptionOptions() throws Throwable .build(); try (Cluster cluster = builder().withNodes(1).withConfig(c -> { - c.with(Feature.JMX).set("jmx_encryption_options", encryptionOptionsMap); + c.with(Feature.JMX).set("jmx_server_options", of("enabled", + true, + "jmx_encryption_options", encryptionOptionsMap)); }).start()) { Map<String, Object> jmxEnv = new HashMap<>(); @@ -85,7 +93,8 @@ public void testClientAuth() throws Throwable .build(); try (Cluster cluster = builder().withNodes(1).withConfig(c -> { - c.with(Feature.JMX).set("jmx_encryption_options", encryptionOptionsMap); + c.with(Feature.JMX).set("jmx_server_options", of("enabled", true, + "jmx_encryption_options", encryptionOptionsMap)); }).start()) { Map<String, Object> jmxEnv = new HashMap<>(); @@ -99,8 +108,8 @@ public void testClientAuth() throws Throwable public void testSystemSettings() throws Throwable { COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_CIPHER_SUITES.reset(); - try (WithProperties withProperties = JMXSslPropertiesUtil.use(true, false, - "TLSv1.2,TLSv1.3,TLSv1.1")) + try (WithProperties withProperties = JMXSslPropertiesUtil.use(true, false, "TLSv1.2,TLSv1.3,TLSv1.1") + .set(CASSANDRA_JMX_LOCAL_PORT, 7199)) { setKeystoreProperties(withProperties); try (Cluster cluster = builder().withNodes(1).withConfig(c -> { @@ -120,13 +129,11 @@ public void testSystemSettings() throws Throwable public void testInvalidKeystorePath() throws Throwable { try (Cluster cluster = builder().withNodes(1).withConfig(c -> { - c.with(Feature.JMX).set("jmx_encryption_options", - ImmutableMap.<String, Object>builder() - .put("enabled", true) - .put("keystore", "/path/to/bad/keystore/that/should/not/exist") - .put("keystore_password", "cassandra") - .put("accepted_protocols", Arrays.asList("TLSv1.2", "TLSv1.3", "TLSv1.1")) - .build()); + c.with(Feature.JMX).set("jmx_server_options", of("enabled", true, + "jmx_encryption_options", of("enabled", true, + "keystore", "/path/to/bad/keystore/that/should/not/exist", + "keystore_password", "cassandra", + "accepted_protocols", Arrays.asList("TLSv1.2", "TLSv1.3", "TLSv1.1")))); }).createWithoutStarting()) { assertCannotStartDueToConfigurationException(cluster); @@ -141,12 +148,11 @@ public void testInvalidKeystorePath() throws Throwable public void testDisabledEncryptionOptions() throws Throwable { try (Cluster cluster = builder().withNodes(1).withConfig(c -> { - c.with(Feature.JMX).set("jmx_encryption_options", - ImmutableMap.builder() - .put("enabled", false) - .put("keystore", "/path/to/bad/keystore/that/should/not/exist") - .put("keystore_password", "cassandra") - .build()); + c.with(Feature.JMX).set("jmx_server_options", of("enabled", true, + "jmx_encryption_options", + of("enabled", false, + "keystore", "/path/to/bad/keystore/that/should/not/exist", + "keystore_password", "cassandra"))); }).start()) { JMXTestsUtil.testAllValidGetters(cluster, null); @@ -155,10 +161,10 @@ public void testDisabledEncryptionOptions() throws Throwable private void setKeystoreProperties(WithProperties properties) { - properties.with("javax.net.ssl.trustStore", (String) validFileBasedKeystores.get("truststore"), - "javax.net.ssl.trustStorePassword", (String) validFileBasedKeystores.get("truststore_password"), - "javax.net.ssl.keyStore", (String) validFileBasedKeystores.get("keystore"), - "javax.net.ssl.keyStorePassword", (String) validFileBasedKeystores.get("keystore_password")); + properties.with(JAVAX_NET_SSL_TRUSTSTORE.getKey(), (String) validFileBasedKeystores.get("truststore"), + JAVAX_NET_SSL_TRUSTSTOREPASSWORD.getKey(), (String) validFileBasedKeystores.get("truststore_password"), + JAVAX_NET_SSL_KEYSTORE.getKey(), (String) validFileBasedKeystores.get("keystore"), + JAVAX_NET_SSL_KEYSTOREPASSWORD.getKey(), (String) validFileBasedKeystores.get("keystore_password")); } @SuppressWarnings("unchecked") diff --git a/test/resources/auth/cassandra-test-jaas.conf b/test/resources/auth/cassandra-test-jaas.conf index ccb8b6a90a01..59e00d291620 100644 --- a/test/resources/auth/cassandra-test-jaas.conf +++ b/test/resources/auth/cassandra-test-jaas.conf @@ -1,4 +1,4 @@ // Delegates authentication to a stub login module, hardcoded to authenticate as a particular user - see JMXAuthTest TestLogin { - org.apache.cassandra.auth.jmx.JMXAuthTest$StubLoginModule REQUIRED role_name=test_role; + org.apache.cassandra.auth.jmx.AbstractJMXAuthTest$StubLoginModule REQUIRED role_name=test_role; }; diff --git a/test/unit/org/apache/cassandra/auth/jmx/JMXAuthTest.java b/test/unit/org/apache/cassandra/auth/jmx/AbstractJMXAuthTest.java similarity index 87% rename from test/unit/org/apache/cassandra/auth/jmx/JMXAuthTest.java rename to test/unit/org/apache/cassandra/auth/jmx/AbstractJMXAuthTest.java index fece405f5f25..0d75ce6a6223 100644 --- a/test/unit/org/apache/cassandra/auth/jmx/JMXAuthTest.java +++ b/test/unit/org/apache/cassandra/auth/jmx/AbstractJMXAuthTest.java @@ -19,14 +19,16 @@ package org.apache.cassandra.auth.jmx; import java.lang.reflect.Field; -import java.nio.file.Paths; import java.rmi.server.RMISocketFactory; import java.util.HashMap; import java.util.Map; import javax.management.JMX; import javax.management.MBeanServerConnection; import javax.management.ObjectName; -import javax.management.remote.*; +import javax.management.remote.JMXConnector; +import javax.management.remote.JMXConnectorFactory; +import javax.management.remote.JMXConnectorServer; +import javax.management.remote.JMXServiceURL; import javax.security.auth.Subject; import javax.security.auth.callback.CallbackHandler; import javax.security.auth.login.LoginException; @@ -34,76 +36,35 @@ import com.google.common.collect.ImmutableSet; import org.junit.Before; -import org.junit.BeforeClass; +import org.junit.Ignore; import org.junit.Test; -import org.apache.cassandra.auth.*; +import org.apache.cassandra.auth.AuthenticatedUser; +import org.apache.cassandra.auth.CassandraPrincipal; +import org.apache.cassandra.auth.IAuthorizer; +import org.apache.cassandra.auth.JMXResource; +import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.auth.RoleResource; +import org.apache.cassandra.auth.StubAuthorizer; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.JMXServerOptions; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.db.ColumnFamilyStoreMBean; import org.apache.cassandra.utils.JMXServerUtils; -import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_AUTHORIZER; -import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_REMOTE_LOGIN_CONFIG; -import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_AUTHENTICATE; -import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_SECURITY_AUTH_LOGIN_CONFIG; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; -public class JMXAuthTest extends CQLTester +@Ignore +public abstract class AbstractJMXAuthTest extends CQLTester { private static JMXConnectorServer jmxServer; private static MBeanServerConnection connection; + private RoleResource role; private String tableName; private JMXResource tableMBean; - @FunctionalInterface - private interface MBeanAction - { - void execute(); - } - - @BeforeClass - public static void setupClass() throws Exception - { - setupAuthorizer(); - setupJMXServer(); - } - - private static void setupAuthorizer() - { - try - { - IAuthorizer authorizer = new StubAuthorizer(); - Field authorizerField = DatabaseDescriptor.class.getDeclaredField("authorizer"); - authorizerField.setAccessible(true); - authorizerField.set(null, authorizer); - DatabaseDescriptor.setPermissionsValidity(0); - } - catch (IllegalAccessException | NoSuchFieldException e) - { - throw new RuntimeException(e); - } - } - - private static void setupJMXServer() throws Exception - { - String config = Paths.get(ClassLoader.getSystemResource("auth/cassandra-test-jaas.conf").toURI()).toString(); - COM_SUN_MANAGEMENT_JMXREMOTE_AUTHENTICATE.setBoolean(true); - JAVA_SECURITY_AUTH_LOGIN_CONFIG.setString(config); - CASSANDRA_JMX_REMOTE_LOGIN_CONFIG.setString("TestLogin"); - CASSANDRA_JMX_AUTHORIZER.setString(NoSuperUserAuthorizationProxy.class.getName()); - jmxServer = JMXServerUtils.createJMXServer(9999, "localhost", true); - jmxServer.start(); - - JMXServiceURL jmxUrl = new JMXServiceURL("service:jmx:rmi:///jndi/rmi://localhost:9999/jmxrmi"); - Map<String, Object> env = new HashMap<>(); - env.put("com.sun.jndi.rmi.factory.socket", RMISocketFactory.getDefaultSocketFactory()); - JMXConnector jmxc = JMXConnectorFactory.connect(jmxUrl, env); - connection = jmxc.getMBeanServerConnection(); - } - @Before public void setup() throws Throwable { @@ -193,6 +154,42 @@ public void executeMethod() throws Throwable assertPermissionOnResource(Permission.EXECUTE, JMXResource.root(), proxy::estimateKeys); } + + protected static void setupJMXServer(JMXServerOptions jmxServerOptions) throws Exception + { + jmxServerOptions.jmx_encryption_options.applyConfig(); + jmxServer = JMXServerUtils.createJMXServer(jmxServerOptions, "localhost"); + jmxServer.start(); + + JMXServiceURL jmxUrl = new JMXServiceURL("service:jmx:rmi:///jndi/rmi://localhost:9999/jmxrmi"); + Map<String, Object> env = new HashMap<>(); + env.put("com.sun.jndi.rmi.factory.socket", RMISocketFactory.getDefaultSocketFactory()); + JMXConnector jmxc = JMXConnectorFactory.connect(jmxUrl, env); + connection = jmxc.getMBeanServerConnection(); + } + + protected static void setupAuthorizer() + { + try + { + IAuthorizer authorizer = new StubAuthorizer(); + Field authorizerField = DatabaseDescriptor.class.getDeclaredField("authorizer"); + authorizerField.setAccessible(true); + authorizerField.set(null, authorizer); + DatabaseDescriptor.setPermissionsValidity(0); + } + catch (IllegalAccessException | NoSuchFieldException e) + { + throw new RuntimeException(e); + } + } + + @FunctionalInterface + private interface MBeanAction + { + void execute(); + } + private void assertPermissionOnResource(Permission permission, JMXResource resource, MBeanAction action) @@ -238,12 +235,14 @@ public static class StubLoginModule implements LoginModule private CassandraPrincipal principal; private Subject subject; - public StubLoginModule(){} + public StubLoginModule() + { + } public void initialize(Subject subject, CallbackHandler callbackHandler, Map<String, ?> sharedState, Map<String, ?> options) { this.subject = subject; - principal = new CassandraPrincipal((String)options.get("role_name")); + principal = new CassandraPrincipal((String) options.get("role_name")); } public boolean login() throws LoginException diff --git a/test/unit/org/apache/cassandra/auth/jmx/JMXAuthJMXServerOptionsTest.java b/test/unit/org/apache/cassandra/auth/jmx/JMXAuthJMXServerOptionsTest.java new file mode 100644 index 000000000000..cb83a61eda17 --- /dev/null +++ b/test/unit/org/apache/cassandra/auth/jmx/JMXAuthJMXServerOptionsTest.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.auth.jmx; + +import java.nio.file.Paths; + +import org.junit.BeforeClass; + +import org.apache.cassandra.config.EncryptionOptions; +import org.apache.cassandra.config.JMXServerOptions; + +/** + * Tests via server options normally constructed in cassandra.yaml. + */ +public class JMXAuthJMXServerOptionsTest extends AbstractJMXAuthTest +{ + @BeforeClass + public static void setupClass() throws Exception + { + setupAuthorizer(); + setupJMXServer(getJMXServerOptions()); + } + + private static JMXServerOptions getJMXServerOptions() throws Exception + { + String config = Paths.get(ClassLoader.getSystemResource("auth/cassandra-test-jaas.conf").toURI()).toString(); + + return new JMXServerOptions(true, false, 9999, 0, true, + new EncryptionOptions(), "TestLogin", config, null, null, + NoSuperUserAuthorizationProxy.class.getName()); + } +} diff --git a/test/unit/org/apache/cassandra/auth/jmx/JMXAuthSystemPropertiesTest.java b/test/unit/org/apache/cassandra/auth/jmx/JMXAuthSystemPropertiesTest.java new file mode 100644 index 000000000000..f527f2e609a3 --- /dev/null +++ b/test/unit/org/apache/cassandra/auth/jmx/JMXAuthSystemPropertiesTest.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.auth.jmx; + +import java.nio.file.Paths; + +import org.junit.BeforeClass; + +import org.apache.cassandra.config.JMXServerOptions; + +import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_AUTHORIZER; +import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_LOCAL_PORT; +import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_REMOTE_LOGIN_CONFIG; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_AUTHENTICATE; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_SECURITY_AUTH_LOGIN_CONFIG; + +/** + * Tests via system properties normally set in cassandra-env.sh + */ +public class JMXAuthSystemPropertiesTest extends AbstractJMXAuthTest +{ + @BeforeClass + public static void setupClass() throws Exception + { + setupAuthorizer(); + setupJMXServer(getJMXServerOptions()); + } + + private static JMXServerOptions getJMXServerOptions() throws Exception + { + String config = Paths.get(ClassLoader.getSystemResource("auth/cassandra-test-jaas.conf").toURI()).toString(); + COM_SUN_MANAGEMENT_JMXREMOTE_AUTHENTICATE.setBoolean(true); + JAVA_SECURITY_AUTH_LOGIN_CONFIG.setString(config); + CASSANDRA_JMX_REMOTE_LOGIN_CONFIG.setString("TestLogin"); + CASSANDRA_JMX_AUTHORIZER.setString(NoSuperUserAuthorizationProxy.class.getName()); + CASSANDRA_JMX_LOCAL_PORT.setInt(9999); + return JMXServerOptions.createParsingSystemProperties(); + } +} diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java index dea0d867eefe..96b164a32256 100644 --- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java +++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java @@ -144,6 +144,7 @@ public class DatabaseDescriptorRefTest "org.apache.cassandra.config.GuardrailsOptions$Config", "org.apache.cassandra.config.GuardrailsOptions$ConsistencyLevels", "org.apache.cassandra.config.GuardrailsOptions$TableProperties", + "org.apache.cassandra.config.JMXServerOptions", "org.apache.cassandra.config.ParameterizedClass", "org.apache.cassandra.config.RepairConfig", "org.apache.cassandra.config.RepairRetrySpec", diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index 2f5857ae9792..9410564b83dd 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -115,6 +115,7 @@ import org.apache.cassandra.config.DataStorageSpec; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.EncryptionOptions; +import org.apache.cassandra.config.JMXServerOptions; import org.apache.cassandra.config.YamlConfigurationLoader; import org.apache.cassandra.cql3.functions.FunctionName; import org.apache.cassandra.cql3.functions.types.ParseUtils; @@ -190,7 +191,6 @@ import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; -import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_LOCAL_PORT; import static org.apache.cassandra.config.CassandraRelevantProperties.TEST_DRIVER_CONNECTION_TIMEOUT_MS; import static org.apache.cassandra.config.CassandraRelevantProperties.TEST_DRIVER_READ_TIMEOUT_MS; import static org.apache.cassandra.config.CassandraRelevantProperties.TEST_RANDOM_SEED; @@ -397,7 +397,7 @@ public static void startJMXServer() throws Exception InetAddress loopback = InetAddress.getLoopbackAddress(); jmxHost = loopback.getHostAddress(); jmxPort = getAutomaticallyAllocatedPort(loopback); - jmxServer = JMXServerUtils.createJMXServer(jmxPort, true); + jmxServer = JMXServerUtils.createJMXServer(JMXServerOptions.fromDescriptor(true, true, jmxPort)); jmxServer.start(); } @@ -518,7 +518,7 @@ protected void resetSchema() throws Throwable public static List<String> buildNodetoolArgs(List<String> args) { - int port = jmxPort == 0 ? CASSANDRA_JMX_LOCAL_PORT.getInt(7199) : jmxPort; + int port = jmxPort == 0 ? DatabaseDescriptor.getJmxServerOptions().jmx_port : jmxPort; String host = jmxHost == null ? "127.0.0.1" : jmxHost; List<String> allArgs = new ArrayList<>(); allArgs.add("bin/nodetool"); diff --git a/test/unit/org/apache/cassandra/db/virtual/SettingsTableTest.java b/test/unit/org/apache/cassandra/db/virtual/SettingsTableTest.java index 1645e3150e59..71b9172da798 100644 --- a/test/unit/org/apache/cassandra/db/virtual/SettingsTableTest.java +++ b/test/unit/org/apache/cassandra/db/virtual/SettingsTableTest.java @@ -32,6 +32,7 @@ import org.apache.cassandra.config.Config; import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions.InternodeEncryption; +import org.apache.cassandra.config.JMXServerOptions; import org.apache.cassandra.config.ParameterizedClass; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.security.SSLFactory; @@ -52,7 +53,8 @@ public void config() config = new Config(); config.client_encryption_options.applyConfig(); config.server_encryption_options.applyConfig(); - config.jmx_encryption_options.applyConfig(); + config.jmx_server_options = new JMXServerOptions(); + config.jmx_server_options.jmx_encryption_options.applyConfig(); config.sstable_preemptive_open_interval = null; config.index_summary_resize_interval = null; config.cache_load_timeout = new DurationSpec.IntSecondsBound(0); diff --git a/test/unit/org/apache/cassandra/tools/ToolsEnvsConfigsTest.java b/test/unit/org/apache/cassandra/tools/ToolsEnvsConfigsTest.java index 6e593ccb9b71..1ee7192478e7 100644 --- a/test/unit/org/apache/cassandra/tools/ToolsEnvsConfigsTest.java +++ b/test/unit/org/apache/cassandra/tools/ToolsEnvsConfigsTest.java @@ -21,9 +21,9 @@ import java.util.Collections; import com.google.common.collect.ImmutableMap; - import org.junit.Test; +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.tools.ToolRunner.ToolResult; @@ -35,6 +35,7 @@ public class ToolsEnvsConfigsTest @Test public void testJDKEnvInfoDefaultCleaners() { + DatabaseDescriptor.daemonInitialization(); ToolResult tool = ToolRunner.invoke(ImmutableMap.of("_JAVA_OPTIONS", "-Djava.net.preferIPv4Stack=true"), null, CQLTester.buildNodetoolArgs(Collections.emptyList())); diff --git a/test/unit/org/apache/cassandra/tools/nodetool/SjkTest.java b/test/unit/org/apache/cassandra/tools/nodetool/SjkTest.java index 9812ef250bf6..65dcbede9555 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/SjkTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/SjkTest.java @@ -19,6 +19,7 @@ import org.junit.Test; +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.tools.ToolRunner; public class SjkTest @@ -26,6 +27,7 @@ public class SjkTest @Test public void sjkHelpReturnsRc0() { + DatabaseDescriptor.daemonInitialization(); ToolRunner.ToolResult tool = ToolRunner.invokeNodetool("sjk", "--help"); tool.assertOnExitCode(); } diff --git a/test/unit/org/apache/cassandra/utils/jmx/DuplicateJMXConfigurationTest.java b/test/unit/org/apache/cassandra/utils/jmx/DuplicateJMXConfigurationTest.java new file mode 100644 index 000000000000..976507badb3b --- /dev/null +++ b/test/unit/org/apache/cassandra/utils/jmx/DuplicateJMXConfigurationTest.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils.jmx; + +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.distributed.shared.WithProperties; +import org.apache.cassandra.exceptions.ConfigurationException; + +import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_CONFIG; +import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_REMOTE_PORT; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_AUTHENTICATE; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_CIPHER_SUITES; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_PROTOCOLS; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL_NEED_CLIENT_AUTH; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_NET_SSL_KEYSTORE; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_NET_SSL_KEYSTOREPASSWORD; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_NET_SSL_TRUSTSTORE; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_NET_SSL_TRUSTSTOREPASSWORD; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public class DuplicateJMXConfigurationTest +{ + @Test + public void testDuplicateConfiguration() + { + String enabledProtocols = "TLSv1.2,TLSv1.3,TLSv1.1"; + String cipherSuites = "TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256"; + + try (WithProperties props = new WithProperties().set(CASSANDRA_CONFIG, "cassandra-jmx-sslconfig.yaml") + .set(CASSANDRA_JMX_REMOTE_PORT, 7199) + .set(COM_SUN_MANAGEMENT_JMXREMOTE_AUTHENTICATE, true) + .set(COM_SUN_MANAGEMENT_JMXREMOTE_SSL, true) + .set(COM_SUN_MANAGEMENT_JMXREMOTE_SSL_NEED_CLIENT_AUTH, true) + .set(COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_PROTOCOLS, enabledProtocols) + .set(COM_SUN_MANAGEMENT_JMXREMOTE_SSL_ENABLED_CIPHER_SUITES, cipherSuites) + .set(JAVAX_NET_SSL_KEYSTORE, "test/conf/cassandra_ssl_test.keystore") + .set(JAVAX_NET_SSL_TRUSTSTORE, "test/conf/cassandra_ssl_test.truststore") + .set(JAVAX_NET_SSL_KEYSTOREPASSWORD, "cassandra") + .set(JAVAX_NET_SSL_TRUSTSTOREPASSWORD, "cassandra")) + { + assertThatThrownBy(DatabaseDescriptor::daemonInitialization) + .isInstanceOf(ConfigurationException.class) + .hasMessageContaining("Configure either jmx_server_options in cassandra.yaml and comment out configure_jmx function " + + "call in cassandra-env.sh or keep cassandra-env.sh to call configure_jmx function but you have to keep " + + "jmx_server_options in cassandra.yaml commented out."); + } + } +} diff --git a/test/unit/org/apache/cassandra/utils/jmx/JMXSslConfigTest.java b/test/unit/org/apache/cassandra/utils/jmx/JMXSslConfigTest.java index 2e739c558ba5..fb6a248dab96 100644 --- a/test/unit/org/apache/cassandra/utils/jmx/JMXSslConfigTest.java +++ b/test/unit/org/apache/cassandra/utils/jmx/JMXSslConfigTest.java @@ -30,10 +30,15 @@ import org.junit.Test; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.JMXServerOptions; import org.apache.cassandra.distributed.shared.WithProperties; import org.apache.cassandra.utils.JMXServerUtils; import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_CONFIG; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_NET_SSL_KEYSTORE; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_NET_SSL_KEYSTOREPASSWORD; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_NET_SSL_TRUSTSTORE; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_NET_SSL_TRUSTSTOREPASSWORD; import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES; import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS; @@ -68,10 +73,15 @@ public void testRemoteJmxSystemConfig() throws SSLException String enabledProtocols = "TLSv1.2,TLSv1.3,TLSv1.1"; String cipherSuites = "TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256"; - try (WithProperties ignored = JMXSslPropertiesUtil.use(true, true, enabledProtocols, - cipherSuites)) + try (WithProperties ignored = JMXSslPropertiesUtil.use(true, true, enabledProtocols, cipherSuites) + .set(JAVAX_NET_SSL_KEYSTORE, "test/conf/cassandra_ssl_test.keystore") + .set(JAVAX_NET_SSL_TRUSTSTORE, "test/conf/cassandra_ssl_test.truststore") + .set(JAVAX_NET_SSL_KEYSTOREPASSWORD, "cassandra") + .set(JAVAX_NET_SSL_TRUSTSTOREPASSWORD, "cassandra")) { - Map<String, Object> env = JMXServerUtils.configureJmxSocketFactories(serverAddress, false); + JMXServerOptions options = JMXServerOptions.createParsingSystemProperties(); + options.jmx_encryption_options.applyConfig(); + Map<String, Object> env = JMXServerUtils.configureJmxSocketFactories(serverAddress, options); Assert.assertNotNull("ServerSocketFactory must not be null", env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE)); Assert.assertTrue("RMI_SERVER_SOCKET_FACTORY must be of SslRMIServerSocketFactory type", env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE) instanceof SslRMIServerSocketFactory); Assert.assertNotNull("ClientSocketFactory must not be null", env.get(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE)); @@ -90,7 +100,8 @@ public void testLocalJmxServer() throws SSLException InetAddress serverAddress = InetAddress.getLoopbackAddress(); try (WithProperties ignored = JMXSslPropertiesUtil.use(false)) { - Map<String, Object> env = JMXServerUtils.configureJmxSocketFactories(serverAddress, true); + JMXServerOptions options = JMXServerOptions.fromDescriptor(true, true, 7199); + Map<String, Object> env = JMXServerUtils.configureJmxSocketFactories(serverAddress, options); Assert.assertNull("ClientSocketFactory must be null", env.get(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE)); Assert.assertNull("com.sun.jndi.rmi.factory.socket must not be set in the env", env.get("com.sun.jndi.rmi.factory.socket")); diff --git a/test/unit/org/apache/cassandra/utils/jmx/JMXSslConfiguredWithYamlFileOptionsTest.java b/test/unit/org/apache/cassandra/utils/jmx/JMXSslConfiguredWithYamlFileOptionsTest.java index 3855183a9259..c9d2d9f09184 100644 --- a/test/unit/org/apache/cassandra/utils/jmx/JMXSslConfiguredWithYamlFileOptionsTest.java +++ b/test/unit/org/apache/cassandra/utils/jmx/JMXSslConfiguredWithYamlFileOptionsTest.java @@ -31,9 +31,8 @@ import org.junit.Test; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.config.EncryptionOptions; +import org.apache.cassandra.config.JMXServerOptions; import org.apache.cassandra.distributed.shared.WithProperties; -import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.utils.JMXServerUtils; import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_CONFIG; @@ -66,15 +65,15 @@ public static void tearDownDatabaseDescriptor() @Test public void testYamlFileJmxEncryptionOptions() throws SSLException { - EncryptionOptions jmxEncryptionOptions = DatabaseDescriptor.getJmxEncryptionOptions(); - String expectedProtocols = StringUtils.join(jmxEncryptionOptions.getAcceptedProtocols(), ","); - String expectedCipherSuites = StringUtils.join(jmxEncryptionOptions.cipherSuitesArray(), ","); + JMXServerOptions serverOptions = DatabaseDescriptor.getJmxServerOptions(); + String expectedProtocols = StringUtils.join(serverOptions.jmx_encryption_options.getAcceptedProtocols(), ","); + String expectedCipherSuites = StringUtils.join(serverOptions.jmx_encryption_options.cipherSuitesArray(), ","); InetAddress serverAddress = InetAddress.getLoopbackAddress(); try (WithProperties ignored = JMXSslPropertiesUtil.use(false)) { - Map<String, Object> env = JMXServerUtils.configureJmxSocketFactories(serverAddress, false); + Map<String, Object> env = JMXServerUtils.configureJmxSocketFactories(serverAddress, serverOptions); Assert.assertTrue("com.sun.management.jmxremote.ssl must be true", COM_SUN_MANAGEMENT_JMXREMOTE_SSL.getBoolean()); Assert.assertNotNull("ServerSocketFactory must not be null", env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE)); Assert.assertTrue("RMI_SERVER_SOCKET_FACTORY must be of JMXSslRMIServerSocketFactory type", env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE) instanceof SslRMIServerSocketFactory); @@ -84,21 +83,4 @@ public void testYamlFileJmxEncryptionOptions() throws SSLException Assert.assertEquals("javax.rmi.ssl.client.enabledCipherSuites must match", expectedCipherSuites, JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES.getString()); } } - - /** - * Tests for the error scenario when the JMX SSL configuration is provided as - * system configuration as well as encryption_options. - * - * @throws SSLException - */ - @Test(expected = ConfigurationException.class) - public void testDuplicateConfig() throws SSLException - { - InetAddress serverAddress = InetAddress.getLoopbackAddress(); - - try (WithProperties ignored = JMXSslPropertiesUtil.use(true)) - { - JMXServerUtils.configureJmxSocketFactories(serverAddress, false); - } - } } diff --git a/test/unit/org/apache/cassandra/utils/jmx/JMXSslDisabledEncryptionOptionsTest.java b/test/unit/org/apache/cassandra/utils/jmx/JMXSslDisabledEncryptionOptionsTest.java index ef2070f2aa20..841a93357fc1 100644 --- a/test/unit/org/apache/cassandra/utils/jmx/JMXSslDisabledEncryptionOptionsTest.java +++ b/test/unit/org/apache/cassandra/utils/jmx/JMXSslDisabledEncryptionOptionsTest.java @@ -29,6 +29,7 @@ import org.junit.Test; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.JMXServerOptions; import org.apache.cassandra.distributed.shared.WithProperties; import org.apache.cassandra.utils.JMXServerUtils; @@ -71,7 +72,7 @@ public void testDisabledJmxSslConfigs() throws SSLException try (WithProperties ignored = JMXSslPropertiesUtil.use(false)) { - Map<String, Object> env = JMXServerUtils.configureJmxSocketFactories(serverAddress, false); + Map<String, Object> env = JMXServerUtils.configureJmxSocketFactories(serverAddress, JMXServerOptions.fromDescriptor(true, false, 7199)); Assert.assertTrue("no properties must be set", env.isEmpty()); Assert.assertFalse("com.sun.management.jmxremote.ssl must be false", COM_SUN_MANAGEMENT_JMXREMOTE_SSL.getBoolean()); Assert.assertNull("javax.rmi.ssl.client.enabledProtocols must be null", JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS.getString()); @@ -90,7 +91,7 @@ public void testFallbackToLocalJmxServer() throws SSLException try (WithProperties ignored = JMXSslPropertiesUtil.use(false)) { - Map<String, Object> env = JMXServerUtils.configureJmxSocketFactories(serverAddress, true); + Map<String, Object> env = JMXServerUtils.configureJmxSocketFactories(serverAddress, JMXServerOptions.fromDescriptor(true, true, 7199)); Assert.assertFalse("com.sun.management.jmxremote.ssl must be false", COM_SUN_MANAGEMENT_JMXREMOTE_SSL.getBoolean()); Assert.assertNull("com.sun.jndi.rmi.factory.socket must be null", env.get("com.sun.jndi.rmi.factory.socket")); Assert.assertNotNull("ServerSocketFactory must not be null", env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE)); diff --git a/test/unit/org/apache/cassandra/utils/jmx/JMXSslPEMConfigTest.java b/test/unit/org/apache/cassandra/utils/jmx/JMXSslPEMConfigTest.java index c7db7b2c724b..01459954e61d 100644 --- a/test/unit/org/apache/cassandra/utils/jmx/JMXSslPEMConfigTest.java +++ b/test/unit/org/apache/cassandra/utils/jmx/JMXSslPEMConfigTest.java @@ -31,7 +31,7 @@ import org.junit.Test; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.config.EncryptionOptions; +import org.apache.cassandra.config.JMXServerOptions; import org.apache.cassandra.distributed.shared.WithProperties; import org.apache.cassandra.utils.JMXServerUtils; @@ -63,15 +63,15 @@ public static void tearDownDatabaseDescriptor() @Test public void testPEMBasedJmxSslConfig() throws SSLException { - EncryptionOptions jmxEncryptionOptions = DatabaseDescriptor.getJmxEncryptionOptions(); - String expectedProtocols = StringUtils.join(jmxEncryptionOptions.getAcceptedProtocols(), ","); - String expectedCipherSuites = StringUtils.join(jmxEncryptionOptions.cipherSuitesArray(), ","); + JMXServerOptions serverOptions = DatabaseDescriptor.getJmxServerOptions(); + String expectedProtocols = StringUtils.join(serverOptions.jmx_encryption_options.getAcceptedProtocols(), ","); + String expectedCipherSuites = StringUtils.join(serverOptions.jmx_encryption_options.cipherSuitesArray(), ","); InetAddress serverAddress = InetAddress.getLoopbackAddress(); try (WithProperties ignored = JMXSslPropertiesUtil.use(false)) { - Map<String, Object> env = JMXServerUtils.configureJmxSocketFactories(serverAddress, false); + Map<String, Object> env = JMXServerUtils.configureJmxSocketFactories(serverAddress, serverOptions); Assert.assertTrue("com.sun.management.jmxremote.ssl must be true", COM_SUN_MANAGEMENT_JMXREMOTE_SSL.getBoolean()); Assert.assertNotNull("ServerSocketFactory must not be null", env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE)); Assert.assertTrue("RMI_SERVER_SOCKET_FACTORY must be of JMXSslRMIServerSocketFactory type", env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE) instanceof SslRMIServerSocketFactory); From 48dcf5e0921b8a5b7a73fa15ebfb2341d48c857b Mon Sep 17 00:00:00 2001 From: Sam Tunnicliffe <samt@apache.org> Date: Tue, 18 Jun 2024 17:29:46 +0100 Subject: [PATCH 054/225] Snitch re-implementation Deprecate IEndpointSnitch entirely, to be replaced with new interfaces: * Locator for endpoint -> location mapping * InitialLocationProvider to supply the DC & rack for registration * NodeProximity for sorting endpoints and replicas at query time For migration/upgrade/deprecation, nodes can still be configured with an IEndpointSnitch implementation via endpoint_snitch in config, but we hide this with a facade and only present the new interfaces. Patch by Sam Tunnicliffe and Marcus Eriksson; reviewed by Sam Tunnicliffe and Marcus Eriksson for CASSANDRA-19488 Co-authored-by: Marcus Eriksson <marcuse@apache.org> Co-authored-by: Sam Tunnicliffe <samt@apache.org> --- NEWS.txt | 43 ++-- conf/cassandra.yaml | 94 ++++++++ conf/cassandra_latest.yaml | 172 +++++++------- .../org/apache/cassandra/config/Config.java | 5 + .../cassandra/config/DatabaseDescriptor.java | 163 ++++++++++---- .../cassandra/config/EncryptionOptions.java | 20 +- .../cql3/statements/DescribeStatement.java | 9 +- .../apache/cassandra/db/ConsistencyLevel.java | 11 +- .../db/CounterMutationVerbHandler.java | 2 +- .../apache/cassandra/db/SystemKeyspace.java | 22 +- .../apache/cassandra/db/view/ViewUtils.java | 6 +- .../db/virtual/InternodeInboundTable.java | 6 +- .../db/virtual/InternodeOutboundTable.java | 6 +- .../db/virtual/PendingHintsTable.java | 11 +- .../apache/cassandra/dht/BootStrapper.java | 2 +- .../org/apache/cassandra/dht/Datacenters.java | 2 +- .../dht/RangeFetchMapCalculator.java | 9 +- .../apache/cassandra/dht/RangeStreamer.java | 44 ++-- .../tokenallocator/OfflineTokenAllocator.java | 33 +-- .../dht/tokenallocator/TokenAllocation.java | 73 +++--- .../org/apache/cassandra/gms/Gossiper.java | 5 +- ...AbstractCloudMetadataServiceConnector.java | 8 +- .../AbstractCloudMetadataServiceSnitch.java | 60 +---- .../locator/AbstractEndpointSnitch.java | 23 +- .../AbstractNetworkTopologySnitch.java | 41 +--- .../locator/AlibabaCloudLocationProvider.java | 50 +++++ .../cassandra/locator/AlibabaCloudSnitch.java | 9 +- .../locator/AzureCloudLocationProvider.java | 100 +++++++++ .../apache/cassandra/locator/AzureSnitch.java | 54 +---- .../cassandra/locator/BaseProximity.java | 58 +++++ .../CloudMetadataLocationProvider.java | 68 ++++++ .../locator/CloudstackLocationProvider.java | 135 +++++++++++ .../cassandra/locator/CloudstackSnitch.java | 92 +------- .../locator/DynamicEndpointSnitch.java | 65 ++---- .../locator/Ec2LocationProvider.java | 162 +++++++++++++ .../locator/Ec2MultiRegionAddressConfig.java | 97 ++++++++ .../locator/Ec2MultiRegionSnitch.java | 47 ++-- .../apache/cassandra/locator/Ec2Snitch.java | 100 +-------- .../cassandra/locator/EndpointSnitchInfo.java | 13 +- .../locator/EndpointSnitchInfoMBean.java | 3 + .../locator/GoogleCloudLocationProvider.java | 54 +++++ .../cassandra/locator/GoogleCloudSnitch.java | 18 +- .../locator/GossipingPropertyFileSnitch.java | 105 ++------- .../cassandra/locator/IEndpointSnitch.java | 10 +- .../org/apache/cassandra/locator/InOurDc.java | 31 ++- .../locator/InitialLocationProvider.java | 44 ++++ .../cassandra/locator/LocationInfo.java | 66 ++++++ .../cassandra/locator/LocationInfoMBean.java | 51 +++++ .../org/apache/cassandra/locator/Locator.java | 195 ++++++++++++++++ .../locator/NetworkTopologyProximity.java | 51 +++++ .../cassandra/locator/NoOpProximity.java | 37 +++ .../cassandra/locator/NodeAddressConfig.java | 47 ++++ .../cassandra/locator/NodeProximity.java | 38 ++++ .../cassandra/locator/PropertyFileSnitch.java | 139 +----------- .../locator/RackDCFileLocationProvider.java | 72 ++++++ .../locator/RackInferringSnitch.java | 32 ++- .../locator/ReconnectableSnitchHelper.java | 34 +-- .../cassandra/locator/ReplicaLayout.java | 4 +- .../cassandra/locator/ReplicaPlans.java | 91 ++++---- .../apache/cassandra/locator/Replicas.java | 12 +- .../locator/SimpleLocationProvider.java | 32 +++ .../cassandra/locator/SimpleSnitch.java | 31 ++- .../cassandra/locator/SnitchAdapter.java | 84 +++++++ .../apache/cassandra/locator/SnitchUtils.java | 9 +- .../locator/TopologyFileLocationProvider.java | 160 +++++++++++++ .../cassandra/metrics/MessagingMetrics.java | 2 +- .../net/OutboundConnectionSettings.java | 18 +- .../apache/cassandra/repair/RepairJob.java | 4 +- .../cassandra/repair/SharedContext.java | 12 +- .../DistributedMetadataLogKeyspace.java | 6 + .../cassandra/schema/DistributedSchema.java | 21 +- .../cassandra/schema/ReplicationParams.java | 6 + .../cassandra/service/CassandraDaemon.java | 24 +- .../DatacenterSyncWriteResponseHandler.java | 12 +- .../org/apache/cassandra/service/Rebuild.java | 6 +- .../cassandra/service/StartupChecks.java | 42 ---- .../cassandra/service/StorageProxy.java | 10 +- .../cassandra/service/StorageService.java | 58 +++-- .../service/StorageServiceMBean.java | 22 +- .../apache/cassandra/service/TokenRange.java | 12 +- .../cassandra/service/paxos/PaxosCommit.java | 4 +- .../cassandra/service/paxos/PaxosRepair.java | 7 +- .../cassandra/service/reads/ReadCallback.java | 2 +- .../reads/range/ReplicaPlanMerger.java | 4 +- .../streaming/DataMovementVerbHandler.java | 4 +- .../cassandra/streaming/StreamManager.java | 7 +- .../apache/cassandra/tcm/ClusterMetadata.java | 5 + .../cassandra/tcm/ClusterMetadataService.java | 10 +- .../cassandra/tcm/RegistrationStatus.java | 67 ++++++ .../org/apache/cassandra/tcm/Startup.java | 3 +- .../tcm/StubClusterMetadataService.java | 4 +- .../tcm/compatibility/TokenRingUtils.java | 2 +- .../tcm/listeners/LegacyStateListener.java | 12 +- .../tcm/listeners/SchemaListener.java | 3 +- .../apache/cassandra/tcm/log/LocalLog.java | 18 +- .../cassandra/tcm/membership/Directory.java | 1 + .../cassandra/tcm/membership/Location.java | 6 + .../tcm/membership/NodeAddresses.java | 1 - .../apache/cassandra/tcm/sequences/Move.java | 5 +- .../cassandra/tcm/serialization/Version.java | 1 + .../tcm/transformations/Register.java | 26 ++- .../transformations/cms/PreInitialize.java | 60 +++-- .../org/apache/cassandra/tools/NodeProbe.java | 13 ++ .../tools/nodetool/DescribeCluster.java | 13 +- .../apache/cassandra/utils/FBUtilities.java | 7 + .../distributed/action/GossipHelper.java | 2 +- ...istributedTestInitialLocationProvider.java | 42 ++++ .../impl/DistributedTestSnitch.java | 128 ----------- .../cassandra/distributed/impl/Instance.java | 10 +- .../distributed/impl/InstanceConfig.java | 26 ++- .../distributed/impl/TestEndpointCache.java | 54 +++++ .../distributed/impl/UnsafeGossipHelper.java | 2 +- .../mock/nodetool/InternalNodeProbe.java | 5 +- .../distributed/shared/ClusterUtils.java | 3 +- .../distributed/test/GossipTest.java | 2 +- .../distributed/test/IPMembershipTest.java | 7 +- .../test/ReconnectToInternalIPTest.java | 198 ++++++++++++++++ .../test/log/BounceResetHostIdTest.java | 12 +- .../test/log/CMSMembershipMetricsTest.java | 8 +- .../distributed/test/log/CMSTestBase.java | 23 +- .../test/log/ClusterMetadataTestHelper.java | 20 +- .../test/log/CoordinatorPathTestBase.java | 2 +- .../log/MetadataChangeSimulationTest.java | 67 +----- .../test/log/NTSSimulationTest.java | 71 ++++++ .../log/SimpleStrategySimulationTest.java | 81 +++++++ .../topology/DecommissionAvoidTimeouts.java | 2 +- .../DynamicEndpointSnitchLongTest.java | 4 +- .../simulator/systems/SimulatedSnitch.java | 2 + .../org/apache/cassandra/ServerTestUtils.java | 27 +-- .../batchlog/BatchlogEndpointFilterTest.java | 52 ++--- .../config/DatabaseDescriptorRefTest.java | 7 + .../org/apache/cassandra/cql3/PagingTest.java | 26 +-- .../statements/DescribeStatementTest.java | 6 +- .../org/apache/cassandra/db/CleanupTest.java | 19 -- .../db/ColumnFamilyStoreClientModeTest.java | 3 - .../cassandra/db/SystemKeyspaceTest.java | 4 +- .../cassandra/db/filter/ColumnFilterTest.java | 12 - ...GuardrailMaximumReplicationFactorTest.java | 21 +- ...GuardrailMinimumReplicationFactorTest.java | 23 +- .../cassandra/db/view/ViewUtilsTest.java | 1 - .../cassandra/dht/BootStrapperTest.java | 2 +- .../dht/RangeFetchMapCalculatorTest.java | 53 ++--- .../tokenallocator/TokenAllocationTest.java | 193 ++++++++-------- .../locator/AlibabaCloudSnitchTest.java | 42 ++-- .../AssureSufficientLiveNodesTest.java | 25 +-- .../locator/AzureConnectorMockingTest.java | 18 +- .../locator/CloudstackSnitchTest.java | 42 ++-- .../locator/DynamicEndpointSnitchTest.java | 4 +- .../cassandra/locator/Ec2SnitchTest.java | 212 ++++++++++-------- .../locator/GoogleCloudSnitchTest.java | 46 ++-- .../GossipingPropertyFileSnitchTest.java | 32 +-- .../locator/NetworkTopologyStrategyTest.java | 85 ++++--- .../cassandra/locator/PendingRangesTest.java | 32 +-- .../locator/PropertyFileSnitchTest.java | 49 ++-- .../ReconnectableSnitchHelperTest.java | 2 +- .../cassandra/locator/ReplicaPlansTest.java | 115 ++++------ .../cassandra/locator/SimpleStrategyTest.java | 13 -- .../cassandra/locator/SnitchUtilsTest.java | 11 + .../cassandra/locator/WithPartitioner.java | 1 + .../net/AsyncStreamingOutputPlusTest.java | 2 +- .../apache/cassandra/net/ConnectionTest.java | 1 + .../apache/cassandra/net/HandshakeTest.java | 2 + .../cassandra/net/MessagingServiceTest.java | 1 + .../net/OutboundConnectionSettingsTest.java | 77 +++---- .../net/OutboundConnectionsTest.java | 6 + .../net/ProxyHandlerConnectionsTest.java | 3 + .../apache/cassandra/repair/FuzzTestBase.java | 14 +- .../service/StorageServiceServerTest.java | 28 --- .../cassandra/service/StorageServiceTest.java | 28 +-- .../service/WriteResponseHandlerTest.java | 40 +--- .../WriteResponseHandlerTransientTest.java | 53 +---- .../service/reads/ReadExecutorTest.java | 4 + .../reads/range/ReplicaPlanMergerTest.java | 14 +- .../reads/repair/AbstractReadRepairTest.java | 18 -- .../service/reads/repair/ReadRepairTest.java | 5 + .../ClusterMetadataTransformationTest.java | 8 - .../apache/cassandra/tcm/GetLogStateTest.java | 2 +- .../apache/cassandra/tcm/LogStateTest.java | 2 +- .../cassandra/tools/nodetool/RingTest.java | 6 +- .../cassandra/tools/nodetool/StatusTest.java | 8 +- 180 files changed, 3897 insertions(+), 2284 deletions(-) create mode 100644 src/java/org/apache/cassandra/locator/AlibabaCloudLocationProvider.java create mode 100644 src/java/org/apache/cassandra/locator/AzureCloudLocationProvider.java create mode 100644 src/java/org/apache/cassandra/locator/BaseProximity.java create mode 100644 src/java/org/apache/cassandra/locator/CloudMetadataLocationProvider.java create mode 100644 src/java/org/apache/cassandra/locator/CloudstackLocationProvider.java create mode 100644 src/java/org/apache/cassandra/locator/Ec2LocationProvider.java create mode 100644 src/java/org/apache/cassandra/locator/Ec2MultiRegionAddressConfig.java create mode 100644 src/java/org/apache/cassandra/locator/GoogleCloudLocationProvider.java create mode 100644 src/java/org/apache/cassandra/locator/InitialLocationProvider.java create mode 100644 src/java/org/apache/cassandra/locator/LocationInfo.java create mode 100644 src/java/org/apache/cassandra/locator/LocationInfoMBean.java create mode 100644 src/java/org/apache/cassandra/locator/Locator.java create mode 100644 src/java/org/apache/cassandra/locator/NetworkTopologyProximity.java create mode 100644 src/java/org/apache/cassandra/locator/NoOpProximity.java create mode 100644 src/java/org/apache/cassandra/locator/NodeAddressConfig.java create mode 100644 src/java/org/apache/cassandra/locator/NodeProximity.java create mode 100644 src/java/org/apache/cassandra/locator/RackDCFileLocationProvider.java create mode 100644 src/java/org/apache/cassandra/locator/SimpleLocationProvider.java create mode 100644 src/java/org/apache/cassandra/locator/SnitchAdapter.java create mode 100644 src/java/org/apache/cassandra/locator/TopologyFileLocationProvider.java create mode 100644 src/java/org/apache/cassandra/tcm/RegistrationStatus.java create mode 100644 test/distributed/org/apache/cassandra/distributed/impl/DistributedTestInitialLocationProvider.java delete mode 100644 test/distributed/org/apache/cassandra/distributed/impl/DistributedTestSnitch.java create mode 100644 test/distributed/org/apache/cassandra/distributed/impl/TestEndpointCache.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/ReconnectToInternalIPTest.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/log/NTSSimulationTest.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/log/SimpleStrategySimulationTest.java diff --git a/NEWS.txt b/NEWS.txt index b493e48d5187..22921153da2f 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -108,7 +108,6 @@ New features the respective configuration sections and by commenting out `configure_jmx` function call in cassandra-env.sh. Enabling both ways of configuring JMX will result in a node failing to start. - Upgrading --------- [The following is a placeholder, to be revised asap] @@ -119,12 +118,12 @@ Upgrading 2. The Cluster Metadata Service (CMS) has been enabled The first step is just a regular upgrade, there are no changes to external APIs, SSTable formats to consider. Step 2 requires an operator to run a new nodetool subcommand, intializecms, on one node in the cluster. - - > nodetool initializecms - Doing so creates the CMS with the local node as its only member. The initializecms command cannot be executed until + - > nodetool cms initialize + Doing so creates the CMS with the local node as its only member. The initialize command cannot be executed until step 1 is complete and all nodes are running 5.1 as migrating metadata management over to the new TCM system requires all nodes to be in agreement over the current state of the cluster. Essentially this means agreement on - schema and topology. Once the upgrade has started, but before initializecms is run, metadata-changing operations are - not permitted and if attempted from an upgraded node will be rejected. + schema and topology. Once the upgrade has started, but before the initialize command is run, metadata-changing + operations are not permitted and if attempted from an upgraded node will be rejected. Prohibited operations include: - schema changes - node replacement @@ -136,22 +135,22 @@ Upgrading a previous version, though this is planned before release. Any automation which can trigger these operations should be disabled for the cluster prior to starting the upgrade. Should any of the prohibited operations be executed (i.e. on a node that is still running a pre-5.1 version) before - the CMS migration, nodes which are DOWN or which have been upgraded will not process the metadata changes. However, - nodes still UP and running the old version will. This will eventually cause the migration to fail, as the cluster - will not be in agreement. - - > nodetool initializecms + the CMS intialization, nodes which are DOWN or which have been upgraded will not process the metadata changes. + However, nodes still UP and running the old version will. This will eventually cause the migration to fail, as the + cluster will not be in agreement. + - > nodetool cms initialize Got mismatching cluster metadatas from [/x.x.x.x:7000] aborting migration See 'nodetool help' or 'nodetool help <command>'. - If the initializecms command fails, it will indicate which nodes’ current metadata does not agree with the node + If the cms initialize command fails, it will indicate which nodes’ current metadata does not agree with the node where the command was executed. To mitigate this situation, bring any mismatching nodes DOWN and rerun the - initializecms command with the additional —ignore flag. - - nodetool intializecms -ignore x.x.x.x + cms initialize command with the additional —ignore flag. + - nodetool cms intialize -ignore x.x.x.x Once the command has run successfully the ignored nodes can be restarted but any metadata changes that they accepted/and or applied whilst the cluster was in mixed mode will be lost. We plan to improve this before beta, but in the meantime operators should ensure no schema or membership/ownership changes are performed during the upgrade. Although the restrictions on metadata-changing operations will be removed as soon as the initial CMS migration is complete, at that point the CMS will only contain a single member, which is not suitable for real clusters. To - modify the membership of the CMS a second nodetool subcommand, reconfigurecms, should be run. This command allows + modify the membership of the CMS a second nodetool subcommand, cms reconfigure, should be run. This command allows the number of required number of CMS members to be specified for each datacenter. Consensus for metadata operations by the CMS is obtained via Paxos, operating at SERIAL/QUORUM consistency, so the minimum safe size for the CMS is 3 nodes. In multi-DC deployments, the CMS members may be distributed across DCs to ensure resilience in the case of a @@ -173,7 +172,23 @@ Upgrading Deprecation ----------- - `use_deterministic_table_id` is no longer supported and should be removed from cassandra.yaml. Table IDs may still be supplied explicitly on CREATE. - + - IEndpointSnitch has been deprecated as ClusterMetadata is now the source of truth regarding topology information. + The responsibilities of the snitch have been broken out to a handful of new classes: + * o.a.c.locator.Locator provides datacenter and rack info for endpoints. This is not configurable as topology is + always sourced from ClusterMetadata. + * o.a.c.locator.InitialLocationProvider supplies the local datacenter and rack for a new node joining a cluster + for the first time. This will be used exactly once to register the Location of the node in ClusterMetadata and + is configurable using the `initial_location_provider` yaml setting. + * o.a.c.locator.NodeProximity handles sorting and ranking of replica lists. It is configurable via the + `node_proximity` yaml setting. + * o.a.c.locator.NodeAddressConfig exists mainly to support the functionality of ReconnectableSnitchHelper and + Ec2MultiRegionSnitch which dynamically configures the broadcast address based on cloud metadata. Optionally + configurable using the `address_config` yaml setting. + For migration and to allow us to deprecate snitches in a controlled way, use of endpoint_snitch in yaml remains + fully supported and all existing IEndpointSnitch implementations should continue to work seamlessly and no action + is required by operators at upgrade time. The in-tree snitches have been refactored to extract implementations of + the new interfaces so that their functionality can also be used via the new `initial_location_provider`, + `node_proximity` and `address_config` settings. 5.0 === diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index f8d9f354cbf0..6ffe1fb9b6b2 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -1465,6 +1465,9 @@ slow_query_log_timeout: 500ms # most users should never need to adjust this. # phi_convict_threshold: 8 +# IEndpointSnitch has been deprecated in Cassandra 5.1 +# Configuring a cluster with an IEndpointSnitch implementation using the endpoint_snitch setting remains supported, +# but is superceded by the new settings detailed below. # endpoint_snitch -- Set this to a class that implements # IEndpointSnitch. The snitch has two functions: # @@ -1550,6 +1553,97 @@ slow_query_log_timeout: 500ms # of the snitch, which will be assumed to be on your classpath. endpoint_snitch: SimpleSnitch +# The settings in the following section are intended to supercede the use of endpoint_snitch: +# initial_location_provider +# node_proximity +# addresses_config (optional) +# prefer_local_connections (optional, defaults to false) + +# The initial location provider supplies the datacenter and rack for a new node joining the +# cluster for the first time. This DC/rack is used to register the node with the cluster and +# is then persisted in and propagated by cluster metadata. The initial location provider is +# not used after this initial phase of the node lifecycle is complete. Cassandra provides the +# following implementations: +# +# SimpleLocationProvider: +# Hardcoded to supply a static location of `datacenter1/rack1`. Replicates the behaviour of +# SimpleSnitch. +# +# RackDCFileLocationProvider: +# The rack and datacenter for the local node are defined in cassandra-rackdc.properties, +# and is backwards compatible with the configuration of GossipingPropertyFileSnitch. +# +# TopologyFileLocationProvider: +# The rack and datacenter for the local node are defined in cassandra-topology.properties, +# and is backwards compatible with the configuration of PropertyFileSnitch. Only the location +# info for the local node is relevant now, rack/dc specifications for other endpoints are +# ignored. +# +# AlibabaCloudLocationProvider: +# Obtains datacenter and rack of the node from the metadata service of Alibaba cloud. This +# maps the ECS region to datacenter and the ECS availability zone to the rack. Intended to +# replace use of AlibabaCloudSnitch. +# +# AzureCloudLocationProvider: +# Maps datacenter to `location` and rack to `zone` fields of the `compute` object obtained +# from the Azure metadata service. If the availablity zone is not enabled, it will use the +# failure domain and get its respective value. Intended to replace use of AzureSnitch. +# +# CloudstackLocationProvider: +# Assumes a Cloudstack Zone follows the typical convention country-location-az, using a +# country/location tuple as a datacenter and the availability zone as a rack. Intended to +# replace use of CloudstackSnitch. +# WARNING: This legacy snitch has been deprecated and is scheduled to be removed in a future +# version of Cassandra. +# +# Ec2LocationProvider: +# Loads Region and Availability Zone information from the EC2 API. The Region is treated as the +# datacenter and the AvailablilityZone as the rack. Intended to replace use of Ec2Snitch and +# Ec2MultiRegionSnitch. +# +# GoogleCloudLocationProvider: +# Fetches source data from the metadata service of Google Cloud. This provider maps the GCE +# region to datacenter and the availibility zone to the rack. Intended to replace use of +# GoogleCloudSnitch. +#initial_location_provider: SimpleLocationProvider + +# Node proximity controls sorting and ranking of replica lists in order to route requests efficiently. +# Dynamic snitch (not configured here) provides an additional level of sorting on top of whatever is +# specified here of sorting based on recorded latencies between peers. +# +# Out of the box Cassandra provides: +# +# NoOpProximity: +# Performs no reordering of replica lists and assigns every endpoint equal proximity, regardless of +# actual topology. This replicates the behaviour of SimpleSnitch. +# +# NetworkTopologyProximity: +# Ranks proximity of nodes based on datacenter and rack. Nodes in the same rack are considered +# closest, followed by those in the same datacenter. Nodes in different datacenters are +# considered furthest apart. Replicates the behaviour of topology aware snitches such as +# GossipingPropertyFileSnitch, PropertyFileSnitch and the multiple cloud platform snitches. +#node_proximity: NetworkTopologyProximity + +# Address configuration (public and private endpoint addresses) is almost always derived directly +# from this configuration file using broadcast_address. In certain deployments, both the public and +# private addresses may be configured from an external source. For instance, in EC2 a node will have +# discover both of its own addresses at startup by querying the appropriate cloud metadata service. +# In multi-region EC2 deployments, this should be set to Ec2MultiRegionAddressConfig, to replicate the +# behaviour of Ec2MultiRegionSnitch. +#addresses_config: Ec2MultiRegionAddressConfig + +# Controls whether or not to ensure that connections to peers in the same datacenter are established +# using private addresses. Typically, this is for situations like EC2 where a node will have a public +# address and a private address. We may initially connect on the public, then discover the private, and +# reconnect on the private. +# In Ec2MultiRegionSnitch (now deprecated), this behaviour was hard coded so if migrating from that +# snitch to modern config, set this to true. +# In GossipingPropertyFileSnitch (now deprecated), this behaviour was configured by the `prefer_local` +# property so if migrating from that snitch to modern config, set this accordingly. +# Note that all of the deprecated in-tree snitches can still be used in configuration, so any config +# migration is currently optional. +#prefer_local_connections: false + # controls how often to perform the more expensive part of host score # calculation # Min unit: ms diff --git a/conf/cassandra_latest.yaml b/conf/cassandra_latest.yaml index 109811956db4..4f8879a43c02 100644 --- a/conf/cassandra_latest.yaml +++ b/conf/cassandra_latest.yaml @@ -1445,90 +1445,94 @@ slow_query_log_timeout: 500ms # most users should never need to adjust this. # phi_convict_threshold: 8 -# endpoint_snitch -- Set this to a class that implements -# IEndpointSnitch. The snitch has two functions: -# -# - it teaches Cassandra enough about your network topology to route -# requests efficiently -# - it allows Cassandra to spread replicas around your cluster to avoid -# correlated failures. It does this by grouping machines into -# "datacenters" and "racks." Cassandra will do its best not to have -# more than one replica on the same "rack" (which may not actually -# be a physical location) -# -# CASSANDRA WILL NOT ALLOW YOU TO SWITCH TO AN INCOMPATIBLE SNITCH -# ONCE DATA IS INSERTED INTO THE CLUSTER. This would cause data loss. -# This means that if you start with the default SimpleSnitch, which -# locates every node on "rack1" in "datacenter1", your only options -# if you need to add another datacenter are GossipingPropertyFileSnitch -# (and the older PFS). From there, if you want to migrate to an -# incompatible snitch like Ec2Snitch you can do it by adding new nodes -# under Ec2Snitch (which will locate them in a new "datacenter") and -# decommissioning the old ones. -# -# Out of the box, Cassandra provides: -# -# SimpleSnitch: -# Treats Strategy order as proximity. This can improve cache -# locality when disabling read repair. Only appropriate for -# single-datacenter deployments. -# -# GossipingPropertyFileSnitch -# This should be your go-to snitch for production use. The rack -# and datacenter for the local node are defined in -# cassandra-rackdc.properties and propagated to other nodes via -# gossip. If cassandra-topology.properties exists, it is used as a -# fallback, allowing migration from the PropertyFileSnitch. -# -# PropertyFileSnitch: -# Proximity is determined by rack and data center, which are -# explicitly configured in cassandra-topology.properties. -# -# AlibabaCloudSnitch: -# Snitch for getting dc and rack of a node from metadata service of Alibaba cloud. -# This snitch that assumes an ECS region is a DC and an ECS availability_zone is a rack. -# -# AzureSnitch: -# Gets datacenter from 'location' and rack from 'zone' fields of 'compute' object -# from instance metadata service. If the availability zone is not enabled, it will use the fault -# domain and get its respective value. -# -# CloudstackSnitch: -# A snitch that assumes a Cloudstack Zone follows the typical convention -# country-location-az and uses a country/location tuple as a datacenter -# and the availability zone as a rack. -# WARNING: This snitch is deprecated and it is scheduled to be removed -# in the next major version of Cassandra. -# -# Ec2Snitch: -# Appropriate for EC2 deployments in a single Region. Loads Region -# and Availability Zone information from the EC2 API. The Region is -# treated as the datacenter, and the Availability Zone as the rack. -# Only private IPs are used, so this will not work across multiple -# Regions. -# -# Ec2MultiRegionSnitch: -# Uses public IPs as broadcast_address to allow cross-region -# connectivity. (Thus, you should set seed addresses to the public -# IP as well.) You will need to open the storage_port or -# ssl_storage_port on the public IP firewall. (For intra-Region -# traffic, Cassandra will switch to the private IP after -# establishing a connection.) -# -# GoogleCloudSnitch: -# Snitch for getting dc and rack of a node from metadata service of Google cloud. -# This snitch that assumes an GCE region is a DC and an GCE availability_zone is a rack. -# -# RackInferringSnitch: -# Proximity is determined by rack and data center, which are -# assumed to correspond to the 3rd and 2nd octet of each node's IP -# address, respectively. Unless this happens to match your -# deployment conventions, this is best used as an example of -# writing a custom Snitch class and is provided in that spirit. -# -# You can use a custom Snitch by setting this to the full class name -# of the snitch, which will be assumed to be on your classpath. -endpoint_snitch: SimpleSnitch +# The settings in the following section are intended to supercede the use of endpoint_snitch: +# initial_location_provider +# node_proximity +# addresses_config (optional) +# prefer_local_connections (optional, defaults to false) + +# The initial location provider supplies the datacenter and rack for a new node joining the +# cluster for the first time. This DC/rack is used to register the node with the cluster and +# is then persisted in and propagated by cluster metadata. The initial location provider is +# not used after this initial phase of the node lifecycle is complete. Cassandra provides the +# following implementations: +# +# SimpleLocationProvider: +# Hardcoded to supply a static location of `datacenter1/rack1`. Replicates the behaviour of +# SimpleSnitch. +# +# RackDCFileLocationProvider: +# The rack and datacenter for the local node are defined in cassandra-rackdc.properties, +# and is backwards compatible with the configuration of GossipingPropertyFileSnitch. +# +# TopologyFileLocationProvider: +# The rack and datacenter for the local node are defined in cassandra-topology.properties, +# and is backwards compatible with the configuration of PropertyFileSnitch. Only the location +# info for the local node is relevant now, rack/dc specifications for other endpoints are +# ignored. +# +# AlibabaCloudLocationProvider: +# Obtains datacenter and rack of the node from the metadata service of Alibaba cloud. This +# maps the ECS region to datacenter and the ECS availability zone to the rack. Intended to +# replace use of AlibabaCloudSnitch. +# +# AzureCloudLocationProvider: +# Maps datacenter to `location` and rack to `zone` fields of the `compute` object obtained +# from the Azure metadata service. If the availablity zone is not enabled, it will use the +# failure domain and get its respective value. Intended to replace use of AzureSnitch. +# +# CloudstackLocationProvider: +# Assumes a Cloudstack Zone follows the typical convention country-location-az, using a +# country/location tuple as a datacenter and the availability zone as a rack. Intended to +# replace use of CloudstackSnitch. +# WARNING: This legacy snitch has been deprecated and is scheduled to be removed in a future +# version of Cassandra. +# +# Ec2LocationProvider: +# Loads Region and Availability Zone information from the EC2 API. The Region is treated as the +# datacenter and the AvailablilityZone as the rack. Intended to replace use of Ec2Snitch and +# Ec2MultiRegionSnitch. +# +# GoogleCloudLocationProvider: +# Fetches source data from the metadata service of Google Cloud. This provider maps the GCE +# region to datacenter and the availibility zone to the rack. Intended to replace use of +# GoogleCloudSnitch. +initial_location_provider: SimpleLocationProvider + +# Node proximity controls sorting and ranking of replica lists in order to route requests efficiently. +# Dynamic snitch (not configured here) provides an additional level of sorting on top of whatever is +# specified here of sorting based on recorded latencies between peers. +# +# Out of the box Cassandra provides: +# +# NoOpProximity: +# Performs no reordering of replica lists and assigns every endpoint equal proximity, regardless of +# actual topology. This replicates the behaviour of SimpleSnitch. +# +# NetworkTopologyProximity: +# Ranks proximity of nodes based on datacenter and rack. Nodes in the same rack are considered +# closest, followed by those in the same datacenter. Nodes in different datacenters are +# considered furthest apart. Replicates the behaviour of topology aware snitches such as +# GossipingPropertyFileSnitch, PropertyFileSnitch and the multiple cloud platform snitches. +node_proximity: NetworkTopologyProximity + +# Address configuration (public and private endpoint addresses) is almost always derived directly +# from this configuration file using broadcast_address. In certain deployments, both the public and +# private addresses may be configured from an external source. For instance, in EC2 a node will have +# discover both of its own addresses at startup by querying the appropriate cloud metadata service. +# In multi-region EC2 deployments, this should be set to Ec2MultiRegionAddressConfig, to replicate the +# behaviour of Ec2MultiRegionSnitch. +#addresses_config: Ec2MultiRegionAddressConfig + +# Controls whether or not to ensure that connections to peers in the same datacenter are established +# using private addresses. Typically, this is for situations like EC2 where a node will have a public +# address and a private address. We may initially connect on the public, then discover the private, and +# reconnect on the private. +# In Ec2MultiRegionSnitch (now deprecated), this behaviour was hard coded so if migrating from that +# snitch to modern config, set this to true. +# In GossipingPropertyFileSnitch (now deprecated), this behaviour was configured by the `prefer_local` +# property so if migrating from that snitch to modern config, set this accordingly. +#prefer_local_connections: false # controls how often to perform the more expensive part of host score # calculation diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index aaf6167ebb27..4d4f6efafd8e 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -430,6 +430,11 @@ public static class SSTableConfig public DurationSpec.IntMillisecondsBound dynamic_snitch_reset_interval = new DurationSpec.IntMillisecondsBound("10m"); public double dynamic_snitch_badness_threshold = 1.0; + public String node_proximity; + public String initial_location_provider; + public String addresses_config; + public boolean prefer_local_connections = false; + public String failure_detector = "FailureDetector"; public EncryptionOptions.ServerEncryptionOptions server_encryption_options = new EncryptionOptions.ServerEncryptionOptions(); diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 169fc1380f17..fe44930ed0f4 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -32,7 +32,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Comparator; import java.util.Enumeration; import java.util.HashMap; import java.util.HashSet; @@ -90,6 +89,7 @@ import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.fql.FullQueryLoggerOptions; +import org.apache.cassandra.gms.IEndpointStateChangeSubscriber; import org.apache.cassandra.gms.IFailureDetector; import org.apache.cassandra.gms.VersionedValue; import org.apache.cassandra.io.FSWriteError; @@ -105,8 +105,14 @@ import org.apache.cassandra.locator.EndpointSnitchInfo; import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.locator.Replica; +import org.apache.cassandra.locator.Locator; +import org.apache.cassandra.locator.LocationInfo; +import org.apache.cassandra.locator.InitialLocationProvider; +import org.apache.cassandra.locator.NodeAddressConfig; +import org.apache.cassandra.locator.ReconnectableSnitchHelper; import org.apache.cassandra.locator.SeedProvider; +import org.apache.cassandra.locator.NodeProximity; +import org.apache.cassandra.locator.SnitchAdapter; import org.apache.cassandra.security.AbstractCryptoProvider; import org.apache.cassandra.security.EncryptionContext; import org.apache.cassandra.security.JREProvider; @@ -114,6 +120,7 @@ import org.apache.cassandra.service.CacheService.CacheType; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.service.paxos.Paxos; +import org.apache.cassandra.tcm.RegistrationStatus; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.MBeanWrapper; import org.apache.cassandra.utils.Pair; @@ -185,7 +192,10 @@ public class DatabaseDescriptor static final DurationSpec.LongMillisecondsBound LOWEST_ACCEPTED_TIMEOUT = new DurationSpec.LongMillisecondsBound(10L); private static Supplier<IFailureDetector> newFailureDetector; - private static IEndpointSnitch snitch; + private static NodeProximity nodeProximity; + private static Locator initializationLocator; + private static InitialLocationProvider initialLocationProvider; + private static IEndpointStateChangeSubscriber localAddressReconnector; private static InetAddress listenAddress; // leave null so we can fall through to getLocalHost private static InetAddress broadcastAddress; private static InetAddress rpcAddress; @@ -218,8 +228,6 @@ public class DatabaseDescriptor private static long counterCacheSizeInMiB; private static long indexSummaryCapacityInMiB; - private static String localDC; - private static Comparator<Replica> localComparator; private static EncryptionContext encryptionContext; private static boolean hasLoggedConfig; @@ -292,6 +300,7 @@ private static void clear() sstableFormats = null; clearMBean("org.apache.cassandra.db:type=DynamicEndpointSnitch"); clearMBean("org.apache.cassandra.db:type=EndpointSnitchInfo"); + clearMBean("org.apache.cassandra.db:type=LocationInfo"); } private static void clearMBean(String name) @@ -352,6 +361,8 @@ public static void toolInitialization(boolean failIfDaemonOrClient) applySnitch(); + applyFailureDetector(); + applyEncryptionContext(); } @@ -518,6 +529,8 @@ private static void applyAll() throws ConfigurationException applySnitch(); + applyFailureDetector(); + applyTokensConfig(); applySeedProvider(); @@ -1484,24 +1497,50 @@ else if (conf.num_tokens == null) // definitely not safe for tools + clients - implicitly instantiates StorageService public static void applySnitch() { - /* end point snitch */ - if (conf.endpoint_snitch == null) + boolean hasLegacyConfig = conf.endpoint_snitch != null; + boolean hasModernConfig = conf.initial_location_provider != null && conf.node_proximity != null; + + if (hasLegacyConfig == hasModernConfig) + throw new ConfigurationException("Configuration must specify either node_proximity and " + + "initial_location_provider or endpoint_snitch but not both. "); + + NodeProximity proximity; + NodeAddressConfig addressConfig; + if (hasLegacyConfig) { - throw new ConfigurationException("Missing endpoint_snitch directive", false); + logger.info("Use of endpoint_snitch in configuration is deprecated and should be replaced by " + + "initial_location_provider and node_proximity"); + SnitchAdapter adapter = new SnitchAdapter(createEndpointSnitch(conf.endpoint_snitch)); + proximity = adapter; + initialLocationProvider = adapter; + addressConfig = adapter; } - snitch = createEndpointSnitch(conf.dynamic_snitch, conf.endpoint_snitch); + else + { + proximity = createProximityImpl(conf.node_proximity); + initialLocationProvider = createInitialLocationProvider(conf.initial_location_provider); + addressConfig = conf.addresses_config != null + ? createAddressConfig(conf.addresses_config) + : NodeAddressConfig.DEFAULT; + } + // this is done here and not in applyAddressConfig as historically, Ec2MultiRegionSnitch is + // responsible for querying the cloud metadata service to get the public IP used for + // broadcast_address and we only want to instantiate the snitch here. + addressConfig.configureAddresses(); + initializationLocator = new Locator(RegistrationStatus.instance, + FBUtilities.getBroadcastAddressAndPort(), + initialLocationProvider); + nodeProximity = conf.dynamic_snitch ? new DynamicEndpointSnitch(proximity) : proximity; + localAddressReconnector = addressConfig.preferLocalConnections() + ? new ReconnectableSnitchHelper(initializationLocator, true) + : new IEndpointStateChangeSubscriber() { /* NO-OP */ }; + EndpointSnitchInfo.create(); + LocationInfo.create(); + } - localDC = snitch.getLocalDatacenter(); - localComparator = (replica1, replica2) -> { - boolean local1 = localDC.equals(snitch.getDatacenter(replica1)); - boolean local2 = localDC.equals(snitch.getDatacenter(replica2)); - if (local1 && !local2) - return -1; - if (local2 && !local1) - return 1; - return 0; - }; + public static void applyFailureDetector() + { newFailureDetector = () -> createFailureDetector(conf.failure_detector); } @@ -1717,12 +1756,36 @@ private static long tryGetSpace(String dir, PathUtils.IOToLongFunction<FileStore }); } - public static IEndpointSnitch createEndpointSnitch(boolean dynamic, String snitchClassName) throws ConfigurationException + public static IEndpointSnitch createEndpointSnitch(String snitchClassName) throws ConfigurationException { if (!snitchClassName.contains(".")) snitchClassName = "org.apache.cassandra.locator." + snitchClassName; IEndpointSnitch snitch = FBUtilities.construct(snitchClassName, "snitch"); - return dynamic ? new DynamicEndpointSnitch(snitch) : snitch; + return snitch; + } + + public static NodeProximity createProximityImpl(String className) throws ConfigurationException + { + if (!className.contains(".")) + className = "org.apache.cassandra.locator." + className; + NodeProximity sorter = FBUtilities.construct(className, "node proximity measurement"); + return sorter; + } + + public static InitialLocationProvider createInitialLocationProvider(String className) throws ConfigurationException + { + if (!className.contains(".")) + className = "org.apache.cassandra.locator." + className; + InitialLocationProvider provider = FBUtilities.construct(className, "initial location provider"); + return provider; + } + + public static NodeAddressConfig createAddressConfig(String className) throws ConfigurationException + { + if (!className.contains(".")) + className = "org.apache.cassandra.locator." + className; + NodeAddressConfig config = FBUtilities.construct(className, "node address config"); + return config; } private static IFailureDetector createFailureDetector(String detectorClassName) throws ConfigurationException @@ -2083,14 +2146,49 @@ public static IPartitioner setOnlyPartitionerUnsafe(IPartitioner newPartitioner) return old; } - public static IEndpointSnitch getEndpointSnitch() + public static IEndpointStateChangeSubscriber getLocalAddressReconnectionHelper() { - return snitch; + return localAddressReconnector; } - public static void setEndpointSnitch(IEndpointSnitch eps) + public static boolean preferLocalConnections() { - snitch = eps; + return conf.prefer_local_connections; + } + + /** + * Return a Locator that can be used from the moment a node begins its initial startup. + * It is not initialized with a ClusterMetadata instance (as there may not be one yet), so + * instead it always performs lookups using the current metadata. This means it is possible + * for the value of a lookup for the same endpoint to change between calls e.g. if a peer + * were to register or change broadcast address between the two calls being made. + * @return Locator + */ + public static Locator getLocator() + { + if (initializationLocator == null && isClientInitialized()) + return Locator.forClients(); + return initializationLocator; + } + + /** + * Used to provide the location (dc/rack) of the local node during its initial startup + * for the purpose of registering it with ClusterMetadata. + * See: org.apache.cassandra.locator.Locator + */ + public static InitialLocationProvider getInitialLocationProvider() + { + return initialLocationProvider; + } + + public static NodeProximity getNodeProximity() + { + return nodeProximity; + } + + public static void setNodeProximity(NodeProximity proximity) + { + nodeProximity = proximity; } public static IFailureDetector newFailureDetector() @@ -3708,7 +3806,7 @@ public static void setBatchlogReplayThrottleInKiB(int throttleInKiB) public static boolean isDynamicEndpointSnitch() { // not using config.dynamic_snitch because snitch can be changed via JMX - return snitch instanceof DynamicEndpointSnitch; + return nodeProximity instanceof DynamicEndpointSnitch; } public static Config.BatchlogEndpointStrategy getBatchlogEndpointStrategy() @@ -4038,18 +4136,7 @@ public static void setSkipStreamDiskSpaceCheck(boolean value) public static String getLocalDataCenter() { - return localDC; - } - - @VisibleForTesting - public static void setLocalDataCenter(String value) - { - localDC = value; - } - - public static Comparator<Replica> getLocalComparator() - { - return localComparator; + return initializationLocator == null ? null : initializationLocator.local().datacenter; } public static Config.InternodeCompression internodeCompression() diff --git a/src/java/org/apache/cassandra/config/EncryptionOptions.java b/src/java/org/apache/cassandra/config/EncryptionOptions.java index 8d922aa96e2f..1cd3b7adddaa 100644 --- a/src/java/org/apache/cassandra/config/EncryptionOptions.java +++ b/src/java/org/apache/cassandra/config/EncryptionOptions.java @@ -32,10 +32,11 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.Locator; import org.apache.cassandra.security.DisableSslContextFactory; import org.apache.cassandra.security.ISslContextFactory; +import org.apache.cassandra.tcm.membership.Location; import org.apache.cassandra.utils.FBUtilities; import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; @@ -786,7 +787,15 @@ private ServerEncryptionOptions applyConfigInternal() public boolean shouldEncrypt(InetAddressAndPort endpoint) { - IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch(); + // When a node is started for the very first time, it has no way to determine whether the seed nodes + // it makes its initial connections to are in a local or remote datacenter and/or rack. When the node is + // in this specific state, Locator will return the constant Location.UNKNOWN for any lookup of a peer's + // location. This is intended to ensure that _all_ peers are treated as remote during this initial phase and + // that the most strict encryption settings allowable by the internode_encryption settings are applied. + // Any connections established during this phase, of which there should be few, will be dropped and + // re-established as soon as the node initialises its local ClusterMetadata and so is able to get accurate + // topology information for peers. + Locator locator = DatabaseDescriptor.getLocator(); switch (internode_encryption) { case none: @@ -794,13 +803,14 @@ public boolean shouldEncrypt(InetAddressAndPort endpoint) case all: break; case dc: - if (snitch.getDatacenter(endpoint).equals(snitch.getLocalDatacenter())) + if (locator.location(endpoint).datacenter.equals(locator.local().datacenter)) return false; break; case rack: // for rack then check if the DC's are the same. - if (snitch.getRack(endpoint).equals(snitch.getLocalRack()) - && snitch.getDatacenter(endpoint).equals(snitch.getLocalDatacenter())) + Location remote = locator.location(endpoint); + Location local = locator.local(); + if (remote.rack.equals(local.rack) && remote.datacenter.equals(local.datacenter)) return false; break; } diff --git a/src/java/org/apache/cassandra/cql3/statements/DescribeStatement.java b/src/java/org/apache/cassandra/cql3/statements/DescribeStatement.java index f32ac48c9c02..cd9ffb11fde9 100644 --- a/src/java/org/apache/cassandra/cql3/statements/DescribeStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/DescribeStatement.java @@ -41,6 +41,8 @@ import org.apache.cassandra.exceptions.RequestValidationException; import org.apache.cassandra.io.util.DataInputBuffer; import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.locator.NodeProximity; +import org.apache.cassandra.locator.SnitchAdapter; import org.apache.cassandra.schema.*; import org.apache.cassandra.service.ClientState; import org.apache.cassandra.service.QueryState; @@ -674,8 +676,11 @@ protected Stream<List<Object>> describe(ClientState state, Keyspaces keyspaces) List<Object> list = new ArrayList<Object>(); list.add(DatabaseDescriptor.getClusterName()); list.add(trimIfPresent(DatabaseDescriptor.getPartitionerName(), "org.apache.cassandra.dht.")); - list.add(trimIfPresent(DatabaseDescriptor.getEndpointSnitch().getClass().getName(), - "org.apache.cassandra.locator.")); + NodeProximity proximity = DatabaseDescriptor.getNodeProximity(); + String nodeProximityClassName = proximity instanceof SnitchAdapter ? ((SnitchAdapter) proximity).snitch.getClass().getName() + : proximity.getClass().getName(); + list.add(trimIfPresent(nodeProximityClassName, + "org.apache.cassandra.locator.")); String useKs = state.getRawKeyspace(); if (mustReturnsRangeOwnerships(useKs)) diff --git a/src/java/org/apache/cassandra/db/ConsistencyLevel.java b/src/java/org/apache/cassandra/db/ConsistencyLevel.java index 73cdeb5c5c3d..aa83742ac847 100644 --- a/src/java/org/apache/cassandra/db/ConsistencyLevel.java +++ b/src/java/org/apache/cassandra/db/ConsistencyLevel.java @@ -18,13 +18,14 @@ package org.apache.cassandra.db; import com.carrotsearch.hppc.ObjectIntHashMap; -import org.apache.cassandra.locator.Endpoints; -import org.apache.cassandra.locator.InOurDc; -import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.locator.AbstractReplicationStrategy; +import org.apache.cassandra.locator.Endpoints; +import org.apache.cassandra.locator.InOurDc; +import org.apache.cassandra.locator.Locator; import org.apache.cassandra.locator.NetworkTopologyStrategy; +import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.transport.ProtocolException; import static org.apache.cassandra.locator.Replicas.addToCountPerDc; @@ -121,10 +122,10 @@ public static ObjectIntHashMap<String> eachQuorumForRead(AbstractReplicationStra } } - public static ObjectIntHashMap<String> eachQuorumForWrite(AbstractReplicationStrategy replicationStrategy, Endpoints<?> pendingWithDown) + public static ObjectIntHashMap<String> eachQuorumForWrite(Locator locator, AbstractReplicationStrategy replicationStrategy, Endpoints<?> pendingWithDown) { ObjectIntHashMap<String> perDc = eachQuorumForRead(replicationStrategy); - addToCountPerDc(perDc, pendingWithDown, 1); + addToCountPerDc(locator, perDc, pendingWithDown, 1); return perDc; } diff --git a/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java b/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java index c8621746254f..d0897afdd5af 100644 --- a/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java +++ b/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java @@ -38,7 +38,7 @@ protected void applyMutation(final Message<CounterMutation> message, InetAddress final CounterMutation cm = message.payload; logger.trace("Applying forwarded {}", cm); - String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getLocalDatacenter(); + String localDataCenter = DatabaseDescriptor.getLocator().local().datacenter; // We should not wait for the result of the write in this thread, // otherwise we could have a distributed deadlock between replicas // running this VerbHandler (see #4578). diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java index 2130d568d2a7..64436a940339 100644 --- a/src/java/org/apache/cassandra/db/SystemKeyspace.java +++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java @@ -82,7 +82,6 @@ import org.apache.cassandra.io.util.DataOutputBuffer; import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.RebufferingInputStream; -import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.MetaStrategy; import org.apache.cassandra.metrics.RestorableMeter; @@ -115,6 +114,7 @@ import org.apache.cassandra.streaming.StreamOperation; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.Epoch; +import org.apache.cassandra.tcm.membership.Location; import org.apache.cassandra.tcm.membership.NodeState; import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.utils.ByteBufferUtil; @@ -653,15 +653,23 @@ public static void persistLocalMetadata() "listen_address," + "listen_port" + ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)"; - IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch(); + + // If this node has not yet been registered in cluster metadata, record the initialization location provided + // by the configured Locator, this will also be used when registering an new node in cluster metadata during its + // intial startup. + // If the node is present in cluster metadata (i.e. has been registered already and we have replayed the + // metadata log), then get the location from there (via the Locator). + // We do this in case either the Locator config or location in metadata has been modified in any way, as cluster + // metadata is the ultimate source of truth. + Location location = DatabaseDescriptor.getLocator().local(); executeOnceInternal(format(req, LOCAL), LOCAL, DatabaseDescriptor.getClusterName(), FBUtilities.getReleaseVersionString(), QueryProcessor.CQL_VERSION.toString(), String.valueOf(ProtocolVersion.CURRENT.asInt()), - snitch.getLocalDatacenter(), - snitch.getLocalRack(), + location.datacenter, + location.rack, DatabaseDescriptor.getPartitioner().getClass().getName(), FBUtilities.getJustBroadcastNativeAddress(), DatabaseDescriptor.getNativeTransportPort(), @@ -950,6 +958,12 @@ public static synchronized void updateSchemaVersion(UUID version) executeInternal(format(req, LOCAL, LOCAL), version); } + public static synchronized void updateRack(String rack) + { + String req = "INSERT INTO system.%s (key, rack) VALUES ('%s', ?)"; + executeInternal(format(req, LOCAL, LOCAL), rack); + } + public static Set<String> tokensAsSet(Collection<Token> tokens) { if (tokens.isEmpty()) diff --git a/src/java/org/apache/cassandra/db/view/ViewUtils.java b/src/java/org/apache/cassandra/db/view/ViewUtils.java index b41150b7a5c4..510939fe6e3e 100644 --- a/src/java/org/apache/cassandra/db/view/ViewUtils.java +++ b/src/java/org/apache/cassandra/db/view/ViewUtils.java @@ -22,13 +22,13 @@ import java.util.function.Predicate; import com.google.common.collect.Iterables; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.dht.Token; import org.apache.cassandra.locator.EndpointsForToken; import org.apache.cassandra.locator.NetworkTopologyStrategy; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.membership.Location; public final class ViewUtils { @@ -60,7 +60,7 @@ private ViewUtils() */ public static Optional<Replica> getViewNaturalEndpoint(ClusterMetadata metadata, String keyspace, Token baseToken, Token viewToken) { - String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getLocalDatacenter(); + Location local = metadata.locator.local(); KeyspaceMetadata keyspaceMetadata = metadata.schema.getKeyspaces().getNullable(keyspace); EndpointsForToken naturalBaseReplicas = metadata.placements.get(keyspaceMetadata.params.replication).reads.forToken(baseToken).get(); @@ -73,7 +73,7 @@ public static Optional<Replica> getViewNaturalEndpoint(ClusterMetadata metadata, // We only select replicas from our own DC // TODO: this is poor encapsulation, leaking implementation details of replication strategy Predicate<Replica> isLocalDC = r -> !(keyspaceMetadata.replicationStrategy instanceof NetworkTopologyStrategy) - || DatabaseDescriptor.getEndpointSnitch().getDatacenter(r).equals(localDataCenter); + || metadata.locator.location(r.endpoint()).sameDatacenter(local); // We have to remove any endpoint which is shared between the base and the view, as it will select itself // and throw off the counts otherwise. diff --git a/src/java/org/apache/cassandra/db/virtual/InternodeInboundTable.java b/src/java/org/apache/cassandra/db/virtual/InternodeInboundTable.java index 0da5870d3dd4..ce6e023f5f10 100644 --- a/src/java/org/apache/cassandra/db/virtual/InternodeInboundTable.java +++ b/src/java/org/apache/cassandra/db/virtual/InternodeInboundTable.java @@ -32,6 +32,7 @@ import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.net.InboundMessageHandlers; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.tcm.membership.Location; public final class InternodeInboundTable extends AbstractVirtualTable { @@ -112,9 +113,8 @@ public DataSet data() private void addRow(SimpleDataSet dataSet, InetAddressAndPort addressAndPort, InboundMessageHandlers handlers) { - String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(addressAndPort); - String rack = DatabaseDescriptor.getEndpointSnitch().getRack(addressAndPort); - dataSet.row(addressAndPort.getAddress(), addressAndPort.getPort(), dc, rack) + Location location = DatabaseDescriptor.getLocator().location(addressAndPort); + dataSet.row(addressAndPort.getAddress(), addressAndPort.getPort(), location.datacenter, location.rack) .column(USING_BYTES, handlers.usingCapacity()) .column(USING_RESERVE_BYTES, handlers.usingEndpointReserveCapacity()) .column(CORRUPT_FRAMES_RECOVERED, handlers.corruptFramesRecovered()) diff --git a/src/java/org/apache/cassandra/db/virtual/InternodeOutboundTable.java b/src/java/org/apache/cassandra/db/virtual/InternodeOutboundTable.java index 687f845687ce..5f0351ff449a 100644 --- a/src/java/org/apache/cassandra/db/virtual/InternodeOutboundTable.java +++ b/src/java/org/apache/cassandra/db/virtual/InternodeOutboundTable.java @@ -34,6 +34,7 @@ import org.apache.cassandra.net.OutboundConnection; import org.apache.cassandra.net.OutboundConnections; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.tcm.membership.Location; public final class InternodeOutboundTable extends AbstractVirtualTable { @@ -112,10 +113,9 @@ public DataSet data() private void addRow(SimpleDataSet dataSet, InetAddressAndPort addressAndPort, OutboundConnections connections) { - String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(addressAndPort); - String rack = DatabaseDescriptor.getEndpointSnitch().getRack(addressAndPort); + Location location = DatabaseDescriptor.getLocator().location(addressAndPort); long pendingBytes = sum(connections, OutboundConnection::pendingBytes); - dataSet.row(addressAndPort.getAddress(), addressAndPort.getPort(), dc, rack) + dataSet.row(addressAndPort.getAddress(), addressAndPort.getPort(), location.datacenter, location.rack) .column(USING_BYTES, pendingBytes) .column(USING_RESERVE_BYTES, connections.usingReserveBytes()) .column(PENDING_COUNT, sum(connections, OutboundConnection::pendingCount)) diff --git a/src/java/org/apache/cassandra/db/virtual/PendingHintsTable.java b/src/java/org/apache/cassandra/db/virtual/PendingHintsTable.java index 9bab1994e1f5..900d903b087b 100644 --- a/src/java/org/apache/cassandra/db/virtual/PendingHintsTable.java +++ b/src/java/org/apache/cassandra/db/virtual/PendingHintsTable.java @@ -36,10 +36,11 @@ import org.apache.cassandra.gms.FailureDetectorMBean; import org.apache.cassandra.hints.HintsService; import org.apache.cassandra.hints.PendingHintsInfo; -import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.Locator; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.tcm.membership.Location; public final class PendingHintsTable extends AbstractVirtualTable { @@ -81,7 +82,7 @@ public PendingHintsTable(String keyspace) public DataSet data() { List<PendingHintsInfo> pendingHints = HintsService.instance.getPendingHintsInfo(); - IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch(); + Locator locator = DatabaseDescriptor.getLocator(); SimpleDataSet result = new SimpleDataSet(metadata()); @@ -91,6 +92,7 @@ public DataSet data() else simpleStates = Collections.emptyMap(); + Location location = Location.UNKNOWN; for (PendingHintsInfo info : pendingHints) { InetAddressAndPort addressAndPort = StorageService.instance.getEndpointForHostId(info.hostId); @@ -101,10 +103,11 @@ public DataSet data() String status = "Unknown"; if (addressAndPort != null) { + location = locator.location(addressAndPort); address = addressAndPort.getAddress(); port = addressAndPort.getPort(); - rack = snitch.getRack(addressAndPort); - dc = snitch.getDatacenter(addressAndPort); + rack = location.rack; + dc = location.datacenter; status = simpleStates.getOrDefault(addressAndPort.toString(), status); } result.row(info.hostId) diff --git a/src/java/org/apache/cassandra/dht/BootStrapper.java b/src/java/org/apache/cassandra/dht/BootStrapper.java index 6c306a3ff063..ebf04dddf89f 100644 --- a/src/java/org/apache/cassandra/dht/BootStrapper.java +++ b/src/java/org/apache/cassandra/dht/BootStrapper.java @@ -121,7 +121,7 @@ public Future<StreamState> bootstrap(StreamStateStore stateStore, boolean useStr RangeStreamer streamer = new RangeStreamer(metadata, StreamOperation.BOOTSTRAP, useStrictConsistency, - DatabaseDescriptor.getEndpointSnitch(), + DatabaseDescriptor.getNodeProximity(), stateStore, true, DatabaseDescriptor.getStreamingConnectionsPerHost(), diff --git a/src/java/org/apache/cassandra/dht/Datacenters.java b/src/java/org/apache/cassandra/dht/Datacenters.java index 2c0328e8d324..ab44c8a2ce43 100644 --- a/src/java/org/apache/cassandra/dht/Datacenters.java +++ b/src/java/org/apache/cassandra/dht/Datacenters.java @@ -28,7 +28,7 @@ public class Datacenters { private static class DCHandle { - private static final String thisDc = DatabaseDescriptor.getEndpointSnitch().getLocalDatacenter(); + private static final String thisDc = DatabaseDescriptor.getLocator().local().datacenter; } public static String thisDatacenter() diff --git a/src/java/org/apache/cassandra/dht/RangeFetchMapCalculator.java b/src/java/org/apache/cassandra/dht/RangeFetchMapCalculator.java index 949cf99f0973..185ec19b9030 100644 --- a/src/java/org/apache/cassandra/dht/RangeFetchMapCalculator.java +++ b/src/java/org/apache/cassandra/dht/RangeFetchMapCalculator.java @@ -33,12 +33,12 @@ import org.apache.cassandra.locator.EndpointsByRange; import org.apache.cassandra.locator.EndpointsForRange; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.Locator; import org.apache.cassandra.locator.Replica; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.locator.Replicas; import org.psjava.algo.graph.flownetwork.FordFulkersonAlgorithm; import org.psjava.algo.graph.flownetwork.MaximumFlowAlgorithm; @@ -82,14 +82,17 @@ public class RangeFetchMapCalculator private final Vertex sourceVertex = OuterVertex.getSourceVertex(); private final Vertex destinationVertex = OuterVertex.getDestinationVertex(); private final Set<Range<Token>> trivialRanges; + private final Locator locator; public RangeFetchMapCalculator(EndpointsByRange rangesWithSources, Collection<RangeStreamer.SourceFilter> sourceFilters, - String keyspace) + String keyspace, + Locator locator) { this.rangesWithSources = rangesWithSources; this.sourceFilters = Predicates.and(sourceFilters); this.keyspace = keyspace; + this.locator = locator; this.trivialRanges = rangesWithSources.keySet() .stream() .filter(RangeFetchMapCalculator::isTrivial) @@ -374,7 +377,7 @@ private boolean addEndpoints(MutableCapacityGraph<Vertex, Integer> capacityGraph private boolean isInLocalDC(Replica replica) { - return DatabaseDescriptor.getLocalDataCenter().equals(DatabaseDescriptor.getEndpointSnitch().getDatacenter(replica)); + return locator.local().sameDatacenter(locator.location(replica.endpoint())); } /** diff --git a/src/java/org/apache/cassandra/dht/RangeStreamer.java b/src/java/org/apache/cassandra/dht/RangeStreamer.java index 7cf919b91f6d..7d57b709565c 100644 --- a/src/java/org/apache/cassandra/dht/RangeStreamer.java +++ b/src/java/org/apache/cassandra/dht/RangeStreamer.java @@ -48,15 +48,16 @@ import org.apache.cassandra.locator.EndpointsByRange; import org.apache.cassandra.locator.EndpointsByReplica; import org.apache.cassandra.locator.EndpointsForRange; -import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.LocalStrategy; +import org.apache.cassandra.locator.Locator; import org.apache.cassandra.locator.NetworkTopologyStrategy; import org.apache.cassandra.locator.RangesAtEndpoint; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.locator.ReplicaCollection; import org.apache.cassandra.locator.ReplicaCollection.Builder.Conflict; import org.apache.cassandra.locator.Replicas; +import org.apache.cassandra.locator.NodeProximity; import org.apache.cassandra.schema.ReplicationParams; import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.streaming.StreamOperation; @@ -93,7 +94,7 @@ public class RangeStreamer private final List<SourceFilter> sourceFilters = new ArrayList<>(); private final StreamPlan streamPlan; private final boolean useStrictConsistency; - private final IEndpointSnitch snitch; + private final NodeProximity proximity; private final StreamStateStore stateStore; private final MovementMap movements; private final MovementMap strictMovements; @@ -178,18 +179,18 @@ public String message(Replica replica) public static class SingleDatacenterFilter implements SourceFilter { private final String sourceDc; - private final IEndpointSnitch snitch; + private final Locator locator; - public SingleDatacenterFilter(IEndpointSnitch snitch, String sourceDc) + public SingleDatacenterFilter(Locator locator, String sourceDc) { this.sourceDc = sourceDc; - this.snitch = snitch; + this.locator = locator; } @Override public boolean apply(Replica replica) { - return snitch.getDatacenter(replica).equals(sourceDc); + return locator.location(replica.endpoint()).datacenter.equals(sourceDc); } @Override @@ -204,19 +205,19 @@ public String message(Replica replica) */ public static class ExcludeLocalDatacenterFilter implements SourceFilter { - private final IEndpointSnitch snitch; + private final Locator locator; private final String localDc; - public ExcludeLocalDatacenterFilter(IEndpointSnitch snitch) + public ExcludeLocalDatacenterFilter(Locator locator) { - this.snitch = snitch; - this.localDc = snitch.getLocalDatacenter(); + this.locator = locator; + this.localDc = locator.local().datacenter; } @Override public boolean apply(Replica replica) { - return !snitch.getDatacenter(replica).equals(localDc); + return !locator.location(replica.endpoint()).datacenter.equals(localDc); } @Override @@ -292,21 +293,21 @@ public String message(Replica replica) public RangeStreamer(ClusterMetadata metadata, StreamOperation streamOperation, boolean useStrictConsistency, - IEndpointSnitch snitch, + NodeProximity proximity, StreamStateStore stateStore, boolean connectSequentially, int connectionsPerHost, MovementMap movements, MovementMap strictMovements) { - this(metadata, streamOperation, useStrictConsistency, snitch, stateStore, + this(metadata, streamOperation, useStrictConsistency, proximity, stateStore, FailureDetector.instance, connectSequentially, connectionsPerHost, movements, strictMovements); } RangeStreamer(ClusterMetadata metadata, StreamOperation streamOperation, boolean useStrictConsistency, - IEndpointSnitch snitch, + NodeProximity proximity, StreamStateStore stateStore, IFailureDetector failureDetector, boolean connectSequentially, @@ -319,7 +320,7 @@ public RangeStreamer(ClusterMetadata metadata, this.description = streamOperation.getDescription(); this.streamPlan = new StreamPlan(streamOperation, connectionsPerHost, connectSequentially, null, PreviewKind.NONE); this.useStrictConsistency = useStrictConsistency; - this.snitch = snitch; + this.proximity = proximity; this.stateStore = stateStore; this.movements = movements; this.strictMovements = strictMovements; @@ -368,7 +369,7 @@ public void addKeyspaceToFetch(String keyspaceName) } boolean useStrictSource = useStrictSourcesForRanges(keyspace.getMetadata().params.replication, strat); - EndpointsByReplica fetchMap = calculateRangesToFetchWithPreferredEndpoints(snitch::sortedByProximity, + EndpointsByReplica fetchMap = calculateRangesToFetchWithPreferredEndpoints(proximity::sortedByProximity, keyspace.getReplicationStrategy(), useStrictConsistency, metadata, @@ -389,7 +390,7 @@ public void addKeyspaceToFetch(String keyspaceName) } else { - workMap = getOptimizedWorkMap(fetchMap, sourceFilters, keyspaceName); + workMap = getOptimizedWorkMap(fetchMap, sourceFilters, keyspaceName, metadata.locator); } if (toFetch.put(keyspaceName, workMap) != null) @@ -458,7 +459,7 @@ private static boolean useStrictSourcesForRanges(ReplicationParams params, * consistency. **/ public static EndpointsByReplica - calculateRangesToFetchWithPreferredEndpoints(BiFunction<InetAddressAndPort, EndpointsForRange, EndpointsForRange> snitchGetSortedListByProximity, + calculateRangesToFetchWithPreferredEndpoints(BiFunction<InetAddressAndPort, EndpointsForRange, EndpointsForRange> sortByProximity, AbstractReplicationStrategy strat, boolean useStrictConsistency, ClusterMetadata metadata, @@ -473,7 +474,7 @@ private static boolean useStrictSourcesForRanges(ReplicationParams params, logger.debug("To fetch RN: {}", movements.get(params).keySet()); Predicate<Replica> testSourceFilters = and(sourceFilters); - Function<EndpointsForRange, EndpointsForRange> sorted = endpoints -> snitchGetSortedListByProximity.apply(localAddress, endpoints); + Function<EndpointsForRange, EndpointsForRange> sorted = endpoints -> sortByProximity.apply(localAddress, endpoints); //This list of replicas is just candidates. With strict consistency it's going to be a narrow list. EndpointsByReplica.Builder rangesToFetchWithPreferredEndpoints = new EndpointsByReplica.Builder(); @@ -599,7 +600,8 @@ public static Multimap<InetAddressAndPort, FetchReplica> convertPreferredEndpoin */ private static Multimap<InetAddressAndPort, FetchReplica> getOptimizedWorkMap(EndpointsByReplica rangesWithSources, Collection<SourceFilter> sourceFilters, - String keyspace) + String keyspace, + Locator locator) { //For now we just aren't going to use the optimized range fetch map with transient replication to shrink //the surface area to test and introduce bugs. @@ -613,7 +615,7 @@ private static Multimap<InetAddressAndPort, FetchReplica> getOptimizedWorkMap(En } EndpointsByRange unwrappedView = unwrapped.build(); - RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(unwrappedView, sourceFilters, keyspace); + RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(unwrappedView, sourceFilters, keyspace, locator); Multimap<InetAddressAndPort, Range<Token>> rangeFetchMapMap = calculator.getRangeFetchMap(); logger.info("Output from RangeFetchMapCalculator for keyspace {}", keyspace); validateRangeFetchMap(unwrappedView, rangeFetchMapMap, keyspace); diff --git a/src/java/org/apache/cassandra/dht/tokenallocator/OfflineTokenAllocator.java b/src/java/org/apache/cassandra/dht/tokenallocator/OfflineTokenAllocator.java index d46c043ed58e..0d9eec598a0a 100644 --- a/src/java/org/apache/cassandra/dht/tokenallocator/OfflineTokenAllocator.java +++ b/src/java/org/apache/cassandra/dht/tokenallocator/OfflineTokenAllocator.java @@ -23,7 +23,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.TreeSet; @@ -32,13 +31,17 @@ import com.google.common.collect.Maps; import org.apache.commons.math3.stat.descriptive.SummaryStatistics; +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Token; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.locator.SimpleSnitch; import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.RegistrationStatus; +import org.apache.cassandra.tcm.membership.Location; import org.apache.cassandra.utils.OutputHandler; +import static org.apache.cassandra.locator.SimpleLocationProvider.LOCATION; + public class OfflineTokenAllocator { public static List<FakeNode> allocate(int rf, int numTokens, int[] nodesPerRack, OutputHandler logger, IPartitioner partitioner) @@ -54,6 +57,9 @@ public static List<FakeNode> allocate(int rf, int numTokens, int[] nodesPerRack, Preconditions.checkArgument(nodes >= rf, "not enough nodes %s for rf %s in %s", Arrays.stream(nodesPerRack).sum(), rf, Arrays.toString(nodesPerRack)); + DatabaseDescriptor.setPartitionerUnsafe(partitioner); + // Set RegistrationStatus to REGISTERED so that Locator works exclusively from ClusterMetadata + RegistrationStatus.instance.onRegistration(); List<FakeNode> fakeNodes = new ArrayList<>(nodes); MultinodeAllocator allocator = new MultinodeAllocator(rf, numTokens, logger, partitioner); @@ -114,27 +120,21 @@ public Collection<Token> tokens() private static class MultinodeAllocator { - private final FakeSnitch fakeSnitch; private final TokenAllocation allocation; private final Map<Integer, SummaryStatistics> lastCheckPoint = Maps.newHashMap(); private final OutputHandler logger; private MultinodeAllocator(int rf, int numTokens, OutputHandler logger, IPartitioner partitioner) { - this.fakeSnitch = new FakeSnitch(); - this.allocation = TokenAllocation.create(fakeSnitch, new ClusterMetadata(partitioner), rf, numTokens); + this.allocation = TokenAllocation.create(LOCATION.datacenter, new ClusterMetadata(partitioner), rf, numTokens); this.logger = logger; } private FakeNode allocateTokensForNode(int nodeId, Integer rackId) { - // Update snitch and token metadata info InetAddressAndPort fakeNodeAddressAndPort = getLoopbackAddressWithPort(nodeId); - fakeSnitch.nodeByRack.put(fakeNodeAddressAndPort, rackId); - // todo: - //fakeMetadata.updateTopology(fakeNodeAddressAndPort); - // Allocate tokens + allocation.addNodeToMetadata(fakeNodeAddressAndPort, new Location(LOCATION.datacenter, rackId.toString())); Collection<Token> tokens = allocation.allocate(fakeNodeAddressAndPort); // Validate ownership stats @@ -145,7 +145,7 @@ private FakeNode allocateTokensForNode(int nodeId, Integer rackId) private void validateAllocation(int nodeId, int rackId) { - SummaryStatistics newOwnership = allocation.getAllocationRingOwnership(SimpleSnitch.DATA_CENTER_NAME, Integer.toString(rackId)); + SummaryStatistics newOwnership = allocation.getAllocationRingOwnership(LOCATION.datacenter, Integer.toString(rackId)); SummaryStatistics oldOwnership = lastCheckPoint.put(rackId, newOwnership); if (oldOwnership != null) logger.debug(String.format("Replicated node load in rack=%d before allocating node %d: %s.", rackId, nodeId, @@ -164,17 +164,6 @@ private void validateAllocation(int nodeId, int rackId) } } - private static class FakeSnitch extends SimpleSnitch - { - final Map<InetAddressAndPort, Integer> nodeByRack = new HashMap<>(); - - @Override - public String getRack(InetAddressAndPort endpoint) - { - return Integer.toString(nodeByRack.get(endpoint)); - } - } - private static InetAddressAndPort getLoopbackAddressWithPort(int port) { try diff --git a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java index bad7785ba096..a946cd20effc 100644 --- a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java +++ b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java @@ -24,7 +24,9 @@ import java.util.Map; import java.util.NavigableMap; import java.util.TreeMap; +import java.util.function.Supplier; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Multimap; @@ -34,23 +36,25 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.dht.Token; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.locator.AbstractReplicationStrategy; -import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.Locator; import org.apache.cassandra.locator.NetworkTopologyStrategy; import org.apache.cassandra.locator.SimpleStrategy; import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.membership.Location; +import org.apache.cassandra.tcm.membership.NodeAddresses; import org.apache.cassandra.tcm.membership.NodeId; +import org.apache.cassandra.tcm.membership.NodeVersion; public class TokenAllocation { public static final double WARN_STDEV_GROWTH = 0.05; private static final Logger logger = LoggerFactory.getLogger(TokenAllocation.class); - final ClusterMetadata metadata; + ClusterMetadata metadata; final AbstractReplicationStrategy replicationStrategy; final int numTokens; final Map<String, Map<String, StrategyAdapter>> strategyByRackDc = new HashMap<>(); @@ -75,14 +79,14 @@ public static Collection<Token> allocateTokens(final ClusterMetadata metadata, final InetAddressAndPort endpoint, int numTokens) { - return create(DatabaseDescriptor.getEndpointSnitch(), metadata, replicas, numTokens).allocate(endpoint); + return create(metadata.locator.local().datacenter, metadata, replicas, numTokens).allocate(endpoint); } - static TokenAllocation create(IEndpointSnitch snitch, ClusterMetadata metadata, int replicas, int numTokens) + static TokenAllocation create(String localDatacenter, ClusterMetadata metadata, int replicas, int numTokens) { // We create a fake NTS replication strategy with the specified RF in the local DC HashMap<String, String> options = new HashMap<>(); - options.put(snitch.getLocalDatacenter(), Integer.toString(replicas)); + options.put(localDatacenter, Integer.toString(replicas)); NetworkTopologyStrategy fakeReplicationStrategy = new NetworkTopologyStrategy(null, options); return new TokenAllocation(metadata, fakeReplicationStrategy, numTokens); @@ -93,6 +97,23 @@ static TokenAllocation create(ClusterMetadata metadata, AbstractReplicationStrat return new TokenAllocation(metadata, rs, numTokens); } + @VisibleForTesting + void updateTokensForNode(NodeId id, Collection<Token> tokens) + { + metadata = metadata.transformer() + .proposeToken(id, tokens) + .addToRackAndDC(id) // needed by NetworkTopologyStrategy + .build().metadata; + } + + // For use by OfflineTokenAllocator + void addNodeToMetadata(InetAddressAndPort endpoint, Location location) + { + metadata = metadata.transformer() + .register(new NodeAddresses(endpoint), location, NodeVersion.CURRENT) + .build().metadata; + } + Collection<Token> allocate(InetAddressAndPort endpoint) { StrategyAdapter strategy = getOrCreateStrategy(endpoint); @@ -100,6 +121,8 @@ Collection<Token> allocate(InetAddressAndPort endpoint) tokens = strategy.adjustForCrossDatacenterClashes(tokens); SummaryStatistics os = strategy.replicatedOwnershipStats(); + NodeId nodeId = metadata.directory.peerId(endpoint); + updateTokensForNode(nodeId, tokens); SummaryStatistics ns = strategy.replicatedOwnershipStats(); logger.info("Selected tokens {}", tokens); @@ -136,7 +159,7 @@ abstract class StrategyAdapter implements ReplicationStrategy<InetAddressAndPort // return true iff the provided endpoint occurs in the same virtual token-ring we are allocating for // i.e. the set of the nodes that share ownership with the node we are allocating // alternatively: return false if the endpoint's ownership is independent of the node we are allocating tokens for - abstract boolean inAllocationRing(InetAddressAndPort other); + abstract boolean inAllocationRing(Locator locator, InetAddressAndPort other); final TokenAllocator<InetAddressAndPort> createAllocator() { @@ -145,7 +168,7 @@ final TokenAllocator<InetAddressAndPort> createAllocator() for (Map.Entry<Token, NodeId> en : metadata.tokenMap.asMap().entrySet()) { InetAddressAndPort endpoint = metadata.directory.endpoint(en.getValue()); - if (inAllocationRing(endpoint)) + if (inAllocationRing(metadata.locator, endpoint)) sortedTokens.put(en.getKey(), endpoint); } return TokenAllocatorFactory.createTokenAllocator(sortedTokens, this, metadata.tokenMap.partitioner()); @@ -161,7 +184,7 @@ final Collection<Token> adjustForCrossDatacenterClashes(Collection<Token> tokens { NodeId nodeId = metadata.tokenMap.owner(t); InetAddressAndPort other = metadata.directory.endpoint(nodeId); - if (inAllocationRing(other)) + if (inAllocationRing(metadata.locator, other)) throw new ConfigurationException(String.format("Allocated token %s already assigned to node %s. Is another node also allocating tokens?", t, other)); t = t.nextValidToken(); } @@ -176,7 +199,7 @@ final SummaryStatistics replicatedOwnershipStats() for (Map.Entry<InetAddressAndPort, Double> en : evaluateReplicatedOwnership().entrySet()) { // Filter only in the same allocation ring - if (inAllocationRing(en.getKey())) + if (inAllocationRing(metadata.locator, en.getKey())) { NodeId nodeId = metadata.directory.peerId(en.getKey()); stat.addValue(en.getValue() / metadata.tokenMap.tokens(nodeId).size()); @@ -220,10 +243,8 @@ private void addOwnership(Token current, Token next, Map<InetAddressAndPort, Dou private StrategyAdapter getOrCreateStrategy(InetAddressAndPort endpoint) { - IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch(); - String dc = snitch.getDatacenter(endpoint); - String rack = snitch.getRack(endpoint); - return getOrCreateStrategy(dc, rack); + Location location = metadata.locator.location(endpoint); + return getOrCreateStrategy(location.datacenter, location.rack); } private StrategyAdapter getOrCreateStrategy(String dc, String rack) @@ -236,13 +257,13 @@ private StrategyAdapter createStrategy(String dc, String rack) if (replicationStrategy instanceof NetworkTopologyStrategy) return createStrategy(metadata, (NetworkTopologyStrategy) replicationStrategy, dc, rack); if (replicationStrategy instanceof SimpleStrategy) - return createStrategy((SimpleStrategy) replicationStrategy); + return createStrategy(metadata, (SimpleStrategy) replicationStrategy); throw new ConfigurationException("Token allocation does not support replication strategy " + replicationStrategy.getClass().getSimpleName()); } - private StrategyAdapter createStrategy(final SimpleStrategy rs) + private StrategyAdapter createStrategy(ClusterMetadata metadata, final SimpleStrategy rs) { - return createStrategy(DatabaseDescriptor.getEndpointSnitch(), null, null, rs.getReplicationFactor().allReplicas, false); + return createStrategy(() -> metadata.locator, null, null, rs.getReplicationFactor().allReplicas, false); } private StrategyAdapter createStrategy(ClusterMetadata metadata, NetworkTopologyStrategy strategy, String dc, String rack) @@ -251,6 +272,7 @@ private StrategyAdapter createStrategy(ClusterMetadata metadata, NetworkTopology // if topology hasn't been setup yet for this dc+rack then treat it as a separate unit Multimap<String, InetAddressAndPort> datacenterRacks = metadata.directory.datacenterRacks(dc); + Supplier<Locator> locator = () -> metadata.locator; int racks = datacenterRacks != null && datacenterRacks.containsKey(rack) ? datacenterRacks.asMap().size() : 1; @@ -258,21 +280,21 @@ private StrategyAdapter createStrategy(ClusterMetadata metadata, NetworkTopology if (replicas <= 1) { // each node is treated as separate and replicates once - return createStrategy(DatabaseDescriptor.getEndpointSnitch(), dc, null, 1, false); + return createStrategy(locator, dc, null, 1, false); } else if (racks == replicas) { // each node is treated as separate and replicates once, with separate allocation rings for each rack - return createStrategy(DatabaseDescriptor.getEndpointSnitch(), dc, rack, 1, false); + return createStrategy(locator, dc, rack, 1, false); } else if (racks > replicas) { // group by rack - return createStrategy(DatabaseDescriptor.getEndpointSnitch(), dc, null, replicas, true); + return createStrategy(locator, dc, null, replicas, true); } else if (racks == 1) { - return createStrategy(DatabaseDescriptor.getEndpointSnitch(), dc, null, replicas, false); + return createStrategy(locator, dc, null, replicas, false); } throw new ConfigurationException(String.format("Token allocation failed: the number of racks %d in datacenter %s is lower than its replication factor %d.", @@ -281,7 +303,7 @@ else if (racks == 1) // a null dc will always return true for inAllocationRing(..) // a null rack will return true for inAllocationRing(..) for all nodes in the same dc - private StrategyAdapter createStrategy(IEndpointSnitch snitch, String dc, String rack, int replicas, boolean groupByRack) + private StrategyAdapter createStrategy(Supplier<Locator> locator, String dc, String rack, int replicas, boolean groupByRack) { return new StrategyAdapter() { @@ -294,13 +316,14 @@ public int replicas() @Override public Object getGroup(InetAddressAndPort unit) { - return groupByRack ? snitch.getRack(unit) : unit; + return groupByRack ? locator.get().location(unit).rack : unit; } @Override - public boolean inAllocationRing(InetAddressAndPort other) + public boolean inAllocationRing(Locator locator, InetAddressAndPort other) { - return (dc == null || dc.equals(snitch.getDatacenter(other))) && (rack == null || rack.equals(snitch.getRack(other))); + Location location = locator.location(other); + return (dc == null || dc.equals(location.datacenter)) && (rack == null || rack.equals(location.rack)); } }; } diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java index de4d9b811b99..5274a57ffb53 100644 --- a/src/java/org/apache/cassandra/gms/Gossiper.java +++ b/src/java/org/apache/cassandra/gms/Gossiper.java @@ -1648,13 +1648,12 @@ public void start(int generationNbr, boolean mergeLocalStates) buildSeedsList(); /* initialize the heartbeat state for this localEndpoint */ maybeInitializeLocalState(generationNbr); + register(DatabaseDescriptor.getLocalAddressReconnectionHelper()); + ClusterMetadata metadata = ClusterMetadata.current(); if (mergeLocalStates && metadata.myNodeId() != null) mergeNodeToGossip(metadata.myNodeId(), metadata); - //notify snitches that Gossiper is about to start - DatabaseDescriptor.getEndpointSnitch().gossiperStarting(); - shutdownAnnounced.set(false); scheduledGossipTask = executor.scheduleWithFixedDelay(new GossipTask(), Gossiper.intervalInMillis, diff --git a/src/java/org/apache/cassandra/locator/AbstractCloudMetadataServiceConnector.java b/src/java/org/apache/cassandra/locator/AbstractCloudMetadataServiceConnector.java index 2cea1c7145da..75f0cfd247b7 100644 --- a/src/java/org/apache/cassandra/locator/AbstractCloudMetadataServiceConnector.java +++ b/src/java/org/apache/cassandra/locator/AbstractCloudMetadataServiceConnector.java @@ -36,11 +36,11 @@ import static java.lang.String.format; import static java.nio.charset.StandardCharsets.UTF_8; -abstract class AbstractCloudMetadataServiceConnector +public abstract class AbstractCloudMetadataServiceConnector { - static final String METADATA_URL_PROPERTY = "metadata_url"; - static final String METADATA_REQUEST_TIMEOUT_PROPERTY = "metadata_request_timeout"; - static final String DEFAULT_METADATA_REQUEST_TIMEOUT = "30s"; + public static final String METADATA_URL_PROPERTY = "metadata_url"; + public static final String METADATA_REQUEST_TIMEOUT_PROPERTY = "metadata_request_timeout"; + public static final String DEFAULT_METADATA_REQUEST_TIMEOUT = "30s"; protected final String metadataServiceUrl; protected final int requestTimeoutMs; diff --git a/src/java/org/apache/cassandra/locator/AbstractCloudMetadataServiceSnitch.java b/src/java/org/apache/cassandra/locator/AbstractCloudMetadataServiceSnitch.java index c8563f9b8744..e1d4ae66c0f8 100644 --- a/src/java/org/apache/cassandra/locator/AbstractCloudMetadataServiceSnitch.java +++ b/src/java/org/apache/cassandra/locator/AbstractCloudMetadataServiceSnitch.java @@ -23,70 +23,32 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.tcm.ClusterMetadata; -import org.apache.cassandra.tcm.membership.NodeId; -import org.apache.cassandra.utils.FBUtilities; -import org.apache.cassandra.utils.Pair; - -import static java.lang.String.format; - +/** + * @deprecated See CASSANDRA-19488 + */ +@Deprecated(since = "CEP-21") abstract class AbstractCloudMetadataServiceSnitch extends AbstractNetworkTopologySnitch { static final Logger logger = LoggerFactory.getLogger(AbstractCloudMetadataServiceSnitch.class); - static final String DEFAULT_DC = "UNKNOWN-DC"; - static final String DEFAULT_RACK = "UNKNOWN-RACK"; - - protected final AbstractCloudMetadataServiceConnector connector; - - private final String localRack; - private final String localDc; + protected final CloudMetadataLocationProvider locationProvider; private Map<InetAddressAndPort, Map<String, String>> savedEndpoints; - public AbstractCloudMetadataServiceSnitch(AbstractCloudMetadataServiceConnector connector, Pair<String, String> dcAndRack) - { - this.connector = connector; - this.localDc = dcAndRack.left; - this.localRack = dcAndRack.right; - - logger.info(format("%s using datacenter: %s, rack: %s, connector: %s, properties: %s", - getClass().getName(), getLocalDatacenter(), getLocalRack(), connector, connector.getProperties())); - } - - @Override - public final String getLocalRack() - { - return localRack; - } - - @Override - public final String getLocalDatacenter() + public AbstractCloudMetadataServiceSnitch(CloudMetadataLocationProvider locationProvider) { - return localDc; + this.locationProvider = locationProvider; } @Override - public final String getRack(InetAddressAndPort endpoint) + public String getLocalRack() { - if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort())) - return getLocalRack(); - ClusterMetadata metadata = ClusterMetadata.current(); - NodeId nodeId = metadata.directory.peerId(endpoint); - if (nodeId == null) - return DEFAULT_RACK; - return metadata.directory.location(nodeId).rack; + return locationProvider.initialLocation().rack; } @Override - public final String getDatacenter(InetAddressAndPort endpoint) + public String getLocalDatacenter() { - if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort())) - return getLocalDatacenter(); - ClusterMetadata metadata = ClusterMetadata.current(); - NodeId nodeId = metadata.directory.peerId(endpoint); - if (nodeId == null) - return DEFAULT_DC; - return metadata.directory.location(nodeId).datacenter; + return locationProvider.initialLocation().datacenter; } } diff --git a/src/java/org/apache/cassandra/locator/AbstractEndpointSnitch.java b/src/java/org/apache/cassandra/locator/AbstractEndpointSnitch.java index 2e7408be20ca..d2f4d88e1734 100644 --- a/src/java/org/apache/cassandra/locator/AbstractEndpointSnitch.java +++ b/src/java/org/apache/cassandra/locator/AbstractEndpointSnitch.java @@ -18,12 +18,30 @@ package org.apache.cassandra.locator; import com.google.common.collect.Iterables; + import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.tcm.membership.Location; +/** + * @deprecated + */ +@Deprecated(since = "CEP-21") public abstract class AbstractEndpointSnitch implements IEndpointSnitch { public abstract int compareEndpoints(InetAddressAndPort target, Replica r1, Replica r2); + @Override + public String getRack(InetAddressAndPort endpoint) + { + throw new UnsupportedOperationException("IEndpointSnitch has been deprecated and is no longer in use"); + } + + @Override + public String getDatacenter(InetAddressAndPort endpoint) + { + throw new UnsupportedOperationException("IEndpointSnitch has been deprecated and is no longer in use"); + } + /** * Sorts the <tt>Collection</tt> of node addresses by proximity to the given address * @param address the address to sort by proximity to @@ -53,7 +71,8 @@ public boolean isWorthMergingForRangeQuery(ReplicaCollection<?> merged, ReplicaC private boolean hasRemoteNode(ReplicaCollection<?> l) { - String localDc = DatabaseDescriptor.getLocalDataCenter(); - return Iterables.any(l, replica -> !localDc.equals(getDatacenter(replica))); + Locator locator = DatabaseDescriptor.getLocator(); + Location local = locator.local(); + return Iterables.any(l, replica -> !local.sameDatacenter(locator.location(replica.endpoint()))); } } diff --git a/src/java/org/apache/cassandra/locator/AbstractNetworkTopologySnitch.java b/src/java/org/apache/cassandra/locator/AbstractNetworkTopologySnitch.java index 08c41f097ff8..b6901e27660c 100644 --- a/src/java/org/apache/cassandra/locator/AbstractNetworkTopologySnitch.java +++ b/src/java/org/apache/cassandra/locator/AbstractNetworkTopologySnitch.java @@ -20,48 +20,15 @@ /** * An endpoint snitch tells Cassandra information about network topology that it can use to route * requests more efficiently. + * @deprecated See CASSANDRA-19488 */ +@Deprecated(since = "CEP-21") public abstract class AbstractNetworkTopologySnitch extends AbstractEndpointSnitch { - /** - * Return the rack for which an endpoint resides in - * @param endpoint a specified endpoint - * @return string of rack - */ - abstract public String getRack(InetAddressAndPort endpoint); - - /** - * Return the data center for which an endpoint resides in - * @param endpoint a specified endpoint - * @return string of data center - */ - abstract public String getDatacenter(InetAddressAndPort endpoint); - + private static final NodeProximity proximity = new NetworkTopologyProximity(); @Override public int compareEndpoints(InetAddressAndPort address, Replica r1, Replica r2) { - InetAddressAndPort a1 = r1.endpoint(); - InetAddressAndPort a2 = r2.endpoint(); - if (address.equals(a1) && !address.equals(a2)) - return -1; - if (address.equals(a2) && !address.equals(a1)) - return 1; - - String addressDatacenter = getDatacenter(address); - String a1Datacenter = getDatacenter(a1); - String a2Datacenter = getDatacenter(a2); - if (addressDatacenter.equals(a1Datacenter) && !addressDatacenter.equals(a2Datacenter)) - return -1; - if (addressDatacenter.equals(a2Datacenter) && !addressDatacenter.equals(a1Datacenter)) - return 1; - - String addressRack = getRack(address); - String a1Rack = getRack(a1); - String a2Rack = getRack(a2); - if (addressRack.equals(a1Rack) && !addressRack.equals(a2Rack)) - return -1; - if (addressRack.equals(a2Rack) && !addressRack.equals(a1Rack)) - return 1; - return 0; + return proximity.compareEndpoints(address, r1, r2); } } diff --git a/src/java/org/apache/cassandra/locator/AlibabaCloudLocationProvider.java b/src/java/org/apache/cassandra/locator/AlibabaCloudLocationProvider.java new file mode 100644 index 000000000000..a3232cc29606 --- /dev/null +++ b/src/java/org/apache/cassandra/locator/AlibabaCloudLocationProvider.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.locator; + +import java.io.IOException; + +import org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.DefaultCloudMetadataServiceConnector; + +import static org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.METADATA_URL_PROPERTY; + +public class AlibabaCloudLocationProvider extends CloudMetadataLocationProvider +{ + static final String DEFAULT_METADATA_SERVICE_URL = "http://100.100.100.200"; + static final String ZONE_NAME_QUERY_URL = "/latest/meta-data/zone-id"; + + /** + * Used via reflection by DatabaseDescriptor::createInitialLocationProvider + */ + public AlibabaCloudLocationProvider() throws IOException + { + this(new SnitchProperties()); + } + + public AlibabaCloudLocationProvider(SnitchProperties properties) throws IOException + { + this(new DefaultCloudMetadataServiceConnector(properties.putIfAbsent(METADATA_URL_PROPERTY, + DEFAULT_METADATA_SERVICE_URL))); + } + + public AlibabaCloudLocationProvider(AbstractCloudMetadataServiceConnector connector) throws IOException + { + super(connector, c -> SnitchUtils.parseLocation(c.apiCall(ZONE_NAME_QUERY_URL), c.getProperties().getDcSuffix())); + } +} diff --git a/src/java/org/apache/cassandra/locator/AlibabaCloudSnitch.java b/src/java/org/apache/cassandra/locator/AlibabaCloudSnitch.java index a7a9df4d1e28..61af63ffea2a 100644 --- a/src/java/org/apache/cassandra/locator/AlibabaCloudSnitch.java +++ b/src/java/org/apache/cassandra/locator/AlibabaCloudSnitch.java @@ -22,6 +22,7 @@ import org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.DefaultCloudMetadataServiceConnector; import static org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.METADATA_URL_PROPERTY; +import static org.apache.cassandra.locator.AlibabaCloudLocationProvider.DEFAULT_METADATA_SERVICE_URL; /** * A snitch that assumes an ECS region is a DC and an ECS availability_zone @@ -29,12 +30,11 @@ * format of the zone-id is like 'cn-hangzhou-a' where cn means china, hangzhou * means the hangzhou region, a means the az id. We use 'cn-hangzhou' as the dc, * and 'a' as the zone-id. + * @deprecated See CASSANDRA-19488 */ +@Deprecated(since = "CEP-21") public class AlibabaCloudSnitch extends AbstractCloudMetadataServiceSnitch { - static final String DEFAULT_METADATA_SERVICE_URL = "http://100.100.100.200"; - static final String ZONE_NAME_QUERY_URL = "/latest/meta-data/zone-id"; - public AlibabaCloudSnitch() throws IOException { this(new SnitchProperties()); @@ -48,7 +48,6 @@ public AlibabaCloudSnitch(SnitchProperties properties) throws IOException public AlibabaCloudSnitch(AbstractCloudMetadataServiceConnector connector) throws IOException { - super(connector, SnitchUtils.parseDcAndRack(connector.apiCall(ZONE_NAME_QUERY_URL), - connector.getProperties().getDcSuffix())); + super(new AlibabaCloudLocationProvider(connector)); } } diff --git a/src/java/org/apache/cassandra/locator/AzureCloudLocationProvider.java b/src/java/org/apache/cassandra/locator/AzureCloudLocationProvider.java new file mode 100644 index 000000000000..930a1f139b70 --- /dev/null +++ b/src/java/org/apache/cassandra/locator/AzureCloudLocationProvider.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.locator; + +import java.io.IOException; + +import com.google.common.collect.ImmutableMap; + +import com.fasterxml.jackson.databind.JsonNode; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.DefaultCloudMetadataServiceConnector; +import org.apache.cassandra.tcm.membership.Location; +import org.apache.cassandra.utils.JsonUtils; + +import static java.lang.String.format; +import static org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.METADATA_URL_PROPERTY; + +public class AzureCloudLocationProvider extends CloudMetadataLocationProvider +{ + static final String DEFAULT_METADATA_SERVICE_URL = "http://169.254.169.254"; + static final String METADATA_QUERY_TEMPLATE = "/metadata/instance/compute?api-version=%s&format=json"; + static final String METADATA_HEADER = "Metadata"; + static final String API_VERSION_PROPERTY_KEY = "azure_api_version"; + static final String DEFAULT_API_VERSION = "2021-12-13"; + + /** + * Used via reflection by DatabaseDescriptor::createInitialLocationProvider + */ + public AzureCloudLocationProvider() throws IOException + { + this(new SnitchProperties()); + } + + public AzureCloudLocationProvider(SnitchProperties properties) throws IOException + { + this(new DefaultCloudMetadataServiceConnector(properties.putIfAbsent(METADATA_URL_PROPERTY, + DEFAULT_METADATA_SERVICE_URL))); + } + + public AzureCloudLocationProvider(AbstractCloudMetadataServiceConnector connector) throws IOException + { + super(connector, AzureCloudLocationProvider::resolveLocation); + } + + static Location resolveLocation(AbstractCloudMetadataServiceConnector connector) throws IOException + { + String apiVersion = connector.getProperties().get(API_VERSION_PROPERTY_KEY, DEFAULT_API_VERSION); + String response = connector.apiCall(format(METADATA_QUERY_TEMPLATE, apiVersion), + ImmutableMap.of(METADATA_HEADER, "true")); + JsonNode jsonNode = JsonUtils.JSON_OBJECT_MAPPER.readTree(response); + + JsonNode location = jsonNode.get("location"); + JsonNode zone = jsonNode.get("zone"); + JsonNode platformFaultDomain = jsonNode.get("platformFaultDomain"); + + String datacenter; + String rack; + + if (location == null || location.isNull() || location.asText().isEmpty()) + throw new ConfigurationException("Unable to retrieve initial location from cloud metadata service. " + + "This is required for registration, please check configuration"); + else + datacenter = location.asText(); + + if (zone == null || zone.isNull() || zone.asText().isEmpty()) + { + if (platformFaultDomain == null || platformFaultDomain.isNull() || platformFaultDomain.asText().isEmpty()) + { + throw new ConfigurationException("Unable to retrieve initial zone or platform fault domain from cloud metadata service. " + + "This is required for registration, please check configuration"); + } + else + { + rack = platformFaultDomain.asText(); + } + } + else + { + rack = zone.asText(); + } + + return new Location(datacenter + connector.getProperties().getDcSuffix(), "rack-" + rack); + } +} diff --git a/src/java/org/apache/cassandra/locator/AzureSnitch.java b/src/java/org/apache/cassandra/locator/AzureSnitch.java index 70729e1f3b6f..ae05de625a7c 100644 --- a/src/java/org/apache/cassandra/locator/AzureSnitch.java +++ b/src/java/org/apache/cassandra/locator/AzureSnitch.java @@ -20,15 +20,10 @@ import java.io.IOException; -import com.google.common.collect.ImmutableMap; - -import com.fasterxml.jackson.databind.JsonNode; import org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.DefaultCloudMetadataServiceConnector; -import org.apache.cassandra.utils.JsonUtils; -import org.apache.cassandra.utils.Pair; -import static java.lang.String.format; import static org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.METADATA_URL_PROPERTY; +import static org.apache.cassandra.locator.AzureCloudLocationProvider.DEFAULT_METADATA_SERVICE_URL; /** * AzureSnitch will resolve datacenter and rack by calling {@code /metadata/instance/compute} endpoint returning @@ -38,15 +33,11 @@ * A datacenter is resolved from {@code location} field and a rack is resolved by looking into {@code zone} field first. * When zone is not set, or it is empty string, it will look into {@code platformFaultDomain} field. Such resolved * value is prepended by {@code rack-} string. + * @deprecated See CASSANDRA-19488 */ +@Deprecated(since = "CEP-21") public class AzureSnitch extends AbstractCloudMetadataServiceSnitch { - static final String DEFAULT_METADATA_SERVICE_URL = "http://169.254.169.254"; - static final String METADATA_QUERY_TEMPLATE = "/metadata/instance/compute?api-version=%s&format=json"; - static final String METADATA_HEADER = "Metadata"; - static final String API_VERSION_PROPERTY_KEY = "azure_api_version"; - static final String DEFAULT_API_VERSION = "2021-12-13"; - public AzureSnitch() throws IOException { this(new SnitchProperties()); @@ -60,43 +51,6 @@ public AzureSnitch(SnitchProperties properties) throws IOException public AzureSnitch(AbstractCloudMetadataServiceConnector connector) throws IOException { - super(connector, resolveDcAndRack(connector)); - } - - private static Pair<String, String> resolveDcAndRack(AbstractCloudMetadataServiceConnector connector) throws IOException - { - String apiVersion = connector.getProperties().get(API_VERSION_PROPERTY_KEY, DEFAULT_API_VERSION); - String response = connector.apiCall(format(METADATA_QUERY_TEMPLATE, apiVersion), ImmutableMap.of(METADATA_HEADER, "true")); - JsonNode jsonNode = JsonUtils.JSON_OBJECT_MAPPER.readTree(response); - - JsonNode location = jsonNode.get("location"); - JsonNode zone = jsonNode.get("zone"); - JsonNode platformFaultDomain = jsonNode.get("platformFaultDomain"); - - String datacenter; - String rack; - - if (location == null || location.isNull() || location.asText().isEmpty()) - datacenter = DEFAULT_DC; - else - datacenter = location.asText(); - - if (zone == null || zone.isNull() || zone.asText().isEmpty()) - { - if (platformFaultDomain == null || platformFaultDomain.isNull() || platformFaultDomain.asText().isEmpty()) - { - rack = DEFAULT_RACK; - } - else - { - rack = platformFaultDomain.asText(); - } - } - else - { - rack = zone.asText(); - } - - return Pair.create(datacenter + connector.getProperties().getDcSuffix(), "rack-" + rack); + super(new AzureCloudLocationProvider(connector)); } } diff --git a/src/java/org/apache/cassandra/locator/BaseProximity.java b/src/java/org/apache/cassandra/locator/BaseProximity.java new file mode 100644 index 000000000000..928213bfe9ee --- /dev/null +++ b/src/java/org/apache/cassandra/locator/BaseProximity.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.locator; + +import com.google.common.collect.Iterables; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.tcm.membership.Location; + +public abstract class BaseProximity implements NodeProximity +{ + public abstract int compareEndpoints(InetAddressAndPort target, Replica r1, Replica r2); + + /** + * Sorts the <tt>Collection</tt> of node addresses by proximity to the given address + * @param address the address to sort by proximity to + * @param unsortedAddress the nodes to sort + * @return a new sorted <tt>List</tt> + */ + public <C extends ReplicaCollection<? extends C>> C sortedByProximity(final InetAddressAndPort address, C unsortedAddress) + { + return unsortedAddress.sorted((r1, r2) -> compareEndpoints(address, r1, r2)); + } + + public boolean isWorthMergingForRangeQuery(ReplicaCollection<?> merged, ReplicaCollection<?> l1, ReplicaCollection<?> l2) + { + // Querying remote DC is likely to be an order of magnitude slower than + // querying locally, so 2 queries to local nodes is likely to still be + // faster than 1 query involving remote ones + boolean mergedHasRemote = hasRemoteNode(merged); + return mergedHasRemote + ? hasRemoteNode(l1) || hasRemoteNode(l2) + : true; + } + + private boolean hasRemoteNode(ReplicaCollection<?> l) + { + Locator locator = DatabaseDescriptor.getLocator(); + Location local = locator.local(); + return Iterables.any(l, replica -> !local.sameDatacenter(locator.location(replica.endpoint()))); + } +} diff --git a/src/java/org/apache/cassandra/locator/CloudMetadataLocationProvider.java b/src/java/org/apache/cassandra/locator/CloudMetadataLocationProvider.java new file mode 100644 index 000000000000..67cf17284892 --- /dev/null +++ b/src/java/org/apache/cassandra/locator/CloudMetadataLocationProvider.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.locator; + +import java.io.IOException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.tcm.membership.Location; + +import static java.lang.String.format; + +public class CloudMetadataLocationProvider implements InitialLocationProvider +{ + static final Logger logger = LoggerFactory.getLogger(CloudMetadataLocationProvider.class); + + protected final AbstractCloudMetadataServiceConnector connector; + private final LocationResolver locationResolver; + private volatile Location location; + + public CloudMetadataLocationProvider(AbstractCloudMetadataServiceConnector connector, LocationResolver locationResolver) + { + this.connector = connector; + this.locationResolver = locationResolver; + } + + @Override + public final Location initialLocation() + { + if (location == null) + { + try + { + location = locationResolver.resolve(connector); + logger.info(format("%s using datacenter: %s, rack: %s, connector: %s, properties: %s", + getClass().getName(), location.datacenter, location.rack, connector, connector.getProperties())); + } + catch (IOException e) + { + throw new ConfigurationException("Unable to resolve initial location using cloud metadata service connector", e); + } + } + return location; + } + + public interface LocationResolver + { + Location resolve(AbstractCloudMetadataServiceConnector connector) throws IOException; + } +} diff --git a/src/java/org/apache/cassandra/locator/CloudstackLocationProvider.java b/src/java/org/apache/cassandra/locator/CloudstackLocationProvider.java new file mode 100644 index 000000000000..840aa06ded30 --- /dev/null +++ b/src/java/org/apache/cassandra/locator/CloudstackLocationProvider.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.locator; + +import java.io.BufferedReader; +import java.io.IOException; +import java.net.URI; +import java.util.Properties; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.io.util.FileReader; +import org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.DefaultCloudMetadataServiceConnector; +import org.apache.cassandra.tcm.membership.Location; +import org.apache.cassandra.utils.JVMStabilityInspector; + +import static org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.METADATA_URL_PROPERTY; + +public class CloudstackLocationProvider extends CloudMetadataLocationProvider +{ + private static final Logger logger = LoggerFactory.getLogger(CloudstackLocationProvider.class); + static final String ZONE_NAME_QUERY_URI = "/latest/meta-data/availability-zone"; + + private static final String[] LEASE_FILES = + { + "file:///var/lib/dhcp/dhclient.eth0.leases", + "file:///var/lib/dhclient/dhclient.eth0.leases" + }; + + /** + * Used via reflection by DatabaseDescriptor::createInitialLocationProvider + */ + public CloudstackLocationProvider() throws IOException + { + this(new SnitchProperties(new Properties())); + } + + public CloudstackLocationProvider(SnitchProperties snitchProperties) throws IOException + { + this(new DefaultCloudMetadataServiceConnector(snitchProperties.putIfAbsent(METADATA_URL_PROPERTY, csMetadataEndpoint()))); + } + + public CloudstackLocationProvider(AbstractCloudMetadataServiceConnector connector) throws IOException + { + super(connector, CloudstackLocationProvider::resolveLocation); + logger.warn("{} is deprecated and not actively maintained. It will be removed in the next " + + "major version of Cassandra.", CloudstackSnitch.class.getName()); + } + + private static Location resolveLocation(AbstractCloudMetadataServiceConnector connector) throws IOException + { + String zone = connector.apiCall(ZONE_NAME_QUERY_URI); + String[] zoneParts = zone.split("-"); + + if (zoneParts.length != 3) + throw new ConfigurationException("CloudstackSnitch cannot handle invalid zone format: " + zone); + + return new Location(zoneParts[0] + '-' + zoneParts[1], zoneParts[2]); + } + + private static String csMetadataEndpoint() throws ConfigurationException + { + for (String lease_uri : LEASE_FILES) + { + try + { + File lease_file = new File(new URI(lease_uri)); + if (lease_file.exists()) + { + return csEndpointFromLease(lease_file); + } + } + catch (Exception e) + { + JVMStabilityInspector.inspectThrowable(e); + } + } + + throw new ConfigurationException("No valid DHCP lease file could be found."); + } + + private static String csEndpointFromLease(File lease) throws ConfigurationException + { + String line; + String endpoint = null; + Pattern identifierPattern = Pattern.compile("^[ \t]*option dhcp-server-identifier (.*);$"); + + try (BufferedReader reader = new BufferedReader(new FileReader(lease))) + { + + while ((line = reader.readLine()) != null) + { + Matcher matcher = identifierPattern.matcher(line); + + if (matcher.find()) + { + endpoint = matcher.group(1); + break; + } + } + } + catch (Exception e) + { + throw new ConfigurationException("CloudstackSnitch cannot access lease file."); + } + + if (endpoint == null) + { + throw new ConfigurationException("No metadata server could be found in lease file."); + } + + return "http://" + endpoint; + } +} diff --git a/src/java/org/apache/cassandra/locator/CloudstackSnitch.java b/src/java/org/apache/cassandra/locator/CloudstackSnitch.java index d5cebb69bf2f..bb0772ea5b04 100644 --- a/src/java/org/apache/cassandra/locator/CloudstackSnitch.java +++ b/src/java/org/apache/cassandra/locator/CloudstackSnitch.java @@ -17,21 +17,8 @@ */ package org.apache.cassandra.locator; -import java.io.BufferedReader; import java.io.IOException; -import java.net.URI; import java.util.Properties; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.io.util.File; -import org.apache.cassandra.io.util.FileReader; -import org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.DefaultCloudMetadataServiceConnector; -import org.apache.cassandra.utils.JVMStabilityInspector; -import org.apache.cassandra.utils.Pair; - -import static org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.METADATA_URL_PROPERTY; /** * A snitch that assumes a Cloudstack Zone follows the typical convention @@ -45,14 +32,6 @@ @Deprecated(since = "5.0") public class CloudstackSnitch extends AbstractCloudMetadataServiceSnitch { - static final String ZONE_NAME_QUERY_URI = "/latest/meta-data/availability-zone"; - - private static final String[] LEASE_FILES = - { - "file:///var/lib/dhcp/dhclient.eth0.leases", - "file:///var/lib/dhclient/dhclient.eth0.leases" - }; - public CloudstackSnitch() throws IOException { this(new SnitchProperties(new Properties())); @@ -60,78 +39,15 @@ public CloudstackSnitch() throws IOException public CloudstackSnitch(SnitchProperties snitchProperties) throws IOException { - this(new DefaultCloudMetadataServiceConnector(snitchProperties.putIfAbsent(METADATA_URL_PROPERTY, csMetadataEndpoint()))); + super( new CloudstackLocationProvider(snitchProperties)); + logger.warn("{} is deprecated and not actively maintained. It will be removed in the next " + + "major version of Cassandra.", CloudstackSnitch.class.getName()); } public CloudstackSnitch(AbstractCloudMetadataServiceConnector connector) throws IOException { - super(connector, resolveDcAndRack(connector)); + super(new CloudstackLocationProvider(connector)); logger.warn("{} is deprecated and not actively maintained. It will be removed in the next " + "major version of Cassandra.", CloudstackSnitch.class.getName()); } - - private static Pair<String, String> resolveDcAndRack(AbstractCloudMetadataServiceConnector connector) throws IOException - { - String zone = connector.apiCall(ZONE_NAME_QUERY_URI); - String[] zoneParts = zone.split("-"); - - if (zoneParts.length != 3) - throw new ConfigurationException("CloudstackSnitch cannot handle invalid zone format: " + zone); - - return Pair.create(zoneParts[0] + '-' + zoneParts[1], zoneParts[2]); - } - - private static String csMetadataEndpoint() throws ConfigurationException - { - for (String lease_uri : LEASE_FILES) - { - try - { - File lease_file = new File(new URI(lease_uri)); - if (lease_file.exists()) - { - return csEndpointFromLease(lease_file); - } - } - catch (Exception e) - { - JVMStabilityInspector.inspectThrowable(e); - } - } - - throw new ConfigurationException("No valid DHCP lease file could be found."); - } - - private static String csEndpointFromLease(File lease) throws ConfigurationException - { - String line; - String endpoint = null; - Pattern identifierPattern = Pattern.compile("^[ \t]*option dhcp-server-identifier (.*);$"); - - try (BufferedReader reader = new BufferedReader(new FileReader(lease))) - { - - while ((line = reader.readLine()) != null) - { - Matcher matcher = identifierPattern.matcher(line); - - if (matcher.find()) - { - endpoint = matcher.group(1); - break; - } - } - } - catch (Exception e) - { - throw new ConfigurationException("CloudstackSnitch cannot access lease file."); - } - - if (endpoint == null) - { - throw new ConfigurationException("No metadata server could be found in lease file."); - } - - return "http://" + endpoint; - } } diff --git a/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java b/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java index bb652b6cff15..1d3810613f6d 100644 --- a/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java +++ b/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java @@ -47,8 +47,9 @@ /** * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector + * // TODO rename to DynamicNodeProximity or similar - but take care with jmx interface */ -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements LatencySubscribers.Subscriber, DynamicEndpointSnitchMBean +public class DynamicEndpointSnitch implements NodeProximity, LatencySubscribers.Subscriber, DynamicEndpointSnitchMBean { private static final boolean USE_SEVERITY = !IGNORE_DYNAMIC_SNITCH_SEVERITY.getBoolean(); @@ -69,7 +70,7 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements Lat private volatile HashMap<InetAddressAndPort, Double> scores = new HashMap<>(); private final ConcurrentHashMap<InetAddressAndPort, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>(); - public final IEndpointSnitch subsnitch; + public final NodeProximity delegate; private volatile ScheduledFuture<?> updateSchedular; private volatile ScheduledFuture<?> resetSchedular; @@ -77,33 +78,21 @@ public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements Lat private final Runnable update; private final Runnable reset; - public DynamicEndpointSnitch(IEndpointSnitch snitch) + public DynamicEndpointSnitch(NodeProximity delegate) { - this(snitch, null); + this(delegate, null); } - public DynamicEndpointSnitch(IEndpointSnitch snitch, String instance) + public DynamicEndpointSnitch(NodeProximity delegate, String instance) { mbeanName = "org.apache.cassandra.db:type=DynamicEndpointSnitch"; if (instance != null) mbeanName += ",instance=" + instance; - subsnitch = snitch; - update = new Runnable() - { - public void run() - { - updateScores(); - } - }; - reset = new Runnable() - { - public void run() - { - // we do this so that a host considered bad has a chance to recover, otherwise would we never try - // to read from it, which would cause its score to never change - reset(); - } - }; + this.delegate = delegate; + update = this::updateScores; + // we do this so that a host considered bad has a chance to recover, otherwise would we never try + // to read from it, which would cause its score to never change + reset = this::reset; if (DatabaseDescriptor.isDaemonInitialized()) { @@ -155,22 +144,6 @@ public void close() MBeanWrapper.instance.unregisterMBean(mbeanName); } - @Override - public void gossiperStarting() - { - subsnitch.gossiperStarting(); - } - - public String getRack(InetAddressAndPort endpoint) - { - return subsnitch.getRack(endpoint); - } - - public String getDatacenter(InetAddressAndPort endpoint) - { - return subsnitch.getDatacenter(endpoint); - } - @Override public <C extends ReplicaCollection<? extends C>> C sortedByProximity(final InetAddressAndPort address, C unsortedAddresses) { @@ -196,7 +169,7 @@ private <C extends ReplicaCollection<? extends C>> C sortedByProximityWithBadnes return replicas; // TODO: avoid copy - replicas = subsnitch.sortedByProximity(address, replicas); + replicas = delegate.sortedByProximity(address, replicas); HashMap<InetAddressAndPort, Double> scores = this.scores; // Make sure the score don't change in the middle of the loop below // (which wouldn't really matter here but its cleaner that way). ArrayList<Double> subsnitchOrderedScores = new ArrayList<>(replicas.size()); @@ -250,7 +223,7 @@ private int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2, } if (scored1.equals(scored2)) - return subsnitch.compareEndpoints(target, a1, a2); + return delegate.compareEndpoints(target, a1, a2); if (scored1 < scored2) return -1; else @@ -353,7 +326,10 @@ public double getBadnessThreshold() public String getSubsnitchClassName() { - return subsnitch.getClass().getName(); + Class<?> clazz = delegate.getClass(); + if (delegate instanceof SnitchAdapter) + clazz = ((SnitchAdapter)delegate).snitch.getClass(); + return clazz.getName(); } public List<Double> dumpTimings(String hostname) throws UnknownHostException @@ -401,7 +377,7 @@ public double getSeverity() public boolean isWorthMergingForRangeQuery(ReplicaCollection<?> merged, ReplicaCollection<?> l1, ReplicaCollection<?> l2) { - if (!subsnitch.isWorthMergingForRangeQuery(merged, l1, l2)) + if (!delegate.isWorthMergingForRangeQuery(merged, l1, l2)) return false; // skip checking scores in the single-node case @@ -433,9 +409,4 @@ private double maxScore(ReplicaCollection<?> endpoints) } return maxScore; } - - public boolean validate(Set<String> datacenters, Set<String> racks) - { - return subsnitch.validate(datacenters, racks); - } } diff --git a/src/java/org/apache/cassandra/locator/Ec2LocationProvider.java b/src/java/org/apache/cassandra/locator/Ec2LocationProvider.java new file mode 100644 index 000000000000..b6a3996df0e1 --- /dev/null +++ b/src/java/org/apache/cassandra/locator/Ec2LocationProvider.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.locator; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Set; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.membership.Location; + +public class Ec2LocationProvider extends CloudMetadataLocationProvider +{ + static final String SNITCH_PROP_NAMING_SCHEME = "ec2_naming_scheme"; + static final String EC2_NAMING_LEGACY = "legacy"; + static final String EC2_NAMING_STANDARD = "standard"; + + @VisibleForTesting + public static final String ZONE_NAME_QUERY = "/latest/meta-data/placement/availability-zone"; + + private final boolean usingLegacyNaming; + + /** + * Used via reflection by DatabaseDescriptor::createInitialLocationProvider + */ + public Ec2LocationProvider() throws IOException, ConfigurationException + { + this(new SnitchProperties()); + } + + public Ec2LocationProvider(SnitchProperties props) throws IOException, ConfigurationException + { + this(Ec2MetadataServiceConnector.create(props)); + } + + Ec2LocationProvider(AbstractCloudMetadataServiceConnector connector) throws IOException + { + super(connector, Ec2LocationProvider::getLocation); + usingLegacyNaming = isUsingLegacyNaming(connector.getProperties()); + } + + private static Location getLocation(AbstractCloudMetadataServiceConnector connector) throws IOException + { + // if using the full naming scheme, region name is created by removing letters from the + // end of the availability zone and zone is the full zone name + boolean usingLegacyNaming = isUsingLegacyNaming(connector.getProperties()); + + String az = connector.apiCall(ZONE_NAME_QUERY); + String region; + String localDc; + String localRack; + if (usingLegacyNaming) + { + // Split "us-east-1a" or "asia-1a" into "us-east"/"1a" and "asia"/"1a". + String[] splits = az.split("-"); + localRack = splits[splits.length - 1]; + + // hack for CASSANDRA-4026 + region = az.substring(0, az.length() - 1); + if (region.endsWith("1")) + region = az.substring(0, az.length() - 3); + } + else + { + // grab the region name, which is embedded in the availability zone name. + // thus an AZ of "us-east-1a" yields the region name "us-east-1" + region = az.replaceFirst("[a-z]+$",""); + localRack = az; + } + + localDc = region.concat(connector.getProperties().getDcSuffix()); + + return new Location(localDc, localRack); + } + + private static boolean isUsingLegacyNaming(SnitchProperties props) + { + return props.get(SNITCH_PROP_NAMING_SCHEME, EC2_NAMING_STANDARD).equalsIgnoreCase(EC2_NAMING_LEGACY); + } + + public void validate(ClusterMetadata metadata) + { + // Validate that the settings here match what was used to locate + // and register other nodes in the cluster (if there are any) + Set<String> datacenters = metadata.directory.allDatacenterRacks().keySet(); + Set<String> racks = new HashSet<>(); + for (String dc : datacenters) + racks.addAll(metadata.directory.datacenterRacks(dc).keySet()); + validate(datacenters, racks); + } + + public boolean validate(Set<String> datacenters, Set<String> racks) + { + boolean valid = true; + + for (String dc : datacenters) + { + // predicated on the late-2017 AWS naming 'convention' that all region names end with a digit. + // Unfortunately, life isn't that simple. Since we allow custom datacenter suffixes (CASSANDRA-5155), + // an operator could conceiveably make the suffix "a", and thus create a region name that looks just like + // one of the region's availability zones. (for example, "us-east-1a"). + // Further, we can't make any assumptions of what that suffix might be by looking at this node's + // datacenterSuffix as conceivably their could be many different suffixes in play for a given region. + // + // It is impossible to distinguish standard and legacy names for datacenters in some cases + // as the format didn't change for some regions (us-west-2 for example). + // We can still identify as legacy the dc names without a number as a suffix like us-east" + boolean dcUsesLegacyFormat = dc.matches("^[a-z]+-[a-z]+$"); + if (dcUsesLegacyFormat && !usingLegacyNaming) + { + valid = false; + break; + } + } + + for (String rack : racks) + { + // predicated on late-2017 AWS naming 'convention' that AZs do not have a digit as the first char - + // we had that in our legacy AZ (rack) names. Thus we test to see if the rack is in the legacy format. + // + // NOTE: the allowed custom suffix only applies to datacenter (region) names, not availability zones. + boolean rackUsesLegacyFormat = rack.matches("[\\d][a-z]"); + if (rackUsesLegacyFormat != usingLegacyNaming) + { + valid = false; + break; + } + } + + if (!valid) + { + throw new ConfigurationException(String.format("This ec2-enabled location provider appears to be using the " + + "%s naming scheme for regions, but existing nodes in cluster " + + "are using the opposite: " + + "region(s) = %s, availability zone(s) = %s. " + + "Please check the %s property in the %s configuration file " + + "for more details.", + usingLegacyNaming ? "legacy" : "standard", datacenters, racks, + SNITCH_PROP_NAMING_SCHEME, SnitchProperties.RACKDC_PROPERTY_FILENAME)); + } + return true; + } +} diff --git a/src/java/org/apache/cassandra/locator/Ec2MultiRegionAddressConfig.java b/src/java/org/apache/cassandra/locator/Ec2MultiRegionAddressConfig.java new file mode 100644 index 000000000000..bf2e83dffff5 --- /dev/null +++ b/src/java/org/apache/cassandra/locator/Ec2MultiRegionAddressConfig.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.locator; + +import java.io.IOException; +import java.net.InetAddress; +import java.net.UnknownHostException; + +import com.google.common.annotations.VisibleForTesting; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.utils.FBUtilities; + +public class Ec2MultiRegionAddressConfig implements NodeAddressConfig +{ + private static final Logger logger = LoggerFactory.getLogger(Ec2MultiRegionAddressConfig.class); + + @VisibleForTesting + public static final String PUBLIC_IP_QUERY = "/latest/meta-data/public-ipv4"; + @VisibleForTesting + public static final String PRIVATE_IP_QUERY = "/latest/meta-data/local-ipv4"; + private final String localPublicAddress; + private final String localPrivateAddress; + + /** + * Used via reflection by DatabaseDescriptor::createAddressConfig + */ + public Ec2MultiRegionAddressConfig() throws IOException + { + this(Ec2MetadataServiceConnector.create(new SnitchProperties())); + } + + @VisibleForTesting + public Ec2MultiRegionAddressConfig(AbstractCloudMetadataServiceConnector connector) throws IOException + { + this.localPublicAddress = connector.apiCall(PUBLIC_IP_QUERY); + logger.info("EC2 multi region address config using publicIP as identifier: {}", localPublicAddress); + this.localPrivateAddress = connector.apiCall(PRIVATE_IP_QUERY); + } + + @Override + public void configureAddresses() + { + // use the Public IP to broadcast Address to other nodes. + try + { + InetAddress broadcastAddress = InetAddress.getByName(localPublicAddress); + DatabaseDescriptor.setBroadcastAddress(broadcastAddress); + if (DatabaseDescriptor.getBroadcastRpcAddress() == null) + { + logger.info("broadcast_rpc_address unset, broadcasting public IP as rpc_address: {}", localPublicAddress); + DatabaseDescriptor.setBroadcastRpcAddress(broadcastAddress); + } + } + catch (UnknownHostException e) + { + throw new ConfigurationException("Unable to obtain public address for node from cloud metadata service", e); + } + + try + { + InetAddress privateAddress = InetAddress.getByName(localPrivateAddress); + FBUtilities.setLocalAddress(privateAddress); + } + catch (UnknownHostException e) + { + throw new ConfigurationException("Unable to obtain private address for node from cloud metadata service", e); + } + } + + @Override + public boolean preferLocalConnections() + { + // Always prefer re-connecting on private addresses if in the same datacenter (i.e. region) + return true; + } +} diff --git a/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java b/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java index b9f9f2d16763..234479376823 100644 --- a/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java +++ b/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java @@ -18,16 +18,10 @@ package org.apache.cassandra.locator; import java.io.IOException; -import java.net.InetAddress; -import java.net.UnknownHostException; import com.google.common.annotations.VisibleForTesting; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.gms.ApplicationState; -import org.apache.cassandra.gms.Gossiper; -import org.apache.cassandra.service.StorageService; /** * 1) Snitch will automatically set the public IP by querying the AWS API @@ -39,14 +33,17 @@ * * Operational: All the nodes in this cluster needs to be able to (modify the * Security group settings in AWS) communicate via Public IP's. + * @deprecated See CASSANDRA-19488 */ +@Deprecated(since = "CEP-21") public class Ec2MultiRegionSnitch extends Ec2Snitch { + + private final Ec2MultiRegionAddressConfig addressConfig; @VisibleForTesting static final String PUBLIC_IP_QUERY = "/latest/meta-data/public-ipv4"; @VisibleForTesting static final String PRIVATE_IP_QUERY = "/latest/meta-data/local-ipv4"; - private final String localPrivateAddress; public Ec2MultiRegionSnitch() throws IOException, ConfigurationException { @@ -58,36 +55,22 @@ public Ec2MultiRegionSnitch(SnitchProperties props) throws IOException, Configur this(Ec2MetadataServiceConnector.create(props)); } - Ec2MultiRegionSnitch(AbstractCloudMetadataServiceConnector connector) throws IOException + @VisibleForTesting + public Ec2MultiRegionSnitch(AbstractCloudMetadataServiceConnector connector) throws IOException { super(connector); - InetAddress localPublicAddress = InetAddress.getByName(connector.apiCall(PUBLIC_IP_QUERY)); - logger.info("EC2Snitch using publicIP as identifier: {}", localPublicAddress); - localPrivateAddress = connector.apiCall(PRIVATE_IP_QUERY); - // use the Public IP to broadcast Address to other nodes. - DatabaseDescriptor.setBroadcastAddress(localPublicAddress); - if (DatabaseDescriptor.getBroadcastRpcAddress() == null) - { - logger.info("broadcast_rpc_address unset, broadcasting public IP as rpc_address: {}", localPublicAddress); - DatabaseDescriptor.setBroadcastRpcAddress(localPublicAddress); - } + addressConfig = new Ec2MultiRegionAddressConfig(connector); + } + + @Override + public void configureAddresses() + { + addressConfig.configureAddresses(); } @Override - public void gossiperStarting() + public boolean preferLocalConnections() { - super.gossiperStarting(); - InetAddressAndPort address; - try - { - address = InetAddressAndPort.getByName(localPrivateAddress); - } - catch (UnknownHostException e) - { - throw new RuntimeException(e); - } - Gossiper.instance.addLocalApplicationState(ApplicationState.INTERNAL_ADDRESS_AND_PORT, StorageService.instance.valueFactory.internalAddressAndPort(address)); - Gossiper.instance.addLocalApplicationState(ApplicationState.INTERNAL_IP, StorageService.instance.valueFactory.internalIP(address.getAddress())); - Gossiper.instance.register(new ReconnectableSnitchHelper(this, getLocalDatacenter(), true)); + return addressConfig.preferLocalConnections(); } } diff --git a/src/java/org/apache/cassandra/locator/Ec2Snitch.java b/src/java/org/apache/cassandra/locator/Ec2Snitch.java index 9b64efa041d6..2d30c0bd3bc5 100644 --- a/src/java/org/apache/cassandra/locator/Ec2Snitch.java +++ b/src/java/org/apache/cassandra/locator/Ec2Snitch.java @@ -23,7 +23,6 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.utils.Pair; /** * A snitch that assumes an EC2 region is a DC and an EC2 availability_zone @@ -44,7 +43,9 @@ * to a user is {@link Ec2MetadataServiceConnector.V2Connector#AWS_EC2_METADATA_TOKEN_TTL_SECONDS_HEADER_PROPERTY} * which is by default set to {@link Ec2MetadataServiceConnector.V2Connector#MAX_TOKEN_TIME_IN_SECONDS}. TTL has * to be an integer from the range [30, 21600]. + * @deprecated See CASSANDRA-19488 */ +@Deprecated(since = "CEP-21") public class Ec2Snitch extends AbstractCloudMetadataServiceSnitch { private static final String SNITCH_PROP_NAMING_SCHEME = "ec2_naming_scheme"; @@ -54,8 +55,6 @@ public class Ec2Snitch extends AbstractCloudMetadataServiceSnitch @VisibleForTesting public static final String ZONE_NAME_QUERY = "/latest/meta-data/placement/availability-zone"; - private final boolean usingLegacyNaming; - public Ec2Snitch() throws IOException, ConfigurationException { this(new SnitchProperties()); @@ -68,103 +67,12 @@ public Ec2Snitch(SnitchProperties props) throws IOException, ConfigurationExcept Ec2Snitch(AbstractCloudMetadataServiceConnector connector) throws IOException { - super(connector, getDcAndRack(connector)); - usingLegacyNaming = isUsingLegacyNaming(connector.getProperties()); - } - - private static Pair<String, String> getDcAndRack(AbstractCloudMetadataServiceConnector connector) throws IOException - { - String az = connector.apiCall(ZONE_NAME_QUERY); - - // if using the full naming scheme, region name is created by removing letters from the - // end of the availability zone and zone is the full zone name - boolean usingLegacyNaming = isUsingLegacyNaming(connector.getProperties()); - String region; - String localDc; - String localRack; - if (usingLegacyNaming) - { - // Split "us-east-1a" or "asia-1a" into "us-east"/"1a" and "asia"/"1a". - String[] splits = az.split("-"); - localRack = splits[splits.length - 1]; - - // hack for CASSANDRA-4026 - region = az.substring(0, az.length() - 1); - if (region.endsWith("1")) - region = az.substring(0, az.length() - 3); - } - else - { - // grab the region name, which is embedded in the availability zone name. - // thus an AZ of "us-east-1a" yields the region name "us-east-1" - region = az.replaceFirst("[a-z]+$",""); - localRack = az; - } - - localDc = region.concat(connector.getProperties().getDcSuffix()); - - return Pair.create(localDc, localRack); - } - - private static boolean isUsingLegacyNaming(SnitchProperties props) - { - return props.get(SNITCH_PROP_NAMING_SCHEME, EC2_NAMING_STANDARD).equalsIgnoreCase(EC2_NAMING_LEGACY); + super(new Ec2LocationProvider(connector)); } @Override public boolean validate(Set<String> datacenters, Set<String> racks) { - return validate(datacenters, racks, usingLegacyNaming); - } - - @VisibleForTesting - static boolean validate(Set<String> datacenters, Set<String> racks, boolean usingLegacyNaming) - { - boolean valid = true; - - for (String dc : datacenters) - { - // predicated on the late-2017 AWS naming 'convention' that all region names end with a digit. - // Unfortunately, life isn't that simple. Since we allow custom datacenter suffixes (CASSANDRA-5155), - // an operator could conceiveably make the suffix "a", and thus create a region name that looks just like - // one of the region's availability zones. (for example, "us-east-1a"). - // Further, we can't make any assumptions of what that suffix might be by looking at this node's - // datacenterSuffix as conceivably their could be many different suffixes in play for a given region. - // - // It is impossible to distinguish standard and legacy names for datacenters in some cases - // as the format didn't change for some regions (us-west-2 for example). - // We can still identify as legacy the dc names without a number as a suffix like us-east" - boolean dcUsesLegacyFormat = dc.matches("^[a-z]+-[a-z]+$"); - if (dcUsesLegacyFormat && !usingLegacyNaming) - { - valid = false; - break; - } - } - - for (String rack : racks) - { - // predicated on late-2017 AWS naming 'convention' that AZs do not have a digit as the first char - - // we had that in our legacy AZ (rack) names. Thus we test to see if the rack is in the legacy format. - // - // NOTE: the allowed custom suffix only applies to datacenter (region) names, not availability zones. - boolean rackUsesLegacyFormat = rack.matches("[\\d][a-z]"); - if (rackUsesLegacyFormat != usingLegacyNaming) - { - valid = false; - break; - } - } - - if (!valid) - { - logger.error("This ec2-enabled snitch appears to be using the {} naming scheme for regions, " + - "but existing nodes in cluster are using the opposite: region(s) = {}, availability zone(s) = {}. " + - "Please check the {} property in the {} configuration file for more details.", - usingLegacyNaming ? "legacy" : "standard", datacenters, racks, - SNITCH_PROP_NAMING_SCHEME, SnitchProperties.RACKDC_PROPERTY_FILENAME); - } - - return valid; + return ((Ec2LocationProvider)locationProvider).validate(datacenters, racks); } } diff --git a/src/java/org/apache/cassandra/locator/EndpointSnitchInfo.java b/src/java/org/apache/cassandra/locator/EndpointSnitchInfo.java index d836cd18062b..467d68b66ddc 100644 --- a/src/java/org/apache/cassandra/locator/EndpointSnitchInfo.java +++ b/src/java/org/apache/cassandra/locator/EndpointSnitchInfo.java @@ -22,6 +22,7 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.utils.MBeanWrapper; +@Deprecated(since = "CEP-21") public class EndpointSnitchInfo implements EndpointSnitchInfoMBean { public static void create() @@ -31,26 +32,28 @@ public static void create() public String getDatacenter(String host) throws UnknownHostException { - return DatabaseDescriptor.getEndpointSnitch().getDatacenter(InetAddressAndPort.getByName(host)); + return DatabaseDescriptor.getLocator().location(InetAddressAndPort.getByName(host)).datacenter; } public String getRack(String host) throws UnknownHostException { - return DatabaseDescriptor.getEndpointSnitch().getRack(InetAddressAndPort.getByName(host)); + return DatabaseDescriptor.getLocator().location(InetAddressAndPort.getByName(host)).rack; } public String getDatacenter() { - return DatabaseDescriptor.getEndpointSnitch().getLocalDatacenter(); + return DatabaseDescriptor.getLocator().local().datacenter; } public String getRack() { - return DatabaseDescriptor.getEndpointSnitch().getLocalRack(); + return DatabaseDescriptor.getLocator().local().rack; } public String getSnitchName() { - return DatabaseDescriptor.getEndpointSnitch().getClass().getName(); + NodeProximity proximity = DatabaseDescriptor.getNodeProximity(); + Class<?> clazz = proximity instanceof SnitchAdapter ? ((SnitchAdapter)proximity).snitch.getClass() : proximity.getClass(); + return clazz.getName(); } } diff --git a/src/java/org/apache/cassandra/locator/EndpointSnitchInfoMBean.java b/src/java/org/apache/cassandra/locator/EndpointSnitchInfoMBean.java index d6a18ff78632..addc6a8ca8f6 100644 --- a/src/java/org/apache/cassandra/locator/EndpointSnitchInfoMBean.java +++ b/src/java/org/apache/cassandra/locator/EndpointSnitchInfoMBean.java @@ -21,7 +21,10 @@ /** * MBean exposing standard Snitch info + * + * @deprecated see CASSANDRA-19488 */ +@Deprecated(since = "CEP-21") public interface EndpointSnitchInfoMBean { /** diff --git a/src/java/org/apache/cassandra/locator/GoogleCloudLocationProvider.java b/src/java/org/apache/cassandra/locator/GoogleCloudLocationProvider.java new file mode 100644 index 000000000000..f01955e80376 --- /dev/null +++ b/src/java/org/apache/cassandra/locator/GoogleCloudLocationProvider.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.locator; + +import java.io.IOException; + +import com.google.common.collect.ImmutableMap; + +import org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.DefaultCloudMetadataServiceConnector; + +import static org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.METADATA_URL_PROPERTY; + +public class GoogleCloudLocationProvider extends CloudMetadataLocationProvider +{ + static final String DEFAULT_METADATA_SERVICE_URL = "http://metadata.google.internal"; + static final String ZONE_NAME_QUERY_URL = "/computeMetadata/v1/instance/zone"; + static final ImmutableMap<String, String> HEADERS = ImmutableMap.of("Metadata-Flavor", "Google"); + + /** + * Used via reflection by DatabaseDescriptor::createInitialLocationProvider + */ + public GoogleCloudLocationProvider() throws IOException + { + this(new SnitchProperties()); + } + + public GoogleCloudLocationProvider(SnitchProperties properties) throws IOException + { + this(new DefaultCloudMetadataServiceConnector(properties.putIfAbsent(METADATA_URL_PROPERTY, + DEFAULT_METADATA_SERVICE_URL))); + } + + public GoogleCloudLocationProvider(AbstractCloudMetadataServiceConnector connector) throws IOException + { + super(connector, c -> SnitchUtils.parseLocation(c.apiCall(ZONE_NAME_QUERY_URL, HEADERS), + c.getProperties().getDcSuffix())); + } +} diff --git a/src/java/org/apache/cassandra/locator/GoogleCloudSnitch.java b/src/java/org/apache/cassandra/locator/GoogleCloudSnitch.java index af3257187e55..fdd0991c23f5 100644 --- a/src/java/org/apache/cassandra/locator/GoogleCloudSnitch.java +++ b/src/java/org/apache/cassandra/locator/GoogleCloudSnitch.java @@ -19,21 +19,14 @@ import java.io.IOException; -import com.google.common.collect.ImmutableMap; - -import org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.DefaultCloudMetadataServiceConnector; - -import static org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.METADATA_URL_PROPERTY; - /** * A snitch that assumes an GCE region is a DC and an GCE availability_zone * is a rack. This information is available in the config for the node. + * @deprecated See CASSANDRA-19488 */ +@Deprecated(since = "CEP-21") public class GoogleCloudSnitch extends AbstractCloudMetadataServiceSnitch { - static final String DEFAULT_METADATA_SERVICE_URL = "http://metadata.google.internal"; - static final String ZONE_NAME_QUERY_URL = "/computeMetadata/v1/instance/zone"; - public GoogleCloudSnitch() throws IOException { this(new SnitchProperties()); @@ -41,14 +34,11 @@ public GoogleCloudSnitch() throws IOException public GoogleCloudSnitch(SnitchProperties properties) throws IOException { - this(new DefaultCloudMetadataServiceConnector(properties.putIfAbsent(METADATA_URL_PROPERTY, - DEFAULT_METADATA_SERVICE_URL))); + super(new GoogleCloudLocationProvider(properties)); } public GoogleCloudSnitch(AbstractCloudMetadataServiceConnector connector) throws IOException { - super(connector, SnitchUtils.parseDcAndRack(connector.apiCall(ZONE_NAME_QUERY_URL, - ImmutableMap.of("Metadata-Flavor", "Google")), - connector.getProperties().getDcSuffix())); + super(new GoogleCloudLocationProvider(connector)); } } diff --git a/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java b/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java index bbd110588b53..2e02165906a5 100644 --- a/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java +++ b/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java @@ -18,107 +18,40 @@ package org.apache.cassandra.locator; -import java.util.concurrent.atomic.AtomicReference; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.gms.ApplicationState; -import org.apache.cassandra.gms.Gossiper; -import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.tcm.ClusterMetadata; -import org.apache.cassandra.tcm.membership.NodeId; -import org.apache.cassandra.utils.FBUtilities; - +import org.apache.cassandra.tcm.membership.Location; -public class GossipingPropertyFileSnitch extends AbstractNetworkTopologySnitch// implements IEndpointStateChangeSubscriber +/** + * @deprecated See CASSANDRA-19488 + */ +@Deprecated(since = "CEP-21") +public class GossipingPropertyFileSnitch extends AbstractNetworkTopologySnitch { - private static final Logger logger = LoggerFactory.getLogger(GossipingPropertyFileSnitch.class); - - private final String myDC; - private final String myRack; - private final boolean preferLocal; - private final AtomicReference<ReconnectableSnitchHelper> snitchHelperReference; - private static final String DEFAULT_DC = "UNKNOWN_DC"; - private static final String DEFAULT_RACK = "UNKNOWN_RACK"; + private final Location fromConfig; + public final boolean preferLocal; public GossipingPropertyFileSnitch() throws ConfigurationException { - SnitchProperties properties = loadConfiguration(); - - myDC = properties.get("dc", DEFAULT_DC).trim(); - myRack = properties.get("rack", DEFAULT_RACK).trim(); + SnitchProperties properties = RackDCFileLocationProvider.loadConfiguration(); + fromConfig = new RackDCFileLocationProvider(properties).initialLocation(); preferLocal = Boolean.parseBoolean(properties.get("prefer_local", "false")); - snitchHelperReference = new AtomicReference<>(); } - private static SnitchProperties loadConfiguration() throws ConfigurationException + @Override + public String getLocalRack() { - final SnitchProperties properties = new SnitchProperties(); - if (!properties.contains("dc") || !properties.contains("rack")) - throw new ConfigurationException("DC or rack not found in snitch properties, check your configuration in: " + SnitchProperties.RACKDC_PROPERTY_FILENAME); - - return properties; + return fromConfig.rack; } - /** - * Return the data center for which an endpoint resides in - * - * @param endpoint the endpoint to process - * @return string of data center - */ - public String getDatacenter(InetAddressAndPort endpoint) + @Override + public String getLocalDatacenter() { - if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort())) - return myDC; - - ClusterMetadata metadata = ClusterMetadata.current(); - NodeId nodeId = metadata.directory.peerId(endpoint); - if (nodeId == null) - return DEFAULT_DC; - return metadata.directory.location(nodeId).datacenter; + return fromConfig.datacenter; } - /** - * Return the rack for which an endpoint resides in - * - * @param endpoint the endpoint to process - * @return string of rack - */ - public String getRack(InetAddressAndPort endpoint) + @Override + public boolean preferLocalConnections() { - if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort())) - return myRack; - - ClusterMetadata metadata = ClusterMetadata.current(); - NodeId nodeId = metadata.directory.peerId(endpoint); - if (nodeId == null) - return DEFAULT_RACK; - return metadata.directory.location(nodeId).rack; - } - - public void gossiperStarting() - { - super.gossiperStarting(); - - Gossiper.instance.addLocalApplicationState(ApplicationState.INTERNAL_ADDRESS_AND_PORT, - StorageService.instance.valueFactory.internalAddressAndPort(FBUtilities.getLocalAddressAndPort())); - Gossiper.instance.addLocalApplicationState(ApplicationState.INTERNAL_IP, - StorageService.instance.valueFactory.internalIP(FBUtilities.getJustLocalAddress())); - - loadGossiperState(); - } - - private void loadGossiperState() - { - assert Gossiper.instance != null; - - ReconnectableSnitchHelper pendingHelper = new ReconnectableSnitchHelper(this, myDC, preferLocal); - Gossiper.instance.register(pendingHelper); - - pendingHelper = snitchHelperReference.getAndSet(pendingHelper); - if (pendingHelper != null) - Gossiper.instance.unregister(pendingHelper); + return preferLocal; } } diff --git a/src/java/org/apache/cassandra/locator/IEndpointSnitch.java b/src/java/org/apache/cassandra/locator/IEndpointSnitch.java index 0120391265d1..4d5033681083 100644 --- a/src/java/org/apache/cassandra/locator/IEndpointSnitch.java +++ b/src/java/org/apache/cassandra/locator/IEndpointSnitch.java @@ -26,8 +26,9 @@ * This interface helps determine location of node in the datacenter relative to another node. * Give a node A and another node B it can tell if A and B are on the same rack or in the same * datacenter. + * @deprecated Replaced by Locator and NodeProximity; see CASSANDRA-19488 */ - +@Deprecated(since="CEP-21") public interface IEndpointSnitch { /** @@ -94,4 +95,11 @@ default boolean validate(Set<String> datacenters, Set<String> racks) { return true; } + + default void configureAddresses() {} + default boolean preferLocalConnections() + { + return false; + } } + diff --git a/src/java/org/apache/cassandra/locator/InOurDc.java b/src/java/org/apache/cassandra/locator/InOurDc.java index 34e8ef89c360..4859ebd23a58 100644 --- a/src/java/org/apache/cassandra/locator/InOurDc.java +++ b/src/java/org/apache/cassandra/locator/InOurDc.java @@ -20,8 +20,8 @@ import java.util.function.Predicate; -import static org.apache.cassandra.config.DatabaseDescriptor.getEndpointSnitch; import static org.apache.cassandra.config.DatabaseDescriptor.getLocalDataCenter; +import static org.apache.cassandra.config.DatabaseDescriptor.getLocator; public class InOurDc { @@ -29,49 +29,44 @@ public class InOurDc private static EndpointTester endpoints; final String dc; - final IEndpointSnitch snitch; + final Locator locator; - private InOurDc(String dc, IEndpointSnitch snitch) + private InOurDc(String dc, Locator locator) { this.dc = dc; - this.snitch = snitch; + this.locator = locator; } boolean stale() { - return dc != getLocalDataCenter() - || snitch != getEndpointSnitch() - // this final clause checks if somehow the snitch/localDc have got out of whack; - // presently, this is possible but very unlikely, but this check will also help - // resolve races on these global fields as well - || !dc.equals(snitch.getLocalDatacenter()); + return !dc.equals(getLocalDataCenter()); } private static final class ReplicaTester extends InOurDc implements Predicate<Replica> { - private ReplicaTester(String dc, IEndpointSnitch snitch) + private ReplicaTester(String dc, Locator locator) { - super(dc, snitch); + super(dc, locator); } @Override public boolean test(Replica replica) { - return dc.equals(snitch.getDatacenter(replica.endpoint())); + return dc.equals(locator.location(replica.endpoint()).datacenter); } } private static final class EndpointTester extends InOurDc implements Predicate<InetAddressAndPort> { - private EndpointTester(String dc, IEndpointSnitch snitch) + private EndpointTester(String dc, Locator locator) { - super(dc, snitch); + super(dc, locator); } @Override public boolean test(InetAddressAndPort endpoint) { - return dc.equals(snitch.getDatacenter(endpoint)); + return dc.equals(locator.location(endpoint).datacenter); } } @@ -79,7 +74,7 @@ public static Predicate<Replica> replicas() { ReplicaTester cur = replicas; if (cur == null || cur.stale()) - replicas = cur = new ReplicaTester(getLocalDataCenter(), getEndpointSnitch()); + replicas = cur = new ReplicaTester(getLocalDataCenter(), getLocator()); return cur; } @@ -87,7 +82,7 @@ public static Predicate<InetAddressAndPort> endpoints() { EndpointTester cur = endpoints; if (cur == null || cur.stale()) - endpoints = cur = new EndpointTester(getLocalDataCenter(), getEndpointSnitch()); + endpoints = cur = new EndpointTester(getLocalDataCenter(), getLocator()); return cur; } diff --git a/src/java/org/apache/cassandra/locator/InitialLocationProvider.java b/src/java/org/apache/cassandra/locator/InitialLocationProvider.java new file mode 100644 index 000000000000..17fafd88af2d --- /dev/null +++ b/src/java/org/apache/cassandra/locator/InitialLocationProvider.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.locator; + +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.membership.Location; + +public interface InitialLocationProvider +{ + /** + * Provides the Location with which a new node should register itself with ClusterMetadata. + * After registration, this is no longer necessary and Location is always sourced from + * ClusterMetadata + * @return the datacenter and rack to register with + */ + Location initialLocation(); + + /** + * Validate that the locations of previously registered peers are considered valid by the locally configured + * InitialLocationProvider. In practice, of the in-tree implementations only Ec2LocationProvider overrides this and + * uses it to ensure that the same ec2 naming scheme is used across all peers. + * See CASSANDRA-7839 for origins. + * @param metadata ClusterMetadata at the time of registering + * @return true if the implementation considers the locations of existing nodes compatible with its own + * configuration, false otherwise + */ + default void validate(ClusterMetadata metadata) {} +} diff --git a/src/java/org/apache/cassandra/locator/LocationInfo.java b/src/java/org/apache/cassandra/locator/LocationInfo.java new file mode 100644 index 000000000000..c0b3e7881840 --- /dev/null +++ b/src/java/org/apache/cassandra/locator/LocationInfo.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.locator; + +import java.net.UnknownHostException; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.utils.MBeanWrapper; + +public class LocationInfo implements LocationInfoMBean +{ + public static void create() + { + MBeanWrapper.instance.registerMBean(new LocationInfo(), "org.apache.cassandra.db:type=LocationInfo"); + } + + public String getDatacenter(String host) throws UnknownHostException + { + return DatabaseDescriptor.getLocator().location(InetAddressAndPort.getByName(host)).datacenter; + } + + public String getRack(String host) throws UnknownHostException + { + return DatabaseDescriptor.getLocator().location(InetAddressAndPort.getByName(host)).rack; + } + + public String getDatacenter() + { + return DatabaseDescriptor.getLocator().local().datacenter; + } + + public String getRack() + { + return DatabaseDescriptor.getLocator().local().rack; + } + + public String getNodeProximityName() + { + NodeProximity proximity = DatabaseDescriptor.getNodeProximity(); + Class<?> clazz = proximity instanceof SnitchAdapter ? ((SnitchAdapter)proximity).snitch.getClass() : proximity.getClass(); + return clazz.getName(); + } + + public boolean hasLegacySnitchAdapter() + { + NodeProximity proximity = DatabaseDescriptor.getNodeProximity(); + if (proximity instanceof DynamicEndpointSnitch) + return ((DynamicEndpointSnitch)proximity).delegate instanceof SnitchAdapter; + return proximity instanceof SnitchAdapter; + } +} diff --git a/src/java/org/apache/cassandra/locator/LocationInfoMBean.java b/src/java/org/apache/cassandra/locator/LocationInfoMBean.java new file mode 100644 index 000000000000..e1c7100ecafa --- /dev/null +++ b/src/java/org/apache/cassandra/locator/LocationInfoMBean.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.locator; + +import java.net.UnknownHostException; + +public interface LocationInfoMBean +{ + /** + * Provides the Rack name depending on the respective snitch used, given the host name/ip + * @param host + * @throws UnknownHostException + */ + public String getRack(String host) throws UnknownHostException; + + /** + * Provides the Datacenter name depending on the respective snitch used, given the hostname/ip + * @param host + * @throws UnknownHostException + */ + public String getDatacenter(String host) throws UnknownHostException; + + /** + * Provides the Rack name depending on the respective snitch used for this node + */ + public String getRack(); + + /** + * Provides the Datacenter name depending on the respective snitch used for this node + */ + public String getDatacenter(); + + public String getNodeProximityName(); + public boolean hasLegacySnitchAdapter(); +} diff --git a/src/java/org/apache/cassandra/locator/Locator.java b/src/java/org/apache/cassandra/locator/Locator.java new file mode 100644 index 000000000000..20ad6046923e --- /dev/null +++ b/src/java/org/apache/cassandra/locator/Locator.java @@ -0,0 +1,195 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.locator; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.Epoch; +import org.apache.cassandra.tcm.RegistrationStatus; +import org.apache.cassandra.tcm.membership.Directory; +import org.apache.cassandra.tcm.membership.Location; +import org.apache.cassandra.tcm.membership.NodeId; +import org.apache.cassandra.utils.FBUtilities; + +/** + * Provides Location (datacenter & rack) information for endpoints. Usually this is obtained directly + * from ClusterMetadata.directory, using either a specific directory instance or the most current + * published one. During initial startup, location info for the local node may be derived from some + * other source, such as a config file or cloud metadata api. This is then used to register the node + * and its location in ClusterMetadata, which then becomes the ultimate source of truth. + */ +public class Locator +{ + private final InetAddressAndPort localEndpoint; + + // Indicates whether the *local node* is yet to register itself with ClusterMetadata. + // This is relevant because once a node is registered, it's location is always derived + // from ClusterMetadata. However, before registering the location is obtained from + // configuration. This pre-registration location is what the node supplies to ClusterMetadata + // when registering and is also used to determine DC locality of peers, for instance when + // establishing initial internode connections during the discovery phase. + private final RegistrationStatus state; + + // Source of truth for location lookups. This may be null, in which case every lookup will + // use the most up to date version from ClusterMetadata.current().directory + private final Directory directory; + + // Supplies the Location for this node only during its initial startup. This location will be + // used to register the node with ClusterMetadata and is not used after that has occurred. + // See DatabaseDescriptor::getInitialLocationProvider + private final InitialLocationProvider locationProvider; + + // This is the Location used to register this node during its initial startup. It is lazily initialized + // using the supplied InitialLocationProvider and memoized here as that may be a non-trivial operation. + // Some providers fetch location metadata from remote services etc. It should usually be unnecessary to + // access the initialization location after a node's first startup. + private volatile Location initializationLocation; + + // Set from ClusterMetadata. After initial registration has happened, location of this node itself + // is always taken from ClusterMetadata. + private volatile VersionedLocation local; + + private static class VersionedLocation + { + final Epoch epoch; + final Location location; + private VersionedLocation(Epoch epoch, Location location) + { + this.epoch = epoch; + this.location = location; + } + } + + public static Locator usingDirectory(Directory directory) + { + return new Locator(RegistrationStatus.instance, + FBUtilities.getBroadcastAddressAndPort(), + DatabaseDescriptor.getInitialLocationProvider(), + directory); + } + + public static Locator forClients() + { + return new Locator(RegistrationStatus.instance, + FBUtilities.getBroadcastAddressAndPort(), + () -> Location.UNKNOWN, + null); + } + + /** + * Creates a Locator instance which always uses the Directory from the most current ClusterMetadata. + * This means that the values returned from {@link this#location(InetAddressAndPort)} can change between + * invocations, if interleaved with the publication of updated cluster metadata. + */ + public Locator(RegistrationStatus state, + InetAddressAndPort localEndpoint, + InitialLocationProvider provider) + { + this(state, localEndpoint, provider, null); + } + + /** + * Creates a Locator instance which returns consistent results based on the supplied Directory instance. + * Changes to RegistrationStatus could still have an effect i.e. the node transitioned from UNREGISTERED to + * REGISTERED between two calls to local(), the first would return the Location according to + * DatabaseDescriptor.getInitialLocationProvider, but the second would consult the supplied Directory. + */ + public Locator(RegistrationStatus state, + InetAddressAndPort localEndpoint, + InitialLocationProvider provider, + Directory directory) + { + this.state = state; + this.localEndpoint = localEndpoint; + this.locationProvider = provider; + this.directory = directory; + this.local = new VersionedLocation(Epoch.EMPTY, Location.UNKNOWN); + } + + public Location location(InetAddressAndPort endpoint) + { + switch (state.getCurrent()) + { + case INITIAL: + return endpoint.equals(localEndpoint) ? initialLocation() : Location.UNKNOWN; + case UNREGISTERED: + return endpoint.equals(localEndpoint) ? initialLocation() : fromDirectory(endpoint); + default: + return fromDirectory(endpoint); + } + } + + public Location local() + { + switch (state.getCurrent()) + { + case INITIAL: + case UNREGISTERED: + return initialLocation(); + default: + // For now, local location is immutable and once registered with cluster metadata, it cannot be + // changed. Revisit this if that assumption changes. + VersionedLocation location = local; + if (location.epoch.isAfter(Epoch.EMPTY)) + return location.location; + + local = versionedFromDirectory(localEndpoint); + return local.location; + } + } + + // The distinction between versioned and unversioned may be removed if/when we allow + // a node's Location to be modified after registration. This duplication should not + // be necessary then. + private VersionedLocation versionedFromDirectory(InetAddressAndPort endpoint) + { + Directory source = directory; + if (source == null) + { + ClusterMetadata metadata = ClusterMetadata.currentNullable(); + if (metadata == null) + return new VersionedLocation(Epoch.EMPTY, Location.UNKNOWN); + source = metadata.directory; + } + NodeId nodeId = source.peerId(endpoint); + Location location = nodeId != null ? source.location(nodeId) : Location.UNKNOWN; + return new VersionedLocation(source.lastModified(), location); + } + + private Location fromDirectory(InetAddressAndPort endpoint) + { + Directory source = directory; + if (source == null) + { + ClusterMetadata metadata = ClusterMetadata.currentNullable(); + if (metadata == null) + return Location.UNKNOWN; + source = metadata.directory; + } + NodeId nodeId = source.peerId(endpoint); + return nodeId != null ? source.location(nodeId) : Location.UNKNOWN; + } + + private Location initialLocation() + { + if (initializationLocation == null) + initializationLocation = locationProvider.initialLocation(); + return initializationLocation; + } +} diff --git a/src/java/org/apache/cassandra/locator/NetworkTopologyProximity.java b/src/java/org/apache/cassandra/locator/NetworkTopologyProximity.java new file mode 100644 index 000000000000..eddcb3630364 --- /dev/null +++ b/src/java/org/apache/cassandra/locator/NetworkTopologyProximity.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.locator; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.tcm.membership.Location; + +public class NetworkTopologyProximity extends BaseProximity +{ + public int compareEndpoints(InetAddressAndPort address, Replica r1, Replica r2) + { + InetAddressAndPort a1 = r1.endpoint(); + InetAddressAndPort a2 = r2.endpoint(); + if (address.equals(a1) && !address.equals(a2)) + return -1; + if (address.equals(a2) && !address.equals(a1)) + return 1; + + Locator locator = DatabaseDescriptor.getLocator(); + Location l1 = locator.location(a1); + Location l2 = locator.location(a2); + Location location = locator.location(address); + + if (location.datacenter.equals(l1.datacenter) && !location.datacenter.equals(l2.datacenter)) + return -1; + if (location.datacenter.equals(l2.datacenter) && !location.datacenter.equals(l1.datacenter)) + return 1; + + if (location.rack.equals(l1.rack) && !location.rack.equals(l2.rack)) + return -1; + if (location.rack.equals(l2.rack) && !location.rack.equals(l1.rack)) + return 1; + return 0; + } +} diff --git a/src/java/org/apache/cassandra/locator/NoOpProximity.java b/src/java/org/apache/cassandra/locator/NoOpProximity.java new file mode 100644 index 000000000000..342f12e61958 --- /dev/null +++ b/src/java/org/apache/cassandra/locator/NoOpProximity.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.locator; + +public class NoOpProximity extends BaseProximity +{ + @Override + public <C extends ReplicaCollection<? extends C>> C sortedByProximity(final InetAddressAndPort address, C unsortedAddress) + { + // Optimization to avoid walking the list + return unsortedAddress; + } + + @Override + public int compareEndpoints(InetAddressAndPort target, Replica r1, Replica r2) + { + // Making all endpoints equal ensures we won't change the original ordering (since + // Collections.sort is guaranteed to be stable) + return 0; + } +} diff --git a/src/java/org/apache/cassandra/locator/NodeAddressConfig.java b/src/java/org/apache/cassandra/locator/NodeAddressConfig.java new file mode 100644 index 000000000000..23bf4e99765f --- /dev/null +++ b/src/java/org/apache/cassandra/locator/NodeAddressConfig.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.locator; + +import org.apache.cassandra.config.DatabaseDescriptor; + +public interface NodeAddressConfig +{ + void configureAddresses(); + boolean preferLocalConnections(); + + NodeAddressConfig DEFAULT = new NodeAddressConfig() + { + @Override + public void configureAddresses() + { + // Default is to use whatever addresses are specified in node config already, so this is a no-op + } + + @Override + public boolean preferLocalConnections() + { + // Previously the equivalent config was hard coded for Ec2MultiRegionSnitch and read from + // cassandra-rackdc.properties in the case of GossipingPropertyFileSnitch. In legacy config + // mode, where one of those IEndpointSnitch impls is still specified the original behaviour + // is preserved. For modern config the option can be specified in the main yaml, distinct + // from the NodeProximity and InitialLocationProvider options. + return DatabaseDescriptor.preferLocalConnections(); + } + }; +} diff --git a/src/java/org/apache/cassandra/locator/NodeProximity.java b/src/java/org/apache/cassandra/locator/NodeProximity.java new file mode 100644 index 000000000000..cbb7158aafa8 --- /dev/null +++ b/src/java/org/apache/cassandra/locator/NodeProximity.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.locator; + +public interface NodeProximity +{ + /** + * returns a new <tt>List</tt> sorted by proximity to the given endpoint + */ + public <C extends ReplicaCollection<? extends C>> C sortedByProximity(final InetAddressAndPort address, C addresses); + + /** + * compares two endpoints in relation to the target endpoint, returning as Comparator.compare would + */ + public int compareEndpoints(InetAddressAndPort target, Replica r1, Replica r2); + + /** + * Returns whether for a range query doing a query against merged is likely + * to be faster than 2 sequential queries, one against l1 followed by one against l2. + */ + public boolean isWorthMergingForRangeQuery(ReplicaCollection<?> merged, ReplicaCollection<?> l1, ReplicaCollection<?> l2); +} diff --git a/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java b/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java index 430779f46869..ee70fa8ae4f9 100644 --- a/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java +++ b/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java @@ -17,22 +17,8 @@ */ package org.apache.cassandra.locator; -import java.io.InputStream; -import java.net.UnknownHostException; -import java.util.Map; -import java.util.Properties; - -import com.google.common.annotations.VisibleForTesting; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.membership.Location; -import org.apache.cassandra.tcm.membership.NodeId; -import org.apache.cassandra.utils.FBUtilities; - -import org.apache.commons.lang3.StringUtils; /** * <p> @@ -50,133 +36,28 @@ * this is done automatically with location derived from gossip state (ultimately from system.local). * Once registered, the Rack & DC should not be changed but currently the only safeguards against this are the * StartupChecks which validate the snitch against system.local. + * @deprecated See CASSANDRA-19488 */ +@Deprecated(since = "CEP-21") public class PropertyFileSnitch extends AbstractNetworkTopologySnitch { - private static final Logger logger = LoggerFactory.getLogger(PropertyFileSnitch.class); - - public static final String SNITCH_PROPERTIES_FILENAME = "cassandra-topology.properties"; - // All the defaults - private static final String DEFAULT_PROPERTY = "default"; - @VisibleForTesting - public static final String DEFAULT_DC = "default"; - @VisibleForTesting - public static final String DEFAULT_RACK = "default"; - + // Used only during initialization of a new node. This provides the location it will register in cluster metadata private final Location local; - public PropertyFileSnitch() throws ConfigurationException { - local = loadConfiguration(); - } - - public String getDatacenter(InetAddressAndPort endpoint) - { - if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort())) - return local.datacenter; - - ClusterMetadata metadata = ClusterMetadata.current(); - NodeId nodeId = metadata.directory.peerId(endpoint); - if (nodeId == null) - return DEFAULT_DC; - return metadata.directory.location(nodeId).datacenter; - } - - /** - * Return the rack for which an endpoint resides in - * - * @param endpoint the endpoint to process - * @return string of rack - */ - public String getRack(InetAddressAndPort endpoint) - { - if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort())) - return local.rack; - - ClusterMetadata metadata = ClusterMetadata.current(); - NodeId nodeId = metadata.directory.peerId(endpoint); - if (nodeId == null) - return DEFAULT_RACK; - return metadata.directory.location(nodeId).rack; + local = new TopologyFileLocationProvider().initialLocation(); } - private Location makeLocation(String value) + @Override + public String getLocalRack() { - if (value == null || value.isEmpty()) - return null; - - String[] parts = value.split(":"); - if (parts.length < 2) - { - return new Location(DEFAULT_DC, DEFAULT_RACK); - } - else - { - return new Location(parts[0].trim(), parts[1].trim()); - } + return local.rack; } - private Location loadConfiguration() throws ConfigurationException + @Override + public String getLocalDatacenter() { - Properties properties = new Properties(); - try (InputStream stream = getClass().getClassLoader().getResourceAsStream(SNITCH_PROPERTIES_FILENAME)) - { - properties.load(stream); - } - catch (Exception e) - { - throw new ConfigurationException("Unable to read " + SNITCH_PROPERTIES_FILENAME, e); - } - - // may be null, which is ok unless config doesn't contain the location of the local node - Location defaultLocation = makeLocation(properties.getProperty(DEFAULT_PROPERTY)); - Location local = null; - InetAddressAndPort broadcastAddress = FBUtilities.getBroadcastAddressAndPort(); - for (Map.Entry<Object, Object> entry : properties.entrySet()) - { - String key = (String) entry.getKey(); - String value = (String) entry.getValue(); - if (DEFAULT_PROPERTY.equals(key)) - continue; - - String hostString = StringUtils.remove(key, '/'); - try - { - InetAddressAndPort host = InetAddressAndPort.getByName(hostString); - if (host.equals(broadcastAddress)) - { - local = makeLocation(value); - break; - } - } - catch (UnknownHostException e) - { - throw new ConfigurationException("Unknown host " + hostString, e); - } - - } - - if (local == null) - { - if (defaultLocation == null) - { - throw new ConfigurationException(String.format("Snitch definitions at %s do not define a location for " + - "this node's broadcast address %s, nor does it provides a default", - SNITCH_PROPERTIES_FILENAME, broadcastAddress)); - } - else - { - logger.debug("Broadcast address {} was not present in snitch config, using default location {}. " + - "This only matters on first boot, before registering with the cluster metadata service", - broadcastAddress, defaultLocation); - return defaultLocation; - } - } - - logger.debug("Loaded location {} for broadcast address {} from property file. " + - "This only matters on first boot, before registering with the cluster metadata service", - local, broadcastAddress); - return local; + return local.datacenter; } } diff --git a/src/java/org/apache/cassandra/locator/RackDCFileLocationProvider.java b/src/java/org/apache/cassandra/locator/RackDCFileLocationProvider.java new file mode 100644 index 000000000000..6443de78d464 --- /dev/null +++ b/src/java/org/apache/cassandra/locator/RackDCFileLocationProvider.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.locator; + +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.tcm.membership.Location; + +/** + * Emulates the previous behaviour of GossipingPropertyFileSnitch; reads cassandra-rackdc.properties + * to identify the local datacenter and rack for one time use during node registration. + * + * Based on a properties file in the following format: + * + * dc=DC1 + * rack=RAC1 + */ +public class RackDCFileLocationProvider implements InitialLocationProvider +{ + private final Location local; + private static final Location DEFAULT = new Location("UNKNOWN_DC", "UNKNOWN_RACK"); + + /** + * Used via reflection by DatabaseDescriptor::createInitialLocationProvider + */ + public RackDCFileLocationProvider() + { + this(loadConfiguration()); + } + + /** + * Used in legacy compatibility mode by GossipingPropertyFileSnitch + * @param properties + */ + RackDCFileLocationProvider(SnitchProperties properties) + { + local = new Location(properties.get("dc", DEFAULT.datacenter).trim(), + properties.get("rack", DEFAULT.rack).trim()); + } + + @Override + public Location initialLocation() + { + return local; + } + + public static SnitchProperties loadConfiguration() throws ConfigurationException + { + final SnitchProperties properties = new SnitchProperties(); + if (!properties.contains("dc") || !properties.contains("rack")) + throw new ConfigurationException("DC or rack not found in snitch properties, check your configuration in: " + SnitchProperties.RACKDC_PROPERTY_FILENAME); + + return properties; + } + + +} diff --git a/src/java/org/apache/cassandra/locator/RackInferringSnitch.java b/src/java/org/apache/cassandra/locator/RackInferringSnitch.java index cc47e0e8fb05..88487c333beb 100644 --- a/src/java/org/apache/cassandra/locator/RackInferringSnitch.java +++ b/src/java/org/apache/cassandra/locator/RackInferringSnitch.java @@ -17,9 +17,7 @@ */ package org.apache.cassandra.locator; -import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.membership.Location; -import org.apache.cassandra.tcm.membership.NodeId; import org.apache.cassandra.utils.FBUtilities; /** @@ -29,7 +27,9 @@ * Local location is derived from (broadcast) ip address and added to ClusterMetadata during node * registration. Every member of the cluster is required to do this, hence remote peers' Location * can always be retrieved, consistently. + * @deprecated See CASSANDRA-19488 */ +@Deprecated(since = "CEP-21") public class RackInferringSnitch extends AbstractNetworkTopologySnitch { final Location local; @@ -37,31 +37,25 @@ public class RackInferringSnitch extends AbstractNetworkTopologySnitch public RackInferringSnitch() { InetAddressAndPort localAddress = FBUtilities.getBroadcastAddressAndPort(); - local = new Location(Integer.toString(localAddress.getAddress().getAddress()[1] & 0xFF, 10), - Integer.toString(localAddress.getAddress().getAddress()[2] & 0xFF, 10)); + local = inferLocation(localAddress); } - public String getDatacenter(InetAddressAndPort endpoint) + public static Location inferLocation(InetAddressAndPort address) { - if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort())) - return local.datacenter; + return new Location(Integer.toString(address.getAddress().getAddress()[1] & 0xFF, 10), + Integer.toString(address.getAddress().getAddress()[2] & 0xFF, 10)); - ClusterMetadata metadata = ClusterMetadata.current(); - NodeId nodeId = metadata.directory.peerId(endpoint); - if (nodeId == null) - return Integer.toString(endpoint.getAddress().getAddress()[1] & 0xFF, 10); - return metadata.directory.location(nodeId).datacenter; } + @Override public String getRack(InetAddressAndPort endpoint) { - if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort())) - return local.rack; + return inferLocation(endpoint).rack; + } - ClusterMetadata metadata = ClusterMetadata.current(); - NodeId nodeId = metadata.directory.peerId(endpoint); - if (nodeId == null) - return Integer.toString(endpoint.getAddress().getAddress()[2] & 0xFF, 10); - return metadata.directory.location(nodeId).rack; + @Override + public String getDatacenter(InetAddressAndPort endpoint) + { + return inferLocation(endpoint).datacenter; } } diff --git a/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java b/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java index 4ff726c4654d..346aabe44d8d 100644 --- a/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java +++ b/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java @@ -40,14 +40,12 @@ public class ReconnectableSnitchHelper implements IEndpointStateChangeSubscriber { private static final Logger logger = LoggerFactory.getLogger(ReconnectableSnitchHelper.class); - private final IEndpointSnitch snitch; - private final String localDc; + private final Locator locator; private final boolean preferLocal; - public ReconnectableSnitchHelper(IEndpointSnitch snitch, String localDc, boolean preferLocal) + public ReconnectableSnitchHelper(Locator locator, boolean preferLocal) { - this.snitch = snitch; - this.localDc = localDc; + this.locator = locator; this.preferLocal = preferLocal; } @@ -55,7 +53,7 @@ private void reconnect(InetAddressAndPort publicAddress, VersionedValue localAdd { try { - reconnect(publicAddress, InetAddressAndPort.getByName(localAddressValue.value), snitch, localDc); + reconnect(publicAddress, InetAddressAndPort.getByName(localAddressValue.value), locator); } catch (UnknownHostException e) { @@ -64,7 +62,7 @@ private void reconnect(InetAddressAndPort publicAddress, VersionedValue localAdd } @VisibleForTesting - static void reconnect(InetAddressAndPort publicAddress, InetAddressAndPort localAddress, IEndpointSnitch snitch, String localDc) + static void reconnect(InetAddressAndPort publicAddress, InetAddressAndPort localAddress, Locator locator) { final OutboundConnectionSettings settings = new OutboundConnectionSettings(publicAddress, localAddress).withDefaults(ConnectionCategory.MESSAGING); if (!settings.authenticator().authenticate(settings.to.getAddress(), settings.to.getPort(), null, OUTBOUND_PRECONNECT)) @@ -73,18 +71,13 @@ static void reconnect(InetAddressAndPort publicAddress, InetAddressAndPort local return; } - if (snitch.getDatacenter(publicAddress).equals(localDc)) + if (locator.local().sameDatacenter(locator.location(publicAddress))) { MessagingService.instance().maybeReconnectWithNewIp(publicAddress, localAddress); logger.debug("Initiated reconnect to an Internal IP {} for the {}", localAddress, publicAddress); } } - public void beforeChange(InetAddressAndPort endpoint, EndpointState currentState, ApplicationState newStateKey, VersionedValue newValue) - { - // no-op - } - public void onJoin(InetAddressAndPort endpoint, EndpointState epState) { if (preferLocal && !Gossiper.instance.isDeadState(epState)) @@ -127,19 +120,4 @@ public void onAlive(InetAddressAndPort endpoint, EndpointState state) if (preferLocal && internalIP != null) reconnect(endpoint, internalIPAndPorts != null ? internalIPAndPorts : internalIP); } - - public void onDead(InetAddressAndPort endpoint, EndpointState state) - { - // do nothing. - } - - public void onRemove(InetAddressAndPort endpoint) - { - // do nothing. - } - - public void onRestart(InetAddressAndPort endpoint, EndpointState state) - { - // do nothing. - } } diff --git a/src/java/org/apache/cassandra/locator/ReplicaLayout.java b/src/java/org/apache/cassandra/locator/ReplicaLayout.java index ed5666a26c6b..751737fafb13 100644 --- a/src/java/org/apache/cassandra/locator/ReplicaLayout.java +++ b/src/java/org/apache/cassandra/locator/ReplicaLayout.java @@ -360,7 +360,7 @@ static ReplicaLayout.ForTokenRead forTokenReadLiveSorted(ClusterMetadata metadat EndpointsForToken replicas = keyspace.getMetadata().params.replication.isLocal() ? forLocalStrategyToken(metadata, replicationStrategy, token) : forNonLocalStrategyTokenRead(metadata, keyspace.getMetadata(), token); - replicas = DatabaseDescriptor.getEndpointSnitch().sortedByProximity(FBUtilities.getBroadcastAddressAndPort(), replicas); + replicas = DatabaseDescriptor.getNodeProximity().sortedByProximity(FBUtilities.getBroadcastAddressAndPort(), replicas); replicas = replicas.filter(FailureDetector.isReplicaAlive); return new ReplicaLayout.ForTokenRead(replicationStrategy, replicas); } @@ -376,7 +376,7 @@ static ReplicaLayout.ForRangeRead forRangeReadLiveSorted(ClusterMetadata metadat ? forLocalStrategyRange(metadata, replicationStrategy, range) : forNonLocalStategyRangeRead(metadata, keyspace.getMetadata(), range); - replicas = DatabaseDescriptor.getEndpointSnitch().sortedByProximity(FBUtilities.getBroadcastAddressAndPort(), replicas); + replicas = DatabaseDescriptor.getNodeProximity().sortedByProximity(FBUtilities.getBroadcastAddressAndPort(), replicas); replicas = replicas.filter(FailureDetector.isReplicaAlive); return new ReplicaLayout.ForRangeRead(replicationStrategy, range, replicas); } diff --git a/src/java/org/apache/cassandra/locator/ReplicaPlans.java b/src/java/org/apache/cassandra/locator/ReplicaPlans.java index 51830ffa4b78..62b7f0ca2d44 100644 --- a/src/java/org/apache/cassandra/locator/ReplicaPlans.java +++ b/src/java/org/apache/cassandra/locator/ReplicaPlans.java @@ -68,6 +68,8 @@ import org.apache.cassandra.service.reads.SpeculativeRetryPolicy; import org.apache.cassandra.tcm.Epoch; +import org.apache.cassandra.tcm.membership.Directory; +import org.apache.cassandra.tcm.membership.Location; import org.apache.cassandra.tcm.membership.NodeState; import org.apache.cassandra.utils.FBUtilities; @@ -98,7 +100,7 @@ public class ReplicaPlans logger.warn("System property {} was set to {} but must be 1 or 2. Running with {}", CassandraRelevantProperties.REQUIRED_BATCHLOG_REPLICA_COUNT.getKey(), batchlogReplicaCount, REQUIRED_BATCHLOG_REPLICA_COUNT); } - public static boolean isSufficientLiveReplicasForRead(AbstractReplicationStrategy replicationStrategy, ConsistencyLevel consistencyLevel, Endpoints<?> liveReplicas) + public static boolean isSufficientLiveReplicasForRead(Locator locator, AbstractReplicationStrategy replicationStrategy, ConsistencyLevel consistencyLevel, Endpoints<?> liveReplicas) { switch (consistencyLevel) { @@ -114,7 +116,7 @@ public static boolean isSufficientLiveReplicasForRead(AbstractReplicationStrateg { int fullCount = 0; Collection<String> dcs = ((NetworkTopologyStrategy) replicationStrategy).getDatacenters(); - for (ObjectObjectCursor<String, Replicas.ReplicaCount> entry : countPerDc(dcs, liveReplicas)) + for (ObjectObjectCursor<String, Replicas.ReplicaCount> entry : countPerDc(locator, dcs, liveReplicas)) { Replicas.ReplicaCount count = entry.value; if (!count.hasAtleast(localQuorumFor(replicationStrategy, entry.key), 0)) @@ -130,17 +132,17 @@ public static boolean isSufficientLiveReplicasForRead(AbstractReplicationStrateg } } - static void assureSufficientLiveReplicasForRead(AbstractReplicationStrategy replicationStrategy, ConsistencyLevel consistencyLevel, Endpoints<?> liveReplicas) throws UnavailableException + static void assureSufficientLiveReplicasForRead(Locator locator, AbstractReplicationStrategy replicationStrategy, ConsistencyLevel consistencyLevel, Endpoints<?> liveReplicas) throws UnavailableException { - assureSufficientLiveReplicas(replicationStrategy, consistencyLevel, liveReplicas, consistencyLevel.blockFor(replicationStrategy), 1); + assureSufficientLiveReplicas(locator, replicationStrategy, consistencyLevel, liveReplicas, consistencyLevel.blockFor(replicationStrategy), 1); } - static void assureSufficientLiveReplicasForWrite(AbstractReplicationStrategy replicationStrategy, ConsistencyLevel consistencyLevel, Endpoints<?> allLive, Endpoints<?> pendingWithDown) throws UnavailableException + static void assureSufficientLiveReplicasForWrite(Locator locator, AbstractReplicationStrategy replicationStrategy, ConsistencyLevel consistencyLevel, Endpoints<?> allLive, Endpoints<?> pendingWithDown) throws UnavailableException { - assureSufficientLiveReplicas(replicationStrategy, consistencyLevel, allLive, consistencyLevel.blockForWrite(replicationStrategy, pendingWithDown), 0); + assureSufficientLiveReplicas(locator, replicationStrategy, consistencyLevel, allLive, consistencyLevel.blockForWrite(replicationStrategy, pendingWithDown), 0); } - static void assureSufficientLiveReplicas(AbstractReplicationStrategy replicationStrategy, ConsistencyLevel consistencyLevel, Endpoints<?> allLive, int blockFor, int blockForFullReplicas) throws UnavailableException + static void assureSufficientLiveReplicas(Locator locator, AbstractReplicationStrategy replicationStrategy, ConsistencyLevel consistencyLevel, Endpoints<?> allLive, int blockFor, int blockForFullReplicas) throws UnavailableException { switch (consistencyLevel) { @@ -172,7 +174,7 @@ static void assureSufficientLiveReplicas(AbstractReplicationStrategy replication int total = 0; int totalFull = 0; Collection<String> dcs = ((NetworkTopologyStrategy) replicationStrategy).getDatacenters(); - for (ObjectObjectCursor<String, Replicas.ReplicaCount> entry : countPerDc(dcs, allLive)) + for (ObjectObjectCursor<String, Replicas.ReplicaCount> entry : countPerDc(locator, dcs, allLive)) { int dcBlockFor = localQuorumFor(replicationStrategy, entry.key); Replicas.ReplicaCount dcCount = entry.value; @@ -226,7 +228,7 @@ public static ReplicaPlan.ForWrite forSingleReplicaWrite(ClusterMetadata metadat public static Replica findCounterLeaderReplica(ClusterMetadata metadata, String keyspaceName, DecoratedKey key, String localDataCenter, ConsistencyLevel cl) throws UnavailableException { Keyspace keyspace = Keyspace.open(keyspaceName); - IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch(); + NodeProximity proximity = DatabaseDescriptor.getNodeProximity(); AbstractReplicationStrategy replicationStrategy = keyspace.getReplicationStrategy(); EndpointsForToken replicas = metadata.placements.get(keyspace.getMetadata().params.replication).reads.forToken(key.getToken()).get(); @@ -243,7 +245,7 @@ public static Replica findCounterLeaderReplica(ClusterMetadata metadata, String List<Replica> localReplicas = new ArrayList<>(replicas.size()); for (Replica replica : replicas) - if (snitch.getDatacenter(replica).equals(localDataCenter)) + if (metadata.locator.location(replica.endpoint()).datacenter.equals(localDataCenter)) localReplicas.add(replica); if (localReplicas.isEmpty()) @@ -252,8 +254,8 @@ public static Replica findCounterLeaderReplica(ClusterMetadata metadata, String if (cl.isDatacenterLocal()) throw UnavailableException.create(cl, cl.blockFor(replicationStrategy), 0); - // No endpoint in local DC, pick the closest endpoint according to the snitch - replicas = snitch.sortedByProximity(FBUtilities.getBroadcastAddressAndPort(), replicas); + // No endpoint in local DC, pick the closest endpoint according to the configured proximity measures + replicas = proximity.sortedByProximity(FBUtilities.getBroadcastAddressAndPort(), replicas); return replicas.get(0); } @@ -296,16 +298,17 @@ public static ReplicaPlan.ForWrite forBatchlogWrite(boolean isAny) throws Unavai private static ReplicaLayout.ForTokenWrite liveAndDownForBatchlogWrite(Token token, ClusterMetadata metadata, boolean isAny) { - IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch(); - Multimap<String, InetAddressAndPort> localEndpoints = HashMultimap.create(metadata.directory.allDatacenterRacks() - .get(snitch.getLocalDatacenter())); + Directory directory = metadata.directory; + Location local = metadata.locator.local(); + Multimap<String, InetAddressAndPort> localEndpoints = HashMultimap.create(directory.allDatacenterRacks() + .get(local.datacenter)); // Replicas are picked manually: // - replicas should be alive according to the failure detector // - replicas should be in the local datacenter // - choose min(2, number of qualifying candiates above) // - allow the local node to be the only replica only if it's a single-node DC Collection<InetAddressAndPort> chosenEndpoints = filterBatchlogEndpoints(false, - snitch.getLocalRack(), + local.rack, localEndpoints, Collections::shuffle, (r) -> FailureDetector.isEndpointAlive.test(r) && metadata.directory.peerState(r) == NodeState.JOINED, @@ -343,7 +346,7 @@ public static ReplicaPlan.ForWrite forBatchlogWrite(ClusterMetadata metadata, bo AbstractReplicationStrategy replicationStrategy = liveAndDown.replicationStrategy(); EndpointsForToken contacts = writeAll.select(consistencyLevel, liveAndDown, liveAndDown); - assureSufficientLiveReplicasForWrite(replicationStrategy, consistencyLevel, liveAndDown.all(), liveAndDown.pending()); + assureSufficientLiveReplicasForWrite(metadata.locator, replicationStrategy, consistencyLevel, liveAndDown.all(), liveAndDown.pending()); return new ReplicaPlan.ForWrite(systemKeyspace, replicationStrategy, consistencyLevel, @@ -506,7 +509,7 @@ public static Collection<InetAddressAndPort> filterBatchlogEndpointsDynamic(bool if (racks.isEmpty()) racks.addAll(validated.keySet()); - String rack = DatabaseDescriptor.getEndpointSnitch().getRack(endpoint); + String rack = DatabaseDescriptor.getLocator().location(endpoint).rack; if (!racks.remove(rack)) continue; if (result.contains(endpoint)) @@ -523,7 +526,7 @@ public static Collection<InetAddressAndPort> filterBatchlogEndpointsDynamic(bool public static List<InetAddressAndPort> sortByProximity(Collection<InetAddressAndPort> endpoints) { EndpointsForRange endpointsForRange = SystemReplicas.getSystemReplicas(endpoints); - return DatabaseDescriptor.getEndpointSnitch() + return DatabaseDescriptor.getNodeProximity() .sortedByProximity(FBUtilities.getBroadcastAddressAndPort(), endpointsForRange) .endpointList(); } @@ -538,7 +541,7 @@ public static ReplicaPlan.ForWrite forReadRepair(ReplicaPlan<?, ?> forRead, Clus ReplicaLayout.ForTokenWrite live = liveAndDown.filter(isAlive); EndpointsForToken contacts = selector.select(consistencyLevel, liveAndDown, live); - assureSufficientLiveReplicasForWrite(replicationStrategy, consistencyLevel, live.all(), liveAndDown.pending()); + assureSufficientLiveReplicasForWrite(metadata.locator, replicationStrategy, consistencyLevel, live.all(), liveAndDown.pending()); return new ReplicaPlan.ForWrite(keyspace, replicationStrategy, consistencyLevel, @@ -588,7 +591,7 @@ public static ReplicaPlan.ForWrite forWrite(ClusterMetadata metadata, AbstractReplicationStrategy replicationStrategy = liveAndDown.replicationStrategy(); EndpointsForToken contacts = selector.select(consistencyLevel, liveAndDown, live); - assureSufficientLiveReplicasForWrite(replicationStrategy, consistencyLevel, live.all(), liveAndDown.pending()); + assureSufficientLiveReplicasForWrite(metadata.locator, replicationStrategy, consistencyLevel, live.all(), liveAndDown.pending()); return new ReplicaPlan.ForWrite(keyspace, replicationStrategy, @@ -655,14 +658,15 @@ E select(ConsistencyLevel consistencyLevel, L liveAndDown, L live) * soft-ensure that we reach QUORUM in all DCs we are able to, by writing to every node; * even if we don't wait for ACK, we have in both cases sent sufficient messages. */ - ObjectIntHashMap<String> requiredPerDc = eachQuorumForWrite(liveAndDown.replicationStrategy(), liveAndDown.pending()); - addToCountPerDc(requiredPerDc, live.natural().filter(Replica::isFull), -1); - addToCountPerDc(requiredPerDc, live.pending(), -1); + Locator locator = ClusterMetadata.current().locator; + ObjectIntHashMap<String> requiredPerDc = eachQuorumForWrite(locator, liveAndDown.replicationStrategy(), liveAndDown.pending()); + addToCountPerDc(locator, requiredPerDc, live.natural().filter(Replica::isFull), -1); + addToCountPerDc(locator, requiredPerDc, live.pending(), -1); - IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch(); for (Replica replica : filter(live.natural(), Replica::isTransient)) { - String dc = snitch.getDatacenter(replica); + + String dc = locator.location(replica.endpoint()).datacenter; if (requiredPerDc.addTo(dc, -1) >= 0) contacts.add(replica); } @@ -713,12 +717,12 @@ E select(ConsistencyLevel consistencyLevel, L liveAndDown, L live) } else { - ObjectIntHashMap<String> requiredPerDc = eachQuorumForWrite(liveAndDown.replicationStrategy(), liveAndDown.pending()); - addToCountPerDc(requiredPerDc, contacts.snapshot(), -1); - IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch(); + Locator locator = ClusterMetadata.current().locator; + ObjectIntHashMap<String> requiredPerDc = eachQuorumForWrite(locator, liveAndDown.replicationStrategy(), liveAndDown.pending()); + addToCountPerDc(locator, requiredPerDc, contacts.snapshot(), -1); for (Replica replica : filter(live.all(), r -> !contacts.contains(r))) { - String dc = snitch.getDatacenter(replica); + String dc = locator.location(replica.endpoint()).datacenter; if (requiredPerDc.addTo(dc, -1) >= 0) contacts.add(replica); } @@ -796,18 +800,16 @@ private static <E extends Endpoints<E>> E candidatesForRead(Keyspace keyspace, return indexQueryPlan != null ? IndexStatusManager.instance.filterForQuery(replicas, keyspace, indexQueryPlan, consistencyLevel) : replicas; } - private static <E extends Endpoints<E>> E contactForEachQuorumRead(NetworkTopologyStrategy replicationStrategy, E candidates) + private static <E extends Endpoints<E>> E contactForEachQuorumRead(Locator locator, NetworkTopologyStrategy replicationStrategy, E candidates) { ObjectIntHashMap<String> perDc = eachQuorumForRead(replicationStrategy); - - final IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch(); return candidates.filter(replica -> { - String dc = snitch.getDatacenter(replica); + String dc = locator.location(replica.endpoint()).datacenter; return perDc.addTo(dc, -1) >= 0; }); } - private static <E extends Endpoints<E>> E contactForRead(AbstractReplicationStrategy replicationStrategy, ConsistencyLevel consistencyLevel, boolean alwaysSpeculate, E candidates) + private static <E extends Endpoints<E>> E contactForRead(Locator locator, AbstractReplicationStrategy replicationStrategy, ConsistencyLevel consistencyLevel, boolean alwaysSpeculate, E candidates) { /* * If we are doing an each quorum query, we have to make sure that the endpoints we select @@ -819,7 +821,7 @@ private static <E extends Endpoints<E>> E contactForRead(AbstractReplicationStra * TODO: this is still very inconistently managed between {LOCAL,EACH}_QUORUM and other consistency levels - should address this in a follow-up */ if (consistencyLevel == EACH_QUORUM && replicationStrategy instanceof NetworkTopologyStrategy) - return contactForEachQuorumRead((NetworkTopologyStrategy) replicationStrategy, candidates); + return contactForEachQuorumRead(locator, (NetworkTopologyStrategy) replicationStrategy, candidates); int count = consistencyLevel.blockFor(replicationStrategy) + (alwaysSpeculate ? 1 : 0); return candidates.subList(0, Math.min(count, candidates.size())); @@ -903,10 +905,10 @@ private static ReplicaPlan.ForTokenRead forRead(ClusterMetadata metadata, Keyspa AbstractReplicationStrategy replicationStrategy = keyspace.getReplicationStrategy(); ReplicaLayout.ForTokenRead forTokenRead = ReplicaLayout.forTokenReadLiveSorted(metadata, keyspace, replicationStrategy, token); EndpointsForToken candidates = candidatesForRead(keyspace, indexQueryPlan, consistencyLevel, forTokenRead.natural()); - EndpointsForToken contacts = contactForRead(replicationStrategy, consistencyLevel, retry.equals(AlwaysSpeculativeRetryPolicy.INSTANCE), candidates); + EndpointsForToken contacts = contactForRead(metadata.locator, replicationStrategy, consistencyLevel, retry.equals(AlwaysSpeculativeRetryPolicy.INSTANCE), candidates); if (throwOnInsufficientLiveReplicas) - assureSufficientLiveReplicasForRead(replicationStrategy, consistencyLevel, contacts); + assureSufficientLiveReplicasForRead(metadata.locator, replicationStrategy, consistencyLevel, contacts); return new ReplicaPlan.ForTokenRead(keyspace, replicationStrategy, consistencyLevel, candidates, contacts, (newClusterMetadata) -> forRead(newClusterMetadata, keyspace, token, indexQueryPlan, consistencyLevel, retry, false), @@ -941,10 +943,10 @@ public static ReplicaPlan.ForRangeRead forRangeRead(ClusterMetadata metadata, AbstractReplicationStrategy replicationStrategy = keyspace.getReplicationStrategy(); ReplicaLayout.ForRangeRead forRangeRead = ReplicaLayout.forRangeReadLiveSorted(metadata, keyspace, replicationStrategy, range); EndpointsForRange candidates = candidatesForRead(keyspace, indexQueryPlan, consistencyLevel, forRangeRead.natural()); - EndpointsForRange contacts = contactForRead(replicationStrategy, consistencyLevel, false, candidates); + EndpointsForRange contacts = contactForRead(metadata.locator, replicationStrategy, consistencyLevel, false, candidates); if (throwOnInsufficientLiveReplicas) - assureSufficientLiveReplicasForRead(replicationStrategy, consistencyLevel, contacts); + assureSufficientLiveReplicasForRead(metadata.locator, replicationStrategy, consistencyLevel, contacts); return new ReplicaPlan.ForRangeRead(keyspace, replicationStrategy, @@ -986,7 +988,8 @@ public static ReplicaPlan.ForRangeRead forFullRangeRead(Keyspace keyspace, /** * Take two range read plans for adjacent ranges, and check if it is OK (and worthwhile) to combine them into a single plan */ - public static ReplicaPlan.ForRangeRead maybeMerge(Keyspace keyspace, + public static ReplicaPlan.ForRangeRead maybeMerge(ClusterMetadata metadata, + Keyspace keyspace, ConsistencyLevel consistencyLevel, ReplicaPlan.ForRangeRead left, ReplicaPlan.ForRangeRead right) @@ -998,16 +1001,16 @@ public static ReplicaPlan.ForRangeRead maybeMerge(Keyspace keyspace, EndpointsForRange mergedCandidates = left.readCandidates().keep(right.readCandidates().endpoints()); AbstractReplicationStrategy replicationStrategy = keyspace.getReplicationStrategy(); - EndpointsForRange contacts = contactForRead(replicationStrategy, consistencyLevel, false, mergedCandidates); + EndpointsForRange contacts = contactForRead(metadata.locator, replicationStrategy, consistencyLevel, false, mergedCandidates); // Estimate whether merging will be a win or not - if (!DatabaseDescriptor.getEndpointSnitch().isWorthMergingForRangeQuery(contacts, left.contacts(), right.contacts())) + if (!DatabaseDescriptor.getNodeProximity().isWorthMergingForRangeQuery(contacts, left.contacts(), right.contacts())) return null; AbstractBounds<PartitionPosition> newRange = left.range().withNewRight(right.range().right); // Check if there are enough shared endpoints for the merge to be possible. - if (!isSufficientLiveReplicasForRead(replicationStrategy, consistencyLevel, mergedCandidates)) + if (!isSufficientLiveReplicasForRead(metadata.locator, replicationStrategy, consistencyLevel, mergedCandidates)) return null; int newVnodeCount = left.vnodeCount() + right.vnodeCount(); diff --git a/src/java/org/apache/cassandra/locator/Replicas.java b/src/java/org/apache/cassandra/locator/Replicas.java index c53815c56351..e280e9980de0 100644 --- a/src/java/org/apache/cassandra/locator/Replicas.java +++ b/src/java/org/apache/cassandra/locator/Replicas.java @@ -25,8 +25,8 @@ import com.carrotsearch.hppc.ObjectIntHashMap; import com.carrotsearch.hppc.ObjectObjectHashMap; + import com.google.common.collect.Iterables; -import org.apache.cassandra.config.DatabaseDescriptor; import static com.google.common.collect.Iterables.all; @@ -88,16 +88,15 @@ public static ReplicaCount countInOurDc(ReplicaCollection<?> replicas) /** * count the number of full and transient replicas, separately, for each DC */ - public static ObjectObjectHashMap<String, ReplicaCount> countPerDc(Collection<String> dataCenters, Iterable<Replica> replicas) + public static ObjectObjectHashMap<String, ReplicaCount> countPerDc(Locator locator, Collection<String> dataCenters, Iterable<Replica> replicas) { ObjectObjectHashMap<String, ReplicaCount> perDc = new ObjectObjectHashMap<>(dataCenters.size()); for (String dc: dataCenters) perDc.put(dc, new ReplicaCount()); - IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch(); for (Replica replica : replicas) { - String dc = snitch.getDatacenter(replica); + String dc = locator.location(replica.endpoint()).datacenter; perDc.get(dc).increment(replica); } return perDc; @@ -106,12 +105,11 @@ public static ObjectObjectHashMap<String, ReplicaCount> countPerDc(Collection<St /** * increment each of the map's DC entries for each matching replica provided */ - public static void addToCountPerDc(ObjectIntHashMap<String> perDc, Iterable<Replica> replicas, int add) + public static void addToCountPerDc(Locator locator, ObjectIntHashMap<String> perDc, Iterable<Replica> replicas, int add) { - IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch(); for (Replica replica : replicas) { - String dc = snitch.getDatacenter(replica); + String dc = locator.location(replica.endpoint()).datacenter; perDc.addTo(dc, add); } } diff --git a/src/java/org/apache/cassandra/locator/SimpleLocationProvider.java b/src/java/org/apache/cassandra/locator/SimpleLocationProvider.java new file mode 100644 index 000000000000..3217b2d16b0a --- /dev/null +++ b/src/java/org/apache/cassandra/locator/SimpleLocationProvider.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.locator; + +import org.apache.cassandra.tcm.membership.Location; + +public class SimpleLocationProvider implements InitialLocationProvider +{ + public static final Location LOCATION = new Location("datacenter1", "rack1"); + + @Override + public Location initialLocation() + { + return LOCATION; + } +} diff --git a/src/java/org/apache/cassandra/locator/SimpleSnitch.java b/src/java/org/apache/cassandra/locator/SimpleSnitch.java index f9bf21525847..e06316fa2666 100644 --- a/src/java/org/apache/cassandra/locator/SimpleSnitch.java +++ b/src/java/org/apache/cassandra/locator/SimpleSnitch.java @@ -21,34 +21,41 @@ * A simple endpoint snitch implementation that treats Strategy order as proximity, * allowing non-read-repaired reads to prefer a single endpoint, which improves * cache locality. + * @deprecated See CASSANDRA-19488 */ +@Deprecated(since = "CEP-21") public class SimpleSnitch extends AbstractEndpointSnitch { - public static final String DATA_CENTER_NAME = "datacenter1"; - public static final String RACK_NAME = "rack1"; + private static final NodeProximity sorter = new NoOpProximity(); + private static final SimpleLocationProvider provider = new SimpleLocationProvider(); - public String getRack(InetAddressAndPort endpoint) + @Override + public String getLocalRack() { - return RACK_NAME; + return provider.initialLocation().rack; } - public String getDatacenter(InetAddressAndPort endpoint) + @Override + public String getLocalDatacenter() { - return DATA_CENTER_NAME; + return provider.initialLocation().datacenter; } @Override - public <C extends ReplicaCollection<? extends C>> C sortedByProximity(final InetAddressAndPort address, C unsortedAddress) + public <C extends ReplicaCollection<? extends C>> C sortedByProximity(InetAddressAndPort address, C unsortedAddress) { - // Optimization to avoid walking the list - return unsortedAddress; + return sorter.sortedByProximity(address, unsortedAddress); } @Override public int compareEndpoints(InetAddressAndPort target, Replica r1, Replica r2) { - // Making all endpoints equal ensures we won't change the original ordering (since - // Collections.sort is guaranteed to be stable) - return 0; + return sorter.compareEndpoints(target, r1, r2); + } + + @Override + public boolean isWorthMergingForRangeQuery(ReplicaCollection<?> merged, ReplicaCollection<?> l1, ReplicaCollection<?> l2) + { + return sorter.isWorthMergingForRangeQuery(merged, l1, l2); } } diff --git a/src/java/org/apache/cassandra/locator/SnitchAdapter.java b/src/java/org/apache/cassandra/locator/SnitchAdapter.java new file mode 100644 index 000000000000..1a32dd64970c --- /dev/null +++ b/src/java/org/apache/cassandra/locator/SnitchAdapter.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.locator; + +import java.util.HashSet; +import java.util.Set; + +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.membership.Location; + +public class SnitchAdapter implements InitialLocationProvider, NodeProximity, NodeAddressConfig +{ + public final IEndpointSnitch snitch; + + public SnitchAdapter(IEndpointSnitch snitch) + { + this.snitch = snitch; + } + + @Override + public Location initialLocation() + { + return new Location(snitch.getLocalDatacenter(), snitch.getLocalRack()); + } + + @Override + public void validate(ClusterMetadata metadata) + { + Set<String> datacenters = metadata.directory.allDatacenterRacks().keySet(); + Set<String> racks = new HashSet<>(); + for (String dc : datacenters) + racks.addAll(metadata.directory.datacenterRacks(dc).keySet()); + if (!snitch.validate(datacenters, racks)) + throw new ConfigurationException("Initial location provider rejected registration location, " + + "please check the system log for errors"); + } + + @Override + public <C extends ReplicaCollection<? extends C>> C sortedByProximity(InetAddressAndPort address, C addresses) + { + return snitch.sortedByProximity(address, addresses); + } + + @Override + public int compareEndpoints(InetAddressAndPort target, Replica r1, Replica r2) + { + return snitch.compareEndpoints(target, r1, r2); + } + + @Override + public boolean isWorthMergingForRangeQuery(ReplicaCollection<?> merged, ReplicaCollection<?> l1, ReplicaCollection<?> l2) + { + return snitch.isWorthMergingForRangeQuery(merged, l1, l2); + } + + @Override + public void configureAddresses() + { + snitch.configureAddresses(); + } + + @Override + public boolean preferLocalConnections() + { + return snitch.preferLocalConnections(); + } +} diff --git a/src/java/org/apache/cassandra/locator/SnitchUtils.java b/src/java/org/apache/cassandra/locator/SnitchUtils.java index 65c1f28c0641..11d9bfa68f36 100644 --- a/src/java/org/apache/cassandra/locator/SnitchUtils.java +++ b/src/java/org/apache/cassandra/locator/SnitchUtils.java @@ -18,6 +18,7 @@ package org.apache.cassandra.locator; +import org.apache.cassandra.tcm.membership.Location; import org.apache.cassandra.utils.Pair; import static java.lang.String.format; @@ -26,7 +27,13 @@ public class SnitchUtils { private SnitchUtils() {} - static Pair<String, String> parseDcAndRack(String response, String dcSuffix) + public static Location parseLocation(String response, String dcSuffix) + { + Pair<String, String> pair = parseDcAndRack(response, dcSuffix); + return new Location(pair.left, pair.right); + } + + public static Pair<String, String> parseDcAndRack(String response, String dcSuffix) { String[] splits = response.split("/"); String az = splits[splits.length - 1]; diff --git a/src/java/org/apache/cassandra/locator/TopologyFileLocationProvider.java b/src/java/org/apache/cassandra/locator/TopologyFileLocationProvider.java new file mode 100644 index 000000000000..9ee4b696f59f --- /dev/null +++ b/src/java/org/apache/cassandra/locator/TopologyFileLocationProvider.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.locator; + +import java.io.InputStream; +import java.net.UnknownHostException; +import java.util.Map; +import java.util.Properties; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.commons.lang3.StringUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.tcm.membership.Location; +import org.apache.cassandra.utils.FBUtilities; + +/** + * Emulates the previous behaviour of PropertyFileSnitch; reads cassandra-topology.properties + * to identify the local datacenter and rack for one time use during node registration. + * + * Based on a properties file in the following format: + * + * 10.0.0.13=DC1:RAC2 + * 10.21.119.14=DC3:RAC2 + * 10.20.114.15=DC2:RAC2 + * default=DC1:r1 + * + * Post CEP-21, only the local rack and DC are loaded from file. Each peer in the cluster is required to register + * itself with the Cluster Metadata Service and provide its Location (Rack + DC) before joining. During upgrades, + * this is done automatically with location derived from gossip state (ultimately from system.local). + * Once registered, the Rack & DC should not be changed but currently the only safeguards against this are the + * StartupChecks which validate the snitch against system.local. + */ +public class TopologyFileLocationProvider implements InitialLocationProvider +{ + private static final Logger logger = LoggerFactory.getLogger(TopologyFileLocationProvider.class); + + public static final String PROPERTIES_FILENAME = "cassandra-topology.properties"; + // All the defaults + private static final String DEFAULT_PROPERTY = "default"; + @VisibleForTesting + public static final String DEFAULT_DC = "default"; + @VisibleForTesting + public static final String DEFAULT_RACK = "default"; + + // Used only during initialization of a new node. This provides the location it will register in cluster metadata + private final Location local; + + public TopologyFileLocationProvider() + { + local = loadConfiguration(); + } + + @Override + public Location initialLocation() + { + return local; + } + + private Location makeLocation(String value) + { + if (value == null || value.isEmpty()) + return null; + + String[] parts = value.split(":"); + if (parts.length < 2) + { + return new Location(DEFAULT_DC, DEFAULT_RACK); + } + else + { + return new Location(parts[0].trim(), parts[1].trim()); + } + } + + private Location loadConfiguration() throws ConfigurationException + { + Properties properties = new Properties(); + try (InputStream stream = getClass().getClassLoader().getResourceAsStream(PROPERTIES_FILENAME)) + { + properties.load(stream); + } + catch (Exception e) + { + throw new ConfigurationException("Unable to read " + PROPERTIES_FILENAME, e); + } + + Location local = null; + InetAddressAndPort broadcastAddress = FBUtilities.getBroadcastAddressAndPort(); + for (Map.Entry<Object, Object> entry : properties.entrySet()) + { + String key = (String) entry.getKey(); + String value = (String) entry.getValue(); + if (DEFAULT_PROPERTY.equals(key)) + continue; + + String hostString = StringUtils.remove(key, '/'); + try + { + InetAddressAndPort host = InetAddressAndPort.getByName(hostString); + if (host.equals(broadcastAddress)) + { + local = makeLocation(value); + break; + } + } + catch (UnknownHostException e) + { + throw new ConfigurationException("Unknown host " + hostString, e); + } + + } + + // This may be null, which is ok unless config doesn't contain the location of the local node + Location defaultLocation = makeLocation(properties.getProperty(DEFAULT_PROPERTY)); + + if (local == null) + { + if (defaultLocation == null) + { + throw new ConfigurationException(String.format("Snitch definitions at %s do not define a location for " + + "this node's broadcast address %s, nor does it provides a default", + PROPERTIES_FILENAME, broadcastAddress)); + } + else + { + logger.debug("Broadcast address {} was not present in snitch config, using default location {}. " + + "This only matters on first boot, before registering with the cluster metadata service", + broadcastAddress, defaultLocation); + local = defaultLocation; + } + } + + logger.debug("Loaded location {} for broadcast address {} from property file. " + + "This only matters on first boot, before registering with the cluster metadata service", + local, broadcastAddress); + + return local; + } + +} diff --git a/src/java/org/apache/cassandra/metrics/MessagingMetrics.java b/src/java/org/apache/cassandra/metrics/MessagingMetrics.java index 6151f57ee4d0..8f122d6dd73f 100644 --- a/src/java/org/apache/cassandra/metrics/MessagingMetrics.java +++ b/src/java/org/apache/cassandra/metrics/MessagingMetrics.java @@ -114,7 +114,7 @@ public MessagingMetrics() public DCLatencyRecorder internodeLatencyRecorder(InetAddressAndPort from) { - String dcName = DatabaseDescriptor.getEndpointSnitch().getDatacenter(from); + String dcName = DatabaseDescriptor.getLocator().location(from).datacenter; DCLatencyRecorder dcUpdater = dcLatency.get(dcName); if (dcUpdater == null) dcUpdater = dcLatency.computeIfAbsent(dcName, k -> new DCLatencyRecorder(Metrics.timer(factory.createMetricName(dcName + "-Latency")), allLatency)); diff --git a/src/java/org/apache/cassandra/net/OutboundConnectionSettings.java b/src/java/org/apache/cassandra/net/OutboundConnectionSettings.java index ccc74f0aba4f..23ac75e71712 100644 --- a/src/java/org/apache/cassandra/net/OutboundConnectionSettings.java +++ b/src/java/org/apache/cassandra/net/OutboundConnectionSettings.java @@ -28,11 +28,10 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions; import org.apache.cassandra.db.SystemKeyspace; -import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.Locator; import org.apache.cassandra.utils.FBUtilities; -import static org.apache.cassandra.config.DatabaseDescriptor.getEndpointSnitch; import static org.apache.cassandra.net.MessagingService.instance; import static org.apache.cassandra.utils.FBUtilities.getBroadcastAddressAndPort; @@ -420,7 +419,7 @@ public boolean tcpNoDelay() if (tcpNoDelay != null) return tcpNoDelay; - if (DatabaseDescriptor.isClientOrToolInitialized() || isInLocalDC(getEndpointSnitch(), getBroadcastAddressAndPort(), to)) + if (DatabaseDescriptor.isClientOrToolInitialized() || isInLocalDC(getBroadcastAddressAndPort(), to)) return INTRADC_TCP_NODELAY; return DatabaseDescriptor.getInterDCTcpNoDelay(); @@ -459,7 +458,7 @@ public Framing framing(ConnectionCategory category) if (category.isStreaming()) return Framing.UNPROTECTED; - return shouldCompressConnection(getEndpointSnitch(), getBroadcastAddressAndPort(), to) + return shouldCompressConnection(getBroadcastAddressAndPort(), to) ? Framing.LZ4 : Framing.CRC; } @@ -479,10 +478,11 @@ public OutboundConnectionSettings withDefaults(ConnectionCategory category) from(), socketFactory(), callbacks(), debug(), endpointToVersion()); } - private static boolean isInLocalDC(IEndpointSnitch snitch, InetAddressAndPort localHost, InetAddressAndPort remoteHost) + private static boolean isInLocalDC(InetAddressAndPort localHost, InetAddressAndPort remoteHost) { - String remoteDC = snitch.getDatacenter(remoteHost); - String localDC = snitch.getDatacenter(localHost); + Locator locator = DatabaseDescriptor.getLocator(); + String remoteDC = locator.location(remoteHost).datacenter; + String localDC = locator.location(localHost).datacenter; return remoteDC != null && remoteDC.equals(localDC); } @@ -494,10 +494,10 @@ static ServerEncryptionOptions defaultEncryptionOptions(InetAddressAndPort endpo } @VisibleForTesting - static boolean shouldCompressConnection(IEndpointSnitch snitch, InetAddressAndPort localHost, InetAddressAndPort remoteHost) + static boolean shouldCompressConnection(InetAddressAndPort localHost, InetAddressAndPort remoteHost) { return (DatabaseDescriptor.internodeCompression() == Config.InternodeCompression.all) - || ((DatabaseDescriptor.internodeCompression() == Config.InternodeCompression.dc) && !isInLocalDC(snitch, localHost, remoteHost)); + || ((DatabaseDescriptor.internodeCompression() == Config.InternodeCompression.dc) && !isInLocalDC(localHost, remoteHost)); } } diff --git a/src/java/org/apache/cassandra/repair/RepairJob.java b/src/java/org/apache/cassandra/repair/RepairJob.java index 63b7b96ec518..7b60df94160c 100644 --- a/src/java/org/apache/cassandra/repair/RepairJob.java +++ b/src/java/org/apache/cassandra/repair/RepairJob.java @@ -488,7 +488,7 @@ static List<SyncTask> createOptimisedSyncingSyncTasks(SharedContext ctx, private String getDC(InetAddressAndPort address) { - return ctx.snitch().getDatacenter(address); + return ctx.locator().location(address).datacenter; } /** @@ -573,7 +573,7 @@ private Future<List<TreeResponse>> sendDCAwareValidationRequest(Collection<InetA Map<String, Queue<InetAddressAndPort>> requestsByDatacenter = new HashMap<>(); for (InetAddressAndPort endpoint : endpoints) { - String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(endpoint); + String dc = DatabaseDescriptor.getLocator().location(endpoint).datacenter; Queue<InetAddressAndPort> queue = requestsByDatacenter.computeIfAbsent(dc, k -> new LinkedList<>()); queue.add(endpoint); } diff --git a/src/java/org/apache/cassandra/repair/SharedContext.java b/src/java/org/apache/cassandra/repair/SharedContext.java index 6c13ae99c9e4..54ec4214570f 100644 --- a/src/java/org/apache/cassandra/repair/SharedContext.java +++ b/src/java/org/apache/cassandra/repair/SharedContext.java @@ -32,8 +32,8 @@ import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.gms.IFailureDetector; import org.apache.cassandra.gms.IGossiper; -import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.Locator; import org.apache.cassandra.net.MessageDelivery; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.service.ActiveRepairService; @@ -73,7 +73,7 @@ public MessageDelivery messaging() }; } IFailureDetector failureDetector(); - IEndpointSnitch snitch(); + Locator locator(); IGossiper gossiper(); ICompactionManager compactionManager(); ActiveRepairService repair(); @@ -147,9 +147,9 @@ public IFailureDetector failureDetector() } @Override - public IEndpointSnitch snitch() + public Locator locator() { - return DatabaseDescriptor.getEndpointSnitch(); + return DatabaseDescriptor.getLocator(); } @Override @@ -270,9 +270,9 @@ public IFailureDetector failureDetector() } @Override - public IEndpointSnitch snitch() + public Locator locator() { - return delegate().snitch(); + return delegate().locator(); } @Override diff --git a/src/java/org/apache/cassandra/schema/DistributedMetadataLogKeyspace.java b/src/java/org/apache/cassandra/schema/DistributedMetadataLogKeyspace.java index 4607a0fcf5c4..37af87f3e400 100644 --- a/src/java/org/apache/cassandra/schema/DistributedMetadataLogKeyspace.java +++ b/src/java/org/apache/cassandra/schema/DistributedMetadataLogKeyspace.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Collections; import java.util.Set; import java.util.function.Supplier; @@ -224,4 +225,9 @@ public static KeyspaceMetadata initialMetadata(Set<String> knownDatacenters) { return KeyspaceMetadata.create(SchemaConstants.METADATA_KEYSPACE_NAME, new KeyspaceParams(true, ReplicationParams.simpleMeta(1, knownDatacenters)), Tables.of(Log)); } + + public static KeyspaceMetadata initialMetadata(String datacenter) + { + return initialMetadata(Collections.singleton(datacenter)); + } } \ No newline at end of file diff --git a/src/java/org/apache/cassandra/schema/DistributedSchema.java b/src/java/org/apache/cassandra/schema/DistributedSchema.java index ec0c65eb3a2e..e4eead15b5ea 100644 --- a/src/java/org/apache/cassandra/schema/DistributedSchema.java +++ b/src/java/org/apache/cassandra/schema/DistributedSchema.java @@ -62,13 +62,20 @@ public static final DistributedSchema empty() public static DistributedSchema first(Set<String> knownDatacenters) { + // During upgrades from pre-5.1 versions, the replication params of the system_cluster_metadata + // keyspace using one of the existing DCs. This is so that this keyspace does not cause issues + // for tooling, clients or control plane systems which may inspect schema and have specific + // expectations about DC layout. This keyspace is unused until the CMS is initialized. + // For new clusters which start out on 5.1 or later, this is not necessary to the initial + // replication params use a empty string for the placeholder DC name. + + // During CMS initialization, the replication of this keyspace will be set for real using + // the DC of the first node to become a CMS member. This happens in the PreInitialize + // transformation when executed on the first CMS member. + if (knownDatacenters.isEmpty()) - { - if (DatabaseDescriptor.getLocalDataCenter() != null) - knownDatacenters = Collections.singleton(DatabaseDescriptor.getLocalDataCenter()); - else - knownDatacenters = Collections.singleton("DC1"); - } + knownDatacenters = Collections.singleton(""); + return new DistributedSchema(Keyspaces.of(DistributedMetadataLogKeyspace.initialMetadata(knownDatacenters)), Epoch.FIRST); } @@ -216,7 +223,7 @@ public void initializeKeyspaceInstances(DistributedSchema prev, boolean loadSSTa }); // Avoid system table side effects during initialization - if (epoch.isEqualOrAfter(Epoch.FIRST)) + if (epoch.isEqualOrAfter(Epoch.FIRST) && !DatabaseDescriptor.isClientOrToolInitialized()) { Collection<Mutation> mutations = SchemaKeyspace.convertSchemaDiffToMutations(ksDiff, FBUtilities.timestampMicros()); SchemaKeyspace.applyChanges(mutations); diff --git a/src/java/org/apache/cassandra/schema/ReplicationParams.java b/src/java/org/apache/cassandra/schema/ReplicationParams.java index cf445a4b78a0..277c95914617 100644 --- a/src/java/org/apache/cassandra/schema/ReplicationParams.java +++ b/src/java/org/apache/cassandra/schema/ReplicationParams.java @@ -18,6 +18,7 @@ package org.apache.cassandra.schema; import java.io.IOException; +import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.Map; @@ -121,6 +122,11 @@ static ReplicationParams simple(String replicationFactor) return new ReplicationParams(SimpleStrategy.class, ImmutableMap.of("replication_factor", replicationFactor)); } + public static ReplicationParams simpleMeta(int replicationFactor, String datacenter) + { + return simpleMeta(replicationFactor, Collections.singleton(datacenter)); + } + public static ReplicationParams simpleMeta(int replicationFactor, Set<String> knownDatacenters) { if (replicationFactor <= 0) diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java index 360c26bdcc33..df73f14b40c1 100644 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@ -65,6 +65,11 @@ import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.Locator; +import org.apache.cassandra.tcm.CMSOperations; +import org.apache.cassandra.tcm.ClusterMetadataService; +import org.apache.cassandra.tcm.RegistrationStatus; +import org.apache.cassandra.tcm.Startup; import org.apache.cassandra.metrics.CassandraMetricsRegistry; import org.apache.cassandra.metrics.DefaultNameFactory; import org.apache.cassandra.net.StartupClusterConnectivityChecker; @@ -74,11 +79,8 @@ import org.apache.cassandra.service.paxos.PaxosState; import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.streaming.StreamManager; -import org.apache.cassandra.tcm.CMSOperations; import org.apache.cassandra.tcm.ClusterMetadata; -import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.MultiStepOperation; -import org.apache.cassandra.tcm.Startup; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.JMXServerUtils; import org.apache.cassandra.utils.JVMStabilityInspector; @@ -265,6 +267,8 @@ protected void setup() Startup.initialize(DatabaseDescriptor.getSeeds()); disableAutoCompaction(Schema.instance.distributedKeyspaces().names()); CMSOperations.initJmx(); + if (ClusterMetadata.current().myNodeId() != null) + RegistrationStatus.instance.onRegistration(); } catch (InterruptedException | ExecutionException | IOException e) { @@ -369,9 +373,13 @@ protected void setup() exitOrFail(1, "Fatal configuration error", e); } + // The local rack may have been changed at some point, which will now be reflected in cluster metadata. Update + // the system.local table just in case the actual value doesn't match what the configured location provided + // reported when the earlier call to SystemKeyspace::persistLocalMetadata was made prior to initialising cluster + // metadata. + SystemKeyspace.updateRack(ClusterMetadata.current().locator.local().rack); ScheduledExecutors.optionalTasks.execute(() -> ClusterMetadataService.instance().processor().fetchLogAndWait()); - // TODO: (TM/alexp), this can be made time-dependent // Because we are writing to the system_distributed keyspace, this should happen after that is created, which // happens in StorageService.instance.initServer() Runnable viewRebuild = () -> { @@ -383,11 +391,6 @@ protected void setup() }; ScheduledExecutors.optionalTasks.schedule(viewRebuild, StorageService.RING_DELAY_MILLIS, TimeUnit.MILLISECONDS); - - // TODO: (TM/alexp), we do not need to wait for gossip settlement anymore -// if (!FBUtilities.getBroadcastAddressAndPort().equals(InetAddressAndPort.getLoopbackAddress())) -// Gossiper.waitToSettle(); - StorageService.instance.doAuthSetup(); // re-enable auto-compaction after replay, so correct disk boundaries are used @@ -637,8 +640,9 @@ public void start() { StartupClusterConnectivityChecker connectivityChecker = StartupClusterConnectivityChecker.create(DatabaseDescriptor.getBlockForPeersTimeoutInSeconds(), DatabaseDescriptor.getBlockForPeersInRemoteDatacenters()); + Locator locator = DatabaseDescriptor.getLocator(); Set<InetAddressAndPort> peers = new HashSet<>(ClusterMetadata.current().directory.allJoinedEndpoints()); - connectivityChecker.execute(peers, DatabaseDescriptor.getEndpointSnitch()::getDatacenter); + connectivityChecker.execute(peers, ep -> locator.location(ep).datacenter); // check to see if transports may start else return without starting. This is needed when in survey mode or // when bootstrap has not completed. diff --git a/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java b/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java index e4b208b582fb..d035286d7d92 100644 --- a/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java +++ b/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java @@ -24,7 +24,7 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.Mutation; -import org.apache.cassandra.locator.IEndpointSnitch; +import org.apache.cassandra.locator.Locator; import org.apache.cassandra.locator.NetworkTopologyStrategy; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.locator.ReplicaPlan; @@ -38,7 +38,7 @@ */ public class DatacenterSyncWriteResponseHandler<T> extends AbstractWriteResponseHandler<T> { - private static final IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch(); + private static final Locator locator = DatabaseDescriptor.getLocator(); private final Map<String, AtomicInteger> responses = new HashMap<String, AtomicInteger>(); private final AtomicInteger acks = new AtomicInteger(0); @@ -64,14 +64,14 @@ public DatacenterSyncWriteResponseHandler(ReplicaPlan.ForWrite replicaPlan, } else { - responses.put(DatabaseDescriptor.getLocalDataCenter(), new AtomicInteger(ConsistencyLevel.quorumFor(replicaPlan.replicationStrategy()))); + responses.put(locator.local().datacenter, new AtomicInteger(ConsistencyLevel.quorumFor(replicaPlan.replicationStrategy()))); } // During bootstrap, we have to include the pending endpoints or we may fail the consistency level // guarantees (see #833) for (Replica pending : replicaPlan.pending()) { - responses.get(snitch.getDatacenter(pending)).incrementAndGet(); + responses.get(locator.location(pending.endpoint()).datacenter).incrementAndGet(); } } @@ -80,8 +80,8 @@ public void onResponse(Message<T> message) try { String dataCenter = message == null - ? DatabaseDescriptor.getLocalDataCenter() - : snitch.getDatacenter(message.from()); + ? locator.local().datacenter + : locator.location(message.from()).datacenter; responses.get(dataCenter).getAndDecrement(); acks.incrementAndGet(); diff --git a/src/java/org/apache/cassandra/service/Rebuild.java b/src/java/org/apache/cassandra/service/Rebuild.java index f089aded1360..421595bab637 100644 --- a/src/java/org/apache/cassandra/service/Rebuild.java +++ b/src/java/org/apache/cassandra/service/Rebuild.java @@ -104,17 +104,17 @@ public static void rebuild(String sourceDc, String keyspace, String tokens, Stri RangeStreamer streamer = new RangeStreamer(metadata, StreamOperation.REBUILD, false, // no strict consistency when rebuilding - DatabaseDescriptor.getEndpointSnitch(), + DatabaseDescriptor.getNodeProximity(), StorageService.instance.streamStateStore, false, DatabaseDescriptor.getStreamingConnectionsPerHost(), rebuildMovements, null); if (sourceDc != null) - streamer.addSourceFilter(new RangeStreamer.SingleDatacenterFilter(DatabaseDescriptor.getEndpointSnitch(), sourceDc)); + streamer.addSourceFilter(new RangeStreamer.SingleDatacenterFilter(metadata.locator, sourceDc)); if (excludeLocalDatacenterNodes) - streamer.addSourceFilter(new RangeStreamer.ExcludeLocalDatacenterFilter(DatabaseDescriptor.getEndpointSnitch())); + streamer.addSourceFilter(new RangeStreamer.ExcludeLocalDatacenterFilter(metadata.locator)); if (keyspace == null) { diff --git a/src/java/org/apache/cassandra/service/StartupChecks.java b/src/java/org/apache/cassandra/service/StartupChecks.java index a16864cc2ae9..4ec301946578 100644 --- a/src/java/org/apache/cassandra/service/StartupChecks.java +++ b/src/java/org/apache/cassandra/service/StartupChecks.java @@ -144,8 +144,6 @@ public enum StartupCheckType checkDataDirs, checkSSTablesFormat, checkSystemKeyspaceState, - checkDatacenter, - checkRack, checkLegacyAuthTables, new DataResurrectionCheck()); @@ -738,46 +736,6 @@ public void execute(StartupChecksOptions options) throws StartupException } }; - public static final StartupCheck checkDatacenter = new StartupCheck() - { - @Override - public void execute(StartupChecksOptions options) throws StartupException - { - String storedDc = SystemKeyspace.getDatacenter(); - if (storedDc != null) - { - String currentDc = DatabaseDescriptor.getEndpointSnitch().getLocalDatacenter(); - if (!storedDc.equals(currentDc)) - { - String formatMessage = "Cannot start node if snitch's data center (%s) differs from previous data center (%s). " + - "Please fix the snitch configuration, decommission and rebootstrap this node"; - - throw new StartupException(StartupException.ERR_WRONG_CONFIG, String.format(formatMessage, currentDc, storedDc)); - } - } - } - }; - - public static final StartupCheck checkRack = new StartupCheck() - { - @Override - public void execute(StartupChecksOptions options) throws StartupException - { - String storedRack = SystemKeyspace.getRack(); - if (storedRack != null) - { - String currentRack = DatabaseDescriptor.getEndpointSnitch().getLocalRack(); - if (!storedRack.equals(currentRack)) - { - String formatMessage = "Cannot start node if snitch's rack (%s) differs from previous rack (%s). " + - "Please fix the snitch configuration, decommission and rebootstrap this node"; - - throw new StartupException(StartupException.ERR_WRONG_CONFIG, String.format(formatMessage, currentRack, storedRack)); - } - } - } - }; - public static final StartupCheck checkLegacyAuthTables = new StartupCheck() { @Override diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java index e62c1bb3bd81..cd03fae55226 100644 --- a/src/java/org/apache/cassandra/service/StorageProxy.java +++ b/src/java/org/apache/cassandra/service/StorageProxy.java @@ -877,7 +877,7 @@ public static void mutate(List<? extends IMutation> mutations, ConsistencyLevel throws UnavailableException, OverloadedException, WriteTimeoutException, WriteFailureException { Tracing.trace("Determining replicas for mutation"); - final String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getLocalDatacenter(); + final String localDataCenter = DatabaseDescriptor.getLocator().local().datacenter; List<AbstractWriteResponseHandler<IMutation>> responseHandlers = new ArrayList<>(mutations.size()); WriteType plainWriteType = mutations.size() <= 1 ? WriteType.SIMPLE : WriteType.UNLOGGED_BATCH; @@ -1011,7 +1011,7 @@ public static void mutateMV(ByteBuffer dataKey, Collection<Mutation> mutations, throws UnavailableException, OverloadedException, WriteTimeoutException { Tracing.trace("Determining replicas for mutation"); - final String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getLocalDatacenter(); + final String localDataCenter = DatabaseDescriptor.getLocator().local().datacenter; long startTime = nanoTime(); @@ -1350,7 +1350,7 @@ private static void asyncWriteBatchedMutations(List<WriteResponseHandlerWrapper> private static void syncWriteBatchedMutations(List<WriteResponseHandlerWrapper> wrappers, Stage stage, Dispatcher.RequestTime requestTime) throws WriteTimeoutException, OverloadedException { - String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getLocalDatacenter(); + String localDataCenter = DatabaseDescriptor.getLocator().local().datacenter; for (WriteResponseHandlerWrapper wrapper : wrappers) { @@ -1528,7 +1528,7 @@ public static void sendToHintedReplicas(final Mutation mutation, Collections.singletonList(MessageFlag.CALL_BACK_ON_FAILURE)); } - String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(destination); + String dc = DatabaseDescriptor.getLocator().location(destination.endpoint()).datacenter; // direct writes to local DC or old Cassandra versions // (1.1 knows how to forward old-style String message IDs; updated to int in 2.0) @@ -2403,7 +2403,7 @@ public static boolean shouldHint(Replica replica, boolean tryEnablePersistentWin Set<String> disabledDCs = DatabaseDescriptor.hintedHandoffDisabledDCs(); if (!disabledDCs.isEmpty()) { - final String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(replica); + final String dc = DatabaseDescriptor.getLocator().location(replica.endpoint()).datacenter; if (disabledDCs.contains(dc)) { Tracing.trace("Not hinting {} since its data center {} has been disabled {}", replica, dc, disabledDCs); diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 28d9d124e92c..12981f724ac2 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -47,6 +47,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; import javax.annotation.Nullable; @@ -137,7 +138,6 @@ import org.apache.cassandra.locator.EndpointsByRange; import org.apache.cassandra.locator.EndpointsForRange; import org.apache.cassandra.locator.EndpointsForToken; -import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.LocalStrategy; import org.apache.cassandra.locator.MetaStrategy; @@ -145,6 +145,8 @@ import org.apache.cassandra.locator.RangesByEndpoint; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.locator.Replicas; +import org.apache.cassandra.locator.NodeProximity; +import org.apache.cassandra.locator.SnitchAdapter; import org.apache.cassandra.locator.SystemReplicas; import org.apache.cassandra.metrics.Sampler; import org.apache.cassandra.metrics.SamplingManager; @@ -178,6 +180,7 @@ import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.MultiStepOperation; +import org.apache.cassandra.tcm.RegistrationStatus; import org.apache.cassandra.tcm.Transformation; import org.apache.cassandra.tcm.compatibility.GossipHelper; import org.apache.cassandra.tcm.compatibility.TokenRingUtils; @@ -821,6 +824,7 @@ public void runMayThrow() throws InterruptedException, ExecutionException, IOExc self = Register.maybeRegister(); } + RegistrationStatus.instance.onRegistration(); Startup.maybeExecuteStartupTransformation(self); try @@ -3429,8 +3433,8 @@ private UUID getPreferredHintsStreamTarget() } else { - // stream to the closest peer as chosen by the snitch - candidates = DatabaseDescriptor.getEndpointSnitch().sortedByProximity(getBroadcastAddressAndPort(), candidates); + // stream to the closest peer as chosen by the configured proximity measures + candidates = DatabaseDescriptor.getNodeProximity().sortedByProximity(getBroadcastAddressAndPort(), candidates); InetAddressAndPort hintsDestinationHost = candidates.get(0).endpoint(); return ClusterMetadata.current().directory.peerId(hintsDestinationHost).toUUID(); } @@ -4144,12 +4148,12 @@ public Map<String, String> getViewBuildStatusesWithPort(String keyspace, String public void setDynamicUpdateInterval(int dynamicUpdateInterval) { - if (DatabaseDescriptor.getEndpointSnitch() instanceof DynamicEndpointSnitch) + if (DatabaseDescriptor.getNodeProximity() instanceof DynamicEndpointSnitch) { try { - updateSnitch(null, true, dynamicUpdateInterval, null, null); + updateProximityInternal(null, true, dynamicUpdateInterval, null, null); } catch (ClassNotFoundException e) { @@ -4164,6 +4168,18 @@ public int getDynamicUpdateInterval() } public void updateSnitch(String epSnitchClassName, Boolean dynamic, Integer dynamicUpdateInterval, Integer dynamicResetInterval, Double dynamicBadnessThreshold) throws ClassNotFoundException + { + Supplier<NodeProximity> factory = () -> new SnitchAdapter(DatabaseDescriptor.createEndpointSnitch(epSnitchClassName)); + updateProximityInternal(factory, dynamic, dynamicUpdateInterval, dynamicResetInterval, dynamicBadnessThreshold); + } + + public void updateNodeProximity(String npsClassName, Boolean dynamic, Integer dynamicUpdateInterval, Integer dynamicResetInterval, Double dynamicBadnessThreshold) throws ClassNotFoundException + { + Supplier<NodeProximity> factory = () -> DatabaseDescriptor.createProximityImpl(npsClassName); + updateProximityInternal(factory, dynamic, dynamicUpdateInterval, dynamicResetInterval, dynamicBadnessThreshold); + } + + private void updateProximityInternal(Supplier<NodeProximity> implSupplier, Boolean dynamic, Integer dynamicUpdateInterval, Integer dynamicResetInterval, Double dynamicBadnessThreshold) throws ClassNotFoundException { // apply dynamic snitch configuration if (dynamicUpdateInterval != null) @@ -4173,51 +4189,53 @@ public void updateSnitch(String epSnitchClassName, Boolean dynamic, Integer dyna if (dynamicBadnessThreshold != null) DatabaseDescriptor.setDynamicBadnessThreshold(dynamicBadnessThreshold); - IEndpointSnitch oldSnitch = DatabaseDescriptor.getEndpointSnitch(); + NodeProximity oldProximity = DatabaseDescriptor.getNodeProximity(); // new snitch registers mbean during construction - if(epSnitchClassName != null) + if(implSupplier != null) { // need to unregister the mbean _before_ the new dynamic snitch is instantiated (and implicitly initialized // and its mbean registered) - if (oldSnitch instanceof DynamicEndpointSnitch) - ((DynamicEndpointSnitch)oldSnitch).close(); + if (oldProximity instanceof DynamicEndpointSnitch) + ((DynamicEndpointSnitch)oldProximity).close(); - IEndpointSnitch newSnitch; + NodeProximity newProximity; try { - newSnitch = DatabaseDescriptor.createEndpointSnitch(dynamic != null && dynamic, epSnitchClassName); + newProximity = implSupplier.get(); + if (dynamic != null && dynamic) + newProximity = new DynamicEndpointSnitch(newProximity); } catch (ConfigurationException e) { throw new ClassNotFoundException(e.getMessage()); } - if (newSnitch instanceof DynamicEndpointSnitch) + if (newProximity instanceof DynamicEndpointSnitch) { logger.info("Created new dynamic snitch {} with update-interval={}, reset-interval={}, badness-threshold={}", - ((DynamicEndpointSnitch)newSnitch).subsnitch.getClass().getName(), DatabaseDescriptor.getDynamicUpdateInterval(), + ((DynamicEndpointSnitch)newProximity).delegate.getClass().getName(), DatabaseDescriptor.getDynamicUpdateInterval(), DatabaseDescriptor.getDynamicResetInterval(), DatabaseDescriptor.getDynamicBadnessThreshold()); } else { - logger.info("Created new non-dynamic snitch {}", newSnitch.getClass().getName()); + logger.info("Created new non-dynamic snitch {}", newProximity.getClass().getName()); } - // point snitch references to the new instance - DatabaseDescriptor.setEndpointSnitch(newSnitch); + // point reference to the new instance + DatabaseDescriptor.setNodeProximity(newProximity); } else { - if (oldSnitch instanceof DynamicEndpointSnitch) + if (oldProximity instanceof DynamicEndpointSnitch) { logger.info("Applying config change to dynamic snitch {} with update-interval={}, reset-interval={}, badness-threshold={}", - ((DynamicEndpointSnitch)oldSnitch).subsnitch.getClass().getName(), DatabaseDescriptor.getDynamicUpdateInterval(), + ((DynamicEndpointSnitch)oldProximity).delegate.getClass().getName(), DatabaseDescriptor.getDynamicUpdateInterval(), DatabaseDescriptor.getDynamicResetInterval(), DatabaseDescriptor.getDynamicBadnessThreshold()); - DynamicEndpointSnitch snitch = (DynamicEndpointSnitch)oldSnitch; - snitch.applyConfigChanges(); + DynamicEndpointSnitch proximity = (DynamicEndpointSnitch)oldProximity; + proximity.applyConfigChanges(); } } } diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java index 46ecc0e05aac..3e556db04a3d 100644 --- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java +++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java @@ -652,15 +652,35 @@ default int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, * * The parameters {@code dynamicUpdateInterval}, {@code dynamicResetInterval} and {@code dynamicBadnessThreshold} * can be specified individually to update the parameters of the dynamic snitch during runtime. - * * @param epSnitchClassName the canonical path name for a class implementing IEndpointSnitch * @param dynamic boolean that decides whether dynamicsnitch is used or not - only valid, if {@code epSnitchClassName} is specified * @param dynamicUpdateInterval integer, in ms (defaults to the value configured in cassandra.yaml, which defaults to 100) * @param dynamicResetInterval integer, in ms (defaults to the value configured in cassandra.yaml, which defaults to 600,000) * @param dynamicBadnessThreshold double, (defaults to the value configured in cassandra.yaml, which defaults to 0.0) + * @deprecated See CASSANDRA-19488 */ + @Deprecated(since = "5.1") public void updateSnitch(String epSnitchClassName, Boolean dynamic, Integer dynamicUpdateInterval, Integer dynamicResetInterval, Double dynamicBadnessThreshold) throws ClassNotFoundException; + /** + * Change NodeProximity class and dynamic-ness (and dynamic attributes) at runtime. + * This method supercedes the deprecated updateSnitch method. + * + * This method is used to change the proximity implementation and/or dynamic proximity parameters. + * If {@code proximityClassName} is specified, it will configure a new instance and make it a + * 'dynamic snitch' if {@code dynamic} is specified and {@code true}. + * + * The parameters {@code dynamicUpdateInterval}, {@code dynamicResetInterval} and {@code dynamicBadnessThreshold} + * can be specified individually to update the parameters of the dynamic snitch during runtime. + * + * @param proximityClassName the canonical path name for a class implementing NodeProximity + * @param dynamic boolean that decides whether dynamicsnitch is used or not - only valid, if {@code epSnitchClassName} is specified + * @param dynamicUpdateInterval integer, in ms (defaults to the value configured in cassandra.yaml, which defaults to 100) + * @param dynamicResetInterval integer, in ms (defaults to the value configured in cassandra.yaml, which defaults to 600,000) + * @param dynamicBadnessThreshold double, (defaults to the value configured in cassandra.yaml, which defaults to 0.0) + */ + public void updateNodeProximity(String proximityClassName, Boolean dynamic, Integer dynamicUpdateInterval, Integer dynamicResetInterval, Double dynamicBadnessThreshold) throws ClassNotFoundException; + /* Update dynamic_snitch_update_interval in ms */ diff --git a/src/java/org/apache/cassandra/service/TokenRange.java b/src/java/org/apache/cassandra/service/TokenRange.java index 37971f5e4f6c..27d1485094bb 100644 --- a/src/java/org/apache/cassandra/service/TokenRange.java +++ b/src/java/org/apache/cassandra/service/TokenRange.java @@ -22,8 +22,9 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; -import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.Locator; +import org.apache.cassandra.tcm.membership.Location; /** * Holds token range informations for the sake of {@link StorageService#describeRing}. @@ -57,12 +58,15 @@ private String toStr(Token tk) public static TokenRange create(Token.TokenFactory tokenFactory, Range<Token> range, List<InetAddressAndPort> endpoints, boolean withPorts) { List<EndpointDetails> details = new ArrayList<>(endpoints.size()); - IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch(); + Locator locator = DatabaseDescriptor.getLocator(); for (InetAddressAndPort ep : endpoints) + { + Location location = locator.location(ep); details.add(new EndpointDetails(ep, StorageService.instance.getNativeaddress(ep, withPorts), - snitch.getDatacenter(ep), - snitch.getRack(ep))); + location.datacenter, + location.rack)); + } return new TokenRange(tokenFactory, range, details); } diff --git a/src/java/org/apache/cassandra/service/paxos/PaxosCommit.java b/src/java/org/apache/cassandra/service/paxos/PaxosCommit.java index 943b04c30c1b..b5ce86794dbb 100644 --- a/src/java/org/apache/cassandra/service/paxos/PaxosCommit.java +++ b/src/java/org/apache/cassandra/service/paxos/PaxosCommit.java @@ -33,6 +33,7 @@ import org.apache.cassandra.locator.EndpointsForToken; import org.apache.cassandra.locator.InOurDc; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.Locator; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.net.IVerbHandler; import org.apache.cassandra.net.Message; @@ -214,7 +215,8 @@ private boolean isSelfOrSend(Message<Agreed> commitMessage, Message<Mutation> mu private static boolean isInLocalDc(InetAddressAndPort destination) { - return DatabaseDescriptor.getLocalDataCenter().equals(DatabaseDescriptor.getEndpointSnitch().getDatacenter(destination)); + Locator locator = DatabaseDescriptor.getLocator(); + return locator.local().sameDatacenter(locator.location(destination)); } /** diff --git a/src/java/org/apache/cassandra/service/paxos/PaxosRepair.java b/src/java/org/apache/cassandra/service/paxos/PaxosRepair.java index 0e3a732185d5..016086cbb18d 100644 --- a/src/java/org/apache/cassandra/service/paxos/PaxosRepair.java +++ b/src/java/org/apache/cassandra/service/paxos/PaxosRepair.java @@ -550,12 +550,13 @@ public static boolean hasSufficientLiveNodesForTopologyChange(Keyspace keyspace, ReplicationParams replication = keyspace.getMetadata().params.replication; // Special case meta keyspace as it uses a custom partitioner/tokens, but the paxos table and repairs // are based on the system partitioner + ClusterMetadata metadata = ClusterMetadata.current(); Collection<InetAddressAndPort> allEndpoints = replication.isMeta() - ? ClusterMetadata.current().fullCMSMembers() - : ClusterMetadata.current().placements.get(replication).reads.forRange(range).endpoints(); + ? metadata.fullCMSMembers() + : metadata.placements.get(replication).reads.forRange(range).endpoints(); return hasSufficientLiveNodesForTopologyChange(allEndpoints, liveEndpoints, - DatabaseDescriptor.getEndpointSnitch()::getDatacenter, + ep -> metadata.locator.location(ep).datacenter, DatabaseDescriptor.paxoTopologyRepairNoDcChecks(), DatabaseDescriptor.paxoTopologyRepairStrictEachQuorum()); } diff --git a/src/java/org/apache/cassandra/service/reads/ReadCallback.java b/src/java/org/apache/cassandra/service/reads/ReadCallback.java index 4d5c92280b86..ca25d1a3fd85 100644 --- a/src/java/org/apache/cassandra/service/reads/ReadCallback.java +++ b/src/java/org/apache/cassandra/service/reads/ReadCallback.java @@ -258,7 +258,7 @@ public boolean invokeOnFailure() private void assertWaitingFor(InetAddressAndPort from) { assert !replicaPlan().consistencyLevel().isDatacenterLocal() - || DatabaseDescriptor.getLocalDataCenter().equals(DatabaseDescriptor.getEndpointSnitch().getDatacenter(from)) + || DatabaseDescriptor.getLocator().local().sameDatacenter(DatabaseDescriptor.getLocator().location(from)) : "Received read response from unexpected replica: " + from; } } diff --git a/src/java/org/apache/cassandra/service/reads/range/ReplicaPlanMerger.java b/src/java/org/apache/cassandra/service/reads/range/ReplicaPlanMerger.java index f2dacd0ccf1e..20e9562f9311 100644 --- a/src/java/org/apache/cassandra/service/reads/range/ReplicaPlanMerger.java +++ b/src/java/org/apache/cassandra/service/reads/range/ReplicaPlanMerger.java @@ -27,6 +27,7 @@ import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.locator.ReplicaPlan; import org.apache.cassandra.locator.ReplicaPlans; +import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.utils.AbstractIterator; class ReplicaPlanMerger extends AbstractIterator<ReplicaPlan.ForRangeRead> @@ -49,6 +50,7 @@ protected ReplicaPlan.ForRangeRead computeNext() return endOfData(); ReplicaPlan.ForRangeRead current = ranges.next(); + ClusterMetadata metadata = ClusterMetadata.current(); // getRestrictedRange has broken the queried range into per-[vnode] token ranges, but this doesn't take // the replication factor into account. If the intersection of live endpoints for 2 consecutive ranges @@ -64,7 +66,7 @@ protected ReplicaPlan.ForRangeRead computeNext() break; ReplicaPlan.ForRangeRead next = ranges.peek(); - ReplicaPlan.ForRangeRead merged = ReplicaPlans.maybeMerge(keyspace, consistency, current, next); + ReplicaPlan.ForRangeRead merged = ReplicaPlans.maybeMerge(metadata, keyspace, consistency, current, next); if (merged == null) break; diff --git a/src/java/org/apache/cassandra/streaming/DataMovementVerbHandler.java b/src/java/org/apache/cassandra/streaming/DataMovementVerbHandler.java index 9415f8928529..1bccf29044f2 100644 --- a/src/java/org/apache/cassandra/streaming/DataMovementVerbHandler.java +++ b/src/java/org/apache/cassandra/streaming/DataMovementVerbHandler.java @@ -57,7 +57,9 @@ public void doVerb(Message<DataMovement> message) throws IOException assert local.isSelf(); boolean transientAdded = false; boolean fullAdded = false; - for (Replica remote : DatabaseDescriptor.getEndpointSnitch().sortedByProximity(local.endpoint(), endpoints).filter(ep -> FailureDetector.instance.isAlive(ep.endpoint()))) + for (Replica remote : DatabaseDescriptor.getNodeProximity() + .sortedByProximity(local.endpoint(), endpoints) + .filter(ep -> FailureDetector.instance.isAlive(ep.endpoint()))) { assert !remote.isSelf(); if (remote.isFull() && !fullAdded) diff --git a/src/java/org/apache/cassandra/streaming/StreamManager.java b/src/java/org/apache/cassandra/streaming/StreamManager.java index fec8b2de21f2..6bfecf8aa813 100644 --- a/src/java/org/apache/cassandra/streaming/StreamManager.java +++ b/src/java/org/apache/cassandra/streaming/StreamManager.java @@ -114,9 +114,10 @@ private StreamRateLimiter(InetAddressAndPort peer, RateLimiter limiter, RateLimi this.interDCLimiter = interDCLimiter; this.throughput = throughput; this.interDCThroughput = interDCThroughput; - if (DatabaseDescriptor.getLocalDataCenter() != null && DatabaseDescriptor.getEndpointSnitch() != null) - isLocalDC = DatabaseDescriptor.getLocalDataCenter().equals( - DatabaseDescriptor.getEndpointSnitch().getDatacenter(peer)); + if (DatabaseDescriptor.getLocalDataCenter() != null && DatabaseDescriptor.getLocator() != null) + isLocalDC = DatabaseDescriptor.getLocator() + .local() + .sameDatacenter(DatabaseDescriptor.getLocator().location(peer)); else isLocalDC = true; } diff --git a/src/java/org/apache/cassandra/tcm/ClusterMetadata.java b/src/java/org/apache/cassandra/tcm/ClusterMetadata.java index 9862e51ecb40..e2b50483ce92 100644 --- a/src/java/org/apache/cassandra/tcm/ClusterMetadata.java +++ b/src/java/org/apache/cassandra/tcm/ClusterMetadata.java @@ -46,6 +46,7 @@ import org.apache.cassandra.locator.EndpointsForRange; import org.apache.cassandra.locator.EndpointsForToken; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.Locator; import org.apache.cassandra.locator.MetaStrategy; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.net.CMSIdentifierMismatchException; @@ -96,6 +97,9 @@ public class ClusterMetadata public final InProgressSequences inProgressSequences; public final ImmutableMap<ExtensionKey<?,?>, ExtensionValue<?>> extensions; + // This isn't serialized as part of ClusterMetadata it's really just a view over the Directory. + public final Locator locator; + // These fields are lazy but only for the test purposes, since their computation requires initialization of the log ks private EndpointsForRange fullCMSReplicas; private Set<InetAddressAndPort> fullCMSEndpoints; @@ -174,6 +178,7 @@ private ClusterMetadata(int metadataIdentifier, this.lockedRanges = lockedRanges; this.inProgressSequences = inProgressSequences; this.extensions = ImmutableMap.copyOf(extensions); + this.locator = Locator.usingDirectory(directory); } public Set<InetAddressAndPort> fullCMSMembers() diff --git a/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java b/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java index 845b6741f9a7..555cdb7f9383 100644 --- a/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java +++ b/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java @@ -98,6 +98,9 @@ public static void setInstance(ClusterMetadataService newInstance) throw new IllegalStateException(String.format("Cluster metadata is already initialized to %s.", instance), trace); instance = newInstance; + RegistrationStatus.instance.onInitialized(); + if (newInstance.metadata().myNodeId() != null) + RegistrationStatus.instance.onRegistration(); trace = new RuntimeException("Previously initialized trace"); } @@ -105,6 +108,7 @@ public static void setInstance(ClusterMetadataService newInstance) public static ClusterMetadataService unsetInstance() { ClusterMetadataService tmp = instance(); + RegistrationStatus.instance.resetState(); instance = null; return tmp; } @@ -245,7 +249,8 @@ public static void initializeForTools(boolean loadSSTables) { if (instance != null) return; - ClusterMetadata emptyFromSystemTables = emptyWithSchemaFromSystemTables(Collections.singleton("DC1")) + String localDC = DatabaseDescriptor.getLocalDataCenter(); + ClusterMetadata emptyFromSystemTables = emptyWithSchemaFromSystemTables(Collections.singleton(localDC)) .forceEpoch(Epoch.EMPTY); LocalLog.LogSpec logSpec = LocalLog.logSpec() @@ -274,7 +279,7 @@ public void notifyPostCommit(ClusterMetadata prev, ClusterMetadata next, boolean new PeerLogFetcher(log)); log.readyUnchecked(); - log.bootstrap(FBUtilities.getBroadcastAddressAndPort()); + log.bootstrap(FBUtilities.getBroadcastAddressAndPort(), localDC); ClusterMetadataService.setInstance(cms); } @@ -292,6 +297,7 @@ public static void initializeForClients(DistributedSchema initialSchema) if (instance != null) return; + ClusterMetadataService.setInstance(StubClusterMetadataService.forClientTools(initialSchema)); } diff --git a/src/java/org/apache/cassandra/tcm/RegistrationStatus.java b/src/java/org/apache/cassandra/tcm/RegistrationStatus.java new file mode 100644 index 000000000000..736a0267832f --- /dev/null +++ b/src/java/org/apache/cassandra/tcm/RegistrationStatus.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.tcm; + +import java.util.concurrent.atomic.AtomicReference; + +import com.google.common.annotations.VisibleForTesting; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.net.MessagingService; + +public class RegistrationStatus +{ + public enum State { INITIAL, UNREGISTERED, REGISTERED } + + private static final Logger logger = LoggerFactory.getLogger(RegistrationStatus.class); + public static final RegistrationStatus instance = new RegistrationStatus(); + private final AtomicReference<RegistrationStatus.State> state = new AtomicReference<>(State.INITIAL); + + public RegistrationStatus.State getCurrent() + { + return state.get(); + } + + @VisibleForTesting + public void resetState() + { + state.set(State.INITIAL); + } + + public void onInitialized() + { + logger.info("Node is initialized, moving to UNREGISTERED state"); + if (!state.compareAndSet(State.INITIAL, State.UNREGISTERED)) + throw new IllegalStateException(String.format("Cannot move to UNREGISTERED state (%s)", state.get())); + } + + public void onRegistration() + { + // This may have been done already if the metadata log replay at start up included our registration + RegistrationStatus.State current = state.get(); + if (current == State.REGISTERED) + return; + + logger.info("This node is registered, moving state to REGISTERED and interrupting any previously established peer connections"); + state.getAndSet(RegistrationStatus.State.REGISTERED); + MessagingService.instance().channelManagers.keySet().forEach(MessagingService.instance()::interruptOutbound); + } +} diff --git a/src/java/org/apache/cassandra/tcm/Startup.java b/src/java/org/apache/cassandra/tcm/Startup.java index 2b60c1561d13..b184866f5725 100644 --- a/src/java/org/apache/cassandra/tcm/Startup.java +++ b/src/java/org/apache/cassandra/tcm/Startup.java @@ -135,7 +135,8 @@ public static void initialize(Set<InetAddressAndPort> seeds, public static void initializeAsFirstCMSNode() { InetAddressAndPort addr = FBUtilities.getBroadcastAddressAndPort(); - ClusterMetadataService.instance().log().bootstrap(addr); + String datacenter = DatabaseDescriptor.getLocator().local().datacenter; + ClusterMetadataService.instance().log().bootstrap(addr, datacenter); ClusterMetadata metadata = ClusterMetadata.current(); assert ClusterMetadataService.state() == LOCAL : String.format("Can't initialize as node hasn't transitioned to CMS state. State: %s.\n%s", ClusterMetadataService.state(), metadata); diff --git a/src/java/org/apache/cassandra/tcm/StubClusterMetadataService.java b/src/java/org/apache/cassandra/tcm/StubClusterMetadataService.java index 8e191307d196..855cce0d363a 100644 --- a/src/java/org/apache/cassandra/tcm/StubClusterMetadataService.java +++ b/src/java/org/apache/cassandra/tcm/StubClusterMetadataService.java @@ -44,7 +44,6 @@ public class StubClusterMetadataService extends ClusterMetadataService public static StubClusterMetadataService forClientTools() { - DatabaseDescriptor.setLocalDataCenter("DC1"); KeyspaceMetadata ks = DistributedMetadataLogKeyspace.initialMetadata(Collections.singleton("DC1")); DistributedSchema schema = new DistributedSchema(Keyspaces.of(ks)); return new StubClusterMetadataService(new ClusterMetadata(DatabaseDescriptor.getPartitioner(), @@ -54,7 +53,6 @@ public static StubClusterMetadataService forClientTools() public static StubClusterMetadataService forClientTools(DistributedSchema initialSchema) { - DatabaseDescriptor.setLocalDataCenter("DC1"); ClusterMetadata metadata = new ClusterMetadata(DatabaseDescriptor.getPartitioner()); metadata = metadata.transformer().with(initialSchema).build().metadata.forceEpoch(Epoch.EMPTY); return new StubClusterMetadataService(metadata); @@ -167,6 +165,7 @@ public static class Builder public StubClusterMetadataService build() { if (initial == null) + { initial = new ClusterMetadata(Epoch.EMPTY, partitioner, DistributedSchema.empty(), @@ -176,6 +175,7 @@ public StubClusterMetadataService build() LockedRanges.EMPTY, InProgressSequences.EMPTY, ImmutableMap.of()); + } return new StubClusterMetadataService(new UniformRangePlacement(), snapshots != null ? snapshots : MetadataSnapshots.NO_OP, LocalLog.logSpec().withInitialState(initial).createLog(), diff --git a/src/java/org/apache/cassandra/tcm/compatibility/TokenRingUtils.java b/src/java/org/apache/cassandra/tcm/compatibility/TokenRingUtils.java index 4a15925bb9e1..676915d3f2d3 100644 --- a/src/java/org/apache/cassandra/tcm/compatibility/TokenRingUtils.java +++ b/src/java/org/apache/cassandra/tcm/compatibility/TokenRingUtils.java @@ -157,7 +157,7 @@ public static Collection<Range<Token>> getPrimaryRangesForEndpoint(String keyspa public static Collection<Range<Token>> getPrimaryRangeForEndpointWithinDC(String keyspace, InetAddressAndPort referenceEndpoint) { ClusterMetadata metadata = ClusterMetadata.current(); - String localDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(referenceEndpoint); + String localDC = DatabaseDescriptor.getLocator().location(referenceEndpoint).datacenter; Collection<InetAddressAndPort> localDcNodes = metadata.directory.datacenterEndpoints(localDC); AbstractReplicationStrategy strategy = Keyspace.open(keyspace).getReplicationStrategy(); diff --git a/src/java/org/apache/cassandra/tcm/listeners/LegacyStateListener.java b/src/java/org/apache/cassandra/tcm/listeners/LegacyStateListener.java index 25aca06f00b4..f0fced7ae6d5 100644 --- a/src/java/org/apache/cassandra/tcm/listeners/LegacyStateListener.java +++ b/src/java/org/apache/cassandra/tcm/listeners/LegacyStateListener.java @@ -34,6 +34,7 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tcm.ClusterMetadata; @@ -50,6 +51,7 @@ import static org.apache.cassandra.tcm.membership.NodeState.BOOT_REPLACING; import static org.apache.cassandra.tcm.membership.NodeState.LEFT; import static org.apache.cassandra.tcm.membership.NodeState.MOVING; +import static org.apache.cassandra.tcm.membership.NodeState.REGISTERED; public class LegacyStateListener implements ChangeListener.Async { @@ -115,8 +117,14 @@ public void notifyPostCommit(ClusterMetadata prev, ClusterMetadata next, boolean Gossiper.instance.addLocalApplicationState(SCHEMA, StorageService.instance.valueFactory.schema(next.schema.getVersion())); } - - if (next.directory.peerState(change) == LEFT) + if (next.directory.peerState(change) == REGISTERED) + { + // Re-establish any connections made prior to this node registering + InetAddressAndPort endpoint = next.directory.endpoint(change); + logger.info("Peer with address {} has registered, interrupting any previously established connections", endpoint); + MessagingService.instance().interruptOutbound(endpoint); + } + else if (next.directory.peerState(change) == LEFT) { Gossiper.instance.mergeNodeToGossip(change, next, prev.tokenMap.tokens(change)); InetAddressAndPort endpoint = prev.directory.endpoint(change); diff --git a/src/java/org/apache/cassandra/tcm/listeners/SchemaListener.java b/src/java/org/apache/cassandra/tcm/listeners/SchemaListener.java index 2dadbb5007d9..a506cb18774e 100644 --- a/src/java/org/apache/cassandra/tcm/listeners/SchemaListener.java +++ b/src/java/org/apache/cassandra/tcm/listeners/SchemaListener.java @@ -26,6 +26,7 @@ import org.apache.cassandra.schema.SchemaDiagnostics; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.Epoch; public class SchemaListener implements ChangeListener { @@ -44,7 +45,7 @@ public void notifyPreCommit(ClusterMetadata prev, ClusterMetadata next, boolean protected void notifyInternal(ClusterMetadata prev, ClusterMetadata next, boolean fromSnapshot, boolean loadSSTables) { - if (!fromSnapshot && next.schema.lastModified().equals(prev.schema.lastModified())) + if (next.epoch.isAfter(Epoch.FIRST) && !fromSnapshot && next.schema.lastModified().equals(prev.schema.lastModified())) return; next.schema.initializeKeyspaceInstances(prev.schema, loadSSTables); } diff --git a/src/java/org/apache/cassandra/tcm/log/LocalLog.java b/src/java/org/apache/cassandra/tcm/log/LocalLog.java index 36b6e41011e3..6e54e4657fa5 100644 --- a/src/java/org/apache/cassandra/tcm/log/LocalLog.java +++ b/src/java/org/apache/cassandra/tcm/log/LocalLog.java @@ -279,14 +279,24 @@ private LocalLog(LogSpec logSpec) entryFilters = Lists.newCopyOnWriteArrayList(); } - public void bootstrap(InetAddressAndPort addr) + @VisibleForTesting + public void unsafeBootstrapForTesting(InetAddressAndPort addr) + { + bootstrap(addr, ""); + } + + /** + * + * @param addr + * @param datacenter + */ + public void bootstrap(InetAddressAndPort addr, String datacenter) { ClusterMetadata metadata = metadata(); assert metadata.epoch.isBefore(FIRST) : String.format("Metadata epoch %s should be before first", metadata.epoch); - Transformation transform = PreInitialize.withFirstCMS(addr); + Transformation transform = PreInitialize.withFirstCMS(addr, datacenter); append(new Entry(Entry.Id.NONE, FIRST, transform)); - waitForHighestConsecutive(); - metadata = metadata(); + metadata = waitForHighestConsecutive(); assert metadata.epoch.is(Epoch.FIRST) : String.format("Epoch: %s. CMS: %s", metadata.epoch, metadata.fullCMSMembers()); } diff --git a/src/java/org/apache/cassandra/tcm/membership/Directory.java b/src/java/org/apache/cassandra/tcm/membership/Directory.java index 2b3dc967e41e..87a6bde05320 100644 --- a/src/java/org/apache/cassandra/tcm/membership/Directory.java +++ b/src/java/org/apache/cassandra/tcm/membership/Directory.java @@ -38,6 +38,7 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; + import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.MetadataValue; import org.apache.cassandra.tcm.serialization.MetadataSerializer; diff --git a/src/java/org/apache/cassandra/tcm/membership/Location.java b/src/java/org/apache/cassandra/tcm/membership/Location.java index 7e38db79cb3b..faf8230d94fe 100644 --- a/src/java/org/apache/cassandra/tcm/membership/Location.java +++ b/src/java/org/apache/cassandra/tcm/membership/Location.java @@ -30,6 +30,7 @@ public class Location { public static final Serializer serializer = new Serializer(); + public static final Location UNKNOWN = new Location("UNKNOWN_DC", "UNKNOWN_RACK"); public final String datacenter; public final String rack; @@ -40,6 +41,11 @@ public Location(String datacenter, String rack) this.rack = rack; } + public boolean sameDatacenter(Location other) + { + return datacenter.equals(other.datacenter); + } + @Override public boolean equals(Object o) { diff --git a/src/java/org/apache/cassandra/tcm/membership/NodeAddresses.java b/src/java/org/apache/cassandra/tcm/membership/NodeAddresses.java index 53f2d1acbaac..dcbb90eb685c 100644 --- a/src/java/org/apache/cassandra/tcm/membership/NodeAddresses.java +++ b/src/java/org/apache/cassandra/tcm/membership/NodeAddresses.java @@ -84,7 +84,6 @@ public boolean identityMatches(NodeAddresses other) public boolean conflictsWith(NodeAddresses other) { return broadcastAddress.equals(other.broadcastAddress) || - localAddress.equals(other.localAddress) || nativeAddress.equals(other.nativeAddress); } diff --git a/src/java/org/apache/cassandra/tcm/sequences/Move.java b/src/java/org/apache/cassandra/tcm/sequences/Move.java index 09811fba80f7..99ada06f9356 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/Move.java +++ b/src/java/org/apache/cassandra/tcm/sequences/Move.java @@ -367,8 +367,9 @@ private static MovementMap movementMap(IFailureDetector fd, DataPlacements place // if we are not running with strict consistency, try to find other sources for streaming if (needsRelaxedSources.get()) { - for (Replica source : DatabaseDescriptor.getEndpointSnitch().sortedByProximity(FBUtilities.getBroadcastAddressAndPort(), - oldOwners.forRange(destination.range()).get())) + for (Replica source : DatabaseDescriptor.getNodeProximity() + .sortedByProximity(FBUtilities.getBroadcastAddressAndPort(), + oldOwners.forRange(destination.range()).get())) { if (fd.isAlive(source.endpoint()) && !source.endpoint().equals(destination.endpoint())) { diff --git a/src/java/org/apache/cassandra/tcm/serialization/Version.java b/src/java/org/apache/cassandra/tcm/serialization/Version.java index ed601009f65f..55bbbfb04e79 100644 --- a/src/java/org/apache/cassandra/tcm/serialization/Version.java +++ b/src/java/org/apache/cassandra/tcm/serialization/Version.java @@ -48,6 +48,7 @@ public enum Version V4(4), /** * - AlterSchema includes execution timestamp + * - PreInitialize includes datacenter (affects local serialization on first CMS node only) */ V5(5), diff --git a/src/java/org/apache/cassandra/tcm/transformations/Register.java b/src/java/org/apache/cassandra/tcm/transformations/Register.java index f53eb9a1f6f2..389f62c69801 100644 --- a/src/java/org/apache/cassandra/tcm/transformations/Register.java +++ b/src/java/org/apache/cassandra/tcm/transformations/Register.java @@ -31,7 +31,6 @@ import org.apache.cassandra.db.SystemKeyspace; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.Transformation; @@ -96,30 +95,37 @@ public static NodeId register(NodeAddresses nodeAddresses) return register(nodeAddresses, NodeVersion.CURRENT); } + @VisibleForTesting + public static NodeId register(NodeAddresses nodeAddresses, Location location) + { + return register(nodeAddresses, location, NodeVersion.CURRENT); + } + @VisibleForTesting public static NodeId register(NodeAddresses nodeAddresses, NodeVersion nodeVersion) { - IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch(); - Location location = new Location(snitch.getLocalDatacenter(), snitch.getLocalRack()); + return register(nodeAddresses, DatabaseDescriptor.getLocator().local(), nodeVersion); + } + private static NodeId register(NodeAddresses nodeAddresses, Location location, NodeVersion nodeVersion) + { ClusterMetadata metadata = ClusterMetadata.current(); + DatabaseDescriptor.getInitialLocationProvider().validate(metadata); NodeId nodeId = metadata.directory.peerId(nodeAddresses.broadcastAddress); if (nodeId == null || metadata.directory.peerState(nodeId) == NodeState.LEFT) { if (nodeId != null) - ClusterMetadataService.instance() - .commit(new Unregister(nodeId, EnumSet.of(NodeState.LEFT))); - nodeId = ClusterMetadataService.instance() - .commit(new Register(nodeAddresses, location, nodeVersion)) - .directory - .peerId(nodeAddresses.broadcastAddress); + ClusterMetadataService.instance().commit(new Unregister(nodeId, EnumSet.of(NodeState.LEFT))); + + Register registration = new Register(nodeAddresses, location, nodeVersion); + nodeId = ClusterMetadataService.instance().commit(registration).directory.peerId(nodeAddresses.broadcastAddress); } else { throw new IllegalStateException(String.format("A node with address %s already exists, cancelling join. Use cassandra.replace_address if you want to replace this node.", nodeAddresses.broadcastAddress)); } - logger.info("Registering with endpoint {}", nodeAddresses.broadcastAddress); + logger.info("Registered with endpoint {}, node id: {}", nodeAddresses.broadcastAddress, nodeId); return nodeId; } diff --git a/src/java/org/apache/cassandra/tcm/transformations/cms/PreInitialize.java b/src/java/org/apache/cassandra/tcm/transformations/cms/PreInitialize.java index 50aa3aef4222..ed25d33cdf13 100644 --- a/src/java/org/apache/cassandra/tcm/transformations/cms/PreInitialize.java +++ b/src/java/org/apache/cassandra/tcm/transformations/cms/PreInitialize.java @@ -26,6 +26,9 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.MetaStrategy; import org.apache.cassandra.locator.Replica; +import org.apache.cassandra.schema.DistributedMetadataLogKeyspace; +import org.apache.cassandra.schema.DistributedSchema; +import org.apache.cassandra.schema.Keyspaces; import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.Transformation; import org.apache.cassandra.tcm.serialization.AsymmetricMetadataSerializer; @@ -41,25 +44,27 @@ public class PreInitialize implements Transformation public static Serializer serializer = new Serializer(); public final InetAddressAndPort addr; + public final String datacenter; - private PreInitialize(InetAddressAndPort addr) + private PreInitialize(InetAddressAndPort addr, String datacenter) { this.addr = addr; + this.datacenter = datacenter; } public static PreInitialize forTesting() { - return new PreInitialize(null); + return new PreInitialize(null, null); } public static PreInitialize blank() { - return new PreInitialize(null); + return new PreInitialize(null, null); } - public static PreInitialize withFirstCMS(InetAddressAndPort addr) + public static PreInitialize withFirstCMS(InetAddressAndPort addr, String datacenter) { - return new PreInitialize(addr); + return new PreInitialize(addr, datacenter); } public Kind kind() @@ -72,8 +77,24 @@ public Result execute(ClusterMetadata metadata) assert metadata.epoch.isBefore(Epoch.FIRST); ClusterMetadata.Transformer transformer = metadata.transformer(); + if (addr != null) { + // If addr != null, then this is being executed on the peer which is actually initializing the log + // for the very first time. + + // addr and datacenter are only used to bootstrap the replication of the distributed metatada + // keyspace on the first CMS node. They are never serialized into the distributed metadata log or + // passed to any other peer. + // + // PRE_INITIALIZE_CMS @ Epoch.FIRST, must be followed in the log by INITIALIZE_CMS @ (Epoch.FIRST + 1). + // The serialization of INITIALIZE_CMS includes the full ClusterMetadata at that point, which is + // obviously minimal, but will necessarily include the distributed metadata keyspace definition with + // the replication settings bootstrapped by PRE_INITIALIZE. This full ClusterMetadata becomes the + // starting point upon which further log entries are applied. So this means that once INITIALIZE_CMS + // has been committed to the log, the actual content of PRE_INITIALIZE_CMS is irrelevant, even on + // the first CMS node if it happens to replay it from its local storage after a restart. + DataPlacement.Builder dataPlacementBuilder = DataPlacement.builder(); Replica replica = new Replica(addr, MetaStrategy.partitioner.getMinimumToken(), @@ -81,10 +102,18 @@ public Result execute(ClusterMetadata metadata) true); dataPlacementBuilder.reads.withReplica(Epoch.FIRST, replica); dataPlacementBuilder.writes.withReplica(Epoch.FIRST, replica); - DataPlacements initialPlacement = metadata.placements.unbuild().with(ReplicationParams.meta(metadata), dataPlacementBuilder.build()).build(); + DataPlacements initialPlacement = metadata.placements.unbuild() + .with(ReplicationParams.simpleMeta(1, datacenter), + dataPlacementBuilder.build()).build(); transformer.with(initialPlacement); + // re-initialise the schema distributed metadata keyspace so it gets the + // correct replication settings based on the DC of the initial CMS node + Keyspaces updated = metadata.schema.getKeyspaces() + .withAddedOrReplaced(DistributedMetadataLogKeyspace.initialMetadata(datacenter)); + transformer.with(new DistributedSchema(updated)); } + ClusterMetadata.Transformer.Transformed transformed = transformer.build(); metadata = transformed.metadata.forceEpoch(Epoch.FIRST); assert metadata.epoch.is(Epoch.FIRST) : metadata.epoch; @@ -94,12 +123,10 @@ public Result execute(ClusterMetadata metadata) public String toString() { - return "PreInitialize{" + - "addr=" + addr + - '}'; + return "PreInitialize"; } - static class Serializer implements AsymmetricMetadataSerializer<Transformation, PreInitialize> + public static class Serializer implements AsymmetricMetadataSerializer<Transformation, PreInitialize> { public void serialize(Transformation t, DataOutputPlus out, Version version) throws IOException @@ -108,7 +135,9 @@ public void serialize(Transformation t, DataOutputPlus out, Version version) thr PreInitialize bcms = (PreInitialize)t; out.writeBoolean(bcms.addr != null); if (bcms.addr != null) - InetAddressAndPort.MetadataSerializer.serializer.serialize(((PreInitialize)t).addr, out, version); + InetAddressAndPort.MetadataSerializer.serializer.serialize(bcms.addr, out, version); + if (bcms.datacenter != null && version.isAtLeast(Version.V5)) + out.writeUTF(bcms.datacenter); } public PreInitialize deserialize(DataInputPlus in, Version version) throws IOException @@ -118,7 +147,8 @@ public PreInitialize deserialize(DataInputPlus in, Version version) throws IOExc return PreInitialize.blank(); InetAddressAndPort addr = InetAddressAndPort.MetadataSerializer.serializer.deserialize(in, version); - return new PreInitialize(addr); + String datacenter = version.isAtLeast(Version.V5) ? in.readUTF() : ""; + return new PreInitialize(addr, datacenter); } public long serializedSize(Transformation t, Version version) @@ -126,7 +156,11 @@ public long serializedSize(Transformation t, Version version) PreInitialize bcms = (PreInitialize)t; long size = TypeSizes.sizeof(bcms.addr != null); - return size + (bcms.addr != null ? InetAddressAndPort.MetadataSerializer.serializer.serializedSize(((PreInitialize)t).addr, version) : 0); + if (bcms.addr != null) + size += InetAddressAndPort.MetadataSerializer.serializer.serializedSize(bcms.addr, version); + if (bcms.datacenter != null && version.isAtLeast(Version.V5)) + size += TypeSizes.sizeof(bcms.datacenter); + return size; } } } \ No newline at end of file diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index 8191bb73ecc3..d0ba27ad51bc 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -92,6 +92,7 @@ import org.apache.cassandra.hints.HintsServiceMBean; import org.apache.cassandra.locator.DynamicEndpointSnitchMBean; import org.apache.cassandra.locator.EndpointSnitchInfoMBean; +import org.apache.cassandra.locator.LocationInfoMBean; import org.apache.cassandra.metrics.CIDRAuthorizerMetrics; import org.apache.cassandra.metrics.CassandraMetricsRegistry; import org.apache.cassandra.metrics.StorageMetrics; @@ -1219,6 +1220,18 @@ public DynamicEndpointSnitchMBean getDynamicEndpointSnitchInfoProxy() } } + public LocationInfoMBean getLocationInfoProxy() + { + try + { + return JMX.newMBeanProxy(mbeanServerConn, new ObjectName("org.apache.cassandra.db:type=LocationInfo"), LocationInfoMBean.class); + } + catch (MalformedObjectNameException e) + { + throw new RuntimeException(e); + } + } + public ColumnFamilyStoreMBean getCfsProxy(String ks, String cf) { ColumnFamilyStoreMBean cfsProxy = null; diff --git a/src/java/org/apache/cassandra/tools/nodetool/DescribeCluster.java b/src/java/org/apache/cassandra/tools/nodetool/DescribeCluster.java index 23100ea74e16..1904eeb50e03 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/DescribeCluster.java +++ b/src/java/org/apache/cassandra/tools/nodetool/DescribeCluster.java @@ -28,6 +28,7 @@ import io.airlift.airline.Command; import org.apache.cassandra.locator.DynamicEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.LocationInfoMBean; import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.tools.NodeTool; import org.apache.cassandra.tools.NodeTool.NodeToolCmd; @@ -46,15 +47,19 @@ public void execute(NodeProbe probe) // display cluster name, snitch and partitioner out.println("Cluster Information:"); out.println("\tName: " + probe.getClusterName()); - String snitch = probe.getEndpointSnitchInfoProxy().getSnitchName(); + LocationInfoMBean locationInfoProxy = probe.getLocationInfoProxy(); + String nodeProximityName = locationInfoProxy.getNodeProximityName(); boolean dynamicSnitchEnabled = false; - if (snitch.equals(DynamicEndpointSnitch.class.getName())) + boolean legacySnitchAdapter = locationInfoProxy.hasLegacySnitchAdapter(); + if (nodeProximityName.equals(DynamicEndpointSnitch.class.getName())) { - snitch = probe.getDynamicEndpointSnitchInfoProxy().getSubsnitchClassName(); + nodeProximityName = probe.getDynamicEndpointSnitchInfoProxy().getSubsnitchClassName(); dynamicSnitchEnabled = true; } - out.println("\tSnitch: " + snitch); + out.println("\tSnitch: " + nodeProximityName); out.println("\tDynamicEndPointSnitch: " + (dynamicSnitchEnabled ? "enabled" : "disabled")); + out.println("\tNodeProximity: " + nodeProximityName); + out.println("\tLegacySnitchAdapter: " + (legacySnitchAdapter ? "enabled" : "disabled")); out.println("\tPartitioner: " + probe.getPartitioner()); // display schema version for each node diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java index d52cfeb30b29..d1eb0798d896 100644 --- a/src/java/org/apache/cassandra/utils/FBUtilities.java +++ b/src/java/org/apache/cassandra/utils/FBUtilities.java @@ -224,6 +224,13 @@ public static InetAddressAndPort getLocalAddressAndPort() return localInetAddressAndPort; } + public static void setLocalAddress(InetAddress localAddress) + { + localInetAddress = localAddress; + // null out localInetAddressAndPort, it will be re-initalized next time it's accessed + localInetAddressAndPort = null; + } + /** * Retrieve just the broadcast address but not the port. This is almost always the wrong thing to be using because * it's ambiguous since you need the address and port to identify a node. You want getBroadcastAddressAndPort diff --git a/test/distributed/org/apache/cassandra/distributed/action/GossipHelper.java b/test/distributed/org/apache/cassandra/distributed/action/GossipHelper.java index e41b3eca5220..9e5e51fa43f7 100644 --- a/test/distributed/org/apache/cassandra/distributed/action/GossipHelper.java +++ b/test/distributed/org/apache/cassandra/distributed/action/GossipHelper.java @@ -43,7 +43,7 @@ import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.FBUtilities; -import static org.apache.cassandra.distributed.impl.DistributedTestSnitch.toCassandraInetAddressAndPort; +import static org.apache.cassandra.distributed.impl.TestEndpointCache.toCassandraInetAddressAndPort; public class GossipHelper { diff --git a/test/distributed/org/apache/cassandra/distributed/impl/DistributedTestInitialLocationProvider.java b/test/distributed/org/apache/cassandra/distributed/impl/DistributedTestInitialLocationProvider.java new file mode 100644 index 000000000000..af413797ac41 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/impl/DistributedTestInitialLocationProvider.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.impl; + + +import org.apache.cassandra.distributed.shared.NetworkTopology; +import org.apache.cassandra.locator.InitialLocationProvider; +import org.apache.cassandra.tcm.membership.Location; +import org.apache.cassandra.utils.FBUtilities; + +public class DistributedTestInitialLocationProvider implements InitialLocationProvider +{ + private static NetworkTopology mapping = null; + public static void assign(NetworkTopology topology) + { + mapping = topology; + } + + @Override + public Location initialLocation() + { + String dc = mapping.localDC(FBUtilities.getBroadcastAddressAndPort()); + String rack = mapping.localRack(FBUtilities.getBroadcastAddressAndPort()); + return new Location(dc, rack); + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/impl/DistributedTestSnitch.java b/test/distributed/org/apache/cassandra/distributed/impl/DistributedTestSnitch.java deleted file mode 100644 index 678af72b7fa5..000000000000 --- a/test/distributed/org/apache/cassandra/distributed/impl/DistributedTestSnitch.java +++ /dev/null @@ -1,128 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.distributed.impl; - -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -import org.apache.cassandra.config.Config; -import org.apache.cassandra.db.SystemKeyspace; -import org.apache.cassandra.distributed.shared.NetworkTopology; -import org.apache.cassandra.gms.ApplicationState; -import org.apache.cassandra.gms.EndpointState; -import org.apache.cassandra.gms.Gossiper; -import org.apache.cassandra.locator.AbstractNetworkTopologySnitch; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.utils.FBUtilities; - -public class DistributedTestSnitch extends AbstractNetworkTopologySnitch -{ - private static NetworkTopology mapping = null; - private static final Map<InetAddressAndPort, InetSocketAddress> cache = new ConcurrentHashMap<>(); - private static final Map<InetSocketAddress, InetAddressAndPort> cacheInverse = new ConcurrentHashMap<>(); - - public static InetAddressAndPort toCassandraInetAddressAndPort(InetSocketAddress addressAndPort) - { - InetAddressAndPort m = cacheInverse.get(addressAndPort); - if (m == null) - { - m = InetAddressAndPort.getByAddressOverrideDefaults(addressAndPort.getAddress(), addressAndPort.getPort()); - cache.put(m, addressAndPort); - } - return m; - } - - public static InetSocketAddress fromCassandraInetAddressAndPort(InetAddressAndPort addressAndPort) - { - InetSocketAddress m = cache.get(addressAndPort); - if (m == null) - { - m = NetworkTopology.addressAndPort(addressAndPort.getAddress(), addressAndPort.getPort()); - cache.put(addressAndPort, m); - } - return m; - } - - private Map<InetAddressAndPort, Map<String, String>> savedEndpoints; - private static final String DEFAULT_DC = "UNKNOWN_DC"; - private static final String DEFAULT_RACK = "UNKNOWN_RCK"; // TODO must be =< 12 chars to preserve nodetool output required by tests - - public String getRack(InetAddress endpoint) - { - int storage_port = Config.getOverrideLoadConfig().get().storage_port; - return getRack(InetAddressAndPort.getByAddressOverrideDefaults(endpoint, storage_port)); - } - - public String getRack(InetAddressAndPort endpoint) - { - assert mapping != null : "network topology must be assigned before using snitch"; - return maybeGetFromEndpointState(mapping.localRack(fromCassandraInetAddressAndPort(endpoint)), endpoint, ApplicationState.RACK, DEFAULT_RACK); - } - - public String getDatacenter(InetAddress endpoint) - { - int storage_port = Config.getOverrideLoadConfig().get().storage_port; - return getDatacenter(InetAddressAndPort.getByAddressOverrideDefaults(endpoint, storage_port)); - } - - public String getDatacenter(InetAddressAndPort endpoint) - { - assert mapping != null : "network topology must be assigned before using snitch"; - return maybeGetFromEndpointState(mapping.localDC(fromCassandraInetAddressAndPort(endpoint)), endpoint, ApplicationState.DC, DEFAULT_DC); - } - - // Here, the logic is slightly different from what we have in GossipingPropertyFileSnitch since we have a different - // goal. Passed argument (topology that was set on the node) overrides anything that is passed elsewhere. - private String maybeGetFromEndpointState(String current, InetAddressAndPort endpoint, ApplicationState state, String defaultValue) - { - if (current != null) - return current; - - EndpointState epState = Gossiper.instance.getEndpointStateForEndpoint(endpoint); - if (epState == null || epState.getApplicationState(state) == null) - { - if (savedEndpoints == null) - savedEndpoints = SystemKeyspace.loadDcRackInfo(); - if (savedEndpoints.containsKey(endpoint)) - return savedEndpoints.get(endpoint).get("data_center"); - - return defaultValue; - } - - return epState.getApplicationState(state).value; - } - - static void assign(NetworkTopology newMapping) - { - mapping = new NetworkTopology(newMapping); - } - - public void gossiperStarting() - { - super.gossiperStarting(); - - Gossiper.instance.addLocalApplicationState(ApplicationState.INTERNAL_ADDRESS_AND_PORT, - StorageService.instance.valueFactory.internalAddressAndPort(FBUtilities.getLocalAddressAndPort())); - Gossiper.instance.addLocalApplicationState(ApplicationState.INTERNAL_IP, - StorageService.instance.valueFactory.internalIP(FBUtilities.getJustLocalAddress())); - } -} diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java index 1d2272ad4d2a..df8acbfbd73f 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java @@ -140,6 +140,7 @@ import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.EpochAwareDebounce; +import org.apache.cassandra.tcm.RegistrationStatus; import org.apache.cassandra.tcm.Startup; import org.apache.cassandra.tcm.membership.NodeId; import org.apache.cassandra.tcm.membership.NodeState; @@ -178,8 +179,8 @@ import static org.apache.cassandra.distributed.api.Feature.JMX; import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL; import static org.apache.cassandra.distributed.api.Feature.NETWORK; -import static org.apache.cassandra.distributed.impl.DistributedTestSnitch.fromCassandraInetAddressAndPort; -import static org.apache.cassandra.distributed.impl.DistributedTestSnitch.toCassandraInetAddressAndPort; +import static org.apache.cassandra.distributed.impl.TestEndpointCache.fromCassandraInetAddressAndPort; +import static org.apache.cassandra.distributed.impl.TestEndpointCache.toCassandraInetAddressAndPort; import static org.apache.cassandra.net.Verb.BATCH_STORE_REQ; import static org.apache.cassandra.service.CassandraDaemon.logSystemInfo; @@ -622,7 +623,7 @@ public void startup(ICluster cluster) { assert config.networkTopology().contains(config.broadcastAddress()) : String.format("Network topology %s doesn't contain the address %s", config.networkTopology(), config.broadcastAddress()); - DistributedTestSnitch.assign(config.networkTopology()); + DistributedTestInitialLocationProvider.assign(config.networkTopology()); CassandraDaemon.getInstanceForTesting().activate(false); // TODO: filters won't work for the messages dispatched during startup registerInboundFilter(cluster); @@ -706,7 +707,7 @@ protected void partialStartup(ICluster<?> cluster) throws IOException, NoSuchFie assert config.networkTopology().contains(config.broadcastAddress()) : String.format("Network topology %s doesn't contain the address %s", config.networkTopology(), config.broadcastAddress()); - DistributedTestSnitch.assign(config.networkTopology()); + DistributedTestInitialLocationProvider.assign(config.networkTopology()); DatabaseDescriptor.daemonInitialization(); if (config.has(JMX)) startJmx(); @@ -811,6 +812,7 @@ protected void partialStartup(ICluster<?> cluster) throws IOException, NoSuchFie Schema.instance.saveSystemKeyspace(); ClusterMetadataService.instance().processor().fetchLogAndWait(); NodeId self = Register.maybeRegister(); + RegistrationStatus.instance.onRegistration(); boolean joinRing = config.get(Constants.KEY_DTEST_JOIN_RING) == null || (boolean) config.get(Constants.KEY_DTEST_JOIN_RING); if (ClusterMetadata.current().directory.peerState(self) != NodeState.JOINED && joinRing) { diff --git a/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java b/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java index 7f16d056801e..8afd90faf78b 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java @@ -37,6 +37,7 @@ import org.apache.cassandra.distributed.shared.NetworkTopology; import org.apache.cassandra.distributed.upgrade.UpgradeTestBase; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.NetworkTopologyProximity; import org.apache.cassandra.locator.SimpleSeedProvider; public class InstanceConfig implements IInstanceConfig @@ -105,7 +106,8 @@ private InstanceConfig(int num, .set("commitlog_sync_period_in_ms", 10000) .set("storage_port", storage_port) .set("native_transport_port", native_transport_port) - .set("endpoint_snitch", DistributedTestSnitch.class.getName()) + .set("initial_location_provider", DistributedTestInitialLocationProvider.class.getName()) + .set("node_proximity", NetworkTopologyProximity.class.getName()) .set("seed_provider", new ParameterizedClass(SimpleSeedProvider.class.getName(), Collections.singletonMap("seeds", seedIp + ':' + seedPort))) .set("discovery_timeout", "3s") @@ -184,7 +186,7 @@ private InstanceConfig(InstanceConfig copy) @Override public InetSocketAddress broadcastAddress() { - return DistributedTestSnitch.fromCassandraInetAddressAndPort(getBroadcastAddressAndPort()); + return TestEndpointCache.fromCassandraInetAddressAndPort(getBroadcastAddressAndPort()); } public void unsetBroadcastAddressAndPort() @@ -345,12 +347,22 @@ private static String[] datadirs(int datadirCount, Path root, int nodeNum) public InstanceConfig forVersion(Semver version) { // Versions before 4.0 need to set 'seed_provider' without specifying the port - if (UpgradeTestBase.v40.compareTo(version) < 0) + // Versions before 5.0 need to set 'endpoint_snitch', not initial_location_provider + node_proximity + if (version.compareTo(UpgradeTestBase.v51) >= 0) return this; - else - return new InstanceConfig(this) - .set("seed_provider", new ParameterizedClass(SimpleSeedProvider.class.getName(), - Collections.singletonMap("seeds", "127.0.0.1"))); + + InstanceConfig config = new InstanceConfig(this); + config.remove("initial_location_provider"); + config.remove("node_proximity"); + config.set("endpoint_snitch", "org.apache.cassandra.distributed.impl.DistributedTestSnitch"); + + // 4.0+ has seed_provider without port + if (version.compareTo(UpgradeTestBase.v40) >= 0) + return config; + + config.set("seed_provider", new ParameterizedClass(SimpleSeedProvider.class.getName(), + Collections.singletonMap("seeds", "127.0.0.1"))); + return config; } public String toString() diff --git a/test/distributed/org/apache/cassandra/distributed/impl/TestEndpointCache.java b/test/distributed/org/apache/cassandra/distributed/impl/TestEndpointCache.java new file mode 100644 index 000000000000..a7d6b06be186 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/impl/TestEndpointCache.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.impl; + +import java.net.InetSocketAddress; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.cassandra.distributed.shared.NetworkTopology; +import org.apache.cassandra.locator.InetAddressAndPort; + +public class TestEndpointCache +{ + private static final Map<InetAddressAndPort, InetSocketAddress> cache = new ConcurrentHashMap<>(); + private static final Map<InetSocketAddress, InetAddressAndPort> cacheInverse = new ConcurrentHashMap<>(); + + public static InetAddressAndPort toCassandraInetAddressAndPort(InetSocketAddress addressAndPort) + { + InetAddressAndPort m = cacheInverse.get(addressAndPort); + if (m == null) + { + m = InetAddressAndPort.getByAddressOverrideDefaults(addressAndPort.getAddress(), addressAndPort.getPort()); + cacheInverse.put(addressAndPort, m); + } + return m; + } + + public static InetSocketAddress fromCassandraInetAddressAndPort(InetAddressAndPort addressAndPort) + { + InetSocketAddress m = cache.get(addressAndPort); + if (m == null) + { + m = NetworkTopology.addressAndPort(addressAndPort.getAddress(), addressAndPort.getPort()); + cache.put(addressAndPort, m); + } + return m; + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/impl/UnsafeGossipHelper.java b/test/distributed/org/apache/cassandra/distributed/impl/UnsafeGossipHelper.java index c111946bedc5..377ac198802a 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/UnsafeGossipHelper.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/UnsafeGossipHelper.java @@ -42,7 +42,7 @@ import static com.google.common.collect.Iterables.getOnlyElement; import static java.util.Collections.singleton; -import static org.apache.cassandra.distributed.impl.DistributedTestSnitch.toCassandraInetAddressAndPort; +import static org.apache.cassandra.distributed.impl.TestEndpointCache.toCassandraInetAddressAndPort; import static org.apache.cassandra.locator.InetAddressAndPort.getByAddress; public class UnsafeGossipHelper diff --git a/test/distributed/org/apache/cassandra/distributed/mock/nodetool/InternalNodeProbe.java b/test/distributed/org/apache/cassandra/distributed/mock/nodetool/InternalNodeProbe.java index 2f9d628d5480..bf6a0879cedd 100644 --- a/test/distributed/org/apache/cassandra/distributed/mock/nodetool/InternalNodeProbe.java +++ b/test/distributed/org/apache/cassandra/distributed/mock/nodetool/InternalNodeProbe.java @@ -33,9 +33,11 @@ import org.apache.cassandra.gms.FailureDetectorMBean; import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.hints.HintsService; +import org.apache.cassandra.locator.DynamicEndpointSnitch; import org.apache.cassandra.locator.DynamicEndpointSnitchMBean; import org.apache.cassandra.locator.EndpointSnitchInfo; import org.apache.cassandra.locator.EndpointSnitchInfoMBean; +import org.apache.cassandra.locator.SnitchAdapter; import org.apache.cassandra.metrics.CassandraMetricsRegistry; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.service.ActiveRepairService; @@ -104,7 +106,8 @@ public EndpointSnitchInfoMBean getEndpointSnitchInfoProxy() @Override public DynamicEndpointSnitchMBean getDynamicEndpointSnitchInfoProxy() { - return (DynamicEndpointSnitchMBean) DatabaseDescriptor.createEndpointSnitch(true, DatabaseDescriptor.getRawConfig().endpoint_snitch); + // TODO At some point we should change this to use modern config e.g. Locator and InitialLocationProvider + return new DynamicEndpointSnitch(new SnitchAdapter(DatabaseDescriptor.createEndpointSnitch(DatabaseDescriptor.getRawConfig().endpoint_snitch))); } public CacheServiceMBean getCacheServiceMBean() diff --git a/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java b/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java index 4aeb630b6653..313dde0c5450 100644 --- a/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java +++ b/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java @@ -77,7 +77,6 @@ import org.apache.cassandra.tcm.Commit; import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.Transformation; -import org.apache.cassandra.tcm.log.Entry; import org.apache.cassandra.tcm.membership.NodeId; import org.apache.cassandra.tcm.ownership.ReplicaGroups; import org.apache.cassandra.utils.Isolated; @@ -89,7 +88,7 @@ import static org.apache.cassandra.config.CassandraRelevantProperties.BROADCAST_INTERVAL_MS; import static org.apache.cassandra.config.CassandraRelevantProperties.REPLACE_ADDRESS_FIRST_BOOT; import static org.apache.cassandra.config.CassandraRelevantProperties.RING_DELAY; -import static org.apache.cassandra.distributed.impl.DistributedTestSnitch.toCassandraInetAddressAndPort; +import static org.apache.cassandra.distributed.impl.TestEndpointCache.toCassandraInetAddressAndPort; import static org.assertj.core.api.Assertions.assertThat; /** diff --git a/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java b/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java index e66a85a3f2f7..b7f7c5ffb524 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java @@ -56,7 +56,7 @@ import static org.apache.cassandra.distributed.api.Feature.GOSSIP; import static org.apache.cassandra.distributed.api.Feature.NETWORK; import static org.apache.cassandra.distributed.api.TokenSupplier.evenlyDistributedTokens; -import static org.apache.cassandra.distributed.impl.DistributedTestSnitch.toCassandraInetAddressAndPort; +import static org.apache.cassandra.distributed.impl.TestEndpointCache.toCassandraInetAddressAndPort; import static org.apache.cassandra.distributed.shared.ClusterUtils.pauseBeforeCommit; import static org.apache.cassandra.distributed.shared.ClusterUtils.replaceHostAndStart; import static org.apache.cassandra.distributed.shared.ClusterUtils.stopUnchecked; diff --git a/test/distributed/org/apache/cassandra/distributed/test/IPMembershipTest.java b/test/distributed/org/apache/cassandra/distributed/test/IPMembershipTest.java index d72180f90213..9f242b0212d0 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/IPMembershipTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/IPMembershipTest.java @@ -34,6 +34,8 @@ import org.apache.cassandra.distributed.impl.InstanceConfig; import org.apache.cassandra.distributed.shared.ClusterUtils; import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.locator.NoOpProximity; +import org.apache.cassandra.locator.SimpleLocationProvider; import org.apache.cassandra.tools.ToolRunner; import org.assertj.core.api.Assertions; @@ -83,8 +85,9 @@ public void startupNewIP() throws IOException { try (Cluster cluster = Cluster.build(3) .withConfig(c -> c.with(Feature.GOSSIP, Feature.NATIVE_PROTOCOL) - // disable DistributedTestSnitch as it tries to query before we setup - .set("endpoint_snitch", "org.apache.cassandra.locator.SimpleSnitch")) + // disable DistributedTestInitialLocationProvider as it tries to query before we setup + .set("node_proximity", NoOpProximity.class.getName()) + .set("initial_location_provider", SimpleLocationProvider.class.getName())) .start()) { IInvokableInstance nodeToReplace = cluster.get(3); diff --git a/test/distributed/org/apache/cassandra/distributed/test/ReconnectToInternalIPTest.java b/test/distributed/org/apache/cassandra/distributed/test/ReconnectToInternalIPTest.java new file mode 100644 index 000000000000..6d798d5aa984 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/ReconnectToInternalIPTest.java @@ -0,0 +1,198 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.time.Duration; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.gms.ApplicationState; +import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector; +import org.apache.cassandra.locator.Ec2MultiRegionAddressConfig; +import org.apache.cassandra.locator.Ec2MultiRegionSnitch; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.InitialLocationProvider; +import org.apache.cassandra.locator.NetworkTopologyProximity; +import org.apache.cassandra.locator.SnitchProperties; +import org.apache.cassandra.tcm.membership.Location; +import org.awaitility.Awaitility; + +import static org.apache.cassandra.locator.Ec2LocationProvider.ZONE_NAME_QUERY; +import static org.apache.cassandra.locator.Ec2MultiRegionAddressConfig.PRIVATE_IP_QUERY; +import static org.apache.cassandra.locator.Ec2MultiRegionAddressConfig.PUBLIC_IP_QUERY; +import static org.apache.cassandra.utils.Pair.create; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class ReconnectToInternalIPTest extends TestBaseImpl +{ + @Test + public void testWithSnitchConfig() throws Exception + { + try (Cluster cluster = init(builder().withNodes(4) + .withConfig(c -> c.set("endpoint_snitch", TestMultiRegionSnitch.class.getName()) + .set("node_proximity", null) + .set("initial_location_provider", null) + .set("listen_on_broadcast_address", true) + .with(Feature.NETWORK, Feature.GOSSIP)) + .start())) + { + doTest(cluster); + } + } + + @Test + public void testWithModernConfig() throws Exception + { + try (Cluster cluster = init(builder().withNodes(4) + .withConfig(c -> c.set("endpoint_snitch", null) + .set("node_proximity", NetworkTopologyProximity.class.getName()) + .set("initial_location_provider", TestMultiRegionLocationProvider.class.getName()) + .set("listen_on_broadcast_address", true) + .set("addresses_config", TestMultiRegionAddressConfig.class.getName()) + .set("prefer_local_connections", true) + .with(Feature.NETWORK, Feature.GOSSIP)) + .start())) + { + doTest(cluster); + } + } + + private static void doTest(Cluster cluster) throws TimeoutException + { + cluster.schemaChange(withKeyspace("create table %s.tbl (id int primary key)")); + cluster.coordinator(1).execute(withKeyspace("insert into %s.tbl (id) values (1)"), ConsistencyLevel.ALL); + // node1 should only reconnect to node2, but currently this is still driven by gossip so may not happen immediately + InetSocketAddress ep = cluster.get(2).config().broadcastAddress(); + String pattern = "Initiated reconnect to an Internal IP "+toInternalIp(ep)+" for the " + ep; + cluster.get(1).logs().watchFor(Duration.ofSeconds(10), pattern); + + // node1 should not reconnect to node3 or node4 + for (int i = 3; i <= 4; i++) + { + ep = cluster.get(i).config().broadcastAddress(); + pattern = "Initiated reconnect to an Internal IP "+toInternalIp(ep)+" for the " + ep; + assertTrue(cluster.get(1).logs().grep(pattern).getResult().isEmpty()); + } + + cluster.forEach(inst -> inst.runOnInstance(() -> { + for (InetAddressAndPort endpoint : Gossiper.instance.endpointStateMap.keySet()) + { + InetAddressAndPort internal = toInternalIp(endpoint); + Awaitility.await() + .atMost(30L, TimeUnit.SECONDS) + .until(() -> Gossiper.instance.getApplicationState(endpoint, ApplicationState.INTERNAL_ADDRESS_AND_PORT) != null); + InetAddressAndPort fromGossip = InetAddressAndPort.getByNameUnchecked(Gossiper.instance.getApplicationState(endpoint, ApplicationState.INTERNAL_ADDRESS_AND_PORT)); + assertEquals(internal, fromGossip); + } + })); + } + + public static class TestMultiRegionLocationProvider implements InitialLocationProvider + { + @Override + public Location initialLocation() + { + InetAddressAndPort configuredBA = InetAddressAndPort.getByNameUnchecked(DatabaseDescriptor.getRawConfig().broadcast_address); + byte lastByte = configuredBA.addressBytes[configuredBA.addressBytes.length - 1]; + String zone = (lastByte == 1 || lastByte == 2) ? "us-east-1" : "us-west-1"; + return new Location(zone, zone + 'a'); + } + } + + public static class TestMultiRegionAddressConfig extends Ec2MultiRegionAddressConfig + { + public TestMultiRegionAddressConfig() throws IOException + { + super(new TestCloudMetadataConnector()); + } + } + + public static class TestMultiRegionSnitch extends Ec2MultiRegionSnitch + { + public TestMultiRegionSnitch() throws IOException, ConfigurationException + { + super(new TestCloudMetadataConnector()); + } + } + + public static class TestCloudMetadataConnector extends AbstractCloudMetadataServiceConnector + { + public TestCloudMetadataConnector() + { + super(new SnitchProperties(create(METADATA_URL_PROPERTY, "http://apache.org"))); + } + + @Override + public String apiCall(String url, + String query, + String method, + Map<String, String> extraHeaders, + int expectedResponseCode) + { + InetAddressAndPort configuredBA = InetAddressAndPort.getByNameUnchecked(DatabaseDescriptor.getRawConfig().broadcast_address); + switch (query) + { + case ZONE_NAME_QUERY: + // "us-east-1a" + byte lastByte = configuredBA.addressBytes[configuredBA.addressBytes.length - 1]; + if (lastByte == 1 || lastByte == 2) + return "us-east-1a"; + else + return "us-west-1a"; + case PUBLIC_IP_QUERY: + // public ip is configured ip "+ 4" + return toPublicIp(configuredBA).getHostAddress(false); + case PRIVATE_IP_QUERY: + // private ip is configured ip "+ 8" + return toInternalIp(configuredBA).getHostAddress(false); + default: + throw new AssertionError("Bad query: " + query); + } + } + } + + private static InetAddressAndPort toInternalIp(InetSocketAddress ep) + { + return convertIp(ep, 8); + } + + private static InetAddressAndPort toPublicIp(InetSocketAddress ep) + { + return convertIp(ep, 4); + } + + private static InetAddressAndPort convertIp(InetSocketAddress ep, int offset) + { + byte lastByte = ep.getAddress().getAddress()[ep.getAddress().getAddress().length - 1]; + return InetAddressAndPort.getByNameUnchecked("127.0.0." + (lastByte + offset)+":"+ep.getPort()); + } + +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/BounceResetHostIdTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/BounceResetHostIdTest.java index e593a71b6895..0a8a31e3a26c 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/BounceResetHostIdTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/BounceResetHostIdTest.java @@ -32,6 +32,8 @@ import org.apache.cassandra.distributed.shared.AssertUtils; import org.apache.cassandra.distributed.shared.ClusterUtils; import org.apache.cassandra.distributed.test.TestBaseImpl; +import org.apache.cassandra.locator.NoOpProximity; +import org.apache.cassandra.locator.SimpleLocationProvider; import org.apache.cassandra.tcm.membership.NodeId; import static org.apache.cassandra.distributed.shared.AssertUtils.row; @@ -44,8 +46,9 @@ public void swapIpsTest() throws Exception { try (Cluster cluster = builder().withNodes(3) .withConfig(c -> c.with(Feature.GOSSIP, Feature.NATIVE_PROTOCOL) - // disable DistributedTestSnitch as it tries to query before we setup - .set("endpoint_snitch", "org.apache.cassandra.locator.SimpleSnitch")) + // disable DistributedTestInitialLocationProvider as it tries to query before we setup + .set("node_proximity", NoOpProximity.class.getName()) + .set("initial_location_provider", SimpleLocationProvider.class.getName())) .createWithoutStarting()) { // This test relies on node IDs being in the same order as IP addresses @@ -97,8 +100,9 @@ public void swapIpsDirectlyTest() throws Exception { try (Cluster cluster = builder().withNodes(3) .withConfig(c -> c.with(Feature.GOSSIP, Feature.NATIVE_PROTOCOL) - // disable DistributedTestSnitch as it tries to query before we setup - .set("endpoint_snitch", "org.apache.cassandra.locator.SimpleSnitch")) + // disable DistributedTestInitialLocationProvider as it tries to query before we setup + .set("node_proximity", NoOpProximity.class.getName()) + .set("initial_location_provider", SimpleLocationProvider.class.getName())) .createWithoutStarting()) { // This test relies on node IDs being in the same order as IP addresses diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/CMSMembershipMetricsTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/CMSMembershipMetricsTest.java index a165f39848ea..58a33400b7cd 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/CMSMembershipMetricsTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/CMSMembershipMetricsTest.java @@ -26,7 +26,7 @@ import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.api.Feature; import org.apache.cassandra.distributed.api.IInvokableInstance; -import org.apache.cassandra.distributed.impl.DistributedTestSnitch; +import org.apache.cassandra.distributed.impl.TestEndpointCache; import org.apache.cassandra.distributed.test.TestBaseImpl; import org.apache.cassandra.gms.FailureDetector; import org.apache.cassandra.metrics.TCMMetrics; @@ -106,14 +106,14 @@ void assertCMSMembership(int expected, IInvokableInstance... instances) private void markDown(IInvokableInstance down, IInvokableInstance inst) { InetSocketAddress node3Address = down.config().broadcastAddress(); - inst.runOnInstance(() -> FailureDetector.instance.forceConviction(DistributedTestSnitch.toCassandraInetAddressAndPort(node3Address))); + inst.runOnInstance(() -> FailureDetector.instance.forceConviction(TestEndpointCache.toCassandraInetAddressAndPort(node3Address))); } private void markUp(IInvokableInstance down, IInvokableInstance inst) { InetSocketAddress downAddress = down.config().broadcastAddress(); - inst.runOnInstance(() -> FailureDetector.instance.report(DistributedTestSnitch.toCassandraInetAddressAndPort(downAddress))); + inst.runOnInstance(() -> FailureDetector.instance.report(TestEndpointCache.toCassandraInetAddressAndPort(downAddress))); Awaitility.waitAtMost(10, TimeUnit.SECONDS) - .until(() -> inst.callOnInstance(() -> FailureDetector.instance.isAlive(DistributedTestSnitch.toCassandraInetAddressAndPort(downAddress)))); + .until(() -> inst.callOnInstance(() -> FailureDetector.instance.isAlive(TestEndpointCache.toCassandraInetAddressAndPort(downAddress)))); } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/CMSTestBase.java b/test/distributed/org/apache/cassandra/distributed/test/log/CMSTestBase.java index ffc7f96bbac5..538bf4f491e8 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/CMSTestBase.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/CMSTestBase.java @@ -25,11 +25,7 @@ import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.distributed.api.IIsolatedExecutor; import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.locator.IEndpointSnitch; -import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.MetaStrategy; -import org.apache.cassandra.locator.Replica; -import org.apache.cassandra.locator.ReplicaCollection; import org.apache.cassandra.schema.DistributedSchema; import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.Keyspaces; @@ -50,23 +46,6 @@ public class CMSTestBase static { DatabaseDescriptor.daemonInitialization(); - DatabaseDescriptor.setEndpointSnitch(new IEndpointSnitch() - { - public String getRack(InetAddressAndPort endpoint) - { - ClusterMetadata metadata = ClusterMetadata.current(); - return metadata.directory.location(metadata.directory.peerId(endpoint)).rack; - } - public String getDatacenter(InetAddressAndPort endpoint) - { - ClusterMetadata metadata = ClusterMetadata.current(); - return metadata.directory.location(metadata.directory.peerId(endpoint)).datacenter; - } - public <C extends ReplicaCollection<? extends C>> C sortedByProximity(InetAddressAndPort address, C addresses) {return null;} - public int compareEndpoints(InetAddressAndPort target, Replica r1, Replica r2) {return 0;} - public void gossiperStarting() {} - public boolean isWorthMergingForRangeQuery(ReplicaCollection<?> merged, ReplicaCollection<?> l1, ReplicaCollection<?> l2) {return false;} - }); DatabaseDescriptor.setDefaultKeyspaceRF(1); Guardrails.instance.setMinimumReplicationFactorThreshold(1, 1); @@ -107,7 +86,7 @@ public CMSSut(IIsolatedExecutor.SerializableFunction<LocalLog, Processor> proces ClusterMetadataService.setInstance(service); log.readyUnchecked(); - log.bootstrap(FBUtilities.getBroadcastAddressAndPort()); + log.unsafeBootstrapForTesting(FBUtilities.getBroadcastAddressAndPort()); service.commit(new Initialize(ClusterMetadata.current()) { public Result execute(ClusterMetadata prev) { diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/ClusterMetadataTestHelper.java b/test/distributed/org/apache/cassandra/distributed/test/log/ClusterMetadataTestHelper.java index b27952d284a6..3c822864c570 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/ClusterMetadataTestHelper.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/ClusterMetadataTestHelper.java @@ -60,6 +60,7 @@ import org.apache.cassandra.tcm.Commit; import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.MetadataSnapshots; +import org.apache.cassandra.tcm.RegistrationStatus; import org.apache.cassandra.tcm.Transformation; import org.apache.cassandra.tcm.log.LocalLog; import org.apache.cassandra.tcm.membership.Directory; @@ -135,7 +136,7 @@ public static ClusterMetadataService instanceForTest() Commit.Replicator.NO_OP, true); log.readyUnchecked(); - log.bootstrap(FBUtilities.getBroadcastAddressAndPort()); + log.unsafeBootstrapForTesting(FBUtilities.getBroadcastAddressAndPort()); QueryProcessor.registerStatementInvalidatingListener(); service.mark(); return service; @@ -281,11 +282,19 @@ public static NodeId register(int nodeIdx, String dc, String rack) return register(addr(nodeIdx), dc, rack); } + public static NodeId register(InetAddressAndPort endpoint, Location location) + { + return register(endpoint, location.datacenter, location.rack); + } + public static NodeId register(InetAddressAndPort endpoint, String dc, String rack) { try { - return commit(new Register(addr(endpoint), new Location(dc, rack), NodeVersion.CURRENT)).directory.peerId(endpoint); + NodeId id = commit(new Register(addr(endpoint), new Location(dc, rack), NodeVersion.CURRENT)).directory.peerId(endpoint); + if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort())) + RegistrationStatus.instance.onRegistration(); + return id; } catch (Throwable e) { @@ -788,6 +797,11 @@ public static void addEndpoint(InetAddressAndPort endpoint, Collection<Token> to addEndpoint(endpoint, tokens, "dc1", "rack1"); } + public static void addEndpoint(InetAddressAndPort endpoint, Collection<Token> tokens, Location location) + { + addEndpoint(endpoint, tokens, location.datacenter, location.rack); + } + public static void addEndpoint(InetAddressAndPort endpoint, Token t, Location location) { addEndpoint(endpoint, Collections.singleton(t), location.datacenter, location.rack); @@ -804,6 +818,8 @@ public static void addEndpoint(InetAddressAndPort endpoint, Collection<Token> t, { Location l = new Location(dc, rack); commit(new Register(addr(endpoint), l, NodeVersion.CURRENT)); + if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort())) + RegistrationStatus.instance.onRegistration(); lazyJoin(endpoint, new HashSet<>(t)).prepareJoin() .startJoin() .midJoin() diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/CoordinatorPathTestBase.java b/test/distributed/org/apache/cassandra/distributed/test/log/CoordinatorPathTestBase.java index 64c4e629e3f2..af9f1806bf3c 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/CoordinatorPathTestBase.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/CoordinatorPathTestBase.java @@ -664,7 +664,7 @@ public void initWithFakeCms(Node cms, InetAddressAndPort nodeUnderTest) true); ClusterMetadataService.setInstance(service); log.readyUnchecked(); - log.bootstrap(cms.addr()); + log.unsafeBootstrapForTesting(cms.addr()); service.commit(new Initialize(log.metadata())); service.commit(new Register(new NodeAddresses(cms.addr()), new Location(cms.dc(), cms.rack()), NodeVersion.CURRENT)); diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/MetadataChangeSimulationTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/MetadataChangeSimulationTest.java index 931ee320e52a..7e2d1880a27b 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/MetadataChangeSimulationTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/MetadataChangeSimulationTest.java @@ -83,65 +83,12 @@ public class MetadataChangeSimulationTest extends CMSTestBase { private static final Logger logger = LoggerFactory.getLogger(MetadataChangeSimulationTest.class); - private static final long seed; - private static final Random rng; static { - seed = System.nanoTime(); - logger.info("SEED: {}", seed); - rng = new Random(seed); DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); DatabaseDescriptor.setTransientReplicationEnabledUnsafe(true); } - - @Test - public void simulateNTS() throws Throwable - { - // TODO: right now, we pick a candidate only if there is enough rf to execute operation - // but the problem is that if we start multiple operations that would take us under rf, we will screw up the placements - // this was not happening before, and test is crafted now to disallow such states, but this is a bug. - // we should either forbid this, or allow it, but make it work. - for (int concurrency : new int[]{ 1, 3, 5 }) - { - for (int rf : new int[]{ 2, 3, 5 }) - { - for (int trans = 0; trans < rf; trans++) - { - simulate(50, 0, new NtsReplicationFactor(3, rf, trans), concurrency); - } - } - } - } - - @Test - public void simulateSimple() throws Throwable - { - for (int concurrency : new int[]{ 1, 3, 5 }) - { - for (int rf : new int[]{ 2, 3, 5 }) - { - for (int trans = 0; trans < rf; trans++) - { - simulate(50, 0, new SimpleReplicationFactor(rf, trans), concurrency); - } - } - } - } - - @Test - public void simulateSimpleOneTransient() throws Throwable - { - DatabaseDescriptor.setTransientReplicationEnabledUnsafe(true); - simulate(50, 0, new SimpleReplicationFactor(5, 2), 1); - } - - @Test - public void simulateSimpleOneNonTransient() throws Throwable - { - simulate(50, 0, new SimpleReplicationFactor(3), 1); - } - @Test public void testMoveReal() throws Throwable { @@ -395,7 +342,7 @@ public void testReplaceReal(ReplicationFactor rf, int replacementId) throws Thro } } - public void simulate(int toBootstrap, int minSteps, ReplicationFactor rf, int concurrency) throws Throwable + public static void simulate(long seed, Random rng, int toBootstrap, int minSteps, ReplicationFactor rf, int concurrency) throws Throwable { logger.info("RUNNING SIMULATION WITH SEED {}. TO BOOTSTRAP: {}, RF: {}, CONCURRENCY: {}", seed, toBootstrap, rf, concurrency); long startTime = System.currentTimeMillis(); @@ -533,7 +480,7 @@ public void simulateDCAwareBounces() throws Throwable } } - public void simulateBounces(ReplicationFactor rf, CMSPlacementStrategy CMSConfigurationStrategy, Random random) throws Throwable + public static void simulateBounces(ReplicationFactor rf, CMSPlacementStrategy CMSConfigurationStrategy, Random random) throws Throwable { try(CMSSut sut = new CMSSut(AtomicLongBackedProcessor::new, false, rf)) { @@ -631,7 +578,7 @@ public static ModelChecker.Pair<ModelState, Node> registerNewNode(ModelState sta return pair(newState, node); } - private ModelChecker.Pair<ModelState, Node> registerNewNode(ModelState state, CMSSut sut, int tokenIdx, int dcIdx, int rackIdx) + private static ModelChecker.Pair<ModelState, Node> registerNewNode(ModelState state, CMSSut sut, int tokenIdx, int dcIdx, int rackIdx) { ModelState newState = state.transformer().incrementUniqueNodes().transform(); Node node = state.nodeFactory.make(newState.uniqueNodes, dcIdx, rackIdx).withToken(tokenIdx); @@ -639,7 +586,7 @@ private ModelChecker.Pair<ModelState, Node> registerNewNode(ModelState state, CM return pair(newState, node); } - private ModelChecker.Pair<ModelState, Node> registerNewNodeWithToken(ModelState state, CMSSut sut, long token, int dcIdx, int rackIdx) + private static ModelChecker.Pair<ModelState, Node> registerNewNodeWithToken(ModelState state, CMSSut sut, long token, int dcIdx, int rackIdx) { ModelState newState = state.transformer().incrementUniqueNodes().transform(); Node node = state.nodeFactory.make(newState.uniqueNodes, dcIdx, rackIdx).overrideToken(token); @@ -647,17 +594,17 @@ private ModelChecker.Pair<ModelState, Node> registerNewNodeWithToken(ModelState return pair(newState, node); } - private Node getRemovalCandidate(ModelState state, EntropySource entropySource) + private static Node getRemovalCandidate(ModelState state, EntropySource entropySource) { return getCandidate(state, entropySource); } - private Node getMoveCandidate(ModelState state, EntropySource entropySource) + private static Node getMoveCandidate(ModelState state, EntropySource entropySource) { return getCandidate(state, entropySource); } - private Node getCandidate(ModelState modelState, EntropySource entropySource) + private static Node getCandidate(ModelState modelState, EntropySource entropySource) { List<String> dcs = new ArrayList<>(modelState.simulatedPlacements.rf.asMap().keySet()); while (!dcs.isEmpty()) diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/NTSSimulationTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/NTSSimulationTest.java new file mode 100644 index 000000000000..6e055b257cfd --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/log/NTSSimulationTest.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.log; + +import java.util.Random; + +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.dht.Murmur3Partitioner; + +import static org.apache.cassandra.distributed.test.log.MetadataChangeSimulationTest.simulate; +import static org.apache.cassandra.harry.sut.TokenPlacementModel.NtsReplicationFactor; + +public class NTSSimulationTest extends CMSTestBase +{ + private static final Logger logger = LoggerFactory.getLogger(NTSSimulationTest.class); + private long seed; + private Random rng; + static + { + DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); + DatabaseDescriptor.setTransientReplicationEnabledUnsafe(true); + } + + @Before + public void setup() + { + seed = System.nanoTime(); + logger.info("SEED: {}", seed); + rng = new Random(seed); + } + + @Test + public void simulateNTS() throws Throwable + { + // TODO: right now, we pick a candidate only if there is enough rf to execute operation + // but the problem is that if we start multiple operations that would take us under rf, we will screw up the placements + // this was not happening before, and test is crafted now to disallow such states, but this is a bug. + // we should either forbid this, or allow it, but make it work. + for (int concurrency : new int[]{ 1, 3, 5 }) + { + for (int rf : new int[]{ 2, 3, 5 }) + { + for (int trans = 0; trans < rf; trans++) + { + simulate(seed, rng, 50, 0, new NtsReplicationFactor(3, rf, trans), concurrency); + } + } + } + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/SimpleStrategySimulationTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/SimpleStrategySimulationTest.java new file mode 100644 index 000000000000..0aaf8737ebae --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/log/SimpleStrategySimulationTest.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.log; + +import java.util.Random; + +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.dht.Murmur3Partitioner; + +import static org.apache.cassandra.distributed.test.log.MetadataChangeSimulationTest.simulate; +import static org.apache.cassandra.harry.sut.TokenPlacementModel.SimpleReplicationFactor; + +public class SimpleStrategySimulationTest extends CMSTestBase +{ + private static final Logger logger = LoggerFactory.getLogger(SimpleStrategySimulationTest.class); + private long seed; + private Random rng; + static + { + DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); + DatabaseDescriptor.setTransientReplicationEnabledUnsafe(true); + } + + @Before + public void setup() + { + seed = System.nanoTime(); + logger.info("SEED: {}", seed); + rng = new Random(seed); + } + + + @Test + public void simulateSimple() throws Throwable + { + for (int concurrency : new int[]{ 1, 3, 5 }) + { + for (int rf : new int[]{ 2, 3, 5 }) + { + for (int trans = 0; trans < rf; trans++) + { + simulate(seed, rng, 50, 0, new SimpleReplicationFactor(rf, trans), concurrency); + } + } + } + } + + @Test + public void simulateSimpleOneTransient() throws Throwable + { + DatabaseDescriptor.setTransientReplicationEnabledUnsafe(true); + simulate(seed, rng, 50, 0, new SimpleReplicationFactor(5, 2), 1); + } + + @Test + public void simulateSimpleOneNonTransient() throws Throwable + { + simulate(seed, rng, 50, 0, new SimpleReplicationFactor(3), 1); + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/topology/DecommissionAvoidTimeouts.java b/test/distributed/org/apache/cassandra/distributed/test/topology/DecommissionAvoidTimeouts.java index 821b6a1cefd2..69ccafdd458e 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/topology/DecommissionAvoidTimeouts.java +++ b/test/distributed/org/apache/cassandra/distributed/test/topology/DecommissionAvoidTimeouts.java @@ -109,7 +109,7 @@ public void test() throws Exception pending.call(); unpauseCommits(cluster.get(1)); - cluster.forEach(i -> i.runOnInstance(() -> ((DynamicEndpointSnitch) DatabaseDescriptor.getEndpointSnitch()).updateScores())); + cluster.forEach(i -> i.runOnInstance(() -> ((DynamicEndpointSnitch) DatabaseDescriptor.getNodeProximity()).updateScores())); cluster.filters().verbs(Verb.GOSSIP_DIGEST_SYN.id).drop(); nodetool.join(); diff --git a/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java b/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java index bc7eb3b17bc5..1f3edcf446da 100644 --- a/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java +++ b/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java @@ -54,8 +54,8 @@ public void testConcurrency() throws InterruptedException, IOException, Configur // do this because SS needs to be initialized before DES can work properly. StorageService.instance.unsafeInitialize(); - SimpleSnitch ss = new SimpleSnitch(); - DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(ss, String.valueOf(ss.hashCode())); + NodeProximity proximity = new NoOpProximity(); + DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(proximity, String.valueOf(proximity.hashCode())); InetAddressAndPort self = FBUtilities.getBroadcastAddressAndPort(); EndpointsForRange.Builder replicasBuilder = EndpointsForRange.builder(ReplicaUtils.FULL_RANGE); diff --git a/test/simulator/main/org/apache/cassandra/simulator/systems/SimulatedSnitch.java b/test/simulator/main/org/apache/cassandra/simulator/systems/SimulatedSnitch.java index 4000946a949b..55fe73c301f4 100644 --- a/test/simulator/main/org/apache/cassandra/simulator/systems/SimulatedSnitch.java +++ b/test/simulator/main/org/apache/cassandra/simulator/systems/SimulatedSnitch.java @@ -104,6 +104,8 @@ public Cluster.Builder setup(Cluster.Builder builder) if (prev != null) prev.accept(config); config.set("endpoint_snitch", SimulatedSnitch.Instance.class.getName()) + .set("initial_location_provider", null) + .set("node_proximity", null) .set("dynamic_snitch", false); }); } diff --git a/test/unit/org/apache/cassandra/ServerTestUtils.java b/test/unit/org/apache/cassandra/ServerTestUtils.java index 29e1a84a1e1d..596312ff0b11 100644 --- a/test/unit/org/apache/cassandra/ServerTestUtils.java +++ b/test/unit/org/apache/cassandra/ServerTestUtils.java @@ -44,9 +44,9 @@ import org.apache.cassandra.io.sstable.format.big.BigTableReader; import org.apache.cassandra.io.sstable.indexsummary.IndexSummarySupport; import org.apache.cassandra.io.util.File; -import org.apache.cassandra.locator.AbstractEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.Replica; +import org.apache.cassandra.locator.BaseProximity; import org.apache.cassandra.security.ThreadAwareSecurityManager; import org.apache.cassandra.service.EmbeddedCassandraService; import org.apache.cassandra.tcm.AtomicLongBackedProcessor; @@ -56,9 +56,11 @@ import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.MetadataSnapshots; import org.apache.cassandra.tcm.Processor; +import org.apache.cassandra.tcm.RegistrationStatus; import org.apache.cassandra.tcm.log.LocalLog; import org.apache.cassandra.tcm.log.LogStorage; import org.apache.cassandra.tcm.log.SystemKeyspaceStorage; +import org.apache.cassandra.tcm.membership.Location; import org.apache.cassandra.tcm.membership.NodeId; import org.apache.cassandra.tcm.ownership.PlacementProvider; import org.apache.cassandra.tcm.ownership.UniformRangePlacement; @@ -98,23 +100,8 @@ public static void daemonInitialization() public static void initSnitch() { // Register an EndpointSnitch which returns fixed values for test. - DatabaseDescriptor.setEndpointSnitch(new AbstractEndpointSnitch() + DatabaseDescriptor.setNodeProximity(new BaseProximity() { - @Override - public String getRack(InetAddressAndPort endpoint) - { - return RACK1; - } - - @Override - public String getDatacenter(InetAddressAndPort endpoint) - { - if (remoteAddrs.contains(endpoint)) - return DATA_CENTER_REMOTE; - - return DATA_CENTER; - } - @Override public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2) { @@ -135,6 +122,7 @@ public static NodeId registerLocal(Set<Token> tokens) tokens, ClusterMetadataService.instance().placementProvider())); SystemKeyspace.setLocalHostId(nodeId.toUUID()); + RegistrationStatus.instance.onRegistration(); return nodeId; } @@ -274,6 +262,7 @@ public static void initCMS() Function<LocalLog, Processor> processorFactory = AtomicLongBackedProcessor::new; IPartitioner partitioner = DatabaseDescriptor.getPartitioner(); + Location location = DatabaseDescriptor.getLocator().local(); boolean addListeners = true; ClusterMetadata initial = new ClusterMetadata(partitioner); if (!Keyspace.isInitialized()) @@ -293,7 +282,7 @@ public static void initCMS() ClusterMetadataService.setInstance(service); log.readyUnchecked(); - log.bootstrap(FBUtilities.getBroadcastAddressAndPort()); + log.bootstrap(FBUtilities.getBroadcastAddressAndPort(), location.datacenter); service.commit(new Initialize(ClusterMetadata.current())); QueryProcessor.registerStatementInvalidatingListener(); service.mark(); @@ -328,7 +317,7 @@ public static void recreateCMS() ClusterMetadataService.setInstance(cms); ((SystemKeyspaceStorage)LogStorage.SystemKeyspace).truncate(); log.readyUnchecked(); - log.bootstrap(FBUtilities.getBroadcastAddressAndPort()); + log.unsafeBootstrapForTesting(FBUtilities.getBroadcastAddressAndPort()); cms.mark(); } diff --git a/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java b/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java index c5041e6e7033..4b3e04813406 100644 --- a/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java +++ b/test/unit/org/apache/cassandra/batchlog/BatchlogEndpointFilterTest.java @@ -30,11 +30,9 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.ImmutableMultimap; import com.google.common.collect.Iterables; import com.google.common.collect.Multimap; -import com.google.common.collect.Multimaps; import org.junit.After; import org.junit.Assert; @@ -44,14 +42,16 @@ import org.apache.cassandra.config.Config; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; import org.apache.cassandra.locator.DynamicEndpointSnitch; -import org.apache.cassandra.locator.GossipingPropertyFileSnitch; -import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.NoOpProximity; +import org.apache.cassandra.locator.NodeProximity; import org.apache.cassandra.locator.ReplicaPlans; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.StubClusterMetadataService; +import org.apache.cassandra.tcm.membership.Location; import org.apache.cassandra.utils.FBUtilities; import static java.util.concurrent.TimeUnit.MILLISECONDS; @@ -275,7 +275,8 @@ private Collection<InetAddressAndPort> filterBatchlogEndpointsForTests(Multimap< filterBatchlogEndpointsRandomForTests(false, endpoints); } - private Collection<InetAddressAndPort> filterBatchlogEndpointsDynamicForTests(Multimap<String, InetAddressAndPort> endpoints) { + private Collection<InetAddressAndPort> filterBatchlogEndpointsDynamicForTests(Multimap<String, InetAddressAndPort> endpoints) + { return ReplicaPlans.filterBatchlogEndpointsDynamic(false, LOCAL, endpoints, x -> true); } @@ -972,57 +973,42 @@ private Multimap<String, InetAddressAndPort> configure(Config.BatchlogEndpointSt assert !racks.isEmpty(); assert racks.size() <= 10; assert racks.size() == nodesPerRack.length; - + ClusterMetadataService.unsetInstance(); + ClusterMetadataService.setInstance(ClusterMetadataTestHelper.instanceForTest()); ImmutableMultimap.Builder<String, InetAddressAndPort> builder = ImmutableMultimap.builder(); for (int r = 0; r < racks.size(); r++) { String rack = racks.get(r); for (int n = 0; n < nodesPerRack[r]; n++) + { builder.put(rack, endpointAddress(r, n)); + ClusterMetadataTestHelper.register(endpointAddress(r, n), new Location("dc1", rack)); + } } ImmutableMultimap<String, InetAddressAndPort> endpoints = builder.build(); - reconfigure(batchlogEndpointStrategy, dynamicSnitch, endpoints); + reconfigure(batchlogEndpointStrategy, dynamicSnitch); return endpoints; } private void reconfigure(Config.BatchlogEndpointStrategy batchlogEndpointStrategy, - boolean dynamicSnitch, - Multimap<String, InetAddressAndPort> endpoints) + boolean dynamicSnitch) { DatabaseDescriptor.setBatchlogEndpointStrategy(batchlogEndpointStrategy); - if (DatabaseDescriptor.getEndpointSnitch() instanceof DynamicEndpointSnitch) - ((DynamicEndpointSnitch) DatabaseDescriptor.getEndpointSnitch()).close(); + if (DatabaseDescriptor.getNodeProximity() instanceof DynamicEndpointSnitch) + ((DynamicEndpointSnitch) DatabaseDescriptor.getNodeProximity()).close(); - Multimap<InetAddressAndPort, String> endpointRacks = Multimaps.invertFrom(endpoints, ArrayListMultimap.create()); - GossipingPropertyFileSnitch gpfs = new GossipingPropertyFileSnitch() - { - @Override - public String getDatacenter(InetAddressAndPort endpoint) - { - return "dc1"; - } - - @Override - public String getRack(InetAddressAndPort endpoint) - { - return endpointRacks.get(endpoint).iterator().next(); - } - }; - IEndpointSnitch snitch; + NodeProximity nodeProximity = new NoOpProximity(); if (dynamicSnitch) - snitch = dsnitch = new DynamicEndpointSnitch(gpfs, String.valueOf(gpfs.hashCode())); + nodeProximity = dsnitch = new DynamicEndpointSnitch(nodeProximity, String.valueOf(nodeProximity.hashCode())); else - { dsnitch = null; - snitch = gpfs; - } DatabaseDescriptor.setDynamicBadnessThreshold(0); - DatabaseDescriptor.setEndpointSnitch(snitch); + DatabaseDescriptor.setNodeProximity(nodeProximity); DatabaseDescriptor.setBatchlogEndpointStrategy(batchlogEndpointStrategy); } @@ -1046,7 +1032,7 @@ private void withConfigs(Stream<Supplier<Multimap<String, InetAddressAndPort>>> private String configToString(Multimap<String, InetAddressAndPort> endpoints) { return "strategy:" + DatabaseDescriptor.getBatchlogEndpointStrategy() - + " snitch:" + DatabaseDescriptor.getEndpointSnitch().getClass().getSimpleName() + + " proximity:" + DatabaseDescriptor.getNodeProximity().getClass().getSimpleName() + " nodes-per-rack: " + endpoints.asMap().entrySet().stream() .map(e -> e.getKey() + '=' + e.getValue().size()) .collect(Collectors.joining()); diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java index 96b164a32256..89af53b175a8 100644 --- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java +++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java @@ -206,6 +206,7 @@ public class DatabaseDescriptorRefTest "org.apache.cassandra.exceptions.TransportException", "org.apache.cassandra.fql.FullQueryLogger", "org.apache.cassandra.fql.FullQueryLoggerOptions", + "org.apache.cassandra.gms.IEndpointStateChangeSubscriber", "org.apache.cassandra.gms.IFailureDetector", "org.apache.cassandra.io.FSError", "org.apache.cassandra.io.FSWriteError", @@ -269,15 +270,21 @@ public class DatabaseDescriptorRefTest "org.apache.cassandra.io.util.SpinningDiskOptimizationStrategy", "org.apache.cassandra.locator.IEndpointSnitch", "org.apache.cassandra.locator.InetAddressAndPort", + "org.apache.cassandra.locator.Locator", + "org.apache.cassandra.locator.NodeAddressConfig", + "org.apache.cassandra.locator.NodeProximity", + "org.apache.cassandra.locator.InitialLocationProvider", "org.apache.cassandra.locator.Replica", "org.apache.cassandra.locator.ReplicaCollection", "org.apache.cassandra.locator.SeedProvider", "org.apache.cassandra.locator.SimpleSeedProvider", + "org.apache.cassandra.locator.SnitchAdapter", "org.apache.cassandra.security.EncryptionContext", "org.apache.cassandra.security.ISslContextFactory", "org.apache.cassandra.security.SSLFactory", "org.apache.cassandra.service.CacheService$CacheType", "org.apache.cassandra.security.AbstractCryptoProvider", + "org.apache.cassandra.tcm.RegistrationStateCallbacks", "org.apache.cassandra.transport.ProtocolException", "org.apache.cassandra.utils.Closeable", "org.apache.cassandra.utils.CloseableIterator", diff --git a/test/unit/org/apache/cassandra/cql3/PagingTest.java b/test/unit/org/apache/cassandra/cql3/PagingTest.java index 919ef8722b57..eddc7aebd559 100644 --- a/test/unit/org/apache/cassandra/cql3/PagingTest.java +++ b/test/unit/org/apache/cassandra/cql3/PagingTest.java @@ -31,11 +31,11 @@ import com.datastax.driver.core.Statement; import org.apache.cassandra.ServerTestUtils; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.locator.AbstractEndpointSnitch; -import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.locator.ReplicaCollection; +import org.apache.cassandra.locator.NodeProximity; +import org.apache.cassandra.locator.BaseProximity; import org.apache.cassandra.service.EmbeddedCassandraService; import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_CONFIG; @@ -102,23 +102,13 @@ public void testPaging() throws InterruptedException String createTableStatement = "CREATE TABLE IF NOT EXISTS " + table + " (id int, id2 int, id3 int, val text, PRIMARY KEY ((id, id2), id3));"; String dropTableStatement = "DROP TABLE IF EXISTS " + table + ';'; - // custom snitch to avoid merging ranges back together after StorageProxy#getRestrictedRanges splits them up - IEndpointSnitch snitch = new AbstractEndpointSnitch() + // custom proximity impl to avoid merging ranges back together after StorageProxy#getRestrictedRanges splits them up + NodeProximity proximity = new BaseProximity() { - private IEndpointSnitch oldSnitch = DatabaseDescriptor.getEndpointSnitch(); + private final NodeProximity oldProximity = DatabaseDescriptor.getNodeProximity(); public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2) { - return oldSnitch.compareEndpoints(target, a1, a2); - } - - public String getRack(InetAddressAndPort endpoint) - { - return oldSnitch.getRack(endpoint); - } - - public String getDatacenter(InetAddressAndPort endpoint) - { - return oldSnitch.getDatacenter(endpoint); + return oldProximity.compareEndpoints(target, a1, a2); } @Override @@ -127,9 +117,7 @@ public boolean isWorthMergingForRangeQuery(ReplicaCollection merged, ReplicaColl return false; } }; - DatabaseDescriptor.setEndpointSnitch(snitch); -// StorageService.instance.getTokenMetadata().clearUnsafe(); -// StorageService.instance.getTokenMetadata().updateNormalToken(new LongToken(5097162189738624638L), FBUtilities.getBroadcastAddressAndPort()); + DatabaseDescriptor.setNodeProximity(proximity); session.execute(createTableStatement); for (int i = 0; i < 110; i++) diff --git a/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java b/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java index 1bef2d82cfa2..cd1941416685 100644 --- a/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java +++ b/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java @@ -451,12 +451,12 @@ public void testDescribeCluster() throws Throwable assertRowsNet(executeDescribeNet(describeKeyword + " CLUSTER"), row("Test Cluster", trimIfPresent(DatabaseDescriptor.getPartitionerName(), "org.apache.cassandra.dht."), - DatabaseDescriptor.getEndpointSnitch().getClass().getName())); + DatabaseDescriptor.getNodeProximity().getClass().getName())); assertRowsNet(executeDescribeNet("system_virtual_schema", describeKeyword + " CLUSTER"), row("Test Cluster", trimIfPresent(DatabaseDescriptor.getPartitionerName(), "org.apache.cassandra.dht."), - DatabaseDescriptor.getEndpointSnitch().getClass().getName())); + DatabaseDescriptor.getNodeProximity().getClass().getName())); } ClusterMetadata metadata = ClusterMetadata.current(); Token token = metadata.tokenMap.tokens().get(0); @@ -465,7 +465,7 @@ public void testDescribeCluster() throws Throwable assertRowsNet(executeDescribeNet(KEYSPACE_PER_TEST, "DESCRIBE CLUSTER"), row("Test Cluster", trimIfPresent(DatabaseDescriptor.getPartitionerName(), "org.apache.cassandra.dht."), - DatabaseDescriptor.getEndpointSnitch().getClass().getName(), + DatabaseDescriptor.getNodeProximity().getClass().getName(), ImmutableMap.of(token.toString(), ImmutableList.of(addressAndPort.toString())))); } diff --git a/test/unit/org/apache/cassandra/db/CleanupTest.java b/test/unit/org/apache/cassandra/db/CleanupTest.java index f4f94d1a300d..d474fd1a1957 100644 --- a/test/unit/org/apache/cassandra/db/CleanupTest.java +++ b/test/unit/org/apache/cassandra/db/CleanupTest.java @@ -39,7 +39,6 @@ import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.ServerTestUtils; import org.apache.cassandra.Util; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.Operator; import org.apache.cassandra.db.compaction.CompactionManager; import org.apache.cassandra.db.filter.RowFilter; @@ -48,7 +47,6 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.locator.AbstractNetworkTopologySnitch; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.KeyspaceMetadata; @@ -94,23 +92,6 @@ public static void defineSchema() throws Exception KeyspaceParams.simple(1), SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1), SchemaLoader.compositeIndexCFMD(KEYSPACE1, CF_INDEXED1, true)); - - - DatabaseDescriptor.setEndpointSnitch(new AbstractNetworkTopologySnitch() - { - @Override - public String getRack(InetAddressAndPort endpoint) - { - return "RC1"; - } - - @Override - public String getDatacenter(InetAddressAndPort endpoint) - { - return "DC1"; - } - }); - SchemaLoader.createKeyspace(KEYSPACE2, KeyspaceParams.nts("DC1", 1), SchemaLoader.standardCFMD(KEYSPACE2, CF_STANDARD2), diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreClientModeTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreClientModeTest.java index 0550237b7422..cdd7567fa4d3 100644 --- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreClientModeTest.java +++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreClientModeTest.java @@ -30,7 +30,6 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.statements.schema.CreateTableStatement; import org.apache.cassandra.dht.Murmur3Partitioner; -import org.apache.cassandra.locator.SimpleSnitch; import org.apache.cassandra.schema.DistributedSchema; import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.KeyspaceParams; @@ -66,11 +65,9 @@ public static void setUpClass() if (DatabaseDescriptor.getPartitioner() == null) DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); - DatabaseDescriptor.setEndpointSnitch(new SimpleSnitch()); DatabaseDescriptor.getRawConfig().memtable_flush_writers = 1; DatabaseDescriptor.getRawConfig().local_system_data_file_directory = tempFolder.toString(); DatabaseDescriptor.getRawConfig().partitioner = "Murmur3Partitioner"; - DatabaseDescriptor.setLocalDataCenter("DC1"); DatabaseDescriptor.applyPartitioner(); } diff --git a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java index 2047210323cc..7eb8f9c578db 100644 --- a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java +++ b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java @@ -147,8 +147,8 @@ public void testPersistLocalMetadata() assertEquals(FBUtilities.getReleaseVersionString(), row.getString("release_version")); assertEquals(QueryProcessor.CQL_VERSION.toString(), row.getString("cql_version")); assertEquals(String.valueOf(ProtocolVersion.CURRENT.asInt()), row.getString("native_protocol_version")); - assertEquals(DatabaseDescriptor.getEndpointSnitch().getLocalDatacenter(), row.getString("data_center")); - assertEquals(DatabaseDescriptor.getEndpointSnitch().getLocalRack(), row.getString("rack")); + assertEquals(DatabaseDescriptor.getLocator().local().datacenter, row.getString("data_center")); + assertEquals(DatabaseDescriptor.getLocator().local().rack, row.getString("rack")); assertEquals(DatabaseDescriptor.getPartitioner().getClass().getName(), row.getString("partitioner")); assertEquals(FBUtilities.getJustBroadcastNativeAddress(), row.getInetAddress("rpc_address")); assertEquals(DatabaseDescriptor.getNativeTransportPort(), row.getInt("rpc_port")); diff --git a/test/unit/org/apache/cassandra/db/filter/ColumnFilterTest.java b/test/unit/org/apache/cassandra/db/filter/ColumnFilterTest.java index 32fa3b234acd..f2b47cbf6eab 100644 --- a/test/unit/org/apache/cassandra/db/filter/ColumnFilterTest.java +++ b/test/unit/org/apache/cassandra/db/filter/ColumnFilterTest.java @@ -24,18 +24,15 @@ import java.util.function.Consumer; import org.junit.Assert; -import org.junit.Before; import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; import org.junit.runners.Parameterized; import org.apache.cassandra.SchemaLoader; -import org.apache.cassandra.Util; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.db.RegularAndStaticColumns; - import org.apache.cassandra.db.marshal.Int32Type; import org.apache.cassandra.db.marshal.SetType; import org.apache.cassandra.db.rows.CellPath; @@ -43,7 +40,6 @@ import org.apache.cassandra.io.util.DataInputBuffer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputBuffer; -import org.apache.cassandra.locator.SimpleSnitch; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.TableMetadata; @@ -93,18 +89,10 @@ public static void beforeClass() DatabaseDescriptor.daemonInitialization(); SchemaLoader.prepareServer(); DatabaseDescriptor.setSeedProvider(Arrays::asList); - DatabaseDescriptor.setEndpointSnitch(new SimpleSnitch()); DatabaseDescriptor.setDefaultFailureDetector(); DatabaseDescriptor.setPartitionerUnsafe(new Murmur3Partitioner()); } - @Before - public void before() - { - Util.setUpgradeFromVersion("4.0"); - // todo; nothing in this test is actually version dependent anymore - } - // Select all @Test diff --git a/test/unit/org/apache/cassandra/db/guardrails/GuardrailMaximumReplicationFactorTest.java b/test/unit/org/apache/cassandra/db/guardrails/GuardrailMaximumReplicationFactorTest.java index 04a7e33b3e29..d35673a84793 100644 --- a/test/unit/org/apache/cassandra/db/guardrails/GuardrailMaximumReplicationFactorTest.java +++ b/test/unit/org/apache/cassandra/db/guardrails/GuardrailMaximumReplicationFactorTest.java @@ -26,14 +26,10 @@ import org.junit.After; import org.junit.Test; -import org.apache.cassandra.ServerTestUtils; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; -import org.apache.cassandra.locator.AbstractEndpointSnitch; -import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.locator.Replica; import org.apache.cassandra.service.ClientWarn; import org.assertj.core.api.Assertions; @@ -113,21 +109,6 @@ public void testSimpleStrategyAlter() throws Throwable @Test public void testMultipleDatacenter() throws Throwable { - IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch(); - DatabaseDescriptor.setEndpointSnitch(new AbstractEndpointSnitch() - { - public static final String RACK1 = ServerTestUtils.RACK1; - - @Override - public String getRack(InetAddressAndPort endpoint) { return RACK1; } - - @Override - public String getDatacenter(InetAddressAndPort endpoint) { return "datacenter2"; } - - @Override - public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2) { return 0; } - }); - List<String> twoWarnings = Arrays.asList(format("The keyspace ks has a replication factor of 3, above the warning threshold of %s.", MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD), format("The keyspace ks has a replication factor of 3, above the warning threshold of %s.", MAXIMUM_REPLICATION_FACTOR_WARN_THRESHOLD)); @@ -152,7 +133,7 @@ public void testMultipleDatacenter() throws Throwable assertFails("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 2, 'datacenter2' : 5}", 5); assertFails("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 5, 'datacenter2' : 5}", 5); - DatabaseDescriptor.setEndpointSnitch(snitch); +// DatabaseDescriptor.setEndpointSnitch(snitch); } @Test diff --git a/test/unit/org/apache/cassandra/db/guardrails/GuardrailMinimumReplicationFactorTest.java b/test/unit/org/apache/cassandra/db/guardrails/GuardrailMinimumReplicationFactorTest.java index 173f70698b99..9b376e2f0e7a 100644 --- a/test/unit/org/apache/cassandra/db/guardrails/GuardrailMinimumReplicationFactorTest.java +++ b/test/unit/org/apache/cassandra/db/guardrails/GuardrailMinimumReplicationFactorTest.java @@ -27,14 +27,11 @@ import org.junit.Before; import org.junit.Test; -import org.apache.cassandra.ServerTestUtils; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.Keyspace; -import org.apache.cassandra.locator.AbstractEndpointSnitch; -import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.locator.Replica; import org.apache.cassandra.service.ClientWarn; +import org.apache.cassandra.tcm.membership.Location; import org.apache.cassandra.tcm.membership.NodeAddresses; import org.apache.cassandra.tcm.transformations.Register; import org.assertj.core.api.Assertions; @@ -153,26 +150,11 @@ public void testSimpleStrategyAlter() throws Throwable @Test public void testMultipleDatacenter() throws Throwable { - IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch(); - DatabaseDescriptor.setEndpointSnitch(new AbstractEndpointSnitch() - { - public static final String RACK1 = ServerTestUtils.RACK1; - - @Override - public String getRack(InetAddressAndPort endpoint) { return RACK1; } - - @Override - public String getDatacenter(InetAddressAndPort endpoint) { return "datacenter2"; } - - @Override - public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2) { return 0; } - }); - List<String> twoWarnings = Arrays.asList(format("The keyspace %s has a replication factor of 2, below the warning threshold of %d.", KS, MINIMUM_REPLICATION_FACTOR_WARN_THRESHOLD), format("The keyspace %s has a replication factor of 2, below the warning threshold of %d.", KS, MINIMUM_REPLICATION_FACTOR_WARN_THRESHOLD)); InetAddressAndPort ep = InetAddressAndPort.getByName("127.0.0.255"); - Register.register(new NodeAddresses(ep)); + Register.register(new NodeAddresses(ep), new Location("datacenter2", "rack1")); guardrails().setMinimumReplicationFactorThreshold(MINIMUM_REPLICATION_FACTOR_WARN_THRESHOLD, MINIMUM_REPLICATION_FACTOR_FAIL_THRESHOLD); assertValid("CREATE KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 4, 'datacenter2' : 4 }"); execute("DROP KEYSPACE IF EXISTS ks"); @@ -192,7 +174,6 @@ public void testMultipleDatacenter() throws Throwable assertFails("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 4, 'datacenter2' : 1 }", 1); assertFails("ALTER KEYSPACE ks WITH replication = { 'class' : 'NetworkTopologyStrategy', 'datacenter1': 1, 'datacenter2' : 1 }", 1); - DatabaseDescriptor.setEndpointSnitch(snitch); execute("DROP KEYSPACE IF EXISTS ks1"); } diff --git a/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java b/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java index 18463c712277..2e6870e6a638 100644 --- a/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java +++ b/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java @@ -54,7 +54,6 @@ public static void setUp() throws ConfigurationException, IOException DatabaseDescriptor.setPartitionerUnsafe(OrderPreservingPartitioner.instance); ServerTestUtils.cleanupAndLeaveDirs(); Keyspace.setInitialized(); - } @Before diff --git a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java index 1407cb8b973e..f82a51fc584d 100644 --- a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java +++ b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java @@ -129,7 +129,7 @@ public boolean isAlive(InetAddressAndPort ep) RangeStreamer s = new RangeStreamer(metadata, StreamOperation.BOOTSTRAP, true, - DatabaseDescriptor.getEndpointSnitch(), + DatabaseDescriptor.getNodeProximity(), new StreamStateStore(), mockFailureDetector, false, diff --git a/test/unit/org/apache/cassandra/dht/RangeFetchMapCalculatorTest.java b/test/unit/org/apache/cassandra/dht/RangeFetchMapCalculatorTest.java index bb02f4071850..12015249ba99 100644 --- a/test/unit/org/apache/cassandra/dht/RangeFetchMapCalculatorTest.java +++ b/test/unit/org/apache/cassandra/dht/RangeFetchMapCalculatorTest.java @@ -32,39 +32,42 @@ import org.junit.Test; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.locator.AbstractNetworkTopologySnitch; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.Locator; import org.apache.cassandra.locator.Replica; +import org.apache.cassandra.tcm.membership.Location; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; public class RangeFetchMapCalculatorTest { + private static Locator locator; @BeforeClass - public static void setupUpSnitch() + public static void setup() { DatabaseDescriptor.daemonInitialization(); DatabaseDescriptor.setPartitionerUnsafe(RandomPartitioner.instance); - DatabaseDescriptor.setEndpointSnitch(new AbstractNetworkTopologySnitch() + locator = new Locator(null, null, null, null) { - //Odd IPs are in DC1 and Even are in DC2. Endpoints upto .14 will have unique racks and - // then will be same for a set of three. + Location local = new Location("DC1", "RAC1"); + Location remote = new Location("DC2", "RAC1"); @Override - public String getRack(InetAddressAndPort endpoint) + public Location local() { - return "RAC1"; + return local; } @Override - public String getDatacenter(InetAddressAndPort endpoint) + public Location location(InetAddressAndPort endpoint) { + //Odd IPs are in DC1 and Even are in DC2. if (getIPLastPart(endpoint) <= 50) - return DatabaseDescriptor.getLocalDataCenter(); + return local; else if (getIPLastPart(endpoint) % 2 == 0) - return DatabaseDescriptor.getLocalDataCenter(); + return local; else - return DatabaseDescriptor.getLocalDataCenter() + "Remote"; + return remote; } private int getIPLastPart(InetAddressAndPort endpoint) @@ -73,7 +76,7 @@ private int getIPLastPart(InetAddressAndPort endpoint) int index = str.lastIndexOf("."); return Integer.parseInt(str.substring(index + 1).trim()); } - }); + }; } @Test @@ -86,7 +89,7 @@ public void testWithSingleSource() throws Exception addNonTrivialRangeAndSources(rangesWithSources, 31, 40, "127.0.0.4"); addNonTrivialRangeAndSources(rangesWithSources, 41, 50, "127.0.0.5"); - RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Collections.emptyList(), "Test"); + RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Collections.emptyList(), "Test", locator); Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap(); validateRange(rangesWithSources, map); @@ -103,7 +106,7 @@ public void testWithNonOverlappingSource() throws Exception addNonTrivialRangeAndSources(rangesWithSources, 31, 40, "127.0.0.7", "127.0.0.8"); addNonTrivialRangeAndSources(rangesWithSources, 41, 50, "127.0.0.9", "127.0.0.10"); - RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Collections.emptyList(), "Test"); + RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Collections.emptyList(), "Test", locator); Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap(); validateRange(rangesWithSources, map); @@ -118,7 +121,7 @@ public void testWithRFThreeReplacement() throws Exception addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.2", "127.0.0.3"); addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.3", "127.0.0.4"); - RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Collections.emptyList(), "Test"); + RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Collections.emptyList(), "Test", locator); Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap(); validateRange(rangesWithSources, map); @@ -136,7 +139,7 @@ public void testForMultipleRoundsComputation() throws Exception addNonTrivialRangeAndSources(rangesWithSources, 31, 40, "127.0.0.3"); addNonTrivialRangeAndSources(rangesWithSources, 41, 50, "127.0.0.3", "127.0.0.2"); - RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Collections.emptyList(), "Test"); + RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Collections.emptyList(), "Test", locator); Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap(); validateRange(rangesWithSources, map); @@ -158,7 +161,7 @@ public void testForMultipleRoundsComputationWithLocalHost() throws Exception addNonTrivialRangeAndSources(rangesWithSources, 31, 40, "127.0.0.1"); addNonTrivialRangeAndSources(rangesWithSources, 41, 50, "127.0.0.1", "127.0.0.2"); - RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Collections.emptyList(), "Test"); + RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Collections.emptyList(), "Test", locator); Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap(); validateRange(rangesWithSources, map); @@ -178,7 +181,7 @@ public void testForEmptyGraph() throws Exception addNonTrivialRangeAndSources(rangesWithSources, 31, 40, "127.0.0.1"); addNonTrivialRangeAndSources(rangesWithSources, 41, 50, "127.0.0.1"); - RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Collections.emptyList(), "Test"); + RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Collections.emptyList(), "Test", locator); Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap(); //All ranges map to local host so we will not stream anything. assertTrue(map.isEmpty()); @@ -216,7 +219,7 @@ public String message(Replica replica) } }; - RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Arrays.asList(filter), "Test"); + RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Arrays.asList(filter), "Test", locator); Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap(); validateRange(rangesWithSources, map); @@ -249,7 +252,7 @@ public String message(Replica replica) } }; - RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Arrays.asList(allDeadFilter), "Test"); + RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Arrays.asList(allDeadFilter), "Test", locator); calculator.getRangeFetchMap(); } @@ -261,7 +264,7 @@ public void testForLocalDC() throws Exception addNonTrivialRangeAndSources(rangesWithSources, 11, 20, "127.0.0.1", "127.0.0.3", "127.0.0.57"); addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.2", "127.0.0.59", "127.0.0.61"); - RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), new ArrayList<>(), "Test"); + RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), new ArrayList<>(), "Test", locator); Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap(); validateRange(rangesWithSources, map); Assert.assertEquals(2, map.asMap().size()); @@ -303,7 +306,7 @@ public String message(Replica replica) } }; - RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Arrays.asList(localHostFilter), "Test"); + RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Arrays.asList(localHostFilter), "Test", locator); Multimap<InetAddressAndPort, Range<Token>> map = calculator.getRangeFetchMap(); validateRange(rangesWithSources, map); Assert.assertEquals(3, map.asMap().size()); @@ -324,7 +327,7 @@ public void testTrivialRanges() throws UnknownHostException addNonTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.2", "127.0.0.59"); // and a trivial one: addTrivialRangeAndSources(rangesWithSources, 1, 10, "127.0.0.3", "127.0.0.51"); - RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Collections.emptyList(), "Test"); + RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Collections.emptyList(), "Test", locator); Multimap<InetAddressAndPort, Range<Token>> optMap = calculator.getRangeFetchMapForNonTrivialRanges(); Multimap<InetAddressAndPort, Range<Token>> trivialMap = calculator.getRangeFetchMapForTrivialRanges(optMap); assertTrue(trivialMap.get(InetAddressAndPort.getByName("127.0.0.3")).contains(generateTrivialRange(1,10)) ^ @@ -364,7 +367,7 @@ public String message(Replica replica) return "Not 127.0.0.3"; } }; - RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Collections.singleton(filter), "Test"); + RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(rangesWithSources.build(), Collections.singleton(filter), "Test", locator); Multimap<InetAddressAndPort, Range<Token>> optMap = calculator.getRangeFetchMapForNonTrivialRanges(); Multimap<InetAddressAndPort, Range<Token>> trivialMap = calculator.getRangeFetchMapForTrivialRanges(optMap); @@ -378,7 +381,7 @@ public void testTrivalRangeLocalHostStreaming() throws UnknownHostException addTrivialRangeAndSources(rangesWithSources, 21, 30, "127.0.0.2", "127.0.0.1"); addTrivialRangeAndSources(rangesWithSources, 31, 40, "127.0.0.1", "127.0.0.2"); EndpointsByRange ebr = rangesWithSources.build(); - RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(ebr, Collections.emptyList(), "Test"); + RangeFetchMapCalculator calculator = new RangeFetchMapCalculator(ebr, Collections.emptyList(), "Test", locator); RangeStreamer.validateRangeFetchMap(ebr, calculator.getRangeFetchMap(), "Test"); } diff --git a/test/unit/org/apache/cassandra/dht/tokenallocator/TokenAllocationTest.java b/test/unit/org/apache/cassandra/dht/tokenallocator/TokenAllocationTest.java index f0e0cda24d2e..6b4ef404656d 100644 --- a/test/unit/org/apache/cassandra/dht/tokenallocator/TokenAllocationTest.java +++ b/test/unit/org/apache/cassandra/dht/tokenallocator/TokenAllocationTest.java @@ -21,6 +21,7 @@ import java.net.InetAddress; import java.net.UnknownHostException; import java.util.Collection; +import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Random; @@ -33,6 +34,7 @@ import org.junit.BeforeClass; import org.junit.Test; +import org.apache.cassandra.ServerTestUtils; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Murmur3Partitioner; @@ -40,14 +42,15 @@ import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.locator.AbstractReplicationStrategy; -import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.locator.RackInferringSnitch; +import org.apache.cassandra.locator.SimpleLocationProvider; import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; +import org.apache.cassandra.tcm.StubClusterMetadataService; +import org.apache.cassandra.tcm.membership.NodeId; import org.apache.cassandra.utils.FBUtilities; import static org.junit.Assert.assertEquals; @@ -69,7 +72,7 @@ public static void beforeClass() throws ConfigurationException public void before() throws ConfigurationException { ClusterMetadataService.setInstance(ClusterMetadataTestHelper.syncInstanceForTest()); - ClusterMetadataService.instance().log().bootstrap(FBUtilities.getBroadcastAddressAndPort()); + ClusterMetadataService.instance().log().unsafeBootstrapForTesting(FBUtilities.getBroadcastAddressAndPort()); } @Before @@ -86,7 +89,7 @@ public static void afterClass() private static TokenAllocation createForTest(ClusterMetadata metadata, int replicas, int numTokens) { - return TokenAllocation.create(DatabaseDescriptor.getEndpointSnitch(), metadata, replicas, numTokens); + return TokenAllocation.create(metadata.locator.local().datacenter, metadata, replicas, numTokens); } @Test @@ -169,88 +172,72 @@ public void testAllocateTokensNetworkStrategyOneRackOneReplica() throws UnknownH public void testAllocateTokensNetworkStrategy(int rackCount, int replicas) throws UnknownHostException { - IEndpointSnitch oldSnitch = DatabaseDescriptor.getEndpointSnitch(); - try - { - DatabaseDescriptor.setEndpointSnitch(new RackInferringSnitch()); - int vn = 16; - String ks = "TokenAllocationTestNTSKeyspace" + rackCount + replicas; - String dc = "1"; - String otherDc = "15"; - KeyspaceMetadata keyspace = KeyspaceMetadata.create(ks, KeyspaceParams.nts(dc, replicas, otherDc, "15")); - - // register these 2 nodes so that the DCs exist, otherwise the CREATE KEYSPACE will be rejected - // but don't join them, we don't assign any tokens to these nodes - ClusterMetadataTestHelper.register(InetAddressAndPort.getByName("127.1.0.99"), dc, Integer.toString(0)); - ClusterMetadataTestHelper.register(InetAddressAndPort.getByName("127.15.0.99"), otherDc, Integer.toString(0)); - ClusterMetadataTestHelper.addOrUpdateKeyspace(keyspace); - - for (int i = 0; i < rackCount; ++i) - generateFakeEndpoints(10, vn, dc, Integer.toString(i)); - InetAddressAndPort addr = InetAddressAndPort.getByName("127." + dc + ".0.99"); - allocateTokensForKeyspace(vn, ks, ClusterMetadata.current(), addr); - // Note: Not matching replication factor in second datacentre, but this should not affect us. - } finally { - DatabaseDescriptor.setEndpointSnitch(oldSnitch); - } + int vn = 16; + String ks = "TokenAllocationTestNTSKeyspace" + rackCount + replicas; + String dc = "1"; + String otherDc = "15"; + KeyspaceMetadata keyspace = KeyspaceMetadata.create(ks, KeyspaceParams.nts(dc, replicas, otherDc, "15")); + + // register these 2 nodes so that the DCs exist, otherwise the CREATE KEYSPACE will be rejected + // but don't join them, we don't assign any tokens to these nodes + ClusterMetadataTestHelper.register(InetAddressAndPort.getByName("127.1.0.99"), dc, Integer.toString(0)); + ClusterMetadataTestHelper.register(InetAddressAndPort.getByName("127.15.0.99"), otherDc, Integer.toString(0)); + ClusterMetadataTestHelper.addOrUpdateKeyspace(keyspace); + + for (int i = 0; i < rackCount; ++i) + generateFakeEndpoints(10, vn, dc, Integer.toString(i)); + InetAddressAndPort addr = InetAddressAndPort.getByName("127." + dc + ".0.99"); + allocateTokensForKeyspace(vn, ks, ClusterMetadata.current(), addr); + // Note: Not matching replication factor in second datacentre, but this should not affect us. } @Test public void testAllocateTokensRfEqRacks() throws UnknownHostException { - IEndpointSnitch oldSnitch = DatabaseDescriptor.getEndpointSnitch(); - try + int vn = 8; + int replicas = 3; + int rackCount = replicas; + String ks = "TokenAllocationTestNTSKeyspaceRfEqRack"; + String dc = "1"; + String otherDc = "15"; + KeyspaceMetadata keyspace = KeyspaceMetadata.create(ks, KeyspaceParams.nts(dc, replicas, otherDc, "15")); + + // register these 2 nodes so that the DCs exist, otherwise the CREATE KEYSPACE will be rejected + // but don't join them, we don't assign any tokens to these nodes + ClusterMetadataTestHelper.register(InetAddressAndPort.getByName("127.1.0.255"), dc, Integer.toString(0)); + ClusterMetadataTestHelper.register(InetAddressAndPort.getByName("127.15.0.255"), otherDc, Integer.toString(0)); + ClusterMetadataTestHelper.addOrUpdateKeyspace(keyspace); + + int base = 5; + for (int i = 0; i < rackCount; ++i) + generateFakeEndpoints(base << i, vn, dc, Integer.toString(i)); // unbalanced racks + + int cnt = 5; + for (int i = 0; i < cnt; ++i) { - DatabaseDescriptor.setEndpointSnitch(new RackInferringSnitch()); - int vn = 8; - int replicas = 3; - int rackCount = replicas; - String ks = "TokenAllocationTestNTSKeyspaceRfEqRack"; - String dc = "1"; - String otherDc = "15"; - KeyspaceMetadata keyspace = KeyspaceMetadata.create(ks, KeyspaceParams.nts(dc, replicas, otherDc, "15")); - - // register these 2 nodes so that the DCs exist, otherwise the CREATE KEYSPACE will be rejected - // but don't join them, we don't assign any tokens to these nodes - ClusterMetadataTestHelper.register(InetAddressAndPort.getByName("127.1.0.255"), dc, Integer.toString(0)); - ClusterMetadataTestHelper.register(InetAddressAndPort.getByName("127.15.0.255"), otherDc, Integer.toString(0)); - ClusterMetadataTestHelper.addOrUpdateKeyspace(keyspace); - - int base = 5; - for (int i = 0; i < rackCount; ++i) - generateFakeEndpoints(base << i, vn, dc, Integer.toString(i)); // unbalanced racks - - int cnt = 5; - for (int i = 0; i < cnt; ++i) - { - InetAddressAndPort endpoint = InetAddressAndPort.getByName("127." + dc + ".0." + (99 + i)); - Collection<Token> tokens = allocateTokensForKeyspace(vn, ks, ClusterMetadata.current(), endpoint); - - ClusterMetadataTestHelper.register(endpoint, dc, Integer.toString(0)); - ClusterMetadataTestHelper.join(endpoint, new HashSet<>(tokens)); - } - - double target = 1.0 / (base + cnt); - double permittedOver = 1.0 / (2 * vn + 1) + 0.01; - - Map<InetAddress, Float> ownership = StorageService.instance.effectiveOwnership(ks); - boolean failed = false; - for (Map.Entry<InetAddress, Float> o : ownership.entrySet()) - { - int rack = o.getKey().getAddress()[2]; - if (rack != 0) - continue; - - System.out.format("Node %s owns %f ratio to optimal %.2f\n", o.getKey(), o.getValue(), o.getValue() / target); - if (o.getValue()/target > 1 + permittedOver) - failed = true; - } - Assert.assertFalse(String.format("One of the nodes in the rack has over %.2f%% overutilization.", permittedOver * 100), failed); - } finally { - DatabaseDescriptor.setEndpointSnitch(oldSnitch); + InetAddressAndPort endpoint = InetAddressAndPort.getByName("127." + dc + ".0." + (99 + i)); + ClusterMetadataTestHelper.register(endpoint, dc, Integer.toString(0)); + Collection<Token> tokens = allocateTokensForKeyspace(vn, ks, ClusterMetadata.current(), endpoint); + ClusterMetadataTestHelper.join(endpoint, new HashSet<>(tokens)); } - } + double target = 1.0 / (base + cnt); + double permittedOver = 1.0 / (2 * vn + 1) + 0.01; + + Map<InetAddress, Float> ownership = StorageService.instance.effectiveOwnership(ks); + boolean failed = false; + for (Map.Entry<InetAddress, Float> o : ownership.entrySet()) + { + int rack = o.getKey().getAddress()[2]; + if (rack != 0) + continue; + + System.out.format("Node %s owns %f ratio to optimal %.2f\n", o.getKey(), o.getValue(), o.getValue() / target); + if (o.getValue() / target > 1 + permittedOver) + failed = true; + } + Assert.assertFalse(String.format("One of the nodes in the rack has over %.2f%% overutilization.", permittedOver * 100), failed); + } /** * TODO: This scenario isn't supported very well. Investigate a multi-keyspace version of the algorithm. @@ -260,7 +247,19 @@ public void testAllocateTokensMultipleKeyspaces() throws UnknownHostException { final int TOKENS = 16; - ClusterMetadata metadata = generateFakeEndpoints(10, TOKENS); + ClusterMetadataService.unsetInstance(); + ClusterMetadataService.setInstance(StubClusterMetadataService.forClientTools()); + generateFakeEndpoints(10, TOKENS); + // pre-register the nodes which we are going to allocate tokens to this is necessary as + // Location info is obtained by the allocator from cluster metadata, not the snitch + Map<InetAddressAndPort, NodeId> endpointToIdMap = new HashMap<>(10); + for (int i=11; i<=20; ++i) + { + InetAddressAndPort endpoint = InetAddressAndPort.getByName("127.0.0." + (i + 1)); + NodeId id = ClusterMetadataTestHelper.register(endpoint, SimpleLocationProvider.LOCATION); + endpointToIdMap.put(endpoint, id); + } + ClusterMetadata metadata = ClusterMetadata.current(); TokenAllocation rf2Allocator = createForTest(metadata, 2, TOKENS); TokenAllocation rf3Allocator = createForTest(metadata, 3, TOKENS); @@ -271,12 +270,13 @@ public void testAllocateTokensMultipleKeyspaces() throws UnknownHostException TokenAllocation current = rf3Allocator; TokenAllocation next = rf2Allocator; - for (int i=11; i<=20; ++i) + for (Map.Entry<InetAddressAndPort, NodeId> entry : endpointToIdMap.entrySet()) { - InetAddressAndPort endpoint = InetAddressAndPort.getByName("127.0.0." + (i + 1)); + InetAddressAndPort endpoint = entry.getKey(); + NodeId id = entry.getValue(); Set<Token> tokens = new HashSet<>(current.allocate(endpoint)); - ClusterMetadataTestHelper.register(endpoint, "datacenter" + 1, "rack" + 1); - ClusterMetadataTestHelper.join(endpoint, tokens); + // Update tokens on next to verify ownership calculation below + next.updateTokensForNode(id, tokens); TokenAllocation tmp = current; current = next; next = tmp; @@ -297,16 +297,33 @@ private void verifyImprovement(SummaryStatistics os, SummaryStatistics ns) } } - private ClusterMetadata generateFakeEndpoints(int numOldNodes, int numVNodes) throws UnknownHostException + private ClusterMetadata generateFakeEndpoints(int nodes, int vnodes) throws UnknownHostException { - return generateFakeEndpoints(numOldNodes, numVNodes, "0", "0"); + // This is used by tests which previously relied on the ServerTestUtils snitch impl which has + // 127.0.0.4 in datacenter2 and every other node in datacenter1 + IPartitioner p = ClusterMetadata.current().tokenMap.partitioner(); + for (int i = 1; i <= nodes; i++) + { + // leave .1 for myEndpoint + InetAddressAndPort addr = InetAddressAndPort.getByName("127.0.0." + (i + 1)); + String dc = (i + 1 == 4) ? ServerTestUtils.DATA_CENTER_REMOTE : ServerTestUtils.DATA_CENTER; + ClusterMetadataTestHelper.register(addr, dc, ServerTestUtils.RACK1); + Set<Token> tokens = new HashSet<>(vnodes); + for (int j = 0; j < vnodes; ++j) + tokens.add(p.getRandomToken(rand)); + ClusterMetadataTestHelper.join(addr, tokens); + } + // Register (but don't join/allocate tokens for) 127.0.0.1) + ClusterMetadataTestHelper.register(FBUtilities.getBroadcastAddressAndPort(), + ServerTestUtils.DATA_CENTER, + ServerTestUtils.RACK1); + return ClusterMetadata.current(); } private ClusterMetadata generateFakeEndpoints(int nodes, int vnodes, String dc, String rack) throws UnknownHostException { System.out.printf("Adding %d nodes to dc=%s, rack=%s.%n", nodes, dc, rack); - ClusterMetadata metadata = ClusterMetadata.current(); - IPartitioner p = metadata.tokenMap.partitioner(); + IPartitioner p = ClusterMetadata.current().tokenMap.partitioner(); for (int i = 1; i <= nodes; i++) { @@ -318,6 +335,6 @@ private ClusterMetadata generateFakeEndpoints(int nodes, int vnodes, String dc, tokens.add(p.getRandomToken(rand)); ClusterMetadataTestHelper.join(addr, tokens); } - return metadata; + return ClusterMetadata.current(); } } diff --git a/test/unit/org/apache/cassandra/locator/AlibabaCloudSnitchTest.java b/test/unit/org/apache/cassandra/locator/AlibabaCloudSnitchTest.java index 6ca684c7689b..09591f291bd3 100644 --- a/test/unit/org/apache/cassandra/locator/AlibabaCloudSnitchTest.java +++ b/test/unit/org/apache/cassandra/locator/AlibabaCloudSnitchTest.java @@ -19,20 +19,19 @@ import java.io.IOException; +import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; +import org.apache.cassandra.ServerTestUtils; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.dht.Token; import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.DefaultCloudMetadataServiceConnector; -import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.utils.Pair; -import static org.apache.cassandra.config.CassandraRelevantProperties.GOSSIP_DISABLE_THREAD_VALIDATION; import static org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.METADATA_URL_PROPERTY; -import static org.apache.cassandra.locator.AlibabaCloudSnitch.DEFAULT_METADATA_SERVICE_URL; +import static org.apache.cassandra.locator.AlibabaCloudLocationProvider.DEFAULT_METADATA_SERVICE_URL; import static org.junit.Assert.assertEquals; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doReturn; @@ -45,11 +44,16 @@ public class AlibabaCloudSnitchTest @BeforeClass public static void setup() throws Exception { - GOSSIP_DISABLE_THREAD_VALIDATION.setBoolean(true); DatabaseDescriptor.daemonInitialization(); ClusterMetadataTestHelper.setInstanceForTest(); } + @Before + public void resetCMS() + { + ServerTestUtils.resetCMS(); + } + @Test public void testRac() throws IOException, ConfigurationException { @@ -60,18 +64,14 @@ public void testRac() throws IOException, ConfigurationException doReturn(az).when(spiedConnector).apiCall(any()); - AlibabaCloudSnitch snitch = new AlibabaCloudSnitch(spiedConnector); - InetAddressAndPort local = InetAddressAndPort.getByName("127.0.0.1"); - InetAddressAndPort nonlocal = InetAddressAndPort.getByName("127.0.0.7"); - - Token t1 = ClusterMetadata.current().partitioner.getRandomToken(); - ClusterMetadataTestHelper.addEndpoint(nonlocal, t1, "cn-shanghai", "a"); - - assertEquals("cn-shanghai", snitch.getDatacenter(nonlocal)); - assertEquals("a", snitch.getRack(nonlocal)); + // for registering a new node, location is obtained from the cloud metadata service + AlibabaCloudLocationProvider locationProvider = new AlibabaCloudLocationProvider(spiedConnector); + assertEquals("cn-hangzhou", locationProvider.initialLocation().datacenter); + assertEquals("f", locationProvider.initialLocation().rack); - assertEquals("cn-hangzhou", snitch.getDatacenter(local)); - assertEquals("f", snitch.getRack(local)); + AlibabaCloudSnitch snitch = new AlibabaCloudSnitch(spiedConnector); + assertEquals("cn-hangzhou", snitch.getLocalDatacenter()); + assertEquals("f", snitch.getLocalRack()); } @Test @@ -83,9 +83,13 @@ public void testNewRegions() throws IOException, ConfigurationException doReturn(az).when(spiedConnector).apiCall(any()); + // for registering a new node, location is obtained from the cloud metadata service + AlibabaCloudLocationProvider locationProvider = new AlibabaCloudLocationProvider(spiedConnector); + assertEquals("us-east", locationProvider.initialLocation().datacenter); + assertEquals("1a", locationProvider.initialLocation().rack); + AlibabaCloudSnitch snitch = new AlibabaCloudSnitch(spiedConnector); - InetAddressAndPort local = InetAddressAndPort.getByName("127.0.0.1"); - assertEquals("us-east", snitch.getDatacenter(local)); - assertEquals("1a", snitch.getRack(local)); + assertEquals("us-east", snitch.getLocalDatacenter()); + assertEquals("1a", snitch.getLocalRack()); } } diff --git a/test/unit/org/apache/cassandra/locator/AssureSufficientLiveNodesTest.java b/test/unit/org/apache/cassandra/locator/AssureSufficientLiveNodesTest.java index 2bec9b8d295f..9e9ff605df99 100644 --- a/test/unit/org/apache/cassandra/locator/AssureSufficientLiveNodesTest.java +++ b/test/unit/org/apache/cassandra/locator/AssureSufficientLiveNodesTest.java @@ -86,23 +86,6 @@ public static void setUpClass() throws Throwable DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); ServerTestUtils.prepareServerNoRegister(); // Register peers with expected DC for NetworkTopologyStrategy. - - // TODO shouldn't require the snitch setup - DatabaseDescriptor.setEndpointSnitch(new AbstractNetworkTopologySnitch() - { - public String getRack(InetAddressAndPort endpoint) - { - byte[] address = endpoint.addressBytes; - return "rake" + address[1]; - } - - public String getDatacenter(InetAddressAndPort endpoint) - { - byte[] address = endpoint.addressBytes; - return "datacenter" + address[1]; - } - }); - List<InetAddressAndPort> instances = ImmutableList.of( // datacenter 1 InetAddressAndPort.getByName("127.1.0.255"), InetAddressAndPort.getByName("127.1.0.254"), InetAddressAndPort.getByName("127.1.0.253"), @@ -118,6 +101,10 @@ public String getDatacenter(InetAddressAndPort endpoint) String rack = "rake" + ip.addressBytes[1]; ClusterMetadataTestHelper.addEndpoint(ip, new Murmur3Partitioner.LongToken(i), dc, rack); } + // Need to register the local endpoint as ReplicaLayout::for(Range|Token)ReadLiveSorted sorts replicas by proximity to + // the local node. We use constants from SimpleSnitch to preserve previous test behaviour + InetAddressAndPort local = FBUtilities.getBroadcastAddressAndPort(); + ClusterMetadataTestHelper.register(local, SimpleLocationProvider.LOCATION); } @Test @@ -268,13 +255,13 @@ private static void raceOfReplicationStrategyTest(KeyspaceParams init, for (int i = 0; i < loopCount; i++) { // reset the keyspace -// racedKs.setMetadata(initKsMeta); + SchemaTestUtil.addOrUpdateKeyspace(initKsMeta); CountDownLatch trigger = new CountDownLatch(1); // starts 2 runnables that could race Future<?> f1 = es.submit(() -> { Uninterruptibles.awaitUninterruptibly(trigger); // Update replication strategy -// racedKs.setMetadata(alterToKsMeta); + SchemaTestUtil.addOrUpdateKeyspace(alterToKsMeta); }); Future<?> f2 = es.submit(() -> { Uninterruptibles.awaitUninterruptibly(trigger); diff --git a/test/unit/org/apache/cassandra/locator/AzureConnectorMockingTest.java b/test/unit/org/apache/cassandra/locator/AzureConnectorMockingTest.java index 1fa2eb688795..17aa35797944 100644 --- a/test/unit/org/apache/cassandra/locator/AzureConnectorMockingTest.java +++ b/test/unit/org/apache/cassandra/locator/AzureConnectorMockingTest.java @@ -34,9 +34,9 @@ import static java.lang.String.format; import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.METADATA_URL_PROPERTY; -import static org.apache.cassandra.locator.AzureSnitch.API_VERSION_PROPERTY_KEY; -import static org.apache.cassandra.locator.AzureSnitch.METADATA_HEADER; -import static org.apache.cassandra.locator.AzureSnitch.METADATA_QUERY_TEMPLATE; +import static org.apache.cassandra.locator.AzureCloudLocationProvider.API_VERSION_PROPERTY_KEY; +import static org.apache.cassandra.locator.AzureCloudLocationProvider.METADATA_HEADER; +import static org.apache.cassandra.locator.AzureCloudLocationProvider.METADATA_QUERY_TEMPLATE; import static org.junit.Assert.assertEquals; public class AzureConnectorMockingTest @@ -60,8 +60,12 @@ public void testConnector() throws Throwable p.setProperty(METADATA_URL_PROPERTY, "http://127.0.0.1:8080"); SnitchProperties snitchProperties = new SnitchProperties(p); - AzureSnitch azureSnitch = new AzureSnitch(new DefaultCloudMetadataServiceConnector(snitchProperties)); + DefaultCloudMetadataServiceConnector connector = new DefaultCloudMetadataServiceConnector(snitchProperties); + AzureCloudLocationProvider locationProvider = new AzureCloudLocationProvider(connector); + assertEquals("rack-1", locationProvider.initialLocation().rack); + assertEquals("PolandCentral", locationProvider.initialLocation().datacenter); + AzureSnitch azureSnitch = new AzureSnitch(connector); assertEquals("rack-1", azureSnitch.getLocalRack()); assertEquals("PolandCentral", azureSnitch.getLocalDatacenter()); } @@ -83,8 +87,12 @@ public void testMissingZone() throws Throwable p.setProperty(API_VERSION_PROPERTY_KEY, "2021-12-14"); SnitchProperties snitchProperties = new SnitchProperties(p); - AzureSnitch azureSnitch = new AzureSnitch(new DefaultCloudMetadataServiceConnector(snitchProperties)); + DefaultCloudMetadataServiceConnector connector = new DefaultCloudMetadataServiceConnector(snitchProperties); + AzureCloudLocationProvider locationProvider = new AzureCloudLocationProvider(connector); + assertEquals("rack-5", locationProvider.initialLocation().rack); + assertEquals("PolandCentral", locationProvider.initialLocation().datacenter); + AzureSnitch azureSnitch = new AzureSnitch(connector); assertEquals("rack-5", azureSnitch.getLocalRack()); assertEquals("PolandCentral", azureSnitch.getLocalDatacenter()); } diff --git a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java index 012fd938ce39..338a7f08afdd 100644 --- a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java +++ b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java @@ -24,16 +24,13 @@ import org.junit.BeforeClass; import org.junit.Test; -import org.apache.cassandra.ServerTestUtils; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.dht.Token; -import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.DefaultCloudMetadataServiceConnector; -import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.ClusterMetadataService; +import org.apache.cassandra.tcm.StubClusterMetadataService; import org.apache.cassandra.utils.Pair; -import static org.apache.cassandra.config.CassandraRelevantProperties.GOSSIP_DISABLE_THREAD_VALIDATION; import static org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.METADATA_URL_PROPERTY; import static org.junit.Assert.assertEquals; import static org.mockito.ArgumentMatchers.any; @@ -47,15 +44,14 @@ public class CloudstackSnitchTest @BeforeClass public static void setup() throws Exception { - GOSSIP_DISABLE_THREAD_VALIDATION.setBoolean(true); DatabaseDescriptor.daemonInitialization(); - ClusterMetadataTestHelper.setInstanceForTest(); } @Before public void resetCMS() { - ServerTestUtils.resetCMS(); + ClusterMetadataService.unsetInstance(); + ClusterMetadataService.setInstance(StubClusterMetadataService.forTesting()); } @Test @@ -68,18 +64,14 @@ public void testRacks() throws IOException, ConfigurationException doReturn(az).when(spiedConnector).apiCall(any()); - CloudstackSnitch snitch = new CloudstackSnitch(spiedConnector); - InetAddressAndPort local = InetAddressAndPort.getByName("127.0.0.1"); - InetAddressAndPort nonlocal = InetAddressAndPort.getByName("127.0.0.7"); - - Token t1 = ClusterMetadata.current().partitioner.getRandomToken(); - ClusterMetadataTestHelper.addEndpoint(nonlocal, t1, "ch-zrh", "2"); + // for registering a new node, location is obtained from the cloud metadata service + CloudstackLocationProvider locationProvider = new CloudstackLocationProvider(spiedConnector); + assertEquals("ch-gva", locationProvider.initialLocation().datacenter); + assertEquals("1", locationProvider.initialLocation().rack); - assertEquals("ch-zrh", snitch.getDatacenter(nonlocal)); - assertEquals("2", snitch.getRack(nonlocal)); - - assertEquals("ch-gva", snitch.getDatacenter(local)); - assertEquals("1", snitch.getRack(local)); + CloudstackSnitch snitch = new CloudstackSnitch(spiedConnector); + assertEquals("ch-gva", snitch.getLocalDatacenter()); + assertEquals("1", snitch.getLocalRack()); } @Test @@ -92,11 +84,13 @@ public void testNewRegions() throws IOException, ConfigurationException doReturn(az).when(spiedConnector).apiCall(any()); - CloudstackSnitch snitch = new CloudstackSnitch(spiedConnector); - - InetAddressAndPort local = InetAddressAndPort.getByName("127.0.0.1"); + // for registering a new node, location is obtained from the cloud metadata service + CloudstackLocationProvider locationProvider = new CloudstackLocationProvider(spiedConnector); + assertEquals("us-east", locationProvider.initialLocation().datacenter); + assertEquals("1a", locationProvider.initialLocation().rack); - assertEquals("us-east", snitch.getDatacenter(local)); - assertEquals("1a", snitch.getRack(local)); + CloudstackSnitch snitch = new CloudstackSnitch(spiedConnector); + assertEquals("us-east", snitch.getLocalDatacenter()); + assertEquals("1a", snitch.getLocalRack()); } } diff --git a/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java b/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java index 310be2a6b0fb..e1cf4ab517f8 100644 --- a/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java +++ b/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java @@ -87,8 +87,8 @@ public void testSnitch() throws InterruptedException, IOException, Configuration // do this because SS needs to be initialized before DES can work properly. DatabaseDescriptor.setDynamicBadnessThreshold(0.1); StorageService.instance.unsafeInitialize(); - SimpleSnitch ss = new SimpleSnitch(); - DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(ss, String.valueOf(ss.hashCode())); + NodeProximity proximity = new NoOpProximity(); + DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(proximity, String.valueOf(proximity.hashCode())); InetAddressAndPort self = FBUtilities.getBroadcastAddressAndPort(); InetAddressAndPort host1 = InetAddressAndPort.getByName("127.0.0.2"); InetAddressAndPort host2 = InetAddressAndPort.getByName("127.0.0.3"); diff --git a/test/unit/org/apache/cassandra/locator/Ec2SnitchTest.java b/test/unit/org/apache/cassandra/locator/Ec2SnitchTest.java index ef54a60c430d..cb8476da04cd 100644 --- a/test/unit/org/apache/cassandra/locator/Ec2SnitchTest.java +++ b/test/unit/org/apache/cassandra/locator/Ec2SnitchTest.java @@ -18,30 +18,30 @@ package org.apache.cassandra.locator; +import java.io.IOException; import java.util.Collections; -import java.util.HashSet; -import java.util.Set; +import java.util.Map; -import org.junit.AfterClass; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimap; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; -import org.apache.cassandra.ServerTestUtils; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.dht.Token; -import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; -import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.ClusterMetadataService; +import org.apache.cassandra.tcm.StubClusterMetadataService; +import org.apache.cassandra.tcm.membership.Directory; +import org.apache.cassandra.utils.Pair; import org.mockito.stubbing.Answer; -import static org.apache.cassandra.config.CassandraRelevantProperties.GOSSIP_DISABLE_THREAD_VALIDATION; import static org.apache.cassandra.locator.Ec2MultiRegionSnitch.PRIVATE_IP_QUERY; import static org.apache.cassandra.locator.Ec2MultiRegionSnitch.PUBLIC_IP_QUERY; import static org.apache.cassandra.locator.Ec2Snitch.EC2_NAMING_LEGACY; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; @@ -62,22 +62,14 @@ public String get(String propertyName, String defaultValue) @BeforeClass public static void setup() throws Exception { - GOSSIP_DISABLE_THREAD_VALIDATION.setBoolean(true); DatabaseDescriptor.daemonInitialization(); - ClusterMetadataTestHelper.setInstanceForTest(); - } - - - @AfterClass - public static void tearDown() - { - StorageService.instance.stopClient(); } @Before public void resetCMS() { - ServerTestUtils.resetCMS(); + ClusterMetadataService.unsetInstance(); + ClusterMetadataService.setInstance(StubClusterMetadataService.forTesting()); } @Test @@ -86,6 +78,10 @@ public void testLegacyRac() throws Exception Ec2MetadataServiceConnector connectorMock = mock(Ec2MetadataServiceConnector.class); doReturn("us-east-1d").when(connectorMock).apiCall(anyString()); doReturn(legacySnitchProps).when(connectorMock).getProperties(); + + Ec2LocationProvider locationProvider = new Ec2LocationProvider(connectorMock); + testLegacyRacInternal(locationProvider); + Ec2Snitch snitch = new Ec2Snitch(connectorMock); testLegacyRacInternal(snitch); } @@ -100,6 +96,9 @@ public void testMultiregionLegacyRac() throws Exception return (PUBLIC_IP_QUERY.equals(query) || PRIVATE_IP_QUERY.equals(query)) ? "127.0.0.1" : "us-east-1d"; }).when(spy).apiCall(anyString()); + Ec2LocationProvider locationProvider = new Ec2LocationProvider(spy); + testLegacyRacInternal(locationProvider); + Ec2Snitch snitch = new Ec2MultiRegionSnitch(spy); testLegacyRacInternal(snitch); } @@ -110,6 +109,7 @@ public void testLegacyNewRegions() throws Exception Ec2MetadataServiceConnector spy = spy(Ec2MetadataServiceConnector.create(legacySnitchProps)); doReturn(legacySnitchProps).when(spy).getProperties(); doReturn("us-east-2d").when(spy).apiCall(anyString()); + testLegacyNewRegionsInternal(new Ec2LocationProvider(spy)); testLegacyNewRegionsInternal(new Ec2Snitch(spy)); } @@ -123,6 +123,7 @@ public void testLegacyMultiRegionNewRegions() throws Exception return (PUBLIC_IP_QUERY.equals(query) || PRIVATE_IP_QUERY.equals(query)) ? "127.0.0.1" : "us-east-2d"; }).when(spy).apiCall(anyString()); + testLegacyNewRegionsInternal(new Ec2LocationProvider(spy)); testLegacyNewRegionsInternal(new Ec2MultiRegionSnitch(spy)); } @@ -132,142 +133,165 @@ public void testFullNamingScheme() throws Exception Ec2MetadataServiceConnector connectorMock = mock(Ec2MetadataServiceConnector.class); when(connectorMock.apiCall(anyString())).thenReturn("us-east-2d"); when(connectorMock.getProperties()).thenReturn(new SnitchProperties()); - Ec2Snitch snitch = new Ec2Snitch(connectorMock); - InetAddressAndPort local = InetAddressAndPort.getByName("127.0.0.1"); + Ec2LocationProvider locationProvider = new Ec2LocationProvider(connectorMock); + assertEquals("us-east-2", locationProvider.initialLocation().datacenter); + assertEquals("us-east-2d", locationProvider.initialLocation().rack); - assertEquals("us-east-2", snitch.getDatacenter(local)); - assertEquals("us-east-2d", snitch.getRack(local)); + Ec2Snitch snitch = new Ec2Snitch(connectorMock); + // for registering a new node, location is obtained from the cloud metadata service + assertEquals("us-east-2", snitch.getLocalDatacenter()); + assertEquals("us-east-2d", snitch.getLocalRack()); Ec2MetadataServiceConnector multiRegionConnectorMock = mock(Ec2MetadataServiceConnector.class); - when(connectorMock.getProperties()).thenReturn(new SnitchProperties()); + when(multiRegionConnectorMock.getProperties()).thenReturn(new SnitchProperties()); when(multiRegionConnectorMock.apiCall(anyString())).then((Answer<String>) invocation -> { String query = invocation.getArgument(0); return (PUBLIC_IP_QUERY.equals(query) || PRIVATE_IP_QUERY.equals(query)) ? "127.0.0.1" : "us-east-2d"; }); - assertEquals("us-east-2", snitch.getDatacenter(local)); - assertEquals("us-east-2d", snitch.getRack(local)); + Ec2LocationProvider mrLocationProvider = new Ec2LocationProvider(connectorMock); + assertEquals("us-east-2", mrLocationProvider.initialLocation().datacenter); + assertEquals("us-east-2d", mrLocationProvider.initialLocation().rack); + + Ec2MultiRegionSnitch mrSnitch = new Ec2MultiRegionSnitch(multiRegionConnectorMock); + // for registering a new node, location is obtained from the cloud metadata service + assertEquals("us-east-2", mrSnitch.getLocalDatacenter()); + assertEquals("us-east-2d", mrSnitch.getLocalRack()); } @Test - public void validateDatacenter_RequiresLegacy_CorrectAmazonName() + public void validateDatacenter_RequiresLegacy_CorrectAmazonName() throws IOException { - Set<String> datacenters = new HashSet<>(); - datacenters.add("us-east-1"); - assertTrue(Ec2Snitch.validate(datacenters, Collections.emptySet(), true)); + // legacy scheme / standard dc / ignore racks + assertValid(true, "us-east-1"); } @Test - public void validateDatacenter_RequiresLegacy_LegacyName() + public void validateDatacenter_RequiresLegacy_LegacyName() throws IOException { - Set<String> datacenters = new HashSet<>(); - datacenters.add("us-east"); - assertTrue(Ec2Snitch.validate(datacenters, Collections.emptySet(), true)); + // legacy scheme / legacy dc / ignore racks + assertValid(true, "us-east"); } @Test - public void validate_RequiresLegacy_HappyPath() + public void validate_RequiresLegacy_HappyPath() throws IOException { - Set<String> datacenters = new HashSet<>(); - datacenters.add("us-east"); - Set<String> racks = new HashSet<>(); - racks.add("1a"); - assertTrue(Ec2Snitch.validate(datacenters, racks, true)); + // legacy scheme / legacy dc / legacy rack + assertValid(true, "us-east", "1a"); } @Test - public void validate_RequiresLegacy_HappyPathWithDCSuffix() + public void validate_RequiresLegacy_HappyPathWithDCSuffix() throws IOException { - Set<String> datacenters = new HashSet<>(); - datacenters.add("us-east_CUSTOM_SUFFIX"); - Set<String> racks = new HashSet<>(); - racks.add("1a"); - assertTrue(Ec2Snitch.validate(datacenters, racks, true)); + // legacy scheme / custom suffix dc / legacy rack + assertValid(true, "us-east_CUSTOM_SUFFIX", "1a"); } @Test - public void validateRack_RequiresAmazonName_CorrectAmazonName() + public void validateRack_RequiresAmazonName_CorrectAmazonName() throws IOException { - Set<String> racks = new HashSet<>(); - racks.add("us-east-1a"); - assertTrue(Ec2Snitch.validate(Collections.emptySet(), racks, false)); + // standard scheme / ignore dc / non legacy rack + assertValid(false, "", "us-east-1a"); } - @Test - public void validateRack_RequiresAmazonName_LegacyName() + @Test(expected = ConfigurationException.class) + public void validateRack_RequiresAmazonName_LegacyName() throws IOException { - Set<String> racks = new HashSet<>(); - racks.add("1a"); - assertFalse(Ec2Snitch.validate(Collections.emptySet(), racks, false)); + // standard scheme / ignore dc / legacy rack + assertValid(false, "", "1a"); } @Test - public void validate_RequiresAmazonName_HappyPath() + public void validate_RequiresAmazonName_HappyPath() throws IOException { - Set<String> datacenters = new HashSet<>(); - datacenters.add("us-east-1"); - Set<String> racks = new HashSet<>(); - racks.add("us-east-1a"); - assertTrue(Ec2Snitch.validate(datacenters, racks, false)); + // standard scheme / standard dc / standard rack + assertValid(false, "us-east-1", "us-east-1a"); } @Test - public void validate_RequiresAmazonName_HappyPathWithDCSuffix() + public void validate_RequiresAmazonName_HappyPathWithDCSuffix() throws IOException { - Set<String> datacenters = new HashSet<>(); - datacenters.add("us-east-1_CUSTOM_SUFFIX"); - Set<String> racks = new HashSet<>(); - racks.add("us-east-1a"); - assertTrue(Ec2Snitch.validate(datacenters, racks, false)); + // standard scheme / custom suffix dc / standard rack + assertValid(false, "us-east-1_CUSTOM_SUFFIX", "us-east-1a"); } /** * Validate upgrades in legacy mode for regions that didn't change name between the standard and legacy modes. */ @Test - public void validate_RequiresLegacy_DCValidStandardAndLegacy() + public void validate_RequiresLegacy_DCValidStandardAndLegacy() throws IOException { - Set<String> datacenters = new HashSet<>(); - datacenters.add("us-west-2"); - Set<String> racks = new HashSet<>(); - racks.add("2a"); - racks.add("2b"); - assertTrue(Ec2Snitch.validate(datacenters, racks, true)); + // legacy scheme / standard dc / legacy racks + assertValid(true, "us-west-2", "2a", "2b"); } /** * Check that racks names are enough to detect a mismatch in naming conventions. */ - @Test - public void validate_RequiresLegacy_RackInvalidForLegacy() + @Test(expected = ConfigurationException.class) + public void validate_RequiresLegacy_RackInvalidForLegacy() throws Exception { - Set<String> datacenters = new HashSet<>(); - datacenters.add("us-west-2"); - Set<String> racks = new HashSet<>(); - racks.add("us-west-2a"); - assertFalse(Ec2Snitch.validate(datacenters, racks, true)); + // legacy scheme / standard dc / standard rack + assertValid(true, "us-west-2", "us-west-2a"); } - private void testLegacyRacInternal(Ec2Snitch snitch) throws Exception + private static void assertValid(boolean useLegacy, String datacenter, String...racks) throws IOException { - InetAddressAndPort local = InetAddressAndPort.getByName("127.0.0.1"); - InetAddressAndPort nonlocal = InetAddressAndPort.getByName("127.0.0.7"); + SnitchProperties properties = new SnitchProperties(Pair.create(Ec2LocationProvider.SNITCH_PROP_NAMING_SCHEME, + useLegacy ? Ec2LocationProvider.EC2_NAMING_LEGACY + : Ec2LocationProvider.EC2_NAMING_STANDARD)); + Ec2MetadataServiceConnector connectorMock = mock(Ec2MetadataServiceConnector.class); + when(connectorMock.getProperties()).thenReturn(properties); + Ec2LocationProvider provider = new Ec2LocationProvider(connectorMock); + + Multimap<String, InetAddressAndPort> dcRacks = HashMultimap.create(); + for (String rack : racks) + dcRacks.put(rack, null); // endpoints are used + Directory directory = new Directory() + { + @Override + public Map<String, Multimap<String, InetAddressAndPort>> allDatacenterRacks() + { + return Collections.singletonMap(datacenter, dcRacks); + } + + @Override + public Multimap<String, InetAddressAndPort> datacenterRacks(String datacenter) + { + return dcRacks; + } + }; + ClusterMetadata metadata = new ClusterMetadata(Murmur3Partitioner.instance, directory); + + provider.validate(metadata); + } - Token t1 = ClusterMetadata.current().partitioner.getRandomToken(); - ClusterMetadataTestHelper.addEndpoint(nonlocal, t1, "us-west", "1a"); + private void testLegacyRacInternal(Ec2LocationProvider provider) throws Exception + { + // for registering a new node, location is obtained from the cloud metadata service + assertEquals("us-east", provider.initialLocation().datacenter); + assertEquals("1d", provider.initialLocation().rack); + } - assertEquals("us-west", snitch.getDatacenter(nonlocal)); - assertEquals("1a", snitch.getRack(nonlocal)); + private void testLegacyRacInternal(Ec2Snitch snitch) throws Exception + { + // for registering a new node, location is obtained from the cloud metadata service + assertEquals("us-east", snitch.getLocalDatacenter()); + assertEquals("1d", snitch.getLocalRack()); + } - assertEquals("us-east", snitch.getDatacenter(local)); - assertEquals("1d", snitch.getRack(local)); + private void testLegacyNewRegionsInternal(Ec2LocationProvider provider) throws Exception + { + // for registering a new node, location is obtained from the cloud metadata service + assertEquals("us-east-2", provider.initialLocation().datacenter); + assertEquals("2d", provider.initialLocation().rack); } private void testLegacyNewRegionsInternal(Ec2Snitch snitch) throws Exception { - InetAddressAndPort local = InetAddressAndPort.getByName("127.0.0.1"); - assertEquals("us-east-2", snitch.getDatacenter(local)); - assertEquals("2d", snitch.getRack(local)); + // for registering a new node, location is obtained from the cloud metadata service + assertEquals("us-east-2", snitch.getLocalDatacenter()); + assertEquals("2d", snitch.getLocalRack()); } } diff --git a/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java b/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java index b447034e9565..277990a3cbfe 100644 --- a/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java +++ b/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java @@ -21,20 +21,19 @@ import java.io.IOException; +import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.dht.Token; -import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.DefaultCloudMetadataServiceConnector; -import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.ClusterMetadataService; +import org.apache.cassandra.tcm.StubClusterMetadataService; import org.apache.cassandra.utils.Pair; -import static org.apache.cassandra.config.CassandraRelevantProperties.GOSSIP_DISABLE_THREAD_VALIDATION; import static org.apache.cassandra.locator.AbstractCloudMetadataServiceConnector.METADATA_URL_PROPERTY; -import static org.apache.cassandra.locator.AlibabaCloudSnitch.DEFAULT_METADATA_SERVICE_URL; +import static org.apache.cassandra.locator.AlibabaCloudLocationProvider.DEFAULT_METADATA_SERVICE_URL; import static org.junit.Assert.assertEquals; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyMap; @@ -46,9 +45,14 @@ public class GoogleCloudSnitchTest @BeforeClass public static void setup() throws Exception { - GOSSIP_DISABLE_THREAD_VALIDATION.setBoolean(true); DatabaseDescriptor.daemonInitialization(); - ClusterMetadataTestHelper.setInstanceForTest(); + } + + @Before + public void resetCMS() + { + ClusterMetadataService.unsetInstance(); + ClusterMetadataService.setInstance(StubClusterMetadataService.forTesting()); } @Test @@ -61,18 +65,14 @@ public void testRac() throws IOException, ConfigurationException doReturn(az).when(spiedConnector).apiCall(any(), anyMap()); - GoogleCloudSnitch snitch = new GoogleCloudSnitch(spiedConnector); - InetAddressAndPort local = InetAddressAndPort.getByName("127.0.0.1"); - InetAddressAndPort nonlocal = InetAddressAndPort.getByName("127.0.0.7"); - - Token t1 = ClusterMetadata.current().partitioner.getRandomToken(); - ClusterMetadataTestHelper.addEndpoint(nonlocal, t1, "europe-west1", "a"); - - assertEquals("europe-west1", snitch.getDatacenter(nonlocal)); - assertEquals("a", snitch.getRack(nonlocal)); + // for registering a new node, location is obtained from the cloud metadata service + GoogleCloudLocationProvider locationProvider = new GoogleCloudLocationProvider(spiedConnector); + assertEquals("us-central1", locationProvider.initialLocation().datacenter); + assertEquals("a", locationProvider.initialLocation().rack); - assertEquals("us-central1", snitch.getDatacenter(local)); - assertEquals("a", snitch.getRack(local)); + GoogleCloudSnitch snitch = new GoogleCloudSnitch(spiedConnector); + assertEquals("us-central1", snitch.getLocalDatacenter()); + assertEquals("a", snitch.getLocalRack()); } @Test @@ -85,9 +85,13 @@ public void testNewRegions() throws IOException, ConfigurationException doReturn(az).when(spiedConnector).apiCall(any(), anyMap()); + // local endpoint is not yet registered, so location is obtained from the cloud metadata service connector + GoogleCloudLocationProvider locationProvider = new GoogleCloudLocationProvider(spiedConnector); + assertEquals("asia-east1", locationProvider.initialLocation().datacenter); + assertEquals("a", locationProvider.initialLocation().rack); + GoogleCloudSnitch snitch = new GoogleCloudSnitch(spiedConnector); - InetAddressAndPort local = InetAddressAndPort.getByName("127.0.0.1"); - assertEquals("asia-east1", snitch.getDatacenter(local)); - assertEquals("a", snitch.getRack(local)); + assertEquals("asia-east1", snitch.getLocalDatacenter()); + assertEquals("a", snitch.getLocalRack()); } } diff --git a/test/unit/org/apache/cassandra/locator/GossipingPropertyFileSnitchTest.java b/test/unit/org/apache/cassandra/locator/GossipingPropertyFileSnitchTest.java index dcd182d23e36..c8ff6ea45aad 100644 --- a/test/unit/org/apache/cassandra/locator/GossipingPropertyFileSnitchTest.java +++ b/test/unit/org/apache/cassandra/locator/GossipingPropertyFileSnitchTest.java @@ -18,13 +18,14 @@ package org.apache.cassandra.locator; -import java.net.UnknownHostException; +import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.tcm.ClusterMetadataService; +import org.apache.cassandra.tcm.StubClusterMetadataService; import static org.junit.Assert.*; @@ -33,34 +34,25 @@ */ public class GossipingPropertyFileSnitchTest { - @BeforeClass - public static void setupDD() + public static void setup() throws Exception { DatabaseDescriptor.daemonInitialization(); } - public static void checkEndpoint(final AbstractNetworkTopologySnitch snitch, - final String endpointString, final String expectedDatacenter, - final String expectedRack) + @Before + public void resetCMS() { - final InetAddressAndPort endpoint; - try - { - endpoint = InetAddressAndPort.getByName(endpointString); - } - catch (UnknownHostException e) - { - throw new RuntimeException(e); - } - assertEquals(expectedDatacenter, snitch.getDatacenter(endpoint)); - assertEquals(expectedRack, snitch.getRack(endpoint)); + ClusterMetadataService.unsetInstance(); + ClusterMetadataService.setInstance(StubClusterMetadataService.forTesting()); } @Test - public void testLoadConfig() throws Exception + public void testLoadConfig() { final GossipingPropertyFileSnitch snitch = new GossipingPropertyFileSnitch(); - checkEndpoint(snitch, FBUtilities.getBroadcastAddressAndPort().getHostAddressAndPort(), "DC1", "RAC1"); + // for registering a new node, location is obtained from the snitch config + assertEquals("DC1", snitch.getLocalDatacenter()); + assertEquals("RAC1", snitch.getLocalRack()); } } diff --git a/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java b/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java index 9a979998272f..fc235526fcef 100644 --- a/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java +++ b/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java @@ -45,14 +45,18 @@ import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.service.ClientWarn; import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.RegistrationStatus; import org.apache.cassandra.tcm.compatibility.TokenRingUtils; +import org.apache.cassandra.tcm.membership.Directory; import org.apache.cassandra.tcm.membership.Location; +import org.apache.cassandra.tcm.membership.NodeId; import org.apache.cassandra.tcm.ownership.VersionedEndpoints; import org.apache.cassandra.utils.FBUtilities; import static org.apache.cassandra.locator.NetworkTopologyStrategy.REPLICATION_FACTOR; import static org.apache.cassandra.locator.Replica.fullReplica; import static org.apache.cassandra.locator.Replica.transientReplica; +import static org.apache.cassandra.locator.SimpleLocationProvider.LOCATION; import static org.junit.Assert.assertTrue; public class NetworkTopologyStrategyTest @@ -78,8 +82,6 @@ public void teardown() @Test public void testProperties() throws IOException, ConfigurationException { - IEndpointSnitch snitch = new PropertyFileSnitch(); - DatabaseDescriptor.setEndpointSnitch(snitch); createDummyTokens(true); ClusterMetadataTestHelper.createKeyspace("CREATE KEYSPACE " + KEYSPACE + " WITH REPLICATION = {" + @@ -104,8 +106,6 @@ public void testProperties() throws IOException, ConfigurationException @Test public void testPropertiesWithEmptyDC() throws IOException, ConfigurationException { - IEndpointSnitch snitch = new PropertyFileSnitch(); - DatabaseDescriptor.setEndpointSnitch(snitch); createDummyTokens(false); Map<String, String> configOptions = new HashMap<>(); @@ -132,9 +132,6 @@ public void testLargeCluster() throws UnknownHostException, ConfigurationExcepti int[] dcEndpoints = new int[]{128, 256, 512}; int[] dcReplication = new int[]{2, 6, 6}; - IEndpointSnitch snitch = new RackInferringSnitch(); - DatabaseDescriptor.setEndpointSnitch(snitch); - Map<String, String> configOptions = new HashMap<String, String>(); Multimap<InetAddressAndPort, Token> tokens = HashMultimap.create(); @@ -152,7 +149,8 @@ public void testLargeCluster() throws UnknownHostException, ConfigurationExcepti StringToken token = new StringToken(String.format("%02x%02x%02x", ep, rack, dc)); logger.debug("adding node {} at {}", address, token); tokens.put(address, token); - ClusterMetadataTestHelper.addEndpoint(address, token, snitch.getDatacenter(address), snitch.getRack(address)); + Location location = RackInferringSnitch.inferLocation(address); + ClusterMetadataTestHelper.addEndpoint(address, token, location.datacenter, location.rack); } } } @@ -218,8 +216,7 @@ public void testCalculateEndpoints() throws UnknownHostException { ServerTestUtils.resetCMS(); Random rand = new Random(run); - IEndpointSnitch snitch = generateSnitch(datacenters, nodes, rand); - DatabaseDescriptor.setEndpointSnitch(snitch); + Locator locator = generateLocator(datacenters, nodes, rand); for (int i = 0; i < NODES; ++i) // Nodes { @@ -228,14 +225,27 @@ public void testCalculateEndpoints() throws UnknownHostException { tokens.add(Murmur3Partitioner.instance.getRandomToken(rand)); } - ClusterMetadataTestHelper.addEndpoint(nodes.get(i), tokens, snitch.getDatacenter(nodes.get(i)), snitch.getRack(nodes.get(i))); + // Here we fake the registration status because we want all the nodes to be registered in cluster + // metadata using the locations we setup in generateLocator. This registration occurs as a part of + // the addEndpoint call here and behaves as expected for all nodes _except_ the one with the address + // which matches the local broadcast address (i.e. 127.0.0.1, which is #2 in the list of nodes). + // The location we want this to be registered with is {DC: rf5_1, rack: 3}, but while + // RegistrationStatus.instance indicates that the node is yet to be registered, the Locator will + // correctly return the initialization location obtained from + // DatabaseDescriptor::getInitialLocationProvider, which ultimately resolves to + // SimpleLocationProvider (because test/conf/cassandra.yaml specifies use of SimpleSnitch) and so + // we register that one node with the location {DC: datacenter1, rack: rack1}. + // This is purely an artefact of the contrived testing setup and in more realistic scenarios, + // including the majority of tests, isn't an issue. + RegistrationStatus.instance.onRegistration(); + ClusterMetadataTestHelper.addEndpoint(nodes.get(i), tokens, locator.location(nodes.get(i))); } - testEquivalence(ClusterMetadata.current(), snitch, datacenters, rand); + testEquivalence(ClusterMetadata.current(), locator, datacenters, rand); } } } - void testEquivalence(ClusterMetadata metadata, IEndpointSnitch snitch, Map<String, Integer> datacenters, Random rand) + void testEquivalence(ClusterMetadata metadata, Locator locator, Map<String, Integer> datacenters, Random rand) { NetworkTopologyStrategy nts = new NetworkTopologyStrategy("ks", datacenters.entrySet() @@ -244,7 +254,7 @@ void testEquivalence(ClusterMetadata metadata, IEndpointSnitch snitch, Map<Strin for (int i=0; i<1000; ++i) { Token token = Murmur3Partitioner.instance.getRandomToken(rand); - List<InetAddressAndPort> expected = calculateNaturalEndpoints(token, metadata, datacenters, snitch); + List<InetAddressAndPort> expected = calculateNaturalEndpoints(token, metadata, datacenters, locator); List<InetAddressAndPort> actual = new ArrayList<>(nts.calculateNaturalReplicas(token, metadata).endpoints()); if (endpointsDiffer(expected, actual)) { @@ -270,10 +280,11 @@ private boolean endpointsDiffer(List<InetAddressAndPort> ep1, List<InetAddressAn return !s1.equals(s2); } - IEndpointSnitch generateSnitch(Map<String, Integer> datacenters, Collection<InetAddressAndPort> nodes, Random rand) + Locator generateLocator(Map<String, Integer> datacenters, Collection<InetAddressAndPort> nodes, Random rand) { - final Map<InetAddressAndPort, String> nodeToRack = new HashMap<>(); - final Map<InetAddressAndPort, String> nodeToDC = new HashMap<>(); + final Map<NodeId, String> nodeToRack = new HashMap<>(); + final Map<NodeId, String> nodeToDC = new HashMap<>(); + final Map<InetAddressAndPort, NodeId> epToId = new HashMap<>(); Map<String, List<String>> racksPerDC = new HashMap<>(); datacenters.forEach((dc, rf) -> racksPerDC.put(dc, randomRacks(rf, rand))); int rf = datacenters.values().stream().mapToInt(x -> x).sum(); @@ -285,27 +296,33 @@ IEndpointSnitch generateSnitch(Map<String, Integer> datacenters, Collection<Inet dcs[pos++] = dce.getKey(); } + int id = 0; for (InetAddressAndPort node : nodes) { String dc = dcs[rand.nextInt(rf)]; List<String> racks = racksPerDC.get(dc); String rack = racks.get(rand.nextInt(racks.size())); - nodeToRack.put(node, rack); - nodeToDC.put(node, dc); + NodeId nodeId = new NodeId(++id); + nodeToRack.put(nodeId, rack); + nodeToDC.put(nodeId, dc); + epToId.put(node, nodeId); } - return new AbstractNetworkTopologySnitch() + Directory dir = new Directory() { - public String getRack(InetAddressAndPort endpoint) + @Override + public NodeId peerId(InetAddressAndPort endpoint) { - return nodeToRack.get(endpoint); + return epToId.get(endpoint); } - public String getDatacenter(InetAddressAndPort endpoint) + @Override + public Location location(NodeId id) { - return nodeToDC.get(endpoint); + return new Location(nodeToDC.get(id), nodeToRack.get(id)); } }; + return Locator.usingDirectory(dir); } private List<String> randomRacks(int rf, Random rand) @@ -318,7 +335,7 @@ private List<String> randomRacks(int rf, Random rand) } // Copy of older endpoints calculation algorithm for comparison - public static List<InetAddressAndPort> calculateNaturalEndpoints(Token searchToken, ClusterMetadata metadata, Map<String, Integer> datacenters, IEndpointSnitch snitch) + public static List<InetAddressAndPort> calculateNaturalEndpoints(Token searchToken, ClusterMetadata metadata, Map<String, Integer> datacenters, Locator locator) { // we want to preserve insertion order so that the first added endpoint becomes primary Set<InetAddressAndPort> replicas = new LinkedHashSet<>(); @@ -349,7 +366,8 @@ public static List<InetAddressAndPort> calculateNaturalEndpoints(Token searchTok { Token next = tokenIter.next(); InetAddressAndPort ep = metadata.directory.endpoint(metadata.tokenMap.owner(next)); - String dc = snitch.getDatacenter(ep); + Location location = locator.location(ep); + String dc = location.datacenter; // have we already found all replicas for this dc? if (!datacenters.containsKey(dc) || hasSufficientReplicas(dc, dcReplicas, allEndpoints, datacenters)) continue; @@ -361,7 +379,7 @@ public static List<InetAddressAndPort> calculateNaturalEndpoints(Token searchTok } else { - String rack = snitch.getRack(ep); + String rack = location.rack; // is this a new rack? if (seenRacks.get(dc).contains(rack)) { @@ -424,21 +442,18 @@ public void testTransientReplica() throws Exception { try (WithPartitioner m3p = new WithPartitioner(Murmur3Partitioner.instance)) { - IEndpointSnitch snitch = new SimpleSnitch(); - DatabaseDescriptor.setEndpointSnitch(snitch); - List<InetAddressAndPort> endpoints = Lists.newArrayList(InetAddressAndPort.getByName("127.0.0.1"), InetAddressAndPort.getByName("127.0.0.2"), InetAddressAndPort.getByName("127.0.0.3"), InetAddressAndPort.getByName("127.0.0.4")); - ClusterMetadataTestHelper.addEndpoint(endpoints.get(0), tk(100), SimpleSnitch.DATA_CENTER_NAME, SimpleSnitch.RACK_NAME); - ClusterMetadataTestHelper.addEndpoint(endpoints.get(1), tk(200), SimpleSnitch.DATA_CENTER_NAME, SimpleSnitch.RACK_NAME); - ClusterMetadataTestHelper.addEndpoint(endpoints.get(2), tk(300), SimpleSnitch.DATA_CENTER_NAME, SimpleSnitch.RACK_NAME); - ClusterMetadataTestHelper.addEndpoint(endpoints.get(3), tk(400), SimpleSnitch.DATA_CENTER_NAME, SimpleSnitch.RACK_NAME); + ClusterMetadataTestHelper.addEndpoint(endpoints.get(0), tk(100), LOCATION); + ClusterMetadataTestHelper.addEndpoint(endpoints.get(1), tk(200), LOCATION); + ClusterMetadataTestHelper.addEndpoint(endpoints.get(2), tk(300), LOCATION); + ClusterMetadataTestHelper.addEndpoint(endpoints.get(3), tk(400), LOCATION); Map<String, String> configOptions = new HashMap<>(); - configOptions.put(SimpleSnitch.DATA_CENTER_NAME, "3/1"); + configOptions.put(LOCATION.datacenter, "3/1"); NetworkTopologyStrategy strategy = new NetworkTopologyStrategy(KEYSPACE, configOptions); Util.assertRCEquals(EndpointsForRange.of(fullReplica(endpoints.get(0), range(400, 100)), diff --git a/test/unit/org/apache/cassandra/locator/PendingRangesTest.java b/test/unit/org/apache/cassandra/locator/PendingRangesTest.java index 15b82a686ac3..45e00a236a00 100644 --- a/test/unit/org/apache/cassandra/locator/PendingRangesTest.java +++ b/test/unit/org/apache/cassandra/locator/PendingRangesTest.java @@ -80,7 +80,7 @@ public void setup() { ClusterMetadataService.unsetInstance(); ClusterMetadataService.setInstance(ClusterMetadataTestHelper.syncInstanceForTest()); - ClusterMetadataService.instance().log().bootstrap(FBUtilities.getBroadcastAddressAndPort()); + ClusterMetadataService.instance().log().unsafeBootstrapForTesting(FBUtilities.getBroadcastAddressAndPort()); } @Test @@ -598,21 +598,21 @@ private static InetAddressAndPort peer(int addressSuffix) } } - private static IEndpointSnitch snitch() - { - return new AbstractNetworkTopologySnitch() - { - public String getRack(InetAddressAndPort endpoint) - { - return RACK1; - } - - public String getDatacenter(InetAddressAndPort endpoint) - { - return DC1; - } - }; - } +// private static IEndpointSnitch snitch() +// { +// return new AbstractNetworkTopologySnitch() +// { +// public String getRack(InetAddressAndPort endpoint) +// { +// return RACK1; +// } +// +// public String getDatacenter(InetAddressAndPort endpoint) +// { +// return DC1; +// } +// }; +// } private static AbstractReplicationStrategy simpleStrategy(int replicationFactor) { diff --git a/test/unit/org/apache/cassandra/locator/PropertyFileSnitchTest.java b/test/unit/org/apache/cassandra/locator/PropertyFileSnitchTest.java index e5f21e604785..35bdb98d8d0d 100644 --- a/test/unit/org/apache/cassandra/locator/PropertyFileSnitchTest.java +++ b/test/unit/org/apache/cassandra/locator/PropertyFileSnitchTest.java @@ -26,24 +26,21 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.regex.Matcher; import java.util.stream.Collectors; -import com.google.common.collect.ImmutableMap; import org.junit.After; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.StubClusterMetadataService; -import org.apache.cassandra.tcm.membership.MembershipUtils; import org.apache.cassandra.utils.FBUtilities; +import static org.apache.cassandra.locator.TopologyFileLocationProvider.PROPERTIES_FILENAME; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -68,7 +65,7 @@ public void setup() throws ConfigurationException, IOException { ClusterMetadataService.unsetInstance(); ClusterMetadataService.setInstance(StubClusterMetadataService.forTesting()); - String confFile = FBUtilities.resourceToFile(PropertyFileSnitch.SNITCH_PROPERTIES_FILENAME); + String confFile = FBUtilities.resourceToFile(PROPERTIES_FILENAME); effectiveFile = Paths.get(confFile); backupFile = Paths.get(confFile + ".bak"); localAddress = FBUtilities.getBroadcastAddressAndPort(); @@ -77,21 +74,24 @@ public void setup() throws ConfigurationException, IOException } @Test - public void localLocationPresentInConfig() throws IOException + public void localLocationsPresentInConfig() throws IOException { replaceConfigFile(Collections.singletonMap(localAddress.getHostAddressAndPort(), "DC1:RAC2")); PropertyFileSnitch snitch = new PropertyFileSnitch(); - assertEquals("DC1", snitch.getDatacenter(localAddress)); - assertEquals("RAC2", snitch.getRack(localAddress)); + // for registering a new node, location is obtained from the snitch config + assertEquals("DC1", snitch.getLocalDatacenter()); + assertEquals("RAC2", snitch.getLocalRack()); } @Test - public void localLocationNotPresentInConfig() throws IOException + public void locationsNotPresentInConfig() throws IOException { - replaceConfigFile(Collections.singletonMap("default", "DEFAULT_DC:DEFAULT_RACK")); + replaceConfigFile(Collections.singletonMap("default", "XXX_DEFAULT_DC:XXX_DEFAULT_RACK")); PropertyFileSnitch snitch = new PropertyFileSnitch(); - assertEquals("DEFAULT_DC", snitch.getDatacenter(localAddress)); - assertEquals("DEFAULT_RACK", snitch.getRack(localAddress)); + + // for registering a new node, location is obtained from the snitch config + assertEquals("XXX_DEFAULT_DC", snitch.getLocalDatacenter()); + assertEquals("XXX_DEFAULT_RACK", snitch.getLocalRack()); } @Test @@ -107,34 +107,11 @@ public void localAndDefaultLocationNotPresentInConfig() throws IOException { String expectedMessage = String.format("Snitch definitions at %s do not define a location for this node's " + "broadcast address %s, nor does it provides a default", - PropertyFileSnitch.SNITCH_PROPERTIES_FILENAME, localAddress); + PROPERTIES_FILENAME, localAddress); assertTrue(e.getMessage().contains(expectedMessage)); } } - @Test - public void configContainsRemoteConfig() throws IOException - { - // Locations of remote peers should not be accessible from this snitch unless - // they are present in ClusterMetadata - Random r = new Random(System.nanoTime()); - InetAddressAndPort peer = MembershipUtils.endpoint(99); - replaceConfigFile(ImmutableMap.of(localAddress.getHostAddressAndPort(), "DC1:RAC1", - peer.getHostAddressAndPort(), "OTHER_DC1:OTHER_RAC1")); - PropertyFileSnitch snitch = new PropertyFileSnitch(); - assertEquals("DC1", snitch.getDatacenter(localAddress)); - assertEquals("RAC1", snitch.getRack(localAddress)); - - assertEquals(PropertyFileSnitch.DEFAULT_DC, snitch.getDatacenter(peer)); - assertEquals(PropertyFileSnitch.DEFAULT_RACK, snitch.getRack(peer)); - - // Register peer, causing ClusterMetadata to be updated. Note that the location - // here is not the one in the config file, that should still be irrelevant - ClusterMetadataTestHelper.register(peer, "OTHER_DC2", "OTHER_RAC2"); - assertEquals("OTHER_DC2", snitch.getDatacenter(peer)); - assertEquals("OTHER_RAC2", snitch.getRack(peer)); - } - @After public void restoreOrigConfigFile() throws IOException { diff --git a/test/unit/org/apache/cassandra/locator/ReconnectableSnitchHelperTest.java b/test/unit/org/apache/cassandra/locator/ReconnectableSnitchHelperTest.java index d5a0ec36a28a..aa629eba2215 100644 --- a/test/unit/org/apache/cassandra/locator/ReconnectableSnitchHelperTest.java +++ b/test/unit/org/apache/cassandra/locator/ReconnectableSnitchHelperTest.java @@ -50,7 +50,7 @@ public void failedAuthentication() throws Exception DatabaseDescriptor.setInternodeAuthenticator(MessagingServiceTest.ALLOW_NOTHING_AUTHENTICATOR); InetAddressAndPort address = InetAddressAndPort.getByName("127.0.0.250"); //Should tolerate null returns by MS for the connection - ReconnectableSnitchHelper.reconnect(address, address, null, null); + ReconnectableSnitchHelper.reconnect(address, address, null); } @After diff --git a/test/unit/org/apache/cassandra/locator/ReplicaPlansTest.java b/test/unit/org/apache/cassandra/locator/ReplicaPlansTest.java index 7f946abbae1a..c8dbc806dcad 100644 --- a/test/unit/org/apache/cassandra/locator/ReplicaPlansTest.java +++ b/test/unit/org/apache/cassandra/locator/ReplicaPlansTest.java @@ -18,55 +18,51 @@ package org.apache.cassandra.locator; +import java.util.Map; +import java.util.Set; + import com.google.common.base.Predicates; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import org.junit.Before; +import org.junit.Test; + import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ConsistencyLevel; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.dht.Token; +import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.StubClusterMetadataService; -import org.junit.Before; -import org.junit.Test; - -import java.util.Map; -import java.util.Set; - import static org.apache.cassandra.locator.Replica.fullReplica; -import static org.apache.cassandra.locator.ReplicaUtils.*; +import static org.apache.cassandra.locator.ReplicaUtils.EP1; +import static org.apache.cassandra.locator.ReplicaUtils.EP2; +import static org.apache.cassandra.locator.ReplicaUtils.EP3; +import static org.apache.cassandra.locator.ReplicaUtils.EP4; +import static org.apache.cassandra.locator.ReplicaUtils.EP5; +import static org.apache.cassandra.locator.ReplicaUtils.EP6; +import static org.apache.cassandra.locator.ReplicaUtils.R1; +import static org.apache.cassandra.locator.ReplicaUtils.assertEquals; +import static org.apache.cassandra.locator.ReplicaUtils.tk; +import static org.apache.cassandra.locator.ReplicaUtils.trans; public class ReplicaPlansTest { - + static Keyspace ks; static { DatabaseDescriptor.daemonInitialization(); } - // TODO replace use of snitch in determining counts per DC with directory lookup - static class Snitch extends AbstractNetworkTopologySnitch + private static Keyspace ks(Set<InetAddressAndPort> dc1, Map<String, String> replication) { - final Set<InetAddressAndPort> dc1; - Snitch(Set<InetAddressAndPort> dc1) - { - this.dc1 = dc1; - } - @Override - public String getRack(InetAddressAndPort endpoint) - { - return dc1.contains(endpoint) ? "R1" : "R2"; - } - - @Override - public String getDatacenter(InetAddressAndPort endpoint) - { - return dc1.contains(endpoint) ? "DC1" : "DC2"; - } + replication = ImmutableMap.<String, String>builder().putAll(replication).put("class", "NetworkTopologyStrategy").build(); + Keyspace keyspace = Keyspace.mockKS(KeyspaceMetadata.create("blah", KeyspaceParams.create(false, replication))); + return keyspace; } @Before @@ -74,59 +70,42 @@ public void setup() { ClusterMetadataService.unsetInstance(); ClusterMetadataService.setInstance(StubClusterMetadataService.forTesting()); - } - - private static Keyspace ks(Set<InetAddressAndPort> dc1, Map<String, String> replication) - { - replication = ImmutableMap.<String, String>builder().putAll(replication).put("class", "NetworkTopologyStrategy").build(); - Keyspace keyspace = Keyspace.mockKS(KeyspaceMetadata.create("blah", KeyspaceParams.create(false, replication))); - Snitch snitch = new Snitch(dc1); - DatabaseDescriptor.setEndpointSnitch(snitch); - return keyspace; + ClusterMetadataTestHelper.register(EP1, "DC1", "R1"); + ClusterMetadataTestHelper.register(EP2, "DC1", "R1"); + ClusterMetadataTestHelper.register(EP3, "DC1", "R1"); + ClusterMetadataTestHelper.register(EP4, "DC2", "R2"); + ClusterMetadataTestHelper.register(EP5, "DC2", "R2"); + ClusterMetadataTestHelper.register(EP6, "DC2", "R2"); } private static Replica full(InetAddressAndPort ep) { return fullReplica(ep, R1); } - - @Test public void testWriteEachQuorum() { - IEndpointSnitch stash = DatabaseDescriptor.getEndpointSnitch(); final Token token = tk(1L); - try { - { - // all full natural - Keyspace ks = ks(ImmutableSet.of(EP1, EP2, EP3), ImmutableMap.of("DC1", "3", "DC2", "3")); - EndpointsForToken natural = EndpointsForToken.of(token, full(EP1), full(EP2), full(EP3), full(EP4), full(EP5), full(EP6)); - EndpointsForToken pending = EndpointsForToken.empty(token); - ReplicaPlan.ForWrite plan = ReplicaPlans.forWrite(ks, ConsistencyLevel.EACH_QUORUM, (cm) -> natural, (cm) -> pending, null, Predicates.alwaysTrue(), ReplicaPlans.writeNormal); - assertEquals(natural, plan.liveAndDown); - assertEquals(natural, plan.live); - assertEquals(natural, plan.contacts()); - } - { - // all natural and up, one transient in each DC - Keyspace ks = ks(ImmutableSet.of(EP1, EP2, EP3), ImmutableMap.of("DC1", "3", "DC2", "3")); - EndpointsForToken natural = EndpointsForToken.of(token, full(EP1), full(EP2), trans(EP3), full(EP4), full(EP5), trans(EP6)); - EndpointsForToken pending = EndpointsForToken.empty(token); - ReplicaPlan.ForWrite plan = ReplicaPlans.forWrite(ks, ConsistencyLevel.EACH_QUORUM, (cm) -> natural, (cm) -> pending, Epoch.FIRST, Predicates.alwaysTrue(), ReplicaPlans.writeNormal); - assertEquals(natural, plan.liveAndDown); - assertEquals(natural, plan.live); - EndpointsForToken expectContacts = EndpointsForToken.of(token, full(EP1), full(EP2), full(EP4), full(EP5)); - assertEquals(expectContacts, plan.contacts()); - } + // all full natural + Keyspace ks = ks(ImmutableSet.of(EP1, EP2, EP3), ImmutableMap.of("DC1", "3", "DC2", "3")); + EndpointsForToken natural = EndpointsForToken.of(token, full(EP1), full(EP2), full(EP3), full(EP4), full(EP5), full(EP6)); + EndpointsForToken pending = EndpointsForToken.empty(token); + ReplicaPlan.ForWrite plan = ReplicaPlans.forWrite(ks, ConsistencyLevel.EACH_QUORUM, (cm) -> natural, (cm) -> pending, null, Predicates.alwaysTrue(), ReplicaPlans.writeNormal); + assertEquals(natural, plan.liveAndDown); + assertEquals(natural, plan.live); + assertEquals(natural, plan.contacts()); } - finally { - DatabaseDescriptor.setEndpointSnitch(stash); - } - - { - // test simple - + // all natural and up, one transient in each DC + // Note: this is confusing because it looks misconfigured as the Keyspace has never been setup with any + // transient replicas in its replication params. + Keyspace ks = ks(ImmutableSet.of(EP1, EP2, EP3), ImmutableMap.of("DC1", "3", "DC2", "3")); + EndpointsForToken natural = EndpointsForToken.of(token, full(EP1), full(EP2), trans(EP3), full(EP4), full(EP5), trans(EP6)); + EndpointsForToken pending = EndpointsForToken.empty(token); + ReplicaPlan.ForWrite plan = ReplicaPlans.forWrite(ks, ConsistencyLevel.EACH_QUORUM, (cm) -> natural, (cm) -> pending, Epoch.FIRST, Predicates.alwaysTrue(), ReplicaPlans.writeNormal); + assertEquals(natural, plan.liveAndDown); + assertEquals(natural, plan.live); + EndpointsForToken expectContacts = EndpointsForToken.of(token, full(EP1), full(EP2), full(EP4), full(EP5)); + assertEquals(expectContacts, plan.contacts()); } } - } diff --git a/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java b/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java index 83570350009b..874e50d17a56 100644 --- a/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java +++ b/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java @@ -85,12 +85,6 @@ public static void withPartitioner(IPartitioner partitioner) DatabaseDescriptor.setTransientReplicationEnabledUnsafe(true); } - @Before - public void resetSnitch() - { - DatabaseDescriptor.setEndpointSnitch(new SimpleSnitch()); - } - @Test public void tryValidKeyspace() { @@ -128,8 +122,6 @@ public void testStringEndpoints() throws UnknownHostException public void testMultiDCSimpleStrategyEndpoints() throws UnknownHostException { withPartitioner(Murmur3Partitioner.instance); - IEndpointSnitch snitch = new PropertyFileSnitch(); - DatabaseDescriptor.setEndpointSnitch(snitch); // Topology taken directly from the topology_test.test_size_estimates_multidc dtest that regressed Multimap<InetAddressAndPort, Token> dc1 = HashMultimap.create(); @@ -275,8 +267,6 @@ private static Range<Token> range(long l, long r) public void transientReplica() throws Exception { withPartitioner(Murmur3Partitioner.instance); - IEndpointSnitch snitch = new SimpleSnitch(); - DatabaseDescriptor.setEndpointSnitch(snitch); List<InetAddressAndPort> endpoints = Lists.newArrayList(InetAddressAndPort.getByName("127.0.0.1"), InetAddressAndPort.getByName("127.0.0.2"), @@ -321,9 +311,6 @@ public void testSimpleStrategyThrowsConfigurationException() throws Configuratio expectedEx.expect(ConfigurationException.class); expectedEx.expectMessage("SimpleStrategy requires a replication_factor strategy option."); - IEndpointSnitch snitch = new SimpleSnitch(); - DatabaseDescriptor.setEndpointSnitch(snitch); - List<InetAddressAndPort> endpoints = Lists.newArrayList(InetAddressAndPort.getByName("127.0.0.1"), InetAddressAndPort.getByName("127.0.0.2"), InetAddressAndPort.getByName("127.0.0.3")); diff --git a/test/unit/org/apache/cassandra/locator/SnitchUtilsTest.java b/test/unit/org/apache/cassandra/locator/SnitchUtilsTest.java index b5b140098832..c203c2b8519e 100644 --- a/test/unit/org/apache/cassandra/locator/SnitchUtilsTest.java +++ b/test/unit/org/apache/cassandra/locator/SnitchUtilsTest.java @@ -20,6 +20,7 @@ import org.junit.Test; +import org.apache.cassandra.tcm.membership.Location; import org.apache.cassandra.utils.Pair; import static org.assertj.core.api.Assertions.assertThatExceptionOfType; @@ -33,13 +34,23 @@ public void testSnitchUtils() Pair<String, String> result = SnitchUtils.parseDcAndRack("my-dc-rack1", ""); assertEquals("my-dc", result.left); assertEquals("rack1", result.right); + Location location = SnitchUtils.parseLocation("my-dc-rack1", ""); + assertEquals("my-dc", location.datacenter); + assertEquals("rack1", location.rack); result = SnitchUtils.parseDcAndRack("my-rack", ""); assertEquals("my", result.left); assertEquals("rack", result.right); + location = SnitchUtils.parseLocation("my-rack", ""); + assertEquals("my", location.datacenter); + assertEquals("rack", location.rack); assertThatExceptionOfType(IllegalStateException.class) .isThrownBy(() -> SnitchUtils.parseDcAndRack("myresponse", "")) .withMessage("myresponse does not contain at least one '-' to differentiate between datacenter and rack"); + + assertThatExceptionOfType(IllegalStateException.class) + .isThrownBy(() -> SnitchUtils.parseLocation("myresponse", "")) + .withMessage("myresponse does not contain at least one '-' to differentiate between datacenter and rack"); } } diff --git a/test/unit/org/apache/cassandra/locator/WithPartitioner.java b/test/unit/org/apache/cassandra/locator/WithPartitioner.java index 94a07a6e5f83..48738e27fd05 100644 --- a/test/unit/org/apache/cassandra/locator/WithPartitioner.java +++ b/test/unit/org/apache/cassandra/locator/WithPartitioner.java @@ -41,6 +41,7 @@ public void close() { ClusterMetadataService.unsetInstance(); ClusterMetadataService.setInstance(toRestore); + DatabaseDescriptor.setPartitionerUnsafe(toRestore.metadata().partitioner); } } diff --git a/test/unit/org/apache/cassandra/net/AsyncStreamingOutputPlusTest.java b/test/unit/org/apache/cassandra/net/AsyncStreamingOutputPlusTest.java index 4775fd1d67d7..0fd923f53375 100644 --- a/test/unit/org/apache/cassandra/net/AsyncStreamingOutputPlusTest.java +++ b/test/unit/org/apache/cassandra/net/AsyncStreamingOutputPlusTest.java @@ -24,12 +24,12 @@ import java.nio.file.Files; import java.util.Random; -import org.apache.cassandra.io.util.File; import org.junit.Test; import io.netty.buffer.ByteBuf; import io.netty.channel.embedded.EmbeddedChannel; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.io.util.File; import org.apache.cassandra.streaming.StreamManager; import org.apache.cassandra.utils.FBUtilities; diff --git a/test/unit/org/apache/cassandra/net/ConnectionTest.java b/test/unit/org/apache/cassandra/net/ConnectionTest.java index 6225ba64543f..42c137cc5f63 100644 --- a/test/unit/org/apache/cassandra/net/ConnectionTest.java +++ b/test/unit/org/apache/cassandra/net/ConnectionTest.java @@ -122,6 +122,7 @@ public static void startup() { DatabaseDescriptor.daemonInitialization(); ClusterMetadataTestHelper.setInstanceForTest(); + ClusterMetadataTestHelper.register(FBUtilities.getBroadcastAddressAndPort()); CommitLog.instance.start(); } diff --git a/test/unit/org/apache/cassandra/net/HandshakeTest.java b/test/unit/org/apache/cassandra/net/HandshakeTest.java index b66ade056e9b..c84643497d5d 100644 --- a/test/unit/org/apache/cassandra/net/HandshakeTest.java +++ b/test/unit/org/apache/cassandra/net/HandshakeTest.java @@ -75,6 +75,8 @@ public static void startup() { DatabaseDescriptor.daemonInitialization(); ClusterMetadataTestHelper.setInstanceForTest(); + ClusterMetadataTestHelper.register(TO_ADDR); + ClusterMetadataTestHelper.register(FROM_ADDR); CommitLog.instance.start(); } diff --git a/test/unit/org/apache/cassandra/net/MessagingServiceTest.java b/test/unit/org/apache/cassandra/net/MessagingServiceTest.java index f1087efb101c..95c72f4bc0b3 100644 --- a/test/unit/org/apache/cassandra/net/MessagingServiceTest.java +++ b/test/unit/org/apache/cassandra/net/MessagingServiceTest.java @@ -117,6 +117,7 @@ public static void beforeClass() throws UnknownHostException ClusterMetadataTestHelper.setInstanceForTest(); CommitLog.instance.start(); DatabaseDescriptor.setBroadcastAddress(InetAddress.getByName("127.0.0.1")); + ClusterMetadataTestHelper.register(FBUtilities.getBroadcastAddressAndPort(), "datacenter1", "rack1"); originalAuthenticator = DatabaseDescriptor.getInternodeAuthenticator(); originalServerEncryptionOptions = DatabaseDescriptor.getInternodeMessagingEncyptionOptions(); originalListenAddress = InetAddressAndPort.getByAddressOverrideDefaults(DatabaseDescriptor.getListenAddress(), DatabaseDescriptor.getStoragePort()); diff --git a/test/unit/org/apache/cassandra/net/OutboundConnectionSettingsTest.java b/test/unit/org/apache/cassandra/net/OutboundConnectionSettingsTest.java index 89549b4f3f7e..5dae8af48fa0 100644 --- a/test/unit/org/apache/cassandra/net/OutboundConnectionSettingsTest.java +++ b/test/unit/org/apache/cassandra/net/OutboundConnectionSettingsTest.java @@ -18,27 +18,29 @@ package org.apache.cassandra.net; -import java.util.HashMap; -import java.util.Map; import java.util.function.Function; import org.junit.Assert; +import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; import org.apache.cassandra.config.Config; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.commitlog.CommitLog; -import org.apache.cassandra.locator.AbstractEndpointSnitch; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.locator.Replica; +import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; +import org.apache.cassandra.tcm.ClusterMetadataService; +import org.apache.cassandra.tcm.StubClusterMetadataService; -import static org.apache.cassandra.config.DatabaseDescriptor.getEndpointSnitch; import static org.apache.cassandra.net.OutboundConnectionsTest.LOCAL_ADDR; import static org.apache.cassandra.net.OutboundConnectionsTest.REMOTE_ADDR; public class OutboundConnectionSettingsTest { + private static final String DC1 = "dc1"; + private static final String DC2 = "dc2"; + private static final String RACK = "rack1"; + @BeforeClass public static void before() { @@ -46,6 +48,13 @@ public static void before() CommitLog.instance.start(); } + @Before + public void reset() + { + ClusterMetadataService.unsetInstance(); + ClusterMetadataService.setInstance(StubClusterMetadataService.forTesting()); + } + @Test (expected = IllegalArgumentException.class) public void build_SmallSendSize() { @@ -81,65 +90,47 @@ private static void test(Function<OutboundConnectionSettings, OutboundConnection f.apply(new OutboundConnectionSettings(LOCAL_ADDR)).withDefaults(ConnectionCategory.MESSAGING); } - private static class TestSnitch extends AbstractEndpointSnitch - { - private final Map<InetAddressAndPort, String> nodeToDc = new HashMap<>(); - - void add(InetAddressAndPort node, String dc) - { - nodeToDc.put(node, dc); - } - - public String getRack(InetAddressAndPort endpoint) - { - return null; - } - - public String getDatacenter(InetAddressAndPort endpoint) - { - return nodeToDc.get(endpoint); - } - - public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2) - { - return 0; - } - } - @Test public void shouldCompressConnection_None() { DatabaseDescriptor.setInternodeCompression(Config.InternodeCompression.none); - Assert.assertFalse(OutboundConnectionSettings.shouldCompressConnection(getEndpointSnitch(), LOCAL_ADDR, REMOTE_ADDR)); + Assert.assertFalse(OutboundConnectionSettings.shouldCompressConnection(LOCAL_ADDR, REMOTE_ADDR)); } @Test public void shouldCompressConnection_DifferentDc() { - TestSnitch snitch = new TestSnitch(); - snitch.add(LOCAL_ADDR, "dc1"); - snitch.add(REMOTE_ADDR, "dc2"); - DatabaseDescriptor.setEndpointSnitch(snitch); + ClusterMetadataTestHelper.register(LOCAL_ADDR, DC1, RACK); + ClusterMetadataTestHelper.register(REMOTE_ADDR, DC2, RACK); + DatabaseDescriptor.setInternodeCompression(Config.InternodeCompression.dc); + Assert.assertTrue(OutboundConnectionSettings.shouldCompressConnection(LOCAL_ADDR, REMOTE_ADDR)); + } + + @Test + public void shouldCompressConnection_MetadataNotInitialized() + { + // if cluster metadata isn't yet available then we assume that every peer is remote. + // connections will be re-established once the cluster metadata service becomes available + ClusterMetadataTestHelper.register(LOCAL_ADDR, DC1, RACK); + ClusterMetadataTestHelper.register(REMOTE_ADDR, DC2, RACK); DatabaseDescriptor.setInternodeCompression(Config.InternodeCompression.dc); - Assert.assertTrue(OutboundConnectionSettings.shouldCompressConnection(getEndpointSnitch(), LOCAL_ADDR, REMOTE_ADDR)); + Assert.assertTrue(OutboundConnectionSettings.shouldCompressConnection(LOCAL_ADDR, REMOTE_ADDR)); } @Test public void shouldCompressConnection_All() { DatabaseDescriptor.setInternodeCompression(Config.InternodeCompression.all); - Assert.assertTrue(OutboundConnectionSettings.shouldCompressConnection(getEndpointSnitch(), LOCAL_ADDR, REMOTE_ADDR)); + Assert.assertTrue(OutboundConnectionSettings.shouldCompressConnection(LOCAL_ADDR, REMOTE_ADDR)); } @Test public void shouldCompressConnection_SameDc() { - TestSnitch snitch = new TestSnitch(); - snitch.add(LOCAL_ADDR, "dc1"); - snitch.add(REMOTE_ADDR, "dc1"); - DatabaseDescriptor.setEndpointSnitch(snitch); + ClusterMetadataTestHelper.register(LOCAL_ADDR, DC1, RACK); + ClusterMetadataTestHelper.register(REMOTE_ADDR, DC1, RACK); DatabaseDescriptor.setInternodeCompression(Config.InternodeCompression.dc); - Assert.assertFalse(OutboundConnectionSettings.shouldCompressConnection(getEndpointSnitch(), LOCAL_ADDR, REMOTE_ADDR)); + Assert.assertFalse(OutboundConnectionSettings.shouldCompressConnection(LOCAL_ADDR, REMOTE_ADDR)); } } diff --git a/test/unit/org/apache/cassandra/net/OutboundConnectionsTest.java b/test/unit/org/apache/cassandra/net/OutboundConnectionsTest.java index 5afbe8831570..0f184ee8b7d5 100644 --- a/test/unit/org/apache/cassandra/net/OutboundConnectionsTest.java +++ b/test/unit/org/apache/cassandra/net/OutboundConnectionsTest.java @@ -41,6 +41,7 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.utils.FBUtilities; import static org.apache.cassandra.net.MessagingService.current_version; import static org.apache.cassandra.net.OutboundConnections.LARGE_MESSAGE_THRESHOLD; @@ -78,6 +79,11 @@ public static void before() { DatabaseDescriptor.daemonInitialization(); ClusterMetadataTestHelper.setInstanceForTest(); + // register the local broadcast address and REMOTE_ADDRESS addresses so the snitch is able to retrieve them + // from cluster metadata. This is necessary for the default of OutboundConnectionSettings::tcpNoDelay. + // There's no need to register RECONNECT_ADDRESS and it would be incorrect to do so + ClusterMetadataTestHelper.register(FBUtilities.getBroadcastAddressAndPort()); + ClusterMetadataTestHelper.register(REMOTE_ADDR); CommitLog.instance.start(); } diff --git a/test/unit/org/apache/cassandra/net/ProxyHandlerConnectionsTest.java b/test/unit/org/apache/cassandra/net/ProxyHandlerConnectionsTest.java index 47669e192ede..6b49fafe0e6a 100644 --- a/test/unit/org/apache/cassandra/net/ProxyHandlerConnectionsTest.java +++ b/test/unit/org/apache/cassandra/net/ProxyHandlerConnectionsTest.java @@ -41,6 +41,7 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.commitlog.CommitLog; import org.apache.cassandra.db.compaction.CompactionManager; +import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; import org.apache.cassandra.io.IVersionedAsymmetricSerializer; import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; @@ -87,6 +88,8 @@ public static void startup() // call these to initialize everything in case a message is dropped, otherwise we will NPE in the commitlog CommitLog.instance.start(); ServerTestUtils.initCMS(); + // register the local broadcast address so the snitch can retrieve it when setting up connection defaults + ClusterMetadataTestHelper.register(FBUtilities.getBroadcastAddressAndPort()); CompactionManager.instance.getPendingTasks(); } diff --git a/test/unit/org/apache/cassandra/repair/FuzzTestBase.java b/test/unit/org/apache/cassandra/repair/FuzzTestBase.java index b4aa22930f22..ad5e8220e280 100644 --- a/test/unit/org/apache/cassandra/repair/FuzzTestBase.java +++ b/test/unit/org/apache/cassandra/repair/FuzzTestBase.java @@ -95,9 +95,9 @@ import org.apache.cassandra.gms.VersionedValue; import org.apache.cassandra.io.util.DataInputBuffer; import org.apache.cassandra.io.util.DataOutputBuffer; -import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.LocalStrategy; +import org.apache.cassandra.locator.Locator; import org.apache.cassandra.locator.RangesAtEndpoint; import org.apache.cassandra.net.ConnectionType; import org.apache.cassandra.net.IVerbHandler; @@ -137,6 +137,7 @@ import org.apache.cassandra.streaming.StreamingChannel; import org.apache.cassandra.streaming.StreamingDataInputPlus; import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.membership.Location; import org.apache.cassandra.tools.nodetool.Repair; import org.apache.cassandra.utils.AbstractTypeGenerators; import org.apache.cassandra.utils.CassandraGenerators; @@ -195,7 +196,6 @@ public static void setUpClass() DatabaseDescriptor.setCommitLogWriteDiskAccessMode(Config.DiskAccessMode.mmap); DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); // TOOD (coverage): random select - DatabaseDescriptor.setLocalDataCenter("test"); StreamingChannel.Factory.Global.unsafeSet(new StreamingChannel.Factory() { private final AtomicInteger counter = new AtomicInteger(); @@ -674,7 +674,7 @@ static class Cluster final Map<InetAddressAndPort, Node> nodes; private final IFailureDetector failureDetector = Mockito.mock(IFailureDetector.class); - private final IEndpointSnitch snitch = Mockito.mock(IEndpointSnitch.class); + private final Locator locator = Mockito.mock(Locator.class); private final SimulatedExecutorFactory globalExecutor; final ScheduledExecutorPlus unorderedScheduled; final ExecutorPlus orderedExecutor; @@ -732,8 +732,8 @@ static class Cluster String dc = rs.pick(dcs); String rack = "rack"; - Mockito.when(snitch.getDatacenter(Mockito.eq(addressAndPort))).thenReturn(dc); - Mockito.when(snitch.getRack(Mockito.eq(addressAndPort))).thenReturn(rack); + Location location = new Location(dc, rack); + Mockito.when(locator.location(Mockito.eq(addressAndPort))).thenReturn(location); VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(DatabaseDescriptor.getPartitioner()); EndpointState state = new EndpointState(new HeartBeatState(42, 42)); @@ -1221,9 +1221,9 @@ public IFailureDetector failureDetector() } @Override - public IEndpointSnitch snitch() + public Locator locator() { - return snitch; + return locator; } @Override diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java index 42ed428c76ba..83e060fe3246 100644 --- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java +++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java @@ -45,8 +45,6 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.locator.AbstractNetworkTopologySnitch; -import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.WithPartitioner; import org.apache.cassandra.schema.KeyspaceMetadata; @@ -58,7 +56,6 @@ import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; -import org.apache.cassandra.tcm.membership.Location; import org.apache.cassandra.tcm.membership.NodeAddresses; import org.apache.cassandra.tcm.membership.NodeId; import org.apache.cassandra.tcm.membership.NodeVersion; @@ -87,32 +84,7 @@ public static void setUp() throws ConfigurationException, UnknownHostException GOSSIP_DISABLE_THREAD_VALIDATION.setBoolean(true); ServerTestUtils.daemonInitialization(); DatabaseDescriptor.setPartitionerUnsafe(OrderPreservingPartitioner.instance); - IEndpointSnitch snitch = new AbstractNetworkTopologySnitch() - { - @Override - public String getRack(InetAddressAndPort endpoint) - { - return location(endpoint).rack; - } - - @Override - public String getDatacenter(InetAddressAndPort endpoint) - { - return location(endpoint).datacenter; - } - - private Location location(InetAddressAndPort endpoint) - { - ClusterMetadata metadata = ClusterMetadata.current(); - NodeId id = metadata.directory.peerId(endpoint); - if (id == null) - throw new IllegalArgumentException("Unknown endpoint " + endpoint); - return metadata.directory.location(id); - } - }; ServerTestUtils.prepareServerNoRegister(); - DatabaseDescriptor.setEndpointSnitch(snitch); - id1 = InetAddressAndPort.getByName("127.0.0.1"); id2 = InetAddressAndPort.getByName("127.0.0.2"); id3 = InetAddressAndPort.getByName("127.0.0.3"); diff --git a/test/unit/org/apache/cassandra/service/StorageServiceTest.java b/test/unit/org/apache/cassandra/service/StorageServiceTest.java index 725f105db832..0b7ef7b1c471 100644 --- a/test/unit/org/apache/cassandra/service/StorageServiceTest.java +++ b/test/unit/org/apache/cassandra/service/StorageServiceTest.java @@ -30,15 +30,12 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.commitlog.CommitLog; import org.apache.cassandra.distributed.test.TestBaseImpl; -import org.apache.cassandra.locator.AbstractEndpointSnitch; -import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.locator.ReplicaCollection; import org.apache.cassandra.locator.ReplicaMultimap; -import org.apache.cassandra.locator.SimpleSnitch; +import org.apache.cassandra.locator.SimpleLocationProvider; import org.apache.cassandra.tcm.ClusterMetadataService; -import org.apache.cassandra.tcm.membership.Location; import org.apache.cassandra.tcm.membership.NodeAddresses; import org.apache.cassandra.tcm.membership.NodeVersion; import org.apache.cassandra.tcm.transformations.Register; @@ -73,27 +70,8 @@ public static void setUpClass() throws Exception DatabaseDescriptor.setTransientReplicationEnabledUnsafe(true); ClusterMetadataService.instance().commit(new Register(NodeAddresses.current(), - new Location(SimpleSnitch.DATA_CENTER_NAME, SimpleSnitch.RACK_NAME), + SimpleLocationProvider.LOCATION, NodeVersion.CURRENT)); - IEndpointSnitch snitch = new AbstractEndpointSnitch() - { - public int compareEndpoints(InetAddressAndPort target, Replica r1, Replica r2) - { - return 0; - } - - public String getRack(InetAddressAndPort endpoint) - { - return "R1"; - } - - public String getDatacenter(InetAddressAndPort endpoint) - { - return "DC1"; - } - }; - - DatabaseDescriptor.setEndpointSnitch(snitch); CommitLog.instance.start(); } @@ -298,7 +276,7 @@ public void testRebuildFailOnNonExistingDatacenter() { assertEquals(String.format("Provided datacenter '%s' is not a valid datacenter, available datacenters are: %s", nonExistentDC, - SimpleSnitch.DATA_CENTER_NAME), + SimpleLocationProvider.LOCATION.datacenter), ex.getMessage()); } } diff --git a/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java b/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java index 6356f3ce1a9d..03785f3c305b 100644 --- a/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java +++ b/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java @@ -36,13 +36,15 @@ import org.apache.cassandra.db.WriteType; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.exceptions.RequestFailureReason; +import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; import org.apache.cassandra.locator.EndpointsForToken; -import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.NodeProximity; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.locator.ReplicaCollection; import org.apache.cassandra.locator.ReplicaPlans; import org.apache.cassandra.locator.ReplicaUtils; +import org.apache.cassandra.locator.BaseProximity; import org.apache.cassandra.net.Message; import org.apache.cassandra.net.Verb; import org.apache.cassandra.schema.KeyspaceParams; @@ -81,26 +83,8 @@ public static void setUpClass() throws Throwable SchemaLoader.loadSchema(); DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); // Register peers with expected DC for NetworkTopologyStrategy. -// metadata.clearUnsafe(); -// metadata.updateHostId(UUID.randomUUID(), InetAddressAndPort.getByName("127.1.0.255")); -// metadata.updateHostId(UUID.randomUUID(), InetAddressAndPort.getByName("127.2.0.255")); - - DatabaseDescriptor.setEndpointSnitch(new IEndpointSnitch() + NodeProximity sorter = new BaseProximity() { - public String getRack(InetAddressAndPort endpoint) - { - return null; - } - - public String getDatacenter(InetAddressAndPort endpoint) - { - byte[] address = endpoint.getAddress().getAddress(); - if (address[1] == 1) - return "datacenter1"; - else - return "datacenter2"; - } - public <C extends ReplicaCollection<? extends C>> C sortedByProximity(InetAddressAndPort address, C replicas) { return replicas; @@ -111,24 +95,22 @@ public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2) return 0; } - public void gossiperStarting() - { - - } - public boolean isWorthMergingForRangeQuery(ReplicaCollection<?> merged, ReplicaCollection<?> l1, ReplicaCollection<?> l2) { return false; } - }); + }; + DatabaseDescriptor.setNodeProximity(sorter); DatabaseDescriptor.setBroadcastAddress(InetAddress.getByName("127.1.0.1")); - SchemaLoader.createKeyspace("Foo", KeyspaceParams.nts("datacenter1", 3, "datacenter2", 3), SchemaLoader.standardCFMD("Foo", "Bar")); - ks = Keyspace.open("Foo"); - cfs = ks.getColumnFamilyStore("Bar"); targets = EndpointsForToken.of(DatabaseDescriptor.getPartitioner().getToken(ByteBufferUtil.bytes(0)), full("127.1.0.255"), full("127.1.0.254"), full("127.1.0.253"), full("127.2.0.255"), full("127.2.0.254"), full("127.2.0.253")); + for (InetAddressAndPort ep : targets.endpoints()) + ClusterMetadataTestHelper.register(ep, ep.addressBytes[1] == 1 ? "datacenter1" : "datacenter2", "rack1"); pending = EndpointsForToken.empty(DatabaseDescriptor.getPartitioner().getToken(ByteBufferUtil.bytes(0))); + SchemaLoader.createKeyspace("Foo", KeyspaceParams.nts("datacenter1", 3, "datacenter2", 3), SchemaLoader.standardCFMD("Foo", "Bar")); + ks = Keyspace.open("Foo"); + cfs = ks.getColumnFamilyStore("Bar"); } @Before diff --git a/test/unit/org/apache/cassandra/service/WriteResponseHandlerTransientTest.java b/test/unit/org/apache/cassandra/service/WriteResponseHandlerTransientTest.java index 7eef9d7defbc..ed844343a4c9 100644 --- a/test/unit/org/apache/cassandra/service/WriteResponseHandlerTransientTest.java +++ b/test/unit/org/apache/cassandra/service/WriteResponseHandlerTransientTest.java @@ -18,7 +18,6 @@ package org.apache.cassandra.service; -import java.net.InetAddress; import java.net.UnknownHostException; import java.util.Set; import java.util.function.Predicate; @@ -36,10 +35,10 @@ import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.dht.Token; +import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; import org.apache.cassandra.exceptions.UnavailableException; import org.apache.cassandra.locator.EndpointsForRange; import org.apache.cassandra.locator.EndpointsForToken; -import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.locator.ReplicaCollection; @@ -93,49 +92,13 @@ public static void setupClass() throws Throwable DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); // Register peers with expected DC for NetworkTopologyStrategy. - -// metadata.clearUnsafe(); -// metadata.updateHostId(UUID.randomUUID(), InetAddressAndPort.getByName("127.1.0.1")); -// metadata.updateHostId(UUID.randomUUID(), InetAddressAndPort.getByName("127.2.0.1")); - - DatabaseDescriptor.setEndpointSnitch(new IEndpointSnitch() - { - public String getRack(InetAddressAndPort endpoint) - { - return null; - } - - public String getDatacenter(InetAddressAndPort endpoint) - { - byte[] address = endpoint.getAddress().getAddress(); - if (address[1] == 1) - return DC1; - else - return DC2; - } - - public <C extends ReplicaCollection<? extends C>> C sortedByProximity(InetAddressAndPort address, C unsortedAddress) - { - return unsortedAddress; - } - - public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2) - { - return 0; - } - - public void gossiperStarting() - { - - } - - public boolean isWorthMergingForRangeQuery(ReplicaCollection<?> merged, ReplicaCollection<?> l1, ReplicaCollection<?> l2) - { - return false; - } - }); - - DatabaseDescriptor.setBroadcastAddress(InetAddress.getByName("127.1.0.1")); + DatabaseDescriptor.setBroadcastAddress(EP1.getAddress()); + ClusterMetadataTestHelper.register(EP1, DC1, "r1"); + ClusterMetadataTestHelper.register(EP2, DC1, "r1"); + ClusterMetadataTestHelper.register(EP3, DC1, "r1"); + ClusterMetadataTestHelper.register(EP4, DC2, "r1"); + ClusterMetadataTestHelper.register(EP5, DC2, "r1"); + ClusterMetadataTestHelper.register(EP6, DC2, "r1"); SchemaLoader.createKeyspace("ks", KeyspaceParams.nts(DC1, "3/1", DC2, "3/1"), SchemaLoader.standardCFMD("ks", "tbl")); ks = Keyspace.open("ks"); cfs = ks.getColumnFamilyStore("tbl"); diff --git a/test/unit/org/apache/cassandra/service/reads/ReadExecutorTest.java b/test/unit/org/apache/cassandra/service/reads/ReadExecutorTest.java index fdafc9e388ad..046da259e98e 100644 --- a/test/unit/org/apache/cassandra/service/reads/ReadExecutorTest.java +++ b/test/unit/org/apache/cassandra/service/reads/ReadExecutorTest.java @@ -22,8 +22,10 @@ import org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.cassandra.ServerTestUtils; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.dht.Token; +import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; import org.apache.cassandra.locator.ReplicaPlan; import org.junit.Before; import org.junit.BeforeClass; @@ -76,6 +78,8 @@ public static void setUpClass() throws Throwable full(InetAddressAndPort.getByName("127.0.0.254")), full(InetAddressAndPort.getByName("127.0.0.253")) ); + for (InetAddressAndPort ep : targets.endpoints()) + ClusterMetadataTestHelper.register(ep, ServerTestUtils.DATA_CENTER, ServerTestUtils.RACK1); cfs.sampleReadLatencyMicros = 0; } diff --git a/test/unit/org/apache/cassandra/service/reads/range/ReplicaPlanMergerTest.java b/test/unit/org/apache/cassandra/service/reads/range/ReplicaPlanMergerTest.java index 8d4fee6d187e..aaa88c938dd3 100644 --- a/test/unit/org/apache/cassandra/service/reads/range/ReplicaPlanMergerTest.java +++ b/test/unit/org/apache/cassandra/service/reads/range/ReplicaPlanMergerTest.java @@ -18,6 +18,7 @@ package org.apache.cassandra.service.reads.range; +import java.net.UnknownHostException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -27,15 +28,20 @@ import org.junit.Test; import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.ServerTestUtils; +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ConsistencyLevel; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.dht.AbstractBounds; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.dht.Range; +import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.tcm.membership.Location; import static org.apache.cassandra.ServerTestUtils.markCMS; import static org.apache.cassandra.ServerTestUtils.recreateCMS; @@ -65,14 +71,18 @@ public class ReplicaPlanMergerTest private static Keyspace keyspace; @BeforeClass - public static void defineSchema() throws ConfigurationException + public static void defineSchema() throws ConfigurationException, UnknownHostException { ORG_APACHE_CASSANDRA_DISABLE_MBEAN_REGISTRATION.setBoolean(true); - SchemaLoader.prepareServer(); + ServerTestUtils.prepareServerNoRegister(); StorageService.instance.setPartitionerUnsafe(Murmur3Partitioner.instance); recreateCMS(); SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(2)); keyspace = Keyspace.open(KEYSPACE); + // ensure all endpoints used in the tests are located in the same DC + Location local = DatabaseDescriptor.getLocator().local(); + for (int i = 1; i <= 3; i++) + ClusterMetadataTestHelper.register(InetAddressAndPort.getByName("127.0.0." + i), local); markCMS(); } diff --git a/test/unit/org/apache/cassandra/service/reads/repair/AbstractReadRepairTest.java b/test/unit/org/apache/cassandra/service/reads/repair/AbstractReadRepairTest.java index b2ff6723eea2..f0026d35b9c0 100644 --- a/test/unit/org/apache/cassandra/service/reads/repair/AbstractReadRepairTest.java +++ b/test/unit/org/apache/cassandra/service/reads/repair/AbstractReadRepairTest.java @@ -60,7 +60,6 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; import org.apache.cassandra.gms.Gossiper; -import org.apache.cassandra.locator.AbstractNetworkTopologySnitch; import org.apache.cassandra.locator.EndpointsForRange; import org.apache.cassandra.locator.EndpointsForToken; import org.apache.cassandra.locator.InetAddressAndPort; @@ -227,23 +226,6 @@ static void configureClass(ReadRepairStrategy repairStrategy) throws Throwable { ServerTestUtils.prepareServerNoRegister(); - DatabaseDescriptor.setEndpointSnitch(new AbstractNetworkTopologySnitch() - { - public String getRack(InetAddressAndPort endpoint) - { - return "rack1"; - } - - public String getDatacenter(InetAddressAndPort endpoint) - { - byte[] address = endpoint.addressBytes; - if (address[1] == 2) { - return "datacenter2"; - } - return "datacenter1"; - } - }); - target1 = InetAddressAndPort.getByName("127.1.0.255"); target2 = InetAddressAndPort.getByName("127.1.0.254"); target3 = InetAddressAndPort.getByName("127.1.0.253"); diff --git a/test/unit/org/apache/cassandra/service/reads/repair/ReadRepairTest.java b/test/unit/org/apache/cassandra/service/reads/repair/ReadRepairTest.java index e3de74be15ae..5138de03000b 100644 --- a/test/unit/org/apache/cassandra/service/reads/repair/ReadRepairTest.java +++ b/test/unit/org/apache/cassandra/service/reads/repair/ReadRepairTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.Iterables; import org.apache.cassandra.Util; +import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; import org.apache.cassandra.locator.AbstractReplicationStrategy; import org.apache.cassandra.locator.Endpoints; import org.apache.cassandra.locator.EndpointsForRange; @@ -56,6 +57,7 @@ import org.apache.cassandra.schema.SchemaTestUtil; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.Tables; +import org.apache.cassandra.tcm.membership.Location; import org.apache.cassandra.utils.ByteBufferUtil; import static java.util.concurrent.TimeUnit.NANOSECONDS; @@ -138,6 +140,9 @@ public static void setUpClass() throws Throwable cell2 = cell("v", "val2", now); cell3 = cell("v", "val3", now); resolved = mutation(cell1, cell2); + Location local = DatabaseDescriptor.getLocator().local(); + for (InetAddressAndPort endpoint : targets.endpoints()) + ClusterMetadataTestHelper.register(endpoint, local); } private static DecoratedKey dk(int v) diff --git a/test/unit/org/apache/cassandra/tcm/ClusterMetadataTransformationTest.java b/test/unit/org/apache/cassandra/tcm/ClusterMetadataTransformationTest.java index d25d02c80de1..bd91832d00bb 100644 --- a/test/unit/org/apache/cassandra/tcm/ClusterMetadataTransformationTest.java +++ b/test/unit/org/apache/cassandra/tcm/ClusterMetadataTransformationTest.java @@ -24,10 +24,8 @@ import java.util.concurrent.ThreadLocalRandom; import com.google.common.collect.Iterables; -import org.junit.BeforeClass; import org.junit.Test; -import org.apache.cassandra.ServerTestUtils; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.io.util.DataInputBuffer; import org.apache.cassandra.io.util.DataOutputBuffer; @@ -68,12 +66,6 @@ public class ClusterMetadataTransformationTest { - @BeforeClass - public static void init() - { - ServerTestUtils.initSnitch(); - } - long seed = System.nanoTime(); Random random = new Random(seed); diff --git a/test/unit/org/apache/cassandra/tcm/GetLogStateTest.java b/test/unit/org/apache/cassandra/tcm/GetLogStateTest.java index 3ec661f15868..167aa3992b63 100644 --- a/test/unit/org/apache/cassandra/tcm/GetLogStateTest.java +++ b/test/unit/org/apache/cassandra/tcm/GetLogStateTest.java @@ -79,7 +79,7 @@ public static void setup() throws IOException ClusterMetadataService.unsetInstance(); ClusterMetadataService.setInstance(cms); log.readyUnchecked(); - log.bootstrap(FBUtilities.getBroadcastAddressAndPort()); + log.unsafeBootstrapForTesting(FBUtilities.getBroadcastAddressAndPort()); } @Test diff --git a/test/unit/org/apache/cassandra/tcm/LogStateTest.java b/test/unit/org/apache/cassandra/tcm/LogStateTest.java index 9388ad6333b3..187337f1c999 100644 --- a/test/unit/org/apache/cassandra/tcm/LogStateTest.java +++ b/test/unit/org/apache/cassandra/tcm/LogStateTest.java @@ -70,7 +70,7 @@ public void setup() throws IOException ClusterMetadataService.unsetInstance(); ClusterMetadataService.setInstance(cms); log.readyUnchecked(); - log.bootstrap(FBUtilities.getBroadcastAddressAndPort()); + log.unsafeBootstrapForTesting(FBUtilities.getBroadcastAddressAndPort()); } @Test diff --git a/test/unit/org/apache/cassandra/tools/nodetool/RingTest.java b/test/unit/org/apache/cassandra/tools/nodetool/RingTest.java index 28fffd9619ea..f29e61ce82b5 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/RingTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/RingTest.java @@ -25,7 +25,7 @@ import org.junit.Test; import org.apache.cassandra.cql3.CQLTester; -import org.apache.cassandra.locator.SimpleSnitch; +import org.apache.cassandra.locator.SimpleLocationProvider; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tools.ToolRunner; import org.apache.cassandra.utils.FBUtilities; @@ -76,11 +76,11 @@ private void validateRingOutput(String hostForm, String... args) */ String[] lines = tool.getStdout().split("\\R"); - assertThat(lines[1].trim()).endsWith(SimpleSnitch.DATA_CENTER_NAME); + assertThat(lines[1].trim()).endsWith(SimpleLocationProvider.LOCATION.datacenter); assertThat(lines[3]).containsPattern("Address *Rack *Status *State *Load *Owns *Token *"); String hostRing = lines[lines.length-4].trim(); // this command has a couple extra newlines and an empty error message at the end. Not messing with it. assertThat(hostRing).startsWith(hostForm); - assertThat(hostRing).contains(SimpleSnitch.RACK_NAME); + assertThat(hostRing).contains(SimpleLocationProvider.LOCATION.rack); assertThat(hostRing).contains("Up"); assertThat(hostRing).contains("Normal"); assertThat(hostRing).containsPattern("\\d+\\.?\\d+ KiB"); diff --git a/test/unit/org/apache/cassandra/tools/nodetool/StatusTest.java b/test/unit/org/apache/cassandra/tools/nodetool/StatusTest.java index 9d8496c8b102..cc9c14deb481 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/StatusTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/StatusTest.java @@ -24,7 +24,7 @@ import org.junit.Test; import org.apache.cassandra.cql3.CQLTester; -import org.apache.cassandra.locator.SimpleSnitch; +import org.apache.cassandra.locator.SimpleLocationProvider; import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tools.ToolRunner; @@ -86,7 +86,7 @@ public void testOutputWhileBootstrapping() assertThat(hostStatus).containsPattern("\\d+\\.?\\d+ KiB"); assertThat(hostStatus).contains(localHostId); assertThat(hostStatus).contains(token); - assertThat(hostStatus).endsWith(SimpleSnitch.RACK_NAME); + assertThat(hostStatus).endsWith(SimpleLocationProvider.LOCATION.rack); String bootstrappingWarn = lines[lines.length-1].trim(); assertThat(bootstrappingWarn) @@ -106,7 +106,7 @@ private void validateStatusOutput(String hostForm, String... args) UN localhost 45.71 KiB 100.0% 0b1b5e91-ad3b-444e-9c24-50578486978a 1849950853373272258 rack1 */ String[] lines = PATTERN.split(tool.getStdout()); - assertThat(lines[0].trim()).endsWith(SimpleSnitch.DATA_CENTER_NAME); + assertThat(lines[0].trim()).endsWith(SimpleLocationProvider.LOCATION.datacenter); String hostStatus = lines[lines.length-1].trim(); assertThat(hostStatus).startsWith("UN"); assertThat(hostStatus).contains(hostForm); @@ -114,7 +114,7 @@ private void validateStatusOutput(String hostForm, String... args) assertThat(hostStatus).containsPattern("\\d+\\.\\d+%"); assertThat(hostStatus).contains(localHostId); assertThat(hostStatus).contains(token); - assertThat(hostStatus).endsWith(SimpleSnitch.RACK_NAME); + assertThat(hostStatus).endsWith(SimpleLocationProvider.LOCATION.rack); assertThat(hostStatus).doesNotContain("?"); } } From 849245a5620f34566ae999fed94bf10c971870f2 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Mon, 16 Dec 2024 14:28:53 +0100 Subject: [PATCH 055/225] Remove cassandra.jmx.remote.port setting to an empty string in SimulatorRunner patch by Stefan Miklosovic; reviewed by Sam Tunnicliffe for CASSANDRA-11695 --- .../main/org/apache/cassandra/simulator/SimulationRunner.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/test/simulator/main/org/apache/cassandra/simulator/SimulationRunner.java b/test/simulator/main/org/apache/cassandra/simulator/SimulationRunner.java index deaa462df34a..97c2db35a551 100644 --- a/test/simulator/main/org/apache/cassandra/simulator/SimulationRunner.java +++ b/test/simulator/main/org/apache/cassandra/simulator/SimulationRunner.java @@ -56,7 +56,6 @@ import static java.util.concurrent.TimeUnit.NANOSECONDS; import static org.apache.cassandra.config.CassandraRelevantProperties.ALLOW_ALTER_RF_DURING_RANGE_MOVEMENT; import static org.apache.cassandra.config.CassandraRelevantProperties.BATCH_COMMIT_LOG_SYNC_INTERVAL; -import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_REMOTE_PORT; import static org.apache.cassandra.config.CassandraRelevantProperties.CLOCK_GLOBAL; import static org.apache.cassandra.config.CassandraRelevantProperties.CLOCK_MONOTONIC_APPROX; import static org.apache.cassandra.config.CassandraRelevantProperties.CLOCK_MONOTONIC_PRECISE; @@ -111,7 +110,6 @@ public static void beforeAll() for (CassandraRelevantProperties property : Arrays.asList(CLOCK_GLOBAL, CLOCK_MONOTONIC_APPROX, CLOCK_MONOTONIC_PRECISE)) property.setString("org.apache.cassandra.simulator.systems.SimulatedTime$Global"); - CASSANDRA_JMX_REMOTE_PORT.setString(""); RING_DELAY.setInt(0); PAXOS_REPAIR_RETRY_TIMEOUT_IN_MS.setLong(NANOSECONDS.toMillis(Long.MAX_VALUE)); SHUTDOWN_ANNOUNCE_DELAY_IN_MS.setInt(0); From 75ec1ada7a40761e8b6eb948925c2dd2b116876e Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Mon, 16 Dec 2024 23:18:36 +0100 Subject: [PATCH 056/225] Check presence of a snapshot in a case-insensitive manner on macOS platform to prevent hardlinking failures When hardlinks are created for a snapshot with the name "snapshot" and then we take a snapshot with the name "Snapshot", macOS platform thinks that this was already hardlinked because its hardlinking implementation does not seem to be case-sensitive. The fix consists of checking, in a case-insensitive manner, if there is already such snapshot, but only on macOS platform. The patch also does not create empty directories for a given snapshot prematurely before checking its presence so there are not dangling empty directories in case of a failure. patch by Stefan Miklosovic; reviewed by Bernardo Botella, Caleb Rackliffe for CASSANDRA-20146 --- CHANGES.txt | 1 + .../org/apache/cassandra/db/Directories.java | 27 ++++++++++ .../cassandra/db/virtual/SnapshotsTable.java | 6 ++- .../service/snapshot/TableSnapshot.java | 30 ++++++++++- .../service/snapshot/TakeSnapshotTask.java | 2 +- .../distributed/test/SnapshotsTest.java | 54 +++++++++++++++---- 6 files changed, 106 insertions(+), 14 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index f3b51f4972eb..842d8b0d945f 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Check presence of a snapshot in a case-insensitive manner on macOS platform to prevent hardlinking failures (CASSANDRA-20146) * Enable JMX server configuration to be in cassandra.yaml (CASSANDRA-11695) * Parallelized UCS compactions (CASSANDRA-18802) * Avoid prepared statement invalidation race when committing schema changes (CASSANDRA-20116) diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java index 12a64090f853..67b87d14e97c 100644 --- a/src/java/org/apache/cassandra/db/Directories.java +++ b/src/java/org/apache/cassandra/db/Directories.java @@ -650,6 +650,18 @@ public static File getSnapshotDirectory(File location, String snapshotName) } } + public static File getSnapshotDirectoryWithoutCreation(File location, String snapshotName) + { + if (isSecondaryIndexFolder(location)) + { + return getWithoutCreation(location.parent(), SNAPSHOT_SUBDIR, snapshotName, location.name()); + } + else + { + return getWithoutCreation(location, SNAPSHOT_SUBDIR, snapshotName); + } + } + /** * Returns directory to write a snapshot to. If directory does not exist, then it is NOT created. * @@ -683,6 +695,16 @@ public Set<File> getSnapshotDirs(String tag) return snapshotDirs; } + public Set<File> getSnapshotDirsWithoutCreation(String tag) + { + Set<File> snapshotDirs = new HashSet<>(); + + for (File cfDir : getCFDirectories()) + snapshotDirs.add(Directories.getSnapshotDirectoryWithoutCreation(cfDir, tag).toAbsolute()); + + return snapshotDirs; + } + public File getSnapshotManifestFile(String snapshotName) { File snapshotDir = getSnapshotDirectory(getDirectoryForNewSSTables(), snapshotName); @@ -1301,6 +1323,11 @@ public static Optional<File> get(File base, String... subdirs) return dir.exists() ? Optional.of(dir) : Optional.empty(); } + public static File getWithoutCreation(File base, String... subdirs) + { + return subdirs == null || subdirs.length == 0 ? base : new File(base, join(subdirs)); + } + private static String join(String... s) { return StringUtils.join(s, File.pathSeparator()); diff --git a/src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java b/src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java index c757ce7c54f5..dada749e8158 100644 --- a/src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java +++ b/src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java @@ -18,6 +18,7 @@ package org.apache.cassandra.db.virtual; +import java.time.Instant; import java.util.Date; import org.apache.cassandra.db.marshal.BooleanType; @@ -64,12 +65,15 @@ public DataSet data() for (TableSnapshot tableSnapshot : SnapshotManager.instance.getSnapshots(false, true)) { + Instant snapshotCreatedAt = tableSnapshot.getCreatedAt(); + Date createdAt = snapshotCreatedAt != null ? new Date(snapshotCreatedAt.toEpochMilli()) : null; + SimpleDataSet row = result.row(tableSnapshot.getTag(), tableSnapshot.getKeyspaceName(), tableSnapshot.getTableName()) .column(TRUE_SIZE, tableSnapshot.computeTrueSizeBytes()) .column(SIZE_ON_DISK, tableSnapshot.computeSizeOnDiskBytes()) - .column(CREATED_AT, new Date(tableSnapshot.getCreatedAt().toEpochMilli())); + .column(CREATED_AT, createdAt); if (tableSnapshot.isExpiring()) row.column(EXPIRES_AT, new Date(tableSnapshot.getExpiresAt().toEpochMilli())); diff --git a/src/java/org/apache/cassandra/service/snapshot/TableSnapshot.java b/src/java/org/apache/cassandra/service/snapshot/TableSnapshot.java index 94bcd07d9a87..549dbb6db62c 100644 --- a/src/java/org/apache/cassandra/service/snapshot/TableSnapshot.java +++ b/src/java/org/apache/cassandra/service/snapshot/TableSnapshot.java @@ -46,12 +46,16 @@ import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.LocalizeString; import org.apache.cassandra.utils.Throwables; import org.apache.cassandra.utils.concurrent.Refs; +import oshi.PlatformEnum; public class TableSnapshot { private static final Logger logger = LoggerFactory.getLogger(TableSnapshot.class); + private static final PlatformEnum PLATFORM = FBUtilities.getSystemInfo().platform(); private final String keyspaceName; private final String tableName; @@ -139,12 +143,14 @@ public String getTag() public Instant getCreatedAt() { - if (createdAt == null) { long minCreation = 0; for (File snapshotDir : snapshotDirs) { + if (!snapshotDir.exists()) + continue; + long lastModified = snapshotDir.lastModified(); if (lastModified == 0) continue; @@ -377,7 +383,27 @@ public boolean equals(Object o) return Objects.equals(keyspaceName, snapshot.keyspaceName) && Objects.equals(tableName, snapshot.tableName) && Objects.equals(tableId, snapshot.tableId) && - Objects.equals(tag, snapshot.tag); + tagsEqual(tag, snapshot.tag); + } + + private boolean tagsEqual(String tag1, String tag2) + { + if (tag1 == null && tag2 == null) + return true; + + if (tag1 == null || tag2 == null) + return false; + + // When hardlinks are created for a snapshot with the name "snapshot" + // and then we take a snapshot with the name "Snapshot", macOS platform thinks + // that this was already hardlinked because its hardlinking implementation + // does not seem to be case-sensitive. The fix consists of checking, + // in a case-insensitive manner, if there is already such snapshot, + // but only on macOS platform. + if (PLATFORM == PlatformEnum.MACOS) + return LocalizeString.toLowerCaseLocalized(tag1).equals(LocalizeString.toLowerCaseLocalized(tag2)); + + return Objects.equals(tag1, tag2); } @Override diff --git a/src/java/org/apache/cassandra/service/snapshot/TakeSnapshotTask.java b/src/java/org/apache/cassandra/service/snapshot/TakeSnapshotTask.java index 2b1345976598..a0d81a685ccc 100644 --- a/src/java/org/apache/cassandra/service/snapshot/TakeSnapshotTask.java +++ b/src/java/org/apache/cassandra/service/snapshot/TakeSnapshotTask.java @@ -94,7 +94,7 @@ public Map<ColumnFamilyStore, TableSnapshot> getSnapshotsToCreate() for (ColumnFamilyStore cfs : entitiesForSnapshot) { - Set<File> snapshotDirs = cfs.getDirectories().getSnapshotDirs(snapshotName); + Set<File> snapshotDirs = cfs.getDirectories().getSnapshotDirsWithoutCreation(snapshotName); TableSnapshot tableSnapshot = new TableSnapshot(cfs.metadata.keyspace, cfs.metadata.name, diff --git a/test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java b/test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java index 9ca5e052338b..2c5765469580 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java @@ -40,15 +40,21 @@ import org.apache.cassandra.distributed.api.NodeToolResult; import org.apache.cassandra.distributed.shared.WithProperties; import org.apache.cassandra.utils.Clock; +import org.apache.cassandra.utils.FBUtilities; import static java.lang.String.format; import static java.util.concurrent.TimeUnit.SECONDS; import static java.util.stream.Collectors.toList; import static org.apache.cassandra.distributed.shared.ClusterUtils.stopUnchecked; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.awaitility.Awaitility.await; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.not; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.junit.Assume.assumeThat; +import static oshi.PlatformEnum.MACOS; public class SnapshotsTest extends TestBaseImpl { @@ -58,6 +64,9 @@ public class SnapshotsTest extends TestBaseImpl private static final WithProperties properties = new WithProperties(); private static Cluster cluster; + private final String[] exoticSnapshotNamesOnMac = new String[]{ "snapshot", "snapshots", "backup", "backups", + "snapshot.with.dots-and-dashes" }; + private final String[] exoticSnapshotNames = new String[]{ "snapshot", "snapshots", "backup", "backups", "Snapshot", "Snapshots", "Backups", "Backup", "snapshot.with.dots-and-dashes" }; @@ -330,18 +339,24 @@ public void testTTLSnapshotOfDroppedTableAfterRestart() @Test public void testExoticSnapshotNames() { - IInvokableInstance instance = cluster.get(1); - cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (key int, value text, PRIMARY KEY (key))")); - populate(cluster); + assumeThat(FBUtilities.getSystemInfo().platform(), not(MACOS)); + exoticSnapshotNamesInternal(exoticSnapshotNames); + } - for (String tag : exoticSnapshotNames) - { - instance.nodetoolResult("snapshot", - "-t", tag, - "-kt", withKeyspace("%s.tbl")).asserts().success(); + @Test + public void testExoticSnapshotNamesOnMacOS() + { + assumeThat(FBUtilities.getSystemInfo().platform(), is(MACOS)); + exoticSnapshotNamesInternal(exoticSnapshotNamesOnMac); + } - waitForSnapshot(tag, true, true); - } + @Test + public void testDuplicateSnapshotOnMacOS() + { + assumeThat(FBUtilities.getSystemInfo().platform(), is(MACOS)); + exoticSnapshotNamesInternal(new String[]{ "snapshot" }); + assertThatThrownBy(() -> exoticSnapshotNamesInternal(new String[]{ "Snapshot" })) + .hasMessageContaining(withKeyspace("Snapshot Snapshot for %s.tbl already exists.")); } @Test @@ -435,4 +450,23 @@ private boolean waitForSnapshotInternal(String keyspaceName, String tableName, S return expectPresent == lines.stream().anyMatch(line -> line.startsWith(snapshotName)); } + + private void exoticSnapshotNamesInternal(String[] exoticSnapshotNames) + { + IInvokableInstance instance = cluster.get(1); + cluster.schemaChange(withKeyspace("CREATE TABLE IF NOT EXISTS %s.tbl (key int, value text, PRIMARY KEY (key))")); + populate(cluster); + + for (String tag : exoticSnapshotNames) + { + NodeToolResult result = instance.nodetoolResult("snapshot", + "-t", tag, + "-kt", withKeyspace("%s.tbl")); + + if (result.getRc() != 0) + throw new RuntimeException(result.getError()); + + waitForSnapshot(tag, true, true); + } + } } From db94321d71b4e79860f2e381cff558ec436e247f Mon Sep 17 00:00:00 2001 From: Sam Tunnicliffe <samt@apache.org> Date: Fri, 13 Dec 2024 09:12:09 +0000 Subject: [PATCH 057/225] Support downgrading after CMS is initialized Patch by Sam Tunnicliffe; reviewed by Marcus Eriksson for CASSANDRA-20145 --- CHANGES.txt | 2 ++ .../cassandra/gms/GossipDigestSynVerbHandler.java | 3 ++- .../cassandra/tcm/compatibility/GossipHelper.java | 10 +++++++++- 3 files changed, 13 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 842d8b0d945f..f50d0b2f39fe 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,6 @@ 5.1 + * Support downgrading after CMS is initialized (CASSANDRA-20145) + * Deprecate IEndpointSnitch (CASSANDRA-19488) * Check presence of a snapshot in a case-insensitive manner on macOS platform to prevent hardlinking failures (CASSANDRA-20146) * Enable JMX server configuration to be in cassandra.yaml (CASSANDRA-11695) * Parallelized UCS compactions (CASSANDRA-18802) diff --git a/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java b/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java index 50cba9f90f6e..93a8252a983c 100644 --- a/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java +++ b/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java @@ -64,7 +64,8 @@ public void doVerb(Message<GossipDigestSyn> message) return; } - if (gDigestMessage.metadataId != ClusterMetadata.current().metadataIdentifier) + if (gDigestMessage.metadataId != ClusterMetadata.EMPTY_METADATA_IDENTIFIER + && gDigestMessage.metadataId != ClusterMetadata.current().metadataIdentifier) { logger.warn("Cluster metadata identifier mismatch from {} {}!={}", from, gDigestMessage.metadataId, ClusterMetadata.current().metadataIdentifier); return; diff --git a/src/java/org/apache/cassandra/tcm/compatibility/GossipHelper.java b/src/java/org/apache/cassandra/tcm/compatibility/GossipHelper.java index 8f4861cc17d2..31a20d0bd0d7 100644 --- a/src/java/org/apache/cassandra/tcm/compatibility/GossipHelper.java +++ b/src/java/org/apache/cassandra/tcm/compatibility/GossipHelper.java @@ -51,6 +51,8 @@ import org.apache.cassandra.gms.VersionedValue; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.schema.DistributedSchema; +import org.apache.cassandra.schema.Keyspaces; +import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.SchemaKeyspace; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tcm.ClusterMetadata; @@ -275,9 +277,15 @@ private static NodeVersion getVersionFromEndpointState(InetAddressAndPort endpoi public static ClusterMetadata emptyWithSchemaFromSystemTables(Set<String> allKnownDatacenters) { + // If this instance was previously upgraded then subsequently downgraded, the metadata keyspace may have been + // added to system_schema tables. If so, don't include it in the initial schema as this will cause it to be + // incorrectly configured with the global partitioner. It will be created afresh from + // DistributedMetadataLogKeyspace.initialMetadata. + Keyspaces keyspaces = SchemaKeyspace.fetchNonSystemKeyspaces() + .filter(k -> !k.name.equals(SchemaConstants.METADATA_KEYSPACE_NAME)); return new ClusterMetadata(Epoch.UPGRADE_STARTUP, DatabaseDescriptor.getPartitioner(), - DistributedSchema.fromSystemTables(SchemaKeyspace.fetchNonSystemKeyspaces(), allKnownDatacenters), + DistributedSchema.fromSystemTables(keyspaces, allKnownDatacenters), Directory.EMPTY, new TokenMap(DatabaseDescriptor.getPartitioner()), DataPlacements.empty(), From 22af7a74cc79294199eb2272625c7eb80c8a8291 Mon Sep 17 00:00:00 2001 From: manish-m-pillai <manishpillai64@gmail.com> Date: Fri, 13 Dec 2024 13:23:12 +0100 Subject: [PATCH 058/225] Enable sorting of nodetool status output patch by Manish Pillai; reviwed by Stefan Miklosovic, Bernardo Botella, Jordan West for CASSANDRA-20104 Co-authored-by: Stefan Miklosovic <smiklosovic@apache.org> --- CHANGES.txt | 1 + .../cassandra/tools/nodetool/Status.java | 384 +++++++++++++++++- .../test/AbstractNodetoolStatusTest.java | 339 ++++++++++++++++ .../test/NodeToolStatusWithVNodesTest.java | 62 +++ .../test/NodetoolStatusWithoutVNodesTest.java | 61 +++ 5 files changed, 825 insertions(+), 22 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/test/AbstractNodetoolStatusTest.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/NodeToolStatusWithVNodesTest.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/NodetoolStatusWithoutVNodesTest.java diff --git a/CHANGES.txt b/CHANGES.txt index f50d0b2f39fe..797312bf01d1 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Enable sorting of nodetool status output (CASSANDRA-20104) * Support downgrading after CMS is initialized (CASSANDRA-20145) * Deprecate IEndpointSnitch (CASSANDRA-19488) * Check presence of a snapshot in a case-insensitive manner on macOS platform to prevent hardlinking failures (CASSANDRA-20146) diff --git a/src/java/org/apache/cassandra/tools/nodetool/Status.java b/src/java/org/apache/cassandra/tools/nodetool/Status.java index a89ca64b8dbd..496175772116 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/Status.java +++ b/src/java/org/apache/cassandra/tools/nodetool/Status.java @@ -17,26 +17,38 @@ */ package org.apache.cassandra.tools.nodetool; -import io.airlift.airline.Arguments; -import io.airlift.airline.Command; -import io.airlift.airline.Option; - import java.io.PrintStream; import java.net.UnknownHostException; import java.text.DecimalFormat; +import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; +import java.util.HashMap; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.SortedMap; +import java.util.UUID; +import java.util.function.BiFunction; +import java.util.stream.Collectors; +import com.google.common.collect.ArrayListMultimap; + +import io.airlift.airline.Arguments; +import io.airlift.airline.Command; +import io.airlift.airline.Option; +import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.locator.EndpointSnitchInfoMBean; +import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.tools.NodeTool; import org.apache.cassandra.tools.NodeTool.NodeToolCmd; import org.apache.cassandra.tools.nodetool.formatter.TableBuilder; +import org.apache.cassandra.utils.LocalizeString; -import com.google.common.collect.ArrayListMultimap; +import static java.util.stream.Collectors.toMap; @SuppressWarnings("UseOfSystemOutOrSystemErr") @Command(name = "status", description = "Print cluster information (state, load, IDs, ...)") @@ -45,9 +57,24 @@ public class Status extends NodeToolCmd @Arguments(usage = "[<keyspace>]", description = "The keyspace name") private String keyspace = null; - @Option(title = "resolve_ip", name = {"-r", "--resolve-ip"}, description = "Show node domain names instead of IPs") + @Option(title = "resolve_ip", name = { "-r", "--resolve-ip" }, description = "Show node domain names instead of IPs") private boolean resolveIp = false; + @Option(title = "sort", + name = { "-s", "--sort" }, + description = "Sort by one of 'ip', 'host', 'load', 'owns', 'id', 'rack', 'state' or 'token'. " + + "Default ordering is ascending for 'ip', 'host', 'id', 'token', 'rack' and descending for 'load', 'owns', 'state'. " + + "Sorting by token is possible only when cluster does not use vnodes. When using vnodes, default " + + "sorting is by id otherwise by token.", + allowedValues = { "ip", "host", "load", "owns", "id", "rack", "state", "token" }) + private String sortBy = null; + + @Option(title = "sort_order", + name = { "-o", "--order" }, + description = "Sorting order: 'asc' for ascending, 'desc' for descending.", + allowedValues = { "asc", "desc" }) + private String sortOrder = null; + private boolean isTokenPerNode = true; private Collection<String> joiningNodes, leavingNodes, movingNodes, liveNodes, unreachableNodes; private Map<String, String> loadMap, hostIDMap; @@ -57,6 +84,11 @@ public class Status extends NodeToolCmd public void execute(NodeProbe probe) { PrintStream out = probe.output().out; + PrintStream errOut = probe.output().err; + + SortBy sortBy = parseSortBy(this.sortBy, errOut); + SortOrder sortOrder = parseSortOrder(this.sortOrder, errOut); + joiningNodes = probe.getJoiningNodes(true); leavingNodes = probe.getLeavingNodes(true); movingNodes = probe.getMovingNodes(true); @@ -86,37 +118,75 @@ public void execute(NodeProbe probe) } catch (Exception ex) { - out.printf("%nError: %s%n", ex.getMessage()); + errOut.printf("%nError: %s%n", ex.getMessage()); System.exit(1); } } catch (IllegalArgumentException ex) { - out.printf("%nError: %s%n", ex.getMessage()); + errOut.printf("%nError: %s%n", ex.getMessage()); System.exit(1); } SortedMap<String, SetHostStatWithPort> dcs = NodeTool.getOwnershipByDcWithPort(probe, resolveIp, tokensToEndpoints, ownerships); + int nodesOfTokens = tokensToEndpoints.values().size(); + // More tokens than nodes (aka vnodes)? - if (dcs.size() < tokensToEndpoints.size()) + if (hostIDMap.size() < nodesOfTokens) isTokenPerNode = false; + if (sortBy == null) + { + if (isTokenPerNode) + sortBy = SortBy.token; + else + sortBy = SortBy.id; + } + else if (!isTokenPerNode && sortBy == SortBy.token) + { + errOut.printf("%nError: Can not sort by token when there is not token per node.%n"); + System.exit(1); + } + else if (!resolveIp && sortBy == SortBy.host) + { + errOut.printf("%nError: Can not sort by host when there is not -r/--resolve-ip flag used.%n"); + System.exit(1); + } + // Datacenters for (Map.Entry<String, SetHostStatWithPort> dc : dcs.entrySet()) { TableBuilder tableBuilder = sharedTable.next(); addNodesHeader(hasEffectiveOwns, tableBuilder); - ArrayListMultimap<String, HostStatWithPort> hostToTokens = ArrayListMultimap.create(); + ArrayListMultimap<InetAddressAndPort, HostStatWithPort> hostToTokens = ArrayListMultimap.create(); for (HostStatWithPort stat : dc.getValue()) - hostToTokens.put(stat.endpointWithPort.getHostAddressAndPort(), stat); + hostToTokens.put(stat.endpointWithPort, stat); - for (String endpoint : hostToTokens.keySet()) + Map<String, List<Object>> data = new HashMap<>(); + for (InetAddressAndPort endpoint : hostToTokens.keySet()) { - Float owns = ownerships.get(endpoint); + Float owns = ownerships.get(endpoint.getHostAddressAndPort()); List<HostStatWithPort> tokens = hostToTokens.get(endpoint); - addNode(endpoint, owns, tokens.get(0), tokens.size(), hasEffectiveOwns, tableBuilder); + + HostStatWithPort hostStatWithPort = tokens.get(0); + String epDns = hostStatWithPort.ipOrDns(printPort); + List<Object> nodeData = addNode(epDns, endpoint, owns, hostStatWithPort, tokens.size(), hasEffectiveOwns); + data.put(epDns, nodeData); + } + + for (Map.Entry<String, List<Object>> entry : sortBy.tokenPerNode(isTokenPerNode) + .sortOrder(sortOrder) + .sort(data) + .entrySet()) + { + List<Object> values = entry.getValue(); + List<String> row = new ArrayList<>(); + for (int i = 1; i < values.size(); i++) + row.add((String) values.get(i)); + + tableBuilder.add(row); } } @@ -124,7 +194,8 @@ public void execute(NodeProbe probe) boolean first = true; for (Map.Entry<String, SetHostStatWithPort> dc : dcs.entrySet()) { - if (!first) { + if (!first) + { out.println(); } first = false; @@ -153,10 +224,11 @@ private void addNodesHeader(boolean hasEffectiveOwns, TableBuilder tableBuilder) tableBuilder.add("--", "Address", "Load", "Tokens", owns, "Host ID", "Rack"); } - private void addNode(String endpoint, Float owns, HostStatWithPort hostStat, int size, boolean hasEffectiveOwns, - TableBuilder tableBuilder) + private List<Object> addNode(String epDns, InetAddressAndPort addressAndPort, Float owns, + HostStatWithPort hostStat, int size, boolean hasEffectiveOwns) { - String status, state, load, strOwns, hostID, rack, epDns; + String endpoint = addressAndPort.getHostAddressAndPort(); + String status, state, load, strOwns, hostID, rack; if (liveNodes.contains(endpoint)) status = "U"; else if (unreachableNodes.contains(endpoint)) status = "D"; else status = "?"; @@ -179,15 +251,283 @@ private void addNode(String endpoint, Float owns, HostStatWithPort hostStat, int throw new RuntimeException(e); } - epDns = hostStat.ipOrDns(printPort); if (isTokenPerNode) + return List.of(addressAndPort, statusAndState, epDns, load, strOwns, hostID, hostStat.token, rack); + else + return List.of(addressAndPort, statusAndState, epDns, load, String.valueOf(size), strOwns, hostID, rack); + } + + public enum SortOrder + { + asc, + desc + } + + public enum SortBy + { + state(true) + { + @Override + public Map<String, List<Object>> sort(Map<String, List<Object>> data) + { + return sortInternal(data, this::compareByState); + } + }, + ip(false) + { + @Override + public Map<String, List<Object>> sort(Map<String, List<Object>> data) + { + return sortInternal(data, this::compareByAddress); + } + }, + host(false) + { + @Override + public Map<String, List<Object>> sort(Map<String, List<Object>> data) + { + return sortInternal(data, this::compareByHost); + } + }, + load(true) + { + @Override + public Map<String, List<Object>> sort(Map<String, List<Object>> data) + { + return sortInternal(data, this::compareByLoad); + } + }, + owns(true) + { + @Override + public Map<String, List<Object>> sort(Map<String, List<Object>> data) + { + return sortInternal(data, this::compareByOwns); + } + }, + id(false) + { + @Override + public Map<String, List<Object>> sort(Map<String, List<Object>> data) + { + return sortInternal(data, this::compareById); + } + }, + token(false) + { + @Override + public Map<String, List<Object>> sort(Map<String, List<Object>> data) + { + return sortInternal(data, this::compareByToken); + } + }, + rack(false) + { + @Override + public Map<String, List<Object>> sort(Map<String, List<Object>> data) + { + return sortInternal(data, this::compareByRack); + } + }; + + private final boolean descendingByDefault; + boolean tokenPerNode; + SortOrder sortOrder; + + SortBy(boolean descendingByDefault) { - tableBuilder.add(statusAndState, epDns, load, strOwns, hostID, hostStat.token, rack); + this.descendingByDefault = descendingByDefault; } - else + + public abstract Map<String, List<Object>> sort(Map<String, List<Object>> data); + + boolean descending(SortOrder sortOrder) + { + return sortOrder == null ? descendingByDefault : sortOrder == SortOrder.desc; + } + + SortBy sortOrder(SortOrder sortOrder) + { + this.sortOrder = sortOrder; + return this; + } + + SortBy tokenPerNode(boolean tokenPerNode) { - tableBuilder.add(statusAndState, epDns, load, String.valueOf(size), strOwns, hostID, rack); + this.tokenPerNode = tokenPerNode; + return this; + } + + int compareByState(Map.Entry<String, List<Object>> row1, Map.Entry<String, List<Object>> row2) + { + String str1 = (String) row1.getValue().get(1); + String str2 = (String) row2.getValue().get(1); + return evaluateComparision(str1.compareTo(str2)); + } + + int compareByHost(Map.Entry<String, List<Object>> row1, Map.Entry<String, List<Object>> row2) + { + String str1 = (String) row1.getValue().get(2); + String str2 = (String) row2.getValue().get(2); + return evaluateComparision(str1.compareTo(str2)); + } + + int compareByOwns(Map.Entry<String, List<Object>> row1, Map.Entry<String, List<Object>> row2) + { + String str1 = (String) row1.getValue().get(tokenPerNode ? 4 : 5); + String str2 = (String) row2.getValue().get(tokenPerNode ? 4 : 5); + + Optional<Integer> maybeReturn = maybeCompareQuestionMarks(str1, str2); + if (maybeReturn.isPresent()) + return maybeReturn.get(); + + double value1 = Double.parseDouble(str1.replace("%", "")); + double value2 = Double.parseDouble(str2.replace("%", "")); + + return evaluateComparision(Double.compare(value1, value2)); + } + + int compareByAddress(Map.Entry<String, List<Object>> row1, Map.Entry<String, List<Object>> row2) + { + InetAddressAndPort addr1 = (InetAddressAndPort) row1.getValue().get(0); + InetAddressAndPort addr2 = (InetAddressAndPort) row2.getValue().get(0); + + return evaluateComparision(addr1.compareTo(addr2)); + } + + int compareById(Map.Entry<String, List<Object>> row1, Map.Entry<String, List<Object>> row2) + { + UUID id1; + UUID id2; + if (tokenPerNode) + { + id1 = UUID.fromString((String) row1.getValue().get(5)); + id2 = UUID.fromString((String) row2.getValue().get(5)); + } + else + { + id1 = UUID.fromString((String) row1.getValue().get(6)); + id2 = UUID.fromString((String) row2.getValue().get(6)); + } + return evaluateComparision(id1.compareTo(id2)); + } + + int compareByToken(Map.Entry<String, List<Object>> row1, Map.Entry<String, List<Object>> row2) + { + long token1 = Long.parseLong((String) row1.getValue().get(6)); + long token2 = Long.parseLong((String) row2.getValue().get(6)); + return evaluateComparision(Long.compare(token1, token2)); + } + + int compareByRack(Map.Entry<String, List<Object>> row1, Map.Entry<String, List<Object>> row2) + { + String rack1 = (String) row1.getValue().get(7); + String rack2 = (String) row2.getValue().get(7); + int byRack = evaluateComparision(rack1.compareTo(rack2)); + + if (byRack != 0) + return byRack; + + if (tokenPerNode) + return compareByToken(row1, row2); + else + return compareById(row1, row2); + } + + int compareByLoad(Map.Entry<String, List<Object>> row1, Map.Entry<String, List<Object>> row2) + { + String str1 = (String) row1.getValue().get(3); + String str2 = (String) row2.getValue().get(3); + + Optional<Integer> maybeReturn = maybeCompareQuestionMarks(str1, str2); + if (maybeReturn.isPresent()) + return maybeReturn.get(); + + long value1 = FileUtils.parseFileSize(str1); + long value2 = FileUtils.parseFileSize(str2); + + return evaluateComparision(Long.compare(value1, value2)); + } + + Map<String, List<Object>> sortInternal(Map<String, List<Object>> data, + BiFunction<Map.Entry<String, List<Object>>, Map.Entry<String, List<Object>>, Integer> rowComparator) + { + return data.entrySet() + .stream() + .sorted(rowComparator::apply) + .collect(toMap(Map.Entry::getKey, Map.Entry::getValue, (e1, e2) -> e1, LinkedHashMap::new)); + } + + private int evaluateComparision(int comparisionResult) + { + if (comparisionResult < 0) + return descending(sortOrder) ? 1 : -1; + if (comparisionResult > 0) + return descending(sortOrder) ? -1 : 1; + + return 0; + } + + private Optional<Integer> maybeCompareQuestionMarks(String str1, String str2) + { + // Check if str1 or str2 contains a '?' and set a value for it. + boolean containsQuestionMark1 = str1.contains("?"); + boolean containsQuestionMark2 = str2.contains("?"); + // If both contain '?', return 0 (they are considered equal). + if (containsQuestionMark1 && containsQuestionMark2) + return Optional.of(0); + + // If str1 contains '?', ensure it's last (or first depending on descending). + if (containsQuestionMark1) + return Optional.of(descending(sortOrder) ? 1 : -1); + + // If str2 contains '?', ensure it's last (or first depending on descending). + if (containsQuestionMark2) + return Optional.of(descending(sortOrder) ? -1 : 1); + + return Optional.empty(); } } + private static SortBy parseSortBy(String setSortBy, PrintStream out) + { + if (setSortBy == null) + return null; + + try + { + return SortBy.valueOf(LocalizeString.toLowerCaseLocalized(setSortBy)); + } + catch (IllegalArgumentException ex) + { + String enabledValues = Arrays.stream(SortBy.values()) + .map(SortBy::name) + .collect(Collectors.joining(", ")); + out.printf("%nError: Illegal value for -s/--sort used: '" + + setSortBy + "'. Supported values are " + enabledValues + ".%n"); + System.exit(1); + return null; + } + } + + private static SortOrder parseSortOrder(String setSortOrder, PrintStream out) + { + if (setSortOrder == null) + return null; + + try + { + return SortOrder.valueOf(LocalizeString.toLowerCaseLocalized(setSortOrder)); + } + catch (IllegalArgumentException ex) + { + String enabledValues = Arrays.stream(SortOrder.values()) + .map(SortOrder::name) + .collect(Collectors.joining(", ")); + out.printf("%nError: Illegal value for -o/--order used: '" + + setSortOrder + "'. Supported values are " + enabledValues + ".%n"); + System.exit(1); + return null; + } + } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/AbstractNodetoolStatusTest.java b/test/distributed/org/apache/cassandra/distributed/test/AbstractNodetoolStatusTest.java new file mode 100644 index 000000000000..2d60a9355231 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/AbstractNodetoolStatusTest.java @@ -0,0 +1,339 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import java.io.IOException; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import java.util.function.BiConsumer; +import java.util.regex.Pattern; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Test; + +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.api.NodeToolResult; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.tools.nodetool.Status; + +import static java.lang.Double.parseDouble; +import static java.util.stream.Collectors.toList; +import static org.apache.cassandra.tools.nodetool.Status.SortOrder.asc; +import static org.apache.cassandra.tools.nodetool.Status.SortOrder.desc; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +@Ignore +public abstract class AbstractNodetoolStatusTest extends TestBaseImpl +{ + protected static boolean useVNodes = false; + + protected static Cluster CLUSTER; + protected static IInvokableInstance NODE_1; + + // Split on 2 or more spaces + private static final Pattern PATTERN = Pattern.compile("\\s{2,}"); + + @BeforeClass + public static void before() throws IOException + { + Cluster.Builder builder = Cluster.build(); + + if (useVNodes) + builder.withVNodes().withTokenCount(16); + else + builder.withoutVNodes(); + + CLUSTER = init(builder.withRacks(1, 2, 2).start()); + + NODE_1 = CLUSTER.get(1); + } + + @AfterClass + public static void after() + { + if (CLUSTER != null) + CLUSTER.close(); + } + + @Test + public void testCommands() + { + // test all legal combinations + for (Status.SortBy sortBy : Status.SortBy.values()) + { + // sort by tokens when on vnodes is illegal combination + if (sortBy == Status.SortBy.token && useVNodes) + continue; + + // will be tested separately below + if (sortBy == Status.SortBy.host) + continue; + + for (Status.SortOrder sortOrder : Status.SortOrder.values()) + { + // test with and without ordering + assertEquals(0, NODE_1.nodetool("status", "-s", sortBy.name(), "-o", sortOrder.name())); + assertEquals(0, NODE_1.nodetool("status", "-s", sortBy.name())); + } + } + + // test no sorting and no ordering + assertEquals(0, NODE_1.nodetool("status")); + + // test illegal sort by or order + NodeToolResult invalidSortBy = NODE_1.nodetoolResult("status", "--sort", "not_an_option"); + invalidSortBy.asserts().failure(); + Assert.assertTrue(invalidSortBy.getStderr().contains("Illegal value for -s/--sort used: 'not_an_option'. Supported values are state, ip, host, load, owns, id, token, rack.")); + + NodeToolResult invalidSortOrder = NODE_1.nodetoolResult("status", "--sort", "ip", "-o", "not_an_order"); + invalidSortOrder.asserts().failure(); + Assert.assertTrue(invalidSortOrder.getStderr().contains("Illegal value for -o/--order used: 'not_an_order'. Supported values are asc, desc.")); + + // test order alone + assertEquals(0, NODE_1.nodetool("status", "-o", "asc")); + assertEquals(0, NODE_1.nodetool("status", "-o", "desc")); + + // test what happens when we use vnodes, but we want to sort by token + if (useVNodes) + { + NodeToolResult result = NODE_1.nodetoolResult("status", "-s", "token", "-o", "desc"); + result.asserts().failure(); + assertTrue(result.getStderr().contains("Error: Can not sort by token when there is not token per node.")); + } + + // test what happens when we want to sort by host, but we do not resolve ips + + NodeToolResult hostResult = NODE_1.nodetoolResult("status", "-s", "host"); + hostResult.asserts().failure(); + assertTrue(hostResult.getStderr().contains("Error: Can not sort by host when there is not -r/--resolve-ip flag used.")); + + // test that resolving ips together with host sorting works + // we are not testing actual hosts ordering because calling nodetool status -r + // on addresses from 127.0.0.1-4 and having them being resolved to some names + // means that we would need to have a resolver set up to return these names + // which is quite impractical + hostResult = NODE_1.nodetoolResult("status", "-s", "host", "-r"); + hostResult.asserts().success(); + + hostResult = NODE_1.nodetoolResult("status", "-s", "host", "-r", "-o", "desc"); + hostResult.asserts().success(); + + hostResult = NODE_1.nodetoolResult("status", "-s", "host", "-r", "-o", "asc"); + hostResult.asserts().success(); + } + + @Test + public void testSortByIp() + { + assertOrdering(extractColumn(getOutput("status", "-s", "ip"), 1).stream().map(this::parseInetAddress).collect(toList()), asc); + assertOrdering(extractColumn(getOutput("status", "-s", "ip", "-o", "asc"), 1).stream().map(this::parseInetAddress).collect(toList()), asc); + assertOrdering(extractColumn(getOutput("status", "-s", "ip", "-o", "desc"), 1).stream().map(this::parseInetAddress).collect(toList()), desc); + } + + @Test + public void testSortByLoad() + { + assertOrdering(extractLoads(getOutput("status", "-s", "load")), desc); + assertOrdering(extractLoads(getOutput("status", "-s", "load", "-o", "asc")), asc); + assertOrdering(extractLoads(getOutput("status", "-s", "load", "-o", "desc")), desc); + } + + @Test + public void testSortByOwns() + { + assertOrdering(extractColumn(getOutput("status", "-s", "owns"), 3).stream().map(s -> parseDouble(s.replace("%", ""))).collect(toList()), desc); + assertOrdering(extractColumn(getOutput("status", "-s", "owns", "-o", "asc"), 3).stream().map(s -> parseDouble(s.replace("%", ""))).collect(toList()), asc); + assertOrdering(extractColumn(getOutput("status", "-s", "owns", "-o", "desc"), 3).stream().map(s -> parseDouble(s.replace("%", ""))).collect(toList()), desc); + } + + @Test + public void testSortByState() + { + assertOrdering(extractColumn(getOutput("status", "-s", "state"), 0), desc); + assertOrdering(extractColumn(getOutput("status", "-s", "state", "-o", "asc"), 0), asc); + assertOrdering(extractColumn(getOutput("status", "-s", "state", "-o", "desc"), 0), desc); + } + + protected String getOutput(String... args) + { + NodeToolResult result = NODE_1.nodetoolResult(args); + result.asserts().success(); + return result.getStdout(); + } + + private InetAddressAndPort parseInetAddress(String ip) + { + try + { + return InetAddressAndPort.getByName(ip); + } + catch (UnknownHostException e) + { + throw new IllegalArgumentException("Invalid IP address", e); + } + } + + protected List<String> extractColumn(String output, int... columnIndex) + { + List<String> columnValues = new ArrayList<>(); + String[] lines = output.split("\n"); + + // Skip the first five lines as headers + int skippedLines = 0; + for (String line : lines) + { + if (line.trim().isEmpty()) + { + continue; // Skip separator lines and empty lines + } + + // Skip the first five lines as they are headers + if (skippedLines < 5) + { + skippedLines++; + continue; + } + + // Use regular expression to extract columns + // Pattern will match any column with possible varying whitespace + String[] columns = PATTERN.split(line.trim()); // Split on 2 or more spaces + + // Check if the line has enough columns (avoid index out of bounds errors) + for (int i = 0; i < columnIndex.length; i++) + { + if (columns.length > columnIndex[i]) + { + columnValues.add(columns[columnIndex[i]].trim()); + } + } + } + + return columnValues; + } + + private <T extends Comparable<T>> void assertOrdering(List<T> columnOfValues, Status.SortOrder order) + { + List<T> sorted = new ArrayList<>(columnOfValues); + if (order == asc) + { + Collections.sort(sorted); + assertEquals("Not sorted in ascending order", sorted, columnOfValues); + } + else + { + sorted.sort(Collections.reverseOrder()); + assertEquals("Not sorted in descending order", sorted, columnOfValues); + } + } + + protected List<UUID> parseUUIDs(List<String> stringUUIDs) + { + List<UUID> result = new ArrayList<>(); + for (String stringUUID : stringUUIDs) + result.add(UUID.fromString(stringUUID)); + return result; + } + + protected List<Long> parseLongs(List<String> stringLongs) + { + List<Long> result = new ArrayList<>(); + for (String stringLong : stringLongs) + result.add(Long.parseLong(stringLong)); + return result; + } + + private List<Long> extractLoads(String output) + { + // Extract the load values and handle '?' placement + List<String> loads = extractColumn(output, 2); + return loads.stream() + .map(load -> { + // If load contains '?', assign it a value that will be placed first + if (load.contains("?")) + { + return Long.MIN_VALUE; + } + return FileUtils.parseFileSize(load); + }) + .collect(toList()); + } + + protected void compareByRacksInternal(String[] args, Status.SortOrder order, BiConsumer<String, Integer> biConsumer) + { + String output = getOutput(args); + + List<String> expectedRacks; + if (order == Status.SortOrder.desc) + expectedRacks = List.of("rack2", "rack2", "rack1", "rack1"); + else + expectedRacks = List.of("rack1", "rack1", "rack2", "rack2"); + + assertEquals("Rack values are not sorted in " + order.name() + " order", + expectedRacks, + extractColumn(output, 6)); + + int expectedComparision = order == Status.SortOrder.desc ? 1 : -1; + + biConsumer.accept(output, expectedComparision); + } + + + protected void sortByIdInternal(String[] args, Status.SortOrder order, int index) + { + String output = getOutput(args); + List<UUID> ids = parseUUIDs(extractColumn(output, index)); + List<UUID> sortedIds = new ArrayList<>(ids); + if (order == asc) + { + Collections.sort(sortedIds); + assertEquals("Ids are not sorted in ascending order", sortedIds, ids); + } + else + { + sortedIds.sort(Collections.reverseOrder()); + assertEquals("Ids are not sorted in descending order", sortedIds, ids); + } + } + + protected void sortByTokenInternal(String[] args, Status.SortOrder order) + { + String output = getOutput(args); + List<Long> tokens = parseLongs(extractColumn(output, 5)); + List<Long> sortedTokens = new ArrayList<>(tokens); + if (order == asc) + { + Collections.sort(sortedTokens); + assertEquals("Tokens are not sorted in ascending order", sortedTokens, tokens); + } + else + { + sortedTokens.sort(Collections.reverseOrder()); + assertEquals("Tokens are not sorted in descending order", sortedTokens, tokens); + } + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/NodeToolStatusWithVNodesTest.java b/test/distributed/org/apache/cassandra/distributed/test/NodeToolStatusWithVNodesTest.java new file mode 100644 index 000000000000..22b0469836e9 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/NodeToolStatusWithVNodesTest.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.distributed.test; + +import java.io.IOException; +import java.util.List; +import java.util.UUID; + +import org.junit.BeforeClass; +import org.junit.Test; + +import static org.apache.cassandra.tools.nodetool.Status.SortOrder.asc; +import static org.apache.cassandra.tools.nodetool.Status.SortOrder.desc; +import static org.junit.Assert.assertEquals; + +public class NodeToolStatusWithVNodesTest extends AbstractNodetoolStatusTest +{ + @BeforeClass + public static void before() throws IOException + { + useVNodes = true; + AbstractNodetoolStatusTest.before(); + } + + @Test + public void testSortById() + { + sortByIdInternal(new String[]{ "status", "-s", "id" }, asc, 5); + sortByIdInternal(new String[]{ "status", "-s", "id", "-o", "asc" }, asc, 5); + sortByIdInternal(new String[]{ "status", "-s", "id", "-o", "desc" }, desc, 5); + } + + @Test + public void testSortByRack() + { + compareByRacksInternal(new String[]{ "status", "-s", "rack" }, asc, this::idComparator); + compareByRacksInternal(new String[]{ "status", "-s", "rack", "-o", "asc" }, asc, this::idComparator); + compareByRacksInternal(new String[]{ "status", "-s", "rack", "-o", "desc" }, desc, this::idComparator); + } + + private void idComparator(String output, int expectedComparision) + { + List<UUID> uuids = parseUUIDs(extractColumn(output, 5)); + assertEquals(expectedComparision, uuids.get(0).compareTo(uuids.get(1))); + assertEquals(expectedComparision, uuids.get(2).compareTo(uuids.get(3))); + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/NodetoolStatusWithoutVNodesTest.java b/test/distributed/org/apache/cassandra/distributed/test/NodetoolStatusWithoutVNodesTest.java new file mode 100644 index 000000000000..411fa7a08e6e --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/NodetoolStatusWithoutVNodesTest.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import java.util.List; + +import org.junit.Test; + +import static org.apache.cassandra.tools.nodetool.Status.SortOrder.asc; +import static org.apache.cassandra.tools.nodetool.Status.SortOrder.desc; +import static org.junit.Assert.assertEquals; + +public class NodetoolStatusWithoutVNodesTest extends AbstractNodetoolStatusTest +{ + @Test + public void testSortById() + { + sortByIdInternal(new String[]{ "status", "-s", "id" }, asc, 4); + sortByIdInternal(new String[]{ "status", "-s", "id", "-o", "asc" }, asc, 4); + sortByIdInternal(new String[]{ "status", "-s", "id", "-o", "desc" }, desc, 4); + } + + @Test + public void testSortByToken() + { + sortByTokenInternal(new String[]{ "status", "-s", "token" }, asc); + sortByTokenInternal(new String[]{ "status", "-s", "token", "-o", "asc" }, asc); + sortByTokenInternal(new String[]{ "status", "-s", "token", "-o", "desc" }, desc); + } + + @Test + public void testSortByRack() + { + compareByRacksInternal(new String[]{ "status", "-s", "rack" }, asc, this::tokenComparator); + compareByRacksInternal(new String[]{ "status", "-s", "rack", "-o", "asc" }, asc, this::tokenComparator); + compareByRacksInternal(new String[]{ "status", "-s", "rack", "-o", "desc" }, desc, this::tokenComparator); + } + + private void tokenComparator(String output, int expectedComparision) + { + List<String> tokens = extractColumn(output, 5); + assertEquals(expectedComparision, Long.compare(Long.parseLong(tokens.get(0)), Long.parseLong(tokens.get(1)))); + assertEquals(expectedComparision, Long.compare(Long.parseLong(tokens.get(2)), Long.parseLong(tokens.get(3)))); + } +} From a98cb154763341773eba44e2c0d465f5980565d7 Mon Sep 17 00:00:00 2001 From: Sam Tunnicliffe <samt@apache.org> Date: Fri, 13 Dec 2024 11:24:04 +0000 Subject: [PATCH 059/225] Handle partitioned nodes in DiscoverySimulationTest Patch by Sam Tunnicliffe; reviewed by Brandon Williams for CASSANDRA-19505 --- .../test/tcm/CMSPlacementAfterMoveTest.java | 1 + .../tcm/DiscoverySimulationTest.java | 62 ++++++++++++++----- 2 files changed, 48 insertions(+), 15 deletions(-) diff --git a/test/distributed/org/apache/cassandra/distributed/test/tcm/CMSPlacementAfterMoveTest.java b/test/distributed/org/apache/cassandra/distributed/test/tcm/CMSPlacementAfterMoveTest.java index 8b66e97c4136..99b33984e559 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/tcm/CMSPlacementAfterMoveTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/tcm/CMSPlacementAfterMoveTest.java @@ -39,6 +39,7 @@ public void testMoveToCMS() throws IOException { try (Cluster cluster = init(Cluster.build(4) .withConfig(c -> c.with(Feature.GOSSIP, Feature.NETWORK)) + .withoutVNodes() .start())) { cluster.get(1).nodetoolResult("cms", "reconfigure", "3").asserts().success(); diff --git a/test/unit/org/apache/cassandra/tcm/DiscoverySimulationTest.java b/test/unit/org/apache/cassandra/tcm/DiscoverySimulationTest.java index f8380a6cb697..564eb98f8457 100644 --- a/test/unit/org/apache/cassandra/tcm/DiscoverySimulationTest.java +++ b/test/unit/org/apache/cassandra/tcm/DiscoverySimulationTest.java @@ -19,19 +19,18 @@ package org.apache.cassandra.tcm; import java.io.IOException; -import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.lang3.NotImplementedException; -import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; @@ -49,6 +48,9 @@ import org.apache.cassandra.tcm.ownership.UniformRangePlacement; import org.apache.cassandra.utils.concurrent.Future; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + public class DiscoverySimulationTest { static @@ -84,6 +86,9 @@ public void discoveryTest() throws Throwable Set<InetAddressAndPort> seeds = new HashSet<>(); seeds.add(InetAddressAndPort.getByName("127.0.100.1")); seeds.add(InetAddressAndPort.getByName("127.0.100.100")); // add an unreachable node + // Thread per-node to try and avoid nodes which start first finishing + // before later nodes have made first contact with the seed. + Executor executor = Executors.newFixedThreadPool(10); for (int i = 1; i <= 10; i++) { InetAddressAndPort addr = InetAddressAndPort.getByName("127.0.100." + i); @@ -94,15 +99,33 @@ public void discoveryTest() throws Throwable messaging.handlers.put(Verb.TCM_DISCOVER_REQ, discovery.requestHandler); } - List<CompletableFuture<Discovery.DiscoveredNodes>> futures = new ArrayList<>(); - for (Discovery value : nodes.values()) - futures.add(CompletableFuture.supplyAsync(() -> value.discover(5))); - - for (CompletableFuture<Discovery.DiscoveredNodes> future : futures) - future.get(); - - for (CompletableFuture<Discovery.DiscoveredNodes> future : futures) - Assert.assertEquals(nodes.keySet(), future.get().nodes()); + Map<InetAddressAndPort, CompletableFuture<Discovery.DiscoveredNodes>> futures = new HashMap<>(); + nodes.forEach((addr, discovery) -> { + futures.put(addr, CompletableFuture.supplyAsync(() -> discovery.discover(5), executor)); + }); + + Map<InetAddressAndPort, Discovery.DiscoveredNodes> discovered = new HashMap<>(); + for (Map.Entry<InetAddressAndPort, CompletableFuture<Discovery.DiscoveredNodes>> future : futures.entrySet()) + discovered.put(future.getKey(), future.getValue().get()); + + // It's possible that some node(s) in the cluster were completely unable to contact any seed. Therefore, these + // nodes are undiscoverable by the rest of the cluster so we exclude them from the expected results. We should + // also expect those partitioned nodes to have failed to discover any of their peers. + Set<InetAddressAndPort> connected = new HashSet<>(); + cluster.forEach((addr, messaging) -> { + if (messaging.hasSentAtLeastOneRequest) + connected.add(addr); + }); + + for (Map.Entry<InetAddressAndPort, Discovery.DiscoveredNodes> result : discovered.entrySet()) + { + InetAddressAndPort node = result.getKey(); + Set<InetAddressAndPort> peersDiscovered = result.getValue().nodes(); + if (connected.contains(node)) + assertEquals(node + " was able to contact seed, but didn't discover expected peers", connected, peersDiscovered); + else + assertTrue(node + " was unable to contact seed, but discovered " + peersDiscovered, peersDiscovered.isEmpty()); + } } /** @@ -113,12 +136,17 @@ public void discoveryTest() throws Throwable */ public static class FakeMessageDelivery implements MessageDelivery { - private static AtomicInteger counter = new AtomicInteger(); + private final AtomicInteger counter = new AtomicInteger(); private final Map<InetAddressAndPort, FakeMessageDelivery> cluster; private final Map<Long, RequestCallback<?>> callbacks; private final Map<Verb, IVerbHandler<?>> handlers; private final InetAddressAndPort addr; + // If we're unlucky, every attempt by this node to contact the single live peer may + // be randomly chosen to fail. If this happens, no other node is able to discover + // that this one exists and so it should be excluded from the expected set when + // checking results. + public boolean hasSentAtLeastOneRequest = false; public FakeMessageDelivery(Map<InetAddressAndPort, FakeMessageDelivery> cluster, InetAddressAndPort addr) @@ -139,6 +167,7 @@ public <REQ> void send(Message<REQ> message, InetAddressAndPort to) { if (message.verb().isResponse()) { + logger.info("{} sending response to {}", addr, to); cluster.get(to).deliverResponse(Message.forgeIdentityForTests(message, addr)); return; } @@ -154,14 +183,17 @@ public <REQ, RSP> void sendWithCallback(Message<REQ> message, InetAddressAndPort FakeMessageDelivery node = cluster.get(to); if (node != null && // Inject some failures - counter.incrementAndGet() % 2 != 0) + counter.incrementAndGet() % 5 != 0) { + hasSentAtLeastOneRequest = true; + logger.info("{} sending request to {}", addr, to); callbacks.put(message.id(), cb); IVerbHandler<REQ> handler = (IVerbHandler<REQ>) node.handlers.get(message.verb()); handler.doVerb(message); } else { + logger.info("{} simulating failure sending request to {}", addr, to); cb.onFailure(to, RequestFailureReason.TIMEOUT); } } @@ -186,4 +218,4 @@ public <V> void respond(V response, Message<?> message) throw new NotImplementedException(); } } -} +} \ No newline at end of file From 0d05233bc83a641ed7986fa31a678557c1db85b3 Mon Sep 17 00:00:00 2001 From: mck <mck@apache.org> Date: Wed, 11 Dec 2024 17:27:56 +1100 Subject: [PATCH 060/225] Remove REPEATED_TESTS from repeated tests script MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit patch by Mick Semb Wever; reviewed by Štefan Miklošovič for CASSANDRA-20139 --- .build/README.md | 12 ++++++++++-- .build/run-tests.sh | 5 ++--- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/.build/README.md b/.build/README.md index 77c5d8f67dd0..52a8cc2178da 100644 --- a/.build/README.md +++ b/.build/README.md @@ -127,8 +127,12 @@ Repeating tests Just add the '-repeat' suffix to the test type and pass in the repeat arguments - .build/run-tests.sh -a jvm-dtest-repeat -e REPEATED_TESTS_COUNT=2 -e REPEATED_TESTS=BooleanTest - .build/docker/run-tests.sh -a jvm-dtest-repeat -e REPEATED_TESTS_COUNT=2 -e REPEATED_TESTS=BooleanTest -j 11 + .build/run-tests.sh -a jvm-dtest-repeat -t BooleanTest -e REPEATED_TESTS_COUNT=2 + .build/docker/run-tests.sh -a jvm-dtest-repeat -t BooleanTest -e REPEATED_TESTS_COUNT=2 -j 11 + +Fail fast with repeating tests is done with REPEATED_TESTS_STOP_ON_FAILURE + + .build/run-tests.sh -a jvm-dtest-repeat -t BooleanTest -e REPEATED_TESTS_COUNT=2 -e REPEATED_TESTS_STOP_ON_FAILURE=false Running python dtests without docker: @@ -144,6 +148,10 @@ Other python dtest types without docker: .build/run-python-dtests.sh dtest-upgrade-large +Debugging test scripts: + + DEBUG=true .build/docker/run-tests.sh -a test + Running Sonar analysis (experimental) ------------------------------------- diff --git a/.build/run-tests.sh b/.build/run-tests.sh index 9ace42cbb571..756c8b3b2d6f 100755 --- a/.build/run-tests.sh +++ b/.build/run-tests.sh @@ -57,7 +57,6 @@ print_help() { echo " or when there are too many changed tests the CI env to handle." echo " -e <key=value> Environment variables to be used in the repeated runs:" echo " -e REPEATED_TESTS_STOP_ON_FAILURE=false" - echo " -e REPEATED_TESTS=org.apache.cassandra.cql3.ViewTest,ForceCompactionTest" echo " -e REPEATED_TESTS_COUNT=500" echo " If you want to specify multiple environment variables simply add multiple -e options." echo " -i Ignore unknown environment variables" @@ -125,7 +124,7 @@ if $has_env_vars && $check_env_vars; then for entry in $(echo $env_vars | tr "|" "\n"); do key=$(echo $entry | tr "=" "\n" | sed -n 1p) case $key in - "REPEATED_TESTS_STOP_ON_FAILURE" | "REPEATED_TESTS" | "REPEATED_TESTS_COUNT" ) + "REPEATED_TESTS_STOP_ON_FAILURE" | "REPEATED_TESTS_COUNT" ) [[ ${test_target} == *"-repeat" ]] || { error 1 "'-e REPEATED_*' variables only valid against *-repeat target types"; } ;; *) @@ -297,7 +296,7 @@ _main() { if [[ ${test_target} == *"-repeat" ]] ; then [[ "${split_chunk}" =~ ^[0-9]+/[0-9]+$ ]] && { error 1 "Repeated tests not valid with splits"; } if [[ -z "${test_name_regexp}" ]] ; then - test_name_regexp="$(_get_env_var 'REPEATED_TESTS')" + error 1 "Repeated tests requires use of -t option" fi local -r repeat_count="$(_get_env_var 'REPEATED_TESTS_COUNT')" else From 227d294986532d460ee6cadd1a237a3a10b67538 Mon Sep 17 00:00:00 2001 From: Sarma Pydipally <sarma1807@users.noreply.github.com> Date: Sat, 21 Dec 2024 18:15:48 -0500 Subject: [PATCH 061/225] Typo in storage-engine.adoc patch by Sarma Pydipally; reviewd by Bernardo Botella, Mick Semb Wever for CASSANDRA-20168 --- doc/modules/cassandra/pages/architecture/storage-engine.adoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/doc/modules/cassandra/pages/architecture/storage-engine.adoc b/doc/modules/cassandra/pages/architecture/storage-engine.adoc index 51e6f6e9dc21..36af06c746b2 100644 --- a/doc/modules/cassandra/pages/architecture/storage-engine.adoc +++ b/doc/modules/cassandra/pages/architecture/storage-engine.adoc @@ -244,7 +244,7 @@ For implementation docs see (https://github.com/apache/cassandra/blob/cassandra- === Version 5 -* da (5.0): initial version of the BIT format +* da (5.0): initial version of the BTI format === Example Code From 4354feef91eda071d880f7b9dfc4473d88124740 Mon Sep 17 00:00:00 2001 From: mck <mck@apache.org> Date: Wed, 1 Jan 2025 12:03:15 +0100 Subject: [PATCH 062/225] Fix nodetool docs page patch by Mick Semb Wever; reviewed by Brandon Williams for CASSANDRA-20030 --- doc/scripts/gen-nodetool-docs.py | 26 ++++++++++++++++++++------ 1 file changed, 20 insertions(+), 6 deletions(-) diff --git a/doc/scripts/gen-nodetool-docs.py b/doc/scripts/gen-nodetool-docs.py index 835f3f40a225..c4eb10966286 100644 --- a/doc/scripts/gen-nodetool-docs.py +++ b/doc/scripts/gen-nodetool-docs.py @@ -60,12 +60,26 @@ def batched(iterable, n): def create_help_file(): with open(helpfilename, "w+") as output_file: try: - subprocess.check_call([nodetool, "help"], stdout=output_file) + proc = subprocess.Popen([nodetool, "help"], stdout=subprocess.PIPE, stderr=subprocess.PIPE) + out, err = proc.communicate() + if proc.returncode != 0: + print( + 'ERROR: Nodetool failed to run, you likely need to build ' + 'cassandra using ant jar from the top level directory' + ) + raise subprocess.CalledProcessError(proc.returncode, proc.args, output=out, stderr=err) + + # Wrap the initial "usage: ..." block + usage_block = re.search(r'usage:.*?\n\n', out.decode('utf-8'), re.DOTALL | re.MULTILINE) + if usage_block: + output_file.write("[source,console]\n----\n") + output_file.write(usage_block.group(0)) + output_file.write("----\n") + out = out.decode('utf-8').replace(usage_block.group(0), '') + else: + raise ValueError("No usage block matched in nodetool help output") + output_file.write(out) except subprocess.CalledProcessError as cpe: - print( - 'ERROR: Nodetool failed to run, you likely need to build ' - 'cassandra using ant jar from the top level directory' - ) raise cpe # for a given command, create the help file and an ADOC file to contain it @@ -89,7 +103,7 @@ def create_adoc(command): with open(helpfilename, "r+") as helpfile: output.write("= Nodetool\n\n== Usage\n\n") for commandLine in helpfile: - command = command_re.sub(r'\nxref:modules/cassandra/pages/managing/tools/nodetool/\2.adoc[\2] - ',commandLine) + command = command_re.sub(r'\nxref:cassandra:managing/tools/nodetool/\2.adoc[\2] - ',commandLine) output.write(command) # create the command usage pages From 0e7bd24f5c7e17ef8db3e5613d0889e7277a93ab Mon Sep 17 00:00:00 2001 From: vsricharan16 <cs16btech11044@iith.ac.in> Date: Sat, 20 Jan 2024 03:17:15 +0000 Subject: [PATCH 063/225] Fix typo in UNLOGGED BATCH section of CQL DML page patch by V Sri Charan Reddy ; reviewed by Erick Ramirez for CASSANDRA-17846 --- doc/modules/cassandra/pages/developing/cql/dml.adoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/doc/modules/cassandra/pages/developing/cql/dml.adoc b/doc/modules/cassandra/pages/developing/cql/dml.adoc index ef76cdbb38ff..674ede814518 100644 --- a/doc/modules/cassandra/pages/developing/cql/dml.adoc +++ b/doc/modules/cassandra/pages/developing/cql/dml.adoc @@ -456,7 +456,7 @@ only isolated within a single partition). There is a performance penalty for batch atomicity when a batch spans multiple partitions. If you do not want to incur this penalty, you can tell Cassandra to skip the batchlog with the `UNLOGGED` option. If the -`UNLOGGED` option is used, a failed batch might leave the patch only +`UNLOGGED` option is used, a failed batch might leave the batch only partly applied. === `COUNTER` batches From ac36ddd7f644bddafe1493666f592651a57604ca Mon Sep 17 00:00:00 2001 From: Brad Schoening <bschoeni@gmail.com> Date: Sat, 18 May 2024 04:35:03 -0400 Subject: [PATCH 064/225] added summary to storage-engine doc patch by Brad Schoening; reviewed by Mick Semb Wever for CASSANDRA-19640 --- .../cassandra/pages/architecture/storage-engine.adoc | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/doc/modules/cassandra/pages/architecture/storage-engine.adoc b/doc/modules/cassandra/pages/architecture/storage-engine.adoc index 36af06c746b2..61bd549af191 100644 --- a/doc/modules/cassandra/pages/architecture/storage-engine.adoc +++ b/doc/modules/cassandra/pages/architecture/storage-engine.adoc @@ -1,6 +1,14 @@ = Storage Engine -{cassandra} processes data at several stages on the write path, starting with the immediate logging of a write and ending in with a write of data to disk: +The {Cassandra} storage engine is optimized for high performance, write-oriented workloads. The architecture is based on Log Structured Merge (LSM) trees, which utilize an append-only approach instead of the traditional relational database design with B-trees. This creates a write path free of read lookups and bottlenecks. + +While the write path is highly optimized, it comes with tradeoffs in terms of read performance and write amplification. To enhance read operations, Cassandra uses Bloom filters when accessing data from stables. Bloom filters are remarkably efficient, leading to generally well-balanced performance for both reads and writes. + +Compaction is a necessary background activity required by the ‘merge’ phase of Log Structured Merge trees. Compaction creates write amplification when several small SSTables on disk are read, merged, updates and deletes processed, and a new ssstable is re-written. Every write of data in Cassandra is re-written multiple times, known as write amplification, and this adds background I/O to the database workload. + +The core storage engine consists of memtables for in-memory data and immutable SSTables (Sorted String Tables) on disk. Data in SSTables is stored sorted to enable efficent merge sort during compaction. Additionally, a write-ahead log (WAL), referred to as the commit log, ensures resiliency for crash and transaction recovery. + +The sequence of the steps in the write path: * Logging data in the commit log * Writing data to the memtable From 2841842f0a55010241e705fad5aae37cb4131d0b Mon Sep 17 00:00:00 2001 From: Maxim Chanturiay <maxim.chanturiay@ibm.com> Date: Sun, 3 Sep 2023 17:34:40 +0300 Subject: [PATCH 065/225] Fixed the link to non-existing cleaner Runnable patch by Maxim Chanturiay; reviewed by Mick Semb Wever for CASSANDRA-18801 --- .../apache/cassandra/utils/memory/MemtableCleanerThread.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/utils/memory/MemtableCleanerThread.java b/src/java/org/apache/cassandra/utils/memory/MemtableCleanerThread.java index dbc23e576fcd..1add03343074 100644 --- a/src/java/org/apache/cassandra/utils/memory/MemtableCleanerThread.java +++ b/src/java/org/apache/cassandra/utils/memory/MemtableCleanerThread.java @@ -35,7 +35,7 @@ /** * A thread that reclaims memory from a MemtablePool on demand. The actual reclaiming work is delegated to the - * cleaner Runnable, e.g., FlushLargestColumnFamily + * cleaner Runnable, e.g., MemtableCleaner {@link AbstractAllocatorMemtable#flushLargestMemtable()}. */ public class MemtableCleanerThread<P extends MemtablePool> implements Interruptible { From 51696d239aa27e933d7e99f131adb45f7139f4a8 Mon Sep 17 00:00:00 2001 From: Andrew Hogg <andrew.hogg@datastax.com> Date: Tue, 25 Jul 2023 09:43:49 +0100 Subject: [PATCH 066/225] Remove 2 metrics listed in 3.11 documentation that weren't added until 4.0 patch by Andrew Hogg; reviewed Lorina Poland for CASSANDRA-18689 --- doc/modules/cassandra/pages/operating/metrics.adoc | 6 ------ 1 file changed, 6 deletions(-) diff --git a/doc/modules/cassandra/pages/operating/metrics.adoc b/doc/modules/cassandra/pages/operating/metrics.adoc index 1eb8156dc20a..78b153f5c36c 100644 --- a/doc/modules/cassandra/pages/operating/metrics.adoc +++ b/doc/modules/cassandra/pages/operating/metrics.adoc @@ -913,12 +913,6 @@ Reported name format: |Name |Type |Description |connectedNativeClients |Gauge<Integer> |Number of clients connected to this nodes native protocol server - -|connections |Gauge<List<Map<String, String>> |List of all connections -and their state information - -|connectedNativeClientsByUser |Gauge<Map<String, Int> |Number of -connnective native clients by username |=== == Batch Metrics From c853efffa8b173a3afe1b966456bb77db5a68883 Mon Sep 17 00:00:00 2001 From: Abe Ratnofsky <abe@aber.io> Date: Fri, 6 Dec 2024 13:41:31 -0500 Subject: [PATCH 067/225] Support audit logging for JMX operations Patch by Abe Ratnofsky; reviewed by Bernardo Botella, Doug Rohrer, Francisco Guerrero for CASSANDRA-20128 --- CHANGES.txt | 1 + build.xml | 2 +- .../audit/AuditLogEntryCategory.java | 2 +- .../cassandra/audit/AuditLogEntryType.java | 3 +- .../cassandra/audit/AuditLogFilter.java | 16 +- .../cassandra/audit/AuditLogManager.java | 131 ++++++++++++++++- .../cassandra/auth/CassandraPrincipal.java | 2 +- .../auth/jmx/AuthorizationProxy.java | 56 ++++--- .../cassandra/utils/JMXServerUtils.java | 6 +- .../utils/JmxInvocationListener.java | 50 +++++++ .../cassandra/audit/AuditLoggerTest.java | 138 +++++++++++++++++- .../auth/jmx/AbstractJMXAuthTest.java | 34 +++++ 12 files changed, 411 insertions(+), 30 deletions(-) create mode 100644 src/java/org/apache/cassandra/utils/JmxInvocationListener.java diff --git a/CHANGES.txt b/CHANGES.txt index 797312bf01d1..cf594214283c 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Support audit logging for JMX operations (CASSANDRA-20128) * Enable sorting of nodetool status output (CASSANDRA-20104) * Support downgrading after CMS is initialized (CASSANDRA-20145) * Deprecate IEndpointSnitch (CASSANDRA-19488) diff --git a/build.xml b/build.xml index 45af0462603b..ccc38717d28c 100644 --- a/build.xml +++ b/build.xml @@ -338,7 +338,7 @@ <!-- needed to compile org.apache.cassandra.utils.JMXServerUtils --> <!-- needed to compile org.apache.cassandra.distributed.impl.Instance--> <!-- needed to compile org.apache.cassandra.utils.memory.BufferPool --> - <property name="jdk11plus-javac-exports" value="--add-exports java.rmi/sun.rmi.registry=ALL-UNNAMED --add-exports java.rmi/sun.rmi.transport.tcp=ALL-UNNAMED --add-exports java.base/jdk.internal.ref=ALL-UNNAMED --add-exports java.base/sun.nio.ch=ALL-UNNAMED" /> + <property name="jdk11plus-javac-exports" value="--add-exports java.rmi/sun.rmi.registry=ALL-UNNAMED --add-exports java.rmi/sun.rmi.transport.tcp=ALL-UNNAMED --add-exports java.base/jdk.internal.ref=ALL-UNNAMED --add-exports java.base/sun.nio.ch=ALL-UNNAMED --add-exports java.management/com.sun.jmx.remote.security=ALL-UNNAMED" /> <!-- Add all the dependencies. diff --git a/src/java/org/apache/cassandra/audit/AuditLogEntryCategory.java b/src/java/org/apache/cassandra/audit/AuditLogEntryCategory.java index 9db4ce05e9c7..b848440607a5 100644 --- a/src/java/org/apache/cassandra/audit/AuditLogEntryCategory.java +++ b/src/java/org/apache/cassandra/audit/AuditLogEntryCategory.java @@ -23,5 +23,5 @@ */ public enum AuditLogEntryCategory { - QUERY, DML, DDL, DCL, OTHER, AUTH, ERROR, PREPARE + QUERY, DML, DDL, DCL, OTHER, AUTH, ERROR, PREPARE, JMX } diff --git a/src/java/org/apache/cassandra/audit/AuditLogEntryType.java b/src/java/org/apache/cassandra/audit/AuditLogEntryType.java index 17d4c98feafe..484895a21b28 100644 --- a/src/java/org/apache/cassandra/audit/AuditLogEntryType.java +++ b/src/java/org/apache/cassandra/audit/AuditLogEntryType.java @@ -71,7 +71,8 @@ public enum AuditLogEntryType LOGIN_ERROR(AuditLogEntryCategory.AUTH), UNAUTHORIZED_ATTEMPT(AuditLogEntryCategory.AUTH), LOGIN_SUCCESS(AuditLogEntryCategory.AUTH), - LIST_SUPERUSERS(AuditLogEntryCategory.DCL); + LIST_SUPERUSERS(AuditLogEntryCategory.DCL), + JMX(AuditLogEntryCategory.JMX); private final AuditLogEntryCategory category; diff --git a/src/java/org/apache/cassandra/audit/AuditLogFilter.java b/src/java/org/apache/cassandra/audit/AuditLogFilter.java index d240e78c83fc..b775ac7785cf 100644 --- a/src/java/org/apache/cassandra/audit/AuditLogFilter.java +++ b/src/java/org/apache/cassandra/audit/AuditLogFilter.java @@ -127,7 +127,21 @@ private IncludeExcludeHolder(ImmutableSet<String> includedSet, ImmutableSet<Stri } /** - * Checks whether a give AuditLog Entry is filtered or not + * Checks whether a given AuditLogEntryCategory is filtered or not. + * + * This is useful when creating an audit log entry might be expensive, and checking the category before formatting + * is less costly. + * + * @param category AuditLogEntryCategory to verify + * @return true if it is filtered, false otherwise + */ + boolean isFiltered(AuditLogEntryCategory category) + { + return isFiltered(category.toString(), includedCategories, excludedCategories); + } + + /** + * Checks whether a given AuditLog Entry is filtered or not * * @param auditLogEntry AuditLogEntry to verify * @return true if it is filtered, false otherwise diff --git a/src/java/org/apache/cassandra/audit/AuditLogManager.java b/src/java/org/apache/cassandra/audit/AuditLogManager.java index 0f49a540603d..5d82bd94532c 100644 --- a/src/java/org/apache/cassandra/audit/AuditLogManager.java +++ b/src/java/org/apache/cassandra/audit/AuditLogManager.java @@ -18,13 +18,26 @@ package org.apache.cassandra.audit; +import java.lang.reflect.InvocationHandler; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.lang.reflect.Proxy; import java.nio.ByteBuffer; +import java.security.AccessControlContext; +import java.security.AccessController; +import java.security.Principal; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Objects; +import java.util.StringJoiner; import java.util.UUID; + import javax.annotation.Nullable; +import javax.management.MBeanServer; import javax.management.openmbean.CompositeData; +import javax.management.remote.MBeanServerForwarder; +import javax.security.auth.Subject; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; @@ -49,15 +62,18 @@ import org.apache.cassandra.transport.Message; import org.apache.cassandra.transport.messages.ResultMessage; import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.JmxInvocationListener; import org.apache.cassandra.utils.MBeanWrapper; import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; +import static org.apache.cassandra.audit.AuditLogEntryType.JMX; + /** * Central location for managing the logging of client/user-initated actions (like queries, log in commands, and so on). * */ -public class AuditLogManager implements QueryEvents.Listener, AuthEvents.Listener, AuditLogManagerMBean +public class AuditLogManager implements QueryEvents.Listener, AuthEvents.Listener, AuditLogManagerMBean, JmxInvocationListener { private static final Logger logger = LoggerFactory.getLogger(AuditLogManager.class); @@ -69,6 +85,9 @@ public class AuditLogManager implements QueryEvents.Listener, AuthEvents.Listene private volatile AuditLogFilter filter; private volatile AuditLogOptions auditLogOptions; + // Only reset in tests + private MBeanServerForwarder mbsf = createMBeanServerForwarder(); + private AuditLogManager() { auditLogOptions = DatabaseDescriptor.getAuditLoggingOptions(); @@ -159,7 +178,7 @@ else if (e instanceof AuthenticationException) { builder.setType(AuditLogEntryType.LOGIN_ERROR); } - else + else if (logEntry.getType() != JMX) { builder.setType(AuditLogEntryType.REQUEST_FAILURE); } @@ -400,4 +419,112 @@ else if (e instanceof PasswordGuardrail.PasswordGuardrailException) return PasswordObfuscator.obfuscate(e.getMessage()); } + + private static class JmxFormatter + { + private static String user(Subject subject) + { + if (subject == null) + return "null"; + StringJoiner joiner = new StringJoiner(", "); + for (Principal principal : subject.getPrincipals()) + joiner.add(Objects.toString(principal)); + return joiner.toString(); + } + + private static String method(Method method, Object[] args) + { + String argsFmt = ""; + if (args != null) + { + StringJoiner joiner = new StringJoiner(", "); + for (Object arg : args) + joiner.add(Objects.toString(arg)); + argsFmt = joiner.toString(); + } + return String.format("%s#%s(%s)", method.getDeclaringClass().getCanonicalName(), method.getName(), argsFmt); + } + } + + @Override + public void onInvocation(Subject subject, Method method, Object[] args) + { + if (filter.isFiltered(AuditLogEntryCategory.JMX)) + return; + + AuditLogEntry entry = new AuditLogEntry.Builder(JMX) + .setOperation(String.format("JMX INVOCATION: %s", JmxFormatter.method(method, args))) + .setUser(JmxFormatter.user(subject)) + .build(); + log(entry); + } + + @Override + public void onFailure(Subject subject, Method method, Object[] args, Exception exception) + { + if (filter.isFiltered(AuditLogEntryCategory.JMX)) + return; + + AuditLogEntry entry = new AuditLogEntry.Builder(JMX) + .setOperation(String.format("JMX FAILURE: %s due to %s", JmxFormatter.method(method, args), exception.getClass().getSimpleName())) + .setUser(JmxFormatter.user(subject)) + .build(); + log(entry, exception); + } + + private class JmxHandler implements InvocationHandler + { + private MBeanServer mbs = null; + + @Override + public Object invoke(Object proxy, Method method, Object[] args) throws Throwable + { + // See AuthorizationProxy.invoke + if ("setMBeanServer".equals(method.getName())) + { + if (args[0] == null) + throw new IllegalArgumentException("Null MBeanServer"); + + if (mbs != null) + throw new IllegalArgumentException("MBeanServer already initialized"); + + mbs = (MBeanServer) args[0]; + return null; + } + + AccessControlContext acc = AccessController.getContext(); + Subject subject = Subject.getSubject(acc); + + try + { + Object invoke = method.invoke(mbs, args); + AuditLogManager.this.onInvocation(subject, method, args); + return invoke; + } + catch (InvocationTargetException e) + { + AuditLogManager.instance.onFailure(subject, method, args, e); + throw e.getCause(); + } + } + } + + private MBeanServerForwarder createMBeanServerForwarder() + { + InvocationHandler handler = new JmxHandler(); + Class<?>[] interfaces = { MBeanServerForwarder.class }; + Object proxy = Proxy.newProxyInstance(MBeanServerForwarder.class.getClassLoader(), interfaces, handler); + return (MBeanServerForwarder) proxy; + } + + @VisibleForTesting + void resetMBeanServerForwarder() + { + this.mbsf = createMBeanServerForwarder(); + } + + public MBeanServerForwarder getMBeanServerForwarder() + { + return mbsf; + } } diff --git a/src/java/org/apache/cassandra/auth/CassandraPrincipal.java b/src/java/org/apache/cassandra/auth/CassandraPrincipal.java index 41de802a27dd..8dd0f5a8c25e 100644 --- a/src/java/org/apache/cassandra/auth/CassandraPrincipal.java +++ b/src/java/org/apache/cassandra/auth/CassandraPrincipal.java @@ -84,7 +84,7 @@ public String getName() @Override public String toString() { - return ("CassandraPrincipal: " + name); + return ("CassandraPrincipal: " + name); } /** diff --git a/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java b/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java index c6ea2d90a185..75576f6aef7f 100644 --- a/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java +++ b/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java @@ -39,9 +39,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.audit.AuditLogManager; import org.apache.cassandra.auth.*; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.utils.JmxInvocationListener; import org.apache.cassandra.utils.MBeanWrapper; /** @@ -140,43 +142,57 @@ public class AuthorizationProxy implements InvocationHandler */ protected BooleanSupplier isAuthSetupComplete = () -> StorageService.instance.isAuthSetupComplete(); + protected JmxInvocationListener listener = AuditLogManager.instance; + @Override public Object invoke(Object proxy, Method method, Object[] args) throws Throwable { String methodName = method.getName(); - if ("getMBeanServer".equals(methodName)) - throw new SecurityException("Access denied"); - - // Corresponds to MBeanServer.invoke - if (methodName.equals("invoke") && args.length == 4) - checkVulnerableMethods(args); - // Retrieve Subject from current AccessControlContext AccessControlContext acc = AccessController.getContext(); Subject subject = Subject.getSubject(acc); - // Allow setMBeanServer iff performed on behalf of the connector server itself - if (("setMBeanServer").equals(methodName)) + try { - if (subject != null) + if ("getMBeanServer".equals(methodName)) throw new SecurityException("Access denied"); - if (args[0] == null) - throw new IllegalArgumentException("Null MBeanServer"); + // Corresponds to MBeanServer.invoke + if (methodName.equals("invoke") && args.length == 4) + checkVulnerableMethods(args); - if (mbs != null) - throw new IllegalArgumentException("MBeanServer already initialized"); + // Allow setMBeanServer iff performed on behalf of the connector server itself + if (("setMBeanServer").equals(methodName)) + { + if (subject != null) + throw new SecurityException("Access denied"); - mbs = (MBeanServer) args[0]; - return null; - } + if (args[0] == null) + throw new IllegalArgumentException("Null MBeanServer"); - if (authorize(subject, methodName, args)) - return invoke(method, args); + if (mbs != null) + throw new IllegalArgumentException("MBeanServer already initialized"); - throw new SecurityException("Access Denied"); + mbs = (MBeanServer) args[0]; + return null; + } + + if (authorize(subject, methodName, args)) + { + Object invoke = invoke(method, args); + listener.onInvocation(subject, method, args); + return invoke; + } + + throw new SecurityException("Access Denied"); + } + catch (Exception e) + { + listener.onFailure(subject, method, args, e); + throw e; + } } /** diff --git a/src/java/org/apache/cassandra/utils/JMXServerUtils.java b/src/java/org/apache/cassandra/utils/JMXServerUtils.java index be8f943d2b45..0d61f2877d52 100644 --- a/src/java/org/apache/cassandra/utils/JMXServerUtils.java +++ b/src/java/org/apache/cassandra/utils/JMXServerUtils.java @@ -53,6 +53,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.audit.AuditLogManager; import org.apache.cassandra.auth.jmx.AuthenticationProxy; import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.config.JMXServerOptions; @@ -135,6 +136,9 @@ public static JMXConnectorServer createJMXServer(JMXServerOptions options, Strin // If a custom authz proxy was created, attach it to the server now. if (authzProxy != null) jmxServer.setMBeanServerForwarder(authzProxy); + else + jmxServer.setMBeanServerForwarder(AuditLogManager.instance.getMBeanServerForwarder()); + jmxServer.start(); registry.setRemoteServerStub(server.toStub()); @@ -274,7 +278,7 @@ private static String getJmxServiceUrl(InetAddress serverAddress, int port) } return String.format(urlTemplate, hostName, port); } - + private static class JMXPluggableAuthenticatorWrapper implements JMXAuthenticator { private static final MethodHandle ctorHandle; diff --git a/src/java/org/apache/cassandra/utils/JmxInvocationListener.java b/src/java/org/apache/cassandra/utils/JmxInvocationListener.java new file mode 100644 index 000000000000..cee473cc5f76 --- /dev/null +++ b/src/java/org/apache/cassandra/utils/JmxInvocationListener.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils; + +import java.lang.reflect.Method; +import javax.security.auth.Subject; + +/** + * Listener for operations executed over JMX. + */ +public interface JmxInvocationListener +{ + /** + * Listener called when an attempted invocation is successful. + * + * @param subject The subject attempting invocation, depends on how JMX authentication is configured + * @param method Invoked method + * @param args Invoked method arguments + */ + default void onInvocation(Subject subject, Method method, Object[] args) + {} + + /** + * Listener called when an attempted invocation throws an exception. This could happen before or after the + * underlying method is invoked, due to invocation wrappers such as {@link org.apache.cassandra.auth.jmx.AuthorizationProxy#invoke(Object, Method, Object[])}. + * + * @param subject The subject attempting invocation, depends on how JMX authentication is configured + * @param method Invoked method + * @param args Invoked method arguments + * @param cause Exception thrown by the attempted invocation + */ + default void onFailure(Subject subject, Method method, Object[] args, Exception cause) + {} +} diff --git a/test/unit/org/apache/cassandra/audit/AuditLoggerTest.java b/test/unit/org/apache/cassandra/audit/AuditLoggerTest.java index 6cfaa52e6482..46345d88fc7d 100644 --- a/test/unit/org/apache/cassandra/audit/AuditLoggerTest.java +++ b/test/unit/org/apache/cassandra/audit/AuditLoggerTest.java @@ -17,12 +17,24 @@ */ package org.apache.cassandra.audit; -import org.junit.After; import java.io.IOException; +import java.lang.reflect.Field; +import java.net.InetAddress; import java.nio.file.Files; import java.nio.file.Path; +import java.nio.file.Paths; +import java.rmi.server.RMISocketFactory; import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import javax.management.JMX; +import javax.management.MBeanServerConnection; +import javax.management.ObjectName; +import javax.management.remote.JMXConnector; +import javax.management.remote.JMXConnectorFactory; +import javax.management.remote.JMXServiceURL; +import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; @@ -36,15 +48,35 @@ import com.datastax.driver.core.exceptions.SyntaxError; import net.openhft.chronicle.queue.RollCycles; import org.apache.cassandra.auth.AuthEvents; +import org.apache.cassandra.auth.AuthenticatedUser; +import org.apache.cassandra.auth.IAuthorizer; +import org.apache.cassandra.auth.JMXResource; +import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.auth.RoleResource; +import org.apache.cassandra.auth.StubAuthorizer; +import org.apache.cassandra.auth.jmx.AbstractJMXAuthTest; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.JMXServerOptions; import org.apache.cassandra.config.ParameterizedClass; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.cql3.QueryEvents; +import org.apache.cassandra.db.ColumnFamilyStoreMBean; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.service.StorageService; - +import org.apache.cassandra.utils.JMXServerUtils; +import org.assertj.core.api.Assertions; + +import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_AUTHORIZER; +import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_LOCAL_PORT; +import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_REMOTE_LOGIN_CONFIG; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_AUTHENTICATE; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_SECURITY_AUTH_LOGIN_CONFIG; import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.Matchers.emptyCollectionOf; +import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.stringContainsInOrder; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNull; @@ -758,6 +790,108 @@ public void testJMXArchiveCommand() throws IOException assertEquals("/xyz/not/null", AuditLogManager.instance.getAuditLogOptions().archive_command); } + @Test + public void testJMXAuditLogs() throws Throwable + { + // Need to use distinct ports, otherwise would get RMI registry object ID collision, even with server shutdown between + testJMXAuditLogs(false, getAutomaticallyAllocatedPort(InetAddress.getLoopbackAddress())); + testJMXAuditLogs(true, getAutomaticallyAllocatedPort(InetAddress.getLoopbackAddress())); + } + + private void testJMXAuditLogs(boolean enableAuthorizationProxy, int port) throws Throwable + { + if (enableAuthorizationProxy) + { + // Set up JMX; see AbstractJMXAuthTest.setupAuthorizer + IAuthorizer authorizer = new StubAuthorizer(); + Field authorizerField = DatabaseDescriptor.class.getDeclaredField("authorizer"); + authorizerField.setAccessible(true); + authorizerField.set(null, authorizer); + DatabaseDescriptor.setPermissionsValidity(0); + } + + JMXResource tableMBean = JMXResource.mbean("org.apache.cassandra.db:type=Tables,keyspace=system_auth,table=roles"); + + if (enableAuthorizationProxy) + { + DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER, + Permission.ALL, + tableMBean, + RoleResource.role("test_role")); + } + + DatabaseDescriptor.setAuditLoggingOptions(new AuditLogOptions.Builder() + .withEnabled(true) + .withBlock(true) + .withLogger("InMemoryAuditLogger", null) + .build()); + + if (enableAuthorizationProxy) + { + String config = Paths.get(ClassLoader.getSystemResource("auth/cassandra-test-jaas.conf").toURI()).toString(); + COM_SUN_MANAGEMENT_JMXREMOTE_AUTHENTICATE.setBoolean(true); + JAVA_SECURITY_AUTH_LOGIN_CONFIG.setString(config); + CASSANDRA_JMX_REMOTE_LOGIN_CONFIG.setString("TestLogin"); + CASSANDRA_JMX_AUTHORIZER.setString(AbstractJMXAuthTest.NoSuperUserAuthorizationProxy.class.getName()); + } + CASSANDRA_JMX_LOCAL_PORT.setInt(port); + JMXServerOptions options = JMXServerOptions.createParsingSystemProperties(); + options.jmx_encryption_options.applyConfig(); + JMXServerUtils.createJMXServer(options, "localhost").start(); + + JMXServiceURL jmxUrl = new JMXServiceURL(String.format("service:jmx:rmi:///jndi/rmi://localhost:%d/jmxrmi", port)); + Map<String, Object> env = new HashMap<>(); + env.put("com.sun.jndi.rmi.factory.socket", RMISocketFactory.getDefaultSocketFactory()); + JMXConnector jmxc = JMXConnectorFactory.connect(jmxUrl, env); + MBeanServerConnection connection = jmxc.getMBeanServerConnection(); + + // Setting up the connection will cause a few JMX methods to be called, so need to reset to empty + Assert.assertThat(((InMemoryAuditLogger) AuditLogManager.instance.getLogger()).inMemQueue, not(emptyCollectionOf(AuditLogEntry.class))); + ((InMemoryAuditLogger) AuditLogManager.instance.getLogger()).inMemQueue.clear(); + + // Do an operation that doesn't fail + ColumnFamilyStoreMBean proxy = JMX.newMBeanProxy(connection, + ObjectName.getInstance(tableMBean.getObjectName()), + ColumnFamilyStoreMBean.class); + proxy.getTableName(); + AuditLogEntry logEntry = ((InMemoryAuditLogger) AuditLogManager.instance.getLogger()).inMemQueue.poll(); + assertEquals(AuditLogEntryType.JMX, logEntry.getType()); + assertThat(logEntry.getOperation(), containsString("JMX INVOCATION")); + if (enableAuthorizationProxy) + assertThat(logEntry.getUser(), is("CassandraPrincipal: test_role")); + else + assertThat(logEntry.getUser(), is("null")); + assertEquals(0, ((InMemoryAuditLogger) AuditLogManager.instance.getLogger()).inMemQueue.size()); + + // Do an operation that fails + tableMBean = JMXResource.mbean("org.apache.cassandra.db:type=Tables,keyspace=system_auth,table=roles"); + proxy = JMX.newMBeanProxy(connection, + ObjectName.getInstance(tableMBean.getObjectName()), + ColumnFamilyStoreMBean.class); + + ColumnFamilyStoreMBean finalProxy = proxy; + Assertions.assertThatThrownBy(() -> finalProxy.setMinimumCompactionThreshold(Integer.MAX_VALUE)) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("min_compaction_threshold cannot be larger than the max_compaction_threshold"); + + logEntry = ((InMemoryAuditLogger) AuditLogManager.instance.getLogger()).inMemQueue.poll(); + assertEquals(AuditLogEntryType.JMX, logEntry.getType()); + assertThat(logEntry.getOperation(), stringContainsInOrder("JMX INVOCATION", "getClassLoaderFor")); + assertThat(logEntry.getUser(), containsString("null")); + + // 2 JMX calls are produced, one to search for class, then one to invoke + logEntry = ((InMemoryAuditLogger) AuditLogManager.instance.getLogger()).inMemQueue.poll(); + assertEquals(AuditLogEntryType.JMX, logEntry.getType()); + assertThat(logEntry.getOperation(), stringContainsInOrder("JMX FAILURE", "setAttribute")); + if (enableAuthorizationProxy) + assertThat(logEntry.getUser(), is("CassandraPrincipal: test_role")); + else + assertThat(logEntry.getUser(), is("null")); + Assertions.assertThat(((InMemoryAuditLogger) AuditLogManager.instance.getLogger()).inMemQueue).isEmpty(); + + AuditLogManager.instance.resetMBeanServerForwarder(); + } + /** * Helper methods for Audit Log CQL Testing */ diff --git a/test/unit/org/apache/cassandra/auth/jmx/AbstractJMXAuthTest.java b/test/unit/org/apache/cassandra/auth/jmx/AbstractJMXAuthTest.java index 0d75ce6a6223..625795c10abc 100644 --- a/test/unit/org/apache/cassandra/auth/jmx/AbstractJMXAuthTest.java +++ b/test/unit/org/apache/cassandra/auth/jmx/AbstractJMXAuthTest.java @@ -22,6 +22,7 @@ import java.rmi.server.RMISocketFactory; import java.util.HashMap; import java.util.Map; +import java.util.Queue; import javax.management.JMX; import javax.management.MBeanServerConnection; import javax.management.ObjectName; @@ -39,6 +40,11 @@ import org.junit.Ignore; import org.junit.Test; +import org.apache.cassandra.audit.AuditLogEntry; +import org.apache.cassandra.audit.AuditLogEntryType; +import org.apache.cassandra.audit.AuditLogManager; +import org.apache.cassandra.audit.AuditLogOptions; +import org.apache.cassandra.audit.InMemoryAuditLogger; import org.apache.cassandra.auth.AuthenticatedUser; import org.apache.cassandra.auth.CassandraPrincipal; import org.apache.cassandra.auth.IAuthorizer; @@ -51,6 +57,7 @@ import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.db.ColumnFamilyStoreMBean; import org.apache.cassandra.utils.JMXServerUtils; +import org.assertj.core.api.Assertions; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -64,6 +71,7 @@ public abstract class AbstractJMXAuthTest extends CQLTester private RoleResource role; private String tableName; private JMXResource tableMBean; + private Queue<AuditLogEntry> auditLogs; @Before public void setup() throws Throwable @@ -73,6 +81,8 @@ public void setup() throws Throwable tableName = createTable("CREATE TABLE %s (k int, v int, PRIMARY KEY (k))"); tableMBean = JMXResource.mbean(String.format("org.apache.cassandra.db:type=Tables,keyspace=%s,table=%s", KEYSPACE, tableName)); + AuditLogManager.instance.enable(DatabaseDescriptor.getAuditLoggingOptions()); + auditLogs = ((InMemoryAuditLogger) AuditLogManager.instance.getLogger()).internalQueue(); } @Test @@ -157,6 +167,12 @@ public void executeMethod() throws Throwable protected static void setupJMXServer(JMXServerOptions jmxServerOptions) throws Exception { + DatabaseDescriptor.setAuditLoggingOptions(new AuditLogOptions.Builder() + .withEnabled(true) + .withBlock(true) + .withLogger("InMemoryAuditLogger", null) + .build()); + jmxServerOptions.jmx_encryption_options.applyConfig(); jmxServer = JMXServerUtils.createJMXServer(jmxServerOptions, "localhost"); jmxServer.start(); @@ -210,6 +226,10 @@ private void grantPermission(Permission permission, JMXResource resource, RoleRe private void assertAuthorized(MBeanAction action) { action.execute(); + AuditLogEntry entry = nextAuditEvent(auditLogs); + Assertions.assertThat(entry.getType()).isSameAs(AuditLogEntryType.JMX); + Assertions.assertThat(entry.getUser()).contains(role.getRoleName()); + Assertions.assertThat(entry.getOperation()).contains("JMX INVOCATION"); } private void assertUnauthorized(MBeanAction action) @@ -222,9 +242,23 @@ private void assertUnauthorized(MBeanAction action) catch (SecurityException e) { assertEquals("Access Denied", e.getLocalizedMessage()); + AuditLogEntry entry = nextAuditEvent(auditLogs); + Assertions.assertThat(entry.getType()).isSameAs(AuditLogEntryType.JMX); + Assertions.assertThat(entry.getUser()).contains(role.getRoleName()); + Assertions.assertThat(entry.getOperation()).contains("JMX FAILURE"); + Assertions.assertThat(entry.getOperation()).contains(e.getLocalizedMessage()); } } + private static AuditLogEntry nextAuditEvent(Queue<AuditLogEntry> auditLogs) + { + // When creating a new proxy, classloaders are loaded over JMX, so ignore those for permission assertions + AuditLogEntry next = auditLogs.remove(); + if (next.getOperation().startsWith("JMX INVOCATION: javax.management.MBeanServer#getClassLoaderFor")) + return auditLogs.remove(); + return next; + } + private void clearAllPermissions() { ((StubAuthorizer) DatabaseDescriptor.getAuthorizer()).clear(); From ea79f7b13fb0e3da3acc61847b8451777656c92b Mon Sep 17 00:00:00 2001 From: mck <mck@apache.org> Date: Fri, 3 Jan 2025 11:48:25 +0100 Subject: [PATCH 068/225] =?UTF-8?q?ninja-fix=20=E2=80=93=205.0=20docs=20ar?= =?UTF-8?q?e=20not=20preview=20anymore?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- doc/antora.yml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/doc/antora.yml b/doc/antora.yml index 59a37abc6c76..d01016f59340 100644 --- a/doc/antora.yml +++ b/doc/antora.yml @@ -1,7 +1,6 @@ name: Cassandra version: '5.0' display_version: '5.0' -prerelease: true asciidoc: attributes: cass_url: 'http://cassandra.apache.org/' @@ -11,4 +10,4 @@ asciidoc: nav: - modules/ROOT/nav.adoc -- modules/cassandra/nav.adoc \ No newline at end of file +- modules/cassandra/nav.adoc From 4ae1aee39bfa5d1805920faa4e30f4660b49fabb Mon Sep 17 00:00:00 2001 From: Dmitry Konstantinov <konstantinov@netcracker.com> Date: Mon, 23 Dec 2024 10:34:16 +0000 Subject: [PATCH 069/225] Avoid memory allocation in NativeCell.valueSize() and NativeClustering.dataSize() patch by Dmitry Konstantinov; reviewed by Branimir Lambov, Mick Semb Wever for CASSANDRA-20162 --- CHANGES.txt | 1 + .../apache/cassandra/db/NativeClustering.java | 6 ++++++ .../apache/cassandra/db/rows/NativeCell.java | 5 +++++ .../apache/cassandra/db/NativeCellTest.java | 21 +++++++++++++++++++ 4 files changed, 33 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index b038387817d9..b76e5bed8afa 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.3 + * Avoid memory allocation in offheap_object's NativeCell.valueSize() and NativeClustering.dataSize() (CASSANDRA-20162) * Add flag to avoid invalidating key cache on sstable deletions (CASSANDRA-20068) * Interpret inet, bigint, varint, and decimal as non-reversed types for query construction and post-filtering (CASSANDRA-20100) * Fix delayed gossip shutdown messages clobbering startup states that leave restarted nodes appearing down (CASSANDRA-20033) diff --git a/src/java/org/apache/cassandra/db/NativeClustering.java b/src/java/org/apache/cassandra/db/NativeClustering.java index f3a838047036..e7c7e8893a17 100644 --- a/src/java/org/apache/cassandra/db/NativeClustering.java +++ b/src/java/org/apache/cassandra/db/NativeClustering.java @@ -93,6 +93,12 @@ public int size() return MemoryUtil.getShort(peer); } + public int dataSize() + { + int dataSizeOffset = (size() * 2) + 2; // metadataSize - 2 + return MemoryUtil.getShort(peer + dataSizeOffset); + } + public ByteBuffer get(int i) { // offset at which we store the dataOffset diff --git a/src/java/org/apache/cassandra/db/rows/NativeCell.java b/src/java/org/apache/cassandra/db/rows/NativeCell.java index a876e7d59577..b0613f33f6da 100644 --- a/src/java/org/apache/cassandra/db/rows/NativeCell.java +++ b/src/java/org/apache/cassandra/db/rows/NativeCell.java @@ -145,6 +145,11 @@ public ValueAccessor<ByteBuffer> accessor() return ByteBufferAccessor.instance; // FIXME: add native accessor } + public int valueSize() + { + return MemoryUtil.getInt(peer + LENGTH); + } + public CellPath path() { if (!hasPath()) diff --git a/test/unit/org/apache/cassandra/db/NativeCellTest.java b/test/unit/org/apache/cassandra/db/NativeCellTest.java index d93dea42fecc..cf2ba4ed0dd1 100644 --- a/test/unit/org/apache/cassandra/db/NativeCellTest.java +++ b/test/unit/org/apache/cassandra/db/NativeCellTest.java @@ -18,6 +18,7 @@ package org.apache.cassandra.db; import java.nio.ByteBuffer; +import java.util.Iterator; import java.util.Random; import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; @@ -163,9 +164,29 @@ private static void test(Row row) Assert.assertEquals(row.clustering(), brow.clustering()); Assert.assertEquals(nrow.clustering(), brow.clustering()); + Assert.assertEquals(row.clustering().dataSize(), nrow.clustering().dataSize()); + Assert.assertEquals(row.clustering().dataSize(), brow.clustering().dataSize()); + ClusteringComparator comparator = new ClusteringComparator(UTF8Type.instance); Assert.assertEquals(0, comparator.compare(row.clustering(), nrow.clustering())); Assert.assertEquals(0, comparator.compare(row.clustering(), brow.clustering())); Assert.assertEquals(0, comparator.compare(nrow.clustering(), brow.clustering())); + + assertCellsDataSize(row, nrow); + assertCellsDataSize(row, brow); + } + + private static void assertCellsDataSize(Row row1, Row row2) + { + Iterator<Cell<?>> row1Iterator = row1.cells().iterator(); + Iterator<Cell<?>> row2Iterator = row2.cells().iterator(); + while (row1Iterator.hasNext()) + { + Cell cell1 = row1Iterator.next(); + Cell cell2 = row2Iterator.next(); + Assert.assertEquals(cell1.dataSize(), cell2.dataSize()); + } + } + } From 3cdd540bfead9d1c1b119c3234f08f4e6be15c27 Mon Sep 17 00:00:00 2001 From: mck <mck@apache.org> Date: Fri, 20 Dec 2024 19:10:37 +1100 Subject: [PATCH 070/225] Timing info for each test split in Jenkinsfile patch by Mick Semb Wever; reviewed by Brandon Williams for CASSANDRA-20157 --- .jenkins/Jenkinsfile | 24 ++++++++++++++++++------ 1 file changed, 18 insertions(+), 6 deletions(-) diff --git a/.jenkins/Jenkinsfile b/.jenkins/Jenkinsfile index 4f8ad6c26899..d7c84eba8050 100644 --- a/.jenkins/Jenkinsfile +++ b/.jenkins/Jenkinsfile @@ -157,6 +157,11 @@ def tasks() { } def testSteps = [ + // Each splits size need to be high enough to avoid the one hour per split timeout, + // and low enough so test time is factors more than the setup+build time in each split. + // Splits can also be poorly balanced: splitting or renaming test classes is the best tactic. + // On unsaturated systems 10 minutes per split is optimal, higher with saturation + // (some buffer on the heaviest split under the 1h max is required, ref `timeout(…)` in `test(…)`) 'cqlsh-test': [splits: 1], 'fqltool-test': [splits: 1, size: 'small'], 'test-cdc': [splits: 8], @@ -383,13 +388,20 @@ def test(command, cell) { } script_vars = fetchDTestsSource(command, script_vars) timeout(time: 1, unit: 'HOURS') { // best throughput with each cell at ~10 minutes - buildJVMDTestJars(cell, script_vars, logfile) - def status = sh label: "RUNNING TESTS ${cell.step}...", script: "${script_vars} .build/docker/run-tests.sh -a ${cell.step} -c '${cell.split}/${splits}' -j ${cell.jdk} 2>&1 | tee >( xz -c > build/${logfile} )", returnStatus: true - dir("build") { - archiveArtifacts artifacts: "${logfile}", fingerprint: true - copyToNightlies("${logfile}", "${cell.step}/${cell.arch}/jdk${cell.jdk}/python${cell.python}/cython_${cell.cython}/" + "split_${cell.split}_${splits}".replace("/", "_")) + def timer = System.currentTimeMillis() + try { + buildJVMDTestJars(cell, script_vars, logfile) + def status = sh label: "RUNNING TESTS ${cell.step}...", script: "${script_vars} .build/docker/run-tests.sh -a ${cell.step} -c '${cell.split}/${splits}' -j ${cell.jdk} 2>&1 | tee >( xz -c > build/${logfile} )", returnStatus: true + dir("build") { + archiveArtifacts artifacts: "${logfile}", fingerprint: true + copyToNightlies("${logfile}", "${cell.step}/${cell.arch}/jdk${cell.jdk}/python${cell.python}/cython_${cell.cython}/" + "split_${cell.split}_${splits}".replace("/", "_")) + } + if (0 != status) { error("Stage ${cell.step}${cell_suffix} failed with exit status ${status}") } + } finally { + def duration = System.currentTimeMillis() - timer + def formattedTime = String.format("%tT.%tL", duration, duration) + echo "Time ${cell.step}${cell_suffix}: ${formattedTime}" } - if (0 != status) { error("Stage ${cell.step}${cell_suffix} failed with exit status ${status}") } } dir("build") { sh """ From 4fc8bb29fcda935728d8863a4499fa0e9d924b82 Mon Sep 17 00:00:00 2001 From: Sunil Ramchandra Pawar <pawar_sr@apple.com> Date: Tue, 7 Jan 2025 08:58:48 -0800 Subject: [PATCH 071/225] IndexOutOfBoundsException when accessing partition where the column was deleted patch by Sunil Ramchandra Pawar; reviewed by Caleb Rackliffe, David Capwell for CASSANDRA-20108 --- CHANGES.txt | 1 + .../apache/cassandra/db/filter/RowFilter.java | 32 ++++++++++--------- .../validation/operations/SelectTest.java | 30 +++++++++++++++++ .../index/sasi/plan/OperationTest.java | 2 +- 4 files changed, 49 insertions(+), 16 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 23dcc154f8e6..7eb0068d7960 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,5 +1,6 @@ 4.0.16 * Fix gossip issue with gossip-only and bootstrapping nodes missing DC/Rack/Host ID endpoint state (CASSANDRA-19983) + * IndexOutOfBoundsException when accessing partition where the column was deleted (CASSANDRA-20108) 4.0.15 diff --git a/src/java/org/apache/cassandra/db/filter/RowFilter.java b/src/java/org/apache/cassandra/db/filter/RowFilter.java index 68a1d57fe1ac..69a57918cfcb 100644 --- a/src/java/org/apache/cassandra/db/filter/RowFilter.java +++ b/src/java/org/apache/cassandra/db/filter/RowFilter.java @@ -175,7 +175,7 @@ public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, for (Expression e : expressions) { - if (!e.isSatisfiedBy(metadata, partitionKey, purged)) + if (!e.isSatisfiedBy(metadata, partitionKey, purged, nowInSec)) return false; } return true; @@ -301,7 +301,7 @@ protected BaseRowIterator<?> applyToPartition(BaseRowIterator<?> partition) // Short-circuit all partitions that won't match based on static and partition keys for (Expression e : partitionLevelExpressions) - if (!e.isSatisfiedBy(metadata, partition.partitionKey(), partition.staticRow())) + if (!e.isSatisfiedBy(metadata, partition.partitionKey(), partition.staticRow(), nowInSec)) { partition.close(); return null; @@ -327,7 +327,7 @@ public Row applyToRow(Row row) return null; for (Expression e : rowLevelExpressions) - if (!e.isSatisfiedBy(metadata, pk, purged)) + if (!e.isSatisfiedBy(metadata, pk, purged, nowInSec)) return null; return row; @@ -437,9 +437,9 @@ public void validateForIndexing() * (i.e. it should come from a RowIterator). * @return whether the row is satisfied by this expression. */ - public abstract boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, Row row); + public abstract boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, Row row, int nowInSec); - protected ByteBuffer getValue(TableMetadata metadata, DecoratedKey partitionKey, Row row) + protected ByteBuffer getValue(TableMetadata metadata, DecoratedKey partitionKey, Row row, int nowInSec) { switch (column.kind) { @@ -451,7 +451,7 @@ protected ByteBuffer getValue(TableMetadata metadata, DecoratedKey partitionKey, return row.clustering().bufferAt(column.position()); default: Cell<?> cell = row.getCell(column); - return cell == null ? null : cell.buffer(); + return cell == null || cell.isTombstone() || !cell.isLive(nowInSec) ? null : cell.buffer(); } } @@ -594,7 +594,7 @@ public static class SimpleExpression extends Expression super(column, operator, value); } - public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, Row row) + public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, Row row, int nowInSec) { // We support null conditions for LWT (in ColumnCondition) but not for RowFilter. // TODO: we should try to merge both code someday. @@ -614,7 +614,7 @@ public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, // representation. See CASSANDRA-11629 if (column.type.isCounter()) { - ByteBuffer foundValue = getValue(metadata, partitionKey, row); + ByteBuffer foundValue = getValue(metadata, partitionKey, row, nowInSec); if (foundValue == null) return false; @@ -624,7 +624,7 @@ public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, else { // Note that CQL expression are always of the form 'x < 4', i.e. the tested value is on the left. - ByteBuffer foundValue = getValue(metadata, partitionKey, row); + ByteBuffer foundValue = getValue(metadata, partitionKey, row, nowInSec); return foundValue != null && operator.isSatisfiedBy(column.type, foundValue, value); } } @@ -635,7 +635,7 @@ public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, case LIKE_MATCHES: { assert !column.isComplex() : "Only CONTAINS and CONTAINS_KEY are supported for 'complex' types"; - ByteBuffer foundValue = getValue(metadata, partitionKey, row); + ByteBuffer foundValue = getValue(metadata, partitionKey, row, nowInSec); // Note that CQL expression are always of the form 'x < 4', i.e. the tested value is on the left. return foundValue != null && operator.isSatisfiedBy(column.type, foundValue, value); } @@ -665,7 +665,7 @@ public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, } else { - ByteBuffer foundValue = getValue(metadata, partitionKey, row); + ByteBuffer foundValue = getValue(metadata, partitionKey, row, nowInSec); if (foundValue == null) return false; @@ -692,7 +692,7 @@ public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, } else { - ByteBuffer foundValue = getValue(metadata, partitionKey, row); + ByteBuffer foundValue = getValue(metadata, partitionKey, row, nowInSec); return foundValue != null && mapType.getSerializer().getSerializedValue(foundValue, value, mapType.getKeysType()) != null; } @@ -765,7 +765,8 @@ public ByteBuffer getIndexValue() return CompositeType.build(ByteBufferAccessor.instance, key, value); } - public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, Row row) + @Override + public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, Row row, int nowInSec) { assert key != null; // We support null conditions for LWT (in ColumnCondition) but not for RowFilter. @@ -783,7 +784,7 @@ public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, } else { - ByteBuffer serializedMap = getValue(metadata, partitionKey, row); + ByteBuffer serializedMap = getValue(metadata, partitionKey, row, nowInSec); if (serializedMap == null) return false; @@ -879,7 +880,8 @@ protected Kind kind() } // Filtering by custom expressions isn't supported yet, so just accept any row - public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, Row row) + @Override + public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, Row row, int nowInSec) { return true; } diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java index d0493cf8fc75..0a7b507df109 100644 --- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java +++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java @@ -2301,6 +2301,36 @@ public void filteringOnStaticColumnTest() throws Throwable }); } + @Test + public void filteringOnDeletedStaticColumnValue() throws Throwable + { + // Create table with int-only columns + createTable("CREATE TABLE %s (pk0 int, pk1 int, ck0 int, ck1 int, s0 tinyint static, v0 int, v1 int, PRIMARY KEY ((pk0, pk1), ck0, ck1))"); + + // Insert rows + execute("INSERT INTO %s (pk0, pk1, s0, ck0, ck1, v0, v1) VALUES (?, ?, ?, ?, ?, ?, ?)", 1000, 2000, (byte) 126, 100, 1, 20, 30); + execute("INSERT INTO %s (pk0, pk1, s0, ck0, ck1, v0, v1) VALUES (?, ?, ?, ?, ?, ?, ?)", 1000, 2000, (byte) 125, 200, 2, 40, 50); + execute("INSERT INTO %s (pk0, pk1, s0, ck0, ck1, v0, v1) VALUES (?, ?, ?, ?, ?, ?, ?)", 1000, 3000, (byte) 122, 300, 3, 60, 70); + execute("DELETE s0,v0,v1 FROM %s WHERE pk0=1000 AND pk1=2000 and ck0=100 and ck1=1"); + + beforeAndAfterFlush(() -> { + // Verify the columns are deleted + assertRows(execute("SELECT pk0, pk1, s0, ck0, ck1, v0, v1 FROM %s WHERE s0=? ALLOW FILTERING", (byte) 122), + row(1000, 3000, (byte) 122, 300, 3, 60, 70)); + }); + + execute("DELETE v0 FROM %s WHERE pk0=1000 AND pk1=3000 AND ck0=300 AND ck1=3"); + + beforeAndAfterFlush(() -> { + assertRows(execute("SELECT pk0, pk1, s0, ck0, ck1, v0, v1 FROM %s WHERE s0=? ALLOW FILTERING", (byte) 122), + row(1000, 3000, (byte) 122, 300, 3, null, 70)); + + assertRows(execute("SELECT pk0, pk1, s0, ck0, ck1, v0, v1 FROM %s WHERE pk0=1000 AND pk1=3000 AND ck0=300 AND ck1=3"), + row(1000, 3000, (byte) 122, 300, 3, null, 70)); + }); + + } + @Test public void containsFilteringOnNonClusteringColumn() throws Throwable { createTable("CREATE TABLE %s (a int, b int, c int, d list<int>, PRIMARY KEY (a, b, c))"); diff --git a/test/unit/org/apache/cassandra/index/sasi/plan/OperationTest.java b/test/unit/org/apache/cassandra/index/sasi/plan/OperationTest.java index 4468f2c2198e..e80e68fb76f4 100644 --- a/test/unit/org/apache/cassandra/index/sasi/plan/OperationTest.java +++ b/test/unit/org/apache/cassandra/index/sasi/plan/OperationTest.java @@ -647,7 +647,7 @@ protected Kind kind() } @Override - public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, Row row) + public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, Row row, int nowInSec) { throw new UnsupportedOperationException(); } From b074675c37b2d76ef18f921c7e29f013fc35e011 Mon Sep 17 00:00:00 2001 From: Brandon Williams <brandonwilliams@apache.org> Date: Fri, 17 May 2024 14:28:31 -0500 Subject: [PATCH 072/225] Add in-jvm dtest for durable writes Patch by brandonwilliams; reviewed by smiklosovic for CASSANDRA-19601 --- .../distributed/test/DurableWritesTest.java | 59 +++++++++++++++++++ 1 file changed, 59 insertions(+) create mode 100644 test/distributed/org/apache/cassandra/distributed/test/DurableWritesTest.java diff --git a/test/distributed/org/apache/cassandra/distributed/test/DurableWritesTest.java b/test/distributed/org/apache/cassandra/distributed/test/DurableWritesTest.java new file mode 100644 index 000000000000..b3debad19525 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/DurableWritesTest.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import org.junit.Test; +import org.junit.Assert; + +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.commitlog.CommitLog; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.schema.TableId; + +public class DurableWritesTest extends TestBaseImpl +{ + @Test + public void durableWritesDisabledTest() throws Throwable + { + try (Cluster cluster = init(Cluster.build(1) + .withConfig(c -> c.set("commitlog_segment_size_in_mb", 1) + ) + .start())) + { + cluster.schemaChange(withKeyspace("DROP KEYSPACE %s")); + cluster.schemaChange(withKeyspace("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1} AND DURABLE_WRITES = false")); + cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (k INT PRIMARY KEY, a INT, b INT, c INT)")); + + for (int i = 1; i <= 1000; i++) + { + cluster.get(1).executeInternal(withKeyspace("INSERT INTO %s.tbl (k, a, b, c) VALUES (1, 1, 1, 1)")); + } + + cluster.get(1).runOnInstance(() -> { + TableId wanted = TableId.fromString(Keyspace.open(KEYSPACE).getColumnFamilyStore("tbl").metadata.id.toString()); + boolean containsTbl = CommitLog.instance.segmentManager + .getActiveSegments() + .stream() + .anyMatch(s -> s.getDirtyTableIds().contains(wanted)); + Assert.assertFalse("Commitlog should not contain data from 'tbl'", containsTbl); + }); + } + } + +} From f41c2e29d240a7f0cafe6455722d287edc80b1be Mon Sep 17 00:00:00 2001 From: Ke Han <kehan5800@gmail.com> Date: Wed, 25 Oct 2023 01:25:56 +0000 Subject: [PATCH 073/225] Fix incorrect column identifier bytes problem when renaming a column patch by Ke Han; reviewed by Stefan Miklosovic and Brandon Williams for CASSANDRA-18956 --- CHANGES.txt | 1 + src/java/org/apache/cassandra/cql3/Cql.g | 12 +- .../cql3/statements/AlterTableStatement.java | 10 +- .../schema/LegacySchemaMigratorBaseTest.java | 816 ++++++++++++++++++ .../schema/LegacySchemaMigratorTest.java | 742 +--------------- .../LegacySchemaMigratorThriftTest.java | 90 ++ 6 files changed, 929 insertions(+), 742 deletions(-) create mode 100644 test/unit/org/apache/cassandra/schema/LegacySchemaMigratorBaseTest.java create mode 100644 test/unit/org/apache/cassandra/schema/LegacySchemaMigratorThriftTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 7ce9a425071c..9804496fcb6c 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.31 + * Fix incorrect column identifier bytes problem when renaming a column (CASSANDRA-18965) * Upgrade OWASP to 10.0.0 (CASSANDRA-19738) Merged from 2.2: * Add termin-8-jdk as a valid jdk8 candidate in the debian package (CASSANDRA-19752) diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g index 02343274b5cd..2d10c950662f 100644 --- a/src/java/org/apache/cassandra/cql3/Cql.g +++ b/src/java/org/apache/cassandra/cql3/Cql.g @@ -816,7 +816,7 @@ alterTableStatement returns [AlterTableStatement expr] @init { AlterTableStatement.Type type = null; TableAttributes attrs = new TableAttributes(); - Map<ColumnIdentifier.Raw, ColumnIdentifier> renames = new HashMap<ColumnIdentifier.Raw, ColumnIdentifier>(); + Map<ColumnIdentifier.Raw, ColumnIdentifier.Raw> renames = new HashMap<ColumnIdentifier.Raw, ColumnIdentifier.Raw>(); boolean isStatic = false; Long dropTimestamp = null; } @@ -829,8 +829,8 @@ alterTableStatement returns [AlterTableStatement expr] | K_DROP K_COMPACT K_STORAGE { type = AlterTableStatement.Type.DROP_COMPACT_STORAGE; } | K_WITH properties[attrs] { type = AlterTableStatement.Type.OPTS; } | K_RENAME { type = AlterTableStatement.Type.RENAME; } - id1=cident K_TO toId1=ident { renames.put(id1, toId1); } - ( K_AND idn=cident K_TO toIdn=ident { renames.put(idn, toIdn); } )* + id1=schema_cident K_TO toId1=schema_cident { renames.put(id1, toId1); } + ( K_AND idn=schema_cident K_TO toIdn=schema_cident { renames.put(idn, toIdn); } )* ) { $expr = new AlterTableStatement(cf, type, id, v, attrs, renames, isStatic, dropTimestamp); @@ -1180,6 +1180,12 @@ cident returns [ColumnIdentifier.Raw id] | EMPTY_QUOTED_NAME { $id = new ColumnIdentifier.Literal("", false); } ; +schema_cident returns [ColumnIdentifier.Raw id] + : t=IDENT { $id = new ColumnIdentifier.Literal($t.text, false); } + | t=QUOTED_NAME { $id = new ColumnIdentifier.Literal($t.text, true); } + | k=unreserved_keyword { $id = new ColumnIdentifier.Literal(k, false); } + ; + // Column identifiers where the comparator is known to be text ident returns [ColumnIdentifier id] : t=IDENT { $id = ColumnIdentifier.getInterned($t.text, false); } diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java index cd1041e6c1bb..2f516f784729 100644 --- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java @@ -71,7 +71,7 @@ public enum Type public final CQL3Type.Raw validator; public final ColumnIdentifier.Raw rawColumnName; private final TableAttributes attrs; - private final Map<ColumnIdentifier.Raw, ColumnIdentifier> renames; + private final Map<ColumnIdentifier.Raw, ColumnIdentifier.Raw> renames; private final boolean isStatic; // Only for ALTER ADD private final Long deleteTimestamp; @@ -80,7 +80,7 @@ public AlterTableStatement(CFName name, ColumnIdentifier.Raw columnName, CQL3Type.Raw validator, TableAttributes attrs, - Map<ColumnIdentifier.Raw, ColumnIdentifier> renames, + Map<ColumnIdentifier.Raw, ColumnIdentifier.Raw> renames, boolean isStatic, Long deleteTimestamp) { @@ -328,10 +328,10 @@ public Event.SchemaChange announceMigration(QueryState queryState, boolean isLoc case RENAME: cfm = meta.copy(); - for (Map.Entry<ColumnIdentifier.Raw, ColumnIdentifier> entry : renames.entrySet()) + for (Map.Entry<ColumnIdentifier.Raw, ColumnIdentifier.Raw> entry : renames.entrySet()) { ColumnIdentifier from = entry.getKey().prepare(cfm); - ColumnIdentifier to = entry.getValue(); + ColumnIdentifier to = entry.getValue().prepare(cfm); cfm.renameColumn(from, to); @@ -342,7 +342,7 @@ public Event.SchemaChange announceMigration(QueryState queryState, boolean isLoc ViewDefinition viewCopy = view.copy(); ColumnIdentifier viewFrom = entry.getKey().prepare(viewCopy.metadata); - ColumnIdentifier viewTo = entry.getValue(); + ColumnIdentifier viewTo = entry.getValue().prepare(viewCopy.metadata); viewCopy.renameColumn(viewFrom, viewTo); if (viewUpdates == null) diff --git a/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorBaseTest.java b/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorBaseTest.java new file mode 100644 index 000000000000..49d99f84759c --- /dev/null +++ b/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorBaseTest.java @@ -0,0 +1,816 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.schema; + +import java.nio.ByteBuffer; +import java.util.*; +import java.util.stream.Collectors; + +import com.google.common.collect.ImmutableList; + +import org.junit.Ignore; + +import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.config.Schema; +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.functions.*; +import org.apache.cassandra.db.*; +import org.apache.cassandra.db.marshal.*; +import org.apache.cassandra.index.internal.CassandraIndex; +import org.apache.cassandra.thrift.ThriftConversion; +import org.apache.cassandra.utils.*; + +import static java.lang.String.format; +import static junit.framework.Assert.assertFalse; +import static junit.framework.Assert.assertTrue; +import static org.apache.cassandra.utils.ByteBufferUtil.bytes; +import static org.apache.cassandra.utils.FBUtilities.json; + +@SuppressWarnings("deprecation") +@Ignore +public abstract class LegacySchemaMigratorBaseTest +{ + private static final long TIMESTAMP = 1435908994000000L; + + private static final String KEYSPACE_PREFIX = "LegacySchemaMigratorTest"; + + public static void loadLegacySchemaTables() + { + KeyspaceMetadata systemKeyspace = Schema.instance.getKSMetaData(SystemKeyspace.NAME); + + Tables systemTables = systemKeyspace.tables; + for (CFMetaData table : LegacySchemaMigrator.LegacySchemaTables) + systemTables = systemTables.with(table); + + LegacySchemaMigrator.LegacySchemaTables.forEach(Schema.instance::load); + + Schema.instance.setKeyspaceMetadata(systemKeyspace.withSwapped(systemTables)); + } + + public static Keyspaces keyspacesToMigrate() + { + Keyspaces.Builder keyspaces = Keyspaces.builder(); + + // A whole bucket of shorthand + String ks1 = KEYSPACE_PREFIX + "Keyspace1"; + String ks2 = KEYSPACE_PREFIX + "Keyspace2"; + String ks3 = KEYSPACE_PREFIX + "Keyspace3"; + String ks4 = KEYSPACE_PREFIX + "Keyspace4"; + String ks5 = KEYSPACE_PREFIX + "Keyspace5"; + String ks6 = KEYSPACE_PREFIX + "Keyspace6"; + String ks_rcs = KEYSPACE_PREFIX + "RowCacheSpace"; + String ks_nocommit = KEYSPACE_PREFIX + "NoCommitlogSpace"; + String ks_prsi = KEYSPACE_PREFIX + "PerRowSecondaryIndex"; + String ks_cql = KEYSPACE_PREFIX + "cql_keyspace"; + + // Make it easy to test compaction + Map<String, String> compactionOptions = new HashMap<>(); + compactionOptions.put("tombstone_compaction_interval", "1"); + + Map<String, String> leveledOptions = new HashMap<>(); + leveledOptions.put("sstable_size_in_mb", "1"); + + keyspaces.add(KeyspaceMetadata.create(ks1, + KeyspaceParams.simple(1), + Tables.of(SchemaLoader.standardCFMD(ks1, "Standard1") + .compaction(CompactionParams.scts(compactionOptions)), + SchemaLoader.standardCFMD(ks1, "StandardGCGS0").gcGraceSeconds(0), + SchemaLoader.standardCFMD(ks1, "StandardLong1"), + SchemaLoader.keysIndexCFMD(ks1, "Indexed1", true), + SchemaLoader.keysIndexCFMD(ks1, "Indexed2", false), + SchemaLoader.jdbcCFMD(ks1, "JdbcUtf8", UTF8Type.instance) + .addColumnDefinition(SchemaLoader.utf8Column(ks1, "JdbcUtf8")), + SchemaLoader.jdbcCFMD(ks1, "JdbcLong", LongType.instance), + SchemaLoader.jdbcCFMD(ks1, "JdbcBytes", BytesType.instance), + SchemaLoader.jdbcCFMD(ks1, "JdbcAscii", AsciiType.instance), + SchemaLoader.standardCFMD(ks1, "StandardLeveled") + .compaction(CompactionParams.lcs(leveledOptions)), + SchemaLoader.standardCFMD(ks1, "legacyleveled") + .compaction(CompactionParams.lcs(leveledOptions)), + SchemaLoader.standardCFMD(ks1, "StandardLowIndexInterval") + .minIndexInterval(8) + .maxIndexInterval(256) + .caching(CachingParams.CACHE_NOTHING)))); + + // Keyspace 2 + keyspaces.add(KeyspaceMetadata.create(ks2, + KeyspaceParams.simple(1), + Tables.of(SchemaLoader.standardCFMD(ks2, "Standard1"), + SchemaLoader.keysIndexCFMD(ks2, "Indexed1", true), + SchemaLoader.compositeIndexCFMD(ks2, "Indexed2", true), + SchemaLoader.compositeIndexCFMD(ks2, "Indexed3", true) + .gcGraceSeconds(0)))); + + // Keyspace 3 + keyspaces.add(KeyspaceMetadata.create(ks3, + KeyspaceParams.simple(5), + Tables.of(SchemaLoader.standardCFMD(ks3, "Standard1"), + SchemaLoader.keysIndexCFMD(ks3, "Indexed1", true)))); + + // Keyspace 4 + keyspaces.add(KeyspaceMetadata.create(ks4, + KeyspaceParams.simple(3), + Tables.of(SchemaLoader.standardCFMD(ks4, "Standard1")))); + + // Keyspace 5 + keyspaces.add(KeyspaceMetadata.create(ks5, + KeyspaceParams.simple(2), + Tables.of(SchemaLoader.standardCFMD(ks5, "Standard1")))); + + // Keyspace 6 + keyspaces.add(KeyspaceMetadata.create(ks6, + KeyspaceParams.simple(1), + Tables.of(SchemaLoader.keysIndexCFMD(ks6, "Indexed1", true)))); + + // RowCacheSpace + keyspaces.add(KeyspaceMetadata.create(ks_rcs, + KeyspaceParams.simple(1), + Tables.of(SchemaLoader.standardCFMD(ks_rcs, "CFWithoutCache") + .caching(CachingParams.CACHE_NOTHING), + SchemaLoader.standardCFMD(ks_rcs, "CachedCF") + .caching(CachingParams.CACHE_EVERYTHING), + SchemaLoader.standardCFMD(ks_rcs, "CachedIntCF") + .caching(new CachingParams(true, 100))))); + + keyspaces.add(KeyspaceMetadata.create(ks_nocommit, + KeyspaceParams.simpleTransient(1), + Tables.of(SchemaLoader.standardCFMD(ks_nocommit, "Standard1")))); + + // PerRowSecondaryIndexTest + keyspaces.add(KeyspaceMetadata.create(ks_prsi, + KeyspaceParams.simple(1), + Tables.of(SchemaLoader.perRowIndexedCFMD(ks_prsi, "Indexed1")))); + + // CQLKeyspace + keyspaces.add(KeyspaceMetadata.create(ks_cql, + KeyspaceParams.simple(1), + Tables.of(CFMetaData.compile("CREATE TABLE table1 (" + + "k int PRIMARY KEY," + + "v1 text," + + "v2 int" + + ')', ks_cql), + + CFMetaData.compile("CREATE TABLE table2 (" + + "k text," + + "c text," + + "v text," + + "PRIMARY KEY (k, c))", ks_cql), + + CFMetaData.compile("CREATE TABLE foo (" + + "bar text, " + + "baz text, " + + "qux text, " + + "PRIMARY KEY(bar, baz) ) " + + "WITH COMPACT STORAGE", ks_cql), + + CFMetaData.compile("CREATE TABLE compact_pkonly (" + + "k int, " + + "c int, " + + "PRIMARY KEY (k, c)) " + + "WITH COMPACT STORAGE", + ks_cql), + + CFMetaData.compile("CREATE TABLE foofoo (" + + "bar text, " + + "baz text, " + + "qux text, " + + "quz text, " + + "foo text, " + + "PRIMARY KEY((bar, baz), qux, quz) ) " + + "WITH COMPACT STORAGE", ks_cql)))); + + // NTS keyspace + keyspaces.add(KeyspaceMetadata.create("nts", KeyspaceParams.nts("dc1", 1, "dc2", 2))); + + keyspaces.add(keyspaceWithDroppedCollections()); + keyspaces.add(keyspaceWithTriggers()); + keyspaces.add(keyspaceWithUDTs()); + keyspaces.add(keyspaceWithUDFs()); + keyspaces.add(keyspaceWithUDFsAndUDTs()); + keyspaces.add(keyspaceWithUDAs()); + keyspaces.add(keyspaceWithUDAsAndUDTs()); + + return keyspaces.build(); + } + + private static KeyspaceMetadata keyspaceWithDroppedCollections() + { + String keyspace = KEYSPACE_PREFIX + "DroppedCollections"; + + CFMetaData table = + CFMetaData.compile("CREATE TABLE dropped_columns (" + + "foo text," + + "bar text," + + "map1 map<text, text>," + + "map2 map<int, int>," + + "set1 set<ascii>," + + "list1 list<blob>," + + "PRIMARY KEY ((foo), bar))", + keyspace); + + String[] collectionColumnNames = { "map1", "map2", "set1", "list1" }; + for (String name : collectionColumnNames) + { + ColumnDefinition column = table.getColumnDefinition(bytes(name)); + table.recordColumnDrop(column, FBUtilities.timestampMicros(), false); + table.removeColumnDefinition(column); + } + + return KeyspaceMetadata.create(keyspace, KeyspaceParams.simple(1), Tables.of(table)); + } + + private static KeyspaceMetadata keyspaceWithTriggers() + { + String keyspace = KEYSPACE_PREFIX + "Triggers"; + + Triggers.Builder triggers = Triggers.builder(); + CFMetaData table = SchemaLoader.standardCFMD(keyspace, "WithTriggers"); + for (int i = 0; i < 10; i++) + triggers.add(new TriggerMetadata("trigger" + i, "DummyTrigger" + i)); + table.triggers(triggers.build()); + + return KeyspaceMetadata.create(keyspace, KeyspaceParams.simple(1), Tables.of(table)); + } + + private static KeyspaceMetadata keyspaceWithUDTs() + { + String keyspace = KEYSPACE_PREFIX + "UDTs"; + + UserType udt1 = new UserType(keyspace, + bytes("udt1"), + new ArrayList<ByteBuffer>() + {{ + add(bytes("col1")); + add(bytes("col2")); + }}, + new ArrayList<AbstractType<?>>() + {{ + add(UTF8Type.instance); + add(Int32Type.instance); + }}); + + UserType udt2 = new UserType(keyspace, + bytes("udt2"), + new ArrayList<ByteBuffer>() + {{ + add(bytes("col3")); + add(bytes("col4")); + }}, + new ArrayList<AbstractType<?>>() + {{ + add(BytesType.instance); + add(BooleanType.instance); + }}); + + UserType udt3 = new UserType(keyspace, + bytes("udt3"), + new ArrayList<ByteBuffer>() + {{ + add(bytes("col5")); + }}, + new ArrayList<AbstractType<?>>() + {{ + add(AsciiType.instance); + }}); + + return KeyspaceMetadata.create(keyspace, + KeyspaceParams.simple(1), + Tables.none(), + Views.none(), + Types.of(udt1, udt2, udt3), + Functions.none()); + } + + private static KeyspaceMetadata keyspaceWithUDFs() + { + String keyspace = KEYSPACE_PREFIX + "UDFs"; + + UDFunction udf1 = UDFunction.create(new FunctionName(keyspace, "udf"), + ImmutableList.of(new ColumnIdentifier("col1", false), new ColumnIdentifier("col2", false)), + ImmutableList.of(BytesType.instance, Int32Type.instance), + LongType.instance, + false, + "java", + "return 42L;"); + + // an overload with the same name, not a typo + UDFunction udf2 = UDFunction.create(new FunctionName(keyspace, "udf"), + ImmutableList.of(new ColumnIdentifier("col3", false), new ColumnIdentifier("col4", false)), + ImmutableList.of(AsciiType.instance, LongType.instance), + Int32Type.instance, + true, + "java", + "return 42;"); + + UDFunction udf3 = UDFunction.create(new FunctionName(keyspace, "udf3"), + ImmutableList.of(new ColumnIdentifier("col4", false)), + ImmutableList.of(UTF8Type.instance), + BooleanType.instance, + false, + "java", + "return true;"); + + return KeyspaceMetadata.create(keyspace, + KeyspaceParams.simple(1), + Tables.none(), + Views.none(), + Types.none(), + Functions.of(udf1, udf2, udf3)); + } + + private static KeyspaceMetadata keyspaceWithUDAs() + { + String keyspace = KEYSPACE_PREFIX + "UDAs"; + + UDFunction udf1 = UDFunction.create(new FunctionName(keyspace, "udf1"), + ImmutableList.of(new ColumnIdentifier("col1", false), new ColumnIdentifier("col2", false)), + ImmutableList.of(Int32Type.instance, Int32Type.instance), + Int32Type.instance, + false, + "java", + "return 42;"); + + UDFunction udf2 = UDFunction.create(new FunctionName(keyspace, "udf2"), + ImmutableList.of(new ColumnIdentifier("col1", false), new ColumnIdentifier("col2", false)), + ImmutableList.of(LongType.instance, Int32Type.instance), + LongType.instance, + false, + "java", + "return 42L;"); + + UDFunction udf3 = UDFunction.create(new FunctionName(keyspace, "udf3"), + ImmutableList.of(new ColumnIdentifier("col1", false)), + ImmutableList.of(LongType.instance), + DoubleType.instance, + false, + "java", + "return 42d;"); + + Functions udfs = Functions.builder().add(udf1).add(udf2).add(udf3).build(); + + UDAggregate uda1 = UDAggregate.create(udfs, new FunctionName(keyspace, "uda1"), + ImmutableList.of(udf1.argTypes().get(1)), + udf1.returnType(), + udf1.name(), + null, + udf1.argTypes().get(0), + null + ); + + UDAggregate uda2 = UDAggregate.create(udfs, new FunctionName(keyspace, "uda2"), + ImmutableList.of(udf2.argTypes().get(1)), + udf3.returnType(), + udf2.name(), + udf3.name(), + udf2.argTypes().get(0), + LongType.instance.decompose(0L) + ); + + return KeyspaceMetadata.create(keyspace, + KeyspaceParams.simple(1), + Tables.none(), + Views.none(), + Types.none(), + Functions.of(udf1, udf2, udf3, uda1, uda2)); + } + + private static KeyspaceMetadata keyspaceWithUDFsAndUDTs() + { + String keyspace = KEYSPACE_PREFIX + "UDFUDTs"; + + UserType udt1 = new UserType(keyspace, + bytes("udt1"), + new ArrayList<ByteBuffer>() + {{ + add(bytes("col1")); + add(bytes("col2")); + }}, + new ArrayList<AbstractType<?>>() + {{ + add(UTF8Type.instance); + add(Int32Type.instance); + }}); + + UserType udt2 = new UserType(keyspace, + bytes("udt2"), + new ArrayList<ByteBuffer>() + {{ + add(bytes("col1")); + add(bytes("col2")); + }}, + new ArrayList<AbstractType<?>>() + {{ + add(ListType.getInstance(udt1, false)); + add(Int32Type.instance); + }}); + + UDFunction udf1 = UDFunction.create(new FunctionName(keyspace, "udf"), + ImmutableList.of(new ColumnIdentifier("col1", false), new ColumnIdentifier("col2", false)), + ImmutableList.of(udt1, udt2), + LongType.instance, + false, + "java", + "return 42L;"); + + // an overload with the same name, not a typo + UDFunction udf2 = UDFunction.create(new FunctionName(keyspace, "udf"), + ImmutableList.of(new ColumnIdentifier("col3", false), new ColumnIdentifier("col4", false)), + ImmutableList.of(AsciiType.instance, LongType.instance), + Int32Type.instance, + true, + "java", + "return 42;"); + + UDFunction udf3 = UDFunction.create(new FunctionName(keyspace, "udf3"), + ImmutableList.of(new ColumnIdentifier("col4", false)), + ImmutableList.of(new TupleType(Arrays.asList(udt1, udt2))), + BooleanType.instance, + false, + "java", + "return true;"); + + return KeyspaceMetadata.create(keyspace, + KeyspaceParams.simple(1), + Tables.none(), + Views.none(), + Types.of(udt1, udt2), + Functions.of(udf1, udf2, udf3)); + } + + private static KeyspaceMetadata keyspaceWithUDAsAndUDTs() + { + String keyspace = KEYSPACE_PREFIX + "UDAUDTs"; + + UserType udt1 = new UserType(keyspace, + bytes("udt1"), + new ArrayList<ByteBuffer>() + {{ + add(bytes("col1")); + add(bytes("col2")); + }}, + new ArrayList<AbstractType<?>>() + {{ + add(UTF8Type.instance); + add(Int32Type.instance); + }}); + + UserType udt2 = new UserType(keyspace, + bytes("udt2"), + new ArrayList<ByteBuffer>() + {{ + add(bytes("col1")); + add(bytes("col2")); + }}, + new ArrayList<AbstractType<?>>() + {{ + add(ListType.getInstance(udt1, false)); + add(Int32Type.instance); + }}); + + UDFunction udf1 = UDFunction.create(new FunctionName(keyspace, "udf1"), + ImmutableList.of(new ColumnIdentifier("col1", false), new ColumnIdentifier("col2", false)), + ImmutableList.of(udt1, udt2), + udt1, + false, + "java", + "return null;"); + + UDFunction udf2 = UDFunction.create(new FunctionName(keyspace, "udf2"), + ImmutableList.of(new ColumnIdentifier("col1", false), new ColumnIdentifier("col2", false)), + ImmutableList.of(udt2, udt1), + udt2, + false, + "java", + "return null;"); + + UDFunction udf3 = UDFunction.create(new FunctionName(keyspace, "udf3"), + ImmutableList.of(new ColumnIdentifier("col1", false)), + ImmutableList.of(udt2), + DoubleType.instance, + false, + "java", + "return 42d;"); + + Functions udfs = Functions.builder().add(udf1).add(udf2).add(udf3).build(); + + UDAggregate uda1 = UDAggregate.create(udfs, new FunctionName(keyspace, "uda1"), + ImmutableList.of(udf1.argTypes().get(1)), + udf1.returnType(), + udf1.name(), + null, + udf1.argTypes().get(0), + null + ); + + ByteBuffer twoNullEntries = ByteBuffer.allocate(8); + twoNullEntries.putInt(-1); + twoNullEntries.putInt(-1); + twoNullEntries.flip(); + UDAggregate uda2 = UDAggregate.create(udfs, new FunctionName(keyspace, "uda2"), + ImmutableList.of(udf2.argTypes().get(1)), + udf3.returnType(), + udf2.name(), + udf3.name(), + udf2.argTypes().get(0), + twoNullEntries + ); + + return KeyspaceMetadata.create(keyspace, + KeyspaceParams.simple(1), + Tables.none(), + Views.none(), + Types.of(udt1, udt2), + Functions.of(udf1, udf2, udf3, uda1, uda2)); + } + + /* + * Serializing keyspaces + */ + + public void legacySerializeKeyspace(KeyspaceMetadata keyspace) + { + makeLegacyCreateKeyspaceMutation(keyspace, TIMESTAMP).apply(); + setLegacyIndexStatus(keyspace); + } + + public Mutation makeLegacyCreateKeyspaceMutation(KeyspaceMetadata keyspace, long timestamp) + { + // Note that because Keyspaces is a COMPACT TABLE, we're really only setting static columns internally and shouldn't set any clustering. + RowUpdateBuilder adder = new RowUpdateBuilder(SystemKeyspace.LegacyKeyspaces, timestamp, keyspace.name); + + adder.add("durable_writes", keyspace.params.durableWrites) + .add("strategy_class", keyspace.params.replication.klass.getName()) + .add("strategy_options", json(keyspace.params.replication.options)); + + Mutation mutation = adder.build(); + + keyspace.tables.forEach(table -> addTableToSchemaMutation(table, timestamp, true, mutation)); + keyspace.types.forEach(type -> addTypeToSchemaMutation(type, timestamp, mutation)); + keyspace.functions.udfs().forEach(udf -> addFunctionToSchemaMutation(udf, timestamp, mutation)); + keyspace.functions.udas().forEach(uda -> addAggregateToSchemaMutation(uda, timestamp, mutation)); + + return mutation; + } + + /* + * Serializing tables + */ + + public void addTableToSchemaMutation(CFMetaData table, long timestamp, boolean withColumnsAndTriggers, Mutation mutation) + { + // For property that can be null (and can be changed), we insert tombstones, to make sure + // we don't keep a property the user has removed + RowUpdateBuilder adder = new RowUpdateBuilder(SystemKeyspace.LegacyColumnfamilies, timestamp, mutation) + .clustering(table.cfName); + + adder.add("cf_id", table.cfId) + .add("type", table.isSuper() ? "Super" : "Standard"); + + if (table.isSuper()) + { + adder.add("comparator", table.comparator.subtype(0).toString()) + .add("subcomparator", ((MapType) table.compactValueColumn().type).getKeysType().toString()); + } + else + { + adder.add("comparator", LegacyLayout.makeLegacyComparator(table).toString()); + } + + adder.add("bloom_filter_fp_chance", table.params.bloomFilterFpChance) + .add("caching", cachingToString(table.params.caching)) + .add("comment", table.params.comment) + .add("compaction_strategy_class", table.params.compaction.klass().getName()) + .add("compaction_strategy_options", json(table.params.compaction.options())) + .add("compression_parameters", json(ThriftConversion.compressionParametersToThrift(table.params.compression))) + .add("default_time_to_live", table.params.defaultTimeToLive) + .add("gc_grace_seconds", table.params.gcGraceSeconds) + .add("key_validator", table.getKeyValidator().toString()) + .add("local_read_repair_chance", table.params.dcLocalReadRepairChance) + .add("max_compaction_threshold", table.params.compaction.maxCompactionThreshold()) + .add("max_index_interval", table.params.maxIndexInterval) + .add("memtable_flush_period_in_ms", table.params.memtableFlushPeriodInMs) + .add("min_compaction_threshold", table.params.compaction.minCompactionThreshold()) + .add("min_index_interval", table.params.minIndexInterval) + .add("read_repair_chance", table.params.readRepairChance) + .add("speculative_retry", table.params.speculativeRetry.toString()); + + for (Map.Entry<ByteBuffer, CFMetaData.DroppedColumn> entry : table.getDroppedColumns().entrySet()) + { + String name = UTF8Type.instance.getString(entry.getKey()); + CFMetaData.DroppedColumn column = entry.getValue(); + adder.addMapEntry("dropped_columns", name, column.droppedTime); + } + + adder.add("is_dense", table.isDense()); + + adder.add("default_validator", table.makeLegacyDefaultValidator().toString()); + + if (withColumnsAndTriggers) + { + for (ColumnDefinition column : table.allColumns()) + addColumnToSchemaMutation(table, column, timestamp, mutation); + + for (TriggerMetadata trigger : table.getTriggers()) + addTriggerToSchemaMutation(table, trigger, timestamp, mutation); + } + + adder.build(); + } + + private static String cachingToString(CachingParams caching) + { + return format("{\"keys\":\"%s\", \"rows_per_partition\":\"%s\"}", + caching.keysAsString(), + caching.rowsPerPartitionAsString()); + } + + public void addColumnToSchemaMutation(CFMetaData table, ColumnDefinition column, long timestamp, Mutation mutation) + { + // We need to special case pk-only dense tables. See CASSANDRA-9874. + String name = table.isDense() && column.kind == ColumnDefinition.Kind.REGULAR && column.type instanceof EmptyType + ? "" + : column.name.toString(); + + final RowUpdateBuilder adder = new RowUpdateBuilder(SystemKeyspace.LegacyColumns, timestamp, mutation).clustering(table.cfName, name); + adder.add("validator", column.type.toString()) + .add("type", serializeKind(column.kind, table.isDense())) + .add("component_index", column.position()); + Optional<IndexMetadata> index = findIndexForColumn(table.getIndexes(), table, column); + if (index.isPresent()) + { + IndexMetadata i = index.get(); + adder.add("index_name", i.name); + adder.add("index_type", i.kind.toString()); + adder.add("index_options", json(i.options)); + } + else + { + adder.add("index_name", null); + adder.add("index_type", null); + adder.add("index_options", null); + } + + adder.build(); + } + + private static Optional<IndexMetadata> findIndexForColumn(Indexes indexes, + CFMetaData table, + ColumnDefinition column) + { + // makes the assumptions that the string option denoting the + // index targets can be parsed by CassandraIndex.parseTarget + // which should be true for any pre-3.0 index + for (IndexMetadata index : indexes) + if (CassandraIndex.parseTarget(table, index).left.equals(column)) + return Optional.of(index); + + return Optional.empty(); + } + + public abstract String serializeKind(ColumnDefinition.Kind kind, boolean isDense); + + + private static void addTriggerToSchemaMutation(CFMetaData table, TriggerMetadata trigger, long timestamp, Mutation mutation) + { + new RowUpdateBuilder(SystemKeyspace.LegacyTriggers, timestamp, mutation) + .clustering(table.cfName, trigger.name) + .addMapEntry("trigger_options", "class", trigger.classOption) + .build(); + } + + /* + * Serializing types + */ + + private static void addTypeToSchemaMutation(UserType type, long timestamp, Mutation mutation) + { + RowUpdateBuilder adder = new RowUpdateBuilder(SystemKeyspace.LegacyUsertypes, timestamp, mutation) + .clustering(type.getNameAsString()); + + adder.resetCollection("field_names") + .resetCollection("field_types"); + + for (int i = 0; i < type.size(); i++) + { + adder.addListEntry("field_names", type.fieldName(i)) + .addListEntry("field_types", type.fieldType(i).toString()); + } + + adder.build(); + } + + /* + * Serializing functions + */ + + private static void addFunctionToSchemaMutation(UDFunction function, long timestamp, Mutation mutation) + { + RowUpdateBuilder adder = new RowUpdateBuilder(SystemKeyspace.LegacyFunctions, timestamp, mutation) + .clustering(function.name().name, functionSignatureWithTypes(function)); + + adder.add("body", function.body()) + .add("language", function.language()) + .add("return_type", function.returnType().toString()) + .add("called_on_null_input", function.isCalledOnNullInput()); + + adder.resetCollection("argument_names") + .resetCollection("argument_types"); + + for (int i = 0; i < function.argNames().size(); i++) + { + adder.addListEntry("argument_names", function.argNames().get(i).bytes) + .addListEntry("argument_types", function.argTypes().get(i).toString()); + } + + adder.build(); + } + + /* + * Serializing aggregates + */ + + private static void addAggregateToSchemaMutation(UDAggregate aggregate, long timestamp, Mutation mutation) + { + RowUpdateBuilder adder = new RowUpdateBuilder(SystemKeyspace.LegacyAggregates, timestamp, mutation) + .clustering(aggregate.name().name, functionSignatureWithTypes(aggregate)); + + adder.resetCollection("argument_types"); + + adder.add("return_type", aggregate.returnType().toString()) + .add("state_func", aggregate.stateFunction().name().name); + + if (aggregate.stateType() != null) + adder.add("state_type", aggregate.stateType().toString()); + if (aggregate.finalFunction() != null) + adder.add("final_func", aggregate.finalFunction().name().name); + if (aggregate.initialCondition() != null) + adder.add("initcond", aggregate.initialCondition()); + + for (AbstractType<?> argType : aggregate.argTypes()) + adder.addListEntry("argument_types", argType.toString()); + + adder.build(); + } + + // We allow method overloads, so a function is not uniquely identified by its name only, but + // also by its argument types. To distinguish overloads of given function name in the schema + // we use a "signature" which is just a list of it's CQL argument types. + public static ByteBuffer functionSignatureWithTypes(AbstractFunction fun) + { + List<String> arguments = + fun.argTypes() + .stream() + .map(argType -> argType.asCQL3Type().toString()) + .collect(Collectors.toList()); + + return ListType.getInstance(UTF8Type.instance, false).decompose(arguments); + } + + private static void setLegacyIndexStatus(KeyspaceMetadata keyspace) + { + keyspace.tables.forEach(LegacySchemaMigratorBaseTest::setLegacyIndexStatus); + } + + private static void setLegacyIndexStatus(CFMetaData table) + { + table.getIndexes().forEach((index) -> setLegacyIndexStatus(table.ksName, table.cfName, index)); + } + + private static void setLegacyIndexStatus(String keyspace, String table, IndexMetadata index) + { + SystemKeyspace.setIndexBuilt(keyspace, table + '.' + index.name); + } + + public static void verifyIndexBuildStatus(KeyspaceMetadata keyspace) + { + keyspace.tables.forEach(LegacySchemaMigratorBaseTest::verifyIndexBuildStatus); + } + + private static void verifyIndexBuildStatus(CFMetaData table) + { + table.getIndexes().forEach(index -> verifyIndexBuildStatus(table.ksName, table.cfName, index)); + } + + private static void verifyIndexBuildStatus(String keyspace, String table, IndexMetadata index) + { + assertFalse(SystemKeyspace.isIndexBuilt(keyspace, table + '.' + index.name)); + assertTrue(SystemKeyspace.isIndexBuilt(keyspace, index.name)); + } +} diff --git a/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java b/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java index 7643456ea00c..c25f39cdeb5b 100644 --- a/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java +++ b/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java @@ -18,42 +18,23 @@ package org.apache.cassandra.schema; import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.*; -import java.util.stream.Collectors; -import com.google.common.collect.ImmutableList; +import org.apache.cassandra.config.ColumnDefinition; import org.junit.Assert; import org.junit.Test; -import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.config.CFMetaData; -import org.apache.cassandra.config.ColumnDefinition; -import org.apache.cassandra.config.Schema; import org.apache.cassandra.cql3.CQLTester; -import org.apache.cassandra.cql3.ColumnIdentifier; -import org.apache.cassandra.cql3.functions.*; import org.apache.cassandra.db.*; -import org.apache.cassandra.db.marshal.*; -import org.apache.cassandra.index.internal.CassandraIndex; -import org.apache.cassandra.thrift.ThriftConversion; -import org.apache.cassandra.utils.*; import static java.lang.String.format; import static junit.framework.Assert.assertEquals; -import static junit.framework.Assert.assertFalse; import static junit.framework.Assert.assertTrue; import static org.apache.cassandra.cql3.QueryProcessor.executeOnceInternal; -import static org.apache.cassandra.utils.ByteBufferUtil.bytes; -import static org.apache.cassandra.utils.FBUtilities.json; @SuppressWarnings("deprecation") -public class LegacySchemaMigratorTest +public class LegacySchemaMigratorTest extends LegacySchemaMigratorBaseTest { - private static final long TIMESTAMP = 1435908994000000L; - - private static final String KEYSPACE_PREFIX = "LegacySchemaMigratorTest"; - /* * 1. Write a variety of different keyspaces/tables/types/function in the legacy manner, using legacy schema tables * 2. Run the migrator @@ -66,10 +47,10 @@ public void testMigrate() throws IOException { CQLTester.cleanupAndLeaveDirs(); - Keyspaces expected = keyspacesToMigrate(); + Keyspaces expected = LegacySchemaMigratorBaseTest.keyspacesToMigrate(); // write the keyspaces into the legacy tables - expected.forEach(LegacySchemaMigratorTest::legacySerializeKeyspace); + expected.forEach(this::legacySerializeKeyspace); // run the migration LegacySchemaMigrator.migrate(); @@ -78,7 +59,7 @@ public void testMigrate() throws IOException Keyspaces actual = SchemaKeyspace.fetchNonSystemKeyspaces(); // need to load back CFMetaData of those tables (CFS instances will still be loaded) - loadLegacySchemaTables(); + LegacySchemaMigratorBaseTest.loadLegacySchemaTables(); // verify that nothing's left in the old schema tables for (CFMetaData table : LegacySchemaMigrator.LegacySchemaTables) @@ -94,7 +75,7 @@ public void testMigrate() throws IOException // check that the build status of all indexes has been updated to use the new // format of index name: the index_name column of system.IndexInfo used to // contain table_name.index_name. Now it should contain just the index_name. - expected.forEach(LegacySchemaMigratorTest::verifyIndexBuildStatus); + expected.forEach(LegacySchemaMigratorBaseTest::verifyIndexBuildStatus); } @Test @@ -122,588 +103,8 @@ public void testMigrateLegacyCachingOptions() throws IOException } } - private static void loadLegacySchemaTables() - { - KeyspaceMetadata systemKeyspace = Schema.instance.getKSMetaData(SystemKeyspace.NAME); - - Tables systemTables = systemKeyspace.tables; - for (CFMetaData table : LegacySchemaMigrator.LegacySchemaTables) - systemTables = systemTables.with(table); - - LegacySchemaMigrator.LegacySchemaTables.forEach(Schema.instance::load); - - Schema.instance.setKeyspaceMetadata(systemKeyspace.withSwapped(systemTables)); - } - - private static Keyspaces keyspacesToMigrate() - { - Keyspaces.Builder keyspaces = Keyspaces.builder(); - - // A whole bucket of shorthand - String ks1 = KEYSPACE_PREFIX + "Keyspace1"; - String ks2 = KEYSPACE_PREFIX + "Keyspace2"; - String ks3 = KEYSPACE_PREFIX + "Keyspace3"; - String ks4 = KEYSPACE_PREFIX + "Keyspace4"; - String ks5 = KEYSPACE_PREFIX + "Keyspace5"; - String ks6 = KEYSPACE_PREFIX + "Keyspace6"; - String ks_rcs = KEYSPACE_PREFIX + "RowCacheSpace"; - String ks_nocommit = KEYSPACE_PREFIX + "NoCommitlogSpace"; - String ks_prsi = KEYSPACE_PREFIX + "PerRowSecondaryIndex"; - String ks_cql = KEYSPACE_PREFIX + "cql_keyspace"; - - // Make it easy to test compaction - Map<String, String> compactionOptions = new HashMap<>(); - compactionOptions.put("tombstone_compaction_interval", "1"); - - Map<String, String> leveledOptions = new HashMap<>(); - leveledOptions.put("sstable_size_in_mb", "1"); - - keyspaces.add(KeyspaceMetadata.create(ks1, - KeyspaceParams.simple(1), - Tables.of(SchemaLoader.standardCFMD(ks1, "Standard1") - .compaction(CompactionParams.scts(compactionOptions)), - SchemaLoader.standardCFMD(ks1, "StandardGCGS0").gcGraceSeconds(0), - SchemaLoader.standardCFMD(ks1, "StandardLong1"), - SchemaLoader.keysIndexCFMD(ks1, "Indexed1", true), - SchemaLoader.keysIndexCFMD(ks1, "Indexed2", false), - SchemaLoader.jdbcCFMD(ks1, "JdbcUtf8", UTF8Type.instance) - .addColumnDefinition(SchemaLoader.utf8Column(ks1, "JdbcUtf8")), - SchemaLoader.jdbcCFMD(ks1, "JdbcLong", LongType.instance), - SchemaLoader.jdbcCFMD(ks1, "JdbcBytes", BytesType.instance), - SchemaLoader.jdbcCFMD(ks1, "JdbcAscii", AsciiType.instance), - SchemaLoader.standardCFMD(ks1, "StandardLeveled") - .compaction(CompactionParams.lcs(leveledOptions)), - SchemaLoader.standardCFMD(ks1, "legacyleveled") - .compaction(CompactionParams.lcs(leveledOptions)), - SchemaLoader.standardCFMD(ks1, "StandardLowIndexInterval") - .minIndexInterval(8) - .maxIndexInterval(256) - .caching(CachingParams.CACHE_NOTHING)))); - - // Keyspace 2 - keyspaces.add(KeyspaceMetadata.create(ks2, - KeyspaceParams.simple(1), - Tables.of(SchemaLoader.standardCFMD(ks2, "Standard1"), - SchemaLoader.keysIndexCFMD(ks2, "Indexed1", true), - SchemaLoader.compositeIndexCFMD(ks2, "Indexed2", true), - SchemaLoader.compositeIndexCFMD(ks2, "Indexed3", true) - .gcGraceSeconds(0)))); - - // Keyspace 3 - keyspaces.add(KeyspaceMetadata.create(ks3, - KeyspaceParams.simple(5), - Tables.of(SchemaLoader.standardCFMD(ks3, "Standard1"), - SchemaLoader.keysIndexCFMD(ks3, "Indexed1", true)))); - - // Keyspace 4 - keyspaces.add(KeyspaceMetadata.create(ks4, - KeyspaceParams.simple(3), - Tables.of(SchemaLoader.standardCFMD(ks4, "Standard1")))); - - // Keyspace 5 - keyspaces.add(KeyspaceMetadata.create(ks5, - KeyspaceParams.simple(2), - Tables.of(SchemaLoader.standardCFMD(ks5, "Standard1")))); - - // Keyspace 6 - keyspaces.add(KeyspaceMetadata.create(ks6, - KeyspaceParams.simple(1), - Tables.of(SchemaLoader.keysIndexCFMD(ks6, "Indexed1", true)))); - - // RowCacheSpace - keyspaces.add(KeyspaceMetadata.create(ks_rcs, - KeyspaceParams.simple(1), - Tables.of(SchemaLoader.standardCFMD(ks_rcs, "CFWithoutCache") - .caching(CachingParams.CACHE_NOTHING), - SchemaLoader.standardCFMD(ks_rcs, "CachedCF") - .caching(CachingParams.CACHE_EVERYTHING), - SchemaLoader.standardCFMD(ks_rcs, "CachedIntCF") - .caching(new CachingParams(true, 100))))); - - keyspaces.add(KeyspaceMetadata.create(ks_nocommit, - KeyspaceParams.simpleTransient(1), - Tables.of(SchemaLoader.standardCFMD(ks_nocommit, "Standard1")))); - - // PerRowSecondaryIndexTest - keyspaces.add(KeyspaceMetadata.create(ks_prsi, - KeyspaceParams.simple(1), - Tables.of(SchemaLoader.perRowIndexedCFMD(ks_prsi, "Indexed1")))); - - // CQLKeyspace - keyspaces.add(KeyspaceMetadata.create(ks_cql, - KeyspaceParams.simple(1), - Tables.of(CFMetaData.compile("CREATE TABLE table1 (" - + "k int PRIMARY KEY," - + "v1 text," - + "v2 int" - + ')', ks_cql), - - CFMetaData.compile("CREATE TABLE table2 (" - + "k text," - + "c text," - + "v text," - + "PRIMARY KEY (k, c))", ks_cql), - - CFMetaData.compile("CREATE TABLE foo (" - + "bar text, " - + "baz text, " - + "qux text, " - + "PRIMARY KEY(bar, baz) ) " - + "WITH COMPACT STORAGE", ks_cql), - - CFMetaData.compile("CREATE TABLE compact_pkonly (" - + "k int, " - + "c int, " - + "PRIMARY KEY (k, c)) " - + "WITH COMPACT STORAGE", - ks_cql), - - CFMetaData.compile("CREATE TABLE foofoo (" - + "bar text, " - + "baz text, " - + "qux text, " - + "quz text, " - + "foo text, " - + "PRIMARY KEY((bar, baz), qux, quz) ) " - + "WITH COMPACT STORAGE", ks_cql)))); - - // NTS keyspace - keyspaces.add(KeyspaceMetadata.create("nts", KeyspaceParams.nts("dc1", 1, "dc2", 2))); - - keyspaces.add(keyspaceWithDroppedCollections()); - keyspaces.add(keyspaceWithTriggers()); - keyspaces.add(keyspaceWithUDTs()); - keyspaces.add(keyspaceWithUDFs()); - keyspaces.add(keyspaceWithUDFsAndUDTs()); - keyspaces.add(keyspaceWithUDAs()); - keyspaces.add(keyspaceWithUDAsAndUDTs()); - - return keyspaces.build(); - } - - private static KeyspaceMetadata keyspaceWithDroppedCollections() - { - String keyspace = KEYSPACE_PREFIX + "DroppedCollections"; - - CFMetaData table = - CFMetaData.compile("CREATE TABLE dropped_columns (" - + "foo text," - + "bar text," - + "map1 map<text, text>," - + "map2 map<int, int>," - + "set1 set<ascii>," - + "list1 list<blob>," - + "PRIMARY KEY ((foo), bar))", - keyspace); - - String[] collectionColumnNames = { "map1", "map2", "set1", "list1" }; - for (String name : collectionColumnNames) - { - ColumnDefinition column = table.getColumnDefinition(bytes(name)); - table.recordColumnDrop(column, FBUtilities.timestampMicros(), false); - table.removeColumnDefinition(column); - } - - return KeyspaceMetadata.create(keyspace, KeyspaceParams.simple(1), Tables.of(table)); - } - - private static KeyspaceMetadata keyspaceWithTriggers() - { - String keyspace = KEYSPACE_PREFIX + "Triggers"; - - Triggers.Builder triggers = Triggers.builder(); - CFMetaData table = SchemaLoader.standardCFMD(keyspace, "WithTriggers"); - for (int i = 0; i < 10; i++) - triggers.add(new TriggerMetadata("trigger" + i, "DummyTrigger" + i)); - table.triggers(triggers.build()); - - return KeyspaceMetadata.create(keyspace, KeyspaceParams.simple(1), Tables.of(table)); - } - - private static KeyspaceMetadata keyspaceWithUDTs() - { - String keyspace = KEYSPACE_PREFIX + "UDTs"; - - UserType udt1 = new UserType(keyspace, - bytes("udt1"), - new ArrayList<ByteBuffer>() {{ add(bytes("col1")); add(bytes("col2")); }}, - new ArrayList<AbstractType<?>>() {{ add(UTF8Type.instance); add(Int32Type.instance); }}); - - UserType udt2 = new UserType(keyspace, - bytes("udt2"), - new ArrayList<ByteBuffer>() {{ add(bytes("col3")); add(bytes("col4")); }}, - new ArrayList<AbstractType<?>>() {{ add(BytesType.instance); add(BooleanType.instance); }}); - - UserType udt3 = new UserType(keyspace, - bytes("udt3"), - new ArrayList<ByteBuffer>() {{ add(bytes("col5")); }}, - new ArrayList<AbstractType<?>>() {{ add(AsciiType.instance); }}); - - return KeyspaceMetadata.create(keyspace, - KeyspaceParams.simple(1), - Tables.none(), - Views.none(), - Types.of(udt1, udt2, udt3), - Functions.none()); - } - - private static KeyspaceMetadata keyspaceWithUDFs() - { - String keyspace = KEYSPACE_PREFIX + "UDFs"; - - UDFunction udf1 = UDFunction.create(new FunctionName(keyspace, "udf"), - ImmutableList.of(new ColumnIdentifier("col1", false), new ColumnIdentifier("col2", false)), - ImmutableList.of(BytesType.instance, Int32Type.instance), - LongType.instance, - false, - "java", - "return 42L;"); - - // an overload with the same name, not a typo - UDFunction udf2 = UDFunction.create(new FunctionName(keyspace, "udf"), - ImmutableList.of(new ColumnIdentifier("col3", false), new ColumnIdentifier("col4", false)), - ImmutableList.of(AsciiType.instance, LongType.instance), - Int32Type.instance, - true, - "java", - "return 42;"); - - UDFunction udf3 = UDFunction.create(new FunctionName(keyspace, "udf3"), - ImmutableList.of(new ColumnIdentifier("col4", false)), - ImmutableList.of(UTF8Type.instance), - BooleanType.instance, - false, - "java", - "return true;"); - - return KeyspaceMetadata.create(keyspace, - KeyspaceParams.simple(1), - Tables.none(), - Views.none(), - Types.none(), - Functions.of(udf1, udf2, udf3)); - } - - private static KeyspaceMetadata keyspaceWithUDAs() - { - String keyspace = KEYSPACE_PREFIX + "UDAs"; - - UDFunction udf1 = UDFunction.create(new FunctionName(keyspace, "udf1"), - ImmutableList.of(new ColumnIdentifier("col1", false), new ColumnIdentifier("col2", false)), - ImmutableList.of(Int32Type.instance, Int32Type.instance), - Int32Type.instance, - false, - "java", - "return 42;"); - - UDFunction udf2 = UDFunction.create(new FunctionName(keyspace, "udf2"), - ImmutableList.of(new ColumnIdentifier("col1", false), new ColumnIdentifier("col2", false)), - ImmutableList.of(LongType.instance, Int32Type.instance), - LongType.instance, - false, - "java", - "return 42L;"); - - UDFunction udf3 = UDFunction.create(new FunctionName(keyspace, "udf3"), - ImmutableList.of(new ColumnIdentifier("col1", false)), - ImmutableList.of(LongType.instance), - DoubleType.instance, - false, - "java", - "return 42d;"); - - Functions udfs = Functions.builder().add(udf1).add(udf2).add(udf3).build(); - - UDAggregate uda1 = UDAggregate.create(udfs, new FunctionName(keyspace, "uda1"), - ImmutableList.of(udf1.argTypes().get(1)), - udf1.returnType(), - udf1.name(), - null, - udf1.argTypes().get(0), - null - ); - - UDAggregate uda2 = UDAggregate.create(udfs, new FunctionName(keyspace, "uda2"), - ImmutableList.of(udf2.argTypes().get(1)), - udf3.returnType(), - udf2.name(), - udf3.name(), - udf2.argTypes().get(0), - LongType.instance.decompose(0L) - ); - - return KeyspaceMetadata.create(keyspace, - KeyspaceParams.simple(1), - Tables.none(), - Views.none(), - Types.none(), - Functions.of(udf1, udf2, udf3, uda1, uda2)); - } - - private static KeyspaceMetadata keyspaceWithUDFsAndUDTs() - { - String keyspace = KEYSPACE_PREFIX + "UDFUDTs"; - - UserType udt1 = new UserType(keyspace, - bytes("udt1"), - new ArrayList<ByteBuffer>() {{ add(bytes("col1")); add(bytes("col2")); }}, - new ArrayList<AbstractType<?>>() {{ add(UTF8Type.instance); add(Int32Type.instance); }}); - - UserType udt2 = new UserType(keyspace, - bytes("udt2"), - new ArrayList<ByteBuffer>() {{ add(bytes("col1")); add(bytes("col2")); }}, - new ArrayList<AbstractType<?>>() {{ add(ListType.getInstance(udt1, false)); add(Int32Type.instance); }}); - - UDFunction udf1 = UDFunction.create(new FunctionName(keyspace, "udf"), - ImmutableList.of(new ColumnIdentifier("col1", false), new ColumnIdentifier("col2", false)), - ImmutableList.of(udt1, udt2), - LongType.instance, - false, - "java", - "return 42L;"); - - // an overload with the same name, not a typo - UDFunction udf2 = UDFunction.create(new FunctionName(keyspace, "udf"), - ImmutableList.of(new ColumnIdentifier("col3", false), new ColumnIdentifier("col4", false)), - ImmutableList.of(AsciiType.instance, LongType.instance), - Int32Type.instance, - true, - "java", - "return 42;"); - - UDFunction udf3 = UDFunction.create(new FunctionName(keyspace, "udf3"), - ImmutableList.of(new ColumnIdentifier("col4", false)), - ImmutableList.of(new TupleType(Arrays.asList(udt1, udt2))), - BooleanType.instance, - false, - "java", - "return true;"); - - return KeyspaceMetadata.create(keyspace, - KeyspaceParams.simple(1), - Tables.none(), - Views.none(), - Types.of(udt1, udt2), - Functions.of(udf1, udf2, udf3)); - } - - private static KeyspaceMetadata keyspaceWithUDAsAndUDTs() - { - String keyspace = KEYSPACE_PREFIX + "UDAUDTs"; - - UserType udt1 = new UserType(keyspace, - bytes("udt1"), - new ArrayList<ByteBuffer>() {{ add(bytes("col1")); add(bytes("col2")); }}, - new ArrayList<AbstractType<?>>() {{ add(UTF8Type.instance); add(Int32Type.instance); }}); - - UserType udt2 = new UserType(keyspace, - bytes("udt2"), - new ArrayList<ByteBuffer>() {{ add(bytes("col1")); add(bytes("col2")); }}, - new ArrayList<AbstractType<?>>() {{ add(ListType.getInstance(udt1, false)); add(Int32Type.instance); }}); - - UDFunction udf1 = UDFunction.create(new FunctionName(keyspace, "udf1"), - ImmutableList.of(new ColumnIdentifier("col1", false), new ColumnIdentifier("col2", false)), - ImmutableList.of(udt1, udt2), - udt1, - false, - "java", - "return null;"); - - UDFunction udf2 = UDFunction.create(new FunctionName(keyspace, "udf2"), - ImmutableList.of(new ColumnIdentifier("col1", false), new ColumnIdentifier("col2", false)), - ImmutableList.of(udt2, udt1), - udt2, - false, - "java", - "return null;"); - - UDFunction udf3 = UDFunction.create(new FunctionName(keyspace, "udf3"), - ImmutableList.of(new ColumnIdentifier("col1", false)), - ImmutableList.of(udt2), - DoubleType.instance, - false, - "java", - "return 42d;"); - - Functions udfs = Functions.builder().add(udf1).add(udf2).add(udf3).build(); - - UDAggregate uda1 = UDAggregate.create(udfs, new FunctionName(keyspace, "uda1"), - ImmutableList.of(udf1.argTypes().get(1)), - udf1.returnType(), - udf1.name(), - null, - udf1.argTypes().get(0), - null - ); - - ByteBuffer twoNullEntries = ByteBuffer.allocate(8); - twoNullEntries.putInt(-1); - twoNullEntries.putInt(-1); - twoNullEntries.flip(); - UDAggregate uda2 = UDAggregate.create(udfs, new FunctionName(keyspace, "uda2"), - ImmutableList.of(udf2.argTypes().get(1)), - udf3.returnType(), - udf2.name(), - udf3.name(), - udf2.argTypes().get(0), - twoNullEntries - ); - - return KeyspaceMetadata.create(keyspace, - KeyspaceParams.simple(1), - Tables.none(), - Views.none(), - Types.of(udt1, udt2), - Functions.of(udf1, udf2, udf3, uda1, uda2)); - } - - /* - * Serializing keyspaces - */ - - private static void legacySerializeKeyspace(KeyspaceMetadata keyspace) - { - makeLegacyCreateKeyspaceMutation(keyspace, TIMESTAMP).apply(); - setLegacyIndexStatus(keyspace); - } - - private static Mutation makeLegacyCreateKeyspaceMutation(KeyspaceMetadata keyspace, long timestamp) - { - // Note that because Keyspaces is a COMPACT TABLE, we're really only setting static columns internally and shouldn't set any clustering. - RowUpdateBuilder adder = new RowUpdateBuilder(SystemKeyspace.LegacyKeyspaces, timestamp, keyspace.name); - - adder.add("durable_writes", keyspace.params.durableWrites) - .add("strategy_class", keyspace.params.replication.klass.getName()) - .add("strategy_options", json(keyspace.params.replication.options)); - - Mutation mutation = adder.build(); - - keyspace.tables.forEach(table -> addTableToSchemaMutation(table, timestamp, true, mutation)); - keyspace.types.forEach(type -> addTypeToSchemaMutation(type, timestamp, mutation)); - keyspace.functions.udfs().forEach(udf -> addFunctionToSchemaMutation(udf, timestamp, mutation)); - keyspace.functions.udas().forEach(uda -> addAggregateToSchemaMutation(uda, timestamp, mutation)); - - return mutation; - } - - /* - * Serializing tables - */ - - private static void addTableToSchemaMutation(CFMetaData table, long timestamp, boolean withColumnsAndTriggers, Mutation mutation) - { - // For property that can be null (and can be changed), we insert tombstones, to make sure - // we don't keep a property the user has removed - RowUpdateBuilder adder = new RowUpdateBuilder(SystemKeyspace.LegacyColumnfamilies, timestamp, mutation) - .clustering(table.cfName); - - adder.add("cf_id", table.cfId) - .add("type", table.isSuper() ? "Super" : "Standard"); - - if (table.isSuper()) - { - adder.add("comparator", table.comparator.subtype(0).toString()) - .add("subcomparator", ((MapType)table.compactValueColumn().type).getKeysType().toString()); - } - else - { - adder.add("comparator", LegacyLayout.makeLegacyComparator(table).toString()); - } - - adder.add("bloom_filter_fp_chance", table.params.bloomFilterFpChance) - .add("caching", cachingToString(table.params.caching)) - .add("comment", table.params.comment) - .add("compaction_strategy_class", table.params.compaction.klass().getName()) - .add("compaction_strategy_options", json(table.params.compaction.options())) - .add("compression_parameters", json(ThriftConversion.compressionParametersToThrift(table.params.compression))) - .add("default_time_to_live", table.params.defaultTimeToLive) - .add("gc_grace_seconds", table.params.gcGraceSeconds) - .add("key_validator", table.getKeyValidator().toString()) - .add("local_read_repair_chance", table.params.dcLocalReadRepairChance) - .add("max_compaction_threshold", table.params.compaction.maxCompactionThreshold()) - .add("max_index_interval", table.params.maxIndexInterval) - .add("memtable_flush_period_in_ms", table.params.memtableFlushPeriodInMs) - .add("min_compaction_threshold", table.params.compaction.minCompactionThreshold()) - .add("min_index_interval", table.params.minIndexInterval) - .add("read_repair_chance", table.params.readRepairChance) - .add("speculative_retry", table.params.speculativeRetry.toString()); - - for (Map.Entry<ByteBuffer, CFMetaData.DroppedColumn> entry : table.getDroppedColumns().entrySet()) - { - String name = UTF8Type.instance.getString(entry.getKey()); - CFMetaData.DroppedColumn column = entry.getValue(); - adder.addMapEntry("dropped_columns", name, column.droppedTime); - } - - adder.add("is_dense", table.isDense()); - - adder.add("default_validator", table.makeLegacyDefaultValidator().toString()); - - if (withColumnsAndTriggers) - { - for (ColumnDefinition column : table.allColumns()) - addColumnToSchemaMutation(table, column, timestamp, mutation); - - for (TriggerMetadata trigger : table.getTriggers()) - addTriggerToSchemaMutation(table, trigger, timestamp, mutation); - } - - adder.build(); - } - - private static String cachingToString(CachingParams caching) - { - return format("{\"keys\":\"%s\", \"rows_per_partition\":\"%s\"}", - caching.keysAsString(), - caching.rowsPerPartitionAsString()); - } - - private static void addColumnToSchemaMutation(CFMetaData table, ColumnDefinition column, long timestamp, Mutation mutation) - { - // We need to special case pk-only dense tables. See CASSANDRA-9874. - String name = table.isDense() && column.kind == ColumnDefinition.Kind.REGULAR && column.type instanceof EmptyType - ? "" - : column.name.toString(); - - final RowUpdateBuilder adder = new RowUpdateBuilder(SystemKeyspace.LegacyColumns, timestamp, mutation).clustering(table.cfName, name); - - adder.add("validator", column.type.toString()) - .add("type", serializeKind(column.kind, table.isDense())) - .add("component_index", column.position()); - - Optional<IndexMetadata> index = findIndexForColumn(table.getIndexes(), table, column); - if (index.isPresent()) - { - IndexMetadata i = index.get(); - adder.add("index_name", i.name); - adder.add("index_type", i.kind.toString()); - adder.add("index_options", json(i.options)); - } - else - { - adder.add("index_name", null); - adder.add("index_type", null); - adder.add("index_options", null); - } - - adder.build(); - } - - private static Optional<IndexMetadata> findIndexForColumn(Indexes indexes, - CFMetaData table, - ColumnDefinition column) - { - // makes the assumptions that the string option denoting the - // index targets can be parsed by CassandraIndex.parseTarget - // which should be true for any pre-3.0 index - for (IndexMetadata index : indexes) - if (CassandraIndex.parseTarget(table, index).left.equals(column)) - return Optional.of(index); - - return Optional.empty(); - } - - private static String serializeKind(ColumnDefinition.Kind kind, boolean isDense) + @Override + public String serializeKind(ColumnDefinition.Kind kind, boolean isDense) { // For backward compatibility, we special case CLUSTERING and the case where the table is dense. if (kind == ColumnDefinition.Kind.CLUSTERING) @@ -715,131 +116,4 @@ private static String serializeKind(ColumnDefinition.Kind kind, boolean isDense) return kind.toString().toLowerCase(); } - private static void addTriggerToSchemaMutation(CFMetaData table, TriggerMetadata trigger, long timestamp, Mutation mutation) - { - new RowUpdateBuilder(SystemKeyspace.LegacyTriggers, timestamp, mutation) - .clustering(table.cfName, trigger.name) - .addMapEntry("trigger_options", "class", trigger.classOption) - .build(); - } - - /* - * Serializing types - */ - - private static void addTypeToSchemaMutation(UserType type, long timestamp, Mutation mutation) - { - RowUpdateBuilder adder = new RowUpdateBuilder(SystemKeyspace.LegacyUsertypes, timestamp, mutation) - .clustering(type.getNameAsString()); - - adder.resetCollection("field_names") - .resetCollection("field_types"); - - for (int i = 0; i < type.size(); i++) - { - adder.addListEntry("field_names", type.fieldName(i)) - .addListEntry("field_types", type.fieldType(i).toString()); - } - - adder.build(); - } - - /* - * Serializing functions - */ - - private static void addFunctionToSchemaMutation(UDFunction function, long timestamp, Mutation mutation) - { - RowUpdateBuilder adder = new RowUpdateBuilder(SystemKeyspace.LegacyFunctions, timestamp, mutation) - .clustering(function.name().name, functionSignatureWithTypes(function)); - - adder.add("body", function.body()) - .add("language", function.language()) - .add("return_type", function.returnType().toString()) - .add("called_on_null_input", function.isCalledOnNullInput()); - - adder.resetCollection("argument_names") - .resetCollection("argument_types"); - - for (int i = 0; i < function.argNames().size(); i++) - { - adder.addListEntry("argument_names", function.argNames().get(i).bytes) - .addListEntry("argument_types", function.argTypes().get(i).toString()); - } - - adder.build(); - } - - /* - * Serializing aggregates - */ - - private static void addAggregateToSchemaMutation(UDAggregate aggregate, long timestamp, Mutation mutation) - { - RowUpdateBuilder adder = new RowUpdateBuilder(SystemKeyspace.LegacyAggregates, timestamp, mutation) - .clustering(aggregate.name().name, functionSignatureWithTypes(aggregate)); - - adder.resetCollection("argument_types"); - - adder.add("return_type", aggregate.returnType().toString()) - .add("state_func", aggregate.stateFunction().name().name); - - if (aggregate.stateType() != null) - adder.add("state_type", aggregate.stateType().toString()); - if (aggregate.finalFunction() != null) - adder.add("final_func", aggregate.finalFunction().name().name); - if (aggregate.initialCondition() != null) - adder.add("initcond", aggregate.initialCondition()); - - for (AbstractType<?> argType : aggregate.argTypes()) - adder.addListEntry("argument_types", argType.toString()); - - adder.build(); - } - - // We allow method overloads, so a function is not uniquely identified by its name only, but - // also by its argument types. To distinguish overloads of given function name in the schema - // we use a "signature" which is just a list of it's CQL argument types. - public static ByteBuffer functionSignatureWithTypes(AbstractFunction fun) - { - List<String> arguments = - fun.argTypes() - .stream() - .map(argType -> argType.asCQL3Type().toString()) - .collect(Collectors.toList()); - - return ListType.getInstance(UTF8Type.instance, false).decompose(arguments); - } - - private static void setLegacyIndexStatus(KeyspaceMetadata keyspace) - { - keyspace.tables.forEach(LegacySchemaMigratorTest::setLegacyIndexStatus); - } - - private static void setLegacyIndexStatus(CFMetaData table) - { - table.getIndexes().forEach((index) -> setLegacyIndexStatus(table.ksName, table.cfName, index)); - } - - private static void setLegacyIndexStatus(String keyspace, String table, IndexMetadata index) - { - SystemKeyspace.setIndexBuilt(keyspace, table + '.' + index.name); - } - - private static void verifyIndexBuildStatus(KeyspaceMetadata keyspace) - { - keyspace.tables.forEach(LegacySchemaMigratorTest::verifyIndexBuildStatus); - } - - private static void verifyIndexBuildStatus(CFMetaData table) - { - table.getIndexes().forEach(index -> verifyIndexBuildStatus(table.ksName, table.cfName, index)); - } - - private static void verifyIndexBuildStatus(String keyspace, String table, IndexMetadata index) - { - assertFalse(SystemKeyspace.isIndexBuilt(keyspace, table + '.' + index.name)); - assertTrue(SystemKeyspace.isIndexBuilt(keyspace, index.name)); - } - } diff --git a/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorThriftTest.java b/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorThriftTest.java new file mode 100644 index 000000000000..3686024f0f49 --- /dev/null +++ b/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorThriftTest.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.schema; + +import java.io.IOException; + +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.ColumnDefinition; +import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.exceptions.ConfigurationException; + +import org.junit.Test; + +import org.apache.cassandra.config.Schema; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.exceptions.InvalidRequestException; + +import static org.apache.cassandra.SchemaLoader.getCompressionParameters; +import static org.apache.cassandra.cql3.QueryProcessor.executeOnceInternal; + +@SuppressWarnings("deprecation") +public class LegacySchemaMigratorThriftTest extends LegacySchemaMigratorBaseTest +{ + private static final String KEYSPACE_18956 = "ks18956"; + private static final String TABLE_18956 = "table18956"; + + @Test + public void testMigrate18956() throws IOException + { + CQLTester.cleanupAndLeaveDirs(); + Keyspaces expected = keyspacesToMigrate18956(); + expected.forEach(this::legacySerializeKeyspace); + LegacySchemaMigrator.migrate(); + Schema.instance.loadFromDisk(); + LegacySchemaMigratorBaseTest.loadLegacySchemaTables(); + try + { + // This should fail + executeOnceInternal(String.format("ALTER TABLE %s.%s RENAME key TO \"4f\"", KEYSPACE_18956, TABLE_18956)); + assert false; + } + catch (InvalidRequestException e) + { + assert e.toString().contains("another column of that name already exist"); + } + } + + public static Keyspaces keyspacesToMigrate18956() + { + Keyspaces.Builder keyspaces = Keyspaces.builder(); + keyspaces.add(KeyspaceMetadata.create(LegacySchemaMigratorThriftTest.KEYSPACE_18956, + KeyspaceParams.simple(1), + Tables.of( + bytesTypeComparatorCFMD18956(LegacySchemaMigratorThriftTest.KEYSPACE_18956, LegacySchemaMigratorThriftTest.TABLE_18956) + ))); + return keyspaces.build(); + } + + public static CFMetaData bytesTypeComparatorCFMD18956(String ksName, String cfName) throws ConfigurationException + { + return CFMetaData.Builder.createDense(ksName, cfName, false, false) + .addPartitionKey("key", BytesType.instance) + .addClusteringColumn("3d", BytesType.instance) + .addRegularColumn("4f", BytesType.instance) + .build() + .compression(getCompressionParameters()); + } + + @Override + public String serializeKind(ColumnDefinition.Kind kind, boolean isDense) + { + // Using cassandra-cli, it's possible to create legacy without compact_value + return kind == ColumnDefinition.Kind.CLUSTERING ? "clustering_key" : kind.toString().toLowerCase(); + } +} From 184887069f18cb3afb7e747284d99f55cee41ec6 Mon Sep 17 00:00:00 2001 From: Caleb Rackliffe <calebrackliffe@gmail.com> Date: Wed, 8 Jan 2025 11:19:50 -0600 Subject: [PATCH 074/225] Remove v30 and v3X from 5.x in-JVM upgrade tests patch by Caleb Rackliffe; reviewed by Mick Semb Wever and Brandon Williams for CASSANDRA-20103 --- CHANGES.txt | 1 + .../CompactStoragePagingWithProtocolTester.java | 3 +-- .../CompactionHistorySystemTableUpgradeTest.java | 2 +- .../upgrade/ConfigCompatibilityTestGenerate.java | 4 +--- .../DropCompactStorageNullClusteringValuesTest.java | 2 +- .../distributed/upgrade/DropCompactStorageTest.java | 2 +- .../cassandra/distributed/upgrade/GroupByTest.java | 2 +- .../upgrade/MixedModeAvailabilityTestBase.java | 2 +- ...tencyV30Test.java => MixedModeConsistencyTest.java} | 4 ++-- ...gedBatchTest.java => MixedModeLoggedBatchTest.java} | 10 ++-------- ...licationTest.java => MixedModeReplicationTest.java} | 4 ++-- ...dBatchTest.java => MixedModeUnloggedBatchTest.java} | 10 ++-------- .../upgrade/MixedModeWritetimeOrTTLTest.java | 2 +- .../cassandra/distributed/upgrade/UpgradeTest.java | 2 +- .../cassandra/distributed/upgrade/UpgradeTestBase.java | 9 +-------- 15 files changed, 19 insertions(+), 40 deletions(-) rename test/distributed/org/apache/cassandra/distributed/upgrade/{MixedModeConsistencyV30Test.java => MixedModeConsistencyTest.java} (98%) rename test/distributed/org/apache/cassandra/distributed/upgrade/{MixedModeFrom3LoggedBatchTest.java => MixedModeLoggedBatchTest.java} (80%) rename test/distributed/org/apache/cassandra/distributed/upgrade/{MixedModeFrom3ReplicationTest.java => MixedModeReplicationTest.java} (96%) rename test/distributed/org/apache/cassandra/distributed/upgrade/{MixedModeFrom3UnloggedBatchTest.java => MixedModeUnloggedBatchTest.java} (79%) diff --git a/CHANGES.txt b/CHANGES.txt index 8eb1fba30083..0d11571d8142 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.3 + * Remove v30 and v3X from 5.x in-JVM upgrade tests (CASSANDRA-20103) * Avoid memory allocation in offheap_object's NativeCell.valueSize() and NativeClustering.dataSize() (CASSANDRA-20162) * Add flag to avoid invalidating key cache on sstable deletions (CASSANDRA-20068) * Interpret inet, bigint, varint, and decimal as non-reversed types for query construction and post-filtering (CASSANDRA-20100) diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/CompactStoragePagingWithProtocolTester.java b/test/distributed/org/apache/cassandra/distributed/upgrade/CompactStoragePagingWithProtocolTester.java index 691ad4eb1195..358bb8863e5a 100644 --- a/test/distributed/org/apache/cassandra/distributed/upgrade/CompactStoragePagingWithProtocolTester.java +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/CompactStoragePagingWithProtocolTester.java @@ -132,8 +132,7 @@ private void assertRowsWithAllProtocolVersions(Object[]... rows) String query = withKeyspace("SELECT * FROM %s.t"); assertRows(query, ProtocolVersion.V3, rows); assertRows(query, ProtocolVersion.V4, rows); - if (initialVersion().isGreaterThanOrEqualTo(v3X)) - assertRows(query, ProtocolVersion.V5, rows); + assertRows(query, ProtocolVersion.V5, rows); } private static void assertRows(String query, ProtocolVersion protocolVersion, Object[]... expectedRows) diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/CompactionHistorySystemTableUpgradeTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/CompactionHistorySystemTableUpgradeTest.java index 73e92dd42307..2878864d3105 100644 --- a/test/distributed/org/apache/cassandra/distributed/upgrade/CompactionHistorySystemTableUpgradeTest.java +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/CompactionHistorySystemTableUpgradeTest.java @@ -47,7 +47,7 @@ public class CompactionHistorySystemTableUpgradeTest extends UpgradeTestBase @Parameters() public static ArrayList<Semver> versions() { - return Lists.newArrayList(v30, v3X, v40, v41); + return Lists.newArrayList(v40, v41); } @Test diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/ConfigCompatibilityTestGenerate.java b/test/distributed/org/apache/cassandra/distributed/upgrade/ConfigCompatibilityTestGenerate.java index 65169dcf3ac0..841bec92e187 100644 --- a/test/distributed/org/apache/cassandra/distributed/upgrade/ConfigCompatibilityTestGenerate.java +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/ConfigCompatibilityTestGenerate.java @@ -33,8 +33,6 @@ import static org.apache.cassandra.config.ConfigCompatibilityTest.TEST_DIR; import static org.apache.cassandra.config.ConfigCompatibilityTest.dump; import static org.apache.cassandra.config.ConfigCompatibilityTest.toTree; -import static org.apache.cassandra.distributed.upgrade.UpgradeTestBase.v30; -import static org.apache.cassandra.distributed.upgrade.UpgradeTestBase.v3X; import static org.apache.cassandra.distributed.upgrade.UpgradeTestBase.v40; import static org.apache.cassandra.distributed.upgrade.UpgradeTestBase.v41; import static org.apache.cassandra.distributed.upgrade.UpgradeTestBase.v50; @@ -48,7 +46,7 @@ public static void main(String[] args) throws Throwable { ICluster.setup(); Versions versions = Versions.find(); - for (Semver version : Arrays.asList(v30, v3X, v40, v41, v50)) + for (Semver version : Arrays.asList(v40, v41, v50)) { File path = new File(TEST_DIR, "version=" + version + ".yml"); path.getParentFile().mkdirs(); diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/DropCompactStorageNullClusteringValuesTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/DropCompactStorageNullClusteringValuesTest.java index 2e5578d2c24f..ad2133e58af9 100644 --- a/test/distributed/org/apache/cassandra/distributed/upgrade/DropCompactStorageNullClusteringValuesTest.java +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/DropCompactStorageNullClusteringValuesTest.java @@ -33,7 +33,7 @@ public class DropCompactStorageNullClusteringValuesTest extends UpgradeTestBase public void testNullClusteringValues() throws Throwable { new TestCase().nodes(1) - .upgradesToCurrentFrom(v30) + .upgradesToCurrentFrom(OLDEST) .withConfig(config -> config.with(GOSSIP, NETWORK, NATIVE_PROTOCOL).set("enable_drop_compact_storage", true)) .setup(cluster -> { String create = "CREATE TABLE %s.%s(k int, c1 int, c2 int, v int, PRIMARY KEY (k, c1, c2)) " + diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/DropCompactStorageTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/DropCompactStorageTest.java index 9846264e56e6..6d2b90fd4fa4 100644 --- a/test/distributed/org/apache/cassandra/distributed/upgrade/DropCompactStorageTest.java +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/DropCompactStorageTest.java @@ -35,7 +35,7 @@ public void testDropCompactStorage() throws Throwable new TestCase() .nodes(2) .nodesToUpgrade(1, 2) - .upgradesToCurrentFrom(v30) + .upgradesToCurrentFrom(OLDEST) .withConfig(config -> config.with(GOSSIP, NETWORK).set("enable_drop_compact_storage", true)) .setup((cluster) -> { cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, PRIMARY KEY (pk, ck)) WITH COMPACT STORAGE"); diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/GroupByTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/GroupByTest.java index b2971cd15f5a..2b3339fd67cb 100644 --- a/test/distributed/org/apache/cassandra/distributed/upgrade/GroupByTest.java +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/GroupByTest.java @@ -35,7 +35,7 @@ public void testReads() throws Throwable // CASSANDRA-16582: group-by across mixed version cluster would fail with ArrayIndexOutOfBoundException new UpgradeTestBase.TestCase() .nodes(2) - .upgradesToCurrentFrom(v3X) + .upgradesToCurrentFrom(OLDEST) .nodesToUpgrade(1) .withConfig(config -> config.with(GOSSIP, NETWORK)) .setup(cluster -> { diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeAvailabilityTestBase.java b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeAvailabilityTestBase.java index d26f6761d5ea..584af8bba400 100644 --- a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeAvailabilityTestBase.java +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeAvailabilityTestBase.java @@ -61,7 +61,7 @@ public void testAvailability() throws Throwable new TestCase() .nodes(NUM_NODES) .nodesToUpgrade(upgradedCoordinator() ? 1 : 2) - .upgradesToCurrentFrom(v30) + .upgradesToCurrentFrom(OLDEST) .withConfig(config -> config.with(GOSSIP, NETWORK, NATIVE_PROTOCOL) .set("request_timeout_in_ms", MINUTES.toMillis(10)) .set("read_request_timeout_in_ms", MINUTES.toMillis(10)) diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeConsistencyV30Test.java b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeConsistencyTest.java similarity index 98% rename from test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeConsistencyV30Test.java rename to test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeConsistencyTest.java index 5ee80151a631..d1c583eda75e 100644 --- a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeConsistencyV30Test.java +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeConsistencyTest.java @@ -38,7 +38,7 @@ import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows; import static org.apache.cassandra.distributed.shared.AssertUtils.row; -public class MixedModeConsistencyV30Test extends UpgradeTestBase +public class MixedModeConsistencyTest extends UpgradeTestBase { @Test public void testConsistency() throws Throwable @@ -51,7 +51,7 @@ public void testConsistency() throws Throwable new TestCase() .nodes(3) .nodesToUpgrade(1) - .upgradesToCurrentFrom(v30) + .upgradesToCurrentFrom(OLDEST) .withConfig(config -> config.set("read_request_timeout_in_ms", SECONDS.toMillis(30)) .set("write_request_timeout_in_ms", SECONDS.toMillis(30))) .setup(cluster -> { diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeFrom3LoggedBatchTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeLoggedBatchTest.java similarity index 80% rename from test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeFrom3LoggedBatchTest.java rename to test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeLoggedBatchTest.java index 7326f6fec97e..a1180bc68955 100644 --- a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeFrom3LoggedBatchTest.java +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeLoggedBatchTest.java @@ -20,17 +20,11 @@ import org.junit.Test; -public class MixedModeFrom3LoggedBatchTest extends MixedModeBatchTestBase +public class MixedModeLoggedBatchTest extends MixedModeBatchTestBase { - @Test - public void testSimpleStrategy30to3X() throws Throwable - { - testSimpleStrategy(v30, v3X, true); - } - @Test public void testSimpleStrategy() throws Throwable { - testSimpleStrategy(v30, true); + testSimpleStrategy(OLDEST, true); } } diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeFrom3ReplicationTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeReplicationTest.java similarity index 96% rename from test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeFrom3ReplicationTest.java rename to test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeReplicationTest.java index 69d3dbec710e..f4366581226f 100644 --- a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeFrom3ReplicationTest.java +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeReplicationTest.java @@ -28,7 +28,7 @@ import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows; import static org.apache.cassandra.distributed.shared.AssertUtils.row; -public class MixedModeFrom3ReplicationTest extends UpgradeTestBase +public class MixedModeReplicationTest extends UpgradeTestBase { @Test public void testSimpleStrategy() throws Throwable @@ -39,7 +39,7 @@ public void testSimpleStrategy() throws Throwable new TestCase() .nodes(3) .nodesToUpgrade(1, 2) - .upgradesToCurrentFrom(v30) + .upgradesToCurrentFrom(OLDEST) .setup(cluster -> { cluster.schemaChange("CREATE KEYSPACE test_simple WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};"); cluster.schemaChange("CREATE TABLE test_simple.names (key int PRIMARY KEY, name text)"); diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeFrom3UnloggedBatchTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeUnloggedBatchTest.java similarity index 79% rename from test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeFrom3UnloggedBatchTest.java rename to test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeUnloggedBatchTest.java index d16914236742..583a17938839 100644 --- a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeFrom3UnloggedBatchTest.java +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeUnloggedBatchTest.java @@ -20,17 +20,11 @@ import org.junit.Test; -public class MixedModeFrom3UnloggedBatchTest extends MixedModeBatchTestBase +public class MixedModeUnloggedBatchTest extends MixedModeBatchTestBase { - @Test - public void testSimpleStrategy30to3X() throws Throwable - { - testSimpleStrategy(v30, v3X, false); - } - @Test public void testSimpleStrategy() throws Throwable { - testSimpleStrategy(v30, false); + testSimpleStrategy(OLDEST, false); } } diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeWritetimeOrTTLTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeWritetimeOrTTLTest.java index 295ffd29877c..2ae891417a40 100644 --- a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeWritetimeOrTTLTest.java +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeWritetimeOrTTLTest.java @@ -45,7 +45,7 @@ public void testWritetimeOrTTLDuringUpgrade() throws Throwable new TestCase() .nodes(2) .nodesToUpgradeOrdered(1, 2) - .upgradesToCurrentFrom(v30) + .upgradesToCurrentFrom(OLDEST) .setup(cluster -> { ICoordinator coordinator = cluster.coordinator(1); diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTest.java index 55e1f1ea9441..76977a70555c 100644 --- a/test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTest.java +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTest.java @@ -34,7 +34,7 @@ public void simpleUpgradeWithNetworkAndGossipTest() throws Throwable .nodes(2) .nodesToUpgrade(1) .withConfig((cfg) -> cfg.with(Feature.NETWORK, Feature.GOSSIP)) - .upgradesToCurrentFrom(v3X) + .upgradesToCurrentFrom(OLDEST) .setup((cluster) -> { cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))"); cluster.coordinator(1).execute("INSERT INTO " + KEYSPACE + ".tbl (pk, ck, v) VALUES (1, 1, 1)", ConsistencyLevel.ALL); diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTestBase.java b/test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTestBase.java index 3f8f4d47c8e0..1d00af5c041c 100644 --- a/test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTestBase.java +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTestBase.java @@ -87,18 +87,11 @@ public static interface RunOnClusterAndNode public void run(UpgradeableCluster cluster, int node) throws Throwable; } - public static final Semver v30 = new Semver("3.0.0-alpha1", SemverType.LOOSE); - public static final Semver v3X = new Semver("3.11.0", SemverType.LOOSE); public static final Semver v40 = new Semver("4.0-alpha1", SemverType.LOOSE); public static final Semver v41 = new Semver("4.1-alpha1", SemverType.LOOSE); public static final Semver v50 = new Semver("5.0-alpha1", SemverType.LOOSE); protected static final SimpleGraph<Semver> SUPPORTED_UPGRADE_PATHS = new SimpleGraph.Builder<Semver>() - .addEdge(v30, v3X) - .addEdge(v30, v40) - .addEdge(v30, v41) - .addEdge(v3X, v40) - .addEdge(v3X, v41) .addEdge(v40, v41) .addEdge(v40, v50) .addEdge(v41, v50) @@ -404,7 +397,7 @@ private static void triggerGC() protected TestCase allUpgrades(int nodes, int... toUpgrade) { return new TestCase().nodes(nodes) - .upgradesToCurrentFrom(v30) + .upgradesToCurrentFrom(OLDEST) .nodesToUpgrade(toUpgrade); } From 6f90e962f54c4b1a90ad6c3dc0bb6a224843abf0 Mon Sep 17 00:00:00 2001 From: Dmitry Konstantinov <netudima@gmail.com> Date: Mon, 4 Nov 2024 21:17:24 +0000 Subject: [PATCH 075/225] Enforce CQL message size limit on multiframe messages Patch by Dmitry Konstantinov; reviewed by Sam Tunnicliffe, Caleb Rackliffe for CASSANDRA-20052 --- CHANGES.txt | 1 + .../org/apache/cassandra/config/Config.java | 1 + .../cassandra/config/DatabaseDescriptor.java | 52 ++++++ .../OversizedCQLMessageException.java | 27 +++ .../cassandra/net/AbstractMessageHandler.java | 4 +- .../transport/CQLMessageHandler.java | 110 +++++++++++- .../transport/ExceptionHandlers.java | 5 + .../transport/InitialConnectionHandler.java | 3 +- .../transport/PipelineConfigurator.java | 2 + .../cassandra/transport/SimpleClient.java | 1 + .../transport/AuthMessageSizeLimitTest.java | 104 +++++++++++ .../transport/ClientResourceLimitsTest.java | 106 +++-------- .../transport/MessageSizeLimitTest.java | 124 +++++++++++++ .../NativeProtocolLimitsTestBase.java | 165 ++++++++++++++++++ .../cassandra/transport/RateLimitingTest.java | 69 +++----- 15 files changed, 632 insertions(+), 142 deletions(-) create mode 100644 src/java/org/apache/cassandra/exceptions/OversizedCQLMessageException.java create mode 100644 test/unit/org/apache/cassandra/transport/AuthMessageSizeLimitTest.java create mode 100644 test/unit/org/apache/cassandra/transport/MessageSizeLimitTest.java create mode 100644 test/unit/org/apache/cassandra/transport/NativeProtocolLimitsTestBase.java diff --git a/CHANGES.txt b/CHANGES.txt index 3e5597558d1e..45ef4fe379f6 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.1.8 + * Enforce CQL message size limit on multiframe messages (CASSANDRA-20052) * Add nodetool checktokenmetadata command that checks TokenMetadata is insync with Gossip endpointState (CASSANDRA-18758) * Backport Java 11 support for Simulator (CASSANDRA-17178/CASSANDRA-19935) * Equality check for Paxos.Electorate should not depend on collection types (CASSANDRA-19935) diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index d8d1f7e617b4..b64ff079dee6 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -269,6 +269,7 @@ public MemtableOptions() public int native_transport_max_threads = 128; @Replaces(oldName = "native_transport_max_frame_size_in_mb", converter = Converters.MEBIBYTES_DATA_STORAGE_INT, deprecated = true) public DataStorageSpec.IntMebibytesBound native_transport_max_frame_size = new DataStorageSpec.IntMebibytesBound("16MiB"); + public volatile DataStorageSpec.LongBytesBound native_transport_max_message_size = null; /** do bcrypt hashing in a limited pool to prevent cpu load spikes; 0 means that all requests will go to default request executor**/ public int native_transport_max_auth_threads = 0; public volatile long native_transport_max_concurrent_connections = -1L; diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index b85a680cfa9b..3ff03ce80150 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -153,6 +153,9 @@ public class DatabaseDescriptor private static long counterCacheSizeInMiB; private static long indexSummaryCapacityInMiB; + private static volatile long nativeTransportMaxMessageSizeInBytes; + private static volatile boolean nativeTransportMaxMessageSizeConfiguredExplicitly; + private static String localDC; private static Comparator<Replica> localComparator; private static EncryptionContext encryptionContext; @@ -822,6 +825,23 @@ else if (conf.commitlog_segment_size.toMebibytes() >= 2048) else if (conf.commitlog_segment_size.toKibibytes() < 2 * conf.max_mutation_size.toKibibytes()) throw new ConfigurationException("commitlog_segment_size must be at least twice the size of max_mutation_size / 1024", false); + if (conf.native_transport_max_message_size == null) + { + conf.native_transport_max_message_size = new DataStorageSpec.LongBytesBound(calculateDefaultNativeTransportMaxMessageSizeInBytes()); + } + else + { + nativeTransportMaxMessageSizeConfiguredExplicitly = true; + long maxCqlMessageSize = conf.native_transport_max_message_size.toBytes(); + if (maxCqlMessageSize > conf.native_transport_max_request_data_in_flight.toBytes()) + throw new ConfigurationException("native_transport_max_message_size must not exceed native_transport_max_request_data_in_flight", false); + + if (maxCqlMessageSize > conf.native_transport_max_request_data_in_flight_per_ip.toBytes()) + throw new ConfigurationException("native_transport_max_message_size must not exceed native_transport_max_request_data_in_flight_per_ip", false); + + } + nativeTransportMaxMessageSizeInBytes = conf.native_transport_max_message_size.toBytes(); + // native transport encryption options if (conf.client_encryption_options != null) { @@ -2748,6 +2768,30 @@ public static long getNativeTransportMaxRequestDataInFlightPerIpInBytes() return conf.native_transport_max_request_data_in_flight_per_ip.toBytes(); } + public static long getNativeTransportMaxMessageSizeInBytes() + { + // the value of native_transport_max_message_size in bytes is cached + // to avoid conversion overhead during a parsing of each incoming CQL message + return nativeTransportMaxMessageSizeInBytes; + } + + @VisibleForTesting + public static void setNativeTransportMaxMessageSizeInBytes(long maxMessageSizeInBytes) + { + conf.native_transport_max_message_size = new DataStorageSpec.LongBytesBound(maxMessageSizeInBytes); + nativeTransportMaxMessageSizeInBytes = conf.native_transport_max_message_size.toBytes(); + } + + private static long calculateDefaultNativeTransportMaxMessageSizeInBytes() + { + return Math.min(conf.max_mutation_size.toBytes(), + Math.min( + conf.native_transport_max_request_data_in_flight.toBytes(), + conf.native_transport_max_request_data_in_flight_per_ip.toBytes() + ) + ); + } + public static Config.PaxosVariant getPaxosVariant() { return conf.paxos_variant; @@ -2874,6 +2918,10 @@ public static void setNativeTransportMaxRequestDataInFlightPerIpInBytes(long max maxRequestDataInFlightInBytes = Runtime.getRuntime().maxMemory() / 40; conf.native_transport_max_request_data_in_flight_per_ip = new DataStorageSpec.LongBytesBound(maxRequestDataInFlightInBytes); + long newNativeTransportMaxMessageSizeInBytes = nativeTransportMaxMessageSizeConfiguredExplicitly + ? Math.min(maxRequestDataInFlightInBytes, getNativeTransportMaxMessageSizeInBytes()) + : calculateDefaultNativeTransportMaxMessageSizeInBytes(); + setNativeTransportMaxMessageSizeInBytes(newNativeTransportMaxMessageSizeInBytes); } public static long getNativeTransportMaxRequestDataInFlightInBytes() @@ -2887,6 +2935,10 @@ public static void setNativeTransportConcurrentRequestDataInFlightInBytes(long m maxRequestDataInFlightInBytes = Runtime.getRuntime().maxMemory() / 10; conf.native_transport_max_request_data_in_flight = new DataStorageSpec.LongBytesBound(maxRequestDataInFlightInBytes); + long newNativeTransportMaxMessageSizeInBytes = nativeTransportMaxMessageSizeConfiguredExplicitly + ? Math.min(maxRequestDataInFlightInBytes, getNativeTransportMaxMessageSizeInBytes()) + : calculateDefaultNativeTransportMaxMessageSizeInBytes(); + setNativeTransportMaxMessageSizeInBytes(newNativeTransportMaxMessageSizeInBytes); } public static int getNativeTransportMaxRequestsPerSecond() diff --git a/src/java/org/apache/cassandra/exceptions/OversizedCQLMessageException.java b/src/java/org/apache/cassandra/exceptions/OversizedCQLMessageException.java new file mode 100644 index 000000000000..60f9c48835ec --- /dev/null +++ b/src/java/org/apache/cassandra/exceptions/OversizedCQLMessageException.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.exceptions; + +public class OversizedCQLMessageException extends InvalidRequestException +{ + public OversizedCQLMessageException(String message) + { + super(message); + } +} diff --git a/src/java/org/apache/cassandra/net/AbstractMessageHandler.java b/src/java/org/apache/cassandra/net/AbstractMessageHandler.java index e2cf68d6d1ee..5b5b8b7f1ad7 100644 --- a/src/java/org/apache/cassandra/net/AbstractMessageHandler.java +++ b/src/java/org/apache/cassandra/net/AbstractMessageHandler.java @@ -562,7 +562,7 @@ public boolean supply(Frame frame) return size == received; } - private void onIntactFrame(IntactFrame frame) + protected void onIntactFrame(IntactFrame frame) { boolean expires = approxTime.isAfter(expiresAtNanos); if (!isExpired && !isCorrupt) @@ -578,7 +578,7 @@ private void onIntactFrame(IntactFrame frame) isExpired |= expires; } - private void onCorruptFrame() + protected void onCorruptFrame() { if (!isExpired && !isCorrupt) releaseBuffersAndCapacity(); // release resources once we transition from normal state to corrupt diff --git a/src/java/org/apache/cassandra/transport/CQLMessageHandler.java b/src/java/org/apache/cassandra/transport/CQLMessageHandler.java index 65c0282908d7..792f6bf7b638 100644 --- a/src/java/org/apache/cassandra/transport/CQLMessageHandler.java +++ b/src/java/org/apache/cassandra/transport/CQLMessageHandler.java @@ -31,6 +31,7 @@ import io.netty.channel.Channel; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.exceptions.OverloadedException; +import org.apache.cassandra.exceptions.OversizedCQLMessageException; import org.apache.cassandra.metrics.ClientMessageSizeMetrics; import org.apache.cassandra.metrics.ClientMetrics; import org.apache.cassandra.net.AbstractMessageHandler; @@ -81,6 +82,10 @@ public class CQLMessageHandler<M extends Message> extends AbstractMessageHandler public static final int LARGE_MESSAGE_THRESHOLD = FrameEncoder.Payload.MAX_SIZE - 1; public static final TimeUnit RATE_LIMITER_DELAY_UNIT = TimeUnit.NANOSECONDS; + static final String MULTI_FRAME_AUTH_ERROR_MESSAGE_PREFIX = "The connection is not yet in a valid state " + + "to process multi frame CQL Messages, usually this" + + "means that authentication is still pending. "; + private final QueueBackpressure queueBackpressure; private final Envelope.Decoder envelopeDecoder; private final Message.Decoder<M> messageDecoder; @@ -94,6 +99,8 @@ public class CQLMessageHandler<M extends Message> extends AbstractMessageHandler long channelPayloadBytesInFlight; private int consecutiveMessageErrors = 0; + private final ServerConnection serverConnection; + interface MessageConsumer<M extends Message> { void dispatch(Channel channel, M message, Dispatcher.FlushItemConverter toFlushItem, Overload backpressure); @@ -106,6 +113,7 @@ interface ErrorHandler } CQLMessageHandler(Channel channel, + ServerConnection serverConnection, ProtocolVersion version, FrameDecoder decoder, Envelope.Decoder envelopeDecoder, @@ -128,6 +136,7 @@ interface ErrorHandler resources.endpointWaitQueue(), resources.globalWaitQueue(), onClosed); + this.serverConnection = serverConnection; this.envelopeDecoder = envelopeDecoder; this.messageDecoder = messageDecoder; this.payloadAllocator = payloadAllocator; @@ -518,10 +527,28 @@ protected boolean processFirstFrameOfLargeMessage(IntactFrame frame, Limit endpo // max CQL message size defaults to 256mb, so should be safe to downcast int messageSize = Ints.checkedCast(header.bodySizeInBytes); receivedBytes += buf.remaining(); + + if (serverConnection != null && serverConnection.stage() != ConnectionStage.READY) + { + // Disallow any multiframe messages before the connection reaches the READY state. + // This guards against being swamped with oversize messages from unauthenticated + // clients. In this case, we raise a fatal error and close the connection so it does + // not make sense to continue processing subsequent frames + handleError(ProtocolException.toFatalException(new OversizedAuthMessageException( + MULTI_FRAME_AUTH_ERROR_MESSAGE_PREFIX + + "type = " + header.type + ", size = " + header.bodySizeInBytes))); + ClientMetrics.instance.markRequestDiscarded(); + return false; + } LargeMessage largeMessage = new LargeMessage(header); - - if (throwOnOverload) + if (messageSize > DatabaseDescriptor.getNativeTransportMaxMessageSizeInBytes()) + { + ClientMetrics.instance.markRequestDiscarded(); + // Mark as too big so that discard the message after consuming any subsequent frames + largeMessage.markTooBig(); + } + else if (throwOnOverload) { if (!acquireCapacity(header, endpointReserve, globalReserve)) { @@ -567,9 +594,9 @@ else if (!dispatcher.hasQueueCapacity()) } } } - else + else // throwOnOverload = false { - if (acquireCapacity(header, endpointReserve, globalReserve)) + if (acquireCapacityAndQueueOnFailure(header, endpointReserve, globalReserve)) { long delay = -1; Overload backpressure = Overload.NONE; @@ -605,7 +632,14 @@ else if (!dispatcher.hasQueueCapacity()) } else { - noSpamLogger.error("Could not aquire capacity while processing native protocol message"); + // we checked previously that messageSize <= native_transport_max_message_size + // and native_transport_max_message_size <= native_transport_max_request_data_in_flight + // and native_transport_max_message_size <= native_transport_max_request_data_in_flight_per_ip + // so, a starvation is not possible for the following case: + // a connection is blocked forever if somebody tries to send a single too big message > total rate limiting capacity. + // Once other messages in the same or other CQL connections are processed and capacity is returned to the limits + // we have enough capacity to acquire it for the current large message. + return false; } } @@ -712,6 +746,7 @@ private class LargeMessage extends AbstractMessageHandler.LargeMessage<Envelope. private Overload overload = Overload.NONE; private Overload backpressure = Overload.NONE; + private boolean tooBig = false; private LargeMessage(Envelope.Header header) { @@ -747,18 +782,75 @@ private void markBackpressure(Overload backpressure) this.backpressure = backpressure; } + private void markTooBig() + { + this.tooBig = true; + } + + @Override + protected void onIntactFrame(IntactFrame frame) + { + if (tooBig || overload != Overload.NONE) + // we do not want to add the frame to buffers (to not consume a lot of memory and throw it away later + // we also do not want to release capacity because we haven't accuired it + frame.consume(); + else + super.onIntactFrame(frame); + } + + @Override + protected void onCorruptFrame() + { + if (!isExpired && !isCorrupt && !tooBig) + { + releaseBuffers(); // release resources once we transition from normal state to corrupt + if (overload != Overload.BYTES_IN_FLIGHT) + releaseCapacity(size); + } + isCorrupt = true; + isExpired |= approxTime.isAfter(expiresAtNanos); + } + + + @Override protected void onComplete() { - if (overload != Overload.NONE) - handleErrorAndRelease(buildOverloadedException(endpointReserveCapacity, globalReserveCapacity, overload), header); + if (tooBig) + // we haven't accuired a capacity for too big messages to release it + handleError(buildOversizedCQLMessageException(header.bodySizeInBytes), header); + else if (overload != Overload.NONE) + if (overload == Overload.BYTES_IN_FLIGHT) + // we haven't accuired a capacity successfully to release it + handleError(buildOverloadedException(endpointReserveCapacity, globalReserveCapacity, overload), header); + else + handleErrorAndRelease(buildOverloadedException(endpointReserveCapacity, globalReserveCapacity, overload), header); else if (!isCorrupt) processRequest(assembleFrame(), backpressure); } + @Override protected void abort() { - if (!isCorrupt) - releaseBuffersAndCapacity(); // release resources if in normal state when abort() is invoked + if (!isCorrupt && !tooBig && overload == Overload.NONE) + releaseBuffers(); + + if (overload == Overload.NONE || overload == Overload.BYTES_IN_FLIGHT) + releaseCapacity(size); + } + + private OversizedCQLMessageException buildOversizedCQLMessageException(long messageBodySize) + { + return new OversizedCQLMessageException("CQL Message of size " + messageBodySize + + " bytes exceeds allowed maximum of " + + DatabaseDescriptor.getNativeTransportMaxMessageSizeInBytes() + " bytes"); + } + } + + static class OversizedAuthMessageException extends ProtocolException + { + OversizedAuthMessageException(String message) + { + super(message); } } } diff --git a/src/java/org/apache/cassandra/transport/ExceptionHandlers.java b/src/java/org/apache/cassandra/transport/ExceptionHandlers.java index 4f063924ead7..4d36fa6cd78c 100644 --- a/src/java/org/apache/cassandra/transport/ExceptionHandlers.java +++ b/src/java/org/apache/cassandra/transport/ExceptionHandlers.java @@ -38,6 +38,7 @@ import io.netty.channel.unix.Errors; import org.apache.cassandra.exceptions.OverloadedException; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.exceptions.OversizedCQLMessageException; import org.apache.cassandra.metrics.ClientMetrics; import org.apache.cassandra.net.FrameEncoder; import org.apache.cassandra.transport.messages.ErrorMessage; @@ -130,6 +131,10 @@ else if (Throwables.anyCauseMatches(cause, t -> t instanceof OverloadedException // Once the threshold for overload is breached, it will very likely spam the logs... NoSpamLogger.log(logger, NoSpamLogger.Level.INFO, 1, TimeUnit.MINUTES, cause.getMessage()); } + else if (Throwables.anyCauseMatches(cause, t -> t instanceof OversizedCQLMessageException)) + { + NoSpamLogger.log(logger, NoSpamLogger.Level.INFO, 1, TimeUnit.MINUTES, cause.getMessage()); + } else if (Throwables.anyCauseMatches(cause, t -> t instanceof Errors.NativeIoException)) { ClientMetrics.instance.markUnknownException(); diff --git a/src/java/org/apache/cassandra/transport/InitialConnectionHandler.java b/src/java/org/apache/cassandra/transport/InitialConnectionHandler.java index 576af3e6dcb3..463d824c4c0a 100644 --- a/src/java/org/apache/cassandra/transport/InitialConnectionHandler.java +++ b/src/java/org/apache/cassandra/transport/InitialConnectionHandler.java @@ -104,6 +104,7 @@ protected void decode(ChannelHandlerContext ctx, ByteBuf buffer, List<Object> li attrConn.set(connection); } assert connection instanceof ServerConnection; + ServerConnection serverConnection = (ServerConnection) connection; StartupMessage startup = (StartupMessage) Message.Decoder.decodeMessage(ctx.channel(), inbound); InetAddress remoteAddress = ((InetSocketAddress) ctx.channel().remoteAddress()).getAddress(); @@ -120,7 +121,7 @@ protected void decode(ChannelHandlerContext ctx, ByteBuf buffer, List<Object> li if (future.isSuccess()) { logger.trace("Response to STARTUP sent, configuring pipeline for {}", inbound.header.version); - configurator.configureModernPipeline(ctx, allocator, inbound.header.version, startup.options); + configurator.configureModernPipeline(ctx, serverConnection, allocator, inbound.header.version, startup.options); allocator.release(inbound.header.bodySizeInBytes); } else diff --git a/src/java/org/apache/cassandra/transport/PipelineConfigurator.java b/src/java/org/apache/cassandra/transport/PipelineConfigurator.java index 10ca8187199b..6ad00471c72a 100644 --- a/src/java/org/apache/cassandra/transport/PipelineConfigurator.java +++ b/src/java/org/apache/cassandra/transport/PipelineConfigurator.java @@ -270,6 +270,7 @@ protected void channelIdle(ChannelHandlerContext ctx, IdleStateEvent evt) } public void configureModernPipeline(ChannelHandlerContext ctx, + ServerConnection serverConnection, ClientResourceLimits.Allocator resourceAllocator, ProtocolVersion version, Map<String, String> options) @@ -311,6 +312,7 @@ public void configureModernPipeline(ChannelHandlerContext ctx, CQLMessageHandler.MessageConsumer<Message.Request> messageConsumer = messageConsumer(); CQLMessageHandler<Message.Request> processor = new CQLMessageHandler<>(ctx.channel(), + serverConnection, version, frameDecoder, envelopeDecoder, diff --git a/src/java/org/apache/cassandra/transport/SimpleClient.java b/src/java/org/apache/cassandra/transport/SimpleClient.java index f9c9b7f5d765..a7227c89a55a 100644 --- a/src/java/org/apache/cassandra/transport/SimpleClient.java +++ b/src/java/org/apache/cassandra/transport/SimpleClient.java @@ -504,6 +504,7 @@ public void release() CQLMessageHandler<Message.Response> processor = new CQLMessageHandler<Message.Response>(ctx.channel(), + null, version, frameDecoder, envelopeDecoder, diff --git a/test/unit/org/apache/cassandra/transport/AuthMessageSizeLimitTest.java b/test/unit/org/apache/cassandra/transport/AuthMessageSizeLimitTest.java new file mode 100644 index 000000000000..5eaee511d5ab --- /dev/null +++ b/test/unit/org/apache/cassandra/transport/AuthMessageSizeLimitTest.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.transport; + +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import com.datastax.driver.core.Authenticator; +import com.datastax.driver.core.EndPoint; +import com.datastax.driver.core.PlainTextAuthProvider; +import org.apache.cassandra.Util; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.net.FrameEncoder; +import org.apache.cassandra.transport.messages.AuthResponse; +import org.apache.cassandra.transport.messages.QueryMessage; +import org.assertj.core.api.Assertions; + +public class AuthMessageSizeLimitTest extends NativeProtocolLimitsTestBase +{ + private static final int TOO_BIG_MULTI_FRAME_AUTH_MESSAGE_SIZE = 2 * FrameEncoder.Payload.MAX_SIZE; + + // set MAX_CQL_MESSAGE_SIZE bigger than TOO_BIG_MULTI_FRAME_AUTH_MESSAGE_SIZE to ensure what the auth message size check is more restrictive + private static final int MAX_CQL_MESSAGE_SIZE = TOO_BIG_MULTI_FRAME_AUTH_MESSAGE_SIZE * 2; + + @BeforeClass + public static void setUp() + { + DatabaseDescriptor.setNativeTransportReceiveQueueCapacityInBytes(1); + DatabaseDescriptor.setNativeTransportMaxRequestDataInFlightPerIpInBytes(MAX_CQL_MESSAGE_SIZE); + DatabaseDescriptor.setNativeTransportConcurrentRequestDataInFlightInBytes(MAX_CQL_MESSAGE_SIZE); + DatabaseDescriptor.setNativeTransportMaxMessageSizeInBytes(MAX_CQL_MESSAGE_SIZE); + requireNetwork(); + requireAuthentication(); + } + + @Before + public void setLimits() + { + ClientResourceLimits.setGlobalLimit(MAX_CQL_MESSAGE_SIZE); + ClientResourceLimits.setEndpointLimit(MAX_CQL_MESSAGE_SIZE); + } + + @Test + public void sendSmallAuthMessage() + { + doTest((client) -> + { + AuthResponse authResponse = createAuthMessage("cassandra", "cassandra"); + client.execute(authResponse); + createTable(client); + + int valueLessThanMessageMaxSize = MAX_CQL_MESSAGE_SIZE - 500; + QueryMessage queryMessage = queryMessage(valueLessThanMessageMaxSize); + client.execute(queryMessage); + } + ); + } + + @Test + public void sendTooBigAuthMultiFrameMessage() + { + doTest((client) -> + { + AuthResponse authResponse = createAuthMessage("cassandra", createIncorrectLongPassword(TOO_BIG_MULTI_FRAME_AUTH_MESSAGE_SIZE)); + Assertions.assertThatThrownBy(() -> client.execute(authResponse)) + .hasCauseInstanceOf(ProtocolException.class) + .hasMessageContaining(CQLMessageHandler.MULTI_FRAME_AUTH_ERROR_MESSAGE_PREFIX); + Util.spinAssertEquals(false, () -> client.connection.channel().isOpen(), 10); + } + ); + } + + private static String createIncorrectLongPassword(int length) + { + StringBuilder password = new StringBuilder(length); + for (int i = 0; i < length; i++) + password.append('a'); + return password.toString(); + } + + private AuthResponse createAuthMessage(String username, String password) + { + PlainTextAuthProvider authProvider = new PlainTextAuthProvider(username, password); + Authenticator authenticator = authProvider.newAuthenticator((EndPoint) null, null); + return new AuthResponse(authenticator.initialResponse()); + } +} diff --git a/test/unit/org/apache/cassandra/transport/ClientResourceLimitsTest.java b/test/unit/org/apache/cassandra/transport/ClientResourceLimitsTest.java index 8e9499723950..1407ab05ac8f 100644 --- a/test/unit/org/apache/cassandra/transport/ClientResourceLimitsTest.java +++ b/test/unit/org/apache/cassandra/transport/ClientResourceLimitsTest.java @@ -18,7 +18,6 @@ package org.apache.cassandra.transport; -import java.io.IOException; import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.TimeUnit; @@ -32,9 +31,6 @@ import org.junit.*; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.cql3.CQLTester; -import org.apache.cassandra.cql3.QueryOptions; -import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.db.marshal.Int32Type; import org.apache.cassandra.db.marshal.UTF8Type; import org.apache.cassandra.db.virtual.*; @@ -50,21 +46,11 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -public class ClientResourceLimitsTest extends CQLTester +public class ClientResourceLimitsTest extends NativeProtocolLimitsTestBase { private static final long LOW_LIMIT = 600L; private static final long HIGH_LIMIT = 5000000000L; - private static final QueryOptions V5_DEFAULT_OPTIONS = - QueryOptions.create(QueryOptions.DEFAULT.getConsistency(), - QueryOptions.DEFAULT.getValues(), - QueryOptions.DEFAULT.skipMetadata(), - QueryOptions.DEFAULT.getPageSize(), - QueryOptions.DEFAULT.getPagingState(), - QueryOptions.DEFAULT.getSerialConsistency(), - ProtocolVersion.V5, - KEYSPACE); - @BeforeClass public static void setUp() { @@ -89,54 +75,6 @@ public void setLimits() ClientResourceLimits.setEndpointLimit(LOW_LIMIT); } - @After - public void dropCreatedTable() - { - try - { - QueryProcessor.executeOnceInternal("DROP TABLE " + KEYSPACE + ".atable"); - } - catch (Throwable t) - { - // ignore - } - } - - @SuppressWarnings("resource") - private SimpleClient client(boolean throwOnOverload) - { - try - { - return SimpleClient.builder(nativeAddr.getHostAddress(), nativePort) - .protocolVersion(ProtocolVersion.V5) - .useBeta() - .build() - .connect(false, throwOnOverload); - } - catch (IOException e) - { - throw new RuntimeException("Error initializing client", e); - } - } - - @SuppressWarnings({"resource", "SameParameterValue"}) - private SimpleClient client(boolean throwOnOverload, int largeMessageThreshold) - { - try - { - return SimpleClient.builder(nativeAddr.getHostAddress(), nativePort) - .protocolVersion(ProtocolVersion.V5) - .useBeta() - .largeMessageThreshold(largeMessageThreshold) - .build() - .connect(false, throwOnOverload); - } - catch (IOException e) - { - throw new RuntimeException("Error initializing client", e); - } - } - @Test public void testQueryExecutionWithThrowOnOverload() { @@ -153,10 +91,8 @@ private void testQueryExecution(boolean throwOnOverload) { try (SimpleClient client = client(throwOnOverload)) { - QueryMessage queryMessage = new QueryMessage("CREATE TABLE atable (pk int PRIMARY KEY, v text)", - V5_DEFAULT_OPTIONS); - client.execute(queryMessage); - queryMessage = new QueryMessage("SELECT * FROM atable", V5_DEFAULT_OPTIONS); + createTable(client); + QueryMessage queryMessage = new QueryMessage("SELECT * FROM atable", queryOptions()); client.execute(queryMessage); } } @@ -189,7 +125,7 @@ private void backPressureTest(Runnable limitLifter, Consumer<ClientResourceLimit { // The first query does not trigger backpressure/pause the connection: QueryMessage queryMessage = - new QueryMessage("CREATE TABLE atable (pk int PRIMARY KEY, v text)", V5_DEFAULT_OPTIONS); + new QueryMessage("CREATE TABLE atable (pk int PRIMARY KEY, v text)", queryOptions()); Message.Response belowThresholdResponse = client.execute(queryMessage); assertEquals(0, getPausedConnectionsGauge().getValue().intValue()); assertNoWarningContains(belowThresholdResponse, "bytes in flight"); @@ -261,7 +197,11 @@ public void testOverloadedExceptionWhenGlobalLimitByMultiFrameMessage() { // Bump the per-endpoint limit to make sure we exhaust the global ClientResourceLimits.setEndpointLimit(HIGH_LIMIT); - testOverloadedException(() -> client(true, Ints.checkedCast(LOW_LIMIT / 2))); + // test message = 2/3 x + // emulated concurrent message = 2/3 x + // test message + emulated concurrent message = 4/3 x > x set as a global limit + emulateInFlightConcurrentMessage(LOW_LIMIT * 2 / 3); + testOverloadedException(() -> client(true, Ints.checkedCast(LOW_LIMIT / 2)), LOW_LIMIT * 2 / 3); } @Test @@ -269,18 +209,25 @@ public void testOverloadedExceptionWhenEndpointLimitByMultiFrameMessage() { // Make sure we can only exceed the per-endpoint limit ClientResourceLimits.setGlobalLimit(HIGH_LIMIT); - testOverloadedException(() -> client(true, Ints.checkedCast(LOW_LIMIT / 2))); + // test message = 2/3 x + // emulated concurrent message = 2/3 x + // test message + emulated concurrent message = 4/3 x > x set as an endpoint limit + emulateInFlightConcurrentMessage(LOW_LIMIT * 2 / 3); + testOverloadedException(() -> client(true, Ints.checkedCast(LOW_LIMIT / 2)), LOW_LIMIT * 2 / 3); } private void testOverloadedException(Supplier<SimpleClient> clientSupplier) + { + testOverloadedException(clientSupplier, LOW_LIMIT * 2); + } + + private void testOverloadedException(Supplier<SimpleClient> clientSupplier, long limit) { try (SimpleClient client = clientSupplier.get()) { - QueryMessage queryMessage = new QueryMessage("CREATE TABLE atable (pk int PRIMARY KEY, v text)", - V5_DEFAULT_OPTIONS); - client.execute(queryMessage); + createTable(client); - queryMessage = queryMessage(); + QueryMessage queryMessage = queryMessage(limit); try { client.execute(queryMessage); @@ -295,11 +242,7 @@ private void testOverloadedException(Supplier<SimpleClient> clientSupplier) private QueryMessage queryMessage() { - StringBuilder query = new StringBuilder("INSERT INTO atable (pk, v) VALUES (1, '"); - for (int i=0; i < LOW_LIMIT * 2; i++) - query.append('a'); - query.append("')"); - return new QueryMessage(query.toString(), V5_DEFAULT_OPTIONS); + return queryMessage(LOW_LIMIT * 2); } @Test @@ -341,7 +284,7 @@ public void testQueryUpdatesConcurrentMetricsUpdate() throws Throwable VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(table, ImmutableList.of(vt1))); final QueryMessage queryMessage = new QueryMessage(String.format("SELECT * FROM %s.%s", table, table), - V5_DEFAULT_OPTIONS); + queryOptions()); try { Thread tester = new Thread(() -> client.execute(queryMessage)); @@ -367,8 +310,9 @@ public void testChangingLimitsAtRuntime() try { QueryMessage smallMessage = new QueryMessage(String.format("CREATE TABLE %s.atable (pk int PRIMARY KEY, v text)", KEYSPACE), - V5_DEFAULT_OPTIONS); + queryOptions()); client.execute(smallMessage); + createTable(client); try { client.execute(queryMessage()); diff --git a/test/unit/org/apache/cassandra/transport/MessageSizeLimitTest.java b/test/unit/org/apache/cassandra/transport/MessageSizeLimitTest.java new file mode 100644 index 000000000000..8b7b2999f47e --- /dev/null +++ b/test/unit/org/apache/cassandra/transport/MessageSizeLimitTest.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.transport; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.net.FrameEncoder; +import org.apache.cassandra.transport.messages.QueryMessage; +import org.assertj.core.api.Assertions; + +public class MessageSizeLimitTest extends NativeProtocolLimitsTestBase +{ + private static final int MAX_CQL_MESSAGE_SIZE = FrameEncoder.Payload.MAX_SIZE * 3; + private static final int TOO_BIG_MESSAGE_SIZE = MAX_CQL_MESSAGE_SIZE * 2; + private static final int NORMAL_MESSAGE_SIZE = MAX_CQL_MESSAGE_SIZE - 500; + + @BeforeClass + public static void setUp() + { + DatabaseDescriptor.setNativeTransportReceiveQueueCapacityInBytes(1); + DatabaseDescriptor.setNativeTransportMaxRequestDataInFlightPerIpInBytes(MAX_CQL_MESSAGE_SIZE); + DatabaseDescriptor.setNativeTransportConcurrentRequestDataInFlightInBytes(MAX_CQL_MESSAGE_SIZE); + DatabaseDescriptor.setNativeTransportMaxMessageSizeInBytes(MAX_CQL_MESSAGE_SIZE); + requireNetwork(); + } + + @Before + public void setLimits() + { + ClientResourceLimits.setGlobalLimit(MAX_CQL_MESSAGE_SIZE); + ClientResourceLimits.setEndpointLimit(MAX_CQL_MESSAGE_SIZE); + } + + @Test + public void sendMessageWithSizeMoreThanMaxMessageSize() + { + runClientLogic((client) -> + { + QueryMessage tooBigQueryMessage = queryMessage(TOO_BIG_MESSAGE_SIZE); + Assertions.assertThatThrownBy(() -> client.execute(tooBigQueryMessage)) + .hasCauseInstanceOf(InvalidRequestException.class); + // InvalidRequestException: CQL Message of size 524362 bytes exceeds allowed maximum of 262144 bytes + + // we send one more message to check that the server continues to process new messages in the opened connection + QueryMessage queryMessage = queryMessage(NORMAL_MESSAGE_SIZE); + client.execute(queryMessage); + } + ); + } + + @Test(timeout = 30_000) + public void checkThatThereIsNoStarvationForMultiFrameMessages() throws InterruptedException + { + runClientLogic((client) -> {}, true); // to create table + AtomicInteger completedSuccessfully = new AtomicInteger(0); + int threadsCount = 2; + List<Thread> threads = new ArrayList<>(); + for (int i = 0; i < threadsCount; i++) + { + threads.add(new Thread(() -> runClientLogic((client) -> { + sendMessages(client, 100, NORMAL_MESSAGE_SIZE); + completedSuccessfully.incrementAndGet(); + }, false)) + ); + } + for (Thread thread : threads) + thread.start(); + + for (Thread thread : threads) + thread.join(); + + Assert.assertEquals("not all messages were sent successfully by all threads", + threadsCount, completedSuccessfully.get()); + } + + private void sendMessages(SimpleClient client, int messagesCount, int messageSize) + { + for (int i = 0; i < messagesCount; i++) + { + QueryMessage queryMessage1 = queryMessage(messageSize); + client.execute(queryMessage1); + } + } + + @Test + public void sendMessageWithSizeBelowLimit() + { + runClientLogic((client) -> + { + QueryMessage queryMessage = queryMessage(NORMAL_MESSAGE_SIZE); + client.execute(queryMessage); + + // run one more time, to validate that the connection is still alive + queryMessage = queryMessage(NORMAL_MESSAGE_SIZE); + client.execute(queryMessage); + } + ); + } +} diff --git a/test/unit/org/apache/cassandra/transport/NativeProtocolLimitsTestBase.java b/test/unit/org/apache/cassandra/transport/NativeProtocolLimitsTestBase.java new file mode 100644 index 000000000000..221bf97b2341 --- /dev/null +++ b/test/unit/org/apache/cassandra/transport/NativeProtocolLimitsTestBase.java @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.transport; + +import java.io.IOException; + +import org.junit.After; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.cql3.QueryProcessor; +import org.apache.cassandra.net.FrameEncoder; +import org.apache.cassandra.transport.messages.QueryMessage; +import org.apache.cassandra.utils.FBUtilities; + +public abstract class NativeProtocolLimitsTestBase extends CQLTester +{ + protected final ProtocolVersion version; + + protected long emulatedUsedCapacity; + + public NativeProtocolLimitsTestBase() + { + this(ProtocolVersion.V5); + } + + public NativeProtocolLimitsTestBase(ProtocolVersion version) + { + this.version = version; + } + + @After + public void dropCreatedTable() + { + if (emulatedUsedCapacity > 0) + { + releaseEmulatedCapacity(emulatedUsedCapacity); + } + try + { + QueryProcessor.executeOnceInternal("DROP TABLE " + KEYSPACE + ".atable"); + } + catch (Throwable t) + { + // ignore + } + } + + public QueryOptions queryOptions() + { + return QueryOptions.create(QueryOptions.DEFAULT.getConsistency(), + QueryOptions.DEFAULT.getValues(), + QueryOptions.DEFAULT.skipMetadata(), + QueryOptions.DEFAULT.getPageSize(), + QueryOptions.DEFAULT.getPagingState(), + QueryOptions.DEFAULT.getSerialConsistency(), + version, + KEYSPACE); + } + + public SimpleClient client() + { + return client(false); + } + + @SuppressWarnings({"resource", "SameParameterValue"}) + public SimpleClient client(boolean throwOnOverload) + { + return client(throwOnOverload, FrameEncoder.Payload.MAX_SIZE); + } + + @SuppressWarnings({"resource", "SameParameterValue"}) + public SimpleClient client(boolean throwOnOverload, int largeMessageThreshold) + { + try + { + return SimpleClient.builder(nativeAddr.getHostAddress(), nativePort) + .protocolVersion(version) + .useBeta() + .largeMessageThreshold(largeMessageThreshold) + .build() + .connect(false, throwOnOverload); + } + catch (IOException e) + { + throw new RuntimeException("Error initializing client", e); + } + } + + public void runClientLogic(ClientLogic clientLogic) + { + runClientLogic(clientLogic, true); + } + + public void runClientLogic(ClientLogic clientLogic, boolean createTable) + { + try (SimpleClient client = client()) + { + if (createTable) + createTable(client); + clientLogic.run(client); + } + } + + public void createTable(SimpleClient client) + { + QueryMessage queryMessage = new QueryMessage("CREATE TABLE IF NOT EXISTS " + + KEYSPACE + ".atable (pk int PRIMARY KEY, v text)", + queryOptions()); + client.execute(queryMessage); + } + + public void doTest(ClientLogic testLogic) + { + try (SimpleClient client = client()) + { + testLogic.run(client); + } + } + public interface ClientLogic + { + void run(SimpleClient simpleClient); + } + + public QueryMessage queryMessage(long valueSize) + { + StringBuilder query = new StringBuilder("INSERT INTO " + KEYSPACE + ".atable (pk, v) VALUES (1, '"); + for (int i = 0; i < valueSize; i++) + query.append('a'); + query.append("')"); + return new QueryMessage(query.toString(), queryOptions()); + } + + protected void emulateInFlightConcurrentMessage(long length) + { + ClientResourceLimits.Allocator allocator = ClientResourceLimits.getAllocatorForEndpoint(FBUtilities.getJustLocalAddress()); + ClientResourceLimits.ResourceProvider resourceProvider = new ClientResourceLimits.ResourceProvider.Default(allocator); + resourceProvider.globalLimit().allocate(length); + resourceProvider.endpointLimit().allocate(length); + emulatedUsedCapacity += length; + } + + protected void releaseEmulatedCapacity(long length) + { + ClientResourceLimits.Allocator allocator = ClientResourceLimits.getAllocatorForEndpoint(FBUtilities.getJustLocalAddress()); + ClientResourceLimits.ResourceProvider resourceProvider = new ClientResourceLimits.ResourceProvider.Default(allocator); + resourceProvider.globalLimit().release(length); + resourceProvider.endpointLimit().release(length); + } +} diff --git a/test/unit/org/apache/cassandra/transport/RateLimitingTest.java b/test/unit/org/apache/cassandra/transport/RateLimitingTest.java index 0b3fb34a79ed..1e8e16a662ac 100644 --- a/test/unit/org/apache/cassandra/transport/RateLimitingTest.java +++ b/test/unit/org/apache/cassandra/transport/RateLimitingTest.java @@ -28,7 +28,9 @@ import com.codahale.metrics.Meter; import com.google.common.base.Ticker; + import org.awaitility.Awaitility; + import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -36,12 +38,9 @@ import org.junit.runners.Parameterized; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.cql3.CQLTester; -import org.apache.cassandra.cql3.QueryOptions; import org.apache.cassandra.exceptions.OverloadedException; import org.apache.cassandra.metrics.CassandraMetricsRegistry; import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.transport.messages.QueryMessage; import org.apache.cassandra.utils.Throwables; import static org.junit.Assert.assertEquals; @@ -54,7 +53,7 @@ @SuppressWarnings("UnstableApiUsage") @RunWith(Parameterized.class) -public class RateLimitingTest extends CQLTester +public class RateLimitingTest extends NativeProtocolLimitsTestBase { public static final String BACKPRESSURE_WARNING_SNIPPET = "Request breached global limit"; @@ -63,9 +62,6 @@ public class RateLimitingTest extends CQLTester private static final long MAX_LONG_CONFIG_VALUE = Long.MAX_VALUE - 1; - @Parameterized.Parameter - public ProtocolVersion version; - @Parameterized.Parameters(name="{0}") public static Collection<Object[]> versions() { @@ -74,6 +70,11 @@ public static Collection<Object[]> versions() .collect(Collectors.toList()); } + public RateLimitingTest(ProtocolVersion version) + { + super(version); + } + private AtomicLong tick; private Ticker ticker; @@ -105,6 +106,7 @@ public long read() ClientResourceLimits.setGlobalLimit(MAX_LONG_CONFIG_VALUE); } + @Test public void shouldThrowOnOverloadSmallMessages() throws Exception { @@ -147,15 +149,19 @@ public void shouldReleaseLargeMessageOnBytesInFlightOverload() throws Exception private void testBytesInFlightOverload(int payloadSize) throws Exception { - try (SimpleClient client = client().connect(false, true)) + int emulatedConcurrentMessageSize = payloadSize * 3 / 2; + try (SimpleClient client = client(true, LARGE_PAYLOAD_THRESHOLD_BYTES)) { StorageService.instance.setNativeTransportRateLimitingEnabled(false); - QueryMessage queryMessage = new QueryMessage("CREATE TABLE IF NOT EXISTS " + KEYSPACE + ".atable (pk int PRIMARY KEY, v text)", queryOptions()); - client.execute(queryMessage); + createTable(client); StorageService.instance.setNativeTransportRateLimitingEnabled(true); ClientResourceLimits.GLOBAL_REQUEST_LIMITER.setRate(OVERLOAD_PERMITS_PER_SECOND, ticker); - ClientResourceLimits.setGlobalLimit(1); + // test message = 1x + // emulated concurrent message = 1.5x + // test message + emulated concurrent message = 2.5x > 2x set as a global limit + ClientResourceLimits.setGlobalLimit(payloadSize * 2); + emulateInFlightConcurrentMessage(emulatedConcurrentMessageSize); try { @@ -170,18 +176,17 @@ private void testBytesInFlightOverload(int payloadSize) throws Exception finally { // Sanity check bytes in flight limiter. - Awaitility.await().untilAsserted(() -> assertEquals(0, ClientResourceLimits.getCurrentGlobalUsage())); + Awaitility.await().untilAsserted(() -> assertEquals(emulatedConcurrentMessageSize, ClientResourceLimits.getCurrentGlobalUsage())); StorageService.instance.setNativeTransportRateLimitingEnabled(false); } } private void testOverload(int payloadSize, boolean throwOnOverload) throws Exception { - try (SimpleClient client = client().connect(false, throwOnOverload)) + try (SimpleClient client = client(throwOnOverload, LARGE_PAYLOAD_THRESHOLD_BYTES)) { StorageService.instance.setNativeTransportRateLimitingEnabled(false); - QueryMessage queryMessage = new QueryMessage("CREATE TABLE IF NOT EXISTS " + KEYSPACE + ".atable (pk int PRIMARY KEY, v text)", queryOptions()); - client.execute(queryMessage); + createTable(client); StorageService.instance.setNativeTransportRateLimitingEnabled(true); ClientResourceLimits.GLOBAL_REQUEST_LIMITER.setRate(OVERLOAD_PERMITS_PER_SECOND, ticker); @@ -286,40 +291,6 @@ private void testThrowOnOverload(int payloadSize, SimpleClient client) assertEquals(dispatchedPrior + 2, getRequestDispatchedMeter().getCount()); } - private QueryMessage queryMessage(int length) - { - StringBuilder query = new StringBuilder("INSERT INTO " + KEYSPACE + ".atable (pk, v) VALUES (1, '"); - - for (int i = 0; i < length; i++) - { - query.append('a'); - } - - query.append("')"); - return new QueryMessage(query.toString(), queryOptions()); - } - - private SimpleClient client() - { - return SimpleClient.builder(nativeAddr.getHostAddress(), nativePort) - .protocolVersion(version) - .useBeta() - .largeMessageThreshold(LARGE_PAYLOAD_THRESHOLD_BYTES) - .build(); - } - - private QueryOptions queryOptions() - { - return QueryOptions.create(QueryOptions.DEFAULT.getConsistency(), - QueryOptions.DEFAULT.getValues(), - QueryOptions.DEFAULT.skipMetadata(), - QueryOptions.DEFAULT.getPageSize(), - QueryOptions.DEFAULT.getPagingState(), - QueryOptions.DEFAULT.getSerialConsistency(), - version, - KEYSPACE); - } - protected static Meter getRequestDispatchedMeter() { String metricName = "org.apache.cassandra.metrics.Client.RequestDispatched"; From 64b8d6b9add607b80752cd1a8fbce51839af9ec4 Mon Sep 17 00:00:00 2001 From: Brandon Williams <brandonwilliams@apache.org> Date: Tue, 7 Jan 2025 13:50:39 -0600 Subject: [PATCH 076/225] Allow hint delivery during schema mismatch Patch by brandonwilliams; reviewed by smiklosovic for CASSANDRA-20188 --- CHANGES.txt | 1 + .../apache/cassandra/hints/HintsDispatchTrigger.java | 10 +--------- 2 files changed, 2 insertions(+), 9 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 7eb0068d7960..1ef6a902e4fc 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0.16 + * Allow hint delivery during schema mismatch (CASSANDRA-20188) * Fix gossip issue with gossip-only and bootstrapping nodes missing DC/Rack/Host ID endpoint state (CASSANDRA-19983) * IndexOutOfBoundsException when accessing partition where the column was deleted (CASSANDRA-20108) diff --git a/src/java/org/apache/cassandra/hints/HintsDispatchTrigger.java b/src/java/org/apache/cassandra/hints/HintsDispatchTrigger.java index ca38c0c319fe..fbaeaebcb803 100644 --- a/src/java/org/apache/cassandra/hints/HintsDispatchTrigger.java +++ b/src/java/org/apache/cassandra/hints/HintsDispatchTrigger.java @@ -19,20 +19,13 @@ import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.cassandra.gms.ApplicationState; -import org.apache.cassandra.gms.Gossiper; -import org.apache.cassandra.schema.Schema; - -import static org.apache.cassandra.utils.FBUtilities.getBroadcastAddressAndPort; - /** * A simple dispatch trigger that's being run every 10 seconds. * * Goes through all hint stores and schedules for dispatch all the hints for hosts that are: * 1. Not currently scheduled for dispatch, and * 2. Either have some hint files, or an active hint writer, and - * 3. Are live, and - * 4. Have matching schema versions + * 3. Are live * * What does triggering a hints store for dispatch mean? * - If there are existing hint files, it means submitting them for dispatch; @@ -65,7 +58,6 @@ public void run() .filter(store -> !isScheduled(store)) .filter(HintsStore::isLive) .filter(store -> store.isWriting() || store.hasFiles()) - .filter(store -> Schema.instance.isSameVersion(Gossiper.instance.getSchemaVersion(store.address()))) .forEach(this::schedule); } From 3c1e10b74e64b4fd9ca8bb203d503120c56f0e77 Mon Sep 17 00:00:00 2001 From: Caleb Rackliffe <calebrackliffe@gmail.com> Date: Mon, 13 Jan 2025 18:05:10 -0600 Subject: [PATCH 077/225] AST library for CQL-based fuzz tests patch by Caleb Rackliffe; reviewed by David Capwell for CASSANDRA-20198 Co-authored-by: Caleb Rackliffe <calebrackliffe@gmail.com> Co-authored-by: David Capwell <dcapwell@apache.org> --- CHANGES.txt | 1 + .../cql3/ast/AssignmentOperator.java | 122 +++ .../org/apache/cassandra/cql3/ast/Bind.java | 71 ++ .../cassandra/cql3/ast/CQLFormatter.java | 128 +++ .../cql3/ast/CQLFormatterPrettyPrintTest.java | 172 ++++ .../cassandra/cql3/ast/CasCondition.java | 66 ++ .../org/apache/cassandra/cql3/ast/Cast.java | 67 ++ .../cassandra/cql3/ast/CollectionAccess.java | 58 ++ .../cassandra/cql3/ast/Conditional.java | 437 ++++++++++ .../cassandra/cql3/ast/CreateIndexDDL.java | 271 ++++++ .../org/apache/cassandra/cql3/ast/DDL.java | 23 + .../apache/cassandra/cql3/ast/Element.java | 63 ++ .../apache/cassandra/cql3/ast/Elements.java | 32 + .../apache/cassandra/cql3/ast/Expression.java | 35 + .../cql3/ast/ExpressionEvaluator.java | 146 ++++ .../cassandra/cql3/ast/ExpressionTest.java | 125 +++ .../cassandra/cql3/ast/FunctionCall.java | 133 +++ .../apache/cassandra/cql3/ast/Literal.java | 71 ++ .../apache/cassandra/cql3/ast/Mutation.java | 817 ++++++++++++++++++ .../apache/cassandra/cql3/ast/Operator.java | 106 +++ .../apache/cassandra/cql3/ast/Reference.java | 182 ++++ .../cql3/ast/ReferenceExpression.java | 31 + .../org/apache/cassandra/cql3/ast/Select.java | 417 +++++++++ .../cassandra/cql3/ast/StandardVisitors.java | 50 ++ .../apache/cassandra/cql3/ast/Statement.java | 110 +++ .../org/apache/cassandra/cql3/ast/Symbol.java | 182 ++++ .../cassandra/cql3/ast/TableReference.java | 57 ++ .../org/apache/cassandra/cql3/ast/Txn.java | 399 +++++++++ .../apache/cassandra/cql3/ast/TypeHint.java | 84 ++ .../org/apache/cassandra/cql3/ast/Value.java | 36 + .../apache/cassandra/cql3/ast/Visitor.java | 159 ++++ 31 files changed, 4651 insertions(+) create mode 100644 test/unit/org/apache/cassandra/cql3/ast/AssignmentOperator.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/Bind.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/CQLFormatter.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/CQLFormatterPrettyPrintTest.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/CasCondition.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/Cast.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/CollectionAccess.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/Conditional.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/CreateIndexDDL.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/DDL.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/Element.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/Elements.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/Expression.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/ExpressionEvaluator.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/ExpressionTest.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/FunctionCall.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/Literal.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/Mutation.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/Operator.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/Reference.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/ReferenceExpression.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/Select.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/StandardVisitors.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/Statement.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/Symbol.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/TableReference.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/Txn.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/TypeHint.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/Value.java create mode 100644 test/unit/org/apache/cassandra/cql3/ast/Visitor.java diff --git a/CHANGES.txt b/CHANGES.txt index dddb461b5385..f4f3d72b6f0a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * AST library for CQL-based fuzz tests (CASSANDRA-20198) * Support audit logging for JMX operations (CASSANDRA-20128) * Enable sorting of nodetool status output (CASSANDRA-20104) * Support downgrading after CMS is initialized (CASSANDRA-20145) diff --git a/test/unit/org/apache/cassandra/cql3/ast/AssignmentOperator.java b/test/unit/org/apache/cassandra/cql3/ast/AssignmentOperator.java new file mode 100644 index 000000000000..f72fceb817eb --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/AssignmentOperator.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.util.EnumSet; +import java.util.stream.Stream; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.CollectionType; +import org.apache.cassandra.db.marshal.ListType; +import org.apache.cassandra.db.marshal.MapType; +import org.apache.cassandra.db.marshal.SetType; + +//TODO this is hacky as Mutation takes Expression then figures out where to put it... +// this can only be applied to UPDATE ... SET +public class AssignmentOperator implements Expression +{ + public enum Kind + { + ADD('+'), + SUBTRACT('-'); + + private final char value; + + Kind(char value) + { + this.value = value; + } + } + + public final Kind kind; + public final Expression right; + + public AssignmentOperator(Kind kind, Expression right) + { + this.kind = kind; + this.right = right; + } + + public static EnumSet<Kind> supportsOperators(AbstractType<?> type) + { + type = type.unwrap(); + EnumSet<Kind> result = EnumSet.noneOf(Kind.class); + for (Operator.Kind supported : Operator.supportsOperators(type)) + { + Kind kind = toKind(supported); + if (kind != null) + result.add(kind); + } + if (result.isEmpty()) + { + if (type instanceof CollectionType && type.isMultiCell()) + { + if (type instanceof SetType || type instanceof ListType) + return EnumSet.of(Kind.ADD, Kind.SUBTRACT); + if (type instanceof MapType) + { + // map supports subtract, but not map - map; only map - set! + // since this is annoying to support, for now dropping - + return EnumSet.of(Kind.ADD); + } + throw new AssertionError("Unexpected collection type: " + type); + } + } + return result; + } + + private static Kind toKind(Operator.Kind kind) + { + switch (kind) + { + case ADD: return Kind.ADD; + case SUBTRACT: return Kind.SUBTRACT; + default: return null; + } + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + sb.append(' ').append(kind.value).append("= "); + right.toCQL(sb, formatter); + } + + @Override + public Stream<? extends Element> stream() + { + return Stream.of(right); + } + + @Override + public AbstractType<?> type() + { + return right.type(); + } + + @Override + public Expression visit(Visitor v) + { + var u = v.visit(this); + if (u != this) return u; + var r = right.visit(v); + if (r == right) return this; + return new AssignmentOperator(kind, r); + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/Bind.java b/test/unit/org/apache/cassandra/cql3/ast/Bind.java new file mode 100644 index 000000000000..ac58dfc7df46 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/Bind.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.nio.ByteBuffer; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.Int32Type; + +public class Bind implements Value +{ + private final Object value; + private final AbstractType<?> type; + + public Bind(Object value, AbstractType<?> type) + { + this.value = value; + this.type = type; + } + + public static Bind of(int value) + { + return new Bind(value, Int32Type.instance); + } + + @Override + public Object value() + { + return value; + } + + @Override + public AbstractType<?> type() + { + return type; + } + + @Override + public ByteBuffer valueEncoded() + { + return value instanceof ByteBuffer ? (ByteBuffer) value : ((AbstractType) type).decompose(value); + } + + @Override + public Bind with(Object value, AbstractType<?> type) + { + return new Bind(value, type); + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + sb.append('?'); + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/CQLFormatter.java b/test/unit/org/apache/cassandra/cql3/ast/CQLFormatter.java new file mode 100644 index 000000000000..a163e7ceec9e --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/CQLFormatter.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import static org.apache.cassandra.cql3.ast.Elements.newLine; + +public interface CQLFormatter +{ + void group(StringBuilder sb); + void endgroup(StringBuilder sb); + + void subgroup(StringBuilder sb); + void endsubgroup(StringBuilder sb); + + void section(StringBuilder sb); + void element(StringBuilder sb); + + class PrettyPrint implements CQLFormatter + { + private static final int SPACE_PER_GROUP = 2; + private int indent; + private int subgroup = -1; + + @Override + public void group(StringBuilder sb) + { + indent += SPACE_PER_GROUP; + } + + @Override + public void endgroup(StringBuilder sb) + { + if (indent == 0) + throw new IllegalStateException("Unable to end group; more endgroup calls than group calls"); + indent -= SPACE_PER_GROUP; + } + + @Override + public void subgroup(StringBuilder sb) + { + if (subgroup != -1) + throw new IllegalStateException("Can not have nested subgroups"); + int offset = sb.lastIndexOf("\n"); + if (offset == -1) + throw new IllegalStateException("subgroup called without a previous section"); + // offset is before the current indent, so this group will already account for indent + subgroup = indent; + indent = sb.length() - offset - 1; + } + + @Override + public void endsubgroup(StringBuilder sb) + { + // this is for Txn.LET + indent = subgroup; + subgroup = -1; + } + + @Override + public void section(StringBuilder sb) + { + newLine(sb, indent); + } + + @Override + public void element(StringBuilder sb) + { + newLine(sb, indent); + } + } + + enum None implements CQLFormatter + { + instance; + + @Override + public void group(StringBuilder sb) + { + + } + + @Override + public void endgroup(StringBuilder sb) + { + + } + + @Override + public void subgroup(StringBuilder sb) + { + + } + + @Override + public void endsubgroup(StringBuilder sb) + { + + } + + @Override + public void section(StringBuilder sb) + { + sb.append(' '); + } + + @Override + public void element(StringBuilder sb) + { + sb.append(' '); + } + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/CQLFormatterPrettyPrintTest.java b/test/unit/org/apache/cassandra/cql3/ast/CQLFormatterPrettyPrintTest.java new file mode 100644 index 000000000000..f14336d7798c --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/CQLFormatterPrettyPrintTest.java @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import org.junit.Test; + +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.schema.TableMetadata; +import org.assertj.core.api.Assertions; + +public class CQLFormatterPrettyPrintTest +{ + private static final TableMetadata TBL1 = TableMetadata.builder("ks", "tbl") + .addPartitionKeyColumn("pk", Int32Type.instance) + .addClusteringColumn("ck", Int32Type.instance) + .addRegularColumn("v0", Int32Type.instance) + .addRegularColumn("v1", Int32Type.instance) + .build(); + private static final Symbol pk = new Symbol("pk", Int32Type.instance); + private static final Symbol ck = new Symbol("ck", Int32Type.instance); + private static final Symbol v0 = new Symbol("v0", Int32Type.instance); + private static final Symbol v1 = new Symbol("v1", Int32Type.instance); + + @Test + public void selectTest() + { + Select select = select(); + Assertions.assertThat(select.toCQL(format())).isEqualTo("SELECT *\n" + + "FROM ks.tbl\n" + + "WHERE foo = ?\n" + + "LIMIT ?"); + } + + @Test + public void insertTest() + { + Mutation.InsertBuilder builder = Mutation.insert(TBL1); + builder.value("pk", 0); + builder.value("v1", 0); + builder.ttl(Literal.of(42)); + Assertions.assertThat(builder.build().toCQL(format())).isEqualTo("INSERT INTO ks.tbl (pk, v1)\n" + + "VALUES (?, ?)\n" + + "USING TTL 42"); + } + + @Test + public void updatTest() + { + Mutation.UpdateBuilder builder = Mutation.update(TBL1); + builder.value("pk", 0); + builder.set("v0", 0); + builder.set("v1", 0); + builder.ttl(Literal.of(42)); + Assertions.assertThat(builder.build().toCQL(format())).isEqualTo("UPDATE ks.tbl\n" + + "USING TTL 42\n" + + "SET\n" + + " v0=?,\n" + + " v1=?\n" + + "WHERE \n" + + " pk = ?"); + + } + + @Test + public void updateWithInClause() + { + Mutation.UpdateBuilder builder = Mutation.update(TBL1); + builder.set("v0", 0); + builder.in("pk", Bind.of(42), Literal.of(78)); + Assertions.assertThat(builder.build().toCQL(format())).isEqualTo("UPDATE ks.tbl\n" + + "SET\n" + + " v0=?\n" + + "WHERE \n" + + " pk IN (?, 78)"); + } + + @Test + public void updateWithBetweenClause() + { + Mutation.UpdateBuilder builder = Mutation.update(TBL1); + builder.set("v0", 0); + builder.value("pk", 0); + builder.between("ck", Bind.of(42), Literal.of(78)); + Assertions.assertThat(builder.build().toCQL(format())).isEqualTo("UPDATE ks.tbl\n" + + "SET\n" + + " v0=?\n" + + "WHERE \n" + + " pk = ? AND \n" + + " ck BETWEEN ? AND 78"); + } + + @Test + public void updateWithIsClause() + { + Mutation.UpdateBuilder builder = Mutation.update(TBL1); + builder.set("v0", 0); + builder.value("pk", 0); + builder.is(new Symbol("ck", Int32Type.instance), + Conditional.Is.Kind.Null); + Assertions.assertThat(builder.build().toCQL(format())).isEqualTo("UPDATE ks.tbl\n" + + "SET\n" + + " v0=?\n" + + "WHERE \n" + + " pk = ? AND \n" + + " ck IS NULL"); + } + + @Test + public void deleteTest() + { + Mutation.DeleteBuilder builder = Mutation.delete(TBL1); + builder.column(v1, v0); + builder.value("pk", 0); + Assertions.assertThat(builder.build().toCQL(format())).isEqualTo("DELETE v1, v0\n" + + "FROM ks.tbl\n" + + "WHERE \n" + + " pk = ?"); + } + + @Test + public void txnTest() + { + Txn.Builder builder = Txn.builder(); + builder.addLet("a", select()); + builder.addLet("b", select()); + builder.addReturn(select()); + Assertions.assertThat(builder.build().toCQL(format())).isEqualTo("BEGIN TRANSACTION\n" + + " LET a = (SELECT *\n" + + " FROM ks.tbl\n" + + " WHERE foo = ?\n" + + " LIMIT ?);\n" + + " LET b = (SELECT *\n" + + " FROM ks.tbl\n" + + " WHERE foo = ?\n" + + " LIMIT ?);\n" + + " SELECT *\n" + + " FROM ks.tbl\n" + + " WHERE foo = ?\n" + + " LIMIT ?;\n" + + "COMMIT TRANSACTION"); + } + + private static CQLFormatter.PrettyPrint format() + { + return new CQLFormatter.PrettyPrint(); + } + + private static Select select() + { + return Select.builder() + .table("ks", "tbl") + .value("foo",42) + .limit(1) + .build(); + } +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/cql3/ast/CasCondition.java b/test/unit/org/apache/cassandra/cql3/ast/CasCondition.java new file mode 100644 index 000000000000..9f8d5867e0b9 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/CasCondition.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.util.stream.Stream; + +public interface CasCondition extends Element +{ + enum Simple implements CasCondition + { + NotExists("IF NOT EXISTS"), + Exists("IF EXISTS"); + + private final String cql; + + Simple(String s) + { + this.cql = s; + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + sb.append(cql); + } + } + + class IfCondition implements CasCondition + { + private final Conditional conditional; + + public IfCondition(Conditional conditional) + { + this.conditional = conditional; + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + sb.append("IF "); + conditional.toCQL(sb, formatter); + } + + @Override + public Stream<? extends Element> stream() + { + return Stream.of(conditional); + } + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/Cast.java b/test/unit/org/apache/cassandra/cql3/ast/Cast.java new file mode 100644 index 000000000000..433ffd13c27e --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/Cast.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.util.stream.Stream; + +import org.apache.cassandra.db.marshal.AbstractType; + +public class Cast implements Expression +{ + private final Expression e; + private final AbstractType<?> type; + + public Cast(Expression e, AbstractType<?> type) + { + this.e = e; + this.type = type; + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + sb.append("CAST ("); + e.toCQL(sb, formatter); + sb.append(" AS "); + sb.append(type.asCQL3Type()); + sb.append(")"); + } + + @Override + public Stream<? extends Element> stream() + { + return Stream.of(e); + } + + @Override + public AbstractType<?> type() + { + return type; + } + + @Override + public Expression visit(Visitor v) + { + var u = v.visit(this); + if (u != this) return u; + Expression e = this.e.visit(v); + if (e == this.e) return this; + return new Cast(e, type); + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/CollectionAccess.java b/test/unit/org/apache/cassandra/cql3/ast/CollectionAccess.java new file mode 100644 index 000000000000..19c8176f6e5b --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/CollectionAccess.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.util.stream.Stream; + +import org.apache.cassandra.db.marshal.AbstractType; + +public class CollectionAccess implements ReferenceExpression +{ + private final ReferenceExpression column; + private final ReferenceExpression element; + private final AbstractType<?> type; + + public CollectionAccess(ReferenceExpression column, ReferenceExpression element, AbstractType<?> type) + { + this.column = column; + this.element = element; + this.type = type; + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + column.toCQL(sb, formatter); + sb.append('['); + element.toCQL(sb, formatter); + sb.append(']'); + } + + @Override + public AbstractType<?> type() + { + return type; + } + + @Override + public Stream<? extends Element> stream() + { + return Stream.of(column, element); + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/Conditional.java b/test/unit/org/apache/cassandra/cql3/ast/Conditional.java new file mode 100644 index 000000000000..61241adc51a3 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/Conditional.java @@ -0,0 +1,437 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import accord.utils.Invariants; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.BooleanType; +import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.marshal.Int32Type; + +public interface Conditional extends Expression +{ + @Override + default AbstractType<?> type() + { + return BooleanType.instance; + } + + @Override + default Conditional visit(Visitor v) + { + return v.visit(this); + } + + default List<Conditional> simplify() + { + return Collections.singletonList(this); + } + + class Where implements Conditional + { + public enum Inequality + { + EQUAL("="), + NOT_EQUAL("!="), + GREATER_THAN(">"), + GREATER_THAN_EQ(">="), + LESS_THAN("<"), + LESS_THAN_EQ("<="); + + public final String value; + + Inequality(String value) + { + this.value = value; + } + } + + public final Inequality kind; + public final Expression lhs; + public final Expression rhs; + + private Where(Inequality kind, Expression lhs, Expression rhs) + { + this.kind = kind; + this.lhs = lhs; + this.rhs = rhs; + } + + public static Where create(Inequality kind, Expression ref, Expression expression) + { + return new Where(kind, ref, expression); + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + lhs.toCQL(sb, formatter); + sb.append(' ').append(kind.value).append(' '); + rhs.toCQL(sb, formatter); + } + + @Override + public Stream<? extends Element> stream() + { + return Stream.of(lhs, rhs); + } + + @Override + public Conditional visit(Visitor v) + { + var u = v.visit(this); + if (u != this) return u; + var lhs = this.lhs.visit(v); + var rhs = this.rhs.visit(v); + if (lhs == this.lhs && rhs == this.rhs) return this; + return new Where(kind, lhs, rhs); + } + } + + class Between implements Conditional + { + public final Expression ref, start, end; + + public Between(Expression ref, Expression start, Expression end) + { + this.ref = ref; + this.start = start; + this.end = end; + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + ref.toCQL(sb, formatter); + sb.append(" BETWEEN "); + start.toCQL(sb, formatter); + sb.append(" AND "); + end.toCQL(sb, formatter); + } + + @Override + public Stream<? extends Element> stream() + { + return Stream.of(ref, start, end); + } + + @Override + public Conditional visit(Visitor v) + { + var u = v.visit(this); + if (u != this) return u; + Expression ref = this.ref.visit(v); + Expression start = this.start.visit(v); + Expression end = this.end.visit(v); + if (ref == this.ref && start == this.start && end == this.end) return this; + return new Between(ref, start, end); + } + } + + class In implements Conditional + { + public final ReferenceExpression ref; + public final List<? extends Expression> expressions; + + public In(ReferenceExpression ref, List<? extends Expression> expressions) + { + Invariants.checkArgument(!expressions.isEmpty()); + this.ref = ref; + this.expressions = expressions; + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + ref.toCQL(sb, formatter); + sb.append(" IN "); + sb.append('('); + for (Expression e : expressions) + { + e.toCQL(sb, formatter); + sb.append(", "); + } + sb.setLength(sb.length() - 2); // ", " + sb.append(')'); + } + + @Override + public Stream<? extends Element> stream() + { + List<Element> es = new ArrayList<>(expressions.size() + 1); + es.add(ref); + es.addAll(expressions); + return es.stream(); + } + + @Override + public Conditional visit(Visitor v) + { + var u = v.visit(this); + if (u != this) return u; + var symbol = this.ref.visit(v); + boolean updated = symbol == this.ref; + List<Expression> expressions = new ArrayList<>(this.expressions.size()); + for (Expression e : this.expressions) + { + var e2 = e.visit(v); + updated |= e == e2; + expressions.add(e2); + } + if (!updated) return this; + return new In(symbol, expressions); + } + } + + class Is implements Conditional + { + public enum Kind + { + Null("NULL"), + NotNull("NOT NULL"); + + private final String cql; + + Kind(String s) + { + this.cql = s; + } + } + + public final ReferenceExpression reference; + public final Kind kind; + + public Is(String symbol, Kind kind) + { + this(Symbol.unknownType(symbol), kind); + } + + public Is(ReferenceExpression reference, Kind kind) + { + this.reference = reference; + this.kind = kind; + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + reference.toCQL(sb, formatter); + sb.append(" IS ").append(kind.cql); + } + } + + class And implements Conditional + { + public final Conditional left, right; + + public And(Conditional left, Conditional right) + { + this.left = left; + this.right = right; + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + left.toCQL(sb, formatter); + sb.append(" AND "); + right.toCQL(sb, formatter); + } + + @Override + public Stream<? extends Element> stream() + { + return Stream.of(left, right); + } + + @Override + public Conditional visit(Visitor v) + { + var u = v.visit(this); + if (u != this) return u; + Conditional left = this.left.visit(v); + Conditional right = this.right.visit(v); + if (this.left == left && this.right == right) return this; + return new And(left, right); + } + + @Override + public List<Conditional> simplify() + { + List<Conditional> result = new ArrayList<>(); + result.addAll(left.simplify()); + result.addAll(right.simplify()); + return result; + } + } + + interface EqBuilder<T extends EqBuilder<T>> + { + T value(Symbol symbol, Expression e); + default T value(Symbol symbol, Object value) + { + return value(symbol, new Bind(value, symbol.type())); + } + + default T value(String symbol, int e) + { + return value(new Symbol(symbol, Int32Type.instance), Bind.of(e)); + } + } + + interface ConditionalBuilder<T extends ConditionalBuilder<T>> extends EqBuilder<T> + { + + T where(Expression ref, Where.Inequality kind, Expression expression); + default T where(Expression ref, Where.Inequality kind, Object value) + { + return where(ref, kind, new Bind(value, ref.type())); + } + + default <Type> T where(String name, Where.Inequality kind, Type value, AbstractType<Type> type) + { + return where(new Symbol(name, type), kind, new Bind(value, type)); + } + + default T where(String name, Where.Inequality kind, int value) + { + return where(name, kind, value, Int32Type.instance); + } + + T between(Expression ref, Expression start, Expression end); + + default T between(String name, Expression start, Expression end) + { + return between(new Symbol(name, start.type()), start, end); + } + + T in(ReferenceExpression ref, List<? extends Expression> expressions); + + default T in(ReferenceExpression ref, Expression... expressions) + { + return in(ref, Arrays.asList(expressions)); + } + + default T in(String name, Expression... expressions) + { + if (expressions == null || expressions.length == 0) + throw new IllegalArgumentException("expressions may not be empty"); + return in(new Symbol(name, expressions[0].type()), expressions); + } + + default T in(String name, int... values) + { + return in(name, Int32Type.instance, IntStream.of(values).boxed().collect(Collectors.toList())); + } + + default <Type> T in(String name, AbstractType<Type> type, Type... values) + { + return in(name, type, Arrays.asList(values)); + } + + default <Type> T in(String name, AbstractType<Type> type, List<Type> values) + { + return in(new Symbol(name, type), values.stream().map(v -> new Bind(v, type)).collect(Collectors.toList())); + } + + T is(Symbol ref, Is.Kind kind); + + @Override + default T value(Symbol symbol, Expression e) + { + return where(symbol, Where.Inequality.EQUAL, e); + } + + default T value(String symbol, int e) + { + return value(symbol, e, Int32Type.instance); + } + + default T value(String symbol, ByteBuffer e) + { + return value(symbol, e, BytesType.instance); + } + + default <Type> T value(String symbol, Type value, AbstractType<Type> type) + { + return value(new Symbol(symbol, type), new Bind(value, type)); + } + } + + class Builder implements ConditionalBuilder<Builder> + { + private final List<Conditional> sub = new ArrayList<>(); + + public boolean isEmpty() + { + return sub.isEmpty(); + } + + private Builder add(Conditional conditional) + { + sub.add(conditional); + return this; + } + + @Override + public Builder where(Expression ref, Where.Inequality kind, Expression expression) + { + return add(Where.create(kind, ref, expression)); + } + + @Override + public Builder between(Expression ref, Expression start, Expression end) + { + return add(new Between(ref, start, end)); + } + + @Override + public Builder in(ReferenceExpression symbol, List<? extends Expression> expressions) + { + return add(new In(symbol, expressions)); + } + + @Override + public Builder is(Symbol ref, Is.Kind kind) + { + return add(new Is(ref, kind)); + } + + public Conditional build() + { + if (sub.isEmpty()) + throw new IllegalStateException("Unable to build an empty Conditional"); + if (sub.size() == 1) + return sub.get(0); + Conditional accum = sub.get(0); + for (int i = 1; i < sub.size(); i++) + accum = new And(accum, sub.get(i)); + return accum; + } + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/CreateIndexDDL.java b/test/unit/org/apache/cassandra/cql3/ast/CreateIndexDDL.java new file mode 100644 index 000000000000..252dd99bc951 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/CreateIndexDDL.java @@ -0,0 +1,271 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.util.Arrays; +import java.util.EnumSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Stream; + +import com.google.common.collect.ImmutableSet; + +import org.apache.cassandra.cql3.ast.Symbol.UnquotedSymbol; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.AsciiType; +import org.apache.cassandra.db.marshal.BooleanType; +import org.apache.cassandra.db.marshal.CompositeType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.marshal.UUIDType; +import org.apache.cassandra.index.sai.StorageAttachedIndex; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.TableMetadata; + +//TODO (now): replace with IndexMetadata? Rather than create a custom DDL type can just leverage the existing metadata like Table/Keyspace +public class CreateIndexDDL implements Element +{ + public enum Version { V1, V2 } + public enum QueryType { Eq, Range } + + public interface Indexer + { + enum Kind { legacy, sai } + Kind kind(); + default boolean isCustom() + { + return kind() != Kind.legacy; + } + UnquotedSymbol name(); + default UnquotedSymbol longName() + { + return name(); + } + boolean supported(TableMetadata table, ColumnMetadata column); + default EnumSet<QueryType> supportedQueries(AbstractType<?> type) + { + return EnumSet.of(QueryType.Eq); + } + } + + public static List<Indexer> supportedIndexers() + { + return Arrays.asList(LEGACY, SAI); + } + + private static boolean standardSupported(TableMetadata metadata, ColumnMetadata col) + { + if (metadata.partitionKeyColumns().size() == 1 && col.isPartitionKey()) return false; + AbstractType<?> type = col.type.unwrap(); + if (type.isUDT() && type.isMultiCell()) return false; // non-frozen UDTs are not supported + if (type.referencesDuration()) return false; // Duration is not allowed! See org.apache.cassandra.cql3.statements.schema.CreateIndexStatement.validateIndexTarget + return true; + } + + private static boolean isFrozen(AbstractType<?> type) + { + return !type.subTypes().isEmpty() && !type.isMultiCell(); + } + + public static final Indexer LEGACY = new Indexer() + { + + @Override + public Kind kind() + { + return Kind.legacy; + } + + @Override + public UnquotedSymbol name() + { + return new UnquotedSymbol("legacy_local_table", UTF8Type.instance); + } + + @Override + public boolean supported(TableMetadata table, ColumnMetadata column) + { + return standardSupported(table, column); + } + }; + + private static final Set<AbstractType<?>> SAI_EQ_ONLY = ImmutableSet.of(UTF8Type.instance, AsciiType.instance, + BooleanType.instance, + UUIDType.instance); + + public static final Indexer SAI = new Indexer() + { + @Override + public Kind kind() + { + return Kind.sai; + } + + @Override + public UnquotedSymbol name() + { + return new UnquotedSymbol("SAI", UTF8Type.instance); + } + + @Override + public UnquotedSymbol longName() + { + return new UnquotedSymbol("StorageAttachedIndex", UTF8Type.instance); + } + + @Override + public boolean supported(TableMetadata table, ColumnMetadata col) + { + if (!standardSupported(table, col)) return false; + AbstractType<?> type = col.type.unwrap(); + if (type instanceof CompositeType) + { + // each element must be SUPPORTED_TYPES only... + if (type.subTypes().stream().allMatch(StorageAttachedIndex.SUPPORTED_TYPES::contains)) + return true; + } + else if (((isFrozen(type) && !type.isVector()) || StorageAttachedIndex.SUPPORTED_TYPES.contains(type.asCQL3Type()))) + return true; + return false; + } + + @Override + public EnumSet<QueryType> supportedQueries(AbstractType<?> type) + { + type = type.unwrap(); + if (SAI_EQ_ONLY.contains(type)) + return EnumSet.of(QueryType.Eq); + return EnumSet.allOf(QueryType.class); + } + }; + + public final Version version; + public final Indexer indexer; + public final Optional<Symbol> name; + public final TableReference on; + public final List<ReferenceExpression> references; + public final Map<String, String> options; + + public CreateIndexDDL(Version version, Indexer indexer, Optional<Symbol> name, TableReference on, List<ReferenceExpression> references, Map<String, String> options) + { + this.version = version; + this.indexer = indexer; + this.name = name; + this.on = on; + this.references = references; + this.options = options; + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + switch (version) + { + case V1: + if (indexer.isCustom()) sb.append("CREATE CUSTOM INDEX"); + else sb.append("CREATE INDEX"); + break; + case V2: + sb.append("CREATE INDEX"); + break; + default: + throw new UnsupportedOperationException(version.name()); + } + if (name.isPresent()) + { + sb.append(' '); + name.get().toCQL(sb, formatter); + } + formatter.section(sb); + sb.append("ON "); + on.toCQL(sb, formatter); + sb.append('('); + for (ReferenceExpression ref : references) + { + ref.toCQL(sb, formatter); + sb.append(", "); + } + sb.setLength(sb.length() - 2); // remove last ", " + sb.append(')'); + UnquotedSymbol indexerName = null; + switch (version) + { + case V1: + if (indexer.isCustom()) + indexerName = indexer.longName(); + break; + case V2: + indexerName = indexer.name(); + break; + default: + throw new UnsupportedOperationException(version.name()); + } + if (indexerName != null) + { + formatter.section(sb); + sb.append("USING '"); + indexerName.toCQL(sb, formatter); + sb.append("'"); + } + if (!options.isEmpty()) + { + formatter.section(sb); + sb.append("WITH OPTIONS = {"); + for (Map.Entry<String, String> e : options.entrySet()) + sb.append("'").append(e.getKey()).append("': '").append(e.getValue()).append("', "); + sb.setLength(sb.length() - 2); // remove ", " + sb.append('}'); + } + } + + public static class CollectionReference implements ReferenceExpression + { + public enum Kind { FULL, KEYS, ENTRIES } + + public final Kind kind; + public final ReferenceExpression column; + + public CollectionReference(Kind kind, ReferenceExpression column) + { + this.kind = kind; + this.column = column; + } + + @Override + public AbstractType<?> type() + { + return column.type(); + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + sb.append(kind.name()).append('('); + column.toCQL(sb, formatter); + sb.append(')'); + } + + @Override + public Stream<? extends Element> stream() + { + return Stream.of(column); + } + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/DDL.java b/test/unit/org/apache/cassandra/cql3/ast/DDL.java new file mode 100644 index 000000000000..7575bec7a4af --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/DDL.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +public interface DDL extends Statement +{ +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/Element.java b/test/unit/org/apache/cassandra/cql3/ast/Element.java new file mode 100644 index 000000000000..05bcd92ce550 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/Element.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.util.stream.Stream; + +public interface Element +{ + void toCQL(StringBuilder sb, CQLFormatter formatter); + + default void toCQL(StringBuilder sb) + { + toCQL(sb, CQLFormatter.None.instance); + } + + default String toCQL() + { + StringBuilder sb = new StringBuilder(); + toCQL(sb, CQLFormatter.None.instance); + return sb.toString(); + } + + default String toCQL(CQLFormatter formatter) + { + StringBuilder sb = new StringBuilder(); + toCQL(sb, formatter); + return sb.toString(); + } + + default Stream<? extends Element> stream() + { + return Stream.empty(); + } + + default Stream<? extends Element> streamRecursive() + { + return streamRecursive(false); + } + + default Stream<? extends Element> streamRecursive(boolean includeSelf) + { + Stream<Element> stream = stream().flatMap(e -> Stream.concat(Stream.of(e), e.streamRecursive())); + if (includeSelf) + stream = Stream.concat(Stream.of(this), stream); + return stream; + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/Elements.java b/test/unit/org/apache/cassandra/cql3/ast/Elements.java new file mode 100644 index 000000000000..f750c4efc10d --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/Elements.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +public final class Elements +{ + private Elements() + {} + + public static void newLine(StringBuilder sb, int indent) + { + sb.append('\n'); + for (int i = 0; i < indent; i++) + sb.append(' '); + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/Expression.java b/test/unit/org/apache/cassandra/cql3/ast/Expression.java new file mode 100644 index 000000000000..96bc41a78c9e --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/Expression.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import org.apache.cassandra.db.marshal.AbstractType; + +public interface Expression extends Element +{ + AbstractType<?> type(); + default String name() + { + return toCQL(); + } + + default Expression visit(Visitor v) + { + return v.visit(this); + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/ExpressionEvaluator.java b/test/unit/org/apache/cassandra/cql3/ast/ExpressionEvaluator.java new file mode 100644 index 000000000000..8270b438f46c --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/ExpressionEvaluator.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.Optional; + +import org.apache.cassandra.db.marshal.AbstractType; + +import static java.util.Optional.of; + +public class ExpressionEvaluator +{ + public static Optional<Object> tryEval(Expression e) + { + if (e instanceof Value) + return of(((Value) e).value()); + if (e instanceof TypeHint) + return tryEval(((TypeHint) e).e); + if (e instanceof Operator) + return tryEval((Operator) e); + return Optional.empty(); + } + + public static Optional<Object> tryEval(Operator e) + { + switch (e.kind) + { + case ADD: + { + var lhsOpt = tryEval(e.left); + var rhsOpt = tryEval(e.right); + if (lhsOpt.isEmpty() || rhsOpt.isEmpty()) + return Optional.empty(); + Object lhs = lhsOpt.get(); + Object rhs = rhsOpt.get(); + if (lhs instanceof Byte) + return of((byte) (((Byte) lhs) + ((Byte) rhs))); + if (lhs instanceof Short) + return of((short) (((Short) lhs) + ((Short) rhs))); + if (lhs instanceof Integer) + return of((int) (((Integer) lhs) + ((Integer) rhs))); + if (lhs instanceof Long) + return of((long) (((Long) lhs) + ((Long) rhs))); + if (lhs instanceof Float) + return of((float) (((Float) lhs) + ((Float) rhs))); + if (lhs instanceof Double) + return of((double) (((Double) lhs) + ((Double) rhs))); + if (lhs instanceof BigInteger) + return of(((BigInteger) lhs).add((BigInteger) rhs)); + if (lhs instanceof BigDecimal) + return of(((BigDecimal) lhs).add((BigDecimal) rhs)); + if (lhs instanceof String) + return of(lhs.toString() + rhs.toString()); + throw new UnsupportedOperationException("Unexpected type: " + lhs.getClass()); + } + case SUBTRACT: + { + var lhsOpt = tryEval(e.left); + var rhsOpt = tryEval(e.right); + if (lhsOpt.isEmpty() || rhsOpt.isEmpty()) + return Optional.empty(); + Object lhs = lhsOpt.get(); + Object rhs = rhsOpt.get(); + if (lhs instanceof Byte) + return of((byte) (((Byte) lhs) - ((Byte) rhs))); + if (lhs instanceof Short) + return of((short) (((Short) lhs) - ((Short) rhs))); + if (lhs instanceof Integer) + return of((int) (((Integer) lhs) - ((Integer) rhs))); + if (lhs instanceof Long) + return of((long) (((Long) lhs) - ((Long) rhs))); + if (lhs instanceof Float) + return of((float) (((Float) lhs) - ((Float) rhs))); + if (lhs instanceof Double) + return of((double) (((Double) lhs) - ((Double) rhs))); + if (lhs instanceof BigInteger) + return of(((BigInteger) lhs).subtract((BigInteger) rhs)); + if (lhs instanceof BigDecimal) + return of(((BigDecimal) lhs).subtract((BigDecimal) rhs)); + throw new UnsupportedOperationException("Unexpected type: " + lhs.getClass()); + } + case MULTIPLY: + { + var lhsOpt = tryEval(e.left); + var rhsOpt = tryEval(e.right); + if (lhsOpt.isEmpty() || rhsOpt.isEmpty()) + return Optional.empty(); + Object lhs = lhsOpt.get(); + Object rhs = rhsOpt.get(); + if (lhs instanceof Byte) + return of((byte) (((Byte) lhs) * ((Byte) rhs))); + if (lhs instanceof Short) + return of((short) (((Short) lhs) * ((Short) rhs))); + if (lhs instanceof Integer) + return of((int) (((Integer) lhs) * ((Integer) rhs))); + if (lhs instanceof Long) + return of((long) (((Long) lhs) * ((Long) rhs))); + if (lhs instanceof Float) + return of((float) (((Float) lhs) * ((Float) rhs))); + if (lhs instanceof Double) + return of((double) ((Double) lhs) * ((Double) rhs)); + if (lhs instanceof BigInteger) + return of(((BigInteger) lhs).multiply((BigInteger) rhs)); + if (lhs instanceof BigDecimal) + return of(((BigDecimal) lhs).multiply((BigDecimal) rhs)); + throw new UnsupportedOperationException("Unexpected type: " + lhs.getClass()); + } + default: + throw new UnsupportedOperationException(e.kind.name()); + } + } + + public static Optional<ByteBuffer> tryEvalEncoded(Expression e) + { + return tryEval(e).map(v -> { + if (v instanceof ByteBuffer) return (ByteBuffer) v; + try + { + return ((AbstractType) e.type()).decompose(v); + } + catch (Throwable t) + { + throw t; + } + }); + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/ExpressionTest.java b/test/unit/org/apache/cassandra/cql3/ast/ExpressionTest.java new file mode 100644 index 000000000000..333f3dec8e1d --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/ExpressionTest.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import org.junit.Test; + +import accord.utils.Gen; +import accord.utils.Gens; +import org.apache.cassandra.cql3.ast.Conditional.And; +import org.apache.cassandra.cql3.ast.Conditional.Where; +import org.apache.cassandra.db.marshal.Int32Type; +import org.assertj.core.api.Assertions; + +import static accord.utils.Property.qt; + +public class ExpressionTest +{ + private static final Visitor.CompositeVisitor COMPOSITE_VISITOR = Visitor.CompositeVisitor.of(StandardVisitors.BIND_TO_LITERAL); + + @Test + public void simple() + { + qt().forAll(Gens.random(), expressions()).check((rs, ex) -> { + Visitor v = rs.nextBoolean() ? StandardVisitors.BIND_TO_LITERAL : COMPOSITE_VISITOR; + Expression update = ex.visit(v); + assertNoBind(update); + }); + } + + @Test + public void select() + { + assertNoBind(selectWithBind().visit(StandardVisitors.BIND_TO_LITERAL)); + } + + @Test + public void txn() + { + Txn txn = Txn.builder() + .addLet("a", selectWithBind()) + .addReturn(selectWithBind()) + .build(); + assertNoBind(txn.visit(StandardVisitors.BIND_TO_LITERAL)); + } + + private static Select selectWithBind() + { + return Select.builder() + .table("ks", "tbl") + .value("foo", 42) + .limit(1) + .build(); + } + + private static void assertNoBind(Element update) + { + Assertions.assertThat(update + .streamRecursive(true) + .filter(e -> e instanceof Bind) + .count()).isEqualTo(0); + } + + private static Gen<Conditional> conditionals() + { + return rs -> { + // in / where / and + switch (rs.nextInt(0, 2)) + { + case 0: + return new Conditional.In(new Symbol("col", Int32Type.instance), Gens.lists(expressions()).ofSizeBetween(1, 3).next(rs)); + case 1: + return Where.create(Where.Inequality.EQUAL, new Symbol("col", Int32Type.instance), expressions().next(rs)); + case 2: + { + var gen = conditionals(); + return new And(gen.next(rs), gen.next(rs)); + } + default: + throw new UnsupportedOperationException(); + } + }; + } + + private static Gen<Expression> expressions() + { + return rs -> { + var subExpression = expressions().filter(e -> e.type() == Int32Type.instance); + if (rs.decide(.1)) + { + // operator + return new Operator(Operator.Kind.ADD, subExpression.next(rs), subExpression.next(rs)); + } + if (rs.decide(.1)) + { + return new AssignmentOperator(AssignmentOperator.Kind.ADD, subExpression.next(rs)); + } + if (rs.decide(.1)) + return new Cast(subExpression.next(rs), Int32Type.instance); + if (rs.decide(.1)) + return new FunctionCall("fn", Gens.lists(subExpression).ofSizeBetween(1, 10).next(rs), Int32Type.instance); + if (rs.decide(.1)) + return new TypeHint(subExpression.next(rs)); + if (rs.decide(.1)) + return conditionals().next(rs); + int value = rs.nextInt(); + return rs.nextBoolean() ? Literal.of(value) : Bind.of(value); + }; + } +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/cql3/ast/FunctionCall.java b/test/unit/org/apache/cassandra/cql3/ast/FunctionCall.java new file mode 100644 index 000000000000..afd8bac2a3c8 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/FunctionCall.java @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Stream; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.marshal.LongType; + +public class FunctionCall implements Expression +{ + public final String name; + public final List<? extends Expression> arguments; + public final AbstractType<?> returnType; + + public FunctionCall(String name, List<? extends Expression> arguments, AbstractType<?> returnType) + { + this.name = name; + this.arguments = arguments; + this.returnType = returnType; + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + sb.append(name).append('('); + for (Expression e : arguments) + { + e.toCQL(sb, formatter); + sb.append(", "); + } + if (!arguments.isEmpty()) + sb.setLength(sb.length() - 2); + sb.append(')'); + } + + @Override + public Stream<? extends Element> stream() + { + return arguments.stream(); + } + + @Override + public AbstractType<?> type() + { + return returnType; + } + + @Override + public String name() + { + return name; + } + + @Override + public Expression visit(Visitor v) + { + var u = v.visit(this); + if (u != this) return u; + boolean updated = false; + List<Expression> as = new ArrayList<>(arguments.size()); + for (Expression e : arguments) + { + u = e.visit(v); + if (u != e) + updated = true; + as.add(u); + } + if (!updated) return this; + return new FunctionCall(name, as, returnType); + } + + public static FunctionCall writetime(String column, AbstractType<?> type) + { + return new FunctionCall("writetime", Collections.singletonList(new Symbol(column, type)), LongType.instance); + } + + public static FunctionCall countStar() + { + return new FunctionCall("count(*)", Collections.emptyList(), LongType.instance); + } + + public static FunctionCall count(String symbol) + { + return count(Symbol.unknownType(symbol)); + } + + public static FunctionCall count(Symbol symbol) + { + return new FunctionCall("count", Collections.singletonList(symbol), LongType.instance); + } + + public static FunctionCall tokenByColumns(Symbol... columns) + { + return tokenByColumns(Arrays.asList(columns)); + } + + public static FunctionCall tokenByColumns(List<Symbol> columns) + { + return new FunctionCall("token", columns, BytesType.instance); + } + + public static FunctionCall tokenByValue(Value... values) + { + return tokenByValue(Arrays.asList(values)); + } + + public static FunctionCall tokenByValue(List<? extends Value> values) + { + return new FunctionCall("token", values, BytesType.instance); + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/Literal.java b/test/unit/org/apache/cassandra/cql3/ast/Literal.java new file mode 100644 index 000000000000..ad8e3ddc7448 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/Literal.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.nio.ByteBuffer; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.Int32Type; + +public class Literal implements Value +{ + private final Object value; + private final AbstractType<?> type; + + public Literal(Object value, AbstractType<?> type) + { + this.value = value; + this.type = type; + } + + public static Literal of(int value) + { + return new Literal(value, Int32Type.instance); + } + + @Override + public AbstractType<?> type() + { + return type; + } + + @Override + public Object value() + { + return value; + } + + @Override + public ByteBuffer valueEncoded() + { + return value instanceof ByteBuffer ? (ByteBuffer) value : ((AbstractType) type).decompose(value); + } + + @Override + public Literal with(Object value, AbstractType<?> type) + { + return new Literal(value, type); + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + sb.append(type.asCQL3Type().toCQLLiteral(valueEncoded())); + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/Mutation.java b/test/unit/org/apache/cassandra/cql3/ast/Mutation.java new file mode 100644 index 000000000000..729bb8127a5a --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/Mutation.java @@ -0,0 +1,817 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Stream; +import javax.annotation.Nullable; + +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.TableMetadata; + +public abstract class Mutation implements Statement +{ + public enum Kind + {INSERT, UPDATE, DELETE} + + public final Kind kind; + public final TableReference table; + + protected Mutation(Kind kind, TableReference table) + { + this.kind = kind; + this.table = table; + } + + public abstract boolean isCas(); + + public final Kind mutationKind() + { + return kind; + } + + @Override + public final Statement.Kind kind() + { + return Statement.Kind.MUTATION; + } + + @Override + public String toString() + { + return detailedToString(); + } + + public static InsertBuilder insert(TableMetadata metadata) + { + return new InsertBuilder(metadata); + } + + public static UpdateBuilder update(TableMetadata metadata) + { + return new UpdateBuilder(metadata); + } + + public static DeleteBuilder delete(TableMetadata metadata) + { + return new DeleteBuilder(metadata); + } + + public static LinkedHashSet<Symbol> toSet(Iterable<ColumnMetadata> columns) + { + LinkedHashSet<Symbol> set = new LinkedHashSet<>(); + for (var col : columns) + set.add(Symbol.from(col)); + return set; + } + + private static void toCQLWhere(Conditional conditional, StringBuilder sb, CQLFormatter formatter) + { + formatter.section(sb); + sb.append("WHERE "); + List<Conditional> where = conditional.simplify(); + formatter.group(sb); + for (var c : where) + { + formatter.element(sb); + c.toCQL(sb, formatter); + sb.append(" AND "); + } + sb.setLength(sb.length() - 5); // 5 = " AND " + formatter.endgroup(sb); + } + + public static class TTL implements Element + { + public final Value value; + + public TTL(Value value) + { + this.value = value; + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + sb.append("TTL "); + value.toCQL(sb, formatter); + } + + @Override + public Stream<? extends Element> stream() + { + return Stream.of(value); + } + } + + public static class Timestamp implements Element + { + public final Value value; + + public Timestamp(Value value) + { + this.value = value; + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + sb.append("TIMESTAMP "); + value.toCQL(sb, formatter); + } + + @Override + public Stream<? extends Element> stream() + { + return Stream.of(value); + } + } + + public static class Using implements Element + { + public final Optional<TTL> ttl; + public final Optional<Timestamp> timestamp; + + public Using(Optional<TTL> ttl, Optional<Timestamp> timestamp) + { + this.ttl = ttl; + this.timestamp = timestamp; + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + if (ttl.isEmpty() && timestamp.isEmpty()) + return; + sb.append("USING "); + if (ttl.isPresent()) + ttl.get().toCQL(sb, formatter); + if (timestamp.isPresent()) + { + if (ttl.isPresent()) + sb.append(" AND "); + timestamp.get().toCQL(sb, formatter); + } + } + } + + public static class Insert extends Mutation + { + public final LinkedHashMap<Symbol, Expression> values; + public final boolean ifNotExists; + public final Optional<Using> using; + + public Insert(TableReference table, LinkedHashMap<Symbol, Expression> values, boolean ifNotExists, Optional<Using> using) + { + super(Mutation.Kind.INSERT, table); + this.values = values; + this.ifNotExists = ifNotExists; + this.using = using; + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + // INSERT INTO [keyspace_name.] table_name (column_list) + sb.append("INSERT INTO "); + table.toCQL(sb, formatter); + sb.append(" ("); + for (Symbol col : values.keySet()) + { + col.toCQL(sb, formatter); + sb.append(", "); + } + sb.setLength(sb.length() - 2); // 2 = ", " + sb.append(")"); + // VALUES (column_values) + formatter.section(sb); + sb.append("VALUES ("); + for (Expression e : values.values()) + { + e.toCQL(sb, formatter); + sb.append(", "); + } + sb.setLength(sb.length() - 2); // 2 = ", " + sb.append(")"); + // [IF NOT EXISTS] + if (ifNotExists) + { + formatter.section(sb); + sb.append("IF NOT EXISTS"); + } + // [USING TTL seconds | TIMESTAMP epoch_in_microseconds] + if (using.isPresent()) + { + formatter.section(sb); + using.get().toCQL(sb, formatter); + } + } + + @Override + public Stream<? extends Element> stream() + { + List<Element> elements = new ArrayList<>(1 + (using.isPresent() ? 1 : 0) + (values.size() * 2)); + elements.add(table); + elements.addAll(values.keySet()); + elements.addAll(values.values()); + if (using.isPresent()) + elements.add(using.get()); + return elements.stream(); + } + + @Override + public Statement visit(Visitor v) + { + var u = v.visit(this); + if (u != this) return u; + boolean updated = false; + LinkedHashMap<Symbol, Expression> copied = new LinkedHashMap<>(values.size()); + for (var e : values.entrySet()) + { + Symbol s = e.getKey(); + Symbol s2 = s.visit("INSERT", v); + if (s != s2) + updated = true; + Expression ex = e.getValue(); + Expression ex2 = ex.visit(v); + if (ex != ex2) + updated = true; + copied.put(s2, ex2); + } + if (!updated) return this; + return new Insert(table, copied, ifNotExists, using); + } + + @Override + public boolean isCas() + { + return ifNotExists; + } + } + + public static class Update extends Mutation + { + public final Optional<Using> using; + public final LinkedHashMap<Symbol, Expression> set; + public final Conditional where; + public final Optional<? extends CasCondition> casCondition; + + public Update(TableReference table, Optional<Using> using, LinkedHashMap<Symbol, Expression> set, Conditional where, Optional<? extends CasCondition> casCondition) + { + super(Mutation.Kind.UPDATE, table); + this.using = using; + this.set = set; + this.where = where; + this.casCondition = casCondition; + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + // UPDATE [keyspace_name.] table_name + sb.append("UPDATE "); + table.toCQL(sb, formatter); + // [USING TTL time_value | USING TIMESTAMP timestamp_value] + if (using.isPresent()) + { + formatter.section(sb); + using.get().toCQL(sb, formatter); + } + // SET assignment [, assignment] . . . + formatter.section(sb); + sb.append("SET"); + formatter.group(sb); + for (var e : set.entrySet()) + { + formatter.element(sb); + e.getKey().toCQL(sb, formatter); + // when a AssignmentOperator the `=` is added there so don't add + //TODO this is super hacky... + if (!(e.getValue() instanceof AssignmentOperator)) + sb.append('='); + e.getValue().toCQL(sb, formatter); + sb.append(","); + } + sb.setLength(sb.length() - 1); // 1 = "," + formatter.endgroup(sb); + // WHERE row_specification + toCQLWhere(this.where, sb, formatter); + // [IF EXISTS | IF condition [AND condition] . . .] ; + if (casCondition.isPresent()) + { + formatter.section(sb); + casCondition.get().toCQL(sb, formatter); + } + } + + @Override + public Stream<? extends Element> stream() + { + List<Element> elements = new ArrayList<>(2 + + (using.isPresent() ? 1 : 0) + + (casCondition.isPresent() ? 1 : 0) + + (set.size() * 2)); + elements.add(table); + if (using.isPresent()) + elements.add(using.get()); + for (var e : set.entrySet()) + { + elements.add(e.getKey()); + elements.add(e.getValue()); + } + elements.add(where); + if (casCondition.isPresent()) + elements.add(casCondition.get()); + return elements.stream(); + } + + @Override + public Statement visit(Visitor v) + { + var u = v.visit(this); + if (u != this) return u; + boolean updated = false; + LinkedHashMap<Symbol, Expression> copied = new LinkedHashMap<>(set.size()); + for (var e : set.entrySet()) + { + Symbol s = e.getKey().visit("UPDATE", v); + if (s != e.getKey()) + updated = true; + Expression ex = e.getValue().visit(v); + if (e.getValue() != ex) + updated = true; + copied.put(s, ex); + } + Conditional copiedWhere = where.visit(v); + if (where != copiedWhere) + updated = true; + + if (!updated) return this; + return new Update(table, using, copied, copiedWhere, casCondition); + } + + @Override + public boolean isCas() + { + return casCondition.isPresent(); + } + } + + public static class Delete extends Mutation + { + public final List<Symbol> columns; + public final Optional<Timestamp> using; + public final Conditional where; + public final Optional<? extends CasCondition> casCondition; + + public Delete(List<Symbol> columns, + TableReference table, + Optional<Timestamp> using, + Conditional where, + Optional<? extends CasCondition> casCondition) + { + super(Mutation.Kind.DELETE, table); + this.columns = columns; + this.using = using; + this.where = where; + this.casCondition = casCondition; + } + + /* +DELETE [column_name (term)][, ...] +FROM [keyspace_name.] table_name +[USING TIMESTAMP timestamp_value] +WHERE PK_column_conditions +[IF EXISTS | IF static_column_conditions] + */ + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + // DELETE [column_name (term)][, ...] + sb.append("DELETE"); + if (!columns.isEmpty()) + { + sb.append(" "); + for (var c : columns) + { + c.toCQL(sb, formatter); + sb.append(", "); + } + sb.setLength(sb.length() - 2); // 2 = ", " + } + // FROM [keyspace_name.] table_name + formatter.section(sb); + sb.append("FROM "); + table.toCQL(sb, formatter); + // [USING TIMESTAMP timestamp_value] + if (using.isPresent()) + { + formatter.section(sb); + using.get().toCQL(sb, formatter); + } + // WHERE PK_column_conditions + toCQLWhere(this.where, sb, formatter); + // [IF EXISTS | IF static_column_conditions] + if (casCondition.isPresent()) + { + formatter.section(sb); + casCondition.get().toCQL(sb, formatter); + } + } + + @Override + public Stream<? extends Element> stream() + { + List<Element> elements = new ArrayList<>(columns.size() + 4); + elements.addAll(columns); + elements.add(table); + if (using.isPresent()) + elements.add(using.get()); + elements.add(where); + if (casCondition.isPresent()) + elements.add(casCondition.get()); + return elements.stream(); + } + + @Override + public Statement visit(Visitor v) + { + var u = v.visit(this); + if (u != this) + return u; + boolean updated = false; + List<Symbol> copiedColumns = new ArrayList<>(columns.size()); + for (var s : columns) + { + var s2 = s.visit("DELETE", v); + if (s != s2) + updated = true; + copiedColumns.add(s2); + } + var copiedWhere = where.visit(v); + if (copiedWhere != where) + updated = true; + + if (!updated) return this; + return new Delete(copiedColumns, table, using, copiedWhere, casCondition); + } + + @Override + public boolean isCas() + { + return casCondition.isPresent(); + } + } + + public static abstract class BaseBuilder<T, B extends BaseBuilder<T, B>> + { + private final Kind kind; + private final TableMetadata metadata; + protected final LinkedHashSet<Symbol> partitionColumns, clusteringColumns, primaryColumns, regularAndStatic, allColumns; + private boolean includeKeyspace = true; + private final Set<Symbol> neededPks = new HashSet<>(); + + protected BaseBuilder(Kind kind, TableMetadata table) + { + this.kind = kind; + this.metadata = table; + + // partition key is always required, so validate + this.partitionColumns = toSet(table.partitionKeyColumns()); + this.clusteringColumns = toSet(table.clusteringColumns()); + this.primaryColumns = new LinkedHashSet<>(); + this.primaryColumns.addAll(partitionColumns); + this.primaryColumns.addAll(clusteringColumns); + this.regularAndStatic = new LinkedHashSet<>(); + this.regularAndStatic.addAll(toSet(table.regularAndStaticColumns())); + this.allColumns = toSet(safeColumns(table)); + neededPks.addAll(partitionColumns); + } + + public static List<ColumnMetadata> safeColumns(TableMetadata metadata) + { + List<ColumnMetadata> columns = new ArrayList<>(metadata.columns().size()); + metadata.allColumnsInSelectOrder().forEachRemaining(columns::add); + return columns; + } + + public abstract T build(); + + protected void assertAllPksHaveEq() + { + if (neededPks.isEmpty()) + return; + throw new IllegalStateException("Attempted to create a " + kind + " but not all partition columns have an equality condition; missing " + neededPks); + } + + protected void maybePkEq(Expression symbol) + { + if (symbol instanceof Symbol) + pkEq((Symbol) symbol); + } + + private void pkEq(Symbol symbol) + { + neededPks.remove(symbol); + } + + public B includeKeyspace(boolean value) + { + this.includeKeyspace = value; + return (B) this; + } + + public B includeKeyspace() + { + return includeKeyspace(true); + } + + public B excludeKeyspace() + { + return includeKeyspace(false); + } + + protected TableReference tableRef() + { + return includeKeyspace ? TableReference.from(metadata) : new TableReference(metadata.name); + } + } + + public static class InsertBuilder extends BaseBuilder<Insert, InsertBuilder> implements Conditional.EqBuilder<InsertBuilder> + { + private final LinkedHashMap<Symbol, Expression> values = new LinkedHashMap<>(); + private boolean ifNotExists = false; + private @Nullable TTL ttl; + private @Nullable Timestamp timestamp; + + protected InsertBuilder(TableMetadata table) + { + super(Kind.INSERT, table); + } + + public InsertBuilder ifNotExists() + { + ifNotExists = true; + return this; + } + + public InsertBuilder timestamp(Value value) + { + this.timestamp = new Timestamp(value); + return this; + } + + public InsertBuilder ttl(Value value) + { + this.ttl = new TTL(value); + return this; + } + + public InsertBuilder ttl(int value) + { + return ttl(Bind.of(value)); + } + + @Override + public InsertBuilder value(Symbol ref, Expression e) + { + maybePkEq(ref); + values.put(ref, e); + return this; + } + + @Override + public Insert build() + { + assertAllPksHaveEq(); + return new Insert(tableRef(), + new LinkedHashMap<>(values), + ifNotExists, + (ttl == null && timestamp == null) ? Optional.empty() : Optional.of(new Using(Optional.ofNullable(ttl), Optional.ofNullable(timestamp)))); + } + } + + public static class UpdateBuilder extends BaseBuilder<Update, UpdateBuilder> implements Conditional.ConditionalBuilder<UpdateBuilder> + { + private @Nullable TTL ttl; + private @Nullable Timestamp timestamp; + private final LinkedHashMap<Symbol, Expression> set = new LinkedHashMap<>(); + private final Conditional.Builder where = new Conditional.Builder(); + private @Nullable CasCondition casCondition; + + protected UpdateBuilder(TableMetadata table) + { + super(Kind.UPDATE, table); + } + + public UpdateBuilder timestamp(Value value) + { + this.timestamp = new Timestamp(value); + return this; + } + + public UpdateBuilder ttl(Value value) + { + this.ttl = new TTL(value); + return this; + } + + public UpdateBuilder ttl(int value) + { + return ttl(Bind.of(value)); + } + + public UpdateBuilder ifExists() + { + casCondition = CasCondition.Simple.Exists; + return this; + } + + public UpdateBuilder ifCondition(CasCondition condition) + { + casCondition = condition; + return this; + } + + public UpdateBuilder set(Symbol column, Expression value) + { + if (!regularAndStatic.contains(column)) + throw new IllegalArgumentException("Attempted to set a non regular or static column " + column + "; expected " + regularAndStatic); + set.put(column, value); + return this; + } + + public UpdateBuilder set(String column, int value) + { + return set(new Symbol(column, Int32Type.instance), Bind.of(value)); + } + + @Override + public UpdateBuilder where(Expression ref, Conditional.Where.Inequality kind, Expression expression) + { + if (kind == Conditional.Where.Inequality.EQUAL) + maybePkEq(ref); + where.where(ref, kind, expression); + return this; + } + + @Override + public UpdateBuilder between(Expression ref, Expression start, Expression end) + { + where.between(ref, start, end); + return this; + } + + @Override + public UpdateBuilder in(ReferenceExpression ref, List<? extends Expression> expressions) + { + maybePkEq(ref); + where.in(ref, expressions); + return this; + } + + @Override + public UpdateBuilder is(Symbol ref, Conditional.Is.Kind kind) + { + where.is(ref, kind); + return this; + } + + @Override + public Update build() + { + assertAllPksHaveEq(); + if (set.isEmpty()) + throw new IllegalStateException("Unable to create an Update without a SET section; set function was never called"); + + return new Update(tableRef(), + (ttl == null && timestamp == null) ? Optional.empty() : Optional.of(new Using(Optional.ofNullable(ttl), Optional.ofNullable(timestamp))), + new LinkedHashMap<>(set), + where.build(), + Optional.ofNullable(casCondition)); + } + } + + public static class DeleteBuilder extends BaseBuilder<Delete, DeleteBuilder> implements Conditional.ConditionalBuilder<DeleteBuilder> + { + private final List<Symbol> columns = new ArrayList<>(); + private @Nullable Timestamp timestamp = null; + private final Conditional.Builder where = new Conditional.Builder(); + //TODO (now): casCondition + private @Nullable CasCondition casCondition; + + public DeleteBuilder(TableMetadata table) + { + super(Kind.DELETE, table); + } + + public DeleteBuilder ifExists() + { + casCondition = CasCondition.Simple.Exists; + return this; + } + + public DeleteBuilder ifCondition(CasCondition condition) + { + casCondition = condition; + return this; + } + + public List<Symbol> columns() + { + return Collections.unmodifiableList(columns); + } + + public DeleteBuilder column(Symbol symbol) + { + if (!regularAndStatic.contains(symbol)) + throw new IllegalArgumentException("Can not delete column " + symbol + "; only regular/static columns can be deleted, expected " + regularAndStatic); + columns.add(symbol); + return this; + } + + public DeleteBuilder column(Symbol... symbols) + { + return column(Arrays.asList(symbols)); + } + + public DeleteBuilder column(List<Symbol> symbols) + { + symbols.forEach(this::column); + return this; + } + + public DeleteBuilder timestamp(Value value) + { + this.timestamp = new Timestamp(value); + return this; + } + + @Override + public DeleteBuilder where(Expression ref, Conditional.Where.Inequality kind, Expression expression) + { + if (kind == Conditional.Where.Inequality.EQUAL) + maybePkEq(ref); + where.where(ref, kind, expression); + return this; + } + + @Override + public DeleteBuilder between(Expression ref, Expression start, Expression end) + { + where.between(ref, start, end); + return this; + } + + @Override + public DeleteBuilder in(ReferenceExpression ref, List<? extends Expression> expressions) + { + maybePkEq(ref); + where.in(ref, expressions); + return this; + } + + @Override + public DeleteBuilder is(Symbol ref, Conditional.Is.Kind kind) + { + where.is(ref, kind); + return this; + } + + @Override + public Delete build() + { + assertAllPksHaveEq(); + return new Delete(columns.isEmpty() ? Collections.emptyList() : new ArrayList<>(columns), + tableRef(), + Optional.ofNullable(timestamp), + where.build(), + Optional.ofNullable(casCondition)); + } + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/Operator.java b/test/unit/org/apache/cassandra/cql3/ast/Operator.java new file mode 100644 index 000000000000..d0baa10ec01c --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/Operator.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.util.EnumSet; +import java.util.stream.Stream; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.NumberType; +import org.apache.cassandra.db.marshal.StringType; +import org.apache.cassandra.db.marshal.TemporalType; + +public class Operator implements Expression +{ + public enum Kind + { + ADD('+'), + SUBTRACT('-'), + MULTIPLY('*'); + //TODO support avoiding 42 / 0 and 42 % 0 as these will fail +// DIVIDE('/'), +// MOD('%'); + + private final char value; + + Kind(char value) + { + this.value = value; + } + } + + public final Kind kind; + public final Expression left; + public final Expression right; + + public Operator(Kind kind, Expression left, Expression right) + { + this.kind = kind; + this.left = left; + this.right = right; + if (!left.type().equals(right.type())) + throw new IllegalArgumentException("Types do not match: left=" + left.type() + ", right=" + right.type()); + } + + public static EnumSet<Kind> supportsOperators(AbstractType<?> type) + { + //TODO (coverage): some operators exist but only for different inputs than (type, type) + // date and time both support +- duration; aka date + duration = date, time - duration = time (looks like all temporal types are this way) + type = type.unwrap(); + // see org.apache.cassandra.cql3.functions.OperationFcts.OPERATION + if (type instanceof NumberType) + return EnumSet.allOf(Kind.class); + if (type instanceof TemporalType) + return EnumSet.noneOf(Kind.class); //TODO (coverage): not true, but requires the RHS to be duration type only + if (type instanceof StringType) + return EnumSet.of(Kind.ADD); + return EnumSet.noneOf(Kind.class); + } + + @Override + public AbstractType<?> type() + { + return left.type(); + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + left.toCQL(sb, formatter); + sb.append(' ').append(kind.value).append(' '); + right.toCQL(sb, formatter); + } + + @Override + public Stream<? extends Element> stream() + { + return Stream.of(left, right); + } + + @Override + public Expression visit(Visitor v) + { + var u = v.visit(this); + if (u != this) return u; + var left = this.left.visit(v); + var right = this.right.visit(v); + if (left == this.left && right == this.right) return this; + return new Operator(kind, left, right); + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/Reference.java b/test/unit/org/apache/cassandra/cql3/ast/Reference.java new file mode 100644 index 000000000000..fe836a05ddc1 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/Reference.java @@ -0,0 +1,182 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.function.Function; +import java.util.stream.Stream; + +import com.google.common.collect.ImmutableList; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.BytesType; + +public class Reference implements ReferenceExpression +{ + public final List<ReferenceExpression> path; + + private Reference(List<ReferenceExpression> path) + { + if (path.isEmpty()) + throw new IllegalArgumentException("Reference may not be empty"); + this.path = path; + if (path.stream().anyMatch(e -> e.toCQL().contains("\"\""))) + new Throwable().printStackTrace(); + } + + public static Reference of(ReferenceExpression top) + { + return new Reference(Collections.singletonList(Objects.requireNonNull(top))); + } + + @Override + public AbstractType<?> type() + { + return path.get(path.size() - 1).type(); + } + + public Builder unbuild() + { + return new Builder(path); + } + + public Reference add(String symbol, AbstractType<?> subType) + { + return unbuild().add(symbol, subType).build(); + } + + public Reference add(Expression expression) + { + if (expression instanceof Reference) + { + Reference other = (Reference) expression; + List<ReferenceExpression> path = new ArrayList<>(this.path.size() + other.path.size()); + path.addAll(this.path); + path.addAll(other.path); + return new Reference(path); + } + else if (expression instanceof Symbol) + { + return unbuild().add((Symbol) expression).build(); + } + else + { + return unbuild().add(expression.name(), expression.type()).build(); + } + } + + public Reference lastAsCollection(Function<ReferenceExpression, CollectionAccess> fn) + { + List<ReferenceExpression> path = new ArrayList<>(this.path.size()); + for (int i = 0; i < this.path.size() - 1; i++) + path.add(this.path.get(i)); + ReferenceExpression last = this.path.get(this.path.size() - 1); + path.add(Objects.requireNonNull(fn.apply(last))); + return new Reference(path); + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + path.forEach(p -> { + p.toCQL(sb, formatter); + sb.append('.'); + }); + sb.setLength(sb.length() - 1); // last . + } + + @Override + public Stream<? extends Element> stream() + { + return path.stream(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Reference elements = (Reference) o; + return Objects.equals(path, elements.path); + } + + @Override + public int hashCode() + { + return Objects.hash(path); + } + + @Override + public String toString() + { + return toCQL(); + } + + public static class Builder + { + private final List<ReferenceExpression> path; + + public Builder() + { + path = new ArrayList<>(); + } + + private Builder(List<ReferenceExpression> path) + { + this.path = new ArrayList<>(path.size() + 1); + this.path.addAll(path); + } + + public Builder add(String name) + { + return add(new Symbol(name, BytesType.instance)); + } + + public Builder add(String symbol, AbstractType<?> subType) + { + return add(new Symbol(symbol, subType)); + } + + public Builder add(Symbol symbol) + { + path.add(Objects.requireNonNull(symbol)); + return this; + } + + public Builder add(String[] paths) + { + for (String name : paths) + path.add(new Symbol(name, BytesType.instance)); + return this; + } + + public Reference build() + { + switch (path.size()) + { + case 0: throw new IllegalStateException("Unable to make a Reference from an empty path"); + case 1: return Reference.of(path.get(0)); + default: return new Reference(ImmutableList.copyOf(path)); + } + } + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/ReferenceExpression.java b/test/unit/org/apache/cassandra/cql3/ast/ReferenceExpression.java new file mode 100644 index 000000000000..7a99e030a4e7 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/ReferenceExpression.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +/** + * Marker interface of what is allowed in a reference + */ +public interface ReferenceExpression extends Expression +{ + @Override + default ReferenceExpression visit(Visitor v) + { + return this; + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/Select.java b/test/unit/org/apache/cassandra/cql3/ast/Select.java new file mode 100644 index 000000000000..224c8f6d5682 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/Select.java @@ -0,0 +1,417 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.stream.Stream; +import javax.annotation.Nullable; + +import com.google.common.collect.ImmutableList; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.schema.TableMetadata; + +public class Select implements Statement +{ + /* +SELECT * | select_expression | DISTINCT partition //TODO DISTINCT +FROM [keyspace_name.] table_name +[WHERE partition_value + [AND clustering_filters + [AND static_filters]]] +[ORDER BY PK_column_name ASC|DESC] +[LIMIT N] +[ALLOW FILTERING] + */ + // select + public final List<Expression> selections; + // from + public final Optional<TableReference> source; + // where + public final Optional<Conditional> where; + public final Optional<OrderBy> orderBy; + public final Optional<Value> limit; + public final boolean allowFiltering; + + public Select(List<Expression> selections) + { + this(selections, Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty()); + } + + public Select(List<Expression> selections, Optional<TableReference> source, Optional<Conditional> where, Optional<OrderBy> orderBy, Optional<Value> limit) + { + this(selections, source, where, orderBy, limit, false); + } + + public Select(List<Expression> selections, Optional<TableReference> source, Optional<Conditional> where, Optional<OrderBy> orderBy, Optional<Value> limit, boolean allowFiltering) + { + this.selections = selections; + this.source = source; + this.where = where; + this.orderBy = orderBy; + this.limit = limit; + this.allowFiltering = allowFiltering; + + if (!source.isPresent()) + { + if (where.isPresent()) + throw new IllegalArgumentException("Can not have a WHERE clause when there isn't a FROM"); + if (orderBy.isPresent()) + throw new IllegalArgumentException("Can not have a ORDER BY clause when there isn't a FROM"); + if (limit.isPresent()) + throw new IllegalArgumentException("Can not have a LIMIT clause when there isn't a FROM"); + if (allowFiltering) + throw new IllegalArgumentException("Can not have a ALLOW FILTERING clause when there isn't a FROM"); + } + } + + public static Builder builder() + { + return new Builder(); + } + + public Select withAllowFiltering() + { + return new Select(selections, source, where, orderBy, limit, true); + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + sb.append("SELECT "); + if (selections.isEmpty()) + { + sb.append('*'); + } + else + { + selections.forEach(s -> { + s.toCQL(sb, formatter); + sb.append(", "); + }); + sb.setLength(sb.length() - 2); // last ', ' + } + if (source.isPresent()) + { + formatter.section(sb); + sb.append("FROM "); + source.get().toCQL(sb, formatter); + if (where.isPresent()) + { + formatter.section(sb); + sb.append("WHERE "); + where.get().toCQL(sb, formatter); + } + if (orderBy.isPresent()) + { + formatter.section(sb); + sb.append("ORDER BY "); + orderBy.get().toCQL(sb, formatter); + } + if (limit.isPresent()) + { + formatter.section(sb); + sb.append("LIMIT "); + limit.get().toCQL(sb, formatter); + } + + if (allowFiltering) + { + formatter.section(sb); + sb.append("ALLOW FILTERING"); + } + } + } + + @Override + public Stream<? extends Element> stream() + { + List<Element> es = new ArrayList<>(selections.size() + + (source.isPresent() ? 1 : 0) + + (where.isPresent() ? 1 : 0) + + (orderBy.isPresent() ? 1 : 0) + + (limit.isPresent() ? 1 : 0)); + es.addAll(selections); + if (source.isPresent()) + es.add(source.get()); + if (where.isPresent()) + es.add(where.get()); + if (orderBy.isPresent()) + es.add(orderBy.get()); + if (limit.isPresent()) + es.add(limit.get()); + return es.stream(); + } + + @Override + public String toString() + { + return detailedToString(); + } + + @Override + public Kind kind() + { + return Kind.SELECT; + } + + @Override + public Statement visit(Visitor v) + { + var u = v.visit(this); + if (u != this) return u; + boolean updated = false; + List<Expression> selections = new ArrayList<>(this.selections.size()); + for (Expression e : this.selections) + { + Expression update = e.visit(v); + updated |= e != update; + selections.add(update); + } + Optional<Conditional> where; + if (this.where.isPresent()) + { + var c = this.where.get(); + var update = c.visit(v); + updated |= c != update; + where = Optional.ofNullable(update); + } + else + { + where = this.where; + } + Optional<Value> limit; + if (this.limit.isPresent()) + { + var l = this.limit.get(); + var update = l.visit(v); + updated |= l != update; + limit = Optional.ofNullable(update); + } + else + { + limit = this.limit; + } + if (!updated) return this; + return new Select(selections, source, where, orderBy, limit, allowFiltering); + } + + public static class OrderBy implements Element + { + public enum Ordering + {ASC, DESC} + + public final List<Ordered> ordered; + + public OrderBy(List<Ordered> ordered) + { + if (ordered.isEmpty()) + throw new IllegalArgumentException("Can not ORDER BY an empty list"); + this.ordered = ordered; + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + if (ordered.size() == 1) + { + ordered.get(0).toCQL(sb, formatter); + return; + } + + String postfix = ", "; + for (Ordered o : ordered) + { + o.toCQL(sb, formatter); + sb.append(postfix); + } + sb.setLength(sb.length() - postfix.length()); + } + + @Override + public Stream<? extends Element> stream() + { + return ordered.stream(); + } + + public static class Ordered implements Element + { + public final ReferenceExpression expression; + public final Ordering ordering; + + public Ordered(ReferenceExpression expression, Ordering ordering) + { + this.expression = expression; + this.ordering = ordering; + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + expression.toCQL(sb, formatter); + sb.append(' '); + sb.append(ordering.name()); + } + + @Override + public Stream<? extends Element> stream() + { + return Stream.of(expression); + } + } + + public static class Builder + { + private final List<Ordered> ordered = new ArrayList<>(); + + public boolean isEmpty() + { + return ordered.isEmpty(); + } + + public Builder add(ReferenceExpression expression, Ordering ordering) + { + ordered.add(new Ordered(expression, ordering)); + return this; + } + + public OrderBy build() + { + return new OrderBy(ImmutableList.copyOf(ordered)); + } + } + } + + public static class Builder implements Conditional.ConditionalBuilder<Builder> + { + @Nullable // null means wildcard + private List<Expression> selections = new ArrayList<>(); + private Optional<TableReference> source = Optional.empty(); + private Conditional.Builder where = new Conditional.Builder(); + private OrderBy.Builder orderBy = new OrderBy.Builder(); + private Optional<Value> limit = Optional.empty(); + private boolean allowFiltering = false; + + public Builder wildcard() + { + if (selections != null && !selections.isEmpty()) + throw new IllegalStateException("Attempted to use * for selection but existing selections exist: " + selections); + selections = null; + return this; + } + + public Builder columnSelection(String name, AbstractType<?> type) + { + return selection(new Symbol(name, type)); + } + + public Builder allowFiltering() + { + allowFiltering = true; + return this; + } + + public Builder selection(Expression e) + { + if (selections == null) + throw new IllegalStateException("Unable to add '" + e.name() + "' as a selection as * was already requested"); + selections.add(e); + return this; + } + + public Builder table(TableReference ref) + { + source = Optional.of(ref); + return this; + } + + public Builder table(String ks, String name) + { + return table(new TableReference(Optional.of(ks), name)); + } + + public Builder table(String name) + { + return table(new TableReference(name)); + } + + public Builder table(TableMetadata table) + { + return table(TableReference.from(table)); + } + + @Override + public Builder where(Expression ref, Conditional.Where.Inequality kind, Expression expression) + { + where.where(ref, kind, expression); + return this; + } + + @Override + public Builder between(Expression ref, Expression start, Expression end) + { + where.between(ref, start, end); + return this; + } + + @Override + public Builder in(ReferenceExpression ref, List<? extends Expression> expressions) + { + where.in(ref, expressions); + return this; + } + + @Override + public Builder is(Symbol ref, Conditional.Is.Kind kind) + { + where.is(ref, kind); + return this; + } + + public Builder orderByColumn(String name, AbstractType<?> type, OrderBy.Ordering ordering) + { + orderBy.add(new Symbol(name, type), ordering); + return this; + } + + public Builder limit(Value limit) + { + this.limit = Optional.of(limit); + return this; + } + + public Builder limit(int limit) + { + return limit(Bind.of(limit)); + } + + public Select build() + { + return new Select((selections == null || selections.isEmpty()) ? Collections.emptyList() : ImmutableList.copyOf(selections), + source, + where.isEmpty() ? Optional.empty() : Optional.of(where.build()), + orderBy.isEmpty() ? Optional.empty() : Optional.of(orderBy.build()), + limit, + allowFiltering); + } + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/StandardVisitors.java b/test/unit/org/apache/cassandra/cql3/ast/StandardVisitors.java new file mode 100644 index 000000000000..2a76cd80972d --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/StandardVisitors.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +public class StandardVisitors +{ + public static final Visitor BIND_TO_LITERAL = new Visitor() + { + @Override + public Value visit(Value v) + { + if (!(v instanceof Bind)) return v; + Bind b = (Bind) v; + return new Literal(b.value(), b.type()); + } + }; + + public static final Visitor UNWRAP_TYPE_HINT = new Visitor() + { + @Override + public Expression visit(Expression e) + { + if (!(e instanceof TypeHint)) return e; + TypeHint hint = (TypeHint) e; + if (hint.type.equals(hint.e.type())) + return hint.e; + return e; + } + }; + + public static final Visitor.CompositeVisitor DEBUG = Visitor.CompositeVisitor.of(UNWRAP_TYPE_HINT, BIND_TO_LITERAL); + + private StandardVisitors() {} +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/Statement.java b/test/unit/org/apache/cassandra/cql3/ast/Statement.java new file mode 100644 index 000000000000..ffcba03465b8 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/Statement.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import org.apache.cassandra.utils.ByteBufferUtil; + +public interface Statement extends Element +{ + enum Kind { SELECT, MUTATION, TXN } + Kind kind(); + + default Object[] binds() + { + return streamRecursive() + .filter(e -> e instanceof Bind) + .map(e -> ((Bind) e).value()) + .toArray(Object[]::new); + } + + default ByteBuffer[] bindsEncoded() + { + return streamRecursive() + .filter(e -> e instanceof Bind) + .map(e -> ((Bind) e).valueEncoded()) + .toArray(ByteBuffer[]::new); + } + + default String detailedToString() + { + Object[] binds = binds(); + return "CQL:\n" + toCQL() + "\nBinds:\n" + IntStream.range(0, binds.length) + .mapToObj(i -> i + " -> " + binds[i].getClass().getCanonicalName() + "(" + normalize(binds[i]) + ")") + .collect(Collectors.joining("\n")); + } + + Statement visit(Visitor v); + + static boolean hasByteBuffer(Object value) + { + if (value == null) + return false; + if (value instanceof ByteBuffer) + return true; + else if (value instanceof Collection) + { + Collection<Object> collection = (Collection<Object>) value; + return collection.stream().anyMatch(Statement::hasByteBuffer); + } + else if (value instanceof Map) + { + Map<Object, Object> map = (Map<Object, Object>) value; + return map.entrySet().stream().anyMatch(e -> hasByteBuffer(e.getKey()) || hasByteBuffer(e.getValue())); + } + return false; + } + + static String normalize(Object value) + { + if (value == null) + return null; + if (value instanceof ByteBuffer) + { + ByteBuffer bb = (ByteBuffer) value; + if (bb.remaining() > 100) + { + bb = bb.duplicate(); + bb.limit(bb.position() + 100); + return ByteBufferUtil.bytesToHex(bb) + "..."; + } + return ByteBufferUtil.bytesToHex(bb); + } + else if (value instanceof Collection) + { + Collection<Object> collection = (Collection<Object>) value; + if (hasByteBuffer(collection)) + return collection.stream().map(Statement::normalize).collect(Collectors.toList()).toString(); + } + else if (value instanceof Map) + { + Map<Object, Object> map = (Map<Object, Object>) value; + if (hasByteBuffer(map)) + return map.entrySet().stream() + .map(e -> normalize(e.getKey()) + " -> " + normalize(e.getValue())) + .collect(Collectors.joining(", ", "{", "}")); + } + return value.toString(); + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/Symbol.java b/test/unit/org/apache/cassandra/cql3/ast/Symbol.java new file mode 100644 index 000000000000..f905218c4585 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/Symbol.java @@ -0,0 +1,182 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.util.Objects; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.ReservedKeywords; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.marshal.ReversedType; +import org.apache.cassandra.schema.ColumnMetadata; + +public class Symbol implements ReferenceExpression, Comparable<Symbol> +{ + public final String symbol; + private final AbstractType<?> type; + public final boolean reversed; + + public Symbol(ColumnMetadata column) + { + this(column.name.toString(), column.type); + } + + public Symbol(String symbol, AbstractType<?> type) + { + this.symbol = Objects.requireNonNull(symbol); + this.type = Objects.requireNonNull(type).unwrap(); + this.reversed = type.isReversed(); + } + + public static Symbol from(ColumnMetadata metadata) + { + return new Symbol(metadata.name.toString(), metadata.type); + } + + public static Symbol unknownType(String name) + { + return new Symbol(name, BytesType.instance); + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + maybeQuote(sb, symbol); + } + + public static void maybeQuote(StringBuilder sb, String symbol) + { + sb.append(maybeQuote(symbol)); + } + + public static String maybeQuote(String symbol) + { + if (ReservedKeywords.isReserved(symbol)) + { + return quote(symbol); + } + else + { + return ColumnIdentifier.maybeQuote(symbol); + } + } + + private static final Pattern PATTERN_DOUBLE_QUOTE = Pattern.compile("\"", Pattern.LITERAL); + private static final String ESCAPED_DOUBLE_QUOTE = Matcher.quoteReplacement("\"\""); + + //TODO refactor ColumnIdentifier to expose this + private static String quote(String text) + { + return '"' + PATTERN_DOUBLE_QUOTE.matcher(text).replaceAll(ESCAPED_DOUBLE_QUOTE) + '"'; + } + + @Override + public AbstractType<?> type() + { + return type; + } + + public AbstractType<?> rawType() + { + if (reversed) return ReversedType.getInstance(type); + return type; + } + + @Override + public String name() + { + return symbol; + } + + /** + * Same as {@link #visit(Visitor)} but has the restriction that the return type may not be changed away from {@link Symbol}. + */ + public Symbol visit(String type, Visitor v) + { + var u = v.visit(this); + if (u == this) return this; + if (!(u instanceof Symbol)) + throw new IllegalStateException("Visitor converted a Symbol to " + u.getClass().getSimpleName() + "; this is not supported with " + type); + return (Symbol) u; + } + + public String detailedName() + { + return symbol + " " + type.asCQL3Type() + (reversed ? " (reversed)" : ""); + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Symbol symbol1 = (Symbol) o; + return Objects.equals(symbol, symbol1.symbol); + } + + @Override + public int hashCode() + { + return Objects.hash(symbol); + } + + @Override + public String toString() + { + return toCQL(); + } + + @Override + public int compareTo(Symbol o) + { + return toCQL().compareTo(o.toCQL()); + } + + public static class UnquotedSymbol extends Symbol + { + public UnquotedSymbol(String symbol, AbstractType<?> type) + { + super(symbol, type); + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + sb.append(symbol); + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + UnquotedSymbol symbol1 = (UnquotedSymbol) o; + return Objects.equals(symbol, symbol1.symbol); + } + + @Override + public int hashCode() + { + return Objects.hash(symbol); + } + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/TableReference.java b/test/unit/org/apache/cassandra/cql3/ast/TableReference.java new file mode 100644 index 000000000000..c2d01200b41b --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/TableReference.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.util.Objects; +import java.util.Optional; + +import org.apache.cassandra.schema.TableMetadata; + +public class TableReference implements Element +{ + public final Optional<String> keyspace; + public final String name; + + public TableReference(String name) + { + this(Optional.empty(), name); + } + + public TableReference(Optional<String> keyspace, String name) + { + this.keyspace = Objects.requireNonNull(keyspace); + this.name = Objects.requireNonNull(name); + } + + public static TableReference from(TableMetadata metadata) + { + return new TableReference(Optional.of(metadata.keyspace), metadata.name); + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + if (keyspace.isPresent()) + { + Symbol.maybeQuote(sb, keyspace.get()); + sb.append('.'); + } + Symbol.maybeQuote(sb, name); + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/Txn.java b/test/unit/org/apache/cassandra/cql3/ast/Txn.java new file mode 100644 index 000000000000..2e60574eac36 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/Txn.java @@ -0,0 +1,399 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import org.apache.cassandra.cql3.FieldIdentifier; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.marshal.UserType; + +public class Txn implements Statement +{ + // lets + public final List<Let> lets; + // return + public final Optional<Select> output; + public final Optional<If> ifBlock; + public final List<Mutation> mutations; + + public Txn(List<Let> lets, Optional<Select> output, Optional<If> ifBlock, List<Mutation> mutations) + { + this.lets = lets; + this.output = output; + this.ifBlock = ifBlock; + if (ifBlock.isPresent() && !mutations.isEmpty()) + throw new IllegalArgumentException("Unable to define both IF and non-conditional mutations"); + this.mutations = mutations; + } + + public static Txn wrap(Select select) + { + return new Txn(Collections.emptyList(), Optional.of(select), Optional.empty(), Collections.emptyList()); + } + + public static Txn wrap(Mutation mutation) + { + return new Txn(Collections.emptyList(), Optional.empty(), Optional.empty(), Collections.singletonList(mutation)); + } + + public static Builder builder() + { + return new Builder(); + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + sb.append("BEGIN TRANSACTION"); + + formatter.group(sb); + stream().forEach(e -> { + formatter.section(sb); + e.toCQL(sb, formatter); + if (!(e instanceof If)) + sb.append(';'); + }); + formatter.endgroup(sb); + formatter.section(sb); + sb.append("COMMIT TRANSACTION"); + } + + @Override + public Stream<? extends Element> stream() + { + Stream<? extends Element> ret = lets.stream(); + if (output.isPresent()) + ret = Stream.concat(ret, Stream.of(output.get())); + if (ifBlock.isPresent()) + ret = Stream.concat(ret, Stream.of(ifBlock.get())); + ret = Stream.concat(ret, mutations.stream()); + return ret; + } + + @Override + public String toString() + { + return toCQL(); + } + + @Override + public Kind kind() + { + return Kind.TXN; + } + + @Override + public Statement visit(Visitor v) + { + var u = v.visit(this); + if (u != this) return u; + boolean updated = false; + List<Let> lets = new ArrayList<>(this.lets.size()); + for (Let l : this.lets) + { + Statement update = l.select.visit(v); + if (!(update instanceof Select)) + throw new IllegalArgumentException("Unable to use type " + update.getClass() + " in Let"); + boolean localUpdated = update != l.select; + updated |= localUpdated; + lets.add(!localUpdated ? l : new Let(l.symbol, (Select) update)); + } + Optional<Select> output; + if (this.output.isPresent()) + { + Select select = this.output.get(); + Statement update = select.visit(v); + if (!(update instanceof Select)) + throw new IllegalArgumentException("Unable to use type " + update.getClass() + " in output"); + boolean localUpdated = select != update; + updated |= localUpdated; + output = !localUpdated ? this.output : Optional.ofNullable((Select) update); + } + else + { + output = this.output; + } + Optional<If> ifBlock; + if (this.ifBlock.isPresent()) + { + If block = this.ifBlock.get(); + boolean localUpdated = false; + Conditional c = block.conditional.visit(v); + localUpdated |= c != block.conditional; + List<Mutation> mutations = new ArrayList<>(this.mutations.size()); + for (Mutation m : block.mutations) + { + Statement update = m.visit(v); + if (!(update instanceof Mutation)) + throw new IllegalArgumentException("Unable to use type " + update.getClass() + " where Mutation is expected"); + updated |= update != m; + mutations.add((Mutation) update); + } + ifBlock = !localUpdated ? this.ifBlock : Optional.of(new If(c, mutations)); + } + else + { + ifBlock = this.ifBlock; + } + List<Mutation> mutations = new ArrayList<>(this.mutations.size()); + for (Mutation m : this.mutations) + { + Statement update = m.visit(v); + if (!(update instanceof Mutation)) + throw new IllegalArgumentException("Unable to use type " + update.getClass() + " where Mutation is expected"); + updated |= update != m; + mutations.add((Mutation) update); + } + if (!updated) return this; + return new Txn(lets, output, ifBlock, mutations); + } + + private static void recursiveReferences(Collection<Reference> accum, Reference ref) + { + //NOTE: this doesn't actually collect the references due to lack of support in CQL within a Txn, need to fix those cases + //Coverage: Map/Set/List are a bit annoying as they use real values and not something known in the schema, so may need to handle that different in validation + AbstractType<?> type = ref.type().unwrap(); + if (type.isCollection()) + { + //TODO Caleb to add support for [] like normal read/write supports +// if (type instanceof SetType) +// { +// // [value] syntax +// SetType set = (SetType) type; +// AbstractType subType = set.getElementsType(); + +// } +// else if (type instanceof MapType) +// { +// // [key] syntax +// MapType map = (MapType) type; +// AbstractType keyType = map.getKeysType(); +// AbstractType valueType = map.getValuesType(); +// } + // see Selectable.specForElementOrSlice; ListType is not supported + } + else if (type.isUDT()) + { + //TODO Caleb to support multiple nesting +// UserType udt = (UserType) type; +// for (int i = 0; i < udt.size(); i++) +// { +// Reference subRef = ref.add(udt.fieldName(i).toString(), udt.type(i)); +// accum.add(subRef); +// recursiveReferences(accum, subRef); +// } + } + } + + public static class Builder + { + private final Map<String, Select> lets = new LinkedHashMap<>(); + // no type system so don't need easy lookup to Expression; just existence check + private final Set<Reference> allowedReferences = new HashSet<>(); + private Optional<Select> output = Optional.empty(); + private Optional<If> ifBlock = Optional.empty(); + private final List<Mutation> mutations = new ArrayList<>(); + + public Set<Reference> allowedReferences() + { + return Collections.unmodifiableSet(allowedReferences); + } + + public Map<String, Select> lets() + { + return Collections.unmodifiableMap(lets); + } + + public boolean isEmpty() + { + // don't include output as 'BEGIN TRANSACTION SELECT "000000000000000010000"; COMMIT TRANSACTION' isn't valid +// return lets.isEmpty(); + // TransactionStatement defines empty as no SELECT or updates + return !output.isPresent() && !ifBlock.isPresent() && mutations.isEmpty(); + } + + public Builder addLet(String name, Select.Builder select) + { + return addLet(name, select.build()); + } + + public Builder addLet(String name, Select select) + { + if (lets.containsKey(name)) + throw new IllegalArgumentException("Let name " + name + " already exists"); + lets.put(name, select); + + Reference ref = Reference.of(new Symbol.UnquotedSymbol(name, toNamedTuple(select))); + for (Expression e : select.selections) + addAllowedReference(ref.add(e)); + return this; + } + + private AbstractType<?> toNamedTuple(Select select) + { + //TODO don't rely on UserType... + List<FieldIdentifier> fieldNames = new ArrayList<>(select.selections.size()); + List<AbstractType<?>> fieldTypes = new ArrayList<>(select.selections.size()); + for (Expression e : select.selections) + { + fieldNames.add(FieldIdentifier.forQuoted(e.name())); + fieldTypes.add(e.type()); + } + return new UserType(null, null, fieldNames, fieldTypes, false); + } + + private Builder addAllowedReference(Reference ref) + { + allowedReferences.add(ref); + recursiveReferences(allowedReferences, ref); + return this; + } + + public Builder addReturn(Select select) + { + output = Optional.of(select); + return this; + } + + public Builder addReturnReferences(String... names) + { + Select.Builder builder = new Select.Builder(); + for (String name : names) + builder.selection(ref(name)); + addReturn(builder.build()); + return this; + } + + private Reference ref(String name) + { + if (!name.contains(".")) + return Reference.of(new Symbol(name, BytesType.instance)); + String[] splits = name.split("\\."); + Reference.Builder builder = new Reference.Builder(); + builder.add(splits); + return builder.build(); + } + + public Builder addIf(If block) + { + ifBlock = Optional.of(block); + return this; + } + + public Builder addIf(Conditional conditional, Mutation... mutations) + { + return addIf(conditional, Arrays.asList(mutations)); + } + + public Builder addIf(Conditional conditional, List<Mutation> mutations) + { + return addIf(new If(conditional, mutations)); + } + + public Builder addUpdate(Mutation mutation) + { + this.mutations.add(Objects.requireNonNull(mutation)); + return this; + } + + public Txn build() + { + List<Let> lets = this.lets.entrySet().stream().map(e -> new Let(e.getKey(), e.getValue())).collect(Collectors.toList()); + return new Txn(lets, output, ifBlock, new ArrayList<>(mutations)); + } + } + + public static class If implements Element + { + private final Conditional conditional; + private final List<Mutation> mutations; + + public If(Conditional conditional, List<Mutation> mutations) + { + this.conditional = conditional; + this.mutations = mutations; + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + sb.append("IF "); + conditional.toCQL(sb, formatter); + sb.append(" THEN"); + formatter.group(sb); + for (Mutation mutation : mutations) + { + formatter.element(sb); + mutation.toCQL(sb, formatter); + sb.append(';'); + } + formatter.endgroup(sb); + formatter.section(sb); + sb.append("END IF"); + } + + @Override + public Stream<? extends Element> stream() + { + return Stream.concat(Stream.of(conditional), mutations.stream()); + } + } + + public static class Let implements Element + { + public final String symbol; + public final Select select; + + public Let(String symbol, Select select) + { + this.symbol = symbol; + this.select = select; + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + sb.append("LET ").append(symbol).append(" = ("); + formatter.subgroup(sb); + select.toCQL(sb, formatter); + formatter.endsubgroup(sb); + sb.append(")"); + } + + @Override + public Stream<? extends Element> stream() + { + return Stream.of(select); + } + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/TypeHint.java b/test/unit/org/apache/cassandra/cql3/ast/TypeHint.java new file mode 100644 index 000000000000..d49f30c11817 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/TypeHint.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.util.stream.Stream; + +import org.apache.cassandra.db.marshal.AbstractType; + +public class TypeHint implements Expression +{ + public final Expression e; + public final AbstractType<?> type; + + public TypeHint(Expression e, AbstractType<?> type) + { + this.e = e; + this.type = type; + } + + public TypeHint(Expression e) + { + this.e = e; + this.type = e.type(); + } + + /** + * {@code ? + ?} is not clear to parsing, so rather than assume the type (like we do for literals) we fail and ask + * the user to CAST... so need to {@link TypeHint} when a {@link Bind} is found. + * + * Wait, {@link TypeHint} and not {@link Cast}? See CASSANDRA-17915... + */ + public static Expression maybeApplyTypeHint(Expression e) + { + if (!(e instanceof Bind)) + return e; + // see https://the-asf.slack.com/archives/CK23JSY2K/p1663788235000449 + return new TypeHint(e); + } + + @Override + public void toCQL(StringBuilder sb, CQLFormatter formatter) + { + sb.append('(').append(type.asCQL3Type()).append(") "); + e.toCQL(sb, formatter); + } + + @Override + public AbstractType<?> type() + { + return type; + } + + @Override + public Stream<? extends Element> stream() + { + return Stream.of(e); + } + + @Override + public Expression visit(Visitor v) + { + var u = v.visit(this); + if (u != this) return u; + var e = this.e.visit(v); + if (e == this.e) return this; + return new TypeHint(e, type); + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/Value.java b/test/unit/org/apache/cassandra/cql3/ast/Value.java new file mode 100644 index 000000000000..f04b5fcd3638 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/Value.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.nio.ByteBuffer; + +import org.apache.cassandra.db.marshal.AbstractType; + +public interface Value extends Expression +{ + Object value(); + ByteBuffer valueEncoded(); + Value with(Object value, AbstractType<?> type); + + @Override + default Value visit(Visitor v) + { + return v.visit(this); + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/Visitor.java b/test/unit/org/apache/cassandra/cql3/ast/Visitor.java new file mode 100644 index 000000000000..71e4797c2595 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ast/Visitor.java @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.ast; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Stream; + +import accord.utils.Invariants; + +/** + * Visits and conditionally replaces an element in the ast. Each "visit" method offers the ability to be notified when the + * desired type is found in the ast tree, and allows the visitor to replace the current element (return the element unchanged + * if no mutations are desired, return a different element if a change is desired). + * + * For elements that support visitors, the following pattern should be respected. + * <ol> + * <li>check the top level element first + * {code} + * var u = visitor.visit(this); + * if (u != this) return u; + * {code}</li> + * <li>check each sub element; replacing the top level element if any sub-elements are changed + * {code} + * boolean updated = false; + * var a = a.visit(visitor); + * updated |= (a != this.a); + * ... + * return !updated ? this : copy... + * {code} + * </li> + * </ol> + */ +public interface Visitor +{ + default Statement visit(Statement s) + { + return s; + } + + default Expression visit(Expression e) + { + if (e instanceof Value) return visit((Value) e); + if (e instanceof ReferenceExpression) return visit((ReferenceExpression) e); + return e; + } + + default Conditional visit(Conditional c) { return c; } + + default ReferenceExpression visit(ReferenceExpression r) + { + return r; + } + + default Value visit(Value v) { return v; } + + class CompositeVisitor implements Visitor + { + private final List<Visitor> visitors; + + private CompositeVisitor(List<Visitor> visitors) + { + this.visitors = visitors; + } + + public static CompositeVisitor of(Visitor... visitors) + { + return of(Arrays.asList(visitors)); + } + + public static CompositeVisitor of(List<Visitor> visitors) + { + Invariants.checkArgument(!visitors.isEmpty(), "Visitors may not be empty"); + + if (Stream.of(visitors).noneMatch(v -> v instanceof CompositeVisitor)) + return new CompositeVisitor(visitors); + List<Visitor> flatten = new ArrayList<>(); + for (Visitor v : visitors) + { + if (!(v instanceof CompositeVisitor)) + { + flatten.add(v); + continue; + } + CompositeVisitor cv = (CompositeVisitor) v; + flatten.addAll(cv.visitors); + } + return new CompositeVisitor(flatten); + } + + public CompositeVisitor append(Visitor v) + { + if (v instanceof CompositeVisitor) + { + CompositeVisitor other = (CompositeVisitor) v; + List<Visitor> vs = new ArrayList<>(visitors.size() + other.visitors.size()); + vs.addAll(visitors); + vs.addAll(other.visitors); + return new CompositeVisitor(vs); + } + else + { + List<Visitor> vs = new ArrayList<>(visitors.size() + 1); + vs.addAll(visitors); + vs.add(v); + return new CompositeVisitor(vs); + } + } + + @Override + public Expression visit(Expression e) + { + for (var v : visitors) + e = v.visit(e); + return e; + } + + @Override + public Conditional visit(Conditional c) + { + for (var v : visitors) + c = v.visit(c); + return c; + } + + @Override + public Statement visit(Statement s) + { + for (var v : visitors) + s = v.visit(s); + return s; + } + + @Override + public Value visit(Value c) + { + for (var v : visitors) + c = v.visit(c); + return c; + } + } +} From aa5b8e3d3fdcc55fdde68a205f376673f8ce1f88 Mon Sep 17 00:00:00 2001 From: Abe Ratnofsky <abe@aber.io> Date: Thu, 21 Nov 2024 14:17:16 -0500 Subject: [PATCH 078/225] Periodically disconnect roles that are revoked or have LOGIN=FALSE set patch by Abe Ratnofsky; reviewed by Bernardo Botella Corbi, Francisco Guerrero Hernandez, and Jon Meredith for CASSANDRA-19385 --- CHANGES.txt | 1 + conf/cassandra.yaml | 7 + conf/cassandra_latest.yaml | 7 + .../cassandra/auth/CassandraRoleManager.java | 107 ++++++++++++- .../auth/CassandraRoleManagerMBean.java | 49 ++++++ .../cassandra/service/CassandraDaemon.java | 7 + .../service/NativeTransportService.java | 7 + .../cassandra/service/StorageService.java | 8 +- .../apache/cassandra/transport/Server.java | 22 +++ .../test/auth/RoleRevocationTest.java | 142 ++++++++++++++++++ .../auth/CassandraRoleManagerTest.java | 80 ++++++++++ .../org/apache/cassandra/auth/RolesTest.java | 10 ++ 12 files changed, 445 insertions(+), 2 deletions(-) create mode 100644 src/java/org/apache/cassandra/auth/CassandraRoleManagerMBean.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/auth/RoleRevocationTest.java diff --git a/CHANGES.txt b/CHANGES.txt index f4f3d72b6f0a..ab3b662f6acb 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Periodically disconnect roles that are revoked or have LOGIN=FALSE set (CASSANDRA-19385) * AST library for CQL-based fuzz tests (CASSANDRA-20198) * Support audit logging for JMX operations (CASSANDRA-20128) * Enable sorting of nodetool status output (CASSANDRA-20104) diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index 6ffe1fb9b6b2..55fc50f6c6e6 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -233,6 +233,13 @@ authorizer: # increase system_auth keyspace replication factor if you use this role manager. role_manager: class_name: CassandraRoleManager + parameters: + # Controls how often invalid roles are disconnected, such as when a role is altered with LOGIN=false + # Task is scheduled with period + random(0, max_jitter) delay between executions + # It's recommended to set these longer than the roles cache refresh period, since the invalidation check depends on + # cache contents. Disable by setting period=0h. + # invalid_role_disconnect_task_period: 4h + # invalid_role_disconnect_task_max_jitter: 1h # Network authorization backend, implementing INetworkAuthorizer; used to restrict user # access to certain DCs diff --git a/conf/cassandra_latest.yaml b/conf/cassandra_latest.yaml index 4f8879a43c02..387565fe1851 100644 --- a/conf/cassandra_latest.yaml +++ b/conf/cassandra_latest.yaml @@ -236,6 +236,13 @@ authorizer: # increase system_auth keyspace replication factor if you use this role manager. role_manager: class_name: CassandraRoleManager + parameters: + # Controls how often invalid roles are disconnected, such as when a role is altered with LOGIN=false + # Task is scheduled with period + random(0, max_jitter) delay between executions + # It's recommended to set these longer than the roles cache refresh period, since the invalidation check depends on + # cache contents. Disable by setting period=0h. + invalid_role_disconnect_task_period: 4h + invalid_role_disconnect_task_max_jitter: 1h # Network authorization backend, implementing INetworkAuthorizer; used to restrict user # access to certain DCs diff --git a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java index 3c1b52f8237a..1e1a9ec310e9 100644 --- a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java +++ b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java @@ -28,8 +28,12 @@ import java.util.Objects; import java.util.Set; import java.util.concurrent.Callable; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.function.Function; +import java.util.function.LongSupplier; import java.util.function.Predicate; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -42,8 +46,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.concurrent.ScheduledExecutorPlus; import org.apache.cassandra.concurrent.ScheduledExecutors; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.cql3.CQLStatement; import org.apache.cassandra.cql3.QueryOptions; import org.apache.cassandra.cql3.QueryProcessor; @@ -59,11 +65,13 @@ import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.service.ClientState; import org.apache.cassandra.service.StorageProxy; +import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.transport.Dispatcher; import org.apache.cassandra.transport.messages.ResultMessage; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.MBeanWrapper; import org.apache.cassandra.utils.NoSpamLogger; import org.mindrot.jbcrypt.BCrypt; @@ -95,7 +103,7 @@ * of the password itself (such as storing it in an alternative location) would * be added in overridden createRole and alterRole implementations. */ -public class CassandraRoleManager implements IRoleManager +public class CassandraRoleManager implements IRoleManager, CassandraRoleManagerMBean { private static final Logger logger = LoggerFactory.getLogger(CassandraRoleManager.class); private static final NoSpamLogger nospamLogger = NoSpamLogger.getLogger(logger, 1L, TimeUnit.MINUTES); @@ -103,6 +111,13 @@ public class CassandraRoleManager implements IRoleManager public static final String DEFAULT_SUPERUSER_NAME = "cassandra"; public static final String DEFAULT_SUPERUSER_PASSWORD = "cassandra"; + @VisibleForTesting + static final String PARAM_INVALID_ROLE_DISCONNECT_TASK_PERIOD = "invalid_role_disconnect_task_period"; + @VisibleForTesting + static final String PARAM_INVALID_ROLE_DISCONNECT_TASK_MAX_JITTER = "invalid_role_disconnect_task_max_jitter"; + + public static final String MBEAN_NAME = "org.apache.cassandra.auth:type=CassandraRoleManager"; + /** * We need to treat the default superuser as a special case since during initial node startup, we may end up with * duplicate creation or deletion + re-creation of this user on different nodes unless we check at quorum to see if @@ -149,7 +164,17 @@ static int getGensaltLogRounds() private final Set<Option> supportedOptions; private final Set<Option> alterableOptions; + private volatile ScheduledFuture<?> invalidRoleDisconnectTask; + + private volatile long invalidClientDisconnectPeriodMillis; + private volatile long invalidClientDisconnectMaxJitterMillis; + public CassandraRoleManager() + { + this(Map.of()); + } + + public CassandraRoleManager(Map<String, String> parameters) { supportedOptions = DatabaseDescriptor.getAuthenticator() instanceof PasswordAuthenticator ? ImmutableSet.of(Option.LOGIN, Option.SUPERUSER, Option.PASSWORD, Option.HASHED_PASSWORD, Option.GENERATED_PASSWORD) @@ -157,6 +182,13 @@ public CassandraRoleManager() alterableOptions = DatabaseDescriptor.getAuthenticator() instanceof PasswordAuthenticator ? ImmutableSet.of(Option.PASSWORD, Option.HASHED_PASSWORD, Option.GENERATED_PASSWORD) : ImmutableSet.<Option>of(); + + // Inherit parsing and validation from existing config parser + invalidClientDisconnectPeriodMillis = new DurationSpec.LongMillisecondsBound(parameters.getOrDefault(PARAM_INVALID_ROLE_DISCONNECT_TASK_PERIOD, "0h")).toMilliseconds(); + invalidClientDisconnectMaxJitterMillis = new DurationSpec.LongMillisecondsBound(parameters.getOrDefault(PARAM_INVALID_ROLE_DISCONNECT_TASK_MAX_JITTER, "0h")).toMilliseconds(); + + if (!MBeanWrapper.instance.isRegistered(MBEAN_NAME)) + MBeanWrapper.instance.registerMBean(this, MBEAN_NAME); } @Override @@ -164,6 +196,7 @@ public void setup(boolean asyncRoleSetup) { loadRoleStatement(); loadIdentityStatement(); + scheduleDisconnectInvalidRoleTask(); if (!asyncRoleSetup) { try @@ -732,4 +765,76 @@ public Supplier<Map<RoleResource, Set<Role>>> bulkLoader() return entries; }; } + + protected void disconnectInvalidRoles() + { + // This should always run with jitter, otherwise there's a risk that all nodes disconnect clients at the same time + StorageService.instance.disconnectInvalidRoles(); + } + + protected void invalidRoleDisconnectTask(LongSupplier delayMillis, ScheduledExecutorService executor) + { + try + { + disconnectInvalidRoles(); + } + catch (Exception e) + { + logger.warn("Failed to disconnect invalid roles", e); + } + + long nextDelayMillis = delayMillis.getAsLong(); + logger.info("Scheduling next invalid role disconnection in {} millis", nextDelayMillis); + this.invalidRoleDisconnectTask = executor.schedule(() -> invalidRoleDisconnectTask(delayMillis, executor), nextDelayMillis, TimeUnit.MILLISECONDS); + } + + protected void scheduleDisconnectInvalidRoleTask() + { + // Cancel any pending execution if it exists, since we may have changed period / jitter parameters + if (this.invalidRoleDisconnectTask != null) + { + logger.debug("Canceling previous invalidRoleDisconnectTask"); + this.invalidRoleDisconnectTask.cancel(true); + } + + long period = getInvalidClientDisconnectPeriodMillis(); + long jitter = getInvalidClientDisconnectMaxJitterMillis(); + if (period <= 0) + { + logger.info("Invalid role disconnection is disabled"); + return; + } + LongSupplier delayMillis = () -> period + ThreadLocalRandom.current().nextLong(0, jitter); + long firstDelayMillis = delayMillis.getAsLong(); + ScheduledExecutorPlus executor = ScheduledExecutors.optionalTasks; + + logger.debug("Scheduling first invalid role disconnection in {} millis", firstDelayMillis); + this.invalidRoleDisconnectTask = executor.schedule(() -> invalidRoleDisconnectTask(delayMillis, executor), firstDelayMillis, TimeUnit.MILLISECONDS); + } + + @Override + public long getInvalidClientDisconnectPeriodMillis() + { + return this.invalidClientDisconnectPeriodMillis; + } + + @Override + public void setInvalidClientDisconnectPeriodMillis(long duration) + { + this.invalidClientDisconnectPeriodMillis = duration; + scheduleDisconnectInvalidRoleTask(); + } + + @Override + public long getInvalidClientDisconnectMaxJitterMillis() + { + return this.invalidClientDisconnectMaxJitterMillis; + } + + @Override + public void setInvalidClientDisconnectMaxJitterMillis(long duration) + { + this.invalidClientDisconnectMaxJitterMillis = duration; + scheduleDisconnectInvalidRoleTask(); + } } diff --git a/src/java/org/apache/cassandra/auth/CassandraRoleManagerMBean.java b/src/java/org/apache/cassandra/auth/CassandraRoleManagerMBean.java new file mode 100644 index 000000000000..f869ac70bcf7 --- /dev/null +++ b/src/java/org/apache/cassandra/auth/CassandraRoleManagerMBean.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.auth; + +/** + * MBean utilities for dynamic access to CassandraRoleManager + */ +public interface CassandraRoleManagerMBean +{ + /** + * Get the period between invalid client disconnect attempts + * @return time between attempts in milliseconds + */ + long getInvalidClientDisconnectPeriodMillis(); + + /** + * Set the period between invalid client disconnect attempts + * @param duration time between attempts in milliseconds + */ + void setInvalidClientDisconnectPeriodMillis(long duration); + + /** + * Get the maximum jitter between invalid client disconnect attempts + * @return maximum jitter in milliseconds + */ + long getInvalidClientDisconnectMaxJitterMillis(); + + /** + * Set the maximum jitter between invalid client disconnect attempts + * @param duration maximum jitter in milliseconds + */ + void setInvalidClientDisconnectMaxJitterMillis(long duration); +} diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java index df73f14b40c1..8c44c4286d36 100644 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@ -31,6 +31,7 @@ import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.function.Predicate; import java.util.stream.Stream; import javax.management.StandardMBean; import javax.management.remote.JMXConnectorServer; @@ -45,6 +46,7 @@ import com.codahale.metrics.SharedMetricRegistries; import org.apache.cassandra.audit.AuditLogManager; import org.apache.cassandra.auth.AuthCacheService; +import org.apache.cassandra.auth.AuthenticatedUser; import org.apache.cassandra.concurrent.ScheduledExecutors; import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.config.DatabaseDescriptor; @@ -887,6 +889,11 @@ public void clearConnectionHistory() nativeTransportService.clearConnectionHistory(); } + public void disconnectUser(Predicate<AuthenticatedUser> userPredicate) + { + nativeTransportService.disconnect(userPredicate); + } + private void exitOrFail(int code, String message) { exitOrFail(code, message, null); diff --git a/src/java/org/apache/cassandra/service/NativeTransportService.java b/src/java/org/apache/cassandra/service/NativeTransportService.java index d6089990017e..5766be65c658 100644 --- a/src/java/org/apache/cassandra/service/NativeTransportService.java +++ b/src/java/org/apache/cassandra/service/NativeTransportService.java @@ -19,6 +19,7 @@ import java.net.InetAddress; import java.util.concurrent.TimeUnit; +import java.util.function.Predicate; import com.google.common.annotations.VisibleForTesting; @@ -30,6 +31,7 @@ import io.netty.channel.epoll.EpollEventLoopGroup; import io.netty.channel.nio.NioEventLoopGroup; import io.netty.util.Version; +import org.apache.cassandra.auth.AuthenticatedUser; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.EncryptionOptions; import org.apache.cassandra.metrics.ClientMetrics; @@ -164,4 +166,9 @@ public void clearConnectionHistory() { server.clearConnectionHistory(); } + + public void disconnect(Predicate<AuthenticatedUser> userPredicate) + { + server.disconnect(userPredicate); + } } diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 12981f724ac2..3817077df0d3 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -4767,6 +4767,13 @@ public void clearConnectionHistory() daemon.clearConnectionHistory(); logger.info("Cleared connection history"); } + + public void disconnectInvalidRoles() + { + logger.info("Disconnecting invalid roles"); + daemon.disconnectUser(user -> !user.canLogin()); + } + public void disableAuditLog() { AuditLogManager.instance.disableAuditLog(); @@ -5463,5 +5470,4 @@ public void setEnforceNativeDeadlineForHints(boolean value) { DatabaseDescriptor.setEnforceNativeDeadlineForHints(value); } - } diff --git a/src/java/org/apache/cassandra/transport/Server.java b/src/java/org/apache/cassandra/transport/Server.java index c6f42da3766f..e657dd1257b6 100644 --- a/src/java/org/apache/cassandra/transport/Server.java +++ b/src/java/org/apache/cassandra/transport/Server.java @@ -211,6 +211,11 @@ private void close(boolean force) logger.info("Stop listening for CQL clients"); } + public void disconnect(Predicate<AuthenticatedUser> userPredicate) + { + connectionTracker.disconnectByUser(userPredicate); + } + public static class Builder { private EventLoopGroup workerGroup; @@ -383,6 +388,23 @@ Map<String, Integer> countConnectedClientsByUser() return result; } + void disconnectByUser(Predicate<AuthenticatedUser> userPredicate) + { + for (Channel c : allChannels) + { + Connection connection = c.attr(Connection.attributeKey).get(); + if (connection instanceof ServerConnection) + { + ServerConnection conn = (ServerConnection) connection; + AuthenticatedUser user = conn.getClientState().getUser(); + if (user == null || userPredicate.test(user)) + { + logger.info("Closing channel with remote address {} with user {}", conn.channel().remoteAddress(), user); + connection.channel().close(); + } + } + } + } } private static class LatestEvent diff --git a/test/distributed/org/apache/cassandra/distributed/test/auth/RoleRevocationTest.java b/test/distributed/org/apache/cassandra/distributed/test/auth/RoleRevocationTest.java new file mode 100644 index 000000000000..5b3ae3512824 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/auth/RoleRevocationTest.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.auth; + +import java.util.function.Consumer; + +import org.junit.Test; + +import com.datastax.driver.core.PlainTextAuthProvider; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.policies.ConstantReconnectionPolicy; +import com.datastax.driver.core.policies.ReconnectionPolicy; +import org.apache.cassandra.auth.AuthenticatedUser; +import org.apache.cassandra.auth.IRoleManager; +import org.apache.cassandra.auth.RoleOptions; +import org.apache.cassandra.auth.RoleResource; +import org.apache.cassandra.auth.Roles; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.api.ICluster; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.test.JavaDriverUtils; +import org.apache.cassandra.distributed.test.TestBaseImpl; +import org.apache.cassandra.service.StorageService; +import org.assertj.core.api.Assertions; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.awaitility.Awaitility.await; + +public class RoleRevocationTest extends TestBaseImpl +{ + // Ensure both the coordinator of the DDL and a replica both change connection state accordingly + private static final int CLUSTER_SIZE = 2; + private static final String USERNAME = "revoke_me"; + private static final String PASSWORD = "i_deserve_disconnection"; + private static final PlainTextAuthProvider CLIENT_AUTH_PROVIDER = new PlainTextAuthProvider(USERNAME, PASSWORD); + private static final ReconnectionPolicy RECONNECTION_POLICY = new ConstantReconnectionPolicy(100); + + public static ICluster<IInvokableInstance> cluster() throws Exception + { + Cluster.Builder builder = Cluster.build(CLUSTER_SIZE); + + builder.withConfig(c -> c.set("authenticator.class_name", "org.apache.cassandra.auth.PasswordAuthenticator") + .set("role_manager", "CassandraRoleManager") + .set("authorizer", "CassandraAuthorizer") + .with(Feature.NETWORK, Feature.NATIVE_PROTOCOL, Feature.GOSSIP)); + ICluster<IInvokableInstance> cluster = builder.start(); + + cluster.get(1).runOnInstance(() -> { + RoleOptions opts = new RoleOptions(); + opts.setOption(IRoleManager.Option.PASSWORD, PASSWORD); + opts.setOption(IRoleManager.Option.LOGIN, true); + DatabaseDescriptor.getRoleManager().createRole(AuthenticatedUser.SYSTEM_USER, RoleResource.role(USERNAME), opts); + }); + + return cluster; + } + + @Test + public void alterRoleLoginFalse() throws Exception + { + test(instance -> { + instance.runOnInstance(() -> { + RoleOptions opts = new RoleOptions(); + opts.setOption(IRoleManager.Option.LOGIN, false); + DatabaseDescriptor.getRoleManager().alterRole(AuthenticatedUser.SYSTEM_USER, RoleResource.role(USERNAME), opts); + }); + }); + } + + @Test + public void dropRole() throws Exception + { + test(instance -> { + instance.runOnInstance(() -> { + DatabaseDescriptor.getRoleManager().dropRole(AuthenticatedUser.SYSTEM_USER, RoleResource.role(USERNAME)); + }); + }); + } + + private void test(Consumer<IInvokableInstance> action) throws Exception + { + ICluster<IInvokableInstance> CLUSTER = cluster(); + com.datastax.driver.core.Cluster driver = JavaDriverUtils.create(CLUSTER, null, builder -> builder + .withAuthProvider(CLIENT_AUTH_PROVIDER) + .withReconnectionPolicy(RECONNECTION_POLICY)); + Session session = driver.connect(); + + // One control, one data connection per host + Assertions.assertThat(driver.getMetrics().getOpenConnections().getValue()).isEqualTo(CLUSTER_SIZE + 1); + Assertions.assertThat(driver.getMetrics().getConnectedToHosts().getValue()).isEqualTo(CLUSTER_SIZE); + Assertions.assertThat(driver.getMetrics().getErrorMetrics().getAuthenticationErrors().getCount()).isEqualTo(0); + + action.accept(CLUSTER.get(1)); + + CLUSTER.forEach(instance -> { + instance.runOnInstance(() -> { + Roles.cache.invalidate(); + StorageService.instance.disconnectInvalidRoles(); + }); + }); + + await().pollDelay(100, MILLISECONDS) + .pollInterval(100, MILLISECONDS) + .atMost(10, SECONDS) + .untilAsserted(() -> { + // Should disconnect from both the coordinator of the DDL and the replica that is notified + Assertions.assertThat(driver.getMetrics().getOpenConnections().getValue()).isEqualTo(0); + Assertions.assertThat(driver.getMetrics().getConnectedToHosts().getValue()).isEqualTo(0); + }); + + await().pollDelay(100, MILLISECONDS) + .pollInterval(100, MILLISECONDS) + .atMost(10, SECONDS) + .untilAsserted(() -> { + long authErrors = session.getCluster().getMetrics().getErrorMetrics().getAuthenticationErrors().getCount(); + Assertions.assertThat(authErrors).isGreaterThan(0); + }); + + session.close(); + driver.close(); + CLUSTER.close(); + } +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/auth/CassandraRoleManagerTest.java b/test/unit/org/apache/cassandra/auth/CassandraRoleManagerTest.java index 658d4b2bb85d..0ee958716403 100644 --- a/test/unit/org/apache/cassandra/auth/CassandraRoleManagerTest.java +++ b/test/unit/org/apache/cassandra/auth/CassandraRoleManagerTest.java @@ -18,18 +18,24 @@ package org.apache.cassandra.auth; +import java.util.HashMap; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.service.StorageService; +import org.assertj.core.api.Assertions; import static org.apache.cassandra.auth.AuthTestUtils.*; import static org.junit.Assert.assertEquals; @@ -37,6 +43,8 @@ public class CassandraRoleManagerTest { + private static final Logger logger = LoggerFactory.getLogger(CassandraRoleManagerTest.class); + @BeforeClass public static void setupClass() { @@ -160,4 +168,76 @@ private void assertRoleSet(Set<Role> actual, RoleResource...expected) for (RoleResource expectedRole : expected) assertTrue(actual.stream().anyMatch(role -> role.resource.equals(expectedRole))); } + + @Test + public void disconnectsAttemptedOnPeriodWithJitter() throws InterruptedException + { + AtomicInteger numDisconnectAttempts = new AtomicInteger(); + + // min: 800ms, max: 900ms + Map<String, String> params = Map.of( + CassandraRoleManager.PARAM_INVALID_ROLE_DISCONNECT_TASK_PERIOD, "800ms", + CassandraRoleManager.PARAM_INVALID_ROLE_DISCONNECT_TASK_MAX_JITTER, "100ms" + ); + + CassandraRoleManager crm = new CassandraRoleManager(params) { + @Override + protected void disconnectInvalidRoles() + { + logger.info("Disconnecting invalid roles..."); + numDisconnectAttempts.incrementAndGet(); + } + }; + + crm.scheduleDisconnectInvalidRoleTask(); + Thread.sleep(3_000); + Assertions.assertThat(numDisconnectAttempts.get()).isGreaterThanOrEqualTo(3); + Assertions.assertThat(numDisconnectAttempts.get()).isLessThan(4); + numDisconnectAttempts.set(0); + + crm.setInvalidClientDisconnectPeriodMillis(100); // min: 100ms, max: 200ms + Thread.sleep(3_000); + Assertions.assertThat(numDisconnectAttempts.get()).isGreaterThanOrEqualTo(10); // 15 - padding + Assertions.assertThat(numDisconnectAttempts.get()).isLessThan(30); + + crm.setInvalidClientDisconnectPeriodMillis(0); + int totalDisconnectAttempts = numDisconnectAttempts.get(); + Thread.sleep(3_000); + Assertions.assertThat(numDisconnectAttempts.get()).isEqualTo(totalDisconnectAttempts); + } + + @Test + public void ctorInvalidRoleDisconnectOptions() + { + CassandraRoleManager crm = new CassandraRoleManager(Map.of()); + Assertions.assertThat(crm.getInvalidClientDisconnectPeriodMillis()).isEqualTo(0); + Assertions.assertThat(crm.getInvalidClientDisconnectMaxJitterMillis()).isEqualTo(0); + + crm = new CassandraRoleManager(Map.of( + CassandraRoleManager.PARAM_INVALID_ROLE_DISCONNECT_TASK_PERIOD, "1s", + CassandraRoleManager.PARAM_INVALID_ROLE_DISCONNECT_TASK_MAX_JITTER, "2s" + )); + Assertions.assertThat(crm.getInvalidClientDisconnectPeriodMillis()).isEqualTo(1000); + Assertions.assertThat(crm.getInvalidClientDisconnectMaxJitterMillis()).isEqualTo(2000); + + // Non-duration input + Map<String, String> params = new HashMap<>(); + params.put(CassandraRoleManager.PARAM_INVALID_ROLE_DISCONNECT_TASK_PERIOD, "notduration"); + Assertions.assertThatThrownBy(() -> new CassandraRoleManager(params)).isOfAnyClassIn(IllegalArgumentException.class).hasMessageContaining("Invalid duration: "); + + // Both fields optional + crm = new CassandraRoleManager(Map.of( + CassandraRoleManager.PARAM_INVALID_ROLE_DISCONNECT_TASK_PERIOD, "1s" + // No jitter + )); + Assertions.assertThat(crm.getInvalidClientDisconnectPeriodMillis()).isEqualTo(1000); + Assertions.assertThat(crm.getInvalidClientDisconnectMaxJitterMillis()).isEqualTo(0); + + crm = new CassandraRoleManager(Map.of( + // No period + CassandraRoleManager.PARAM_INVALID_ROLE_DISCONNECT_TASK_MAX_JITTER, "1s" + )); + Assertions.assertThat(crm.getInvalidClientDisconnectPeriodMillis()).isEqualTo(0); + Assertions.assertThat(crm.getInvalidClientDisconnectMaxJitterMillis()).isEqualTo(1000); + } } diff --git a/test/unit/org/apache/cassandra/auth/RolesTest.java b/test/unit/org/apache/cassandra/auth/RolesTest.java index 0a64d1ffc74e..90cf57f05465 100644 --- a/test/unit/org/apache/cassandra/auth/RolesTest.java +++ b/test/unit/org/apache/cassandra/auth/RolesTest.java @@ -31,6 +31,7 @@ import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ConsistencyLevel; +import org.assertj.core.api.Assertions; import static org.apache.cassandra.auth.AuthTestUtils.ALL_ROLES; import static org.apache.cassandra.auth.AuthTestUtils.ROLE_A; @@ -139,4 +140,13 @@ public void testSuperUsers() .map(RoleResource::getRoleName) .collect(Collectors.toSet())); } + + @Test + public void testNonexistentRoleCantLogin() + { + // There can be a reference to a nonexistent role (that has been removed from the cache and the system table) + // via the native transport connection state, make sure there's no NPE on canLogin check + AuthenticatedUser nonexistent = new AuthenticatedUser("nonexistent"); + Assertions.assertThat(nonexistent.canLogin()).isFalse(); + } } From 46ef00a31594312403e8b42e66ae9354e7bf87ef Mon Sep 17 00:00:00 2001 From: Maxwell Guo <cclive1601@gmail.com> Date: Fri, 11 Oct 2024 19:23:41 +0800 Subject: [PATCH 079/225] Implementation of CEP-43 patch by Maxwell Guo; reviewed by Stefan Miklosovic, Benjamin Lerer for CASSANDRA-19964 --- CHANGES.txt | 1 + NEWS.txt | 2 + doc/cql3/CQL.textile | 33 ++ .../examples/BNF/create_table_like.bnf | 3 + .../examples/CQL/create_table_like.cql | 14 + pylib/cqlshlib/cql3handling.py | 22 + pylib/cqlshlib/test/test_cql_parsing.py | 3 + pylib/cqlshlib/test/test_cqlsh_completion.py | 117 +++- src/antlr/Parser.g | 12 + .../cassandra/audit/AuditLogEntryType.java | 1 + .../statements/schema/CopyTableStatement.java | 220 +++++++ .../schema/CreateTableStatement.java | 23 +- .../test/CreateTableNonDeterministicTest.java | 25 + .../test/metric/TableMetricTest.java | 6 + .../cassandra/audit/AuditLoggerTest.java | 23 + .../cassandra/auth/GrantAndRevokeTest.java | 92 +++ .../cql3/AlterSchemaStatementTest.java | 3 +- .../org/apache/cassandra/cql3/CQLTester.java | 73 +++ .../statements/DescribeStatementTest.java | 62 ++ .../createlike/CreateLikeCqlParseTest.java | 51 ++ .../schema/createlike/CreateLikeTest.java | 557 ++++++++++++++++++ .../createlike/CreateLikeWithSessionTest.java | 71 +++ 22 files changed, 1404 insertions(+), 10 deletions(-) create mode 100644 doc/modules/cassandra/examples/BNF/create_table_like.bnf create mode 100644 doc/modules/cassandra/examples/CQL/create_table_like.cql create mode 100644 src/java/org/apache/cassandra/cql3/statements/schema/CopyTableStatement.java create mode 100644 test/unit/org/apache/cassandra/schema/createlike/CreateLikeCqlParseTest.java create mode 100644 test/unit/org/apache/cassandra/schema/createlike/CreateLikeTest.java create mode 100644 test/unit/org/apache/cassandra/schema/createlike/CreateLikeWithSessionTest.java diff --git a/CHANGES.txt b/CHANGES.txt index ab3b662f6acb..02a0be0571cb 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Implementation of CEP-43 (CASSANDRA-19964) * Periodically disconnect roles that are revoked or have LOGIN=FALSE set (CASSANDRA-19385) * AST library for CQL-based fuzz tests (CASSANDRA-20198) * Support audit logging for JMX operations (CASSANDRA-20128) diff --git a/NEWS.txt b/NEWS.txt index 22921153da2f..a4cec79660a9 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -107,6 +107,8 @@ New features JMX to JVM parameters. You have to opt-in to use the configuration via cassandra.yaml by uncommenting the respective configuration sections and by commenting out `configure_jmx` function call in cassandra-env.sh. Enabling both ways of configuring JMX will result in a node failing to start. + - CEP-43 - it is possible to create a table by "copying" as `CREATE TABLE ks.tb_copy LIKE ks.tb;`. + A newly created table will have no data. Upgrading --------- diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile index 4c2b09acc37d..f819ec193d28 100644 --- a/doc/cql3/CQL.textile +++ b/doc/cql3/CQL.textile @@ -406,6 +406,39 @@ h4. Other considerations: * When "inserting":#insertStmt / "updating":#updateStmt a given row, not all columns needs to be defined (except for those part of the key), and missing columns occupy no space on disk. Furthermore, adding new columns (see <a href=#alterStmt><tt>ALTER TABLE</tt></a>) is a constant time operation. There is thus no need to try to anticipate future usage (or to cry when you haven't) when creating a table. +h3(#copyStmt). CREATE TABLE LIKE + +__Syntax:__ + +bc(syntax).. +<copy-table-stmt> ::= CREATE ( TABLE | COLUMNFAMILY ) ( IF NOT EXISTS )? <newtablename> LIKE <oldtablename> + ( WITH <option> ( AND <option>)* )? + +<option> ::= <property> + +p. +__Sample:__ + +bc(sample).. +CREATE TABLE newtb1 LIKE oldtb; + +CREATE TABLE newtb2 LIKE oldtb WITH compaction = { 'class' : 'LeveledCompactionStrategy' }; +p. +The @COPY TABLE@ statement creates a new table which is a clone of old table. The new table have the same column numbers, column names, column data types, column data mask with the old table. The new table is defined by a "name":#copyNewTableName, and the name of the old table being cloned is defined by a "name":#copyOldTableName . The table options of the new table can be defined by setting "copyoptions":#copyTableOptions. Note that the @CREATE COLUMNFAMILY LIKE@ syntax is supported as an alias for @CREATE TABLE like@ (for historical reasons). + +Attempting to create an already existing table will return an error unless the @IF NOT EXISTS@ option is used. If it is used, the statement will be a no-op if the table already exists. + +h4(#copyNewTableName). @<newtablename>@ + +Valid table names are the same as valid "keyspace names":#createKeyspaceStmt (up to 32 characters long alphanumerical identifiers). If the table name is provided alone, the table is created within the current keyspace (see <a href="#useStmt"><tt>USE</tt></a>), but if it is prefixed by an existing keyspace name (see "@<tablename>@":#statements grammar), it is created in the specified keyspace (but does *not* change the current keyspace). + +h4(#copyOldTableName). @<oldtablename>@ + +The old table name defines the already existed table. + +h4(#copyTableOptions). @<copyoptions>@ + +The @COPY TABLE@ statement supports a number of options that controls the configuration of a new table. These options can be specified after the @WITH@ keyword, and all options are the same as those options when creating a table except for id . h3(#alterTableStmt). ALTER TABLE diff --git a/doc/modules/cassandra/examples/BNF/create_table_like.bnf b/doc/modules/cassandra/examples/BNF/create_table_like.bnf new file mode 100644 index 000000000000..56d209c6ef28 --- /dev/null +++ b/doc/modules/cassandra/examples/BNF/create_table_like.bnf @@ -0,0 +1,3 @@ +create_table_statement::= CREATE TABLE [ IF NOT EXISTS ] new_table_name LIKE old_table_name + [ WITH table_options ] +table_options::= options [ AND table_options ] \ No newline at end of file diff --git a/doc/modules/cassandra/examples/CQL/create_table_like.cql b/doc/modules/cassandra/examples/CQL/create_table_like.cql new file mode 100644 index 000000000000..ef65b8b0509c --- /dev/null +++ b/doc/modules/cassandra/examples/CQL/create_table_like.cql @@ -0,0 +1,14 @@ +CREATE TABLE ks.newtb1 LIKE ks.oldtb; + +CREATE TABLE ks1.newtb1 LIKE ks.oldtb; + +USE ks; + +CREATE TABLE newtb1 LIKE oldtb; + +CREATE TABLE IF NOT EXISTS newtb2 LIKE oldtb; + +CREATE TABLE newtb3 LIKE oldtb WITH compaction = { 'class' : 'LeveledCompactionStrategy' } + AND compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 32 } + AND cdc = true; + diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py index 457582f3d379..4d2f4dffcf14 100644 --- a/pylib/cqlshlib/cql3handling.py +++ b/pylib/cqlshlib/cql3handling.py @@ -279,6 +279,7 @@ def dequote_value(cqlword): <schemaChangeStatement> ::= <createKeyspaceStatement> | <createColumnFamilyStatement> + | <copyTableStatement> | <createIndexStatement> | <createMaterializedViewStatement> | <createUserTypeStatement> @@ -1302,6 +1303,27 @@ def create_cf_composite_primary_key_comma_completer(ctxt, cass): return [','] +syntax_rules += r''' +<copyTableStatement> ::= "CREATE" wat=("COLUMNFAMILY" | "TABLE" ) ("IF" "NOT" "EXISTS")? + ( tks=<nonSystemKeyspaceName> dot="." )? tcf=<cfOrKsName> + "LIKE" ( sks=<nonSystemKeyspaceName> dot="." )? scf=<cfOrKsName> + ( "WITH" <property> ( "AND" <property> )* )? + ; +''' + + +@completer_for('copyTableStatement', 'wat') +def create_tb_wat_completer(ctxt, cass): + # would prefer to get rid of the "schema" nomenclature in cql3 + if ctxt.get_binding('partial', '') == '': + return ['TABLE'] + return ['COLUMNFAMILY', 'TABLE'] + + +explain_completion('copyTableStatement', 'tcf', '<new_table_name>') +explain_completion('copyTableStatement', 'scf', '<old_table_name>') + + syntax_rules += r''' <idxName> ::= <identifier> diff --git a/pylib/cqlshlib/test/test_cql_parsing.py b/pylib/cqlshlib/test/test_cql_parsing.py index 00227c83e6c6..b9eb716a7843 100644 --- a/pylib/cqlshlib/test/test_cql_parsing.py +++ b/pylib/cqlshlib/test/test_cql_parsing.py @@ -568,6 +568,9 @@ def test_parse_drop_keyspace(self): def test_parse_create_table(self): pass + def test_parse_create_table_like(self): + pass + def test_parse_drop_table(self): pass diff --git a/pylib/cqlshlib/test/test_cqlsh_completion.py b/pylib/cqlshlib/test/test_cqlsh_completion.py index 112474e7c712..2a02fcf464ba 100644 --- a/pylib/cqlshlib/test/test_cqlsh_completion.py +++ b/pylib/cqlshlib/test/test_cqlsh_completion.py @@ -397,7 +397,7 @@ def test_complete_in_update(self): choices=['EXISTS', '<quotedName>', '<identifier>']) self.trycompletions("UPDATE empty_table SET lonelycol = 'eggs' WHERE TOKEN(lonelykey) <= TOKEN(13) IF EXISTS ", - choices=['>=', '!=', '<=', 'IN','[', ';', '=', '<', '>', '.', 'CONTAINS']) + choices=['>=', '!=', '<=', 'IN', '[', ';', '=', '<', '>', '.', 'CONTAINS']) self.trycompletions("UPDATE empty_table SET lonelycol = 'eggs' WHERE TOKEN(lonelykey) <= TOKEN(13) IF lonelykey ", choices=['>=', '!=', '<=', 'IN', '=', '<', '>', 'CONTAINS']) @@ -464,7 +464,7 @@ def test_complete_in_delete(self): choices=['a', 'b', 'TOKEN(']) self.trycompletions('DELETE FROM twenty_rows_composite_table USING TIMESTAMP 0 WHERE a ', - choices=['<=', '>=', 'BETWEEN', 'CONTAINS', 'IN', 'NOT' , '[', '=', '<', '>', '!=']) + choices=['<=', '>=', 'BETWEEN', 'CONTAINS', 'IN', 'NOT', '[', '=', '<', '>', '!=']) self.trycompletions('DELETE FROM twenty_rows_composite_table USING TIMESTAMP 0 WHERE TOKEN(', immediate='a ') @@ -614,9 +614,9 @@ def create_columnfamily_table_template(self, name): self.trycompletions(prefix + 'IF NOT EXISTS ', choices=['<new_table_name>', self.cqlsh.keyspace]) self.trycompletions(prefix + 'IF NOT EXISTS new_table ', - immediate='( ') + choices=['(', '.', 'LIKE']) - self.trycompletions(prefix + quoted_keyspace, choices=['.', '(']) + self.trycompletions(prefix + quoted_keyspace, choices=['.', '(', 'LIKE']) self.trycompletions(prefix + quoted_keyspace + '( ', choices=['<new_column_name>', '<identifier>', @@ -625,7 +625,7 @@ def create_columnfamily_table_template(self, name): self.trycompletions(prefix + quoted_keyspace + '.', choices=['<new_table_name>']) self.trycompletions(prefix + quoted_keyspace + '.new_table ', - immediate='( ') + choices=['(', 'LIKE']) self.trycompletions(prefix + quoted_keyspace + '.new_table ( ', choices=['<new_column_name>', '<identifier>', '<quotedName>']) @@ -780,6 +780,113 @@ def test_complete_in_create_table(self): self.trycompletions('CREATE TA', immediate='BLE ') self.create_columnfamily_table_template('TABLE') + def test_complete_in_create_table_like(self): + self.trycompletions('CREATE T', choices=['TRIGGER', 'TABLE', 'TYPE']) + self.trycompletions('CREATE TA', immediate='BLE ') + quoted_keyspace = '"' + self.cqlsh.keyspace + '"' + self.trycompletions('CREATE TABLE ', + choices=['IF', self.cqlsh.keyspace, '<new_table_name>']) + self.trycompletions('CREATE TABLE IF ', + immediate='NOT EXISTS ') + self.trycompletions('CREATE TABLE IF NOT EXISTS ', + choices=['<new_table_name>', self.cqlsh.keyspace]) + self.trycompletions('CREATE TABLE IF NOT EXISTS new_table L', + immediate='IKE ') + self.trycompletions('CREATE TABLE ' + quoted_keyspace + '.', + choices=['<new_table_name>']) + self.trycompletions('CREATE TABLE ' + quoted_keyspace + '.new_table L', + immediate='IKE ') + self.trycompletions('CREATE TABLE ' + 'new_table LIKE old_table W', + immediate='ITH ') + self.trycompletions('CREATE TABLE ' + 'new_table LIKE old_table WITH ', + choices=['allow_auto_snapshot', + 'bloom_filter_fp_chance', 'compaction', + 'compression', + 'default_time_to_live', 'gc_grace_seconds', + 'incremental_backups', + 'max_index_interval', + 'memtable', + 'memtable_flush_period_in_ms', + 'caching', 'comment', + 'min_index_interval', 'speculative_retry', 'additional_write_policy', 'cdc', 'read_repair']) + self.trycompletions('CREATE TABLE ' + 'new_table LIKE old_table WITH ', + choices=['allow_auto_snapshot', + 'bloom_filter_fp_chance', 'compaction', + 'compression', + 'default_time_to_live', 'gc_grace_seconds', + 'incremental_backups', + 'max_index_interval', + 'memtable', + 'memtable_flush_period_in_ms', + 'caching', 'comment', + 'min_index_interval', 'speculative_retry', 'additional_write_policy', 'cdc', 'read_repair']) + self.trycompletions('CREATE TABLE ' + 'new_table LIKE old_table WITH bloom_filter_fp_chance ', + immediate='= ') + self.trycompletions('CREATE TABLE ' + 'new_table LIKE old_table WITH bloom_filter_fp_chance = ', + choices=['<float_between_0_and_1>']) + + self.trycompletions('CREATE TABLE ' + 'new_table LIKE old_table WITH compaction ', + immediate="= {'class': '") + self.trycompletions('CREATE TABLE ' + "new_table LIKE old_table WITH compaction = " + + "{'class': '", + choices=['SizeTieredCompactionStrategy', + 'LeveledCompactionStrategy', + 'TimeWindowCompactionStrategy', + 'UnifiedCompactionStrategy']) + self.trycompletions('CREATE TABLE ' + "new_table LIKE old_table WITH compaction = " + + "{'class': 'S", + immediate="izeTieredCompactionStrategy'") + self.trycompletions('CREATE TABLE ' + "new_table LIKE old_table WITH compaction = " + + "{'class': 'SizeTieredCompactionStrategy", + immediate="'") + self.trycompletions('CREATE TABLE ' + "new_table LIKE old_table WITH compaction = " + + "{'class': 'SizeTieredCompactionStrategy'", + choices=['}', ',']) + self.trycompletions('CREATE TABLE ' + "new_table LIKE old_table WITH compaction = " + + "{'class': 'SizeTieredCompactionStrategy', ", + immediate="'") + self.trycompletions('CREATE TABLE ' + "new_table LIKE old_table WITH compaction = " + + "{'class': 'SizeTieredCompactionStrategy', '", + choices=['bucket_high', 'bucket_low', 'class', + 'enabled', 'max_threshold', + 'min_sstable_size', 'min_threshold', + 'tombstone_compaction_interval', + 'tombstone_threshold', + 'unchecked_tombstone_compaction', + 'only_purge_repaired_tombstones', + 'provide_overlapping_tombstones']) + self.trycompletions('CREATE TABLE ' + "new_table LIKE old_table WITH compaction = " + + "{'class': 'SizeTieredCompactionStrategy'}", + choices=[';', 'AND']) + self.trycompletions('CREATE TABLE ' + "new_table LIKE old_table WITH compaction = " + + "{'class': 'SizeTieredCompactionStrategy'} AND ", + choices=['allow_auto_snapshot', 'bloom_filter_fp_chance', 'compaction', + 'compression', + 'default_time_to_live', 'gc_grace_seconds', + 'incremental_backups', + 'max_index_interval', + 'memtable', + 'memtable_flush_period_in_ms', + 'caching', 'comment', + 'min_index_interval', 'speculative_retry', 'additional_write_policy', 'cdc', 'read_repair']) + self.trycompletions('CREATE TABLE ' + "new_table LIKE old_table WITH compaction = " + + "{'class': 'TimeWindowCompactionStrategy', '", + choices=['compaction_window_unit', 'compaction_window_size', + 'timestamp_resolution', 'min_threshold', 'class', 'max_threshold', + 'tombstone_compaction_interval', 'tombstone_threshold', + 'enabled', 'unchecked_tombstone_compaction', + 'only_purge_repaired_tombstones', 'provide_overlapping_tombstones']) + self.trycompletions('CREATE TABLE ' + "new_table LIKE old_table WITH compaction = " + + "{'class': 'UnifiedCompactionStrategy', '", + choices=['scaling_parameters', 'min_sstable_size', + 'flush_size_override', 'base_shard_count', 'class', 'target_sstable_size', + 'sstable_growth', 'max_sstables_to_compact', + 'enabled', 'expired_sstable_check_frequency_seconds', + 'unsafe_aggressive_sstable_expiration', 'overlap_inclusion_method', + 'tombstone_threshold', 'tombstone_compaction_interval', + 'unchecked_tombstone_compaction', 'provide_overlapping_tombstones', + 'max_threshold', 'only_purge_repaired_tombstones']) + def test_complete_in_describe(self): # Cassandra-10733 self.trycompletions('DES', immediate='C') # quoted_keyspace = '"' + self.cqlsh.keyspace + '"' diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g index 15c2a0140bbd..940039573399 100644 --- a/src/antlr/Parser.g +++ b/src/antlr/Parser.g @@ -236,6 +236,7 @@ cqlStatement returns [CQLStatement.Raw stmt] | st42=addIdentityStatement { $stmt = st42; } | st43=dropIdentityStatement { $stmt = st43; } | st44=listSuperUsersStatement { $stmt = st44; } + | st45=copyTableStatement { $stmt = st45; } ; /* @@ -813,6 +814,17 @@ tableClusteringOrder[CreateTableStatement.Raw stmt] : k=ident (K_ASC | K_DESC { ascending = false; } ) { $stmt.extendClusteringOrder(k, ascending); } ; +/** + * CREATE TABLE [IF NOT EXISTS] <NEW_TABLE> LIKE <OLD_TABLE> WITH <property> = <value> AND ...; + */ +copyTableStatement returns [CopyTableStatement.Raw stmt] + @init { boolean ifNotExists = false; } + : K_CREATE K_COLUMNFAMILY (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? + newCf=columnFamilyName K_LIKE oldCf=columnFamilyName + { $stmt = new CopyTableStatement.Raw(newCf, oldCf, ifNotExists); } + ( K_WITH property[stmt.attrs] ( K_AND property[stmt.attrs] )*)? + ; + /** * CREATE TYPE foo ( * <name1> <type1>, diff --git a/src/java/org/apache/cassandra/audit/AuditLogEntryType.java b/src/java/org/apache/cassandra/audit/AuditLogEntryType.java index 484895a21b28..2bbff08429e1 100644 --- a/src/java/org/apache/cassandra/audit/AuditLogEntryType.java +++ b/src/java/org/apache/cassandra/audit/AuditLogEntryType.java @@ -32,6 +32,7 @@ public enum AuditLogEntryType ALTER_KEYSPACE(AuditLogEntryCategory.DDL), DROP_KEYSPACE(AuditLogEntryCategory.DDL), CREATE_TABLE(AuditLogEntryCategory.DDL), + CREATE_TABLE_LIKE(AuditLogEntryCategory.DDL), DROP_TABLE(AuditLogEntryCategory.DDL), PREPARE_STATEMENT(AuditLogEntryCategory.PREPARE), DROP_TRIGGER(AuditLogEntryCategory.DDL), diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CopyTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CopyTableStatement.java new file mode 100644 index 000000000000..0df8d00f45e8 --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/statements/schema/CopyTableStatement.java @@ -0,0 +1,220 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.statements.schema; + +import org.apache.cassandra.audit.AuditLogContext; +import org.apache.cassandra.audit.AuditLogEntryType; +import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.cql3.CQLStatement; +import org.apache.cassandra.cql3.QualifiedName; +import org.apache.cassandra.db.guardrails.Guardrails; +import org.apache.cassandra.db.marshal.VectorType; +import org.apache.cassandra.exceptions.AlreadyExistsException; +import org.apache.cassandra.schema.Indexes; +import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.schema.Keyspaces; +import org.apache.cassandra.schema.MemtableParams; +import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.schema.TableParams; +import org.apache.cassandra.schema.Triggers; +import org.apache.cassandra.schema.UserFunctions; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.reads.repair.ReadRepairStrategy; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.transport.Event.SchemaChange; + +/** + * {@code CREATE TABLE [IF NOT EXISTS] <newtable> LIKE <oldtable> WITH <property> = <value>} + */ +public final class CopyTableStatement extends AlterSchemaStatement +{ + private final String sourceKeyspace; + private final String sourceTableName; + private final String targetKeyspace; + private final String targetTableName; + private final boolean ifNotExists; + private final TableAttributes attrs; + + public CopyTableStatement(String sourceKeyspace, + String targetKeyspace, + String sourceTableName, + String targetTableName, + boolean ifNotExists, + TableAttributes attrs) + { + super(targetKeyspace); + this.sourceKeyspace = sourceKeyspace; + this.targetKeyspace = targetKeyspace; + this.sourceTableName = sourceTableName; + this.targetTableName = targetTableName; + this.ifNotExists = ifNotExists; + this.attrs = attrs; + } + + @Override + SchemaChange schemaChangeEvent(Keyspaces.KeyspacesDiff diff) + { + return new SchemaChange(SchemaChange.Change.CREATED, SchemaChange.Target.TABLE, targetKeyspace, targetTableName); + } + + @Override + public void authorize(ClientState client) + { + client.ensureTablePermission(sourceKeyspace, sourceTableName, Permission.SELECT); + client.ensureAllTablesPermission(targetKeyspace, Permission.CREATE); + } + + @Override + public AuditLogContext getAuditLogContext() + { + return new AuditLogContext(AuditLogEntryType.CREATE_TABLE_LIKE, targetKeyspace, targetTableName); + } + + @Override + public Keyspaces apply(ClusterMetadata metadata) + { + Keyspaces schema = metadata.schema.getKeyspaces(); + KeyspaceMetadata sourceKeyspaceMeta = schema.getNullable(sourceKeyspace); + + if (null == sourceKeyspaceMeta) + throw ire("Source Keyspace '%s' doesn't exist", sourceKeyspace); + + TableMetadata sourceTableMeta = sourceKeyspaceMeta.getTableNullable(sourceTableName); + + if (null == sourceTableMeta) + throw ire("Souce Table '%s.%s' doesn't exist", sourceKeyspace, sourceTableName); + + if (sourceTableMeta.isIndex()) + throw ire("Cannot use CREATE TABLE LIKE on an index table '%s.%s'.", sourceKeyspace, sourceTableName); + + if (sourceTableMeta.isView()) + throw ire("Cannot use CREATE TABLE LIKE on a materialized view '%s.%s'.", sourceKeyspace, sourceTableName); + + KeyspaceMetadata targetKeyspaceMeta = schema.getNullable(targetKeyspace); + if (null == targetKeyspaceMeta) + throw ire("Target Keyspace '%s' doesn't exist", targetKeyspace); + + if (targetKeyspaceMeta.hasTable(targetTableName)) + { + if(ifNotExists) + return schema; + + throw new AlreadyExistsException(targetKeyspace, targetTableName); + } + // todo support udt for differenet ks latter + if (!sourceKeyspace.equalsIgnoreCase(targetKeyspace) && !sourceTableMeta.getReferencedUserTypes().isEmpty()) + throw ire("Cannot use CREATE TABLE LIKE across different keyspace when source table have UDTs."); + + // Guardrail on columns per table + Guardrails.columnsPerTable.guard(sourceTableMeta.columns().size(), targetTableName, false, state); + + sourceTableMeta.columns().forEach(columnMetadata -> { + if (columnMetadata.type.isVector()) + { + Guardrails.vectorTypeEnabled.ensureEnabled(columnMetadata.name.toString(), state); + int dimensions = ((VectorType)columnMetadata.type).dimension; + Guardrails.vectorDimensions.guard(dimensions, columnMetadata.name.toString(), false, state); + } + }); + + // Guardrail to check whether creation of new COMPACT STORAGE tables is allowed + if (sourceTableMeta.isCompactTable()) + Guardrails.compactTablesEnabled.ensureEnabled(state); + + if (sourceKeyspaceMeta.replicationStrategy.hasTransientReplicas() + && sourceTableMeta.params.readRepair != ReadRepairStrategy.NONE) + { + throw ire("read_repair must be set to 'NONE' for transiently replicated keyspaces"); + } + + if (!sourceTableMeta.params.compression.isEnabled()) + Guardrails.uncompressedTablesEnabled.ensureEnabled(state); + + // withInternals can be set to false as it is only used for souce table id, which is not need for target table and the table + // id can be set through create table like cql using WITH ID + String sourceCQLString = sourceTableMeta.toCqlString(false, false, false, false); + + TableMetadata.Builder targetBuilder = CreateTableStatement.parse(sourceCQLString, + targetKeyspace, + targetTableName, + sourceKeyspaceMeta.types, + UserFunctions.none()) + .indexes(Indexes.none()) + .triggers(Triggers.none()); + + TableParams originalParams = targetBuilder.build().params; + TableParams newTableParams = attrs.asAlteredTableParams(originalParams); + + TableMetadata table = targetBuilder.params(newTableParams) + .id(TableId.get(metadata)) + .build(); + table.validate(); + + return schema.withAddedOrUpdated(targetKeyspaceMeta.withSwapped(targetKeyspaceMeta.tables.with(table))); + } + + @Override + public void validate(ClientState state) + { + super.validate(state); + + // If a memtable configuration is specified, validate it against config + if (attrs.hasOption(TableParams.Option.MEMTABLE)) + MemtableParams.get(attrs.getString(TableParams.Option.MEMTABLE.toString())); + + // Guardrail on table properties + Guardrails.tableProperties.guard(attrs.updatedProperties(), attrs::removeProperty, state); + + // Guardrail on number of tables + if (Guardrails.tables.enabled(state)) + { + int totalUserTables = Schema.instance.getUserKeyspaces() + .stream() + .mapToInt(ksm -> ksm.tables.size()) + .sum(); + Guardrails.tables.guard(totalUserTables + 1, targetTableName, false, state); + } + validateDefaultTimeToLive(attrs.asNewTableParams()); + } + + public final static class Raw extends CQLStatement.Raw + { + private final QualifiedName oldName; + private final QualifiedName newName; + private final boolean ifNotExists; + public final TableAttributes attrs = new TableAttributes(); + + public Raw(QualifiedName newName, QualifiedName oldName, boolean ifNotExists) + { + this.newName = newName; + this.oldName = oldName; + this.ifNotExists = ifNotExists; + } + + @Override + public CQLStatement prepare(ClientState state) + { + String oldKeyspace = oldName.hasKeyspace() ? oldName.getKeyspace() : state.getKeyspace(); + String newKeyspace = newName.hasKeyspace() ? newName.getKeyspace() : state.getKeyspace(); + return new CopyTableStatement(oldKeyspace, newKeyspace, oldName.getName(), newName.getName(), ifNotExists, attrs); + } + } +} diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java index 7c705f7fee36..5970360f24d5 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java @@ -479,12 +479,21 @@ public String defaultCompactValueName() } } + public static TableMetadata.Builder parse(String cql, String keyspace, String table, Types types, UserFunctions userFunctions) + { + Raw createTable = CQLFragmentParser.parseAny(CqlParser::createTableStatement, cql, "CREATE TABLE") + .keyspace(keyspace); + + if (table != null) + createTable.table(table); + + return createTable.prepare(null) // works around a messy ClientState/QueryProcessor class init deadlock + .builder(types, userFunctions); + } + public static TableMetadata.Builder parse(String cql, String keyspace) { - return CQLFragmentParser.parseAny(CqlParser::createTableStatement, cql, "CREATE TABLE") - .keyspace(keyspace) - .prepare(null) // works around a messy ClientState/QueryProcessor class init deadlock - .builder(Types.none(), UserFunctions.none()); + return parse(cql, keyspace, null, Types.none(), UserFunctions.none()); } public final static class Raw extends CQLStatement.Raw @@ -538,6 +547,12 @@ public Raw keyspace(String keyspace) return this; } + public Raw table(String table) + { + name.setName(table, true); + return this; + } + public String table() { return name.getName(); diff --git a/test/distributed/org/apache/cassandra/distributed/test/CreateTableNonDeterministicTest.java b/test/distributed/org/apache/cassandra/distributed/test/CreateTableNonDeterministicTest.java index 74c08047fce3..0cf551e2e875 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/CreateTableNonDeterministicTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/CreateTableNonDeterministicTest.java @@ -25,10 +25,13 @@ import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; import org.apache.cassandra.distributed.api.IInvokableInstance; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.tcm.ClusterMetadata; +import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows; +import static org.apache.cassandra.distributed.shared.AssertUtils.row; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; @@ -77,6 +80,28 @@ public void testIdClash() throws IOException } } + @Test + public void testCreateLikeTable() throws IOException + { + try (Cluster cluster = init(Cluster.build(2).start())) + { + cluster.schemaChange(withKeyspace("CREATE TABLE %s.sourcetb (k int primary key, v text)")); + TableId node1id = tableId(cluster.get(1), "sourcetb"); + TableId node2id = tableId(cluster.get(2), "sourcetb"); + assertEquals(node1id, node2id); + cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".targettb LIKE " + KEYSPACE + ".sourcetb"); + TableId node1id2 = tableId(cluster.get(1), "targettb"); + TableId node2id2 = tableId(cluster.get(2), "targettb"); + assertNotEquals(node1id, node1id2); + assertEquals(node1id2, node2id2); + cluster.coordinator(1).execute(withKeyspace("INSERT INTO %s.sourcetb(k, v) VALUES (1, 'v1')"), ConsistencyLevel.QUORUM); + cluster.coordinator(1).execute(withKeyspace("INSERT INTO %s.targettb(k, v) VALUES (1, 'v1')"), ConsistencyLevel.QUORUM); + Object[] row = row(1, "v1"); + assertRows(cluster.coordinator(1).execute(withKeyspace("SELECT * FROM %s.sourcetb"), ConsistencyLevel.QUORUM), row); + assertRows(cluster.coordinator(1).execute(withKeyspace("SELECT * FROM %s.targettb "), ConsistencyLevel.QUORUM), row); + } + } + long epoch(IInvokableInstance inst) { return inst.callOnInstance(() -> ClusterMetadata.current().epoch.getEpoch()); diff --git a/test/distributed/org/apache/cassandra/distributed/test/metric/TableMetricTest.java b/test/distributed/org/apache/cassandra/distributed/test/metric/TableMetricTest.java index 58a3feef4074..1ea737b191bb 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/metric/TableMetricTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/metric/TableMetricTest.java @@ -105,10 +105,16 @@ public void userTables() throws IOException cluster.schemaChange(withKeyspace("ALTER TABLE %s.tbl DROP value")); cluster.forEach(i -> assertTableMetricsExist(i, KEYSPACE, "tbl")); + cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".targettb LIKE " + KEYSPACE + ".tbl"); + cluster.forEach(i -> assertTableMetricsExist(i, KEYSPACE, "targettb")); + // drop and make sure table no longer exists cluster.schemaChange(withKeyspace("DROP TABLE %s.tbl")); cluster.forEach(i -> assertTableMetricsDoesNotExist(i, KEYSPACE, "tbl")); + cluster.schemaChange(withKeyspace("DROP TABLE %s.targettb")); + cluster.forEach(i -> assertTableMetricsDoesNotExist(i, KEYSPACE, "tbl")); + cluster.schemaChange(withKeyspace("DROP KEYSPACE %s")); cluster.forEach(i -> assertKeyspaceMetricDoesNotExists(i, KEYSPACE)); diff --git a/test/unit/org/apache/cassandra/audit/AuditLoggerTest.java b/test/unit/org/apache/cassandra/audit/AuditLoggerTest.java index 46345d88fc7d..f97e0bf5cef2 100644 --- a/test/unit/org/apache/cassandra/audit/AuditLoggerTest.java +++ b/test/unit/org/apache/cassandra/audit/AuditLoggerTest.java @@ -560,6 +560,29 @@ public void testCqlTriggerAuditing() throws Throwable executeAndAssert(cql, AuditLogEntryType.DROP_TRIGGER); } + @Test + public void testCqlCreateTableLikeAuditing() throws Throwable + { + String cql = "CREATE TABLE " + KEYSPACE + "." + createTableName() + " (id int primary key, v1 text, v2 text)"; + executeAndAssert(cql, AuditLogEntryType.CREATE_TABLE); + + cql = "CREATE TABLE IF NOT EXISTS " + KEYSPACE + "." + createTableName() + " (id int primary key, v1 text, v2 text)"; + executeAndAssert(cql, AuditLogEntryType.CREATE_TABLE); + + String sourceTable = currentTable(); + + cql = "CREATE TABLE " + KEYSPACE + "." + createTableName() + " LIKE " + KEYSPACE + "." + sourceTable; + executeAndAssert(cql, AuditLogEntryType.CREATE_TABLE_LIKE); + + cql = "INSERT INTO " + KEYSPACE + '.' + currentTable() + " (id, v1, v2) VALUES (?, ?, ?)"; + executeAndAssertWithPrepare(cql, AuditLogEntryType.UPDATE, 1, "insert_audit", "test"); + + cql = "SELECT id, v1, v2 FROM " + KEYSPACE + '.' + currentTable() + " WHERE id = ?"; + ResultSet rs = executeAndAssertWithPrepare(cql, AuditLogEntryType.SELECT, 1); + + assertEquals(1, rs.all().size()); + } + @Test public void testCqlAggregateAuditing() throws Throwable { diff --git a/test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java b/test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java index 55e1135cd46e..2ac02321391d 100644 --- a/test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java +++ b/test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java @@ -386,4 +386,96 @@ public void testWarnings() throws Throwable res = executeNet("REVOKE SELECT, MODIFY ON KEYSPACE revoke_yeah FROM " + user); assertWarningsContain(res.getExecutionInfo().getWarnings(), "Role '" + user + "' was not granted MODIFY on <keyspace revoke_yeah>"); } + + @Test + public void testCreateTableLikeAuthorize() throws Throwable + { + useSuperUser(); + + // two keyspaces + executeNet("CREATE KEYSPACE ks1 WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}"); + executeNet("CREATE KEYSPACE ks2 WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}"); + executeNet("CREATE TABLE ks1.sourcetb (id int PRIMARY KEY, val text)"); + executeNet("CREATE USER '" + user + "' WITH PASSWORD '" + pass + "'"); + + // same keyspace + // have no select permission on source table + ResultSet res = executeNet("REVOKE SELECT ON TABLE ks1.sourcetb FROM " + user); + assertWarningsContain(res.getExecutionInfo().getWarnings(), "Role '" + user + "' was not granted SELECT on <table ks1.sourcetb>"); + + useUser(user, pass); + // Spin assert for effective auth changes. + Util.spinAssertEquals(false, () -> { + try + { + assertUnauthorizedQuery("User user has no SELECT permission on <table ks1.sourcetb> or any of its parents", + formatQuery("SELECT * FROM ks1.sourcetb LIMIT 1")); + } + catch(Throwable e) + { + return true; + } + return false; + }, 10); + + assertUnauthorizedQuery("User user has no SELECT permission on <table ks1.sourcetb> or any of its parents", + "CREATE TABLE ks1.targetTb LIKE ks1.sourcetb"); + + // have select permission on source table and do not have create permission on target keyspace + useSuperUser(); + executeNet("GRANT SELECT ON TABLE ks1.sourcetb TO " + user); + res = executeNet("REVOKE CREATE ON KEYSPACE ks1 FROM " + user); + assertWarningsContain(res.getExecutionInfo().getWarnings(), "Role '" + user + "' was not granted CREATE on <keyspace ks1>"); + + useUser(user, pass); + Util.spinAssertEquals(false, () -> { + try + { + assertUnauthorizedQuery("User user has no CREATE permission on <all tables in ks1> or any of its parents", + formatQuery("CREATE TABLE ks1.targetTb LIKE ks1.sourcetb")); + } + catch(Throwable e) + { + return true; + } + return false; + }, 10); + + assertUnauthorizedQuery("User user has no CREATE permission on <all tables in ks1> or any of its parents", + "CREATE TABLE ks1.targetTb LIKE ks1.sourcetb"); + + // different keyspaces + // have select permission on source table and do not have create permission on target keyspace + useSuperUser(); + executeNet("GRANT SELECT ON TABLE ks1.sourcetb TO " + user); + res = executeNet("REVOKE CREATE ON KEYSPACE ks2 FROM " + user); + assertWarningsContain(res.getExecutionInfo().getWarnings(), "Role '" + user + "' was not granted CREATE on <keyspace ks2>"); + + useUser(user, pass); + Util.spinAssertEquals(false, () -> { + try + { + assertUnauthorizedQuery("User user has no CREATE permission on <all tables in ks2> or any of its parents", + formatQuery("CREATE TABLE ks2.targetTb LIKE ks1.sourcetb")); + } + catch(Throwable e) + { + return true; + } + return false; + }, 10); + + assertUnauthorizedQuery("User user has no CREATE permission on <all tables in ks2> or any of its parents", + "CREATE TABLE ks2.targetTb LIKE ks1.sourcetb"); + + // source keyspace and table do not exist + assertUnauthorizedQuery("User user has no SELECT permission on <table ks1.tbnotexist> or any of its parents", + "CREATE TABLE ks2.targetTb LIKE ks1.tbnotexist"); + assertUnauthorizedQuery("User user has no SELECT permission on <table ksnotexists.sourcetb> or any of its parents", + "CREATE TABLE ks2.targetTb LIKE ksnotexists.sourcetb"); + // target keyspace does not exist + assertUnauthorizedQuery("User user has no CREATE permission on <all tables in ksnotexists> or any of its parents", + "CREATE TABLE ksnotexists.targetTb LIKE ks1.sourcetb"); + + } } diff --git a/test/unit/org/apache/cassandra/cql3/AlterSchemaStatementTest.java b/test/unit/org/apache/cassandra/cql3/AlterSchemaStatementTest.java index 4c0f833504fe..57d7d2b5a5a8 100644 --- a/test/unit/org/apache/cassandra/cql3/AlterSchemaStatementTest.java +++ b/test/unit/org/apache/cassandra/cql3/AlterSchemaStatementTest.java @@ -39,7 +39,8 @@ public class AlterSchemaStatementTest extends CQLTester "CREATE KEYSPACE ks WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", "CREATE TABLE ks.t1 (k int PRIMARY KEY)", "ALTER MATERIALIZED VIEW ks.v1 WITH compaction = { 'class' : 'LeveledCompactionStrategy' }", - "ALTER TABLE ks.t1 ADD v int" + "ALTER TABLE ks.t1 ADD v int", + "CREATE TABLE ks.tb like ks1.tb" }; private final ClientState clientState = ClientState.forExternalCalls(InetSocketAddress.createUnresolved("127.0.0.1", 1234)); diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index 9410564b83dd..92f865b2029e 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -163,6 +163,7 @@ import org.apache.cassandra.metrics.CassandraMetricsRegistry; import org.apache.cassandra.metrics.ClientMetrics; import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.IndexMetadata; import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.Schema; @@ -1080,6 +1081,25 @@ protected String createTable(String keyspace, String query, String tableName) return currentTable; } + protected String createTableLike(String query, String sourceTable, String sourceKeyspace, String targetKeyspace) + { + return createTableLike(query, sourceTable, sourceKeyspace, null, targetKeyspace); + } + + protected String createTableLike(String query, String sourceTable, String sourceKeyspace, String targetTable, String targetKeyspace) + { + if (!tables.contains(sourceTable)) + { + throw new IllegalArgumentException("Source table " + sourceTable + " does not exist"); + } + + String currentTable = createTableName(targetTable); + String fullQuery = currentTable == null ? query : String.format(query, targetKeyspace + "." + currentTable, sourceKeyspace + "." + sourceTable);; + logger.info(fullQuery); + schemaChange(fullQuery); + return currentTable; + } + protected String createTableName() { return createTableName(null); @@ -1545,6 +1565,11 @@ protected static ResultMessage schemaChange(String query) } } + protected TableMetadata getTableMetadata(String keyspace, String table) + { + return Schema.instance.getTableMetadata(keyspace, table); + } + protected TableMetadata currentTableMetadata() { return Schema.instance.getTableMetadata(KEYSPACE, currentTable()); @@ -1935,6 +1960,54 @@ private static boolean isEmptyContainerNull(DataType type, return false; } + /** + * Determine whether the source and target TableMetadata is equal without compare the table name and dropped columns. + * @param source the source TableMetadata + * @param target the target TableMetadata + * @param compareParams wether compare table params + * @param compareIndexes wether compare table's indexes + * @param compareTrigger wether compare table's triggers + * */ + protected boolean equalsWithoutTableNameAndDropCns(TableMetadata source, TableMetadata target, boolean compareParams, boolean compareIndexes, boolean compareTrigger) + { + return source.partitioner.equals(target.partitioner) + && source.kind == target.kind + && source.flags.equals(target.flags) + && (!compareParams || source.params.equals(target.params)) + && (!compareIndexes || source.indexes.equals(target.indexes)) + && (!compareTrigger || source.triggers.equals(target.triggers)) + && columnsEqualWitoutKsTb(source, target); + } + + // only compare columns + private boolean columnsEqualWitoutKsTb(TableMetadata source, TableMetadata target) + { + if (target.columns().size() != source.columns().size()) + return false; + + Iterator<ColumnMetadata> leftIterator = source.allColumnsInCreateOrder(); + Iterator<ColumnMetadata> rightIterator = target.allColumnsInCreateOrder(); + + while (leftIterator.hasNext() && rightIterator.hasNext()) + { + ColumnMetadata leftCn = leftIterator.next(); + ColumnMetadata rightCn = rightIterator.next(); + if (!equalsWithoutKsTb(leftCn, rightCn)) + return false; + } + + return true; + } + + private boolean equalsWithoutKsTb(ColumnMetadata left, ColumnMetadata right) + { + return left.name.equals(right.name) + && left.kind == right.kind + && left.position() == right.position() + && java.util.Objects.equals(left.getMask(), right.getMask()) + && left.type.equals(right.type); + } + protected void assertRowCountNet(ResultSet r1, int expectedCount) { Assert.assertFalse("Received a null resultset when expected count was > 0", expectedCount > 0 && r1 == null); diff --git a/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java b/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java index cd1941416685..f3141a76402f 100644 --- a/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java +++ b/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java @@ -53,6 +53,7 @@ import static org.apache.cassandra.schema.SchemaConstants.TRACE_KEYSPACE_NAME; import static org.apache.cassandra.schema.SchemaConstants.VIRTUAL_SCHEMA; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -822,6 +823,67 @@ public void testDescribeWithCustomIndex() throws Throwable row(KEYSPACE_PER_TEST, "index", indexWithOptions, expectedIndexStmtWithOptions)); } + @Test + public void testDescribeCreateLikeTable() throws Throwable + { + requireNetwork(); + DatabaseDescriptor.setDynamicDataMaskingEnabled(true); + String souceTable = createTable(KEYSPACE_PER_TEST, + "CREATE TABLE %s (" + + " pk1 text, " + + " pk2 int MASKED WITH DEFAULT, " + + " ck1 int, " + + " ck2 double," + + " s1 float static, " + + " v1 int, " + + " v2 int, " + + "PRIMARY KEY ((pk1, pk2), ck1, ck2 ))"); + TableMetadata source = getTableMetadata(KEYSPACE_PER_TEST, currentTable()); + assertNotNull(source); + String targetTable = createTableLike("CREATE TABLE %s LIKE %s", souceTable, KEYSPACE_PER_TEST, KEYSPACE_PER_TEST); + TableMetadata target = getTableMetadata(KEYSPACE_PER_TEST, currentTable()); + assertNotNull(target); + assertTrue(equalsWithoutTableNameAndDropCns(source, target, true, true, true)); + assertNotEquals(source.id, target.id); + assertNotEquals(source.name, target.name); + + String sourceTableCreateStatement = "CREATE TABLE " + KEYSPACE_PER_TEST + "." + souceTable + " (\n" + + " pk1 text,\n" + + " pk2 int MASKED WITH system.mask_default(),\n" + + " ck1 int,\n" + + " ck2 double,\n" + + " s1 float static,\n" + + " v1 int,\n" + + " v2 int,\n" + + " PRIMARY KEY ((pk1, pk2), ck1, ck2)\n" + + ") WITH ID = " + source.id + "\n" + + " AND CLUSTERING ORDER BY (ck1 ASC, ck2 ASC)\n" + + " AND " + tableParametersCql(); + String targetTableCreateStatement = "CREATE TABLE " + KEYSPACE_PER_TEST + "." + targetTable + " (\n" + + " pk1 text,\n" + + " pk2 int MASKED WITH system.mask_default(),\n" + + " ck1 int,\n" + + " ck2 double,\n" + + " s1 float static,\n" + + " v1 int,\n" + + " v2 int,\n" + + " PRIMARY KEY ((pk1, pk2), ck1, ck2)\n" + + ") WITH ID = " + target.id + "\n" + + " AND CLUSTERING ORDER BY (ck1 ASC, ck2 ASC)\n" + + " AND " + tableParametersCql(); + + assertRowsNet(executeDescribeNet("DESCRIBE TABLE " + KEYSPACE_PER_TEST + "." + souceTable + " WITH INTERNALS"), + row(KEYSPACE_PER_TEST, + "table", + souceTable, + sourceTableCreateStatement)); + assertRowsNet(executeDescribeNet("DESCRIBE TABLE " + KEYSPACE_PER_TEST + "." + targetTable + " WITH INTERNALS"), + row(KEYSPACE_PER_TEST, + "table", + targetTable, + targetTableCreateStatement)); + } + @Test public void testDescribeTableWithColumnMasks() throws Throwable { diff --git a/test/unit/org/apache/cassandra/schema/createlike/CreateLikeCqlParseTest.java b/test/unit/org/apache/cassandra/schema/createlike/CreateLikeCqlParseTest.java new file mode 100644 index 000000000000..67a79870d0df --- /dev/null +++ b/test/unit/org/apache/cassandra/schema/createlike/CreateLikeCqlParseTest.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.schema.createlike; + +import org.junit.Test; + +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.cql3.QueryProcessor; +import org.apache.cassandra.exceptions.SyntaxException; + +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; + +public class CreateLikeCqlParseTest extends CQLTester +{ + // Incorrect use of create table like cql + private static final String[] unSupportedCqls = new String[] + { + "CREATE TABLE ta (a int primary key, b int) LIKE tb", // useless column information + "CREATE TABLE ta (a int primary key, b int MASKED WITH DEFAULT) LIKE tb", + "CREATE TABLE IF NOT EXISTS LIKE tb", // missing target table + "CREATE TABLE IF NOT EXISTS LIKE tb WITH compression = { 'enabled' : 'false'}", + "CREATE TABLE ta IF NOT EXISTS LIKE ", // missing source table + "CREATE TABLE ta LIKE WITH id = '123-111'" // id is not supported + }; + + @Test + public void testUnsupportedCqlParse() + { + for (String cql : unSupportedCqls) + { + assertThatExceptionOfType(SyntaxException.class) + .isThrownBy(() -> QueryProcessor.parseStatement(cql)); + } + } +} diff --git a/test/unit/org/apache/cassandra/schema/createlike/CreateLikeTest.java b/test/unit/org/apache/cassandra/schema/createlike/CreateLikeTest.java new file mode 100644 index 000000000000..02406acdfe07 --- /dev/null +++ b/test/unit/org/apache/cassandra/schema/createlike/CreateLikeTest.java @@ -0,0 +1,557 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.schema.createlike; + +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.cql3.Duration; +import org.apache.cassandra.cql3.validation.operations.CreateTest; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.compaction.LeveledCompactionStrategy; +import org.apache.cassandra.exceptions.AlreadyExistsException; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.schema.CachingParams; +import org.apache.cassandra.schema.CompactionParams; +import org.apache.cassandra.schema.CompressionParams; +import org.apache.cassandra.schema.Indexes; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.schema.TableParams; +import org.apache.cassandra.service.reads.SpeculativeRetryPolicy; +import org.apache.cassandra.service.reads.repair.ReadRepairStrategy; +import org.apache.cassandra.utils.TimeUUID; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +@RunWith(Parameterized.class) +public class CreateLikeTest extends CQLTester +{ + @Parameterized.Parameter + public boolean differentKs; + + @Parameterized.Parameters(name = "{index}: differentKs={0}") + public static Collection<Object[]> data() + { + List<Object[]> result = new ArrayList<>(); + result.add(new Object[]{false}); + result.add(new Object[]{true}); + return result; + } + + private UUID uuid1 = UUID.fromString("62c3e96f-55cd-493b-8c8e-5a18883a1698"); + private UUID uuid2 = UUID.fromString("52c3e96f-55cd-493b-8c8e-5a18883a1698"); + private TimeUUID timeUuid1 = TimeUUID.fromString("00346642-2d2f-11ed-a261-0242ac120002"); + private TimeUUID timeUuid2 = TimeUUID.fromString("10346642-2d2f-11ed-a261-0242ac120002"); + private Duration duration1 = Duration.newInstance(1, 2, 3); + private Duration duration2 = Duration.newInstance(1, 2, 4); + private Date date1 = new Date(); + private Double d1 = Double.valueOf("1.1"); + private Double d2 = Double.valueOf("2.2"); + private Float f1 = Float.valueOf("3.33"); + private Float f2 = Float.valueOf("4.44"); + private BigDecimal decimal1 = BigDecimal.valueOf(1.1); + private BigDecimal decimal2 = BigDecimal.valueOf(2.2); + private Vector<Integer> vector1 = vector(1, 2); + private Vector<Integer> vector2 = vector(3, 4); + private String keyspace1 = "keyspace1"; + private String keyspace2 = "keyspace2"; + private String sourceKs; + private String targetKs; + + @Before + public void before() + { + createKeyspace("CREATE KEYSPACE " + keyspace1 + " WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }"); + createKeyspace("CREATE KEYSPACE " + keyspace2 + " WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }"); + sourceKs = keyspace1; + targetKs = differentKs ? keyspace2 : keyspace1; + } + + @Test + public void testTableSchemaCopy() + { + String sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b duration, c text);"); + String targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a, b, c) VALUES (?, ?, ?)", 1, duration1, "1"); + execute("INSERT INTO " + targetKs + "." + targetTb + " (a, b, c) VALUES (?, ?, ?)", 2, duration2, "2"); + assertRows(execute("SELECT * FROM " + sourceKs + "." + sourceTb), + row(1, duration1, "1")); + assertRows(execute("SELECT * FROM " + targetKs + "." + targetTb), + row(2, duration2, "2")); + + sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY);"); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a) VALUES (1)"); + execute("INSERT INTO " + targetKs + "." + targetTb + " (a) VALUES (2)"); + assertRows(execute("SELECT * FROM " + sourceKs + "." + sourceTb), + row(1)); + assertRows(execute("SELECT * FROM " + targetKs + "." + targetTb), + row(2)); + + sourceTb = createTable(sourceKs, "CREATE TABLE %s (a frozen<map<text, text>> PRIMARY KEY);"); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a) VALUES (?)", map("k", "v")); + execute("INSERT INTO " + targetKs + "." + targetTb + " (a) VALUES (?)", map("nk", "nv")); + assertRows(execute("SELECT * FROM " + sourceKs + "." + sourceTb), + row(map("k", "v"))); + assertRows(execute("SELECT * FROM " + targetKs + "." + targetTb), + row(map("nk", "nv"))); + + sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b set<frozen<list<text>>>, c map<text, int>, d smallint, e duration, f tinyint);"); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", + 1, set(list("1", "2"), list("3", "4")), map("k", 1), (short)2, duration1, (byte)4); + execute("INSERT INTO " + targetKs + "." + targetTb + " (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", + 2, set(list("5", "6"), list("7", "8")), map("nk", 2), (short)3, duration2, (byte)5); + assertRows(execute("SELECT * FROM " + sourceKs + "." + sourceTb), + row(1, set(list("1", "2"), list("3", "4")), map("k", 1), (short)2, duration1, (byte)4)); + assertRows(execute("SELECT * FROM " + targetKs + "." + targetTb), + row(2, set(list("5", "6"), list("7", "8")), map("nk", 2), (short)3, duration2, (byte)5)); + + sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int , b double, c tinyint, d float, e list<text>, f map<text, int>, g duration, PRIMARY KEY((a, b, c), d));"); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a, b, c, d, e, f, g) VALUES (?, ?, ?, ?, ?, ?, ?) ", + 1, d1, (byte)4, f1, list("a", "b"), map("k", 1), duration1); + execute("INSERT INTO " + targetKs + "." + targetTb + " (a, b, c, d, e, f, g) VALUES (?, ?, ?, ?, ?, ?, ?) ", + 2, d2, (byte)5, f2, list("c", "d"), map("nk", 2), duration2); + assertRows(execute("SELECT * FROM " + sourceKs + "." + sourceTb), + row(1, d1, (byte)4, f1, list("a", "b"), map("k", 1), duration1)); + assertRows(execute("SELECT * FROM " + targetKs + "." + targetTb), + row(2, d2, (byte)5, f2, list("c", "d"), map("nk", 2), duration2)); + + sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int , " + + "b text, " + + "c bigint, " + + "d decimal, " + + "e set<text>, " + + "f uuid, " + + "g vector<int, 2>, " + + "h list<float>, " + + "i timeuuid, " + + "j map<text, frozen<set<int>>>, " + + "PRIMARY KEY((a, b), c, d)) " + + "WITH CLUSTERING ORDER BY (c DESC, d ASC);"); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a, b, c, d, e, f, g, h, i, j) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?)", + 1, "b", 100L, decimal1, set("1", "2"), uuid1, vector1, list(1.1F, 2.2F), timeUuid1, map("k", set(1, 2))); + execute("INSERT INTO " + targetKs + "." + targetTb + " (a, b, c, d, e, f, g, h, i, j) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?)", + 2, "nb", 200L, decimal2, set("3", "4"), uuid2, vector2, list(3.3F, 4.4F), timeUuid2, map("nk", set(3, 4))); + assertRows(execute("SELECT * FROM " + sourceKs + "." + sourceTb), + row(1, "b", 100L, decimal1, set("1", "2"), uuid1, vector1, list(1.1F, 2.2F), timeUuid1, map("k", set(1, 2)))); + assertRows(execute("SELECT * FROM " + targetKs + "." + targetTb), + row(2, "nb", 200L, decimal2, set("3", "4"), uuid2, vector2, list(3.3F, 4.4F), timeUuid2, map("nk", set(3, 4)))); + + // test that can create a copy of a copied table + sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b duration, c text);"); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + createTableLike("CREATE TABLE %s LIKE %s", targetTb, targetKs, "newtargettb", sourceKs); + } + + @Test + public void testIfNotExists() throws Throwable + { + String sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int, b text, c duration, d float, PRIMARY KEY(a, b));"); + String targetTb = createTableLike("CREATE TABLE IF NOT EXISTS %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + + createTableLike("CREATE TABLE IF NOT EXISTS %s LIKE %s", sourceTb, sourceKs, targetTb, targetKs); + assertInvalidThrowMessage("Cannot add already existing table \"" + targetTb + "\" to keyspace \"" + targetKs + "\"", AlreadyExistsException.class, + "CREATE TABLE " + targetKs + "." + targetTb + " LIKE " + sourceKs + "." + sourceTb); + } + + @Test + public void testCopyAfterAlterTable() + { + String sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int, b text, c duration, d float, PRIMARY KEY(a, b));"); + String targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + + alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " DROP d"); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + + alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " ADD e uuid"); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + + alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " ADD f float"); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + + execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a, b, c, e, f) VALUES (?, ?, ?, ?, ?)", 1, "1", duration1, uuid1, f1); + execute("INSERT INTO " + targetKs + "." + targetTb + " (a, b, c, e, f) VALUES (?, ?, ?, ?, ?)", 2, "2", duration2, uuid2, f2); + assertRows(execute("SELECT * FROM " + sourceKs + "." + sourceTb), + row(1, "1", duration1, uuid1, f1)); + assertRows(execute("SELECT * FROM " + targetKs + "." + targetTb), + row(2, "2", duration2, uuid2, f2)); + + alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " DROP f USING TIMESTAMP 20000"); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + + alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " RENAME b TO bb "); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + + alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " WITH compaction = {'class':'LeveledCompactionStrategy', 'sstable_size_in_mb':10, 'fanout_size':16} "); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + + execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a, bb, c, e) VALUES (?, ?, ?, ?)", 1, "1", duration1, uuid1); + execute("INSERT INTO " + targetKs + "." + targetTb + " (a, bb, c, e) VALUES (?, ?, ?, ?)", 2, "2", duration2, uuid2); + assertRows(execute("SELECT * FROM " + sourceKs + "." + sourceTb), + row(1, "1", duration1, uuid1)); + assertRows(execute("SELECT * FROM " + targetKs + "." + targetTb), + row(2, "2", duration2, uuid2)); + + } + + @Test + public void testTableOptionsCopy() throws Throwable + { + // compression + String tbCompressionDefault1 = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b))"); + String tbCompressionDefault2 =createTable(sourceKs,"CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + + " WITH compression = { 'enabled' : 'false'};"); + String tbCompressionSnappy1 = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + + " WITH compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 32 };"); + String tbCompressionSnappy2 =createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + + " WITH compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 32, 'enabled' : true };"); + String tbCompressionSnappy3 = createTable(sourceKs,"CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + + " WITH compression = { 'class' : 'SnappyCompressor', 'min_compress_ratio' : 2 };"); + String tbCompressionSnappy4 = createTable(sourceKs,"CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + + " WITH compression = { 'class' : 'SnappyCompressor', 'min_compress_ratio' : 1 };"); + String tbCompressionSnappy5 = createTable(sourceKs,"CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + + " WITH compression = { 'class' : 'SnappyCompressor', 'min_compress_ratio' : 0 };"); + + // memtable + String tableMemtableSkipList = createTable(sourceKs,"CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + + " WITH memtable = 'skiplist';"); + String tableMemtableTrie = createTable(sourceKs,"CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + + " WITH memtable = 'trie';"); + String tableMemtableDefault = createTable(sourceKs,"CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + + " WITH memtable = 'default';"); + + // compaction + String tableCompactionStcs = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b)) WITH compaction = {'class':'SizeTieredCompactionStrategy', 'min_threshold':2, 'enabled':false};"); + String tableCompactionLcs =createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b)) WITH compaction = {'class':'LeveledCompactionStrategy', 'sstable_size_in_mb':1, 'fanout_size':5};"); + String tableCompactionTwcs = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b)) WITH compaction = {'class':'TimeWindowCompactionStrategy', 'min_threshold':2};"); + String tableCompactionUcs =createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b)) WITH compaction = {'class':'UnifiedCompactionStrategy'};"); + + // other options are all different from default + String tableOtherOptions = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b)) WITH" + + " additional_write_policy = '95p' " + + " AND bloom_filter_fp_chance = 0.1 " + + " AND caching = {'keys': 'ALL', 'rows_per_partition': '100'}" + + " AND cdc = true " + + " AND comment = 'test for create like'" + + " AND crc_check_chance = 0.1" + + " AND default_time_to_live = 10" + + " AND compaction = {'class':'UnifiedCompactionStrategy'} " + + " AND compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 32 }" + + " AND gc_grace_seconds = 100" + + " AND incremental_backups = false" + + " AND max_index_interval = 1024" + + " AND min_index_interval = 64" + + " AND speculative_retry = '95p'" + + " AND read_repair = 'NONE'" + + " AND memtable_flush_period_in_ms = 360000" + + " AND memtable = 'default';" ); + + String tbLikeCompressionDefault1 = createTableLike("CREATE TABLE %s LIKE %s", tbCompressionDefault1, sourceKs, targetKs); + String tbLikeCompressionDefault2 = createTableLike("CREATE TABLE %s LIKE %s", tbCompressionDefault2, sourceKs, targetKs); + String tbLikeCompressionSp1 = createTableLike("CREATE TABLE %s LIKE %s", tbCompressionSnappy1, sourceKs, targetKs); + String tbLikeCompressionSp2 = createTableLike("CREATE TABLE %s LIKE %s", tbCompressionSnappy2, sourceKs, targetKs); + String tbLikeCompressionSp3 = createTableLike("CREATE TABLE %s LIKE %s", tbCompressionSnappy3, sourceKs, targetKs); + String tbLikeCompressionSp4 = createTableLike("CREATE TABLE %s LIKE %s", tbCompressionSnappy4, sourceKs, targetKs); + String tbLikeCompressionSp5 = createTableLike("CREATE TABLE %s LIKE %s", tbCompressionSnappy5, sourceKs, targetKs); + String tbLikeMemtableSkipList = createTableLike("CREATE TABLE %s LIKE %s", tableMemtableSkipList, sourceKs, targetKs); + String tbLikeMemtableTrie = createTableLike("CREATE TABLE %s LIKE %s", tableMemtableTrie, sourceKs, targetKs); + String tbLikeMemtableDefault = createTableLike("CREATE TABLE %s LIKE %s", tableMemtableDefault, sourceKs, targetKs); + String tbLikeCompactionStcs = createTableLike("CREATE TABLE %s LIKE %s", tableCompactionStcs, sourceKs, targetKs); + String tbLikeCompactionLcs = createTableLike("CREATE TABLE %s LIKE %s", tableCompactionLcs, sourceKs, targetKs); + String tbLikeCompactionTwcs = createTableLike("CREATE TABLE %s LIKE %s", tableCompactionTwcs, sourceKs, targetKs); + String tbLikeCompactionUcs = createTableLike("CREATE TABLE %s LIKE %s", tableCompactionUcs, sourceKs, targetKs); + String tbLikeCompactionOthers= createTableLike("CREATE TABLE %s LIKE %s", tableOtherOptions, sourceKs, targetKs); + + assertTableMetaEquals(sourceKs, targetKs, tbCompressionDefault1, tbLikeCompressionDefault1); + assertTableMetaEquals(sourceKs, targetKs, tbCompressionDefault2, tbLikeCompressionDefault2); + assertTableMetaEquals(sourceKs, targetKs, tbCompressionSnappy1, tbLikeCompressionSp1); + assertTableMetaEquals(sourceKs, targetKs, tbCompressionSnappy2, tbLikeCompressionSp2); + assertTableMetaEquals(sourceKs, targetKs, tbCompressionSnappy3, tbLikeCompressionSp3); + assertTableMetaEquals(sourceKs, targetKs, tbCompressionSnappy4, tbLikeCompressionSp4); + assertTableMetaEquals(sourceKs, targetKs, tbCompressionSnappy5, tbLikeCompressionSp5); + assertTableMetaEquals(sourceKs, targetKs, tableMemtableSkipList, tbLikeMemtableSkipList); + assertTableMetaEquals(sourceKs, targetKs, tableMemtableTrie, tbLikeMemtableTrie); + assertTableMetaEquals(sourceKs, targetKs, tableMemtableDefault, tbLikeMemtableDefault); + assertTableMetaEquals(sourceKs, targetKs, tableCompactionStcs, tbLikeCompactionStcs); + assertTableMetaEquals(sourceKs, targetKs, tableCompactionLcs, tbLikeCompactionLcs); + assertTableMetaEquals(sourceKs, targetKs, tableCompactionTwcs, tbLikeCompactionTwcs); + assertTableMetaEquals(sourceKs, targetKs, tableCompactionUcs, tbLikeCompactionUcs); + assertTableMetaEquals(sourceKs, targetKs, tableOtherOptions, tbLikeCompactionOthers); + + // a copy of the table with the table parameters set + String tableCopyAndSetCompression = createTableLike("CREATE TABLE %s LIKE %s WITH compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 64 };", + tbCompressionSnappy1, sourceKs, targetKs); + String tableCopyAndSetLCSCompaction = createTableLike("CREATE TABLE %s LIKE %s WITH compaction = {'class':'LeveledCompactionStrategy', 'sstable_size_in_mb':10, 'fanout_size':16};", + tableCompactionLcs, sourceKs, targetKs); + String tableCopyAndSetAllParams = createTableLike("CREATE TABLE %s (a text, b int, c int, primary key (a, b)) WITH" + + " bloom_filter_fp_chance = 0.75 " + + " AND caching = {'keys': 'NONE', 'rows_per_partition': '10'}" + + " AND cdc = true " + + " AND comment = 'test for create like and set params'" + + " AND crc_check_chance = 0.8" + + " AND default_time_to_live = 100" + + " AND compaction = {'class':'SizeTieredCompactionStrategy'} " + + " AND compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 64 }" + + " AND gc_grace_seconds = 1000" + + " AND incremental_backups = true" + + " AND max_index_interval = 128" + + " AND min_index_interval = 16" + + " AND speculative_retry = '96p'" + + " AND read_repair = 'NONE'" + + " AND memtable_flush_period_in_ms = 3600;", + tableOtherOptions, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, tbCompressionDefault1, tableCopyAndSetCompression, false, false, false); + assertTableMetaEquals(sourceKs, targetKs, tableCompactionLcs, tableCopyAndSetLCSCompaction, false, false, false); + assertTableMetaEquals(sourceKs, targetKs, tableOtherOptions, tableCopyAndSetAllParams, false, false, false); + TableParams paramsSetCompression = getTableMetadata(targetKs, tableCopyAndSetCompression).params; + TableParams paramsSetLCSCompaction = getTableMetadata(targetKs, tableCopyAndSetLCSCompaction).params; + TableParams paramsSetAllParams = getTableMetadata(targetKs, tableCopyAndSetAllParams).params; + + assertEquals(paramsSetCompression, TableParams.builder().compression(CompressionParams.snappy(64 * 1024, 0.0)).build()); + assertEquals(paramsSetLCSCompaction, TableParams.builder().compaction(CompactionParams.create(LeveledCompactionStrategy.class, + Map.of("sstable_size_in_mb", "10", + "fanout_size", "16"))) + .build()); + assertEquals(paramsSetAllParams, TableParams.builder().bloomFilterFpChance(0.75) + .caching(new CachingParams(false, 10)) + .cdc(true) + .comment("test for create like and set params") + .crcCheckChance(0.8) + .defaultTimeToLive(100) + .compaction(CompactionParams.stcs(Collections.emptyMap())) + .compression(CompressionParams.snappy(64 * 1024, 0.0)) + .gcGraceSeconds(1000) + .incrementalBackups(true) + .maxIndexInterval(128) + .minIndexInterval(16) + .speculativeRetry(SpeculativeRetryPolicy.fromString("96PERCENTILE")) + .readRepair(ReadRepairStrategy.NONE) + .memtableFlushPeriodInMs(3600) + .build()); + + // table id + TableId id = TableId.generate(); + String tbNormal = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b))"); + assertInvalidThrowMessage("Cannot alter table id.", ConfigurationException.class, + "CREATE TABLE " + targetKs + ".targetnormal LIKE " + sourceKs + "." + tbNormal + " WITH ID = " + id); + } + + @Test + public void testStaticColumnCopy() + { + // create with static column + String sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int , b int , c int static, d int, e list<text>, PRIMARY KEY(a, b));", "tb1"); + String targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + execute("INSERT INTO " + targetKs + "." + targetTb + " (a, b, c, d, e) VALUES (0, 1, 2, 3, ?)", list("1", "2", "3", "4")); + assertRows(execute("SELECT * FROM " + targetKs + "." + targetTb), row(0, 1, 2, 3, list("1", "2", "3", "4"))); + + // add static column + sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b))"); + alterTable("ALTER TABLE " + sourceKs + "." + sourceTb + " ADD d int static"); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + } + + @Test + public void testColumnMaskTableCopy() + { + DatabaseDescriptor.setDynamicDataMaskingEnabled(true); + // masked partition key + String sourceTb = createTable(sourceKs, "CREATE TABLE %s (k int MASKED WITH mask_default() PRIMARY KEY, r int)"); + String targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + + // masked partition key component + sourceTb = createTable(sourceKs, "CREATE TABLE %s (k1 int, k2 text MASKED WITH DEFAULT, r int, PRIMARY KEY(k1, k2))"); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + + // masked clustering key + sourceTb = createTable(sourceKs, "CREATE TABLE %s (k int, c int MASKED WITH mask_default(), r int, PRIMARY KEY (k, c))"); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + + // masked clustering key with reverse order + sourceTb = createTable(sourceKs, "CREATE TABLE %s (k int, c text MASKED WITH mask_default(), r int, PRIMARY KEY (k, c)) " + + "WITH CLUSTERING ORDER BY (c DESC)"); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + + // masked clustering key component + sourceTb = createTable(sourceKs, "CREATE TABLE %s (k int, c1 int, c2 text MASKED WITH DEFAULT, r int, PRIMARY KEY (k, c1, c2))"); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + + // masked regular column + sourceTb = createTable(sourceKs, "CREATE TABLE %s (k int PRIMARY KEY, r1 text MASKED WITH DEFAULT, r2 int)"); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + + // masked static column + sourceTb = createTable(sourceKs, "CREATE TABLE %s (k int, c int, r int, s int STATIC MASKED WITH DEFAULT, PRIMARY KEY (k, c))"); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + + // multiple masked columns + sourceTb = createTable(sourceKs, "CREATE TABLE %s (" + + "k1 int, k2 int MASKED WITH DEFAULT, " + + "c1 int, c2 text MASKED WITH DEFAULT, " + + "r1 int, r2 int MASKED WITH DEFAULT, " + + "s1 int static, s2 int static MASKED WITH DEFAULT, " + + "PRIMARY KEY((k1, k2), c1, c2))"); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + + sourceTb = createTable(sourceKs, "CREATE TABLE %s (k int PRIMARY KEY, " + + "s set<int> MASKED WITH DEFAULT, " + + "l list<int> MASKED WITH DEFAULT, " + + "m map<int, int> MASKED WITH DEFAULT, " + + "fs frozen<set<int>> MASKED WITH DEFAULT, " + + "fl frozen<list<int>> MASKED WITH DEFAULT, " + + "fm frozen<map<int, int>> MASKED WITH DEFAULT)"); + targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + } + + @Test + public void testUDTTableCopy() throws Throwable + { + // normal udt + String udt = createType(sourceKs, "CREATE TYPE %s (a int, b uuid, c text)"); + // collection udt + String udtSet = createType(sourceKs, "CREATE TYPE %s (a int, c frozen <set<text>>)"); + // frozen udt + String udtFrozen = createType(sourceKs, "CREATE TYPE %s (a int, c frozen<" + udt + ">)"); + + String sourceTbUdt = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b duration, c " + udt + ");"); + String sourceTbUdtSet = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b duration, c " + udtSet + ");"); + String sourceTbUdtFrozen = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b duration, c " + udtFrozen + ");"); + + if (differentKs) + { + assertInvalidThrowMessage("Cannot use CREATE TABLE LIKE across different keyspace when source table have UDT", + InvalidRequestException.class, "CREATE TABLE " + targetKs + ".tbudt LIKE " + sourceKs + "." + sourceTbUdt); + assertInvalidThrowMessage("Cannot use CREATE TABLE LIKE across different keyspace when source table have UDT", + InvalidRequestException.class, "CREATE TABLE " + targetKs + ".tbdtset LIKE " + sourceKs + "." + sourceTbUdt); + assertInvalidThrowMessage("Cannot use CREATE TABLE LIKE across different keyspace when source table have UDT", InvalidRequestException.class, + "CREATE TABLE " + targetKs + ".tbudtfrozen LIKE " + sourceKs + "." + sourceTbUdt); + } + else + { + String targetTbUdt = createTableLike("CREATE TABLE %s LIKE %s", sourceTbUdt, sourceKs, "tbudt", targetKs); + String targetTbUdtSet = createTableLike("CREATE TABLE %s LIKE %s", sourceTbUdtSet, sourceKs, "tbdtset", targetKs); + String targetTbUdtFrozen = createTableLike("CREATE TABLE %s LIKE %s", sourceTbUdtFrozen, sourceKs, "tbudtfrozen", targetKs); + assertTableMetaEquals(sourceKs, targetKs, sourceTbUdt, targetTbUdt); + assertTableMetaEquals(sourceKs, targetKs, sourceTbUdtSet, targetTbUdtSet); + assertTableMetaEquals(sourceKs, targetKs, sourceTbUdtFrozen, targetTbUdtFrozen); + } + } + + @Test + public void testIndexOperationOnCopiedTable() + { + // copied table can do index creation + String sourceTb = createTable(sourceKs, "CREATE TABLE %s (id text PRIMARY KEY, val text, num int);"); + String targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + String saiIndex = createIndex(targetKs, "CREATE INDEX ON %s(val) USING 'sai'"); + execute("INSERT INTO " + targetKs + "." + targetTb + " (id, val, num) VALUES ('1', 'value', 1)"); + assertEquals(1, execute("SELECT id FROM " + targetKs + "." + targetTb + " WHERE val = 'value'").size()); + String normalIndex = createIndex(targetKs, "CREATE INDEX ON %s(num)"); + Indexes targetIndexes = getTableMetadata(targetKs, targetTb).indexes; + assertEquals(targetIndexes.size(), 2); + assertNotNull(targetIndexes.get(saiIndex)); + assertNotNull(targetIndexes.get(normalIndex)); + } + + @Test + public void testTriggerOperationOnCopiedTable() + { + String triggerName = "trigger_1"; + String sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a))"); + String targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); + execute("CREATE TRIGGER " + triggerName + " ON " + targetKs + "." + targetTb + " USING '" + CreateTest.TestTrigger.class.getName() + "'"); + assertNotNull(getTableMetadata(targetKs, targetTb).triggers.get(triggerName)); + } + + @Test + public void testUnSupportedSchema() throws Throwable + { + createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b int, c text)", "tb"); + String index = createIndex( "CREATE INDEX ON " + sourceKs + ".tb (c)"); + assertInvalidThrowMessage("Souce Table '" + targetKs + "." + index + "' doesn't exist", InvalidRequestException.class, + "CREATE TABLE " + sourceKs + ".newtb LIKE " + targetKs + "." + index + ";"); + + assertInvalidThrowMessage("System keyspace 'system' is not user-modifiable", InvalidRequestException.class, + "CREATE TABLE system.local_clone LIKE system.local ;"); + assertInvalidThrowMessage("System keyspace 'system_views' is not user-modifiable", InvalidRequestException.class, + "CREATE TABLE system_views.newtb LIKE system_views.snapshots ;"); + } + + private void assertTableMetaEquals(String sourceKs, String targetKs, String sourceTb, String targetTb) + { + assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb, true, true, true); + } + + private void assertTableMetaEquals(String sourceKs, String targetKs, String sourceTb, String targetTb, boolean compareParams, boolean compareIndexes, boolean compareTrigger) + { + TableMetadata left = getTableMetadata(sourceKs, sourceTb); + TableMetadata right = getTableMetadata(targetKs, targetTb); + assertNotNull(left); + assertNotNull(right); + assertTrue(equalsWithoutTableNameAndDropCns(left, right, compareParams, compareIndexes, compareTrigger)); + assertNotEquals(left.id, right.id); + assertNotEquals(left.name, right.name); + } +} diff --git a/test/unit/org/apache/cassandra/schema/createlike/CreateLikeWithSessionTest.java b/test/unit/org/apache/cassandra/schema/createlike/CreateLikeWithSessionTest.java new file mode 100644 index 000000000000..e676961d1855 --- /dev/null +++ b/test/unit/org/apache/cassandra/schema/createlike/CreateLikeWithSessionTest.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.schema.createlike; + +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.cql3.CQLTester; + +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +public class CreateLikeWithSessionTest extends CQLTester +{ + private static String ks1 = "ks1"; + private static String ks2 = "ks2"; + private static String tb1 = "tb1"; + private static String tb2 = "tb2"; + + @BeforeClass + public static void beforeClass() + { + requireNetwork(); + } + + @Test + public void testCreateLikeWithSession() + { + // create two keyspaces and tables + createKeyspace("CREATE KEYSPACE " + ks1 + " WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }"); + createKeyspace("CREATE KEYSPACE " + ks2 + " WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }"); + createTable("CREATE TABLE " + ks1 + "." + tb1 + " (id int PRIMARY KEY, age int);"); + createTable("CREATE TABLE " + ks2 + "." + tb2 + " (name text PRIMARY KEY, address text);"); + + // use ks1 + executeNet("use " + ks1); + executeNet("CREATE TABLE tb3 LIKE " + tb1); + executeNet("CREATE TABLE " + ks1 + ".tb4 LIKE " + tb1); + executeNet("CREATE TABLE tb5 like " + ks1 + "." + tb1); + + executeNet("CREATE TABLE " + ks2 + ".tb6 LIKE " + tb1); + + + assertThatExceptionOfType(com.datastax.driver.core.exceptions.InvalidQueryException.class).isThrownBy(() -> executeNet("CREATE TABLE tb7 LIKE " + ks2 + "." + tb1)) + .withMessage("Souce Table 'ks2.tb1' doesn't exist"); + + assertNotNull(getTableMetadata(ks1, tb1)); + assertNotNull(getTableMetadata(ks1, "tb3")); + assertNotNull(getTableMetadata(ks1, "tb4")); + assertNotNull(getTableMetadata(ks1, "tb5")); + assertNotNull(getTableMetadata(ks2, "tb6")); + assertNull(getTableMetadata(ks2, "tb7")); + } +} From a0d58a9ce8814d096c1bd8a0440e8e28d8ea15a9 Mon Sep 17 00:00:00 2001 From: tengu-alt <olexandr.luzhniy@gmail.com> Date: Tue, 14 Jan 2025 12:46:18 +0200 Subject: [PATCH 080/225] Fix the reference list in the docs' nav bar patch by Oleksandr Luzhniy; reviewed by Mick Semb Wever, Bernardo Botella for CASSANDRA-20201 --- doc/modules/cassandra/nav.adoc | 10 ++++------ doc/modules/cassandra/pages/reference/index.adoc | 1 - 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/doc/modules/cassandra/nav.adoc b/doc/modules/cassandra/nav.adoc index bf2f3c696e19..6a4c7a7a2fe5 100644 --- a/doc/modules/cassandra/nav.adoc +++ b/doc/modules/cassandra/nav.adoc @@ -117,11 +117,9 @@ *** xref:cassandra:troubleshooting/use_tools.adoc[Using external tools to deep-dive] ** xref:reference/index.adoc[] -*** xref:reference/cql-commands/alter-table.adoc[] -*** xref:reference/cql-commands/create-index.adoc[] -*** xref:reference/cql-commands/create-custom-index.adoc[] -*** xref:reference/cql-commands/create-table.adoc[] -*** xref:reference/cql-commands/drop-index.adoc[] -*** xref:reference/cql-commands/drop-table.adoc[] +*** xref:reference/cql-commands/commands-toc.adoc[CQL commands] +*** xref:reference/java17.adoc[Java 17] +*** xref:reference/static.adoc[Static columns] +*** xref:reference/sai-virtual-table-indexes.adoc[SAI virtual table] ** xref:integrating/plugins/index.adoc[] \ No newline at end of file diff --git a/doc/modules/cassandra/pages/reference/index.adoc b/doc/modules/cassandra/pages/reference/index.adoc index 8be11464522a..6c61a37925f5 100644 --- a/doc/modules/cassandra/pages/reference/index.adoc +++ b/doc/modules/cassandra/pages/reference/index.adoc @@ -2,6 +2,5 @@ * xref:reference/cql-commands/commands-toc.adoc[CQL commands] * xref:reference/java17.adoc[Java 17] -* xref:reference/counter-type.adoc[Counter type] * xref:reference/static.adoc[Static columns] * xref:reference/sai-virtual-table-indexes.adoc[SAI virtual table] \ No newline at end of file From b15a1f04b61e11bc3b9719c06bffb9fa672a25cd Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Wed, 18 Dec 2024 11:58:30 +0100 Subject: [PATCH 081/225] Propagate true size of snapshot in SnapshotDetailsTabularData to not call JMX twice in nodetool listsnapshots This patch also opportunistically adds a new MBean method for SnapshotManager to return true snapshot size for a particular snapshot. patch by Stefan Miklosovic; reviewed by Maxim Muzafarov, Bernardo Botella for CASSANDRA-20149 --- CHANGES.txt | 1 + .../snapshot/SnapshotDetailsTabularData.java | 25 +++++--- .../service/snapshot/SnapshotManager.java | 9 +++ .../snapshot/SnapshotManagerMBean.java | 12 +++- .../tools/nodetool/ListSnapshots.java | 21 ++++--- .../cassandra/db/ColumnFamilyStoreTest.java | 62 +++++++++++-------- 6 files changed, 84 insertions(+), 46 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 02a0be0571cb..b20e0f8b1fc8 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Propagate true size of snapshot in SnapshotDetailsTabularData to not call JMX twice in nodetool listsnapshots (CASSANDRA-20149) * Implementation of CEP-43 (CASSANDRA-19964) * Periodically disconnect roles that are revoked or have LOGIN=FALSE set (CASSANDRA-19385) * AST library for CQL-based fuzz tests (CASSANDRA-20198) diff --git a/src/java/org/apache/cassandra/service/snapshot/SnapshotDetailsTabularData.java b/src/java/org/apache/cassandra/service/snapshot/SnapshotDetailsTabularData.java index 23c84e505678..3d9609f2e740 100644 --- a/src/java/org/apache/cassandra/service/snapshot/SnapshotDetailsTabularData.java +++ b/src/java/org/apache/cassandra/service/snapshot/SnapshotDetailsTabularData.java @@ -18,13 +18,18 @@ package org.apache.cassandra.service.snapshot; import java.util.Set; -import javax.management.openmbean.*; +import javax.management.openmbean.CompositeDataSupport; +import javax.management.openmbean.CompositeType; +import javax.management.openmbean.OpenDataException; +import javax.management.openmbean.OpenType; +import javax.management.openmbean.SimpleType; +import javax.management.openmbean.TabularDataSupport; +import javax.management.openmbean.TabularType; import org.apache.cassandra.io.util.FileUtils; public class SnapshotDetailsTabularData { - private static final String[] ITEM_NAMES = new String[]{"Snapshot name", "Keyspace name", "Column family name", @@ -32,7 +37,8 @@ public class SnapshotDetailsTabularData "Size on disk", "Creation time", "Expiration time", - "Ephemeral"}; + "Ephemeral", + "Raw true size"}; private static final String[] ITEM_DESCS = new String[]{"snapshot_name", "keyspace_name", @@ -41,7 +47,8 @@ public class SnapshotDetailsTabularData "TotalDiskSpaceUsed", "created_at", "expires_at", - "ephemeral"}; + "ephemeral", + "raw_true_disk_space_used"}; private static final String TYPE_NAME = "SnapshotDetails"; @@ -57,7 +64,7 @@ public class SnapshotDetailsTabularData { try { - ITEM_TYPES = new OpenType[]{ SimpleType.STRING, SimpleType.STRING, SimpleType.STRING, SimpleType.STRING, SimpleType.STRING, SimpleType.STRING, SimpleType.STRING, SimpleType.STRING }; + ITEM_TYPES = new OpenType[]{ SimpleType.STRING, SimpleType.STRING, SimpleType.STRING, SimpleType.STRING, SimpleType.STRING, SimpleType.STRING, SimpleType.STRING, SimpleType.STRING, SimpleType.LONG }; COMPOSITE_TYPE = new CompositeType(TYPE_NAME, ROW_DESC, ITEM_NAMES, ITEM_DESCS, ITEM_TYPES); @@ -74,14 +81,14 @@ public static void from(TableSnapshot details, TabularDataSupport result, Set<St { try { - final String totalSize = FileUtils.stringifyFileSize(details.computeSizeOnDiskBytes()); + String totalSize = FileUtils.stringifyFileSize(details.computeSizeOnDiskBytes()); long trueSizeBytes = details.computeTrueSizeBytes(files); - final String liveSize = FileUtils.stringifyFileSize(details.computeTrueSizeBytes()); + String liveSize = FileUtils.stringifyFileSize(trueSizeBytes); String createdAt = safeToString(details.getCreatedAt()); String expiresAt = safeToString(details.getExpiresAt()); String ephemeral = Boolean.toString(details.isEphemeral()); result.put(new CompositeDataSupport(COMPOSITE_TYPE, ITEM_NAMES, - new Object[]{ details.getTag(), details.getKeyspaceName(), details.getTableName(), liveSize, totalSize, createdAt, expiresAt, ephemeral })); + new Object[]{ details.getTag(), details.getKeyspaceName(), details.getTableName(), liveSize, totalSize, createdAt, expiresAt, ephemeral, trueSizeBytes })); } catch (OpenDataException e) { @@ -93,4 +100,4 @@ private static String safeToString(Object object) { return object == null ? null : object.toString(); } -} \ No newline at end of file +} diff --git a/src/java/org/apache/cassandra/service/snapshot/SnapshotManager.java b/src/java/org/apache/cassandra/service/snapshot/SnapshotManager.java index bea0c04669e6..f7fa1e4ececa 100644 --- a/src/java/org/apache/cassandra/service/snapshot/SnapshotManager.java +++ b/src/java/org/apache/cassandra/service/snapshot/SnapshotManager.java @@ -471,6 +471,15 @@ public long getTrueSnapshotsSize(String keyspace, String table) return new TrueSnapshotSizeTask(this, s -> s.getKeyspaceName().equals(keyspace) && s.getTableName().equals(table)).call(); } + @Override + public long getTrueSnapshotsSize(String keyspace, String table, String snapshotName) + { + return new TrueSnapshotSizeTask(this, + s -> s.getKeyspaceName().equals(keyspace) + && s.getTableName().equals(table) + && s.getTag().equals(snapshotName)).call(); + } + @Override public void setSnapshotLinksPerSecond(long throttle) { diff --git a/src/java/org/apache/cassandra/service/snapshot/SnapshotManagerMBean.java b/src/java/org/apache/cassandra/service/snapshot/SnapshotManagerMBean.java index 2d69a7ca97aa..214adcc4454f 100644 --- a/src/java/org/apache/cassandra/service/snapshot/SnapshotManagerMBean.java +++ b/src/java/org/apache/cassandra/service/snapshot/SnapshotManagerMBean.java @@ -78,7 +78,7 @@ public interface SnapshotManagerMBean long getTrueSnapshotsSize(String keyspace); /** - * Get the true size take by all snapshots in given keyspace and table. + * Get the true size taken by all snapshots in given keyspace and table. * * @param keyspace keyspace to get true size of all snapshots of * @param table table in a keyspace to get true size of all snapshots of @@ -86,6 +86,16 @@ public interface SnapshotManagerMBean */ long getTrueSnapshotsSize(String keyspace, String table); + /** + * Get the true size of a snapshot in given keyspace and table. + * + * @param keyspace keyspace to get true size of all snapshots of + * @param table table in a keyspace to get true size of all snapshots of + * @param snapshotName name of snapshot in given keyspace and table to get true size of + * @return true size of all snapshots in given keyspace and table + */ + long getTrueSnapshotsSize(String keyspace, String table, String snapshotName); + /** * Set the current hardlink-per-second throttle for snapshots * A setting of zero indicates no throttling diff --git a/src/java/org/apache/cassandra/tools/nodetool/ListSnapshots.java b/src/java/org/apache/cassandra/tools/nodetool/ListSnapshots.java index 6a1c71d38605..e1b61d4b518b 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/ListSnapshots.java +++ b/src/java/org/apache/cassandra/tools/nodetool/ListSnapshots.java @@ -25,7 +25,6 @@ import javax.management.openmbean.TabularData; import io.airlift.airline.Command; - import io.airlift.airline.Option; import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.tools.NodeProbe; @@ -64,15 +63,17 @@ public void execute(NodeProbe probe) return; } - final long trueSnapshotsSize = probe.trueSnapshotsSize(); TableBuilder table = new TableBuilder(); // display column names only once - final List<String> indexNames = snapshotDetails.entrySet().iterator().next().getValue().getTabularType().getIndexNames(); + List<String> indexNames = snapshotDetails.entrySet().iterator().next().getValue().getTabularType().getIndexNames(); + indexNames.subList(0, indexNames.size() - 1); if (includeEphemeral) - table.add(indexNames.toArray(new String[indexNames.size()])); - else table.add(indexNames.subList(0, indexNames.size() - 1).toArray(new String[indexNames.size() - 1])); + else + table.add(indexNames.subList(0, indexNames.size() - 2).toArray(new String[indexNames.size() - 2])); + + long totalTrueDiskSize = 0; for (final Map.Entry<String, TabularData> snapshotDetail : snapshotDetails.entrySet()) { @@ -81,18 +82,20 @@ public void execute(NodeProbe probe) { final List<?> value = (List<?>) eachValue; if (includeEphemeral) - table.add(value.toArray(new String[value.size()])); - else table.add(value.subList(0, value.size() - 1).toArray(new String[value.size() - 1])); + else + table.add(value.subList(0, value.size() - 2).toArray(new String[value.size() - 2])); + + totalTrueDiskSize += (Long) value.get(value.size() - 1); } } table.printTo(out); - out.println("\nTotal TrueDiskSpaceUsed: " + FileUtils.stringifyFileSize(trueSnapshotsSize) + '\n'); + out.println("\nTotal TrueDiskSpaceUsed: " + FileUtils.stringifyFileSize(totalTrueDiskSize) + '\n'); } catch (Exception e) { throw new RuntimeException("Error during list snapshot", e); } } -} \ No newline at end of file +} diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java index ba08437cd658..7f5b28c7596d 100644 --- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java +++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java @@ -382,18 +382,21 @@ private void testTrueSnapshotSizeInternal(String keyspace, String table, boolean assertThat(cfs.trueSnapshotsSize()).isZero(); - SnapshotManager.instance.takeSnapshot("snapshot_without_index", cfs.getKeyspaceTableName()); + SnapshotManager.instance.takeSnapshot("snapshot0_without_index", cfs.getKeyspaceTableName()); - long firstSnapshotsSize = cfs.trueSnapshotsSize(); + long sizeOfOneSnapshot = cfs.trueSnapshotsSize(); + // test that true snapshot size computed from manager is same as true snapshot size from cfs with one snapshot + long trueSnapshotSizeViaManagerWithoutIndex = SnapshotManager.instance.getTrueSnapshotsSize(cfs.getKeyspaceName(), cfs.getTableName(), "snapshot0_without_index"); + Assert.assertEquals(sizeOfOneSnapshot, trueSnapshotSizeViaManagerWithoutIndex); Map<String, TableSnapshot> listedSnapshots = Util.listSnapshots(cfs); - assertThat(firstSnapshotsSize).isPositive(); + assertThat(sizeOfOneSnapshot).isPositive(); assertThat(listedSnapshots.size()).isEqualTo(1); - assertThat(listedSnapshots.get("snapshot_without_index")).isNotNull(); - long withoutIndexSize = listedSnapshots.get("snapshot_without_index").computeSizeOnDiskBytes(); - long withoutIndexTrueSize = listedSnapshots.get("snapshot_without_index").computeTrueSizeBytes(); + assertThat(listedSnapshots.get("snapshot0_without_index")).isNotNull(); + long withoutIndexSize = listedSnapshots.get("snapshot0_without_index").computeSizeOnDiskBytes(); + long withoutIndexTrueSize = listedSnapshots.get("snapshot0_without_index").computeTrueSizeBytes(); assertThat(withoutIndexSize).isGreaterThan(withoutIndexTrueSize); - assertEquals(firstSnapshotsSize, withoutIndexTrueSize); + assertEquals(sizeOfOneSnapshot, withoutIndexTrueSize); // add index, trueSnapshotSize should reflect that ColumnIdentifier col = ColumnIdentifier.getInterned("val", true); @@ -414,36 +417,41 @@ private void testTrueSnapshotSizeInternal(String keyspace, String table, boolean rebuildIndices(cfs); - SnapshotManager.instance.takeSnapshot("snapshot_with_index", new HashMap<>(), cfs.getKeyspaceTableName()); + SnapshotManager.instance.takeSnapshot("snapshot1_with_index", new HashMap<>(), cfs.getKeyspaceTableName()); - long secondSnapshotSize = cfs.trueSnapshotsSize(); - Map<String, TableSnapshot> secondListedSnapshots = Util.listSnapshots(cfs); - assertThat(secondSnapshotSize).isPositive(); - assertThat(secondSnapshotSize).isGreaterThan(firstSnapshotsSize); + long sizeOfTwoSnapshots = cfs.trueSnapshotsSize(); + long trueSnapshotSizeViaManagerWithIndex = SnapshotManager.instance.getTrueSnapshotsSize(cfs.getKeyspaceName(), cfs.getTableName(), "snapshot1_with_index"); - assertThat(secondListedSnapshots.size()).isEqualTo(2); - assertThat(secondListedSnapshots.get("snapshot_with_index")).isNotNull(); - assertThat(secondListedSnapshots.get("snapshot_without_index")).isNotNull(); + assertThat(trueSnapshotSizeViaManagerWithIndex).isGreaterThan(trueSnapshotSizeViaManagerWithoutIndex); - long withIndexSize = secondListedSnapshots.get("snapshot_with_index").computeSizeOnDiskBytes(); - long withIndexTrueSize = secondListedSnapshots.get("snapshot_with_index").computeTrueSizeBytes(); + assertThat(sizeOfTwoSnapshots).isPositive(); + assertThat(sizeOfTwoSnapshots).isGreaterThan(sizeOfOneSnapshot); + + listedSnapshots = Util.listSnapshots(cfs); + assertThat(listedSnapshots.size()).isEqualTo(2); + assertThat(listedSnapshots.get("snapshot1_with_index")).isNotNull(); + assertThat(listedSnapshots.get("snapshot0_without_index")).isNotNull(); + + long withIndexSize = listedSnapshots.get("snapshot1_with_index").computeSizeOnDiskBytes(); + long withIndexTrueSize = listedSnapshots.get("snapshot1_with_index").computeTrueSizeBytes(); assertThat(withIndexSize).isGreaterThan(withIndexTrueSize); - assertEquals(secondSnapshotSize, withIndexTrueSize + withoutIndexTrueSize); + assertEquals(sizeOfTwoSnapshots, withIndexTrueSize + withoutIndexTrueSize); // taking another one is basically a copy of the previous - SnapshotManager.instance.takeSnapshot("another_snapshot_with_index", new HashMap<>(), cfs.getKeyspaceTableName()); + SnapshotManager.instance.takeSnapshot("snapshot2_with_index", new HashMap<>(), cfs.getKeyspaceTableName()); - long thirdSnapshotSize = cfs.trueSnapshotsSize(); - Map<String, TableSnapshot> thirdListedSnapshots = Util.listSnapshots(cfs); - assertThat(thirdSnapshotSize).isPositive(); - assertThat(thirdSnapshotSize).isGreaterThan(secondSnapshotSize); + long sizeOfThreeSnapshots = cfs.trueSnapshotsSize(); + assertThat(sizeOfThreeSnapshots).isPositive(); + assertThat(sizeOfThreeSnapshots).isGreaterThan(sizeOfTwoSnapshots); - long anotherWithIndexSize = thirdListedSnapshots.get("another_snapshot_with_index").computeSizeOnDiskBytes(); - long anotherWithIndexTrueSize = thirdListedSnapshots.get("another_snapshot_with_index").computeTrueSizeBytes(); + listedSnapshots = Util.listSnapshots(cfs); + long anotherWithIndexSize = listedSnapshots.get("snapshot2_with_index").computeSizeOnDiskBytes(); + long anotherWithIndexTrueSize = listedSnapshots.get("snapshot2_with_index").computeTrueSizeBytes(); - assertEquals(withIndexSize, anotherWithIndexSize); - assertEquals(withIndexTrueSize, anotherWithIndexTrueSize); + // TODO CASSANDRA-20209 + assertTrue(withIndexSize == anotherWithIndexSize || (withIndexSize + 4 == anotherWithIndexSize) || (withIndexSize - 4 == anotherWithIndexSize)); + assertTrue(withIndexTrueSize == anotherWithIndexTrueSize || (withIndexTrueSize + 4 == anotherWithIndexTrueSize) || (withIndexTrueSize - 4 == anotherWithIndexTrueSize)); } private void rebuildIndices(ColumnFamilyStore cfs) From b4e9399865eb51077f51d34c4c383b8418a9e6ed Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Wed, 18 Dec 2024 14:25:29 +0100 Subject: [PATCH 082/225] Create manifest upon loading where it does not exist or enrich it When snapshot manifest does not exist, no snapshots would be found, because even they would be loaded, since CASSANDRA-18111, snapshot's presence is logically determined by the existence of at least one manifest.json file in any data dir. That would result in snapshots not being shown e.g. in the output of nodetool listsnapshots. The fix consists of creating a manifest in each snapshot directory when not present. When a manifest.json exists, its older format has not contained created_at field (pre-16789) When this is detected, we will proceed to enrich such manifest by overwriting it with a manifest of new format. patch by Stefan Miklosovic; reviewed by Jordan West, Cheng Wang, Bernardo Botella for CASSANDRA-20150 --- CHANGES.txt | 1 + .../config/CassandraRelevantProperties.java | 6 + .../service/snapshot/TableSnapshot.java | 146 ++++++++++++++- .../distributed/test/SnapshotsTest.java | 167 +++++++++++++++++- .../apache/cassandra/db/DirectoriesTest.java | 10 +- .../service/snapshot/SnapshotLoaderTest.java | 57 ++++-- 6 files changed, 364 insertions(+), 23 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index b20e0f8b1fc8..ee4e28789ec0 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Create manifest upon loading where it does not exist or enrich it (CASSANDRA-20150) * Propagate true size of snapshot in SnapshotDetailsTabularData to not call JMX twice in nodetool listsnapshots (CASSANDRA-20149) * Implementation of CEP-43 (CASSANDRA-19964) * Periodically disconnect roles that are revoked or have LOGIN=FALSE set (CASSANDRA-19385) diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java index 8e5b0c6dd9dc..70a6ea702e08 100644 --- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java +++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java @@ -502,6 +502,12 @@ public enum CassandraRelevantProperties SNAPSHOT_CLEANUP_INITIAL_DELAY_SECONDS("cassandra.snapshot.ttl_cleanup_initial_delay_seconds", "5"), /** snapshots ttl cleanup period in seconds */ SNAPSHOT_CLEANUP_PERIOD_SECONDS("cassandra.snapshot.ttl_cleanup_period_seconds", "60"), + /** + * When there is a snapshot with old / basic format (basically pre-CASSANDRA-16789), + * it will enrich it with more metadata upon snapshot's loading at startup. + * Defaults to true, when set to false, no enriching will be done. + * */ + SNAPSHOT_MANIFEST_ENRICH_OR_CREATE_ENABLED("cassandra.snapshot.enrich.or.create.enabled", "true"), /** minimum allowed TTL for snapshots */ SNAPSHOT_MIN_ALLOWED_TTL_SECONDS("cassandra.snapshot.min_allowed_ttl_seconds", "60"), SSL_ENABLE("ssl.enable"), diff --git a/src/java/org/apache/cassandra/service/snapshot/TableSnapshot.java b/src/java/org/apache/cassandra/service/snapshot/TableSnapshot.java index 549dbb6db62c..7698ea1e3da2 100644 --- a/src/java/org/apache/cassandra/service/snapshot/TableSnapshot.java +++ b/src/java/org/apache/cassandra/service/snapshot/TableSnapshot.java @@ -19,10 +19,13 @@ import java.io.IOException; import java.nio.file.Files; +import java.nio.file.NoSuchFileException; import java.nio.file.Path; import java.time.Instant; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -35,6 +38,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Directories; import org.apache.cassandra.db.Keyspace; @@ -42,10 +46,13 @@ import org.apache.cassandra.io.FSReadError; import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.SSTable; import org.apache.cassandra.io.sstable.SSTableId; +import org.apache.cassandra.io.sstable.format.SSTableFormat; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.utils.Clock; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.LocalizeString; import org.apache.cassandra.utils.Throwables; @@ -262,7 +269,8 @@ private List<Path> listDir(Path dir) } catch (Throwable t) { - logger.error("Could not list directory content {}", dir, t); + if (!(t instanceof NoSuchFileException)) + logger.error("Could not list directory content {}", dir, t); } return paths; @@ -490,8 +498,144 @@ private void loadMetadataFromManifest(File manifestFile) TableSnapshot build() { + maybeCreateOrEnrichManifest(); return new TableSnapshot(keyspaceName, tableName, tableId, tag, createdAt, expiresAt, snapshotDirs, ephemeral); } + + private List<File> getSnapshotDirsWithoutManifest() + { + List<File> snapshotDirNotHavingManifests = new ArrayList<>(); + for (File snapshotDir : snapshotDirs) + { + if (!new File(snapshotDir.toPath().resolve("manifest.json")).exists()) + snapshotDirNotHavingManifests.add(snapshotDir); + } + + return snapshotDirNotHavingManifests; + } + + private void maybeCreateOrEnrichManifest() + { + if (!CassandraRelevantProperties.SNAPSHOT_MANIFEST_ENRICH_OR_CREATE_ENABLED.getBoolean()) + return; + + // this is caused by not reading any manifest or that snapshot had a basic manifest just with sstables + // enumerated (pre CASSANDRA-16789), so we just go ahead and enrich it in each snapshot dir + + List<File> snapshotDirsWithoutManifests = getSnapshotDirsWithoutManifest(); + if (createdAt != null && snapshotDirsWithoutManifests.isEmpty()) + return; + + if (createdAt == null && snapshotDirsWithoutManifests.isEmpty()) + logger.info("Manifest in the old format for snapshot {} for {}.{} was detected, going to enrich it.", + tag, keyspaceName, tableName); + + if (!snapshotDirsWithoutManifests.isEmpty()) + logger.info("There is no manifest for snapshot {} for {}.{} at paths {}, going to create it.", + tag, keyspaceName, tableName, snapshotDirsWithoutManifests); + + long lastModified = createdAt == null ? -1 : createdAt.toEpochMilli(); + + if (lastModified == -1) + { + for (File snapshotDir : snapshotDirs) + { + // we will consider time of the creation the oldest last modified + // timestamp on any snapshot directory + long currentLastModified = snapshotDir.lastModified(); + if ((currentLastModified < lastModified || lastModified == -1) && currentLastModified > 0) + lastModified = currentLastModified; + } + } + + List<String> allDataFiles = new ArrayList<>(); + for (File snapshotDir : snapshotDirs) + { + List<File> dataFiles = new ArrayList<>(); + try + { + List<File> indicesDirs = new ArrayList<>(); + File[] snapshotFiles = snapshotDir.list(file -> { + if (file.isDirectory() && file.name().startsWith(".")) + { + indicesDirs.add(file); + return false; + } + else + { + return file.name().endsWith('-' + SSTableFormat.Components.DATA.type.repr); + } + }); + + Collections.addAll(dataFiles, snapshotFiles); + + for (File indexDir : indicesDirs) + dataFiles.addAll(Arrays.asList(indexDir.list(file -> file.name().endsWith('-' + SSTableFormat.Components.DATA.type.repr)))); + } + catch (IOException ex) + { + logger.error("Unable to list a directory for data components: {}", snapshotDir); + } + + for (File dataFile : dataFiles) + { + Descriptor descriptor = SSTable.tryDescriptorFromFile(dataFile); + if (descriptor != null) + { + String relativeDataFileName = descriptor.relativeFilenameFor(SSTableFormat.Components.DATA); + allDataFiles.add(relativeDataFileName); + } + } + } + + // in any case of not being able to resolve it, set it to current time + if (lastModified < 0) + createdAt = Instant.ofEpochMilli(Clock.Global.currentTimeMillis()); + else + createdAt = Instant.ofEpochMilli(lastModified); + + for (File snapshotDir : snapshotDirs) + { + SnapshotManifest snapshotManifest = new SnapshotManifest(allDataFiles, null, createdAt, ephemeral); + File manifestFile = new File(snapshotDir, "manifest.json"); + if (snapshotDirsWithoutManifests.contains(snapshotDir)) + { + writeManifest(snapshotManifest, manifestFile); + } + else + { + SnapshotManifest existingManifest = readManifest(manifestFile); + if (existingManifest != null && existingManifest.createdAt == null) + writeManifest(snapshotManifest, manifestFile); + } + } + } + + private SnapshotManifest readManifest(File manifestFile) + { + try + { + return SnapshotManifest.deserializeFromJsonFile(manifestFile); + } + catch (IOException ex) + { + logger.error("Unable to read a manifest.json file in {}", manifestFile.absolutePath()); + } + + return null; + } + + private void writeManifest(SnapshotManifest snapshotManifest, File manifestFile) + { + try + { + snapshotManifest.serializeToJsonFile(manifestFile); + } + catch (IOException ex) + { + logger.error("Unable to create a manifest.json file in {}", manifestFile.absolutePath()); + } + } } protected static String buildSnapshotId(String keyspaceName, String tableName, UUID tableId, String tag) diff --git a/test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java b/test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java index 2c5765469580..f04a3403cad1 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java @@ -33,32 +33,40 @@ import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.api.ConsistencyLevel; import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.api.IIsolatedExecutor; import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableCallable; import org.apache.cassandra.distributed.api.NodeToolResult; import org.apache.cassandra.distributed.shared.WithProperties; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.utils.Clock; import org.apache.cassandra.utils.FBUtilities; import static java.lang.String.format; import static java.util.concurrent.TimeUnit.SECONDS; import static java.util.stream.Collectors.toList; +import static org.apache.cassandra.config.CassandraRelevantProperties.SNAPSHOT_MANIFEST_ENRICH_OR_CREATE_ENABLED; import static org.apache.cassandra.distributed.shared.ClusterUtils.stopUnchecked; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.apache.cassandra.schema.SchemaConstants.LOCAL_SYSTEM_KEYSPACE_NAMES; +import static org.apache.cassandra.schema.SchemaConstants.REPLICATED_SYSTEM_KEYSPACE_NAMES; import static org.awaitility.Awaitility.await; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.not; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assume.assumeThat; import static oshi.PlatformEnum.MACOS; public class SnapshotsTest extends TestBaseImpl { - public static final Integer SNAPSHOT_CLEANUP_PERIOD_SECONDS = 1; + public static final Integer SNAPSHOT_CLEANUP_PERIOD_SECONDS = 2; public static final Integer FIVE_SECONDS = 5; public static final Integer TEN_SECONDS = 10; private static final WithProperties properties = new WithProperties(); @@ -130,10 +138,117 @@ public void testEverySnapshotDirHasManifestAndSchema() } } + @Test + public void testLocalOrReplicatedSystemTablesSnapshotsDoNotHaveSchema() + { + cluster.get(1) + .nodetoolResult("snapshot", "-t", "snapshot_with_local_or_replicated") + .asserts() + .success(); + + String[] dataDirs = (String[]) cluster.get(1).config().get("data_file_directories"); + String[] paths = getSnapshotPaths(true); + + for (String dataDir : dataDirs) + { + for (String path : paths) + { + Path snapshotDir = Paths.get(dataDir) + .resolve(path) + .resolve("snapshots") + .resolve("snapshot_with_local_or_replicated"); + + if (snapshotDir.toFile().exists()) + assertFalse(new File(snapshotDir, "schema.cql").exists()); + } + } + } + + @Test + public void testMissingManifestIsCreatedOnStartupWithEnrichmentEnabled() + { + cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (key int, value text, PRIMARY KEY (key))")); + populate(cluster); + + cluster.get(1) + .nodetoolResult("snapshot", "-t", "snapshot_with_local_or_replicated") + .asserts() + .success(); + + String[] dataDirs = (String[]) cluster.get(1).config().get("data_file_directories"); + String[] paths = getSnapshotPaths(false); + + assertManifestsPresence(dataDirs, paths, true); + + // remove all manifest files + removeAllManifests(dataDirs, paths); + assertManifestsPresence(dataDirs, paths, false); + + // restart which should add them back + cluster.get(1).shutdown(true); + cluster.get(1).startup(); + + assertManifestsPresence(dataDirs, paths, true); + + // remove them again and reload by mbean, that should create them too + removeAllManifests(dataDirs, paths); + assertManifestsPresence(dataDirs, paths, false); + cluster.get(1).runOnInstance((IIsolatedExecutor.SerializableRunnable) () -> SnapshotManager.instance.restart(true)); + assertManifestsPresence(dataDirs, paths, true); + + cluster.get(1).shutdown(true); + + // remove manifest only in the first data dir + removeAllManifests(new String[] {dataDirs[0]}, paths); + + // they will be still created for that first dir + cluster.get(1).startup(); + assertManifestsPresence(dataDirs, paths, true); + } + + @Test + public void testMissingManifestIsNotCreatedOnStartupWithEnrichmentDisabled() + { + cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (key int, value text, PRIMARY KEY (key))")); + populate(cluster); + + cluster.get(1) + .nodetoolResult("snapshot", "-t", "snapshot_with_local_or_replicated") + .asserts() + .success(); + + String[] dataDirs = (String[]) cluster.get(1).config().get("data_file_directories"); + String[] paths = getSnapshotPaths(false); + + assertManifestsPresence(dataDirs, paths, true); + + cluster.get(1).shutdown(true); + + // remove all manifest files + removeAllManifests(dataDirs, paths); + assertManifestsPresence(dataDirs, paths, false); + + try (WithProperties ignored = new WithProperties().set(SNAPSHOT_MANIFEST_ENRICH_OR_CREATE_ENABLED, false)) + { + // restart which should NOT add them back because we disabled it by system property + cluster.get(1).startup(); + + // no manifests created + assertManifestsPresence(dataDirs, paths, false); + + cluster.get(1).runOnInstance((IIsolatedExecutor.SerializableRunnable) () -> SnapshotManager.instance.restart(true)); + + // no manifests created, even after restart of SnapshotManager + assertManifestsPresence(dataDirs, paths, false); + } + } + @Test public void testSnapshotsCleanupByTTL() { - cluster.get(1).nodetoolResult("snapshot", "--ttl", format("%ds", FIVE_SECONDS), + cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (key int, value text, PRIMARY KEY (key))")); + populate(cluster, withKeyspace("%s"), "tbl"); + cluster.get(1).nodetoolResult("snapshot", "-kt", withKeyspace("%s.tbl"), "--ttl", format("%ds", FIVE_SECONDS), "-t", "basic").asserts().success(); waitForSnapshotPresent("basic"); waitForSnapshotCleared("basic"); @@ -469,4 +584,52 @@ private void exoticSnapshotNamesInternal(String[] exoticSnapshotNames) waitForSnapshot(tag, true, true); } } + + private void assertManifestsPresence(String[] dataDirs, String[] paths, boolean shouldExist) + { + for (String dataDir : dataDirs) + { + for (String path : paths) + { + Path snapshotDir = Paths.get(dataDir).resolve(path).resolve("snapshots").resolve("snapshot_with_local_or_replicated"); + assertEquals(shouldExist, new File(snapshotDir, "manifest.json").exists()); + } + } + } + + private void removeAllManifests(String[] dataDirs, String[] paths) + { + for (String dataDir : dataDirs) + { + for (String path : paths) + { + Path snapshotDir = Paths.get(dataDir).resolve(path).resolve("snapshots").resolve("snapshot_with_local_or_replicated"); + + File manifest = new File(snapshotDir, "manifest.json"); + assertTrue(manifest.exists()); + manifest.delete(); + assertFalse(manifest.exists()); + } + } + } + + private String[] getSnapshotPaths(boolean forSystemKeyspaces) + { + return cluster.get(1).applyOnInstance((IIsolatedExecutor.SerializableFunction<Boolean, String[]>) forSystem -> { + List<String> result = new ArrayList<>(); + + for (Keyspace keyspace : Keyspace.all()) + { + if (forSystem && !LOCAL_SYSTEM_KEYSPACE_NAMES.contains(keyspace.getName()) && !REPLICATED_SYSTEM_KEYSPACE_NAMES.contains(keyspace.getName())) + continue; + else if (LOCAL_SYSTEM_KEYSPACE_NAMES.contains(keyspace.getName()) || REPLICATED_SYSTEM_KEYSPACE_NAMES.contains(keyspace.getName())) + continue; + + for (ColumnFamilyStore cfs : keyspace.getColumnFamilyStores()) + result.add(format("%s/%s-%s", keyspace.getName(), cfs.name, cfs.metadata().id.toHexString())); + } + + return result.toArray(new String[0]); + }, forSystemKeyspaces); + } } diff --git a/test/unit/org/apache/cassandra/db/DirectoriesTest.java b/test/unit/org/apache/cassandra/db/DirectoriesTest.java index b6d4126fc9b7..06db7e4a02a8 100644 --- a/test/unit/org/apache/cassandra/db/DirectoriesTest.java +++ b/test/unit/org/apache/cassandra/db/DirectoriesTest.java @@ -351,11 +351,11 @@ public void testListSnapshots() throws Exception { // Create snapshot with and without manifest FakeSnapshot snapshot1 = createFakeSnapshot(fakeTable, SNAPSHOT1, true, false); - FakeSnapshot snapshot2 = createFakeSnapshot(fakeTable, SNAPSHOT2, false, false); - // ephemeral without manifst - FakeSnapshot snapshot3 = createFakeSnapshot(fakeTable, SNAPSHOT3, false, true); + FakeSnapshot snapshot2 = createFakeSnapshot(fakeTable, SNAPSHOT2, true, false); + // ephemeral without manifest + FakeSnapshot snapshot3 = createFakeSnapshot(fakeTable, SNAPSHOT3, true, true); - // Both snapshots should be present + // All snapshots should be present Map<String, TableSnapshot> snapshots = listSnapshots(directories); assertThat(snapshots.keySet()).isEqualTo(Sets.newHashSet(SNAPSHOT1, SNAPSHOT2, SNAPSHOT3)); assertThat(snapshots.get(SNAPSHOT1)).isEqualTo(snapshot1.asTableSnapshot()); @@ -459,7 +459,7 @@ public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) } }); // CASSANDRA-17357: include indexes when computing true size of parent table - assertEquals(70L, parentSnapshotDetail.get("test").computeTrueSizeBytes(files)); + assertEquals(70L, parentSnapshotDetail.get("test").computeTrueSizeBytes(files) - parentSnapshotDetail.get("test").getManifestsSize()); // check backup directory File parentBackupDirectory = Directories.getBackupsDirectory(parentDesc); diff --git a/test/unit/org/apache/cassandra/service/snapshot/SnapshotLoaderTest.java b/test/unit/org/apache/cassandra/service/snapshot/SnapshotLoaderTest.java index 54ecf19d98f9..ee76cedd63fa 100644 --- a/test/unit/org/apache/cassandra/service/snapshot/SnapshotLoaderTest.java +++ b/test/unit/org/apache/cassandra/service/snapshot/SnapshotLoaderTest.java @@ -23,6 +23,7 @@ import java.time.Instant; import java.util.Arrays; import java.util.HashSet; +import java.util.List; import java.util.Set; import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; @@ -37,6 +38,7 @@ import org.apache.cassandra.config.DurationSpec; import org.apache.cassandra.db.Directories; import org.apache.cassandra.io.util.File; +import org.apache.cassandra.utils.Clock; import org.assertj.core.util.Lists; import static org.apache.cassandra.service.snapshot.SnapshotLoader.SNAPSHOT_DIR_PATTERN; @@ -110,9 +112,9 @@ public void testNoSnapshots() throws IOException @Test public void testSnapshotsWithoutManifests() throws IOException { - Set<File> tag1Files = new HashSet<>(); - Set<File> tag2Files = new HashSet<>(); - Set<File> tag3Files = new HashSet<>(); + Set<File> firstSnapshotDirs = new HashSet<>(); + Set<File> secondSnapshotDirs = new HashSet<>(); + Set<File> thirdSnapshotDirs = new HashSet<>(); // Create one snapshot per table - without manifests: // - ks1.t1 : tag1 @@ -121,20 +123,26 @@ public void testSnapshotsWithoutManifests() throws IOException File baseDir = new File(tmpDir.newFolder()); for (String dataDir : DATA_DIRS) { - tag1Files.add(createDir(baseDir, dataDir, KEYSPACE_1, tableDirName(TABLE1_NAME, TABLE1_ID), Directories.SNAPSHOT_SUBDIR, TAG1)); - tag2Files.add(createDir(baseDir, dataDir, KEYSPACE_1, tableDirName(TABLE2_NAME, TABLE2_ID), Directories.SNAPSHOT_SUBDIR, TAG2)); - tag3Files.add(createDir(baseDir, dataDir, KEYSPACE_2, tableDirName(TABLE3_NAME, TABLE3_ID), Directories.SNAPSHOT_SUBDIR, TAG3)); + firstSnapshotDirs.add(createDir(baseDir, dataDir, KEYSPACE_1, tableDirName(TABLE1_NAME, TABLE1_ID), Directories.SNAPSHOT_SUBDIR, TAG1)); + secondSnapshotDirs.add(createDir(baseDir, dataDir, KEYSPACE_1, tableDirName(TABLE2_NAME, TABLE2_ID), Directories.SNAPSHOT_SUBDIR, TAG2)); + thirdSnapshotDirs.add(createDir(baseDir, dataDir, KEYSPACE_2, tableDirName(TABLE3_NAME, TABLE3_ID), Directories.SNAPSHOT_SUBDIR, TAG3)); } + Instant createdAt = Instant.ofEpochMilli(Clock.Global.currentTimeMillis()); + + createManifests(firstSnapshotDirs, createdAt); + createManifests(secondSnapshotDirs, createdAt); + createManifests(thirdSnapshotDirs, createdAt); + // Verify all 3 snapshots are found correctly from data directories SnapshotLoader loader = new SnapshotLoader(Arrays.asList(Paths.get(baseDir.toString(), DATA_DIR_1), Paths.get(baseDir.toString(), DATA_DIR_2), Paths.get(baseDir.toString(), DATA_DIR_3))); Set<TableSnapshot> snapshots = loader.loadSnapshots(); assertThat(snapshots).hasSize(3); - assertThat(snapshots).contains(new TableSnapshot(KEYSPACE_1, TABLE1_NAME, TABLE1_ID, TAG1, null, null, tag1Files, false)); - assertThat(snapshots).contains(new TableSnapshot(KEYSPACE_1, TABLE2_NAME, TABLE2_ID, TAG2, null, null, tag2Files, false)); - assertThat(snapshots).contains(new TableSnapshot(KEYSPACE_2, TABLE3_NAME, TABLE3_ID, TAG3, null, null, tag3Files, false)); + assertThat(snapshots).contains(new TableSnapshot(KEYSPACE_1, TABLE1_NAME, TABLE1_ID, TAG1, createdAt, null, firstSnapshotDirs, false)); + assertThat(snapshots).contains(new TableSnapshot(KEYSPACE_1, TABLE2_NAME, TABLE2_ID, TAG2, createdAt, null, secondSnapshotDirs, false)); + assertThat(snapshots).contains(new TableSnapshot(KEYSPACE_2, TABLE3_NAME, TABLE3_ID, TAG3, createdAt, null, thirdSnapshotDirs, false)); // Verify snapshot loading for a specific keyspace loader = new SnapshotLoader(Arrays.asList(Paths.get(baseDir.toString(), DATA_DIR_1), @@ -143,21 +151,37 @@ public void testSnapshotsWithoutManifests() throws IOException snapshots = loader.loadSnapshots(KEYSPACE_1); assertThat(snapshots).hasSize(2); - assertThat(snapshots).contains(new TableSnapshot(KEYSPACE_1, TABLE1_NAME, TABLE1_ID, TAG1, null, null, tag1Files, false)); - assertThat(snapshots).contains(new TableSnapshot(KEYSPACE_1, TABLE2_NAME, TABLE2_ID, TAG2, null, null, tag2Files, false)); + assertThat(snapshots).contains(new TableSnapshot(KEYSPACE_1, TABLE1_NAME, TABLE1_ID, TAG1, createdAt, null, firstSnapshotDirs, false)); + assertThat(snapshots).contains(new TableSnapshot(KEYSPACE_1, TABLE2_NAME, TABLE2_ID, TAG2, createdAt, null, secondSnapshotDirs, false)); loader = new SnapshotLoader(Arrays.asList(Paths.get(baseDir.toString(), DATA_DIR_1), Paths.get(baseDir.toString(), DATA_DIR_2), Paths.get(baseDir.toString(), DATA_DIR_3))); snapshots = loader.loadSnapshots(KEYSPACE_2); assertThat(snapshots).hasSize(1); - assertThat(snapshots).contains(new TableSnapshot(KEYSPACE_2, TABLE3_NAME, TABLE3_ID, TAG3, null, null, tag3Files, false)); + assertThat(snapshots).contains(new TableSnapshot(KEYSPACE_2, TABLE3_NAME, TABLE3_ID, TAG3, createdAt, null, thirdSnapshotDirs, false)); + } + + private void createManifests(Set<File> snapshotDirs, Instant createdAt) + { + SnapshotManifest snapshotManifest = new SnapshotManifest(List.of(), null, createdAt, false); + for (File snapshotDir : snapshotDirs) + { + try + { + snapshotManifest.serializeToJsonFile(new File(snapshotDir, "manifest.json")); + } + catch (Throwable t) + { + throw new RuntimeException("Unable to write manifest file", t); + } + } } @Test public void testEphemeralSnapshotWithoutManifest() throws IOException { - Set<File> tag1Files = new HashSet<>(); + Set<File> snapshotDirs = new HashSet<>(); // Create one snapshot per table - without manifests: // - ks1.t1 : tag1 @@ -166,7 +190,7 @@ public void testEphemeralSnapshotWithoutManifest() throws IOException for (String dataDir : DATA_DIRS) { File dir = createDir(baseDir, dataDir, KEYSPACE_1, tableDirName(TABLE1_NAME, TABLE1_ID), Directories.SNAPSHOT_SUBDIR, TAG1); - tag1Files.add(dir); + snapshotDirs.add(dir); if (!ephemeralFileCreated) { createEphemeralMarkerFile(dir); @@ -179,9 +203,12 @@ public void testEphemeralSnapshotWithoutManifest() throws IOException Paths.get(baseDir.toString(), DATA_DIR_2), Paths.get(baseDir.toString(), DATA_DIR_3))); + Instant createdAt = Instant.ofEpochMilli(Clock.Global.currentTimeMillis()); + createManifests(snapshotDirs, createdAt); + Set<TableSnapshot> snapshots = loader.loadSnapshots(); assertThat(snapshots).hasSize(1); - assertThat(snapshots).contains(new TableSnapshot(KEYSPACE_1, TABLE1_NAME, TABLE1_ID, TAG1, null, null, tag1Files, true)); + assertThat(snapshots).contains(new TableSnapshot(KEYSPACE_1, TABLE1_NAME, TABLE1_ID, TAG1, createdAt, null, snapshotDirs, true)); Assert.assertTrue(snapshots.stream().findFirst().get().isEphemeral()); } From 407dbacb0a0fcd5eca28abe781be635476325a65 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Wed, 18 Dec 2024 21:04:44 +0100 Subject: [PATCH 083/225] Enable filtering of snapshots on keyspace, table and snapshot name in nodetool listsnapshots patch by Stefan Miklosovic; reviewed by Jordan West, Bernardo Botella, Cheng Wang, Maxim Muzafarov for CASSANDRA-20151 --- CHANGES.txt | 3 +- .../snapshot/SnapshotManagerMBean.java | 14 +++- .../tools/nodetool/ListSnapshots.java | 21 +++++ .../distributed/test/SnapshotsTest.java | 78 +++++++++++++++++-- 4 files changed, 109 insertions(+), 7 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index ee4e28789ec0..97c638682119 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,7 +1,8 @@ 5.1 + * Enable filtering of snapshots on keyspace, table and snapshot name in nodetool listsnapshots (CASSANDRA-20151) * Create manifest upon loading where it does not exist or enrich it (CASSANDRA-20150) * Propagate true size of snapshot in SnapshotDetailsTabularData to not call JMX twice in nodetool listsnapshots (CASSANDRA-20149) - * Implementation of CEP-43 (CASSANDRA-19964) + * Implementation of CEP-43 - copying a table via CQL by CREATE TABLE LIKE (CASSANDRA-19964) * Periodically disconnect roles that are revoked or have LOGIN=FALSE set (CASSANDRA-19385) * AST library for CQL-based fuzz tests (CASSANDRA-20198) * Support audit logging for JMX operations (CASSANDRA-20128) diff --git a/src/java/org/apache/cassandra/service/snapshot/SnapshotManagerMBean.java b/src/java/org/apache/cassandra/service/snapshot/SnapshotManagerMBean.java index 214adcc4454f..332ebb4b5480 100644 --- a/src/java/org/apache/cassandra/service/snapshot/SnapshotManagerMBean.java +++ b/src/java/org/apache/cassandra/service/snapshot/SnapshotManagerMBean.java @@ -55,7 +55,19 @@ public interface SnapshotManagerMBean void clearSnapshot(String tag, Map<String, Object> options, String... keyspaceNames) throws IOException; /** - * Get the details of all the snapshots + * Get the details of all the snapshots. Options might be: + * + * <pre> + * no_ttl: "true" or "false" + * include_ephemeral: "true" or "false" + * keyspace: name of keyspace to get snapshots of + * table: name of table to get tables of + * snapshot: name of snapshot to list + * </pre> + * + * There is no requirement as what option has to be specified. + * Values of 'null' for keyspace, table or snapshot do not have any effect / will + * not be part of the filtering. * * @param options map of options used for filtering of snapshots * @return A map of snapshotName to all its details in Tabular form. diff --git a/src/java/org/apache/cassandra/tools/nodetool/ListSnapshots.java b/src/java/org/apache/cassandra/tools/nodetool/ListSnapshots.java index e1b61d4b518b..97ea809fdd9a 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/ListSnapshots.java +++ b/src/java/org/apache/cassandra/tools/nodetool/ListSnapshots.java @@ -44,6 +44,21 @@ public class ListSnapshots extends NodeToolCmd description = "Include ephememeral snapshots") private boolean includeEphemeral = false; + @Option(title = "keyspace", + name = { "-k", "--keyspace" }, + description = "Include snapshots of specified keyspace name") + private String keyspace = null; + + @Option(title = "table", + name = { "-t", "--table" }, + description = "Include snapshots of specified table name") + private String table = null; + + @Option(title = "snapshot", + name = { "-n", "--snapshot"}, + description = "Include snapshots of specified name") + private String snapshotName = null; + @Override public void execute(NodeProbe probe) { @@ -55,6 +70,12 @@ public void execute(NodeProbe probe) Map<String, String> options = new HashMap<>(); options.put("no_ttl", Boolean.toString(noTTL)); options.put("include_ephemeral", Boolean.toString(includeEphemeral)); + if (keyspace != null) + options.put("keyspace", keyspace); + if (table != null) + options.put("table", table); + if (snapshotName != null) + options.put("snapshot", snapshotName); final Map<String, TabularData> snapshotDetails = probe.getSnapshotDetails(options); if (snapshotDetails.isEmpty()) diff --git a/test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java b/test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java index f04a3403cad1..d0b78112a8d5 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java @@ -512,6 +512,49 @@ public void testTakingSnapshoWithSameNameOnDifferentTablesDoesNotFail() cluster.get(1).nodetoolResult("snapshot", "-t", "somename", "-kt", String.format("%s.tbl2", KEYSPACE)).asserts().success(); } + @Test + public void testListingOfSnapshotsByKeyspaceAndTable() + { + IInvokableInstance instance = cluster.get(1); + cluster.schemaChange("CREATE KEYSPACE IF NOT EXISTS ks1 WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1};"); + cluster.schemaChange("CREATE KEYSPACE IF NOT EXISTS ks2 WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1};"); + cluster.schemaChange("CREATE TABLE IF NOT EXISTS ks1.tbl (key int, value text, PRIMARY KEY (key))"); + cluster.schemaChange("CREATE TABLE IF NOT EXISTS ks1.tbl2 (key int, value text, PRIMARY KEY (key))"); + cluster.schemaChange("CREATE TABLE IF NOT EXISTS ks2.tbl (key int, value text, PRIMARY KEY (key))"); + cluster.schemaChange("CREATE TABLE IF NOT EXISTS ks2.tbl2 (key int, value text, PRIMARY KEY (key))"); + + populate(cluster, "ks1", "tbl"); + populate(cluster, "ks1", "tbl2"); + populate(cluster, "ks2", "tbl"); + populate(cluster, "ks2", "tbl2"); + + instance.nodetoolResult("snapshot", "-t", "tagks1tbl", "-kt", "ks1.tbl").asserts().success(); + instance.nodetoolResult("snapshot", "-t", "tagks1tbl2", "-kt", "ks1.tbl2").asserts().success(); + instance.nodetoolResult("snapshot", "-t", "tagks2tbl", "-kt", "ks2.tbl").asserts().success(); + instance.nodetoolResult("snapshot", "-t", "tagks2tbl2", "-kt", "ks2.tbl2").asserts().success(); + + waitForSnapshot("ks1", null, "tagks1tbl", true, false); + waitForSnapshot("ks1", null, "tagks1tbl2", true, false); + waitForSnapshot("ks1", null, "tagks2tbl", false, false); + waitForSnapshot("ks1", null, "tagks2tbl2", false, false); + + waitForSnapshot("ks1", "tbl", "tagks1tbl", true, false); + waitForSnapshot("ks1", "tbl", "tagks1tbl2", false, false); + waitForSnapshot("ks1", "tbl", "tagks2tbl", false, false); + waitForSnapshot("ks1", "tbl", "tagks2tbl2", false, false); + + waitForSnapshot(null, "tbl", "tagks1tbl", true, false); + waitForSnapshot(null, "tbl", "tagks1tbl2", false, false); + waitForSnapshot(null, "tbl", "tagks2tbl", true, false); + waitForSnapshot(null, "tbl", "tagks2tbl2", false, false); + + NodeToolResult nodeToolResult = instance.nodetoolResult("listsnapshots", "-n", "tagks1tbl"); + nodeToolResult.asserts().success(); + List<String> snapshots = extractSnapshots(nodeToolResult.getStdout()); + assertEquals(1, snapshots.size()); + assertTrue(snapshots.get(0).contains("tagks1tbl")); + } + private void populate(Cluster cluster) { for (int i = 0; i < 100; i++) @@ -555,13 +598,27 @@ private boolean waitForSnapshotInternal(String keyspaceName, String tableName, S if (noTTL) args.add("-nt"); + if (keyspaceName != null) + { + args.add("-k"); + args.add(keyspaceName); + } + + if (tableName != null) + { + args.add("-t"); + args.add(tableName); + } + + if (snapshotName != null) + { + args.add("-n"); + args.add(snapshotName); + } + listsnapshots = cluster.get(1).nodetoolResult(args.toArray(new String[0])); - List<String> lines = Arrays.stream(listsnapshots.getStdout().split("\n")) - .filter(line -> !line.isEmpty()) - .filter(line -> !line.startsWith("Snapshot Details:") && !line.startsWith("There are no snapshots")) - .filter(line -> !line.startsWith("Snapshot name") && !line.startsWith("Total TrueDiskSpaceUsed")) - .collect(toList()); + List<String> lines = extractSnapshots(listsnapshots.getStdout()); return expectPresent == lines.stream().anyMatch(line -> line.startsWith(snapshotName)); } @@ -632,4 +689,15 @@ else if (LOCAL_SYSTEM_KEYSPACE_NAMES.contains(keyspace.getName()) || REPLICATED_ return result.toArray(new String[0]); }, forSystemKeyspaces); } + + private List<String> extractSnapshots(String listSnapshotsStdOut) + { + return Arrays.stream(listSnapshotsStdOut.split("\n")) + .filter(line -> !line.isEmpty()) + .filter(line -> !line.startsWith("Snapshot Details:")) + .filter(line -> !line.startsWith("There are no snapshots")) + .filter(line -> !line.startsWith("Snapshot name")) + .filter(line -> !line.startsWith("Total TrueDiskSpaceUsed")) + .collect(toList()); + } } From 7ffbd714b4808f7f05b2cee60d2d88a8b6dee05f Mon Sep 17 00:00:00 2001 From: Kristijonas Zalys <kzalys@uber.com> Date: Sat, 26 Oct 2024 15:14:10 -0700 Subject: [PATCH 084/225] Set setIsShutdownUnsafeForTests(true) in dtests to fix shutdown race condition Patch by Kristijonas Zalys, reviewed by brandonwilliams and mck for CASSANDRA-18440 --- .../org/apache/cassandra/distributed/impl/Instance.java | 1 + 1 file changed, 1 insertion(+) diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java index 39f5e980afab..a8c948c25d73 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java @@ -794,6 +794,7 @@ public Future<Void> shutdown(boolean graceful) error = parallelRun(error, executor, () -> StorageService.instance.setRpcReady(false), + () -> StorageService.instance.setIsShutdownUnsafeForTests(true), CassandraDaemon.getInstanceForTesting()::destroyClientTransports); if (config.has(GOSSIP) || config.has(NETWORK)) From 2a3454a3eae5e5064e63389ffbc268fb06f3f0e9 Mon Sep 17 00:00:00 2001 From: Caleb Rackliffe <calebrackliffe@gmail.com> Date: Wed, 25 Sep 2024 13:58:01 -0400 Subject: [PATCH 085/225] Port Harry v2 to trunk This commit ports to trunk work already done in the cep-15-accord branch, including parts of CASSANDRA-19955, CASSANDRA-20080, and CASSANDRA-20143. patch by Caleb Rackliffe; reviewed by David Capwell and Alex Petrov for CASSANDRA-20200 Co-authored-by: Caleb Rackliffe <calebrackliffe@gmail.com> Co-authored-by: Ariel Weisberg <aweisberg@apple.com> Co-authored-by: Alex Petrov <oleksandr.petrov@gmail.com> Co-authored-by: David Capwell <dcapwell@apache.org> --- CHANGES.txt | 1 + .../statements/ModificationStatement.java | 5 - .../index/sai/plan/QueryController.java | 23 +- .../apache/cassandra/utils/ArrayUtils.java | 68 + .../cassandra/utils/ByteBufferUtil.java | 68 + .../cassandra/distributed/Constants.java | 8 + .../distributed/impl/AbstractCluster.java | 4 + .../distributed/impl/Coordinator.java | 4 +- .../distributed/shared/ClusterUtils.java | 167 ++- .../distributed/test/IntegrationTestBase.java | 85 ++ .../test/log/BounceGossipTest.java | 255 ++-- .../distributed/test/log/CMSTestBase.java | 2 +- .../test/log/CoordinatorPathTest.java | 154 +- .../test/log/CoordinatorPathTestBase.java | 8 +- .../distributed/test/log/FailedLeaveTest.java | 141 +- .../distributed/test/log/FuzzTestBase.java | 26 +- .../log/MetadataChangeSimulationTest.java | 16 +- .../test/log/MetadataKeysTest.java | 2 +- .../distributed/test/log/ModelState.java | 80 +- .../test/log/NTSSimulationTest.java | 2 +- .../test/log/OperationalEquivalenceTest.java | 58 +- .../test/log/PlacementSimulator.java | 10 +- .../test/log/PlacementSimulatorTest.java | 29 +- .../test/log/ResumableStartupTest.java | 221 +-- .../log/SimpleStrategySimulationTest.java | 2 +- .../test/log/SimulatedOperation.java | 2 +- .../test/log/SystemKeyspaceStorageTest.java | 2 +- .../ClusterMetadataUpgradeHarryTest.java | 183 +-- .../examples/RangeTombstoneBurnTest.java | 193 ++- .../fuzz/harry/examples/RepairBurnTest.java | 123 +- .../fuzz/harry/gen/DataGeneratorsTest.java | 504 ------- .../fuzz/harry/gen/EntropySourceTest.java | 158 --- .../fuzz/harry/gen/SurjectionsTest.java | 63 - .../integration/QuickTheoriesAdapter.java | 100 -- .../harry/integration/ddl/SchemaGenTest.java | 215 --- .../dsl/HistoryBuilderIntegrationTest.java | 173 --- ...istoryBuilderOverridesIntegrationTest.java | 359 ----- .../DataGeneratorsIntegrationTest.java | 179 --- ...urrentQuiescentCheckerIntegrationTest.java | 71 - .../model/InJVMTokenAwareExecutorTest.java | 98 -- .../model/IntegrationTestBase.java | 122 -- .../harry/integration/model/MockSchema.java | 117 -- .../integration/model/ModelTestBase.java | 138 -- .../model/QuerySelectorNegativeTest.java | 157 --- .../integration/model/QuerySelectorTest.java | 163 --- .../QuiescentCheckerIntegrationTest.java | 224 --- ...scentLocalStateCheckerIntegrationTest.java | 79 -- .../model/ReconcilerIntegrationTest.java | 93 -- .../reconciler/SimpleReconcilerTest.java | 341 ----- .../harry/integration/op/RowVisitorTest.java | 90 -- .../harry/model/ApproximateClockTest.java | 180 --- .../fuzz/harry/model/OpSelectorsTest.java | 418 ------ .../fuzz/harry/operations/RelationTest.java | 204 --- .../harry/runner/LockingDataTrackerTest.java | 133 -- .../cassandra/fuzz/harry/util/RangesTest.java | 102 -- .../fuzz/ring/ConsistentBootstrapTest.java | 322 +++-- .../fuzz/ring/ConsistentLeaveTest.java | 127 +- .../fuzz/ring/ConsistentMoveTest.java | 141 +- .../cassandra/fuzz/sai/MultiNodeSAITest.java | 84 +- .../fuzz/sai/MultiNodeSAITestBase.java | 85 ++ .../fuzz/sai/PagingSingleNodeSAITest.java | 54 - .../fuzz/sai/PagingStaticsTortureTest.java | 54 - .../cassandra/fuzz/sai/SingleNodeSAITest.java | 288 +--- .../fuzz/sai/SingleNodeSAITestBase.java | 284 ++++ .../fuzz/sai/StaticsTortureTest.java | 239 ++-- .../fuzz/topology/HarryTopologyMixupTest.java | 161 ++- .../fuzz/topology/TopologyMixupTestBase.java | 682 ++++++--- test/harry/main/README.md | 647 --------- .../apache/cassandra/harry/ColumnSpec.java | 610 ++++++++ .../apache/cassandra/harry/HarryHelper.java | 172 --- .../cassandra/harry/MagicConstants.java | 70 + .../org/apache/cassandra/harry/Relations.java | 205 +++ .../apache/cassandra/harry/SchemaSpec.java | 440 ++++++ .../harry/ValueGeneratorHelper.java} | 33 +- .../cassandra/harry/checker/ModelChecker.java | 60 +- .../cassandra/harry/checker/TestHelper.java | 64 + .../harry/clock/ApproximateClock.java | 279 ---- .../cassandra/harry/clock/OffsetClock.java | 94 -- .../cassandra/harry/core/Configuration.java | 1255 ----------------- .../cassandra/harry/core/MetricReporter.java | 51 - .../org/apache/cassandra/harry/core/Run.java | 75 - .../harry/corruptor/AddExtraRowCorruptor.java | 90 -- .../harry/corruptor/ChangeValueCorruptor.java | 85 -- .../harry/corruptor/HideRowCorruptor.java | 49 - .../harry/corruptor/HideValueCorruptor.java | 109 -- .../corruptor/QueryResponseCorruptor.java | 108 -- .../harry/corruptor/RowCorruptor.java | 56 - .../harry/corruptor/ShowValueCorruptor.java | 77 - .../cassandra/harry/cql/DeleteHelper.java | 245 ++++ .../cassandra/harry/cql/SelectHelper.java | 241 ++++ .../cassandra/harry/cql/WriteHelper.java | 208 +++ .../cassandra/harry/data/ResultSetRow.java | 137 -- .../cassandra/harry/ddl/ColumnSpec.java | 460 ------ .../cassandra/harry/ddl/SchemaGenerators.java | 512 ------- .../cassandra/harry/ddl/SchemaSpec.java | 531 ------- .../harry/dsl/BatchOperationBuilder.java | 32 - .../harry/dsl/BatchVisitBuilder.java | 139 -- .../cassandra/harry/dsl/HistoryBuilder.java | 770 ++++------ .../harry/dsl/HistoryBuilderHelper.java | 195 +++ .../harry/dsl/MultiOperationVisitBuilder.java | 170 +++ .../harry/dsl/OverridingBijection.java | 84 -- .../harry/dsl/OverridingCkGenerator.java | 153 -- .../harry/dsl/PartitionVisitState.java | 40 - .../harry/dsl/PartitionVisitStateImpl.java | 115 -- .../harry/dsl/ReplayingHistoryBuilder.java | 143 +- .../harry/dsl/SingleOperationBuilder.java | 109 +- .../dsl/SingleOperationVisitBuilder.java | 666 ++++++--- .../harry/{util => dsl}/TestRunner.java | 34 +- .../dsl/ValueDescriptorIndexGenerator.java | 87 -- .../cassandra/harry/dsl/ValueHelper.java | 74 - .../execution/CQLTesterVisitExecutor.java | 170 +++ .../harry/execution/CQLVisitExecutor.java | 181 +++ .../harry/execution/CompiledStatement.java | 141 ++ .../harry/execution/DataTracker.java | 171 +++ .../execution/InJvmDTestVisitExecutor.java | 424 ++++++ .../execution/QueryBuildingVisitExecutor.java | 198 +++ .../harry/execution/ResultSetRow.java | 128 ++ .../RingAwareInJvmDTestVisitExecutor.java | 214 +++ .../VisitExecutor.java} | 11 +- .../cassandra/harry/gen/Bijections.java | 22 +- .../cassandra/harry/gen/BooleanGenerator.java | 29 - .../cassandra/harry/gen/Collections.java | 249 ---- .../cassandra/harry/gen/DataGenerators.java | 511 ------- .../cassandra/harry/gen/EntropySource.java | 1 + .../cassandra/harry/gen/Generators.java | 347 ++++- .../cassandra/harry/gen/IndexGenerators.java | 130 ++ .../harry/gen/InvertibleGenerator.java | 261 ++++ .../harry/gen/OperationsGenerators.java | 114 ++ .../cassandra/harry/gen/SchemaGenerators.java | 111 ++ .../cassandra/harry/gen/TypeAdapters.java | 124 ++ .../cassandra/harry/gen/ValueGenerators.java | 131 ++ .../harry/gen/distribution/Distribution.java | 136 -- .../harry/gen/rng/JdkRandomEntropySource.java | 24 +- .../cassandra/harry/gen/rng/PcgRSUFast.java | 6 + .../cassandra/harry/gen/rng/PureRng.java | 70 + .../harry/gen/rng/SeedableEntropySource.java | 52 + .../harry/gen/test/EntropySourceTest.java | 136 ++ .../harry/model/AgainstSutChecker.java | 109 -- .../model/AlwaysSamePartitionSelector.java | 92 -- .../apache/cassandra/harry/model/Model.java | 66 +- .../cassandra/harry/model/NoOpChecker.java | 44 - .../cassandra/harry/model/OpSelectors.java | 903 ------------ .../cassandra/harry/model/PartitionState.java | 403 ++++++ .../harry/model/PartitionStateBuilder.java | 153 ++ .../harry/model/QueryingNoOpValidator.java | 65 - .../harry/model/QuiescentChecker.java | 360 +++-- .../model/QuiescentLocalStateCheckerBase.java | 124 -- .../cassandra/harry/model/Reconciler.java | 75 + .../model/RepairingLocalStateValidator.java | 92 -- .../cassandra/harry/model/SelectHelper.java | 348 ----- .../{sut => model}/TokenPlacementModel.java | 116 +- .../model/TokenPlacementModelHelper.java | 66 + .../model/reconciler/PartitionState.java | 310 ---- .../harry/model/reconciler/Reconciler.java | 342 ----- .../harry/model/ring/PlacementSimulator.java | 1052 -------------- .../apache/cassandra/harry/op/Operations.java | 562 ++++++++ .../org/apache/cassandra/harry/op/Visit.java | 72 + .../harry/operations/CompiledStatement.java | 95 -- .../harry/operations/DeleteHelper.java | 190 --- .../harry/operations/FilteringQuery.java | 74 - .../cassandra/harry/operations/Query.java | 557 -------- .../harry/operations/QueryGenerator.java | 176 --- .../cassandra/harry/operations/Relation.java | 299 ---- .../harry/operations/WriteHelper.java | 166 --- .../harry/runner/EarlyExitException.java | 27 - .../cassandra/harry/runner/FlaggedRunner.java | 84 -- .../cassandra/harry/runner/HarryRunner.java | 115 -- .../apache/cassandra/harry/runner/Runner.java | 479 ------- .../cassandra/harry/runner/StagedRunner.java | 92 -- .../harry/runner/TrivialShrinker.java | 189 --- .../cassandra/harry/runner/UpToLtsRunner.java | 111 -- .../cassandra/harry/sut/DoubleWritingSut.java | 60 - .../cassandra/harry/sut/PrintlnSut.java | 67 - .../harry/sut/QueryModifyingSut.java | 80 -- .../cassandra/harry/sut/SystemUnderTest.java | 105 -- .../harry/sut/injvm/ClusterState.java | 24 - .../harry/sut/injvm/ExistingClusterSUT.java | 96 -- .../injvm/InJVMTokenAwareVisitExecutor.java | 165 --- .../cassandra/harry/sut/injvm/InJvmSut.java | 117 -- .../harry/sut/injvm/InJvmSutBase.java | 331 ----- .../sut/injvm/QuiescentLocalStateChecker.java | 90 -- .../harry/test/HistoryBuilderTest.java | 328 +++++ .../harry/test/QueryBuilderTest.java | 48 + .../harry/test/SimpleBijectionTest.java | 94 ++ .../cassandra/harry/test/TestStateTest.java | 50 + .../cassandra/harry/tracker/DataTracker.java | 66 - .../harry/tracker/DefaultDataTracker.java | 193 --- .../harry/tracker/LockingDataTracker.java | 286 ---- .../harry/{dsl => util}/ArrayWrapper.java | 4 +- .../cassandra/harry/util/ByteUtils.java | 40 + .../harry/util/DescriptorRanges.java | 150 -- .../cassandra/harry/util/IteratorsUtil.java | 79 ++ .../cassandra/harry/util/StringUtils.java | 49 +- .../harry/util/ThrowingRunnable.java | 4 +- .../visitors/AllPartitionsValidator.java | 156 -- .../harry/visitors/CorruptingVisitor.java | 89 -- .../visitors/DoubleWriteVisitExecutor.java | 56 - .../harry/visitors/FaultInjectingVisitor.java | 97 -- .../harry/visitors/GeneratingVisitor.java | 232 --- .../harry/visitors/LoggingVisitor.java | 85 -- .../cassandra/harry/visitors/LtsVisitor.java | 83 -- .../harry/visitors/MutatingRowVisitor.java | 115 -- .../harry/visitors/MutatingVisitor.java | 220 --- .../harry/visitors/OperationExecutor.java | 78 - .../visitors/ParallelRecentValidator.java | 149 -- .../harry/visitors/ParallelValidator.java | 103 -- .../cassandra/harry/visitors/QueryLogger.java | 97 -- .../visitors/RandomPartitionValidator.java | 58 - .../harry/visitors/RandomValidator.java | 93 -- .../harry/visitors/RecentValidator.java | 96 -- .../harry/visitors/ReplayingVisitor.java | 73 - .../cassandra/harry/visitors/Sampler.java | 97 -- .../harry/visitors/SingleValidator.java | 66 - .../harry/visitors/SkippingVisitor.java | 56 - .../harry/visitors/VisitExecutor.java | 122 -- .../cassandra/harry/visitors/Visitor.java | 33 - .../simulator/cluster/KeyspaceActions.java | 2 +- .../simulator/harry/HarryValidatingQuery.java | 146 -- .../simulator/test/HarrySimulatorTest.java | 366 +++-- .../simulator/test/HarryValidatingQuery.java | 116 ++ .../test/SimulatedVisitExectuor.java | 175 --- test/unit/accord/utils/Invariants.java | 16 +- test/unit/accord/utils/Property.java | 204 ++- .../cassandra/tcm/ClusterMetadataTest.java | 3 +- .../UniformRangePlacementIntegrationTest.java | 2 +- .../tcm/sequences/ProgressBarrierTest.java | 2 +- 226 files changed, 11913 insertions(+), 24717 deletions(-) create mode 100644 src/java/org/apache/cassandra/utils/ArrayUtils.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/IntegrationTestBase.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/gen/DataGeneratorsTest.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/gen/EntropySourceTest.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/gen/SurjectionsTest.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/integration/QuickTheoriesAdapter.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/integration/ddl/SchemaGenTest.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/integration/dsl/HistoryBuilderIntegrationTest.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/integration/dsl/HistoryBuilderOverridesIntegrationTest.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/integration/generators/DataGeneratorsIntegrationTest.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/integration/model/ConcurrentQuiescentCheckerIntegrationTest.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/integration/model/InJVMTokenAwareExecutorTest.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/integration/model/IntegrationTestBase.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/integration/model/MockSchema.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/integration/model/ModelTestBase.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/integration/model/QuerySelectorNegativeTest.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/integration/model/QuerySelectorTest.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/integration/model/QuiescentCheckerIntegrationTest.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/integration/model/QuiescentLocalStateCheckerIntegrationTest.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/integration/model/ReconcilerIntegrationTest.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/integration/model/reconciler/SimpleReconcilerTest.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/integration/op/RowVisitorTest.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/model/ApproximateClockTest.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/model/OpSelectorsTest.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/operations/RelationTest.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/runner/LockingDataTrackerTest.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/harry/util/RangesTest.java create mode 100644 test/distributed/org/apache/cassandra/fuzz/sai/MultiNodeSAITestBase.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/sai/PagingSingleNodeSAITest.java delete mode 100644 test/distributed/org/apache/cassandra/fuzz/sai/PagingStaticsTortureTest.java create mode 100644 test/distributed/org/apache/cassandra/fuzz/sai/SingleNodeSAITestBase.java delete mode 100644 test/harry/main/README.md create mode 100644 test/harry/main/org/apache/cassandra/harry/ColumnSpec.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/HarryHelper.java create mode 100644 test/harry/main/org/apache/cassandra/harry/MagicConstants.java create mode 100644 test/harry/main/org/apache/cassandra/harry/Relations.java create mode 100644 test/harry/main/org/apache/cassandra/harry/SchemaSpec.java rename test/{distributed/org/apache/cassandra/fuzz/harry/gen/ExtensionsTest.java => harry/main/org/apache/cassandra/harry/ValueGeneratorHelper.java} (58%) create mode 100644 test/harry/main/org/apache/cassandra/harry/checker/TestHelper.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/clock/ApproximateClock.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/clock/OffsetClock.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/core/Configuration.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/core/MetricReporter.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/core/Run.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/corruptor/AddExtraRowCorruptor.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/corruptor/ChangeValueCorruptor.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/corruptor/HideRowCorruptor.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/corruptor/HideValueCorruptor.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/corruptor/QueryResponseCorruptor.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/corruptor/RowCorruptor.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/corruptor/ShowValueCorruptor.java create mode 100644 test/harry/main/org/apache/cassandra/harry/cql/DeleteHelper.java create mode 100644 test/harry/main/org/apache/cassandra/harry/cql/SelectHelper.java create mode 100644 test/harry/main/org/apache/cassandra/harry/cql/WriteHelper.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/data/ResultSetRow.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/ddl/ColumnSpec.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/ddl/SchemaGenerators.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/ddl/SchemaSpec.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/dsl/BatchOperationBuilder.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/dsl/BatchVisitBuilder.java create mode 100644 test/harry/main/org/apache/cassandra/harry/dsl/HistoryBuilderHelper.java create mode 100644 test/harry/main/org/apache/cassandra/harry/dsl/MultiOperationVisitBuilder.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/dsl/OverridingBijection.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/dsl/OverridingCkGenerator.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/dsl/PartitionVisitState.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/dsl/PartitionVisitStateImpl.java rename test/harry/main/org/apache/cassandra/harry/{util => dsl}/TestRunner.java (77%) delete mode 100644 test/harry/main/org/apache/cassandra/harry/dsl/ValueDescriptorIndexGenerator.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/dsl/ValueHelper.java create mode 100644 test/harry/main/org/apache/cassandra/harry/execution/CQLTesterVisitExecutor.java create mode 100644 test/harry/main/org/apache/cassandra/harry/execution/CQLVisitExecutor.java create mode 100644 test/harry/main/org/apache/cassandra/harry/execution/CompiledStatement.java create mode 100644 test/harry/main/org/apache/cassandra/harry/execution/DataTracker.java create mode 100644 test/harry/main/org/apache/cassandra/harry/execution/InJvmDTestVisitExecutor.java create mode 100644 test/harry/main/org/apache/cassandra/harry/execution/QueryBuildingVisitExecutor.java create mode 100644 test/harry/main/org/apache/cassandra/harry/execution/ResultSetRow.java create mode 100644 test/harry/main/org/apache/cassandra/harry/execution/RingAwareInJvmDTestVisitExecutor.java rename test/harry/main/org/apache/cassandra/harry/{dsl/ValueOverrides.java => execution/VisitExecutor.java} (69%) delete mode 100644 test/harry/main/org/apache/cassandra/harry/gen/BooleanGenerator.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/gen/Collections.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/gen/DataGenerators.java create mode 100644 test/harry/main/org/apache/cassandra/harry/gen/IndexGenerators.java create mode 100644 test/harry/main/org/apache/cassandra/harry/gen/InvertibleGenerator.java create mode 100644 test/harry/main/org/apache/cassandra/harry/gen/OperationsGenerators.java create mode 100644 test/harry/main/org/apache/cassandra/harry/gen/SchemaGenerators.java create mode 100644 test/harry/main/org/apache/cassandra/harry/gen/TypeAdapters.java create mode 100644 test/harry/main/org/apache/cassandra/harry/gen/ValueGenerators.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/gen/distribution/Distribution.java create mode 100644 test/harry/main/org/apache/cassandra/harry/gen/rng/PureRng.java create mode 100644 test/harry/main/org/apache/cassandra/harry/gen/rng/SeedableEntropySource.java create mode 100644 test/harry/main/org/apache/cassandra/harry/gen/test/EntropySourceTest.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/model/AgainstSutChecker.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/model/AlwaysSamePartitionSelector.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/model/NoOpChecker.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/model/OpSelectors.java create mode 100644 test/harry/main/org/apache/cassandra/harry/model/PartitionState.java create mode 100644 test/harry/main/org/apache/cassandra/harry/model/PartitionStateBuilder.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/model/QueryingNoOpValidator.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/model/QuiescentLocalStateCheckerBase.java create mode 100644 test/harry/main/org/apache/cassandra/harry/model/Reconciler.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/model/RepairingLocalStateValidator.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/model/SelectHelper.java rename test/harry/main/org/apache/cassandra/harry/{sut => model}/TokenPlacementModel.java (90%) create mode 100644 test/harry/main/org/apache/cassandra/harry/model/TokenPlacementModelHelper.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/model/reconciler/PartitionState.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/model/reconciler/Reconciler.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/model/ring/PlacementSimulator.java create mode 100644 test/harry/main/org/apache/cassandra/harry/op/Operations.java create mode 100644 test/harry/main/org/apache/cassandra/harry/op/Visit.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/operations/CompiledStatement.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/operations/DeleteHelper.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/operations/FilteringQuery.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/operations/Query.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/operations/QueryGenerator.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/operations/Relation.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/operations/WriteHelper.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/runner/EarlyExitException.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/runner/FlaggedRunner.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/runner/HarryRunner.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/runner/Runner.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/runner/StagedRunner.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/runner/TrivialShrinker.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/runner/UpToLtsRunner.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/sut/DoubleWritingSut.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/sut/PrintlnSut.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/sut/QueryModifyingSut.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/sut/SystemUnderTest.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/sut/injvm/ClusterState.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/sut/injvm/ExistingClusterSUT.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/sut/injvm/InJVMTokenAwareVisitExecutor.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/sut/injvm/InJvmSut.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/sut/injvm/InJvmSutBase.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/sut/injvm/QuiescentLocalStateChecker.java create mode 100644 test/harry/main/org/apache/cassandra/harry/test/HistoryBuilderTest.java create mode 100644 test/harry/main/org/apache/cassandra/harry/test/QueryBuilderTest.java create mode 100644 test/harry/main/org/apache/cassandra/harry/test/SimpleBijectionTest.java create mode 100644 test/harry/main/org/apache/cassandra/harry/test/TestStateTest.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/tracker/DataTracker.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/tracker/DefaultDataTracker.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/tracker/LockingDataTracker.java rename test/harry/main/org/apache/cassandra/harry/{dsl => util}/ArrayWrapper.java (95%) delete mode 100644 test/harry/main/org/apache/cassandra/harry/util/DescriptorRanges.java create mode 100644 test/harry/main/org/apache/cassandra/harry/util/IteratorsUtil.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/AllPartitionsValidator.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/CorruptingVisitor.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/DoubleWriteVisitExecutor.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/FaultInjectingVisitor.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/GeneratingVisitor.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/LoggingVisitor.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/LtsVisitor.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/MutatingRowVisitor.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/MutatingVisitor.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/OperationExecutor.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/ParallelRecentValidator.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/ParallelValidator.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/QueryLogger.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/RandomPartitionValidator.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/RandomValidator.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/RecentValidator.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/ReplayingVisitor.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/Sampler.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/SingleValidator.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/SkippingVisitor.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/VisitExecutor.java delete mode 100644 test/harry/main/org/apache/cassandra/harry/visitors/Visitor.java delete mode 100644 test/simulator/main/org/apache/cassandra/simulator/harry/HarryValidatingQuery.java create mode 100644 test/simulator/test/org/apache/cassandra/simulator/test/HarryValidatingQuery.java delete mode 100644 test/simulator/test/org/apache/cassandra/simulator/test/SimulatedVisitExectuor.java diff --git a/CHANGES.txt b/CHANGES.txt index 97c638682119..b4a5206aa326 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Port Harry v2 to trunk (CASSANDRA-20200) * Enable filtering of snapshots on keyspace, table and snapshot name in nodetool listsnapshots (CASSANDRA-20151) * Create manifest upon loading where it does not exist or enrich it (CASSANDRA-20150) * Propagate true size of snapshot in SnapshotDetailsTabularData to not call JMX twice in nodetool listsnapshots (CASSANDRA-20149) diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java index e3662a609571..113135016d8a 100644 --- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java @@ -351,11 +351,6 @@ public List<Operation> getStaticOperations() return operations.staticOperations(); } - public Iterable<Operation> allOperations() - { - return operations; - } - public Iterable<ColumnMetadata> getColumnsWithConditions() { return conditions.getColumns(); diff --git a/src/java/org/apache/cassandra/index/sai/plan/QueryController.java b/src/java/org/apache/cassandra/index/sai/plan/QueryController.java index b462fa3ad5f1..b81c6a5c4f52 100644 --- a/src/java/org/apache/cassandra/index/sai/plan/QueryController.java +++ b/src/java/org/apache/cassandra/index/sai/plan/QueryController.java @@ -25,6 +25,7 @@ import java.util.Collections; import java.util.List; import java.util.NavigableSet; +import java.util.Set; import java.util.stream.Collectors; import javax.annotation.Nullable; @@ -172,6 +173,22 @@ public UnfilteredRowIterator queryStorage(List<PrimaryKey> keys, ReadExecutionCo return partition.queryMemtableAndDisk(cfs, executionController); } + private static Runnable getIndexReleaser(Set<SSTableIndex> referencedIndexes) + { + return new Runnable() + { + boolean closed; + @Override + public void run() + { + if (closed) + return; + closed = true; + referencedIndexes.forEach(SSTableIndex::releaseQuietly); + } + }; + } + /** * Build a {@link KeyRangeIterator.Builder} from the given list of {@link Expression}s. * <p> @@ -200,7 +217,7 @@ public KeyRangeIterator.Builder getIndexQueryResults(Collection<Expression> expr expressions = expressions.stream().filter(e -> e.getIndexOperator() != Expression.IndexOperator.ANN).collect(Collectors.toList()); QueryViewBuilder.QueryView queryView = new QueryViewBuilder(expressions, mergeRange).build(); - Runnable onClose = () -> queryView.referencedIndexes.forEach(SSTableIndex::releaseQuietly); + Runnable onClose = getIndexReleaser(queryView.referencedIndexes); KeyRangeIterator.Builder builder = command.rowFilter().isStrict() ? KeyRangeIntersectionIterator.builder(expressions.size(), onClose) : KeyRangeUnionIterator.builder(expressions.size(), onClose); @@ -319,7 +336,7 @@ public KeyRangeIterator getTopKRows(RowFilter.Expression expression) KeyRangeIterator memtableResults = index.memtableIndexManager().searchMemtableIndexes(queryContext, planExpression, mergeRange); QueryViewBuilder.QueryView queryView = new QueryViewBuilder(Collections.singleton(planExpression), mergeRange).build(); - Runnable onClose = () -> queryView.referencedIndexes.forEach(SSTableIndex::releaseQuietly); + Runnable onClose = getIndexReleaser(queryView.referencedIndexes); try { @@ -359,7 +376,7 @@ private KeyRangeIterator getTopKRows(List<PrimaryKey> rawSourceKeys, RowFilter.E // search memtable before referencing sstable indexes; otherwise we may miss newly flushed memtable index KeyRangeIterator memtableResults = index.memtableIndexManager().limitToTopResults(queryContext, sourceKeys, planExpression); QueryViewBuilder.QueryView queryView = new QueryViewBuilder(Collections.singleton(planExpression), mergeRange).build(); - Runnable onClose = () -> queryView.referencedIndexes.forEach(SSTableIndex::releaseQuietly); + Runnable onClose = getIndexReleaser(queryView.referencedIndexes); try { diff --git a/src/java/org/apache/cassandra/utils/ArrayUtils.java b/src/java/org/apache/cassandra/utils/ArrayUtils.java new file mode 100644 index 000000000000..118e230ba1da --- /dev/null +++ b/src/java/org/apache/cassandra/utils/ArrayUtils.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils; + +import java.util.Arrays; + +public class ArrayUtils +{ + public static int hashCode(Object object) + { + if (object.getClass().isArray()) + { + Class<?> klass = object.getClass(); + if (klass == Object[].class) return Arrays.hashCode((Object[]) object); + else if (klass == byte[].class) return Arrays.hashCode((byte[]) object); + else if (klass == char[].class) return Arrays.hashCode((char[]) object); + else if (klass == short[].class) return Arrays.hashCode((short[]) object); + else if (klass == int[].class) return Arrays.hashCode((int[]) object); + else if (klass == long[].class) return Arrays.hashCode((long[]) object); + else if (klass == double[].class) return Arrays.hashCode((double[]) object); + else if (klass == float[].class) return Arrays.hashCode((float[]) object); + else if (klass == boolean[].class) return Arrays.hashCode((boolean[]) object); + else throw new IllegalArgumentException("Unknown type: " + klass); + } + else + { + return object.hashCode(); + } + } + + public static String toString(Object object) + { + if (object.getClass().isArray()) + { + Class<?> klass = object.getClass(); + if (klass == Object[].class) return Arrays.toString((Object[]) object); + else if (klass == byte[].class) return Arrays.toString((byte[]) object); + else if (klass == char[].class) return Arrays.toString((char[]) object); + else if (klass == short[].class) return Arrays.toString((short[]) object); + else if (klass == int[].class) return Arrays.toString((int[]) object); + else if (klass == long[].class) return Arrays.toString((long[]) object); + else if (klass == double[].class) return Arrays.toString((double[]) object); + else if (klass == float[].class) return Arrays.toString((float[]) object); + else if (klass == boolean[].class) return Arrays.toString((boolean[]) object); + else throw new IllegalArgumentException("Unknown type: " + klass); + } + else + { + return object.toString(); + } + } +} diff --git a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java index 156c1c4bc840..73725a6ce3df 100644 --- a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java +++ b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java @@ -27,6 +27,8 @@ import java.io.EOFException; import java.io.IOException; import java.io.InputStream; +import java.math.BigDecimal; +import java.math.BigInteger; import java.net.InetAddress; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; @@ -34,10 +36,23 @@ import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; import java.util.Arrays; +import java.util.Date; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.UUID; +import java.util.stream.Collectors; import net.nicoulaj.compilecommand.annotations.Inline; import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.db.marshal.BooleanType; +import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.marshal.ListType; +import org.apache.cassandra.db.marshal.MapType; +import org.apache.cassandra.db.marshal.SetType; +import org.apache.cassandra.db.marshal.TimestampType; import org.apache.cassandra.io.compress.BufferType; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; @@ -534,6 +549,8 @@ else if (obj instanceof Float) return ByteBufferUtil.bytes((float) obj); else if (obj instanceof Double) return ByteBufferUtil.bytes((double) obj); + else if (obj instanceof Boolean) + return BooleanType.instance.decompose((Boolean) obj); else if (obj instanceof UUID) return ByteBufferUtil.bytes((UUID) obj); else if (obj instanceof InetAddress) @@ -544,6 +561,57 @@ else if (obj instanceof byte[]) return ByteBuffer.wrap((byte[]) obj); else if (obj instanceof ByteBuffer) return (ByteBuffer) obj; + else if (obj instanceof BigInteger) + return ByteBuffer.wrap(((BigInteger)obj).toByteArray()); + else if (obj instanceof BigDecimal) + { + BigDecimal cast = (BigDecimal) obj; + BigInteger bi = cast.unscaledValue(); + int scale = cast.scale(); + byte[] bibytes = bi.toByteArray(); + + ByteBuffer bytes = ByteBuffer.allocate(4 + bibytes.length); + bytes.putInt(scale); + bytes.put(bibytes); + bytes.rewind(); + return bytes; + } + else if (obj instanceof List) + { + List<?> list = (List<?>) obj; + // convert subtypes to BB + List<ByteBuffer> bbs = list.stream().map(ByteBufferUtil::objectToBytes).collect(Collectors.toList()); + // decompose/serializer doesn't use the isMultiCell, so safe to do this + return ListType.getInstance(BytesType.instance, false).decompose(bbs); + } + else if (obj instanceof Map) + { + Map<?, ?> map = (Map<?, ?>) obj; + // convert subtypes to BB + Map<ByteBuffer, ByteBuffer> bbs = new LinkedHashMap<>(); + for (Map.Entry<?, ?> e : map.entrySet()) + { + Object key = e.getKey(); + ByteBuffer previousValue = bbs.put(objectToBytes(key), objectToBytes(e.getValue())); + if (previousValue != null) + throw new IllegalStateException("Key " + key + " already maps to value " + previousValue); + } + // decompose/serializer doesn't use the isMultiCell, so safe to do this + return MapType.getInstance(BytesType.instance, BytesType.instance, false).decompose(bbs); + } + else if (obj instanceof Set) + { + Set<?> set = (Set<?>) obj; + // convert subtypes to BB + Set<ByteBuffer> bbs = new LinkedHashSet<>(); + for (Object o : set) + if (!bbs.add(objectToBytes(o))) + throw new IllegalStateException("Object " + o + " maps to a buffer that already exists in the set"); + // decompose/serializer doesn't use the isMultiCell, so safe to do this + return SetType.getInstance(BytesType.instance, false).decompose(bbs); + } + else if (obj instanceof Date) + return TimestampType.instance.decompose((Date) obj); else throw new IllegalArgumentException(String.format("Cannot convert value %s of type %s", obj, diff --git a/test/distributed/org/apache/cassandra/distributed/Constants.java b/test/distributed/org/apache/cassandra/distributed/Constants.java index cdd8843d8ba1..04955adbeeaf 100644 --- a/test/distributed/org/apache/cassandra/distributed/Constants.java +++ b/test/distributed/org/apache/cassandra/distributed/Constants.java @@ -42,4 +42,12 @@ public final class Constants public static final String KEY_DTEST_FULL_STARTUP = "dtest.api.startup.full_startup"; public static final String KEY_DTEST_JOIN_RING = "dtest.api.startup.join_ring"; + + /** + * Adds a timeout to instance startup so the tests don't block forever when startup bugs can happen. This should + * almost always be paired with {@link #KEY_DTEST_API_STARTUP_FAILURE_AS_SHUTDOWN} as the startup still needs to cleanup. + * + * {@code c.set(Constants.KEY_DTEST_API_STARTUP_FAILURE_AS_SHUTDOWN, false); } + */ + public static final String KEY_DTEST_STARTUP_TIMEOUT = "dtest.api.startup.timeout"; } diff --git a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java index feca65208acb..7f98f9003d13 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java @@ -680,6 +680,10 @@ public Stream<ICoordinator> coordinators() return stream().map(IInstance::coordinator); } + public I firstAlive() + { + return stream().filter(i -> !i.isShutdown()).findFirst().get(); + } public List<I> get(int... nodes) { if (nodes == null || nodes.length == 0) diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Coordinator.java b/test/distributed/org/apache/cassandra/distributed/impl/Coordinator.java index 71cf5aaa5a3e..20293ad11996 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/Coordinator.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/Coordinator.java @@ -28,6 +28,7 @@ import com.google.common.collect.Iterators; +import accord.utils.Invariants; import org.apache.cassandra.cql3.CQLStatement; import org.apache.cassandra.cql3.QueryOptions; import org.apache.cassandra.cql3.QueryProcessor; @@ -126,7 +127,8 @@ public QueryResult executeWithPagingWithResult(String query, ConsistencyLevel co boundBBValues.add(ByteBufferUtil.objectToBytes(boundValue)); prepared.validate(clientState); - assert prepared instanceof SelectStatement : "Only SELECT statements can be executed with paging"; + Invariants.checkState(prepared instanceof SelectStatement, + "Only SELECT statements can be executed with paging %s", prepared); Dispatcher.RequestTime requestTime = Dispatcher.RequestTime.forImmediateExecution(); SelectStatement selectStatement = (SelectStatement) prepared; diff --git a/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java b/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java index 313dde0c5450..cbb6c420463c 100644 --- a/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java +++ b/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java @@ -43,21 +43,17 @@ import java.util.stream.Collectors; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.Futures; +import org.apache.cassandra.distributed.api.*; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.cassandra.dht.Token; -import org.apache.cassandra.distributed.api.ConsistencyLevel; -import org.apache.cassandra.distributed.api.Feature; -import org.apache.cassandra.distributed.api.ICluster; -import org.apache.cassandra.distributed.api.IInstance; -import org.apache.cassandra.distributed.api.IInstanceConfig; -import org.apache.cassandra.distributed.api.IInvokableInstance; -import org.apache.cassandra.distributed.api.IMessageFilters; -import org.apache.cassandra.distributed.api.NodeToolResult; +import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.impl.AbstractCluster; import org.apache.cassandra.distributed.impl.InstanceConfig; import org.apache.cassandra.distributed.impl.TestChangeListener; @@ -170,6 +166,33 @@ public static <I extends IInstance> void stopAll(ICluster<I> cluster) cluster.stream().forEach(ClusterUtils::stopUnchecked); } + /** + * Create a new instance and add it to the cluster, without starting it. + * + * @param cluster to add to + * @param fn function to add to the config before starting + * @param <I> instance type + * @return the instance added + */ + public static <I extends IInstance> I addInstance(AbstractCluster<I> cluster, Consumer<IInstanceConfig> fn) + { + I inst = cluster.stream().filter(i -> !i.isShutdown()).findFirst().get(); + return addInstance(cluster, inst.config(), fn); + } + + /** + * Create a new instance and add it to the cluster, without starting it. + * + * @param cluster to add to + * @param <I> instance type + * @return the instance added + */ + public static <I extends IInstance> I addInstance(AbstractCluster<I> cluster) + { + I inst = cluster.stream().filter(i -> !i.isShutdown()).findFirst().get(); + return addInstance(cluster, inst.config(), c -> {}); + } + /** * Create a new instance and add it to the cluster, without starting it. * @@ -464,11 +487,14 @@ protected static Callable<Void> pauseBeforeEnacting(IInvokableInstance instance, return () -> { try { + logger.info("EpochPause Waiting before enacting epoch {}", epoch); promise.get(wait, waitUnit); + logger.info("EpochPause stopped waiting before enacting epoch {}", epoch); return null; } catch (Throwable e) { + logger.info("EpochPause Timed out waiting for before enacting epoch {}", epoch); throw new RuntimeException(e); } }; @@ -492,11 +518,14 @@ protected static Callable<Void> pauseAfterEnacting(IInvokableInstance instance, return () -> { try { + logger.info("EpochPause Waiting after enacting epoch {}", epoch); promise.get(wait, waitUnit); + logger.info("EpochPause done waiting after enacting epoch {}", epoch); return null; } catch (Throwable e) { + logger.info("EpochPause Timed out waiting for after enacting epoch {}", epoch); throw new RuntimeException(e); } }; @@ -512,6 +541,7 @@ public static Callable<Epoch> pauseBeforeCommit(IInvokableInstance cmsInstance, return () -> { try { + logger.info("EpochPause Waiting before commit"); return promise.get(30, TimeUnit.SECONDS).getEpoch(); } catch (Throwable e) @@ -594,9 +624,9 @@ private ClusterMetadataVersion(int node, Epoch epoch) public String toString() { return "Version{" + - "node=" + node + - ", epoch=" + epoch + - '}'; + "node=" + node + + ", epoch=" + epoch + + '}'; } } @@ -606,7 +636,7 @@ public static void waitForCMSToQuiesce(ICluster<IInvokableInstance> cluster, int waitForCMSToQuiesce(cluster, maxEpoch(cluster, cmsNodes)); } - private static Epoch maxEpoch(ICluster<IInvokableInstance> cluster, int[] cmsNodes) + public static Epoch maxEpoch(ICluster<IInvokableInstance> cluster, int[] cmsNodes) { Epoch max = null; for (int id : cmsNodes) @@ -623,6 +653,11 @@ private static Epoch maxEpoch(ICluster<IInvokableInstance> cluster, int[] cmsNod } public static void waitForCMSToQuiesce(ICluster<IInvokableInstance> cluster, Epoch awaitedEpoch, int...ignored) + { + waitForCMSToQuiesce(cluster, awaitedEpoch, false, ignored); + } + + public static void waitForCMSToQuiesce(ICluster<IInvokableInstance> cluster, Epoch awaitedEpoch, boolean fetchLogWhenBehind, int...ignored) { List<ClusterMetadataVersion> notMatching = new ArrayList<>(); long deadline = System.nanoTime() + TimeUnit.SECONDS.toNanos(30); @@ -639,9 +674,12 @@ public static void waitForCMSToQuiesce(ICluster<IInvokableInstance> cluster, Epo if (skip) continue; - if (cluster.get(j).isShutdown()) + IInvokableInstance inst = cluster.get(j); + if (inst.isShutdown()) continue; - Epoch version = getClusterMetadataVersion(cluster.get(j)); + Epoch version = getClusterMetadataVersion(inst); + if (fetchLogWhenBehind && version.getEpoch() < awaitedEpoch.getEpoch()) + version = fetchLogFromCMS(inst, awaitedEpoch); if (version.getEpoch() < awaitedEpoch.getEpoch()) notMatching.add(new ClusterMetadataVersion(j, version)); } @@ -653,6 +691,17 @@ public static void waitForCMSToQuiesce(ICluster<IInvokableInstance> cluster, Epo throw new AssertionError(String.format("Some instances have not reached schema agreement with the leader. Awaited %s; diverging nodes: %s. ", awaitedEpoch, notMatching)); } + public static Epoch fetchLogFromCMS(IInvokableInstance inst, Epoch awaitedEpoch) + { + return fetchLogFromCMS(inst, awaitedEpoch.getEpoch()); + } + + public static Epoch fetchLogFromCMS(IInvokableInstance inst, long awaitedEpoch) + { + long latest = inst.callOnInstance(() -> ClusterMetadataService.instance().fetchLogFromCMS(Epoch.create(awaitedEpoch)).epoch.getEpoch()); + return Epoch.create(latest); + } + public static Epoch getCurrentEpoch(IInvokableInstance inst) { return decode(inst.callOnInstance(() -> encode(ClusterMetadata.current().epoch))); @@ -689,8 +738,8 @@ public static Map<String, Epoch> getPeerEpochs(IInvokableInstance requester) return epochs; }); return map.entrySet() - .stream() - .collect(Collectors.toMap(Map.Entry::getKey, e -> decode(e.getValue()))); + .stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> decode(e.getValue()))); } public static Set<String> getCMSMembers(IInvokableInstance inst) @@ -804,10 +853,9 @@ public static List<RingInstanceDetails> assertRingState(IInstance instance, IIns List<RingInstanceDetails> match = ring.stream() .filter(d -> d.address.equals(targetAddress)) .collect(Collectors.toList()); - assertThat(match) - .isNotEmpty() - .as("State was expected to be %s but was not", state) - .anyMatch(r -> r.state.equals(state)); + assertThat(match).isNotEmpty() + .as("State was expected to be %s but was not", state) + .anyMatch(r -> r.state.equals(state)); return ring; } @@ -839,7 +887,19 @@ private static List<RingInstanceDetails> awaitRing(IInstance src, String errorMe } sleepUninterruptibly(1, TimeUnit.SECONDS); } - throw new AssertionError(errorMessage + "\n" + ring); + throw new AssertionError(errorMessage + "\nsrc=" + src + "\nring=" + ring); + } + + /** + * Wait for the target to be in the ring as seen by the source instances + * @param cluster to check on + * @param nodes in the cluster to check + * @param expectedInRing instance to wait for + */ + public static void awaitRingJoin(Cluster cluster, int[] nodes, IInvokableInstance expectedInRing) + { + for (IInvokableInstance inst : cluster.get(nodes)) + awaitRingJoin(inst, expectedInRing); } /** @@ -883,8 +943,8 @@ public static List<RingInstanceDetails> awaitRingJoin(IInstance instance, String public static List<RingInstanceDetails> awaitRingHealthy(IInstance src) { return awaitRing(src, "Timeout waiting for ring to become healthy", - ring -> - ring.stream().allMatch(ClusterUtils::isRingInstanceDetailsHealthy)); + ring -> + ring.stream().allMatch(ClusterUtils::isRingInstanceDetailsHealthy)); } /** @@ -951,8 +1011,8 @@ public static List<RingInstanceDetails> assertRingIs(IInstance instance, IInstan public static List<RingInstanceDetails> assertRingIs(IInstance instance, Collection<? extends IInstance> expectedInRing) { Set<String> expectedRingAddresses = expectedInRing.stream() - .map(i -> i.config().broadcastAddress().getAddress().getHostAddress()) - .collect(Collectors.toSet()); + .map(i -> i.config().broadcastAddress().getAddress().getHostAddress()) + .collect(Collectors.toSet()); return assertRingIs(instance, expectedRingAddresses); } @@ -969,8 +1029,8 @@ public static List<RingInstanceDetails> assertRingIs(IInstance instance, Set<Str List<RingInstanceDetails> ring = ring(instance); Set<String> ringAddresses = ring.stream().map(d -> d.address).collect(Collectors.toSet()); assertThat(ringAddresses) - .as("Ring addreses did not match for instance %s", instance) - .isEqualTo(expectedRingAddresses); + .as("Ring addreses did not match for instance %s", instance) + .isEqualTo(expectedRingAddresses); return ring; } @@ -1085,9 +1145,9 @@ public static void awaitGossipStateMatch(ICluster<? extends IInstance> cluster, for (int i = 0; i < 100; i++) { matches = cluster.stream().map(ClusterUtils::gossipInfo) - .map(gi -> Objects.requireNonNull(gi.get(getBroadcastAddressString(expectedInGossip)))) - .map(m -> m.get(key.name())) - .collect(Collectors.toSet()); + .map(gi -> Objects.requireNonNull(gi.get(getBroadcastAddressString(expectedInGossip)))) + .map(m -> m.get(key.name())) + .collect(Collectors.toSet()); if (matches.isEmpty() || matches.size() == 1) return; sleepUninterruptibly(1, TimeUnit.SECONDS); @@ -1342,7 +1402,7 @@ public static String getBroadcastAddressHostWithPortString(IInstance target) public static InetSocketAddress getNativeInetSocketAddress(IInstance target) { return new InetSocketAddress(target.config().broadcastAddress().getAddress(), - getIntConfig(target.config(), "native_transport_port", 9042)); + getIntConfig(target.config(), "native_transport_port", 9042)); } /** @@ -1445,5 +1505,50 @@ public static void preventSystemExit() { System.setSecurityManager(new PreventSystemExit()); } + + public static void awaitInPeers(Cluster cluster, int[] nodes, IInstance expectedInPeers) + { + for (IInvokableInstance inst : cluster.get(nodes)) + { + if (inst.config().num() == expectedInPeers.config().num()) continue; // ignore self as self is not in peers + awaitInPeers(inst, expectedInPeers); + } + } + + public static void awaitInPeers(IInstance instance, IInstance expectedInPeers) + { + for (int i = 0; i < 100; i++) + { + if (isInPeers(instance, expectedInPeers)) + return; + sleepUninterruptibly(1, TimeUnit.SECONDS); + } + throw new AssertionError("Unable to find " + expectedInPeers.config().broadcastAddress() + " in peers"); + } + + public static boolean isInPeers(IInstance instance, IInstance expectedInPeers) + { + SimpleQueryResult qr = instance.executeInternalWithResult("select tokens, data_center, rack from system.peers WHERE peer=?", expectedInPeers.config().broadcastAddress().getAddress()); + if (!qr.hasNext()) return false; + Row row = qr.next(); + // peer is known, but is it fully defined? + Collection<String> tokens = row.get("tokens"); + String dc = row.getString("data_center"); + String rack = row.getString("rack"); + return tokens != null && !tokens.isEmpty() && !Strings.isNullOrEmpty(dc) && !Strings.isNullOrEmpty(rack); + } + + public static StorageService.Mode mode(IInvokableInstance inst) + { + String name = inst.callOnInstance(() -> StorageService.instance.operationMode().name()); + return StorageService.Mode.valueOf(name); + } + + public static void assertModeJoined(IInvokableInstance inst) + { + Assertions.assertThat(mode(inst)) + .describedAs("Unexpected StorageService operation mode") + .isEqualTo(StorageService.Mode.NORMAL); + } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/IntegrationTestBase.java b/test/distributed/org/apache/cassandra/distributed/test/IntegrationTestBase.java new file mode 100644 index 000000000000..16f4e58f304d --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/IntegrationTestBase.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import java.util.function.Consumer; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import accord.utils.Invariants; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.IInstanceConfig; + +public class IntegrationTestBase extends TestBaseImpl +{ + protected static final Logger logger = LoggerFactory.getLogger(IntegrationTestBase.class); + protected static Cluster cluster; + + private static boolean initialized = false; + + @BeforeClass + public static void before() throws Throwable + { + init(1, defaultConfig()); + } + + protected static void init(int nodes, Consumer<IInstanceConfig> cfg) throws Throwable + { + Invariants.checkState(!initialized); + cluster = Cluster.build() + .withNodes(nodes) + .withConfig(cfg) + .createWithoutStarting(); + cluster.startup(); + cluster = init(cluster); + initialized = true; + } + + @AfterClass + public static void afterClass() + { + if (cluster != null) + cluster.close(); + } + + // TODO: meta-randomize this + public static Consumer<IInstanceConfig> defaultConfig() + { + return (cfg) -> { + cfg.set("row_cache_size", "50MiB") + .set("index_summary_capacity", "50MiB") + .set("counter_cache_size", "50MiB") + .set("key_cache_size", "50MiB") + .set("file_cache_size", "50MiB") + .set("index_summary_capacity", "50MiB") + .set("memtable_heap_space", "128MiB") + .set("memtable_offheap_space", "128MiB") + .set("memtable_flush_writers", 1) + .set("concurrent_compactors", 1) + .set("concurrent_reads", 5) + .set("concurrent_writes", 5) + .set("compaction_throughput_mb_per_sec", 10) + .set("hinted_handoff_enabled", false); + }; + } + +} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/BounceGossipTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/BounceGossipTest.java index 8c188a2feca8..3bcac51f64b0 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/BounceGossipTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/BounceGossipTest.java @@ -19,127 +19,153 @@ package org.apache.cassandra.distributed.test.log; import java.io.IOException; +import java.util.Iterator; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.LockSupport; import com.google.common.util.concurrent.Uninterruptibles; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.Constants; -import org.apache.cassandra.harry.HarryHelper; -import org.apache.cassandra.harry.runner.FlaggedRunner; import org.apache.cassandra.distributed.test.TestBaseImpl; import org.apache.cassandra.gms.ApplicationState; import org.apache.cassandra.gms.EndpointState; import org.apache.cassandra.gms.Gossiper; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.sut.injvm.InJvmSut; -import org.apache.cassandra.harry.sut.injvm.InJvmSutBase; -import org.apache.cassandra.harry.visitors.MutatingVisitor; -import org.apache.cassandra.harry.visitors.QueryLogger; -import org.apache.cassandra.harry.visitors.RandomPartitionValidator; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.op.Visit; +import org.apache.cassandra.harry.dsl.HistoryBuilder; +import org.apache.cassandra.harry.dsl.HistoryBuilderHelper; +import org.apache.cassandra.harry.execution.CQLVisitExecutor; +import org.apache.cassandra.harry.execution.InJvmDTestVisitExecutor; +import org.apache.cassandra.harry.gen.Generator; +import org.apache.cassandra.harry.gen.Generators; +import org.apache.cassandra.harry.gen.SchemaGenerators; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.utils.concurrent.CountDownLatch; -import static java.util.Arrays.asList; import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory; import static org.apache.cassandra.distributed.api.Feature.GOSSIP; import static org.apache.cassandra.distributed.api.Feature.NETWORK; -import static org.apache.cassandra.harry.core.Configuration.VisitorPoolConfiguration.pool; -import static org.apache.cassandra.harry.ddl.ColumnSpec.asciiType; -import static org.apache.cassandra.harry.ddl.ColumnSpec.ck; -import static org.apache.cassandra.harry.ddl.ColumnSpec.int64Type; -import static org.apache.cassandra.harry.ddl.ColumnSpec.pk; -import static org.apache.cassandra.harry.ddl.ColumnSpec.regularColumn; -import static org.apache.cassandra.harry.ddl.ColumnSpec.staticColumn; -import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; +import static org.apache.cassandra.harry.checker.TestHelper.withRandom; import static org.junit.Assert.fail; import static org.psjava.util.AssertStatus.assertTrue; public class BounceGossipTest extends TestBaseImpl { + private static final Logger logger = LoggerFactory.getLogger(BounceGossipTest.class); + @Test public void bounceTest() throws Exception { + Generator<SchemaSpec> schemaGen = SchemaGenerators.schemaSpecGen("bounce_gossip_test", "bounce_test", 1000); try (Cluster cluster = init(builder().withNodes(3) .withConfig(config -> config.with(NETWORK, GOSSIP) .set(Constants.KEY_DTEST_FULL_STARTUP, true)) .start())) { + ExecutorService es = executorFactory().pooled("harry", 1); - SchemaSpec schema = new SchemaSpec("harry", "test_table", - asList(pk("pk1", asciiType), pk("pk1", int64Type)), - asList(ck("ck1", asciiType), ck("ck1", int64Type)), - asList(regularColumn("regular1", asciiType), regularColumn("regular1", int64Type)), - asList(staticColumn("static1", asciiType), staticColumn("static1", int64Type))); - AtomicInteger down = new AtomicInteger(0); - Configuration config = HarryHelper.defaultConfiguration() - .setKeyspaceDdl(String.format("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': %d};", schema.keyspace, 3)) - .setSUT(() -> new InJvmSut(cluster, () -> 1, InJvmSutBase.retryOnTimeout())) - .build(); - - CountDownLatch stopLatch = CountDownLatch.newCountDownLatch(1); - Future<?> f = es.submit(() -> { - try - { - new FlaggedRunner(config.createRun(), - config, - asList(pool("Writer", 1, MutatingVisitor::new), - pool("Reader", 1, (run) -> new RandomPartitionValidator(run, new Configuration.QuiescentCheckerConfig(), QueryLogger.NO_OP))), - stopLatch).run(); - } - catch (Throwable e) + AtomicBoolean stop = new AtomicBoolean(false); + + withRandom(rng -> { + SchemaSpec schema = schemaGen.generate(rng); + Generators.TrackingGenerator<Integer> pkGen = Generators.tracking(Generators.int32(0, Math.min(schema.valueGenerators.pkPopulation(), 1000))); + Generator<Integer> ckGen = Generators.int32(0, Math.min(schema.valueGenerators.ckPopulation(), 1000)); + + cluster.schemaChange("CREATE KEYSPACE " + schema.keyspace + + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 3};"); + cluster.schemaChange(schema.compile()); + + Future<?> f = es.submit(new Runnable() { - throw new RuntimeException(e); - } - }); - Uninterruptibles.sleepUninterruptibly(10, TimeUnit.SECONDS); - - down.set(3); - cluster.get(3).shutdown(true).get(); - cluster.get(1).logs().watchFor("/127.0.0.3:.* is now DOWN"); - Uninterruptibles.sleepUninterruptibly(5, TimeUnit.SECONDS); - cluster.get(3).startup(); - cluster.get(1).logs().watchFor("/127.0.0.3:.* is now UP"); - down.set(0); - stopLatch.decrement(); - f.get(); - - for (int inst = 1; inst <= 3; inst++) - { - cluster.get(inst).runOnInstance(() -> { - for (int i = 1; i <= 3; i++) + final HistoryBuilder history = new HistoryBuilder(schema.valueGenerators); + final Iterator<Visit> iterator = history.iterator(); + final CQLVisitExecutor executor = InJvmDTestVisitExecutor.builder() + .nodeSelector(lts -> 1) + .retryPolicy(InJvmDTestVisitExecutor.RetryPolicy.RETRY_ON_TIMEOUT) + .build(schema, history, cluster); + @Override + public void run() { - boolean stateOk = false; - int tries = 0; - while (!stateOk) + while (!stop.get()) { - EndpointState epstate = Gossiper.instance.getEndpointStateForEndpoint(InetAddressAndPort.getByNameUnchecked("127.0.0." + i)); - stateOk = epstate.getApplicationState(ApplicationState.STATUS_WITH_PORT).value.contains("NORMAL") && - epstate.getApplicationState(ApplicationState.TOKENS) != null && - epstate.getHeartBeatState().getGeneration() > 0; - if (!stateOk) + // Rate limit to ~10 per second + LockSupport.parkNanos(TimeUnit.MILLISECONDS.toNanos(500)); + HistoryBuilderHelper.insertRandomData(schema, pkGen, ckGen, rng, history); + history.selectPartition(pkGen.generate(rng)); + + while (iterator.hasNext() && !stop.get()) { - tries++; - if (tries > 20) + try { - assertTrue(epstate.getApplicationState(ApplicationState.STATUS_WITH_PORT).value.contains("NORMAL")); - assertTrue(epstate.getApplicationState(ApplicationState.TOKENS) != null); - assertTrue(epstate.getHeartBeatState().getGeneration() > 0); - fail("shouldn't reach this, but epstate: "+epstate); + executor.execute(iterator.next()); + } + catch (Throwable e) + { + if (e.getMessage().contains("Can't use shutdown node3")) + { + // ignore and continue + continue; + } + + if (e.getMessage().contains("does not exist")) + { + logger.error(e.getMessage()); + // ignore and continue + continue; + } + throw e; } - Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS); } } } }); + Uninterruptibles.sleepUninterruptibly(10, TimeUnit.SECONDS); - } + cluster.get(3).shutdown(true).get(); + cluster.get(1).logs().watchFor("/127.0.0.3:.* is now DOWN"); + Uninterruptibles.sleepUninterruptibly(5, TimeUnit.SECONDS); + cluster.get(3).startup(); + cluster.get(1).logs().watchFor("/127.0.0.3:.* is now UP"); + stop.set(true); + f.get(); + for (int inst = 1; inst <= 3; inst++) + { + cluster.get(inst).runOnInstance(() -> { + for (int i = 1; i <= 3; i++) + { + boolean stateOk = false; + int tries = 0; + while (!stateOk) + { + EndpointState epstate = Gossiper.instance.getEndpointStateForEndpoint(InetAddressAndPort.getByNameUnchecked("127.0.0." + i)); + stateOk = epstate.getApplicationState(ApplicationState.STATUS_WITH_PORT).value.contains("NORMAL") && + epstate.getApplicationState(ApplicationState.TOKENS) != null && + epstate.getHeartBeatState().getGeneration() > 0; + if (!stateOk) + { + tries++; + if (tries > 20) + { + assertTrue(epstate.getApplicationState(ApplicationState.STATUS_WITH_PORT).value.contains("NORMAL")); + assertTrue(epstate.getApplicationState(ApplicationState.TOKENS) != null); + assertTrue(epstate.getHeartBeatState().getGeneration() > 0); + fail("shouldn't reach this, but epstate: " + epstate); + } + Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS); + } + } + } + }); + } + }); } } @@ -157,7 +183,7 @@ public void gossipPropagatesVersionTest() throws IOException int correctTokensVersion; while ((correctTokensVersion = getGossipTokensVersion(cluster, 2)) == tokensBefore) Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS); // wait for LegacyStateListener to actually update gossip - for (int inst : new int[] {1, 3}) + for (int inst : new int[]{ 1, 3 }) while (correctTokensVersion != getGossipTokensVersion(cluster, inst)) { System.out.println(correctTokensVersion + " ::: " + getGossipTokensVersion(cluster, inst)); @@ -172,75 +198,4 @@ private static int getGossipTokensVersion(Cluster cluster, int instance) return cluster.get(instance).callOnInstance(() -> Gossiper.instance.endpointStateMap.get(InetAddressAndPort.getByNameUnchecked("127.0.0.2")) .getApplicationState(ApplicationState.TOKENS).version); } - - private static String createHarryConf() - { - return "seed: " + currentTimeMillis() + "\n" + - "\n" + - "# Default schema provider generates random schema\n" + - "schema_provider:\n" + - " fixed:\n" + - " keyspace: harry\n" + - " table: test_table\n" + - " partition_keys:\n" + - " pk1: bigint\n" + - " pk2: ascii\n" + - " clustering_keys:\n" + - " ck1: ascii\n" + - " ck2: bigint\n" + - " regular_columns:\n" + - " v1: ascii\n" + - " v2: bigint\n" + - " v3: ascii\n" + - " v4: bigint\n" + - " static_keys:\n" + - " s1: ascii\n" + - " s2: bigint\n" + - " s3: ascii\n" + - " s4: bigint\n" + - "\n" + - "clock:\n" + - " offset:\n" + - " offset: 1000\n" + - "\n" + - "drop_schema: false\n" + - "create_schema: true\n" + - "truncate_table: true\n" + - "\n" + - "partition_descriptor_selector:\n" + - " default:\n" + - " window_size: 10\n" + - " slide_after_repeats: 100\n" + - "\n" + - "clustering_descriptor_selector:\n" + - " default:\n" + - " modifications_per_lts:\n" + - " type: \"constant\"\n" + - " constant: 2\n" + - " rows_per_modification:\n" + - " type: \"constant\"\n" + - " constant: 2\n" + - " operation_kind_weights:\n" + - " DELETE_RANGE: 0\n" + - " DELETE_SLICE: 0\n" + - " DELETE_ROW: 0\n" + - " DELETE_COLUMN: 0\n" + - " DELETE_PARTITION: 0\n" + - " DELETE_COLUMN_WITH_STATICS: 0\n" + - " INSERT_WITH_STATICS: 50\n" + - " INSERT: 50\n" + - " UPDATE_WITH_STATICS: 50\n" + - " UPDATE: 50\n" + - " column_mask_bitsets: null\n" + - " max_partition_size: 1000\n" + - "\n" + - "metric_reporter:\n" + - " no_op: {}\n" + - "\n" + - "data_tracker:\n" + - " locking:\n" + - " max_seen_lts: -1\n" + - " max_complete_lts: -1"; - } - -} +} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/CMSTestBase.java b/test/distributed/org/apache/cassandra/distributed/test/log/CMSTestBase.java index 538bf4f491e8..c89b52c04706 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/CMSTestBase.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/CMSTestBase.java @@ -24,7 +24,7 @@ import org.apache.cassandra.db.guardrails.Guardrails; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.distributed.api.IIsolatedExecutor; -import org.apache.cassandra.harry.sut.TokenPlacementModel; +import org.apache.cassandra.harry.model.TokenPlacementModel; import org.apache.cassandra.locator.MetaStrategy; import org.apache.cassandra.schema.DistributedSchema; import org.apache.cassandra.schema.KeyspaceMetadata; diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/CoordinatorPathTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/CoordinatorPathTest.java index 388185e8d2f1..42630b3a297c 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/CoordinatorPathTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/CoordinatorPathTest.java @@ -27,29 +27,32 @@ import java.util.function.Function; import java.util.stream.Collectors; -import org.apache.cassandra.harry.sut.TokenPlacementModel.Replica; import org.junit.Assert; import org.junit.Test; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.operations.CompiledStatement; -import org.apache.cassandra.harry.operations.WriteHelper; -import org.apache.cassandra.harry.sut.injvm.InJvmSut; -import org.apache.cassandra.harry.util.ByteUtils; -import org.apache.cassandra.harry.util.TokenUtil; import org.apache.cassandra.db.Mutation; import org.apache.cassandra.db.ReadCommand; import org.apache.cassandra.db.ReadResponse; import org.apache.cassandra.distributed.api.ConsistencyLevel; -import org.apache.cassandra.harry.HarryHelper; -import org.apache.cassandra.harry.sut.TokenPlacementModel; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.ValueGeneratorHelper; +import org.apache.cassandra.harry.cql.WriteHelper; +import org.apache.cassandra.harry.dsl.HistoryBuilder; +import org.apache.cassandra.harry.execution.CompiledStatement; +import org.apache.cassandra.harry.gen.Generator; +import org.apache.cassandra.harry.gen.SchemaGenerators; +import org.apache.cassandra.harry.model.TokenPlacementModel; +import org.apache.cassandra.harry.model.TokenPlacementModel.Replica; +import org.apache.cassandra.harry.op.Operations; +import org.apache.cassandra.harry.util.ByteUtils; +import org.apache.cassandra.harry.util.TokenUtil; import org.apache.cassandra.net.Message; import org.apache.cassandra.net.NoPayload; import org.apache.cassandra.net.Verb; import org.apache.cassandra.utils.concurrent.Future; +import static org.apache.cassandra.harry.checker.TestHelper.withRandom; + public class CoordinatorPathTest extends CoordinatorPathTestBase { private static final TokenPlacementModel.SimpleReplicationFactor RF = new TokenPlacementModel.SimpleReplicationFactor(3); @@ -57,11 +60,8 @@ public class CoordinatorPathTest extends CoordinatorPathTestBase @Test public void writeConsistencyTest() throws Throwable { + Generator<SchemaSpec> schemaGen = SchemaGenerators.schemaSpecGen(KEYSPACE, "write_consistency_test", 1000); coordinatorPathTest(RF, (cluster, simulatedCluster) -> { - Configuration.ConfigurationBuilder configBuilder = HarryHelper.defaultConfiguration() - .setSUT(() -> new InJvmSut(cluster)); - Run run = configBuilder.build().createRun(); - for (int ignored : new int[]{ 2, 3, 4, 5 }) simulatedCluster.createNode().register(); @@ -75,68 +75,72 @@ public void writeConsistencyTest() throws Throwable .prepareJoin() .startJoin(); - cluster.schemaChange("CREATE KEYSPACE " + run.schemaSpec.keyspace + - " WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 3};"); - cluster.schemaChange(run.schemaSpec.compile().cql()); - - while (true) - { - long lts = run.clock.nextLts(); - - long pd = run.pdSelector.pd(run.clock.nextLts(), run.schemaSpec); + withRandom(rng -> { + SchemaSpec schema = schemaGen.generate(rng); + cluster.schemaChange("CREATE KEYSPACE " + schema.keyspace + + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 3};"); + cluster.schemaChange(schema.compile()); - ByteBuffer[] pk = ByteUtils.objectsToBytes(run.schemaSpec.inflatePartitionKey(pd)); - long token = TokenUtil.token(ByteUtils.compose(pk)); - if (!prediction.state.get().isWriteTargetFor(token, prediction.node(6).matcher)) - continue; - - simulatedCluster.waitForQuiescense(); - List<Replica> replicas = simulatedCluster.state.get().writePlacementsFor(token); - // At most 2 replicas should respond, so that when the pending node is added, results would be insufficient for recomputed blockFor - BooleanSupplier shouldRespond = atMostResponses(simulatedCluster.state.get().isWriteTargetFor(token, simulatedCluster.node(1).matcher) ? 1 : 2); - List<WaitingAction<?,?>> waiting = simulatedCluster - .filter((n) -> replicas.stream().map(Replica::node).anyMatch(n.matcher) && n.node.idx() != 1) - .map((nodeToBlockOn) -> nodeToBlockOn.blockOnReplica((node) -> new MutationAction(node, shouldRespond))) - .collect(Collectors.toList()); - - Future<?> writeQuery = async(() -> { - long cd = run.descriptorSelector.cd(pd, lts, 0, run.schemaSpec); - CompiledStatement s = WriteHelper.inflateInsert(run.schemaSpec, - pd, - cd, - run.descriptorSelector.vds(pd, cd, lts, 0, OpSelectors.OperationKind.INSERT_WITH_STATICS, run.schemaSpec), - run.descriptorSelector.sds(pd, cd, lts, 0, OpSelectors.OperationKind.INSERT_WITH_STATICS, run.schemaSpec), - run.clock.rts(lts)); - cluster.coordinator(1).execute(s.cql(), ConsistencyLevel.QUORUM, s.bindings()); - return null; - }); - - waiting.forEach(WaitingAction::waitForMessage); - - simulatedCluster.createNode().register(); - simulatedCluster.node(6) - .lazyJoin() - .prepareJoin() - .startJoin(); - - simulatedCluster.waitForQuiescense(); - - waiting.forEach(WaitingAction::resume); - - try - { - writeQuery.get(); - Assert.fail("Should have thrown"); - } - catch (Throwable t) + HistoryBuilder.IndexedValueGenerators valueGenerators = (HistoryBuilder.IndexedValueGenerators) schema.valueGenerators; + for (int i = 0; i < valueGenerators.pkPopulation(); i++) { - if (t.getMessage() == null) - throw t; - Assert.assertTrue("Expected a different error message, but got " + t.getMessage(), - t.getMessage().contains("the ring has changed")); - return; + long pd = valueGenerators.pkGen().descriptorAt(i); + + ByteBuffer[] pk = ByteUtils.objectsToBytes(valueGenerators.pkGen().inflate(pd)); + long token = TokenUtil.token(ByteUtils.compose(pk)); + if (!prediction.state.get().isWriteTargetFor(token, prediction.node(6).matcher)) + continue; + + simulatedCluster.waitForQuiescense(); + List<Replica> replicas = simulatedCluster.state.get().writePlacementsFor(token); + // At most 2 replicas should respond, so that when the pending node is added, results would be insufficient for recomputed blockFor + BooleanSupplier shouldRespond = atMostResponses(simulatedCluster.state.get().isWriteTargetFor(token, simulatedCluster.node(1).matcher) ? 1 : 2); + List<WaitingAction<?,?>> waiting = simulatedCluster + .filter((n) -> replicas.stream().map(Replica::node).anyMatch(n.matcher) && n.node.idx() != 1) + .map((nodeToBlockOn) -> nodeToBlockOn.blockOnReplica((node) -> new MutationAction(node, shouldRespond))) + .collect(Collectors.toList()); + + long lts = 1L; + Future<?> writeQuery = async(() -> { + + CompiledStatement s = WriteHelper.inflateInsert(new Operations.WriteOp(lts, pd, 0, + ValueGeneratorHelper.randomDescriptors(rng, valueGenerators::regularColumnGen, valueGenerators.regularColumnCount()), + ValueGeneratorHelper.randomDescriptors(rng, valueGenerators::staticColumnGen, valueGenerators.staticColumnCount()), + Operations.Kind.INSERT), + schema, + lts); + cluster.coordinator(1).execute(s.cql(), ConsistencyLevel.QUORUM, s.bindings()); + return null; + }); + + waiting.forEach(WaitingAction::waitForMessage); + + simulatedCluster.createNode().register(); + simulatedCluster.node(6) + .lazyJoin() + .prepareJoin() + .startJoin(); + + simulatedCluster.waitForQuiescense(); + + waiting.forEach(WaitingAction::resume); + + try + { + writeQuery.get(); + Assert.fail("Should have thrown"); + } + catch (Throwable t) + { + if (t.getMessage() == null) + throw t; + Assert.assertTrue("Expected a different error message, but got " + t.getMessage(), + t.getMessage().contains("the ring has changed")); + return; + } } - } + + }); }); } @@ -253,4 +257,4 @@ public Message<NoPayload> respondTo(Message<Mutation> request) }); } -} +} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/CoordinatorPathTestBase.java b/test/distributed/org/apache/cassandra/distributed/test/log/CoordinatorPathTestBase.java index af9f1806bf3c..cfa0ec4be7ca 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/CoordinatorPathTestBase.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/CoordinatorPathTestBase.java @@ -64,11 +64,11 @@ import org.apache.cassandra.distributed.test.log.PlacementSimulator.Transformations; import org.apache.cassandra.gms.GossipDigestAck; import org.apache.cassandra.gms.GossipDigestSyn; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.sut.TokenPlacementModel.Node; -import org.apache.cassandra.harry.sut.TokenPlacementModel.NodeFactory; +import org.apache.cassandra.harry.model.TokenPlacementModel; +import org.apache.cassandra.harry.model.TokenPlacementModel.Node; +import org.apache.cassandra.harry.model.TokenPlacementModel.NodeFactory; import org.apache.cassandra.harry.util.ByteUtils; -import org.apache.cassandra.harry.util.TestRunner; +import org.apache.cassandra.harry.dsl.TestRunner; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.SimpleSeedProvider; import org.apache.cassandra.tcm.*; diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/FailedLeaveTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/FailedLeaveTest.java index 5dadb0ed01da..a5f88d92479a 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/FailedLeaveTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/FailedLeaveTest.java @@ -22,46 +22,43 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.BiFunction; import org.junit.Assert; import org.junit.Test; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.sut.injvm.InJVMTokenAwareVisitExecutor; -import org.apache.cassandra.harry.sut.injvm.InJvmSut; -import org.apache.cassandra.harry.sut.injvm.QuiescentLocalStateChecker; -import org.apache.cassandra.harry.visitors.GeneratingVisitor; -import org.apache.cassandra.harry.visitors.MutatingRowVisitor; -import org.apache.cassandra.harry.visitors.Visitor; import net.bytebuddy.ByteBuddy; import net.bytebuddy.dynamic.loading.ClassLoadingStrategy; import net.bytebuddy.implementation.MethodDelegation; import net.bytebuddy.implementation.bind.annotation.SuperCall; import org.apache.cassandra.distributed.Cluster; -import org.apache.cassandra.distributed.api.ConsistencyLevel; import org.apache.cassandra.distributed.api.Feature; import org.apache.cassandra.distributed.api.IInvokableInstance; -import org.apache.cassandra.harry.HarryHelper; import org.apache.cassandra.distributed.shared.ClusterUtils; import org.apache.cassandra.distributed.shared.ClusterUtils.SerializableBiPredicate; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.dsl.HistoryBuilder; +import org.apache.cassandra.harry.dsl.ReplayingHistoryBuilder; +import org.apache.cassandra.harry.execution.InJvmDTestVisitExecutor; +import org.apache.cassandra.harry.gen.Generator; +import org.apache.cassandra.harry.gen.Generators; +import org.apache.cassandra.harry.gen.SchemaGenerators; +import org.apache.cassandra.streaming.IncomingStream; +import org.apache.cassandra.streaming.StreamReceiveTask; import org.apache.cassandra.tcm.Commit; import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.Transformation; import org.apache.cassandra.tcm.transformations.CancelInProgressSequence; import org.apache.cassandra.tcm.transformations.PrepareLeave; -import org.apache.cassandra.streaming.IncomingStream; -import org.apache.cassandra.streaming.StreamReceiveTask; import static net.bytebuddy.matcher.ElementMatchers.named; import static org.apache.cassandra.distributed.shared.ClusterUtils.cancelInProgressSequences; import static org.apache.cassandra.distributed.shared.ClusterUtils.decommission; import static org.apache.cassandra.distributed.shared.ClusterUtils.getClusterMetadataVersion; import static org.apache.cassandra.distributed.shared.ClusterUtils.getSequenceAfterCommit; +import static org.apache.cassandra.harry.checker.TestHelper.withRandom; public class FailedLeaveTest extends FuzzTestBase { @@ -89,7 +86,7 @@ private void failedLeaveTest(BiFunction<ExecutorService, IInvokableInstance, Fut SerializableBiPredicate<Transformation, Commit.Result> actionCommitted) throws Exception { - ExecutorService es = Executors.newSingleThreadExecutor(); + Generator<SchemaSpec> schemaGen = SchemaGenerators.schemaSpecGen(KEYSPACE, "failed_leave_test", 1000); try (Cluster cluster = builder().withNodes(3) .withInstanceInitializer(BB::install) .appendConfig(c -> c.with(Feature.NETWORK)) @@ -98,64 +95,72 @@ private void failedLeaveTest(BiFunction<ExecutorService, IInvokableInstance, Fut IInvokableInstance cmsInstance = cluster.get(1); IInvokableInstance leavingInstance = cluster.get(2); - Configuration.ConfigurationBuilder configBuilder = HarryHelper.defaultConfiguration() - .setSUT(() -> new InJvmSut(cluster)); - Run run = configBuilder.build().createRun(); - - cluster.coordinator(1).execute("CREATE KEYSPACE " + run.schemaSpec.keyspace + - " WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 2};", - ConsistencyLevel.ALL); - cluster.coordinator(1).execute(run.schemaSpec.compile().cql(), ConsistencyLevel.ALL); - ClusterUtils.waitForCMSToQuiesce(cluster, cmsInstance); - - TokenPlacementModel.ReplicationFactor rf = new TokenPlacementModel.SimpleReplicationFactor(2); - QuiescentLocalStateChecker model = new QuiescentLocalStateChecker(run, rf); - Visitor visitor = new GeneratingVisitor(run, new InJVMTokenAwareVisitExecutor(run, - MutatingRowVisitor::new, - SystemUnderTest.ConsistencyLevel.ALL, - rf)); - for (int i = 0; i < WRITES; i++) - visitor.visit(); - - Epoch startEpoch = getClusterMetadataVersion(cmsInstance); - // Configure node 3 to fail when receiving streams, then start decommissioning node 2 - cluster.get(3).runOnInstance(() -> BB.failReceivingStream.set(true)); - Future<Boolean> success = es.submit(() -> decommission(leavingInstance)); - Assert.assertFalse(success.get()); - - // metadata event log should have advanced by 2 entries, PREPARE_LEAVE & START_LEAVE - ClusterUtils.waitForCMSToQuiesce(cluster, cmsInstance); - Epoch currentEpoch = getClusterMetadataVersion(cmsInstance); - Assert.assertEquals(startEpoch.getEpoch() + 2, currentEpoch.getEpoch()); - - // Node 2's leaving failed due to the streaming errors. If decommission is called again on the node, it should - // resume where it left off. Allow streaming to succeed this time and verify that the node is able to - // finish leaving. - cluster.get(3).runOnInstance(() -> BB.failReceivingStream.set(false)); - - // Run the desired action to mitigate the failure (i.e. retry or cancel) - success = runAfterFailure.apply(es, leavingInstance); - - // get the Epoch of the event resulting from that action, so we can wait for it - Epoch nextEpoch = getSequenceAfterCommit(cmsInstance, actionCommitted).call(); - - Assert.assertTrue(success.get()); - - // wait for the cluster to all witness the event submitted after failure - // (i.e. the FINISH_JOIN or CANCEL_SEQUENCE). - ClusterUtils.waitForCMSToQuiesce(cluster, nextEpoch); - - //validate the state of the cluster - for (int i = 0; i < WRITES; i++) - visitor.visit(); - model.validateAll(); + withRandom(rng -> { + SchemaSpec schema = schemaGen.generate(rng); + Generators.TrackingGenerator<Integer> pkGen = Generators.tracking(Generators.int32(0, Math.min(schema.valueGenerators.pkPopulation(), 1000))); + Generator<Integer> ckGen = Generators.int32(0, Math.min(schema.valueGenerators.ckPopulation(), 1000)); + + HistoryBuilder history = new ReplayingHistoryBuilder(schema.valueGenerators, + (hb) -> InJvmDTestVisitExecutor.builder() + .nodeSelector(i -> 1) + .build(schema, hb, cluster)); + history.custom(() -> { + cluster.schemaChange("CREATE KEYSPACE " + schema.keyspace + + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 2};"); + cluster.schemaChange(schema.compile()); + }, "Setup"); + + Runnable writeAndValidate = () -> { + for (int i = 0; i < WRITES; i++) + history.insert(pkGen.generate(rng), ckGen.generate(rng)); + + for (int pk : pkGen.generated()) + history.selectPartition(pk); + }; + writeAndValidate.run(); + + history.customThrowing(() -> { + ExecutorService es = Executors.newSingleThreadExecutor(); + Epoch startEpoch = getClusterMetadataVersion(cmsInstance); + // Configure node 3 to fail when receiving streams, then start decommissioning node 2 + cluster.get(3).runOnInstance(() -> BB.failReceivingStream.set(true)); + Future<Boolean> success = es.submit(() -> decommission(leavingInstance)); + Assert.assertFalse(success.get()); + + // metadata event log should have advanced by 2 entries, PREPARE_LEAVE & START_LEAVE + ClusterUtils.waitForCMSToQuiesce(cluster, cmsInstance); + Epoch currentEpoch = getClusterMetadataVersion(cmsInstance); + Assert.assertEquals(startEpoch.getEpoch() + 2, currentEpoch.getEpoch()); + + // Node 2's leaving failed due to the streaming errors. If decommission is called again on the node, it should + // resume where it left off. Allow streaming to succeed this time and verify that the node is able to + // finish leaving. + cluster.get(3).runOnInstance(() -> BB.failReceivingStream.set(false)); + + // Run the desired action to mitigate the failure (i.e. retry or cancel) + success = runAfterFailure.apply(es, leavingInstance); + + // get the Epoch of the event resulting from that action, so we can wait for it + Epoch nextEpoch = getSequenceAfterCommit(cmsInstance, actionCommitted).call(); + + Assert.assertTrue(success.get()); + + es.shutdown(); + es.awaitTermination(1, TimeUnit.MINUTES); + // wait for the cluster to all witness the event submitted after failure + // (i.e. the FINISH_JOIN or CANCEL_SEQUENCE). + ClusterUtils.waitForCMSToQuiesce(cluster, nextEpoch); + }, "Failed leave"); + + writeAndValidate.run(); + }); } } - public static class BB { static AtomicBoolean failReceivingStream = new AtomicBoolean(false); + public static void install(ClassLoader cl, int instance) { if (instance == 3) @@ -175,4 +180,4 @@ public static void received(IncomingStream stream, @SuperCall Callable<Void> zup zuper.call(); } } -} +} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/FuzzTestBase.java b/test/distributed/org/apache/cassandra/distributed/test/log/FuzzTestBase.java index 86e5f681e1f4..48df35dd7681 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/FuzzTestBase.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/FuzzTestBase.java @@ -24,7 +24,15 @@ import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.test.TestBaseImpl; -import org.apache.cassandra.harry.HarryHelper; + +import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_ALLOW_SIMPLE_STRATEGY; +import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_MINIMUM_REPLICATION_FACTOR; +import static org.apache.cassandra.config.CassandraRelevantProperties.DISABLE_TCACTIVE_OPENSSL; +import static org.apache.cassandra.config.CassandraRelevantProperties.IO_NETTY_TRANSPORT_NONATIVE; +import static org.apache.cassandra.config.CassandraRelevantProperties.LOG4J2_DISABLE_JMX; +import static org.apache.cassandra.config.CassandraRelevantProperties.LOG4J2_DISABLE_JMX_LEGACY; +import static org.apache.cassandra.config.CassandraRelevantProperties.LOG4J_SHUTDOWN_HOOK_ENABLED; +import static org.apache.cassandra.config.CassandraRelevantProperties.ORG_APACHE_CASSANDRA_DISABLE_MBEAN_REGISTRATION; public class FuzzTestBase extends TestBaseImpl { @@ -32,9 +40,23 @@ public class FuzzTestBase extends TestBaseImpl public static void beforeClass() throws Throwable { TestBaseImpl.beforeClass(); - HarryHelper.init(); + init(); } + public static void init() + { + // setting both ways as changes between versions + LOG4J2_DISABLE_JMX.setBoolean(true); + LOG4J2_DISABLE_JMX_LEGACY.setBoolean(true); + LOG4J_SHUTDOWN_HOOK_ENABLED.setBoolean(false); + CASSANDRA_ALLOW_SIMPLE_STRATEGY.setBoolean(true); + CASSANDRA_MINIMUM_REPLICATION_FACTOR.setInt(0); + DISABLE_TCACTIVE_OPENSSL.setBoolean(true); + IO_NETTY_TRANSPORT_NONATIVE.setBoolean(true); + ORG_APACHE_CASSANDRA_DISABLE_MBEAN_REGISTRATION.setBoolean(true); + } + + @Override public Cluster.Builder builder() { return super.builder() diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/MetadataChangeSimulationTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/MetadataChangeSimulationTest.java index 7e2d1880a27b..414f74bbef47 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/MetadataChangeSimulationTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/MetadataChangeSimulationTest.java @@ -49,8 +49,7 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.harry.checker.ModelChecker; import org.apache.cassandra.harry.gen.EntropySource; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.sut.TokenPlacementModel.DCReplicas; +import org.apache.cassandra.harry.model.TokenPlacementModel; import org.apache.cassandra.locator.CMSPlacementStrategy; import org.apache.cassandra.locator.EndpointsForRange; import org.apache.cassandra.locator.InetAddressAndPort; @@ -73,12 +72,13 @@ import org.apache.cassandra.tcm.transformations.TriggerSnapshot; import static org.apache.cassandra.distributed.test.log.PlacementSimulator.SimulatedPlacements; -import static org.apache.cassandra.harry.sut.TokenPlacementModel.Node; -import static org.apache.cassandra.harry.sut.TokenPlacementModel.NtsReplicationFactor; -import static org.apache.cassandra.harry.sut.TokenPlacementModel.ReplicationFactor; -import static org.apache.cassandra.harry.sut.TokenPlacementModel.SimpleReplicationFactor; -import static org.apache.cassandra.harry.sut.TokenPlacementModel.nodeFactory; -import static org.apache.cassandra.harry.sut.TokenPlacementModel.nodeFactoryHumanReadable; +import static org.apache.cassandra.harry.model.TokenPlacementModel.DCReplicas; +import static org.apache.cassandra.harry.model.TokenPlacementModel.Node; +import static org.apache.cassandra.harry.model.TokenPlacementModel.NtsReplicationFactor; +import static org.apache.cassandra.harry.model.TokenPlacementModel.ReplicationFactor; +import static org.apache.cassandra.harry.model.TokenPlacementModel.SimpleReplicationFactor; +import static org.apache.cassandra.harry.model.TokenPlacementModel.nodeFactory; +import static org.apache.cassandra.harry.model.TokenPlacementModel.nodeFactoryHumanReadable; public class MetadataChangeSimulationTest extends CMSTestBase { diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/MetadataKeysTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/MetadataKeysTest.java index 04acd2369a61..acc013aff036 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/MetadataKeysTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/MetadataKeysTest.java @@ -33,7 +33,7 @@ import org.apache.cassandra.harry.gen.Generator; import org.apache.cassandra.harry.gen.Generators; import org.apache.cassandra.harry.gen.rng.JdkRandomEntropySource; -import org.apache.cassandra.harry.sut.TokenPlacementModel; +import org.apache.cassandra.harry.model.TokenPlacementModel; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.tcm.AtomicLongBackedProcessor; import org.apache.cassandra.tcm.ClusterMetadata; diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/ModelState.java b/test/distributed/org/apache/cassandra/distributed/test/log/ModelState.java index 3ad03fb6a0e1..0eb73b86940e 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/ModelState.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/ModelState.java @@ -28,8 +28,10 @@ import java.util.Set; import java.util.TreeMap; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.sut.TokenPlacementModel.DCReplicas; +import static org.apache.cassandra.harry.model.TokenPlacementModel.DCReplicas; +import static org.apache.cassandra.harry.model.TokenPlacementModel.Node; +import static org.apache.cassandra.harry.model.TokenPlacementModel.NodeFactory; +import static org.apache.cassandra.harry.model.TokenPlacementModel.ReplicationFactor; public class ModelState { @@ -40,17 +42,17 @@ public class ModelState public final int[] cancelled; public final int[] finished; public final int bootstrappingCount; - public final List<TokenPlacementModel.Node> currentNodes; - public final Map<String, List<TokenPlacementModel.Node>> nodesByDc; - public final List<TokenPlacementModel.Node> registeredNodes; - public final List<TokenPlacementModel.Node> leavingNodes; - public final List<TokenPlacementModel.Node> movingNodes; + public final List<Node> currentNodes; + public final Map<String, List<Node>> nodesByDc; + public final List<Node> registeredNodes; + public final List<Node> leavingNodes; + public final List<Node> movingNodes; public final List<SimulatedOperation> inFlightOperations; public final PlacementSimulator.SimulatedPlacements simulatedPlacements; - public final TokenPlacementModel.NodeFactory nodeFactory; + public final NodeFactory nodeFactory; - public static ModelState empty(TokenPlacementModel.NodeFactory nodeFactory, int maxClusterSize, int maxConcurrency) + public static ModelState empty(NodeFactory nodeFactory, int maxClusterSize, int maxConcurrency) { return new ModelState(maxClusterSize, maxConcurrency, 0, 0, @@ -61,11 +63,11 @@ public static ModelState empty(TokenPlacementModel.NodeFactory nodeFactory, int nodeFactory); } - public static Map<String, List<TokenPlacementModel.Node>> groupByDc(List<TokenPlacementModel.Node> nodes) + public static Map<String, List<Node>> groupByDc(List<Node> nodes) { // using treemap here since it is much easier to read/debug when it comes to that - Map<String, List<TokenPlacementModel.Node>> grouped = new TreeMap<>(); - for (TokenPlacementModel.Node node : nodes) + Map<String, List<Node>> grouped = new TreeMap<>(); + for (Node node : nodes) { grouped.computeIfAbsent(node.dc(), (k) -> new ArrayList<>()) .add(node); @@ -79,13 +81,13 @@ private ModelState(int maxClusterSize, int rejected, int[] cancelled, int[] finished, - List<TokenPlacementModel.Node> currentNodes, - List<TokenPlacementModel.Node> registeredNodes, - List<TokenPlacementModel.Node> leavingNodes, - List<TokenPlacementModel.Node> movingNodes, + List<Node> currentNodes, + List<Node> registeredNodes, + List<Node> leavingNodes, + List<Node> movingNodes, List<SimulatedOperation> operationStates, PlacementSimulator.SimulatedPlacements simulatedPlacements, - TokenPlacementModel.NodeFactory nodeFactory) + NodeFactory nodeFactory) { this.maxClusterSize = maxClusterSize; this.maxConcurrency = maxConcurrency; @@ -121,30 +123,30 @@ public boolean shouldBootstrap() return withinConcurrencyLimit() && bootstrappingCount + currentNodes.size() < maxClusterSize; } - public boolean shouldLeave(TokenPlacementModel.ReplicationFactor rf, Random rng) + public boolean shouldLeave(ReplicationFactor rf, Random rng) { return canRemove(rf) && rng.nextDouble() > 0.7; } - public boolean shouldMove(TokenPlacementModel.ReplicationFactor rf, Random rng) + public boolean shouldMove(ReplicationFactor rf, Random rng) { return canRemove(rf) && rng.nextDouble() > 0.7; } - public boolean shouldReplace(TokenPlacementModel.ReplicationFactor rf, Random rng) + public boolean shouldReplace(ReplicationFactor rf, Random rng) { return canRemove(rf) && rng.nextDouble() > 0.8; } - private boolean canRemove(TokenPlacementModel.ReplicationFactor rfs) + private boolean canRemove(ReplicationFactor rfs) { if (!withinConcurrencyLimit()) return false; for (Map.Entry<String, DCReplicas> e : rfs.asMap().entrySet()) { String dc = e.getKey(); int rf = e.getValue().totalCount; - List<TokenPlacementModel.Node> nodes = nodesByDc.get(dc); - Set<TokenPlacementModel.Node> nodesInDc = nodes == null ? new HashSet<>() : new HashSet<>(nodes); + List<Node> nodes = nodesByDc.get(dc); + Set<Node> nodesInDc = nodes == null ? new HashSet<>() : new HashSet<>(nodes); for (SimulatedOperation op : inFlightOperations) nodesInDc.removeAll(Arrays.asList(op.nodes)); if (nodesInDc.size() > rf) @@ -182,13 +184,13 @@ public static class Transformer // join/replace/leave/move private int[] cancelled; private int[] finished; - private List<TokenPlacementModel.Node> currentNodes; - private List<TokenPlacementModel.Node> registeredNodes; - private List<TokenPlacementModel.Node> leavingNodes; - private List<TokenPlacementModel.Node> movingNodes; + private List<Node> currentNodes; + private List<Node> registeredNodes; + private List<Node> leavingNodes; + private List<Node> movingNodes; private List<SimulatedOperation> operationStates; private PlacementSimulator.SimulatedPlacements simulatedPlacements; - private TokenPlacementModel.NodeFactory nodeFactory; + private NodeFactory nodeFactory; private Transformer(ModelState source) { @@ -257,26 +259,26 @@ public Transformer removeOperation(SimulatedOperation operation) return this; } - public Transformer withJoined(TokenPlacementModel.Node node) + public Transformer withJoined(Node node) { addToCluster(node); finished[0]++; return this; } - public Transformer recycleRejected(TokenPlacementModel.Node node) + public Transformer recycleRejected(Node node) { registeredNodes = new ArrayList<>(registeredNodes); registeredNodes.add(node); return this; } - public Transformer withMoved(TokenPlacementModel.Node movingNode, TokenPlacementModel.Node movedTo) + public Transformer withMoved(Node movingNode, Node movedTo) { assert currentNodes.contains(movingNode) : movingNode; - List<TokenPlacementModel.Node> tmp = currentNodes; + List<Node> tmp = currentNodes; currentNodes = new ArrayList<>(); - for (TokenPlacementModel.Node n : tmp) + for (Node n : tmp) { if (n.idx() == movingNode.idx()) currentNodes.add(movedTo); @@ -291,14 +293,14 @@ public Transformer withMoved(TokenPlacementModel.Node movingNode, TokenPlacement return this; } - private void addToCluster(TokenPlacementModel.Node node) + private void addToCluster(Node node) { // called during both join and replacement currentNodes = new ArrayList<>(currentNodes); currentNodes.add(node); } - public Transformer markMoving(TokenPlacementModel.Node moving) + public Transformer markMoving(Node moving) { assert currentNodes.contains(moving); movingNodes = new ArrayList<>(movingNodes); @@ -306,7 +308,7 @@ public Transformer markMoving(TokenPlacementModel.Node moving) return this; } - public Transformer markLeaving(TokenPlacementModel.Node leaving) + public Transformer markLeaving(Node leaving) { assert currentNodes.contains(leaving); leavingNodes = new ArrayList<>(leavingNodes); @@ -314,7 +316,7 @@ public Transformer markLeaving(TokenPlacementModel.Node leaving) return this; } - public Transformer withLeft(TokenPlacementModel.Node node) + public Transformer withLeft(Node node) { assert currentNodes.contains(node); // for now... assassinate may change this assertion @@ -324,7 +326,7 @@ public Transformer withLeft(TokenPlacementModel.Node node) return this; } - private void removeFromCluster(TokenPlacementModel.Node node) + private void removeFromCluster(Node node) { // called during both decommission and replacement currentNodes = new ArrayList<>(currentNodes); @@ -333,7 +335,7 @@ private void removeFromCluster(TokenPlacementModel.Node node) leavingNodes.remove(node); } - public Transformer withReplaced(TokenPlacementModel.Node oldNode, TokenPlacementModel.Node newNode) + public Transformer withReplaced(Node oldNode, Node newNode) { addToCluster(newNode); removeFromCluster(oldNode); diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/NTSSimulationTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/NTSSimulationTest.java index 6e055b257cfd..c0ce61226ba5 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/NTSSimulationTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/NTSSimulationTest.java @@ -29,7 +29,7 @@ import org.apache.cassandra.dht.Murmur3Partitioner; import static org.apache.cassandra.distributed.test.log.MetadataChangeSimulationTest.simulate; -import static org.apache.cassandra.harry.sut.TokenPlacementModel.NtsReplicationFactor; +import static org.apache.cassandra.harry.model.TokenPlacementModel.NtsReplicationFactor; public class NTSSimulationTest extends CMSTestBase { diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/OperationalEquivalenceTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/OperationalEquivalenceTest.java index 85519006cfd6..2f3e9f81ef9a 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/OperationalEquivalenceTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/OperationalEquivalenceTest.java @@ -30,7 +30,6 @@ import org.junit.Test; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.harry.sut.TokenPlacementModel; import org.apache.cassandra.locator.EndpointsForRange; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.tcm.AtomicLongBackedProcessor; @@ -44,6 +43,13 @@ import org.apache.cassandra.tcm.transformations.Register; import org.apache.cassandra.tcm.transformations.UnsafeJoin; +import static org.apache.cassandra.harry.model.TokenPlacementModel.Node; +import static org.apache.cassandra.harry.model.TokenPlacementModel.NodeFactory; +import static org.apache.cassandra.harry.model.TokenPlacementModel.NtsReplicationFactor; +import static org.apache.cassandra.harry.model.TokenPlacementModel.ReplicationFactor; +import static org.apache.cassandra.harry.model.TokenPlacementModel.SimpleReplicationFactor; +import static org.apache.cassandra.harry.model.TokenPlacementModel.nodeFactory; + /** * Compare different operations, and make sure that executing operations such as move, bootstrap, etc., * is consistent with bootstrapping nodes with equivalent token ownership. Useful for testing operations @@ -61,30 +67,30 @@ public class OperationalEquivalenceTest extends CMSTestBase @Test public void testMove() throws Exception { - testMove(new TokenPlacementModel.SimpleReplicationFactor(2)); - testMove(new TokenPlacementModel.SimpleReplicationFactor(3)); - testMove(new TokenPlacementModel.SimpleReplicationFactor(5)); - - testMove(new TokenPlacementModel.NtsReplicationFactor(1, 2)); - testMove(new TokenPlacementModel.NtsReplicationFactor(1, 3)); - testMove(new TokenPlacementModel.NtsReplicationFactor(1, 5)); - - testMove(new TokenPlacementModel.NtsReplicationFactor(3, 2)); - testMove(new TokenPlacementModel.NtsReplicationFactor(3, 3)); - testMove(new TokenPlacementModel.NtsReplicationFactor(3, 5)); - - testMove(new TokenPlacementModel.SimpleReplicationFactor(3, 1)); - testMove(new TokenPlacementModel.SimpleReplicationFactor(3, 2)); - testMove(new TokenPlacementModel.NtsReplicationFactor(3, 3, 1)); - testMove(new TokenPlacementModel.NtsReplicationFactor(3, 5, 2)); + testMove(new SimpleReplicationFactor(2)); + testMove(new SimpleReplicationFactor(3)); + testMove(new SimpleReplicationFactor(5)); + + testMove(new NtsReplicationFactor(1, 2)); + testMove(new NtsReplicationFactor(1, 3)); + testMove(new NtsReplicationFactor(1, 5)); + + testMove(new NtsReplicationFactor(3, 2)); + testMove(new NtsReplicationFactor(3, 3)); + testMove(new NtsReplicationFactor(3, 5)); + + testMove(new SimpleReplicationFactor(3, 1)); + testMove(new SimpleReplicationFactor(3, 2)); + testMove(new NtsReplicationFactor(3, 3, 1)); + testMove(new NtsReplicationFactor(3, 5, 2)); } - public void testMove(TokenPlacementModel.ReplicationFactor rf) throws Exception + public void testMove(ReplicationFactor rf) throws Exception { - TokenPlacementModel.NodeFactory nodeFactory = TokenPlacementModel.nodeFactory(); + NodeFactory nodeFactory = nodeFactory(); ClusterMetadata withMove = null; - List<TokenPlacementModel.Node> equivalentNodes = new ArrayList<>(); + List<Node> equivalentNodes = new ArrayList<>(); int nodes = 30; try (CMSSut sut = new CMSSut(AtomicLongBackedProcessor::new, false, rf)) { @@ -92,14 +98,14 @@ public void testMove(TokenPlacementModel.ReplicationFactor rf) throws Exception for (int i = 0; i < nodes; i++) { int dc = toDc(i, rf); - TokenPlacementModel.Node node = nodeFactory.make(counter.incrementAndGet(), dc, 1); + Node node = nodeFactory.make(counter.incrementAndGet(), dc, 1); sut.service.commit(new Register(new NodeAddresses(node.addr()), new Location(node.dc(), node.rack()), NodeVersion.CURRENT)); sut.service.commit(new UnsafeJoin(node.nodeId(), Collections.singleton(node.longToken()), sut.service.placementProvider())); equivalentNodes.add(node); } - TokenPlacementModel.Node toMove = equivalentNodes.get(rng.nextInt(equivalentNodes.size())); - TokenPlacementModel.Node moved = toMove.withNewToken(); + Node toMove = equivalentNodes.get(rng.nextInt(equivalentNodes.size())); + Node moved = toMove.withNewToken(); equivalentNodes.set(equivalentNodes.indexOf(toMove), moved); Move plan = SimulatedOperation.prepareMove(sut, toMove, moved.longToken()).get(); @@ -114,12 +120,12 @@ public void testMove(TokenPlacementModel.ReplicationFactor rf) throws Exception withMove.placements); } - private static ClusterMetadata simulateAndCompare(TokenPlacementModel.ReplicationFactor rf, List<TokenPlacementModel.Node> nodes) throws Exception + private static ClusterMetadata simulateAndCompare(ReplicationFactor rf, List<Node> nodes) throws Exception { Collections.shuffle(nodes, rng); try (CMSSut sut = new CMSSut(AtomicLongBackedProcessor::new, false, rf)) { - for (TokenPlacementModel.Node node : nodes) + for (Node node : nodes) { sut.service.commit(new Register(new NodeAddresses(node.addr()), new Location(node.dc(), node.rack()), NodeVersion.CURRENT)); sut.service.commit(new UnsafeJoin(node.nodeId(), Collections.singleton(node.longToken()), sut.service.placementProvider())); @@ -154,7 +160,7 @@ public static List<Replica> toReplicas(EndpointsForRange ep) { return ep.stream().sorted(Replica::compareTo).collect(Collectors.toList()); } - private static int toDc(int i, TokenPlacementModel.ReplicationFactor rf) + private static int toDc(int i, ReplicationFactor rf) { return (i % rf.dcs()) + 1; } diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/PlacementSimulator.java b/test/distributed/org/apache/cassandra/distributed/test/log/PlacementSimulator.java index d9ca33a75980..6bd3cd7e250e 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/PlacementSimulator.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/PlacementSimulator.java @@ -30,13 +30,13 @@ import java.util.function.Predicate; import java.util.stream.Collectors; -import org.apache.cassandra.harry.sut.TokenPlacementModel.Replica; import org.junit.Assert; -import static org.apache.cassandra.harry.sut.TokenPlacementModel.Node; -import static org.apache.cassandra.harry.sut.TokenPlacementModel.Range; -import static org.apache.cassandra.harry.sut.TokenPlacementModel.ReplicationFactor; -import static org.apache.cassandra.harry.sut.TokenPlacementModel.toRanges; +import static org.apache.cassandra.harry.model.TokenPlacementModel.Node; +import static org.apache.cassandra.harry.model.TokenPlacementModel.Range; +import static org.apache.cassandra.harry.model.TokenPlacementModel.Replica; +import static org.apache.cassandra.harry.model.TokenPlacementModel.ReplicationFactor; +import static org.apache.cassandra.harry.model.TokenPlacementModel.toRanges; /** * A small class that helps to avoid doing mental arithmetics on ranges. diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/PlacementSimulatorTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/PlacementSimulatorTest.java index 2ea853bf01e9..e1013b0509e0 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/PlacementSimulatorTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/PlacementSimulatorTest.java @@ -32,8 +32,6 @@ import org.junit.Test; import org.apache.cassandra.harry.checker.ModelChecker; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.sut.TokenPlacementModel.Replica; import static org.apache.cassandra.distributed.test.log.PlacementSimulator.SimulatedPlacements; import static org.apache.cassandra.distributed.test.log.PlacementSimulator.Transformations; @@ -46,11 +44,14 @@ import static org.apache.cassandra.distributed.test.log.PlacementSimulator.replace; import static org.apache.cassandra.distributed.test.log.PlacementSimulator.split; import static org.apache.cassandra.distributed.test.log.PlacementSimulator.superset; -import static org.apache.cassandra.harry.sut.TokenPlacementModel.Node; -import static org.apache.cassandra.harry.sut.TokenPlacementModel.NodeFactory; -import static org.apache.cassandra.harry.sut.TokenPlacementModel.Range; -import static org.apache.cassandra.harry.sut.TokenPlacementModel.ReplicationFactor; -import static org.apache.cassandra.harry.sut.TokenPlacementModel.SimpleReplicationFactor; +import static org.apache.cassandra.harry.model.TokenPlacementModel.Node; +import static org.apache.cassandra.harry.model.TokenPlacementModel.NodeFactory; +import static org.apache.cassandra.harry.model.TokenPlacementModel.Range; +import static org.apache.cassandra.harry.model.TokenPlacementModel.Replica; +import static org.apache.cassandra.harry.model.TokenPlacementModel.ReplicationFactor; +import static org.apache.cassandra.harry.model.TokenPlacementModel.SimpleReplicationFactor; +import static org.apache.cassandra.harry.model.TokenPlacementModel.nodeFactory; +import static org.apache.cassandra.harry.model.TokenPlacementModel.nodeFactoryHumanReadable; import static org.junit.Assert.assertTrue; public class PlacementSimulatorTest @@ -76,7 +77,7 @@ public void testMove(long minToken) public void testMove(long t1, long t2, long t3, long t4, long newToken, ReplicationFactor rf) { - NodeFactory factory = TokenPlacementModel.nodeFactory(); + NodeFactory factory = nodeFactory(); Node movingNode = factory.make(1, 1, 1).overrideToken(t1); List<Node> orig = Arrays.asList(movingNode, factory.make(2, 1, 1).overrideToken(t2), @@ -135,7 +136,7 @@ public void testBootstrap(long newToken) public void testBootstrap(long t1, long t2, long t3, long t4, long newToken, ReplicationFactor rf) { - NodeFactory factory = TokenPlacementModel.nodeFactory(); + NodeFactory factory = nodeFactory(); List<Node> orig = Arrays.asList(factory.make(1, 1, 1).overrideToken(t1), factory.make(2, 1, 1).overrideToken(t2), factory.make(3, 1, 1).overrideToken(t3), @@ -195,7 +196,7 @@ public void testDecommission(long minToken) public void testDecommission(long t1, long t2, long t3, long t4, long t5, ReplicationFactor rf) { - NodeFactory factory = TokenPlacementModel.nodeFactory(); + NodeFactory factory = nodeFactory(); Node leavingNode = factory.make(1, 1, 1).overrideToken(t1); List<Node> orig = Arrays.asList(leavingNode, factory.make(2, 1, 1).overrideToken(t2), @@ -263,7 +264,7 @@ public void simulate() throws Throwable public void simulate(ReplicationFactor rf) throws Throwable { - NodeFactory factory = TokenPlacementModel.nodeFactory(); + NodeFactory factory = nodeFactory(); List<Node> orig = Collections.singletonList(factory.make(1, 1, 1)); ModelChecker<SimulatedPlacements, SUTState> modelChecker = new ModelChecker<>(); @@ -325,7 +326,7 @@ public void revertPartialBootstrap() throws Throwable for (int n : new int[]{ 2, 3, 5 }) { ReplicationFactor rf = new SimpleReplicationFactor(n); - NodeFactory factory = TokenPlacementModel.nodeFactoryHumanReadable(); + NodeFactory factory = nodeFactoryHumanReadable(); List<Node> nodes = new ArrayList<>(10); for (int i = 1; i <= 10; i++) nodes.add(factory.make(i, 1, 1)); @@ -343,7 +344,7 @@ public void revertPartialLeave() for (int n : new int[]{ 2, 3, 5 }) { ReplicationFactor rf = new SimpleReplicationFactor(n); - NodeFactory factory = TokenPlacementModel.nodeFactoryHumanReadable(); + NodeFactory factory = nodeFactoryHumanReadable(); List<Node> nodes = new ArrayList<>(10); for (int i = 1; i <= 10; i++) nodes.add(factory.make(i, 1, 1)); @@ -360,7 +361,7 @@ public void revertPartialReplacement() for (int n : new int[]{ 2, 3, 5 }) { ReplicationFactor rf = new SimpleReplicationFactor(n); - NodeFactory factory = TokenPlacementModel.nodeFactoryHumanReadable(); + NodeFactory factory = nodeFactoryHumanReadable(); List<Node> nodes = new ArrayList<>(10); for (int i = 1; i <= 10; i++) nodes.add(factory.make(i, 1, 1)); diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/ResumableStartupTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/ResumableStartupTest.java index c1440a13427c..07ec054d54ae 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/ResumableStartupTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/ResumableStartupTest.java @@ -19,46 +19,54 @@ package org.apache.cassandra.distributed.test.log; import java.io.IOException; +import java.util.Iterator; import java.util.concurrent.Callable; +import java.util.concurrent.atomic.AtomicReference; import org.junit.Assert; import org.junit.Test; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.sut.injvm.InJVMTokenAwareVisitExecutor; -import org.apache.cassandra.harry.sut.injvm.InJvmSut; -import org.apache.cassandra.harry.sut.injvm.QuiescentLocalStateChecker; -import org.apache.cassandra.harry.visitors.GeneratingVisitor; -import org.apache.cassandra.harry.visitors.MutatingRowVisitor; -import org.apache.cassandra.harry.visitors.Visitor; import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.Constants; -import org.apache.cassandra.distributed.api.*; -import org.apache.cassandra.harry.HarryHelper; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.api.IInstanceConfig; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.api.TokenSupplier; import org.apache.cassandra.distributed.shared.ClusterUtils; import org.apache.cassandra.distributed.shared.NetworkTopology; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.dsl.HistoryBuilder; +import org.apache.cassandra.harry.execution.DataTracker; +import org.apache.cassandra.harry.execution.RingAwareInJvmDTestVisitExecutor; +import org.apache.cassandra.harry.gen.Generator; +import org.apache.cassandra.harry.gen.Generators; +import org.apache.cassandra.harry.gen.SchemaGenerators; +import org.apache.cassandra.harry.model.QuiescentChecker; +import org.apache.cassandra.harry.model.TokenPlacementModel; +import org.apache.cassandra.harry.op.Visit; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.transformations.PrepareJoin; -import org.apache.cassandra.service.StorageService; import static org.apache.cassandra.distributed.action.GossipHelper.withProperty; import static org.apache.cassandra.distributed.shared.ClusterUtils.getClusterMetadataVersion; import static org.apache.cassandra.distributed.shared.ClusterUtils.getSequenceAfterCommit; +import static org.apache.cassandra.harry.checker.TestHelper.withRandom; public class ResumableStartupTest extends FuzzTestBase { + private static final String KS = "resumable_startup_test"; private static int WRITES = 500; @Test public void bootstrapWithDeferredJoinTest() throws Throwable { + Generator<SchemaSpec> schemaGen = SchemaGenerators.schemaSpecGen(KS, "bootstrap_with_deferred_join", 1000); try (Cluster cluster = builder().withNodes(1) .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(4)) .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(4, "dc0", "rack0")) @@ -66,89 +74,116 @@ public void bootstrapWithDeferredJoinTest() throws Throwable .createWithoutStarting()) { IInvokableInstance cmsInstance = cluster.get(1); - Configuration.ConfigurationBuilder configBuilder = HarryHelper.defaultConfiguration() - .setSUT(() -> new InJvmSut(cluster)); - Run run = configBuilder.build().createRun(); - cmsInstance.config().set("auto_bootstrap", true); cmsInstance.startup(); - cluster.coordinator(1).execute("CREATE KEYSPACE " + run.schemaSpec.keyspace + - " WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 2};", - ConsistencyLevel.ALL); - cluster.coordinator(1).execute(run.schemaSpec.compile().cql(), ConsistencyLevel.ALL); - ClusterUtils.waitForCMSToQuiesce(cluster, cluster.get(1)); - - TokenPlacementModel.ReplicationFactor rf = new TokenPlacementModel.SimpleReplicationFactor(2); - Visitor visitor = new GeneratingVisitor(run, new InJVMTokenAwareVisitExecutor(run, MutatingRowVisitor::new, - SystemUnderTest.ConsistencyLevel.NODE_LOCAL, - rf)); - for (int i = 0; i < WRITES; i++) - visitor.visit(); - - IInstanceConfig config = cluster.newInstanceConfig() - .set("auto_bootstrap", true) - .set(Constants.KEY_DTEST_FULL_STARTUP, true); - IInvokableInstance newInstance = cluster.bootstrap(config); - - withProperty(CassandraRelevantProperties.TEST_WRITE_SURVEY, true, newInstance::startup); - - // Write with ALL, replicate via pending range mechanism - visitor = new GeneratingVisitor(run, new InJVMTokenAwareVisitExecutor(run, - MutatingRowVisitor::new, - SystemUnderTest.ConsistencyLevel.ONE, - rf)); - - for (int i = 0; i < WRITES; i++) - visitor.visit(); - - Epoch currentEpoch = getClusterMetadataVersion(cmsInstance); - // Quick check that schema changes are possible with nodes in write survey mode (i.e. with ranges locked) - cluster.coordinator(1).execute(String.format("ALTER TABLE %s.%s WITH comment = 'Schema alterations which do not affect placements should not be restricted by in flight operations';", run.schemaSpec.keyspace, run.schemaSpec.table), - ConsistencyLevel.ALL); - - final String newAddress = ClusterUtils.getBroadcastAddressHostWithPortString(newInstance); - final String keyspace = run.schemaSpec.keyspace; - boolean newReplicaInCorrectState = cluster.get(1).callOnInstance(() -> { - ClusterMetadata metadata = ClusterMetadata.current(); - KeyspaceMetadata ksm = metadata.schema.getKeyspaceMetadata(keyspace); - boolean isWriteReplica = false; - boolean isReadReplica = false; - for (InetAddressAndPort readReplica : metadata.placements.get(ksm.params.replication).reads.byEndpoint().keySet()) - { - if (readReplica.getHostAddressAndPort().equals(newAddress)) - isReadReplica = true; - } - for (InetAddressAndPort writeReplica : metadata.placements.get(ksm.params.replication).writes.byEndpoint().keySet()) - { - if (writeReplica.getHostAddressAndPort().equals(newAddress)) - isWriteReplica = true; - } - return (isWriteReplica && !isReadReplica); - }); - Assert.assertTrue("Expected new instance to be a write replica only", newReplicaInCorrectState); - - Callable<Epoch> finishedBootstrap = getSequenceAfterCommit(cmsInstance, (e, r) -> e instanceof PrepareJoin.FinishJoin && r.isSuccess()); - newInstance.runOnInstance(() -> { - try - { - StorageService.instance.joinRing(); - } - catch (IOException e) - { - throw new RuntimeException("Error joining ring", e); - } - }); - Epoch next = finishedBootstrap.call(); - Assert.assertEquals(String.format("Expected epoch after schema change, mid join & finish join to be %s, but was %s", - next.getEpoch(), currentEpoch.getEpoch() + 3), - next.getEpoch(), currentEpoch.getEpoch() + 3); - - for (int i = 0; i < WRITES; i++) - visitor.visit(); + withRandom(rng -> { + SchemaSpec schema = schemaGen.generate(rng); + Generators.TrackingGenerator<Integer> pkGen = Generators.tracking(Generators.int32(0, Math.min(schema.valueGenerators.pkPopulation(), 1000))); + Generator<Integer> ckGen = Generators.int32(0, Math.min(schema.valueGenerators.ckPopulation(), 1000)); + + HistoryBuilder history = new HistoryBuilder(schema.valueGenerators); + history.customThrowing(() -> { + cluster.schemaChange(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 2};", schema.keyspace)); + cluster.schemaChange(schema.compile()); + ClusterUtils.waitForCMSToQuiesce(cluster, cluster.get(1)); + }, "Setup"); + + Runnable writeAndValidate = () -> { + for (int i = 0; i < WRITES; i++) + history.insert(pkGen.generate(rng), ckGen.generate(rng)); + + for (int pk : pkGen.generated()) + history.selectPartition(pk); + }; + writeAndValidate.run(); + + // First write with ONE, as we only have 1 node + TokenPlacementModel.ReplicationFactor rf = new TokenPlacementModel.SimpleReplicationFactor(1); + DataTracker tracker = new DataTracker.SequentialDataTracker(); + QuiescentChecker checker = new QuiescentChecker(schema.valueGenerators, tracker, history); + + RingAwareInJvmDTestVisitExecutor executor; + // RF is ONE here since we have no pending nodes + executor = RingAwareInJvmDTestVisitExecutor.builder() + .replicationFactor(rf) + .consistencyLevel(ConsistencyLevel.ONE) + .build(schema, tracker, checker, cluster); + Iterator<Visit> iterator = history.iterator(); + while (iterator.hasNext()) + executor.execute(iterator.next()); + + AtomicReference<IInvokableInstance> newInstance = new AtomicReference<>(); + history.customThrowing(() -> { + IInstanceConfig config = cluster.newInstanceConfig() + .set("auto_bootstrap", true) + .set(Constants.KEY_DTEST_FULL_STARTUP, true); + newInstance.set(cluster.bootstrap(config)); + + withProperty(CassandraRelevantProperties.TEST_WRITE_SURVEY, true, newInstance.get()::startup); + }, "Bootstrap"); + + writeAndValidate.run(); + + rf = new TokenPlacementModel.SimpleReplicationFactor(2); + // RF is ONE here since we have 1 regular and 1 pending node, but want to check for RF2 + executor = RingAwareInJvmDTestVisitExecutor.builder() + .replicationFactor(rf) + .consistencyLevel(ConsistencyLevel.ONE) + .build(schema, tracker, checker, cluster); + while (iterator.hasNext()) + executor.execute(iterator.next()); + + history.customThrowing(() -> { + Epoch currentEpoch = getClusterMetadataVersion(cmsInstance); + // Quick check that schema changes are possible with nodes in write survey mode (i.e. with ranges locked) + cluster.schemaChange(String.format("ALTER TABLE %s.%s WITH comment = 'Schema alterations which do not affect placements should not be restricted by in flight operations';", schema.keyspace, schema.table)); + + final String newAddress = ClusterUtils.getBroadcastAddressHostWithPortString(newInstance.get()); + final String keyspace = schema.keyspace; + boolean newReplicaInCorrectState = cluster.get(1).callOnInstance(() -> { + ClusterMetadata metadata = ClusterMetadata.current(); + KeyspaceMetadata ksm = metadata.schema.getKeyspaceMetadata(keyspace); + boolean isWriteReplica = false; + boolean isReadReplica = false; + for (InetAddressAndPort readReplica : metadata.placements.get(ksm.params.replication).reads.byEndpoint().keySet()) + { + if (readReplica.getHostAddressAndPort().equals(newAddress)) + isReadReplica = true; + } + for (InetAddressAndPort writeReplica : metadata.placements.get(ksm.params.replication).writes.byEndpoint().keySet()) + { + if (writeReplica.getHostAddressAndPort().equals(newAddress)) + isWriteReplica = true; + } + return (isWriteReplica && !isReadReplica); + }); + Assert.assertTrue("Expected new instance to be a write replica only", newReplicaInCorrectState); + + Callable<Epoch> finishedBootstrap = getSequenceAfterCommit(cmsInstance, (e, r) -> e instanceof PrepareJoin.FinishJoin && r.isSuccess()); + newInstance.get().runOnInstance(() -> { + try + { + StorageService.instance.joinRing(); + } + catch (IOException e) + { + throw new RuntimeException("Error joining ring", e); + } + }); + Epoch next = finishedBootstrap.call(); + Assert.assertEquals(String.format("Expected epoch after schema change, mid join & finish join to be %s, but was %s", + next.getEpoch(), currentEpoch.getEpoch() + 3), + next.getEpoch(), currentEpoch.getEpoch() + 3); + + }, "Finish bootstrap"); + + writeAndValidate.run(); + + while (iterator.hasNext()) + executor.execute(iterator.next()); - QuiescentLocalStateChecker model = new QuiescentLocalStateChecker(run, new TokenPlacementModel.SimpleReplicationFactor(3)); - model.validateAll(); + }); } } -} +} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/SimpleStrategySimulationTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/SimpleStrategySimulationTest.java index 0aaf8737ebae..b8f98f68a004 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/SimpleStrategySimulationTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/SimpleStrategySimulationTest.java @@ -29,7 +29,7 @@ import org.apache.cassandra.dht.Murmur3Partitioner; import static org.apache.cassandra.distributed.test.log.MetadataChangeSimulationTest.simulate; -import static org.apache.cassandra.harry.sut.TokenPlacementModel.SimpleReplicationFactor; +import static org.apache.cassandra.harry.model.TokenPlacementModel.SimpleReplicationFactor; public class SimpleStrategySimulationTest extends CMSTestBase { diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/SimulatedOperation.java b/test/distributed/org/apache/cassandra/distributed/test/log/SimulatedOperation.java index c8c2315529c7..77638319331d 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/SimulatedOperation.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/SimulatedOperation.java @@ -53,7 +53,7 @@ import static org.apache.cassandra.dht.Murmur3Partitioner.LongToken; import static org.apache.cassandra.distributed.test.log.CMSTestBase.CMSSut; -import static org.apache.cassandra.harry.sut.TokenPlacementModel.*; +import static org.apache.cassandra.harry.model.TokenPlacementModel.Node; public abstract class SimulatedOperation { diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/SystemKeyspaceStorageTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/SystemKeyspaceStorageTest.java index e8bf239f56a8..2e131bcf497b 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/SystemKeyspaceStorageTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/SystemKeyspaceStorageTest.java @@ -27,7 +27,7 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.distributed.test.ExecUtil; -import org.apache.cassandra.harry.sut.TokenPlacementModel; +import org.apache.cassandra.harry.model.TokenPlacementModel; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.log.Entry; diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeHarryTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeHarryTest.java index e165a0d16e2a..64810654d5cf 100644 --- a/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeHarryTest.java +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeHarryTest.java @@ -18,115 +18,135 @@ package org.apache.cassandra.distributed.upgrade; -import java.util.Collections; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; +import java.time.Duration; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import com.google.common.util.concurrent.Uninterruptibles; - -import org.apache.cassandra.harry.HarryHelper; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.ddl.SchemaSpec; - -import org.apache.cassandra.harry.runner.FlaggedRunner; -import org.apache.cassandra.harry.sut.injvm.ClusterState; -import org.apache.cassandra.harry.sut.injvm.ExistingClusterSUT; - -import org.apache.cassandra.harry.visitors.MutatingVisitor; -import org.apache.cassandra.harry.visitors.QueryLogger; -import org.apache.cassandra.harry.visitors.RandomPartitionValidator; - import org.junit.Test; +import org.apache.cassandra.concurrent.Interruptible; import org.apache.cassandra.distributed.Constants; +import org.apache.cassandra.distributed.api.ConsistencyLevel; import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.dsl.HistoryBuilder; +import org.apache.cassandra.harry.dsl.ReplayingHistoryBuilder; +import org.apache.cassandra.harry.execution.InJvmDTestVisitExecutor; +import org.apache.cassandra.harry.gen.Generator; +import org.apache.cassandra.harry.gen.Generators; import org.apache.cassandra.schema.SchemaConstants; -import org.apache.cassandra.utils.concurrent.CountDownLatch; - +import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException; import static java.util.Arrays.asList; import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory; -import static org.apache.cassandra.harry.core.Configuration.VisitorPoolConfiguration.pool; -import static org.apache.cassandra.harry.ddl.ColumnSpec.asciiType; -import static org.apache.cassandra.harry.ddl.ColumnSpec.int64Type; -import static org.apache.cassandra.harry.ddl.ColumnSpec.ck; -import static org.apache.cassandra.harry.ddl.ColumnSpec.pk; -import static org.apache.cassandra.harry.ddl.ColumnSpec.regularColumn; -import static org.apache.cassandra.harry.ddl.ColumnSpec.staticColumn; +import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.SimulatorSafe.UNSAFE; +import static org.apache.cassandra.harry.ColumnSpec.asciiType; +import static org.apache.cassandra.harry.ColumnSpec.ck; +import static org.apache.cassandra.harry.ColumnSpec.int64Type; +import static org.apache.cassandra.harry.ColumnSpec.pk; +import static org.apache.cassandra.harry.ColumnSpec.regularColumn; +import static org.apache.cassandra.harry.ColumnSpec.staticColumn; +import static org.apache.cassandra.harry.checker.TestHelper.withRandom; import static org.apache.cassandra.tcm.log.SystemKeyspaceStorage.NAME; import static org.junit.Assert.assertEquals; public class ClusterMetadataUpgradeHarryTest extends UpgradeTestBase { @Test - public void simpleUpgradeTest() throws Throwable + public void simpleUpgradeTest() { - ExecutorService es = executorFactory().pooled("harry", 1); + AtomicReference<Interruptible> executor = new AtomicReference<>(); + AtomicLong loops = new AtomicLong(0); Listener listener = new Listener(); - CountDownLatch stopLatch = CountDownLatch.newCountDownLatch(1); - AtomicReference<Future<?>> harryRunner = new AtomicReference<>(); - new UpgradeTestBase.TestCase() - .nodes(3) - .nodesToUpgrade(1, 2, 3) - .withConfig((cfg) -> cfg.with(Feature.NETWORK, Feature.GOSSIP) - .set(Constants.KEY_DTEST_FULL_STARTUP, true)) - .upgradesToCurrentFrom(v41) - .withUpgradeListener(listener) - .setup((cluster) -> { - SchemaSpec schema = new SchemaSpec("harry", "test_table", - asList(pk("pk1", asciiType), pk("pk2", int64Type)), - asList(ck("ck1", asciiType), ck("ck2", int64Type)), - asList(regularColumn("regular1", asciiType), regularColumn("regular2", int64Type)), - asList(staticColumn("static1", asciiType), staticColumn("static2", int64Type))); - - Configuration config = HarryHelper.defaultConfiguration() - .setKeyspaceDdl(String.format("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': %d};", schema.keyspace, 3)) - .setSchemaProvider(new Configuration.FixedSchemaProviderConfiguration(schema)) - .setDataTracker(new Configuration.LockingDataTrackerConfiguration(-1l, -1l, Collections.emptyList())) - .setSUT(new ExistingClusterSUT(cluster, listener)) - .build(); - - Future<?> f = es.submit(() -> { + Runnable awaitHarryProgress = () -> { + long startingLoopCount = loops.get(); + long deadline = startingLoopCount + 100; + long nowNanos = System.nanoTime(); + boolean matched = false; + for (int i = 0; i < 20 && !(matched = loops.get() >= deadline); i++) + { try { - new FlaggedRunner(config.createRun(), - config, - asList(pool("Writer", 1, MutatingVisitor::new), - pool("Reader", 1, (run) -> new RandomPartitionValidator(run, new Configuration.QuiescentCheckerConfig(), QueryLogger.NO_OP.NO_OP))), - stopLatch).run(); + TimeUnit.MILLISECONDS.sleep(500); } - catch (Throwable e) + catch (InterruptedException e) { - throw new RuntimeException(e); + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e); } - }); - harryRunner.set(f); - Uninterruptibles.sleepUninterruptibly(10, TimeUnit.SECONDS); - }) - .runAfterNodeUpgrade((cluster, node) -> { - Uninterruptibles.sleepUninterruptibly(10, TimeUnit.SECONDS); // make sure harry executes in mixed mode - }) - .runAfterClusterUpgrade((cluster) -> { - - // make sure we haven't persisted any events; - cluster.stream().forEach((i) -> { - Object[][] res = i.executeInternal(String.format("select * from %s.%s", SchemaConstants.SYSTEM_KEYSPACE_NAME, NAME)); - assertEquals(0, res.length); - }); - - cluster.get(1).nodetoolResult("cms", "initialize").asserts().success(); - cluster.get(1).nodetoolResult("cms", "reconfigure", "3").asserts().success(); - cluster.schemaChange(withKeyspace("create table %s.xyz (id int primary key)")); - stopLatch.decrement(); - harryRunner.get().get(); - }).run(); + } + if (!matched) + throw new AssertionError("Harry did not make enough progress within " + Duration.ofNanos(System.nanoTime() - nowNanos) + "; starting loops " + startingLoopCount + ", ending loops " + loops.get()); + }; + withRandom(rng -> { + new TestCase() + .nodes(3) + .nodesToUpgrade(1, 2, 3) + .withConfig((cfg) -> cfg.with(Feature.NETWORK, Feature.GOSSIP) + .set(Constants.KEY_DTEST_FULL_STARTUP, true)) + .upgradesToCurrentFrom(v41) + .withUpgradeListener(listener) + .setup((cluster) -> { + SchemaSpec schema = new SchemaSpec(rng.next(), + 10_000, + "harry", "test_table", + asList(pk("pk1", asciiType), pk("pk2", int64Type)), + asList(ck("ck1", asciiType, false), ck("ck2", int64Type, false)), + asList(regularColumn("regular1", asciiType), regularColumn("regular2", int64Type)), + asList(staticColumn("static1", asciiType), staticColumn("static2", int64Type))); + cluster.schemaChange(String.format("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': %d};", schema.keyspace, 3)); + cluster.schemaChange(schema.compile()); + + HistoryBuilder history = new ReplayingHistoryBuilder(schema.valueGenerators, + hb -> InJvmDTestVisitExecutor.builder() + .retryPolicy(retry -> true) + .nodeSelector(lts -> { + while (true) + { + int node = rng.nextInt(0, cluster.size()) + 1; + if (listener.isDown(node)) + continue; + return node; + } + }) + .consistencyLevel(ConsistencyLevel.QUORUM) + .build(schema, hb, cluster)); + + Generator<Integer> pkIdxGen = Generators.int32(0, Math.min(10_000, schema.valueGenerators.ckPopulation())); + + executor.set(executorFactory().infiniteLoop("R/W Worload", + () -> { + history.insert(pkIdxGen.generate(rng)); + history.selectPartition(pkIdxGen.generate(rng)); + loops.incrementAndGet(); + }, UNSAFE)); + + awaitHarryProgress.run(); + }) + .runAfterNodeUpgrade((cluster, node) -> { + awaitHarryProgress.run(); // make sure harry executes in mixed mode + }) + .runAfterClusterUpgrade((cluster) -> { + + // make sure we haven't persisted any events; + cluster.stream().forEach((i) -> { + Object[][] res = i.executeInternal(String.format("select * from %s.%s", SchemaConstants.SYSTEM_KEYSPACE_NAME, NAME)); + assertEquals(0, res.length); + }); + + cluster.get(1).nodetoolResult("cms", "initialize").asserts().success(); + cluster.get(1).nodetoolResult("cms", "reconfigure", "3").asserts().success(); + cluster.schemaChange(withKeyspace("create table %s.xyz (id int primary key)")); + executor.get().shutdownNow(); + executor.get().awaitTermination(1, TimeUnit.MINUTES); + }).run(); + }); } - private static class Listener implements UpgradeListener, ClusterState + private static class Listener implements UpgradeListener { // only ever one node down here. public final AtomicInteger downNode = new AtomicInteger(0); @@ -142,7 +162,6 @@ public void startup(int i) downNode.set(0); } - @Override public boolean isDown(int i) { return downNode.get() == i; diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/examples/RangeTombstoneBurnTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/examples/RangeTombstoneBurnTest.java index ddddb57c0957..8607c2551ddb 100644 --- a/test/distributed/org/apache/cassandra/fuzz/harry/examples/RangeTombstoneBurnTest.java +++ b/test/distributed/org/apache/cassandra/fuzz/harry/examples/RangeTombstoneBurnTest.java @@ -18,120 +18,117 @@ package org.apache.cassandra.fuzz.harry.examples; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Supplier; - import org.junit.Test; +import org.apache.cassandra.distributed.test.IntegrationTestBase; +import org.apache.cassandra.harry.SchemaSpec; import org.apache.cassandra.harry.checker.ModelChecker; -import org.apache.cassandra.harry.ddl.SchemaGenerators; -import org.apache.cassandra.harry.ddl.SchemaSpec; +import org.apache.cassandra.harry.dsl.HistoryBuilderHelper; import org.apache.cassandra.harry.dsl.ReplayingHistoryBuilder; -import org.apache.cassandra.harry.gen.EntropySource; -import org.apache.cassandra.harry.gen.rng.JdkRandomEntropySource; -import org.apache.cassandra.harry.model.AgainstSutChecker; -import org.apache.cassandra.fuzz.harry.integration.model.IntegrationTestBase; -import org.apache.cassandra.harry.sut.QueryModifyingSut; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.tracker.DataTracker; -import org.apache.cassandra.harry.tracker.DefaultDataTracker; +import org.apache.cassandra.harry.dsl.SingleOperationBuilder; +import org.apache.cassandra.harry.execution.InJvmDTestVisitExecutor; +import org.apache.cassandra.harry.gen.Generator; +import org.apache.cassandra.harry.gen.Generators; +import org.apache.cassandra.harry.gen.SchemaGenerators; + +import static org.apache.cassandra.harry.checker.TestHelper.withRandom; public class RangeTombstoneBurnTest extends IntegrationTestBase { - private final long seed = 1; - private final int ITERATIONS = 5; - private final int STEPS_PER_ITERATION = 100; + private final int ITERATIONS = 10; + private final int STEPS_PER_ITERATION = 1000; @Test - public void rangeTombstoneBurnTest() throws Throwable + public void rangeTombstoneBurnTest() { - Supplier<SchemaSpec> supplier = SchemaGenerators.progression(SchemaGenerators.DEFAULT_SWITCH_AFTER); - - for (int i = 0; i < SchemaGenerators.DEFAULT_RUNS; i++) - { - SchemaSpec schema = supplier.get(); - beforeEach(); - SchemaSpec doubleWriteSchema = schema.cloneWithName(schema.keyspace, schema.keyspace + "_debug"); - - sut.schemaChange(schema.compile().cql()); - sut.schemaChange(doubleWriteSchema.compile().cql()); - - QueryModifyingSut sut = new QueryModifyingSut(this.sut, - schema.table, - doubleWriteSchema.table); - + Generator<SchemaSpec> schemaGen = SchemaGenerators.schemaSpecGen(KEYSPACE, "range_tombstone", 100); + withRandom(rng -> { + SchemaSpec schema = schemaGen.generate(rng); cluster.get(1).nodetool("disableautocompaction"); + cluster.schemaChange(schema.compile()); + + int perIteration = Math.min(10, schema.valueGenerators.pkPopulation());; + int maxPartitions = Math.max(perIteration, schema.valueGenerators.pkPopulation()); for (int iteration = 0; iteration < ITERATIONS; iteration++) { - ModelChecker<ReplayingHistoryBuilder, Void> modelChecker = new ModelChecker<>(); - EntropySource entropySource = new JdkRandomEntropySource(iteration); - - int maxPartitionSize = entropySource.nextInt(1, 1 << entropySource.nextInt(5, 11)); - - int[] partitions = new int[10]; - for (int j = 0; j < partitions.length; j++) + Integer[] partitions = new Integer[perIteration]; + for (int j = 0; j < partitions.length && iteration * 10 < maxPartitions; j++) partitions[j] = iteration * partitions.length + j; - float deleteRowChance = entropySource.nextFloat(0.99f, 1.0f); - float deletePartitionChance = entropySource.nextFloat(0.999f, 1.0f); - float deleteRangeChance = entropySource.nextFloat(0.95f, 1.0f); - float flushChance = entropySource.nextFloat(0.999f, 1.0f); - AtomicInteger flushes = new AtomicInteger(); - - TokenPlacementModel.ReplicationFactor rf = new TokenPlacementModel.SimpleReplicationFactor(1); - - DataTracker tracker = new DefaultDataTracker(); - modelChecker.init(new ReplayingHistoryBuilder(seed, maxPartitionSize, STEPS_PER_ITERATION, new DefaultDataTracker(), sut, schema, rf, SystemUnderTest.ConsistencyLevel.ALL)) - .step((history, rng) -> { - int rowIdx = rng.nextInt(maxPartitionSize); - int partitionIdx = partitions[rng.nextInt(partitions.length)]; - history.visitPartition(partitionIdx).insert(rowIdx); - }) - .step((history, rng) -> rng.nextFloat() > deleteRowChance, - (history, rng) -> { - int partitionIdx = partitions[rng.nextInt(partitions.length)]; - history.visitPartition(partitionIdx).deleteRow(); - }) - .step((history, rng) -> rng.nextFloat() > deleteRowChance, - (history, rng) -> { - int partitionIdx = partitions[rng.nextInt(partitions.length)]; - history.visitPartition(partitionIdx).deleteColumns(); - }) - .step((history, rng) -> rng.nextFloat() > deletePartitionChance, - (history, rng) -> { - int partitionIdx = partitions[rng.nextInt(partitions.length)]; - history.visitPartition(partitionIdx).deletePartition(); - }) - .step((history, rng) -> rng.nextFloat() > flushChance, - (history, rng) -> { - cluster.get(1).nodetool("flush", schema.keyspace, schema.table); - flushes.incrementAndGet(); - }) - .step((history, rng) -> rng.nextFloat() > deleteRangeChance, - (history, rng) -> { - int partitionIdx = partitions[rng.nextInt(partitions.length)]; - history.visitPartition(partitionIdx).deleteRowSlice(); - }) - .step((history, rng) -> rng.nextFloat() > deleteRangeChance, - (history, rng) -> { - int row1 = rng.nextInt(maxPartitionSize); - int row2 = rng.nextInt(maxPartitionSize); - int partitionIdx = partitions[rng.nextInt(partitions.length)]; - history.visitPartition(partitionIdx).deleteRowRange(Math.min(row1, row2), - Math.max(row1, row2), - entropySource.nextBoolean(), - entropySource.nextBoolean()); - }) - .afterAll((history) -> { - // Sanity check - history.validate(new AgainstSutChecker(tracker, history.clock(), sut, schema, doubleWriteSchema), - partitions); - history.validate(partitions); - }) - .run(STEPS_PER_ITERATION, seed, entropySource); + float deleteRowChance = rng.nextFloat(0.99f, 1.0f); + float deletePartitionChance = rng.nextFloat(0.999f, 1.0f); + float deleteColumnsChance = rng.nextFloat(0.95f, 1.0f); + float deleteRangeChance = rng.nextFloat(0.95f, 1.0f); + float flushChance = rng.nextFloat(0.999f, 1.0f); + int maxPartitionSize = Math.min(rng.nextInt(1, 1 << rng.nextInt(5, 11)), schema.valueGenerators.ckPopulation()); + + Generator<Integer> partitionPicker = Generators.pick(partitions); + Generator<Integer> rowPicker = Generators.int32(0, maxPartitionSize); + ModelChecker<SingleOperationBuilder, Void> model = new ModelChecker<>(); + ReplayingHistoryBuilder historyBuilder = new ReplayingHistoryBuilder(schema.valueGenerators, + (hb) -> InJvmDTestVisitExecutor.builder().build(schema, hb, cluster)); + + model.init(historyBuilder) + .step((history, rng_) -> { + int pdIdx = partitionPicker.generate(rng); + history.insert(pdIdx, rowPicker.generate(rng)); + history.selectPartition(pdIdx); + }) + .step((history, rng_) -> rng.nextDouble() >= deleteRowChance, + (history, rng_) -> { + int pdIdx = partitionPicker.generate(rng); + history.deleteRow(pdIdx, rowPicker.generate(rng)); + history.selectPartition(pdIdx); + }) + .step((history, rng_) -> rng.nextDouble() >= deletePartitionChance, + (history, rng_) -> { + int pdIdx = partitionPicker.generate(rng); + history.deletePartition(pdIdx); + history.selectPartition(pdIdx); + }) + .step((history, rng_) -> rng.nextDouble() >= deleteColumnsChance, + (history, rng_) -> { + int pdIdx = partitionPicker.generate(rng); + HistoryBuilderHelper.deleteRandomColumns(schema, pdIdx, rowPicker.generate(rng), rng, history); + history.selectPartition(pdIdx); + }) + .step((history, rng_) -> rng.nextDouble() >= deleteRangeChance, + (history, rng_) -> { + int pdIdx = partitionPicker.generate(rng); + history.deleteRowRange(pdIdx, + rowPicker.generate(rng), + rowPicker.generate(rng), + rng.nextInt(schema.clusteringKeys.size()), + rng.nextBoolean(), + rng.nextBoolean() + ); + history.selectPartition(pdIdx); + }) + .step((history, rng_) -> { + int pdIdx = partitionPicker.generate(rng); + history.selectRow(pdIdx, rowPicker.generate(rng)); + }) + .step((history, rng_) -> { + int pdIdx = partitionPicker.generate(rng); + history.selectRowRange(pdIdx, + rowPicker.generate(rng), + rowPicker.generate(rng), + rng.nextInt(schema.clusteringKeys.size()), + rng.nextBoolean(), + rng.nextBoolean()); + }) + .step((history, rng_) -> rng.nextDouble() >= flushChance, + (history, rng_) -> { + history.custom(() -> cluster.get(1).nodetool("flush", schema.keyspace, schema.table), "FLUSH"); + }) + .exitCondition((history) -> { + if (historyBuilder.size() < STEPS_PER_ITERATION) + return false; + return true; + }) + .run(0, Long.MAX_VALUE, rng); } - } + }); } } \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/examples/RepairBurnTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/examples/RepairBurnTest.java index 19f2997d6959..65b1bf444140 100644 --- a/test/distributed/org/apache/cassandra/fuzz/harry/examples/RepairBurnTest.java +++ b/test/distributed/org/apache/cassandra/fuzz/harry/examples/RepairBurnTest.java @@ -18,27 +18,21 @@ package org.apache.cassandra.fuzz.harry.examples; -import java.util.Arrays; -import java.util.Random; - import org.junit.BeforeClass; import org.junit.Test; import org.apache.cassandra.distributed.api.Feature; -import org.apache.cassandra.fuzz.harry.integration.model.IntegrationTestBase; +import org.apache.cassandra.distributed.test.IntegrationTestBase; +import org.apache.cassandra.harry.SchemaSpec; import org.apache.cassandra.harry.checker.ModelChecker; -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.ddl.SchemaSpec; import org.apache.cassandra.harry.dsl.HistoryBuilder; -import org.apache.cassandra.harry.gen.rng.JdkRandomEntropySource; -import org.apache.cassandra.harry.model.Model; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.sut.injvm.InJvmSutBase; -import org.apache.cassandra.harry.tracker.DataTracker; -import org.apache.cassandra.harry.tracker.DefaultDataTracker; -import org.apache.cassandra.harry.visitors.ReplayingVisitor; +import org.apache.cassandra.harry.dsl.HistoryBuilderHelper; +import org.apache.cassandra.harry.execution.InJvmDTestVisitExecutor; +import org.apache.cassandra.harry.gen.Generator; +import org.apache.cassandra.harry.gen.Generators; +import org.apache.cassandra.harry.gen.SchemaGenerators; + +import static org.apache.cassandra.harry.checker.TestHelper.withRandom; public class RepairBurnTest extends IntegrationTestBase { @@ -46,92 +40,45 @@ public class RepairBurnTest extends IntegrationTestBase public static void before() throws Throwable { init(3, - (cfg) -> InJvmSutBase.defaultConfig().accept(cfg.with(Feature.NETWORK, Feature.GOSSIP))); + (cfg) -> defaultConfig().accept(cfg.with(Feature.NETWORK, Feature.GOSSIP))); } - private final long seed = 1L; - @Test - public void repairBurnTest() throws Throwable + public void repairBurnTest() { - SchemaSpec schema = new SchemaSpec("repair_burn_test", - "test_overrides", - Arrays.asList( - ColumnSpec.pk("pk1", ColumnSpec.asciiType(4, 10)), - ColumnSpec.pk("pk2", ColumnSpec.int64Type), - ColumnSpec.pk("pk3", ColumnSpec.int64Type), - ColumnSpec.pk("pk4", ColumnSpec.asciiType(2, 10))), - Arrays.asList( - ColumnSpec.ck("ck1", ColumnSpec.asciiType(2, 0)), - ColumnSpec.ck("ck2", ColumnSpec.asciiType(2, 0)), - ColumnSpec.ck("ck3", ColumnSpec.int64Type), - ColumnSpec.ck("ck4", ColumnSpec.asciiType(4, 100)), - ColumnSpec.ck("ck5", ColumnSpec.asciiType(8, 100)) - ), - Arrays.asList( - ColumnSpec.regularColumn("regular1", ColumnSpec.asciiType(8, 100)), - ColumnSpec.regularColumn("regular2", ColumnSpec.asciiType(8, 100)), - ColumnSpec.regularColumn("regular3", ColumnSpec.asciiType(8, 100)) - ), - Arrays.asList( - ColumnSpec.staticColumn("static1", ColumnSpec.asciiType(8, 100)), - ColumnSpec.staticColumn("static2", ColumnSpec.asciiType(8, 100)), - ColumnSpec.staticColumn("static3", ColumnSpec.asciiType(8, 100)) - )); - - sut.schemaChange("CREATE KEYSPACE " + schema.keyspace + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};"); - sut.schemaChange(schema.compile().cql()); + int maxPartitionSize = 10; + int partitions = 1000; + Generator<SchemaSpec> schemaGen = SchemaGenerators.schemaSpecGen(KEYSPACE, "repair_burn", 1000); - ModelChecker<HistoryBuilder, Void> modelChecker = new ModelChecker<>(); - DataTracker tracker = new DefaultDataTracker(); + withRandom(rng -> { + SchemaSpec schema = schemaGen.generate(rng); - TokenPlacementModel.ReplicationFactor rf = new TokenPlacementModel.SimpleReplicationFactor(3); + Generators.TrackingGenerator<Integer > pkGen = Generators.tracking(Generators.int32(0, Math.min(schema.valueGenerators.pkPopulation(), partitions))); + Generator<Integer> ckGen = Generators.int32(0, Math.min(schema.valueGenerators.ckPopulation(), maxPartitionSize)); - int maxPartitionSize = 10; - int partitions = 1000; + ModelChecker<HistoryBuilder, Void> modelChecker = new ModelChecker<>(); - modelChecker.init(new HistoryBuilder(seed, maxPartitionSize, 10, schema, rf)) - .step((history, rng) -> { - history.visitPartition(rng.nextInt(partitions), - (ps) -> { - Object[][] clusterings = new Object[maxPartitionSize][]; - for (int i = 0; i < clusterings.length; i++) - { - Object[] v = schema.ckGenerator.inflate(rng.next()); - for (int j = 0; j < v.length; j++) - { - if (rng.nextBoolean() && v[j] instanceof String) - { - v[j] = ""; - return; - } - } - clusterings[i] = v; - } - ps.overrideClusterings(clusterings); - }) - .insert(rng.nextInt(maxPartitionSize)); - }) - .step((history, rng) -> { - history.visitPartition(rng.nextInt(partitions)) - .deleteRow(rng.nextInt(maxPartitionSize)); - }) - .exitCondition((history) -> { - if (history.size() < 10_000) - return false; + modelChecker.init(new HistoryBuilder(schema.valueGenerators)) + .step((history, rng_) -> HistoryBuilderHelper.insertRandomData(schema, pkGen, ckGen, rng, history)) + .step((history, rng_) -> history.deleteRow(pkGen.generate(rng), ckGen.generate(rng))) + .exitCondition((history) -> { + if (history.size() < 10_000) + return false; - ReplayingVisitor visitor = history.visitor(tracker, sut, SystemUnderTest.ConsistencyLevel.NODE_LOCAL); - visitor.replayAll(); + history.custom(() -> cluster.get(1).nodetool("repair", "--full"), + "Repair"); - cluster.get(1).nodetool("repair", "--full"); + for (Integer pkIdx : pkGen.generated()) + history.selectPartition(pkIdx); - Model model = history.quiescentLocalChecker(tracker, sut); + cluster.schemaChange(schema.compile()); - for (Long pd : history.visitedPds()) - model.validate(Query.selectAllColumns(history.schema(), pd, false)); + InJvmDTestVisitExecutor.replay(InJvmDTestVisitExecutor.builder().build(schema, history, cluster), + history); - return true; + return true; }) - .run(Integer.MAX_VALUE, seed, new JdkRandomEntropySource(new Random(seed))); + .run(rng); + }); } } \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/gen/DataGeneratorsTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/gen/DataGeneratorsTest.java deleted file mode 100644 index 37aa5a527450..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/gen/DataGeneratorsTest.java +++ /dev/null @@ -1,504 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.gen; - -import java.lang.reflect.Array; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.function.BiConsumer; -import java.util.function.Consumer; - -import org.junit.Assert; -import org.junit.Ignore; -import org.junit.Test; - -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.gen.Bijections; -import org.apache.cassandra.harry.gen.Bytes; -import org.apache.cassandra.harry.gen.DataGenerators; -import org.apache.cassandra.harry.gen.EntropySource; -import org.apache.cassandra.harry.gen.Generator; -import org.apache.cassandra.harry.gen.StringBijection; - -public class DataGeneratorsTest -{ - private static final int RUNS = 100; - private static final EntropySource rand = EntropySource.forTests(1); - - @Test - public void testSingleTypeRoundTrip() - { - for (ColumnSpec.DataType dt : new ColumnSpec.DataType[]{ColumnSpec.int8Type, - ColumnSpec.int16Type, - ColumnSpec.int32Type, - ColumnSpec.int64Type, - ColumnSpec.asciiType, - ColumnSpec.floatType, - ColumnSpec.doubleType}) - { - for (int i = 0; i < RUNS; i++) - { - DataGenerators.SinglePartKeyGenerator gen = new DataGenerators.SinglePartKeyGenerator(Collections.singletonList(ColumnSpec.ck("ck0", dt, false))); - long descriptor = rand.next(); - descriptor = gen.adjustEntropyDomain(descriptor); - Assert.assertEquals(descriptor, - gen.deflate(gen.inflate(descriptor))); - } - } - } - - @Test - public void testRequiredBytes() - { - testRequiredBytes(sizes(4, 1), - ColumnSpec.int32Type, ColumnSpec.int8Type); - testRequiredBytes(sizes(7, 1), - ColumnSpec.int64Type, ColumnSpec.int8Type); - testRequiredBytes(sizes(4, 4), - ColumnSpec.int32Type, ColumnSpec.int64Type); - testRequiredBytes(sizes(4, 4), - ColumnSpec.int32Type, ColumnSpec.int32Type); - testRequiredBytes(sizes(4, 3), - ColumnSpec.int32Type, ColumnSpec.floatType); - testRequiredBytes(sizes(4, 4), - ColumnSpec.int64Type, ColumnSpec.int64Type); - testRequiredBytes(sizes(4, 2, 2), - ColumnSpec.int64Type, ColumnSpec.int16Type, ColumnSpec.int32Type); - testRequiredBytes(sizes(6, 1, 1), - ColumnSpec.int64Type, ColumnSpec.int8Type, ColumnSpec.int8Type); - testRequiredBytes(sizes(4, 2, 2), - ColumnSpec.int32Type, ColumnSpec.int32Type, ColumnSpec.int64Type); - testRequiredBytes(sizes(4, 2, 2), - ColumnSpec.int32Type, ColumnSpec.int32Type, ColumnSpec.int64Type); - testRequiredBytes(sizes(1, 5, 2), - ColumnSpec.int8Type, ColumnSpec.asciiType, ColumnSpec.int64Type); - testRequiredBytes(sizes(1, 1, 6), - ColumnSpec.int8Type, ColumnSpec.int8Type, ColumnSpec.int64Type); - testRequiredBytes(sizes(2, 2, 2, 2), - ColumnSpec.int64Type, ColumnSpec.int32Type, ColumnSpec.int32Type, ColumnSpec.int32Type); - testRequiredBytes(sizes(1, 3, 2, 2), - ColumnSpec.int8Type, ColumnSpec.int32Type, ColumnSpec.int32Type, ColumnSpec.int32Type); - testRequiredBytes(sizes(1, 3, 2, 2), - ColumnSpec.int8Type, ColumnSpec.int32Type, ColumnSpec.int32Type, ColumnSpec.int32Type); - testRequiredBytes(sizes(1, 3, 2, 2), - ColumnSpec.int8Type, ColumnSpec.int32Type, ColumnSpec.int32Type, ColumnSpec.int32Type, ColumnSpec.int64Type); - } - - private static void testRequiredBytes(int[] sizes, - ColumnSpec.DataType<?>... types) - { - int sum = 0; - for (int size : sizes) - sum += size; - Assert.assertTrue(sum > 0); - Assert.assertTrue(sum <= 8); - - List<ColumnSpec<?>> columns = new ArrayList<>(types.length); - for (int i = 0; i < types.length; i++) - columns.add(ColumnSpec.ck("r" + i, types[i], false)); - Assert.assertArrayEquals(columns.toString(), - sizes, - DataGenerators.requiredBytes(columns)); - } - - @Test - public void testSliceStitch() - { - for (int i = 2; i < 5; i++) - { - Iterator<ColumnSpec.DataType[]> iter = permutations(i, - ColumnSpec.DataType.class, - ColumnSpec.int8Type, - ColumnSpec.asciiType, - ColumnSpec.int16Type, - ColumnSpec.int32Type, - ColumnSpec.int32Type, - ColumnSpec.floatType, - ColumnSpec.doubleType); - while (iter.hasNext()) - { - testSliceStitch(iter.next()); - } - } - } - - private static void testSliceStitch(ColumnSpec.DataType... types) - { - List<ColumnSpec<?>> spec = new ArrayList<>(types.length); - for (int i = 0; i < types.length; i++) - spec.add(ColumnSpec.ck("r" + i, types[i], false)); - DataGenerators.MultiPartKeyGenerator gen = new DataGenerators.MultiPartKeyGenerator(spec); - - for (int i = 0; i < RUNS; i++) - { - long orig = gen.adjustEntropyDomain(rand.next()); - long[] sliced = gen.slice(orig); - long stitched = gen.stitch(sliced); - Assert.assertEquals(String.format("Orig: %s. Stitched: %s", - Long.toHexString(orig), - Long.toHexString(stitched)), - orig, stitched); - } - } - - @Test - public void testKeyGenerators() - { - for (int i = 1; i < 5; i++) - { - for (boolean asReversed : new boolean[]{ false, true }) - { - Iterator<ColumnSpec.DataType[]> iter = permutations(i, - ColumnSpec.DataType.class, - ColumnSpec.int8Type, - ColumnSpec.asciiType, - ColumnSpec.int16Type, - ColumnSpec.int32Type, - ColumnSpec.int64Type, - ColumnSpec.floatType, - ColumnSpec.doubleType - ); - - while (iter.hasNext()) - { - ColumnSpec.DataType[] types = iter.next(); - try - { - testKeyGenerators(asReversed, types); - } - catch (Throwable t) - { - throw new AssertionError("Caught error for the type combination " + Arrays.toString(types), t); - } - } - } - } - } - - @Ignore - @Test // this one is mostly useful when the above test fails and you need a quicker turnaround - public void testSomeKeyGenerators() - { - Iterator<ColumnSpec.DataType[]> iter = Collections.singletonList(new ColumnSpec.DataType[]{ ColumnSpec.int32Type, ColumnSpec.int32Type }).iterator(); - - while (iter.hasNext()) - { - ColumnSpec.DataType[] types = iter.next(); - try - { - testKeyGenerators(true, types); - testKeyGenerators(false, types); - } - catch (Throwable t) - { - throw new AssertionError("Caught error for the type combination " + Arrays.toString(types), t); - } - } - } - - static void testKeyGenerators(boolean reversed, ColumnSpec.DataType<?>... types) - { - List<ColumnSpec<?>> spec = new ArrayList<>(types.length); - for (int i = 0; i < types.length; i++) - spec.add(ColumnSpec.ck("r" + i, types[i], reversed)); - - DataGenerators.KeyGenerator keyGenerator = DataGenerators.createKeyGenerator(spec); - - for (int i = 0; i < RUNS; i++) - { - testKeyGenerators(rand.next(), rand.next(), keyGenerator); - // test some edge cases - testKeyGenerators(0, 0, keyGenerator); - testKeyGenerators(0xffffffffffffffffL, 0xffffffffffffffffL, keyGenerator); - testKeyGenerators(keyGenerator.minValue(), keyGenerator.maxValue(), keyGenerator); - testKeyGenerators(0, keyGenerator.minValue(), keyGenerator); - testKeyGenerators(0, keyGenerator.maxValue(), keyGenerator); - long descriptor = rand.next(); - testKeyGenerators(descriptor, 0, keyGenerator); - testKeyGenerators(descriptor, keyGenerator.minValue(), keyGenerator); - testKeyGenerators(descriptor, keyGenerator.maxValue(), keyGenerator); - testKeyGenerators(descriptor, 0xffffffffffffffffL, keyGenerator); - testKeyGenerators(descriptor, descriptor + 1, keyGenerator); - testKeyGenerators(descriptor, descriptor - 1, keyGenerator); - testKeyGenerators(descriptor, descriptor, keyGenerator); - } - - // Fixed prefix, tests sign inversion of subsequent values - if (types.length > 1) - { - - long pattern = Bytes.bytePatternFor(Long.BYTES - DataGenerators.requiredBytes(spec)[0]); - for (int i = 0; i < RUNS; i++) - { - long descriptor = rand.next(); - long descriptor2 = (descriptor & ~pattern) | (rand.next() & pattern); - testKeyGenerators(descriptor, descriptor2, keyGenerator); - } - - } - } - - static void testKeyGenerators(long descriptor1, long descriptor2, DataGenerators.KeyGenerator keyGenerator) - { - descriptor1 = keyGenerator.adjustEntropyDomain(descriptor1); - descriptor2 = keyGenerator.adjustEntropyDomain(descriptor2); - Object[] value1 = keyGenerator.inflate(descriptor1); - Object[] value2 = keyGenerator.inflate(descriptor2); - - assertDescriptorsEqual(descriptor1, - keyGenerator.deflate(value1)); - assertDescriptorsEqual(descriptor2, - keyGenerator.deflate(value2)); - - Assert.assertEquals(String.format("%s %s %s and %s %s %s have different order. ", - Arrays.toString(value1), - toSignString(compare(value1, value2, keyGenerator.columns)), - Arrays.toString(value2), - Long.toHexString(descriptor1), - toSignString(Long.compare(descriptor1, descriptor2)), - Long.toHexString(descriptor2)), - normalize(Long.compare(descriptor1, descriptor2)), - compare(value1, value2, keyGenerator.columns)); - } - - private static void assertDescriptorsEqual(long l, long r) - { - Assert.assertEquals(String.format("Expected %d (0x%s), but got %d (0x%s)", - l, Long.toHexString(l), r, Long.toHexString(r)), - l, r); - } - - @Test - public void int8GeneratorTest() - { - testInverse(Bijections.INT8_GENERATOR); - } - - @Test - public void int16GeneratorTest() - { - testInverse(Bijections.INT16_GENERATOR); - } - - @Test - public void int32GeneratorTest() - { - testInverse(Bijections.INT32_GENERATOR); - } - - @Test - public void int64GeneratorTest() - { - testInverse(Bijections.INT64_GENERATOR); - testOrderPreserving(Bijections.INT64_GENERATOR); - testInverse(new Bijections.ReverseBijection(Bijections.INT64_GENERATOR)); - testOrderPreserving(new Bijections.ReverseBijection(Bijections.INT64_GENERATOR), true); - } - - @Test - public void booleanGenTest() - { - testInverse(Bijections.BOOLEAN_GENERATOR); - testOrderPreserving(Bijections.BOOLEAN_GENERATOR); - } - - @Test - public void floatGeneratorTest() - { - testInverse(Bijections.FLOAT_GENERATOR); - testOrderPreserving(Bijections.FLOAT_GENERATOR, Float::compareTo); - } - - @Test - public void doubleGeneratorTest() - { - testInverse(Bijections.DOUBLE_GENERATOR); - testOrderPreserving(Bijections.DOUBLE_GENERATOR); - } - - - @Test - public void stringGenTest() - { - testInverse(new StringBijection()); - testOrderPreserving(new StringBijection()); - } - - public static <T> void testInverse(Bijections.Bijection<T> gen) - { - test(gen, - (v) -> Assert.assertEquals(gen.adjustEntropyDomain(v.descriptor), gen.deflate(v.value))); - } - - public static <T extends Comparable> void testOrderPreserving(Bijections.Bijection<T> gen) - { - testOrderPreserving(gen, false); - } - - public static <T extends Comparable> void testOrderPreserving(Bijections.Bijection<T> gen, boolean reverse) - { - test(gen, gen, - (v1, v2) -> { - long v1Descriptor = gen.adjustEntropyDomain(v1.descriptor); - long v2Descriptor = gen.adjustEntropyDomain(v2.descriptor); - Assert.assertEquals(String.format("%s (%s) and %s (%s) sort wrong", - v1.value, - Long.toHexString(v1Descriptor), - v2.value, - Long.toHexString(v2Descriptor)), - normalize(Long.compare(v1Descriptor, v2Descriptor)) * (reverse ? -1 : 1), - normalize(v1.value.compareTo(v2.value))); - }); - } - - public static <T extends Comparable> void testOrderPreserving(Bijections.Bijection<T> gen, Comparator<T> comparator) - { - test(gen, gen, - (v1, v2) -> Assert.assertEquals(normalize(Long.compare(gen.adjustEntropyDomain(v1.descriptor), - gen.adjustEntropyDomain(v2.descriptor))), - normalize(comparator.compare(v1.value, v2.value)))); - } - - public static <T1> void test(Bijections.Bijection<T1> gen1, - Consumer<Generator.Value<T1>> validate) - { - - for (int i = 0; i < RUNS; i++) - { - long descriptor1 = rand.next(); - validate.accept(new Generator.Value<T1>(descriptor1, gen1.inflate(gen1.adjustEntropyDomain(descriptor1)))); - } - } - - public static <T1, T2> void test(Bijections.Bijection<T1> gen1, - Bijections.Bijection<T2> gen2, - BiConsumer<Generator.Value<T1>, Generator.Value<T2>> validate) - { - - for (int i = 0; i < RUNS; i++) - { - long descriptor1 = rand.next(); - long descriptor2 = rand.next(); - validate.accept(new Generator.Value<T1>(descriptor1, gen1.inflate(gen1.adjustEntropyDomain(descriptor1))), - new Generator.Value<T2>(descriptor2, gen2.inflate(gen1.adjustEntropyDomain(descriptor2)))); - } - } - - public static int[] sizes(int... ints) - { - return ints; - } - - - public static String toSignString(int l) - { - if (l == 0) - return "="; - else if (l > 0) - return ">"; - return "<"; - } - public static int normalize(int l) - { - if (l == 0) - return 0; - if (l > 0) - return 1; - else - return -1; - } - - static int compare(Object[] a, Object[] b, List<ColumnSpec<?>> spec) - { - assert a.length == b.length; - for (int i = 0; i < a.length; i++) - { - Comparable comparableA = (Comparable) a[i]; - Comparable comparableB = (Comparable) b[i]; - - int cmp = comparableA.compareTo(comparableB); - if (cmp != 0) - { - if (spec.get(i).isReversed()) - cmp *= -1; - - return cmp < 0 ? -1 : 1; - } - } - return 0; - } - - public static <T> Iterator<T[]> permutations(int size, Class<T> klass, T... values) - { - int[] cursors = new int[size]; - return new Iterator<T[]>() - { - int left = 0; - T[] next = fromCursors(); - - public boolean hasNext() - { - for (int i = cursors.length - 1; i >= 0; i--) - { - if (cursors[i] < values.length - 1) - return true; - } - return false; - } - - public T[] computeNext() - { - cursors[left]++; - - for (int i = 0; i < cursors.length; i++) - { - if (cursors[i] == values.length) - { - cursors[i] = 0; - cursors[i + 1]++; - } - } - - return fromCursors(); - } - - public T[] next() - { - if (next == null) - next = computeNext(); - - T[] ret = next; - next = null; - return ret; - } - - public T[] fromCursors() - { - T[] res = (T[]) Array.newInstance(klass, cursors.length); - for (int i = 0; i < cursors.length; i++) - res[i] = values[cursors[i]]; - return res; - } - }; - } -} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/gen/EntropySourceTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/gen/EntropySourceTest.java deleted file mode 100644 index a2b53306d025..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/gen/EntropySourceTest.java +++ /dev/null @@ -1,158 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.gen; - -import java.util.Random; - -import org.junit.Assert; -import org.junit.Test; - -import org.apache.cassandra.harry.gen.EntropySource; -import org.apache.cassandra.harry.gen.distribution.Distribution; -import org.apache.cassandra.harry.gen.rng.PCGFastPure; -import org.apache.cassandra.harry.gen.rng.PcgRSUFast; -import org.apache.cassandra.harry.model.OpSelectors; - -public class EntropySourceTest -{ - private static int RUNS = 100000; - - @Test - public void testScale() - { - Random rand = new Random(); - for (int cycle = 0; cycle < RUNS; cycle++) - { - int a = rand.nextInt(100); - int b = rand.nextInt(100); - while (a == b) - b = rand.nextInt(100); - - int min = Math.min(a, b); - int max = Math.max(a, b); - long[] cardinality = new long[max - min]; - for (int i = 0; i < 100000; i++) - { - long rnd = rand.nextLong(); - long scaled = Distribution.ScaledDistribution.scale(rnd, min, max); - cardinality[(int) scaled - min]++; - } - - for (long c : cardinality) - Assert.assertTrue(c > 0); - } - } - - @Test - public void testShuffleUnshuffle() - { - Random rnd = new Random(); - - for (int i = 1; i < RUNS; i++) - { - long l = rnd.nextLong(); - Assert.assertEquals(l, PCGFastPure.unshuffle(PCGFastPure.shuffle(l))); - } - } - - @Test - public void testImmutableRng() - { - int size = 5; - OpSelectors.PureRng rng = new OpSelectors.PCGFast(1); - for (int stream = 1; stream < RUNS; stream++) - { - long[] generated = new long[size]; - for (int i = 0; i < size; i++) - generated[i] = rng.randomNumber(i, stream); - - Assert.assertEquals(0, rng.sequenceNumber(generated[0], stream)); - Assert.assertEquals(generated[1], rng.next(generated[0], stream)); - - for (int i = 1; i < size; i++) - { - Assert.assertEquals(generated[i], rng.next(generated[i - 1], stream)); - Assert.assertEquals(generated[i - 1], rng.prev(generated[i], stream)); - Assert.assertEquals(i, rng.sequenceNumber(generated[i], stream)); - } - } - } - - @Test - public void testSequenceNumber() - { - int size = 5; - OpSelectors.PureRng rng = new OpSelectors.PCGFast(1); - for (int stream = 1; stream < RUNS; stream++) - { - for (int i = 0; i < size; i++) - Assert.assertEquals(i, rng.sequenceNumber(rng.randomNumber(i, stream), stream)); - } - } - - @Test - public void seekTest() - { - PcgRSUFast rand = new PcgRSUFast(1, 1); - long first = rand.next(); - long last = 0; - for (int i = 0; i < 10; i++) - last = rand.next(); - - rand.advance(-11); - Assert.assertEquals(first, rand.next()); - - rand.advance(9); - Assert.assertEquals(last, rand.next()); - Assert.assertEquals(first, rand.nextAt(0)); - Assert.assertEquals(last, rand.nextAt(10)); - Assert.assertEquals(-10, rand.distance(first)); - } - - @Test - public void shuffleUnshuffleTest() - { - Random rnd = new Random(); - for (int i = 0; i < RUNS; i++) - { - long a = rnd.nextLong(); - Assert.assertEquals(a, PCGFastPure.unshuffle(PCGFastPure.shuffle(a))); - } - } - - @Test - public void testIntBetween() - { - EntropySource rng = new PcgRSUFast(System.currentTimeMillis(), 0); - - int a = 0; - int b = 50; - int[] cardinality = new int[b - a]; - for (int i = 0; i < RUNS; i++) - { - int min = Math.min(a, b); - int max = Math.max(a, b); - cardinality[rng.nextInt(min, max - 1) - min]++; - } - - // Extremely improbable yet possible that some of the values won't be generated - for (int i = 0; i < cardinality.length; i++) - Assert.assertTrue(cardinality[i] > 0); - } -} diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/gen/SurjectionsTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/gen/SurjectionsTest.java deleted file mode 100644 index b749c4140a13..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/gen/SurjectionsTest.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.gen; - -import java.util.HashMap; -import java.util.Map; - -import org.junit.Assert; -import org.junit.Test; - -import org.apache.cassandra.harry.gen.EntropySource; -import org.apache.cassandra.harry.gen.Surjections; -import org.apache.cassandra.harry.gen.rng.PcgRSUFast; - -public class SurjectionsTest -{ - private static int RUNS = 1000000; - - @Test - public void weightedTest() - { - int[] weights = new int[] {50, 40, 10}; - - Surjections.Surjection<String> gen = Surjections.weighted(Surjections.weights(weights), - "a", "b", "c"); - - Map<String, Integer> frequencies = new HashMap<>(); - EntropySource rng = new PcgRSUFast(System.currentTimeMillis(), 0); - - for (int i = 0; i < RUNS; i++) - { - String s = gen.inflate(rng.next()); - frequencies.compute(s, (s1, i1) -> { - if (i1 == null) - return 1; - else - return i1 + 1; - }); - } - - Assert.assertEquals(frequencies.get("a") / 10000, weights[0], 1); - Assert.assertEquals(frequencies.get("b") / 10000, weights[1], 1); - Assert.assertEquals(frequencies.get("c") / 10000, weights[2], 1); - } - - -} diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/integration/QuickTheoriesAdapter.java b/test/distributed/org/apache/cassandra/fuzz/harry/integration/QuickTheoriesAdapter.java deleted file mode 100644 index ecb3513278eb..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/integration/QuickTheoriesAdapter.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.integration; - -import org.apache.cassandra.harry.gen.Generator; -import org.apache.cassandra.harry.gen.EntropySource; -import org.apache.cassandra.harry.gen.rng.RngUtils; -import org.quicktheories.core.Gen; -import org.quicktheories.core.RandomnessSource; -import org.quicktheories.impl.Constraint; - -public class QuickTheoriesAdapter -{ - public static <T> Gen<T> convert(Generator<T> generator) - { - return new Gen<T>() - { - private final RandomnessSourceAdapter<T> adapter = new RandomnessSourceAdapter<>(); - - public T generate(RandomnessSource randomnessSource) - { - return adapter.generate(randomnessSource, generator); - } - }; - } - - public static class RandomnessSourceAdapter<T> implements EntropySource - { - private RandomnessSource rnd; - - public long next() - { - return rnd.next(Constraint.none()); - } - - public void seed(long seed) - { - throw new RuntimeException("Seed is not settable"); - } - - public EntropySource derive() - { - return new RandomnessSourceAdapter<>(); - } - - public int nextInt() - { - return RngUtils.asInt(next()); - } - - public int nextInt(int max) - { - return RngUtils.asInt(next(), max); - } - - public int nextInt(int min, int max) - { - return RngUtils.asInt(next(), min, max); - } - - public long nextLong(long min, long max) - { - return RngUtils.trim(next(), min, max); - } - - public float nextFloat() - { - return RngUtils.asFloat(next()); - } - - public boolean nextBoolean() - { - return RngUtils.asBoolean(next()); - } - - public T generate(RandomnessSource rnd, Generator<T> generate) - { - this.rnd = rnd; - T value = generate.generate(this); - this.rnd = null; - return value; - } - } -} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/integration/ddl/SchemaGenTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/integration/ddl/SchemaGenTest.java deleted file mode 100644 index c39421fb8430..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/integration/ddl/SchemaGenTest.java +++ /dev/null @@ -1,215 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.integration.ddl; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; - -import org.junit.Assert; -import org.junit.Test; - -import org.apache.cassandra.fuzz.harry.integration.QuickTheoriesAdapter; -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.ddl.SchemaGenerators; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.gen.Generator; -import org.apache.cassandra.harry.operations.CompiledStatement; - -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.util.TestRunner; -import org.apache.cassandra.cql3.CQLTester; -import org.apache.cassandra.db.Keyspace; -import org.apache.cassandra.schema.ColumnMetadata; -import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.utils.Pair; -import org.quicktheories.QuickTheory; -import org.quicktheories.core.Gen; - -import static org.quicktheories.generators.SourceDSL.integers; - -public class SchemaGenTest extends CQLTester -{ - private static final int CYCLES = 10; - - // TODO: compact storage tests - @Test - public void testSelectForwardAndReverseIteration() throws Throwable - { - Generator<SchemaSpec> gen = new SchemaGenerators.Builder(KEYSPACE).partitionKeyColumnCount(1, 4) - .clusteringColumnCount(1, 10) - .regularColumnCount(0, 10) - .staticColumnCount(0, 10) - .generator(); - - TestRunner.test(gen, - schemaDefinition -> { - String tableDef = schemaDefinition.compile().cql(); - createTable(tableDef); - - try - { - CompiledStatement statement = Query.selectAllColumns(schemaDefinition, 1, false).toSelectStatement(); - execute(statement.cql(), statement.bindings()); - statement = Query.selectAllColumns(schemaDefinition, 1, true).toSelectStatement(); - execute(statement.cql(), statement.bindings()); - } - catch (Throwable t) - { - throw new AssertionError("Exception caught", t); - } - }); - } - - @Test - public void createTableRoundTrip() throws Throwable - { - Generator<SchemaSpec> gen = new SchemaGenerators.Builder(KEYSPACE).partitionKeyColumnCount(1, 10) - .clusteringColumnCount(1, 10) - .regularColumnCount(0, 10) - .staticColumnCount(0, 10) - .generator(); - - TestRunner.test(gen, - schemaDefinition -> { - String tableDef = schemaDefinition.compile().cql(); - createTable(KEYSPACE, tableDef); - TableMetadata tableMetadata = Keyspace.open(KEYSPACE).getColumnFamilyStore(schemaDefinition.table).metadata.get(); - compareColumns(schemaDefinition.partitionKeys, tableMetadata.partitionKeyColumns()); - compareColumns(schemaDefinition.clusteringKeys, tableMetadata.clusteringColumns()); - compareColumns(schemaDefinition.regularColumns, tableMetadata.regularColumns()); - compareColumns(schemaDefinition.staticColumns, tableMetadata.staticColumns()); - }); - } - - @Test - public void testReverseComparator() - { - SchemaSpec spec = new SchemaSpec(KEYSPACE, "tbl1", - Arrays.asList(ColumnSpec.pk("pk1", ColumnSpec.asciiType), - ColumnSpec.pk("pk2", ColumnSpec.int64Type)), - Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.asciiType, true), - ColumnSpec.ck("ck2", ColumnSpec.int64Type, false)), - Arrays.asList(ColumnSpec.regularColumn("v1", ColumnSpec.asciiType), - ColumnSpec.regularColumn("v2", ColumnSpec.asciiType), - ColumnSpec.regularColumn("v3", ColumnSpec.int64Type), - ColumnSpec.regularColumn("v4", ColumnSpec.int64Type)), - Arrays.asList(ColumnSpec.staticColumn("static1", ColumnSpec.asciiType), - ColumnSpec.staticColumn("static2", ColumnSpec.int64Type))); - - - String tableDef = spec.compile().cql(); - createTable(KEYSPACE, tableDef); - TableMetadata tableMetadata = Keyspace.open(KEYSPACE).getColumnFamilyStore(spec.table).metadata.get(); - compareColumns(spec.partitionKeys, tableMetadata.partitionKeyColumns()); - compareColumns(spec.clusteringKeys, tableMetadata.clusteringColumns()); - compareColumns(spec.regularColumns, tableMetadata.regularColumns()); - compareColumns(spec.staticColumns, tableMetadata.staticColumns()); - } - - - @Test - public void testSchemaGeneration() - { - Gen<Pair<Integer, Integer>> ckCounts = integers().between(0, 4).zip(integers().between(0, 6), Pair::create); - Gen<Pair<Integer, Integer>> regCounts = integers().between(0, 4).zip(integers().between(0, 6), Pair::create); -// Gen<Pair<Integer, Integer>> staticCounts = integers().between(0, 4).zip(integers().between(0, 6), Pair::create); - Gen<Pair<Integer, Integer>> pkCounts = integers().between(1, 4).zip(integers().between(0, 6), Pair::create); - - Gen<SchemaGenerationInputs> inputs = pkCounts.zip(ckCounts, regCounts, - (pks, cks, regs) -> - new SchemaGenerationInputs(pks.left, pks.left + pks.right, - cks.left, cks.left + cks.right, - regs.left, regs.left + regs.right)); - - Gen<Pair<SchemaGenerationInputs, SchemaSpec>> schemaAndInputs = inputs.flatMap(input -> { - Generator<SchemaSpec> gen = new SchemaGenerators.Builder("test") - .partitionKeyColumnCount(input.minPk, input.maxPk) - .clusteringColumnCount(input.minCks, input.maxCks) - .regularColumnCount(input.minRegs, input.maxRegs) - .generator(); - - return QuickTheoriesAdapter.convert(gen).map(schema -> Pair.create(input, schema)); - }); - - qt().forAll(schemaAndInputs) - .check(schemaAndInput -> { - SchemaGenerationInputs input = schemaAndInput.left; - SchemaSpec schema = schemaAndInput.right; - - return schema.partitionKeys.size() <= input.maxPk && schema.partitionKeys.size() >= input.minPk && - schema.clusteringKeys.size() <= input.maxCks && schema.clusteringKeys.size() >= input.minCks && - schema.regularColumns.size() <= input.maxRegs && schema.regularColumns.size() >= input.minRegs; - }); - } - - private static class SchemaGenerationInputs { - private final int minPk; - private final int maxPk; - private final int minCks; - private final int maxCks; - private final int minRegs; - private final int maxRegs; - - public SchemaGenerationInputs(int minPk, int maxPk, int minCks, int maxCks, int minRegs, int maxRegs) - { - this.minPk = minPk; - this.maxPk = maxPk; - this.minCks = minCks; - this.maxCks = maxCks; - this.minRegs = minRegs; - this.maxRegs = maxRegs; - } - } - - private static boolean compareColumns(Collection<ColumnSpec<?>> expectedColl, Collection<ColumnMetadata> actualColl) - { - List<ColumnSpec<?>> expectedSorted = new ArrayList<>(expectedColl); - expectedSorted.sort(Comparator.comparing(Object::toString)); - List<ColumnMetadata> actualSorted = new ArrayList<>(actualColl); - actualSorted.sort(Comparator.comparing(Object::toString)); - - Iterator<ColumnSpec<?>> expectedIter = expectedSorted.iterator(); - Iterator<ColumnMetadata> actualIter = actualSorted.iterator(); - - while (expectedIter.hasNext() && actualIter.hasNext()) - { - ColumnSpec expected = expectedIter.next(); - ColumnMetadata actual = actualIter.next(); - - Assert.assertEquals(expected.kind.toString(), actual.kind.toString()); - Assert.assertEquals(expected.name, actual.name.toString()); - Assert.assertEquals(expected.type.toString(), actual.type.asCQL3Type().toString()); - } - - Assert.assertEquals(String.format("Collections %s and %s have different sizes", expectedColl, actualColl), - expectedIter.hasNext(), actualIter.hasNext()); - return true; - } - - public static QuickTheory qt() - { - return QuickTheory.qt() - .withExamples(CYCLES) - .withShrinkCycles(0); - } -} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/integration/dsl/HistoryBuilderIntegrationTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/integration/dsl/HistoryBuilderIntegrationTest.java deleted file mode 100644 index bd0d2533a905..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/integration/dsl/HistoryBuilderIntegrationTest.java +++ /dev/null @@ -1,173 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.integration.dsl; - -import java.util.Random; -import java.util.function.Supplier; - -import org.junit.Test; - -import org.apache.cassandra.fuzz.harry.integration.model.IntegrationTestBase; -import org.apache.cassandra.harry.checker.ModelChecker; -import org.apache.cassandra.harry.ddl.SchemaGenerators; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.dsl.BatchVisitBuilder; -import org.apache.cassandra.harry.dsl.HistoryBuilder; -import org.apache.cassandra.harry.dsl.SingleOperationBuilder; -import org.apache.cassandra.harry.gen.rng.JdkRandomEntropySource; -import org.apache.cassandra.harry.model.Model; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.tracker.DataTracker; -import org.apache.cassandra.harry.tracker.DefaultDataTracker; -import org.apache.cassandra.harry.visitors.ReplayingVisitor; - -public class HistoryBuilderIntegrationTest extends IntegrationTestBase -{ - private static final long SEED = 1L; - private static final int STEPS_PER_ITERATION = 1_000; - private static final int MAX_PARTITIONS = 50; - - @Test - public void simpleDSLTest() throws Throwable - { - Supplier<SchemaSpec> supplier = SchemaGenerators.progression(SchemaGenerators.DEFAULT_SWITCH_AFTER); - for (int i = 0; i < SchemaGenerators.DEFAULT_RUNS; i++) - { - SchemaSpec schema = supplier.get(); - DataTracker tracker = new DefaultDataTracker(); - beforeEach(); - sut.schemaChange(schema.compile().cql()); - - ModelChecker<SingleOperationBuilder, Void> modelChecker = new ModelChecker<>(); - - TokenPlacementModel.ReplicationFactor rf = new TokenPlacementModel.SimpleReplicationFactor(1); - - int maxPartitionSize = 100; - modelChecker.init(new HistoryBuilder(SEED, maxPartitionSize, 10, schema, rf)) - .step((history) -> { - history.insert(); - }) - .step((history, rng) -> { - history.insert(rng.nextInt(maxPartitionSize)); - }) - .step((history, rng) -> { - int row = rng.nextInt(maxPartitionSize); - long[] vIdxs = new long[schema.regularColumns.size()]; - for (int j = 0; j < schema.regularColumns.size(); j++) - vIdxs[j] = rng.nextInt(20); - - history.insert(row, vIdxs); - }) - .step((history) -> { - history.deleteRow(); - }) - .step((history, rng) -> { - history.deleteRow(rng.nextInt(maxPartitionSize)); - }) - .step(SingleOperationBuilder::deletePartition) - .step(SingleOperationBuilder::deleteColumns) - .step(SingleOperationBuilder::deleteRowSlice) - .step((history) -> { - history.deleteRowRange(); - }) - .step((history, rng) -> { - history.deleteRowRange(rng.nextInt(maxPartitionSize), - rng.nextInt(maxPartitionSize), - rng.nextBoolean(), - rng.nextBoolean()); - }) - .step((history) -> history instanceof HistoryBuilder, - (history) -> ((HistoryBuilder) history).beginBatch()) - .step((history) -> (history instanceof BatchVisitBuilder) && ((BatchVisitBuilder) history).size() > 1, - (history) -> ((BatchVisitBuilder) history).endBatch()) - .exitCondition((history) -> { - if (!(history instanceof HistoryBuilder)) - return false; - - HistoryBuilder historyBuilder = (HistoryBuilder) history; - ReplayingVisitor visitor = historyBuilder.visitor(tracker, sut, SystemUnderTest.ConsistencyLevel.ALL); - visitor.replayAll(); - - if (historyBuilder.visitedPds().size() < MAX_PARTITIONS) - return false; - - Model model = historyBuilder.quiescentChecker(tracker, sut); - - for (Long pd : historyBuilder.visitedPds()) - model.validate(Query.selectAllColumns(historyBuilder.schema(), pd, false)); - - return true; - }) - .run(STEPS_PER_ITERATION, SEED, new JdkRandomEntropySource(new Random(SEED))); - } - } - - @Test - public void overrideCkTest() throws Throwable - { - Supplier<SchemaSpec> supplier = SchemaGenerators.progression(SchemaGenerators.DEFAULT_SWITCH_AFTER); - for (int schemaIdx = 0; schemaIdx < SchemaGenerators.DEFAULT_RUNS; schemaIdx++) - { - SchemaSpec schema = supplier.get(); - DataTracker tracker = new DefaultDataTracker(); - beforeEach(); - sut.schemaChange(schema.compile().cql()); - - ModelChecker<HistoryBuilder, Void> modelChecker = new ModelChecker<>(); - - TokenPlacementModel.ReplicationFactor rf = new TokenPlacementModel.SimpleReplicationFactor(1); - - int maxPartitionSize = 10; - modelChecker.init(new HistoryBuilder(SEED, maxPartitionSize, 10, schema, rf)) - .beforeAll((history, rng) -> { - for (int i = 0; i < MAX_PARTITIONS; i++) - history.forPartition(i).ensureClustering(schema.ckGenerator.inflate(rng.next())); - }) - .step((history, rng) -> history.visitPartition(rng.nextInt(MAX_PARTITIONS)).insert()) - .step((history, rng) -> history.visitPartition(rng.nextInt(MAX_PARTITIONS)).insert(rng.nextInt(maxPartitionSize))) - .step((history, rng) -> history.visitPartition(rng.nextInt(MAX_PARTITIONS)).deleteRow()) - .step((history, rng) -> history.visitPartition(rng.nextInt(MAX_PARTITIONS)).deleteRow(rng.nextInt(maxPartitionSize))) - .step((history, rng) -> history.visitPartition(rng.nextInt(MAX_PARTITIONS)).deletePartition()) - .step((history, rng) -> history.visitPartition(rng.nextInt(MAX_PARTITIONS)).deleteColumns()) - .step((history, rng) -> history.visitPartition(rng.nextInt(MAX_PARTITIONS)).deleteRowRange()) - .step((history, rng) -> history.visitPartition(rng.nextInt(MAX_PARTITIONS)).deleteRowSlice()) - .exitCondition((history) -> { - ReplayingVisitor visitor = history.visitor(tracker, sut, SystemUnderTest.ConsistencyLevel.ALL); - visitor.replayAll(); - - if (history.visitedPds().size() < MAX_PARTITIONS) - return false; - - Model model = history.quiescentChecker(tracker, sut); - - for (Long pd : history.visitedPds()) - { - model.validate(Query.selectAllColumns(history.schema(), pd, false)); - model.validate(Query.selectAllColumnsWildcard(history.schema(), pd, false)); - } - - - return true; - }) - .run(STEPS_PER_ITERATION, SEED, new JdkRandomEntropySource(new Random(SEED))); - } - } -} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/integration/dsl/HistoryBuilderOverridesIntegrationTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/integration/dsl/HistoryBuilderOverridesIntegrationTest.java deleted file mode 100644 index a6cfb74cd85e..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/integration/dsl/HistoryBuilderOverridesIntegrationTest.java +++ /dev/null @@ -1,359 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.integration.dsl; - -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.function.Supplier; - -import org.junit.Assert; -import org.junit.Test; - -import org.apache.cassandra.fuzz.harry.integration.model.IntegrationTestBase; -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.ddl.SchemaGenerators; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.dsl.HistoryBuilder; -import org.apache.cassandra.harry.gen.Bijections; -import org.apache.cassandra.harry.gen.EntropySource; -import org.apache.cassandra.harry.gen.rng.JdkRandomEntropySource; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.tracker.DataTracker; -import org.apache.cassandra.harry.tracker.DefaultDataTracker; -import org.apache.cassandra.harry.visitors.ReplayingVisitor; - -public class HistoryBuilderOverridesIntegrationTest extends IntegrationTestBase -{ - private static final long SEED = 1L; - - public static SchemaSpec SIMPLE_SCHEMA = new SchemaSpec("harry", - "test_overrides", - Arrays.asList(ColumnSpec.pk("pk1", ColumnSpec.asciiType(4, 10)), - ColumnSpec.pk("pk2", ColumnSpec.int64Type), - ColumnSpec.pk("pk3", ColumnSpec.int64Type), - ColumnSpec.pk("pk4", ColumnSpec.asciiType(2, 10)) - ), - Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.asciiType(2, 0)), - ColumnSpec.ck("ck2", ColumnSpec.asciiType(2, 0)), - ColumnSpec.ck("ck3", ColumnSpec.int64Type), - ColumnSpec.ck("ck4", ColumnSpec.asciiType(4, 100)), - ColumnSpec.ck("ck5", ColumnSpec.asciiType(8, 100)) - ), - Arrays.asList(ColumnSpec.regularColumn("regular1", ColumnSpec.asciiType(8, 100)), - ColumnSpec.regularColumn("regular2", ColumnSpec.asciiType(8, 100)), - ColumnSpec.regularColumn("regular3", ColumnSpec.asciiType(8, 100)) - ), - Arrays.asList(ColumnSpec.staticColumn("static1", ColumnSpec.asciiType(8, 100)), - ColumnSpec.staticColumn("static2", ColumnSpec.asciiType(8, 100)), - ColumnSpec.staticColumn("static3", ColumnSpec.asciiType(8, 100)) - )); - @Test - public void simpleCkOverrideTest() - { - SchemaSpec schema = SIMPLE_SCHEMA; - - DataTracker tracker = new DefaultDataTracker(); - beforeEach(); - sut.schemaChange(schema.compile().cql()); - - TokenPlacementModel.ReplicationFactor rf = new TokenPlacementModel.SimpleReplicationFactor(1); - - HistoryBuilder history = new HistoryBuilder(SEED, 5, 10, schema, rf); - Object[] override = new Object[]{ "", "b", -1L, "c", "d" }; - history.forPartition(1).ensureClustering(override); - for (int i = 0; i < 5; i++) - history.visitPartition(1).insert(i); - - history.visitor(tracker, sut, SystemUnderTest.ConsistencyLevel.ALL).replayAll(); - - Object[][] res = sut.execute(Query.selectAllColumns(history.schema(), history.visitedPds().get(0), false).toSelectStatement(), - SystemUnderTest.ConsistencyLevel.ALL); - int found = 0; - for (Object[] row : res) - { - if (Arrays.equals(override, Arrays.copyOfRange(row, 4, 9))) - found++; - } - Assert.assertEquals("Should have mutated exactly one CK", found, 1); - - history.validateAll(tracker, sut); - } - - @Test - public void ckOverrideSortingTest() - { - for (boolean reverse : new boolean[]{ true, false }) - { - SchemaSpec schema = new SchemaSpec("harry", - "test_overrides" + (reverse ? "_reverse" : ""), - Arrays.asList(ColumnSpec.pk("pk1", ColumnSpec.asciiType(4, 10))), - Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.asciiType(2, 0), reverse)), - Arrays.asList(ColumnSpec.regularColumn("regular1", ColumnSpec.asciiType(8, 100))), - Arrays.asList(ColumnSpec.staticColumn("static1", ColumnSpec.asciiType(8, 100)))); - - DataTracker tracker = new DefaultDataTracker(); - beforeEach(); - sut.schemaChange(schema.compile().cql()); - - TokenPlacementModel.ReplicationFactor rf = new TokenPlacementModel.SimpleReplicationFactor(1); - - int partitionSize = 10; - HistoryBuilder history = new HistoryBuilder(SEED, partitionSize, 10, schema, rf); - ReplayingVisitor visitor = history.visitor(tracker, sut, SystemUnderTest.ConsistencyLevel.ALL); - Set<Integer> foundAt = new HashSet<>(); - for (int pdIdx = 0; pdIdx < 128; pdIdx++) - { - Object[] override = new Object[]{ Character.toString(pdIdx) }; - history.forPartition(pdIdx).ensureClustering(override); - for (int i = 0; i < partitionSize; i++) - history.visitPartition(pdIdx).insert(i); - - visitor.replayAll(); - long visitedPd = history.forPartition(pdIdx).pd(); - { - Object[][] res = sut.execute(Query.selectAllColumns(history.schema(), visitedPd, false).toSelectStatement(), - SystemUnderTest.ConsistencyLevel.ALL); - - int found = 0; - for (int i = 0; i < res.length; i++) - { - Object[] row = res[i]; - if (Arrays.equals(override, Arrays.copyOfRange(row, 1, 2))) - { - found++; - foundAt.add(i); - } - } - Assert.assertEquals("Should have mutated exactly one CK", found, 1); - } - history.validateAll(tracker, sut); - } - Assert.assertEquals(10, foundAt.size()); - } - } - - @Test - public void ckOverrideManySortingTest() - { - int counter = 0; - for (boolean reverse : new boolean[]{ true, false }) - { - for (ColumnSpec.DataType type : new ColumnSpec.DataType[]{ ColumnSpec.asciiType(2, 0), ColumnSpec.int64Type }) - { - SchemaSpec schema = new SchemaSpec("harry", - "test_overrides" + (counter++), - Arrays.asList(ColumnSpec.pk("pk1", ColumnSpec.asciiType(4, 10))), - Arrays.asList(ColumnSpec.ck("ck1", type, reverse)), - Arrays.asList(ColumnSpec.regularColumn("regular1", ColumnSpec.asciiType(8, 100))), - Arrays.asList(ColumnSpec.staticColumn("static1", ColumnSpec.asciiType(8, 100)))); - - DataTracker tracker = new DefaultDataTracker(); - beforeEach(); - sut.schemaChange(schema.compile().cql()); - - TokenPlacementModel.ReplicationFactor rf = new TokenPlacementModel.SimpleReplicationFactor(1); - - int partitionSize = 10; - HistoryBuilder history = new HistoryBuilder(SEED, partitionSize, 10, schema, rf); - ReplayingVisitor visitor = history.visitor(tracker, sut, SystemUnderTest.ConsistencyLevel.ALL); - EntropySource rng = new JdkRandomEntropySource(SEED); - for (int pdIdx = 0; pdIdx < 100; pdIdx++) - { - Set<Object> overrides = new HashSet<>(); - for (int i = 0; i < 5; i++) - { - Object override = schema.clusteringKeys.get(0).generator().inflate(rng.next()); - try - { - history.forPartition(pdIdx).ensureClustering(new Object[]{ override }); - overrides.add(override); - } - catch (IllegalStateException t) - { - // could not override twice - } - } - - for (int i = 0; i < partitionSize; i++) - history.visitPartition(pdIdx).insert(i); - - visitor.replayAll(); - long visitedPd = history.forPartition(pdIdx).pd(); - { - Object[][] res = sut.execute(Query.selectAllColumns(history.schema(), visitedPd, false).toSelectStatement(), - SystemUnderTest.ConsistencyLevel.ALL); - - int found = 0; - for (int i = 0; i < res.length; i++) - { - Object[] row = res[i]; - Object v = row[1]; - if (overrides.contains(v)) - found++; - } - Assert.assertEquals("Should have mutated exactly one CK", found, overrides.size()); - } - history.validateAll(tracker, sut); - } - } - } - } - - @Test - public void ckOverrideWithDeleteTestSingleColumn() - { - Supplier<SchemaSpec> supplier = SchemaGenerators.progression(SchemaGenerators.DEFAULT_SWITCH_AFTER); - - int partitionSize = 5; - for (int cnt = 0; cnt < SchemaGenerators.DEFAULT_RUNS; cnt++) - { - SchemaSpec schema = supplier.get(); - beforeEach(); - - DataTracker tracker = new DefaultDataTracker(); - beforeEach(); - sut.schemaChange(schema.compile().cql()); - - TokenPlacementModel.ReplicationFactor rf = new TokenPlacementModel.SimpleReplicationFactor(1); - - HistoryBuilder history = new HistoryBuilder(SEED, partitionSize, 1, schema, rf); - ReplayingVisitor visitor = history.visitor(tracker, sut, SystemUnderTest.ConsistencyLevel.ALL); - - EntropySource rng = new JdkRandomEntropySource(SEED); - for (int i = 0; i < partitionSize; i++) - { - history.visitPartition(1, - (e) -> schema.ckGenerator.inflate(rng.next())) - .insert(i); - } - - for (int i = 0; i < partitionSize; i++) - { - history.visitPartition(1) - .deleteRow(i); - - visitor.replayAll(); - history.validateAll(tracker, sut); - } - } - } - - @Test - public void regularAndStaticOverrideTest() - { - for (ColumnSpec.DataType<?> type : new ColumnSpec.DataType[]{ ColumnSpec.asciiType(2, 0), ColumnSpec.int64Type }) - { - SchemaSpec schema = new SchemaSpec("harry", - "test_overrides", - Arrays.asList(ColumnSpec.pk("pk1", ColumnSpec.asciiType(4, 10))), - Arrays.asList(ColumnSpec.ck("ck1", type, false)), - Arrays.asList(ColumnSpec.regularColumn("regular1", ColumnSpec.asciiType(2, 2)), - ColumnSpec.regularColumn("regular2", ColumnSpec.int64Type)), - Arrays.asList(ColumnSpec.staticColumn("static1", ColumnSpec.asciiType(2, 2)), - ColumnSpec.staticColumn("static2", ColumnSpec.int64Type) - )); - - Map<String, Bijections.Bijection<?>> reGenerators = new HashMap<>(); - reGenerators.put("regular1", ColumnSpec.asciiType(4, 4).generator()); - reGenerators.put("regular2", ColumnSpec.int64Type.generator()); - reGenerators.put("static1", ColumnSpec.asciiType(8, 4).generator()); - reGenerators.put("static2", ColumnSpec.int64Type.generator()); - - DataTracker tracker = new DefaultDataTracker(); - beforeEach(); - sut.schemaChange(schema.compile().cql()); - - TokenPlacementModel.ReplicationFactor rf = new TokenPlacementModel.SimpleReplicationFactor(1); - - int partitionSize = 100; - HistoryBuilder history = new HistoryBuilder(SEED, partitionSize, 10, schema, rf); - ReplayingVisitor visitor = history.visitor(tracker, sut, SystemUnderTest.ConsistencyLevel.ALL); - EntropySource rng = new JdkRandomEntropySource(SEED); - - Map<String, Set<Object>> perColumnOverrides = new HashMap<>(); - for (ColumnSpec<?> column : schema.regularColumns) - { - perColumnOverrides.put(column.name, new HashSet<>()); - for (int i = 0; i < partitionSize; i++) - { - Object override = reGenerators.get(column.name).inflate(rng.next()); - history.valueOverrides().override(column.name, i, override); - perColumnOverrides.get(column.name).add(override); - } - } - - for (ColumnSpec<?> column : schema.staticColumns) - { - perColumnOverrides.put(column.name, new HashSet<>()); - for (int i = 0; i < partitionSize; i++) - { - Object override = reGenerators.get(column.name).inflate(rng.next()); - history.valueOverrides().override(column.name, i, override); - perColumnOverrides.get(column.name).add(override); - } - } - for (int pdIdx = 0; pdIdx < 10; pdIdx++) - { - Map<String, Set<Object>> results = new HashMap<>(); - for (ColumnSpec<?> column : schema.regularColumns) - results.put(column.name, new HashSet<>()); - for (ColumnSpec<?> column : schema.staticColumns) - results.put(column.name, new HashSet<>()); - - for (int i = 0; i < partitionSize; i++) - { - history.visitPartition(pdIdx) - .insert(i, - new long[]{ rng.nextInt(100), rng.nextInt(100) }, - new long[]{ rng.nextInt(100), rng.nextInt(100) }); - } - - visitor.replayAll(); - history.validateAll(tracker, sut); - - long visitedPd = history.forPartition(pdIdx).pd(); - Object[][] res = sut.execute(Query.selectAllColumns(history.schema(), visitedPd, false).toSelectStatement(), - SystemUnderTest.ConsistencyLevel.ALL); - - for (int i = 0; i < res.length; i++) - { - Object[] row = res[i]; - results.get("regular1").add(row[4]); - results.get("regular2").add(row[5]); - results.get("static1").add(row[2]); - results.get("static2").add(row[3]); - } - - for (Map.Entry<String, Set<Object>> e : results.entrySet()) - { - for (Object o : e.getValue()) - { - Assert.assertTrue(String.format("Found a non-overriden value for %s: %s", e.getKey(), e.getValue()), - perColumnOverrides.get(e.getKey()).contains(o)); - } - } - } - } - } -} diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/integration/generators/DataGeneratorsIntegrationTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/integration/generators/DataGeneratorsIntegrationTest.java deleted file mode 100644 index 1ce9994c09a9..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/integration/generators/DataGeneratorsIntegrationTest.java +++ /dev/null @@ -1,179 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.integration.generators; - -import java.util.Random; -import java.util.concurrent.CompletableFuture; - -import com.google.common.collect.Iterators; -import org.junit.Test; - -import org.apache.cassandra.harry.HarryHelper; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.ddl.SchemaGenerators; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.gen.Generator; -import org.apache.cassandra.harry.gen.Surjections; -import org.apache.cassandra.harry.gen.distribution.Distribution; -import org.apache.cassandra.harry.model.NoOpChecker; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.visitors.MutatingVisitor; -import org.apache.cassandra.harry.visitors.MutatingRowVisitor; -import org.apache.cassandra.harry.visitors.SingleValidator; -import org.apache.cassandra.harry.util.TestRunner; -import org.apache.cassandra.harry.visitors.Visitor; -import org.apache.cassandra.cql3.CQLTester; -import org.apache.cassandra.cql3.UntypedResultSet; -import org.apache.cassandra.distributed.impl.RowUtil; - -public class DataGeneratorsIntegrationTest extends CQLTester -{ - @Test - public void testTimestampTieResolution() - { - Random rng = new Random(1); - String ks = "test_timestamp_tie_resolution"; - createKeyspace(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}", ks)); - int counter = 0; - for (ColumnSpec.DataType<?> dataType : new ColumnSpec.DataType[]{ ColumnSpec.int8Type, - ColumnSpec.int16Type, - ColumnSpec.int32Type, - ColumnSpec.int64Type, - ColumnSpec.asciiType, - ColumnSpec.floatType, - ColumnSpec.doubleType }) - { - - - String tbl = "table_" + (counter++); - createTable(String.format("CREATE TABLE %s.%s (pk int PRIMARY KEY, v %s)", - ks, tbl, - dataType)); - for (int i = 0; i < 10_000; i++) - { - long d1 = dataType.generator().adjustEntropyDomain(rng.nextLong()); - long d2 = dataType.generator().adjustEntropyDomain(rng.nextLong()); - for (long d : new long[]{ d1, d2 }) - { - execute(String.format("INSERT INTO %s.%s (pk, v) VALUES (?,?) USING TIMESTAMP 1", ks, tbl), - i, dataType.generator().inflate(d)); - } - - if (dataType.compareLexicographically(d1, d2) > 0) - assertRows(execute(String.format("SELECT v FROM %s.%s WHERE pk=?", ks, tbl), i), - row(dataType.generator().inflate(d1))); - else - assertRows(execute(String.format("SELECT v FROM %s.%s WHERE pk=?", ks, tbl), i), - row(dataType.generator().inflate(d2))); - } - } - } - - @Test - public void queryParseabilityTest() throws Throwable - { - Generator<SchemaSpec> gen = new SchemaGenerators.Builder(KEYSPACE).partitionKeyColumnCount(2, 4) - .clusteringColumnCount(1, 4) - .regularColumnCount(1, 4) - .staticColumnCount(1, 4) - .generator(); - - TestRunner.test(gen, - (schema) -> { - try - { - schema.validate(); - } - catch (AssertionError e) - { - return; - } - createTable(schema.compile().cql()); - - - Configuration.ConfigurationBuilder builder = HarryHelper.defaultConfiguration() - .setDataTracker(new Configuration.NoOpDataTrackerConfiguration()) - .setSchemaProvider(new Configuration.FixedSchemaProviderConfiguration(schema)) - .setSUT(CqlTesterSut::new); - - for (OpSelectors.OperationKind kind : OpSelectors.OperationKind.values()) - { - Run run = builder - .setClusteringDescriptorSelector((rng, schema_) -> { - return new OpSelectors.DefaultDescriptorSelector(rng, - OpSelectors.columnSelectorBuilder().forAll(schema_).build(), - OpSelectors.OperationSelector.weighted(Surjections.weights(100), kind), - new Distribution.ConstantDistribution(2), - 100); - }) - .build() - .createRun(); - - Visitor visitor = new MutatingVisitor(run, MutatingRowVisitor::new); - for (int lts = 0; lts < 100; lts++) - visitor.visit(); - - SingleValidator validator = new SingleValidator(100, run, NoOpChecker::new); - for (int lts = 0; lts < 10; lts++) - validator.visit(lts); - } - }); - } - - public class CqlTesterSut implements SystemUnderTest - { - public boolean isShutdown() - { - return false; - } - - public void shutdown() - { - } - - public void schemaChange(String statement) - { - createTable(statement); - } - - public Object[][] execute(String statement, ConsistencyLevel cl, Object... bindings) - { - try - { - UntypedResultSet res = DataGeneratorsIntegrationTest.this.execute(statement, bindings); - if (res == null) - return new Object[][]{}; - - return Iterators.toArray(RowUtil.toIter(res), Object[].class); - } - catch (Throwable throwable) - { - throw new RuntimeException(throwable); - } - } - - public CompletableFuture<Object[][]> executeAsync(String statement, ConsistencyLevel cl, Object... bindings) - { - return CompletableFuture.completedFuture(execute(statement, cl, bindings)); - } - } -} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/ConcurrentQuiescentCheckerIntegrationTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/ConcurrentQuiescentCheckerIntegrationTest.java deleted file mode 100644 index 192629b0260b..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/ConcurrentQuiescentCheckerIntegrationTest.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.integration.model; - -import java.util.concurrent.TimeUnit; -import java.util.function.Supplier; - -import org.junit.Test; - -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.ddl.SchemaGenerators; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.tracker.LockingDataTracker; -import org.apache.cassandra.harry.runner.Runner; -import org.apache.cassandra.harry.visitors.MutatingVisitor; -import org.apache.cassandra.harry.visitors.QueryLogger; -import org.apache.cassandra.harry.visitors.RandomPartitionValidator; - -import static org.apache.cassandra.harry.core.Configuration.VisitorPoolConfiguration.pool; -import static java.util.Arrays.asList; - -public class ConcurrentQuiescentCheckerIntegrationTest extends ModelTestBase -{ - @Test - public void testConcurrentReadWriteWorkload() throws Throwable - { - Supplier<SchemaSpec> supplier = SchemaGenerators.progression(1); - int writeThreads = 2; - int readThreads = 2; - - for (int i = 0; i < SchemaGenerators.GENERATORS_COUNT; i++) - { - SchemaSpec schema = supplier.get(); - - Configuration config = configuration(i, schema) - .setKeyspaceDdl("CREATE KEYSPACE IF NOT EXISTS " + schema.keyspace + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1};") - .setCreateSchema(true) - .setDropSchema(true) - .setDataTracker(LockingDataTracker::new) - .build(); - - Runner.concurrent(config, - asList(pool("Writer", writeThreads, MutatingVisitor::new), - pool("Reader", readThreads, (run) -> new RandomPartitionValidator(run, modelConfiguration(), QueryLogger.NO_OP))), - 2, TimeUnit.MINUTES) - .run(); - } - } - - @Override - protected Configuration.ModelConfiguration modelConfiguration() - { - return new Configuration.QuiescentCheckerConfig(); - } -} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/InJVMTokenAwareExecutorTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/InJVMTokenAwareExecutorTest.java deleted file mode 100644 index bd7927a13eeb..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/InJVMTokenAwareExecutorTest.java +++ /dev/null @@ -1,98 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.integration.model; - -import java.util.concurrent.TimeUnit; -import java.util.function.Supplier; - -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.ddl.SchemaGenerators; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.QuiescentChecker; -import org.apache.cassandra.harry.model.RepairingLocalStateValidator; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.sut.injvm.InJVMTokenAwareVisitExecutor; -import org.apache.cassandra.harry.sut.injvm.InJvmSut; -import org.apache.cassandra.harry.sut.injvm.InJvmSutBase; -import org.apache.cassandra.harry.runner.Runner; -import org.apache.cassandra.harry.runner.UpToLtsRunner; -import org.apache.cassandra.harry.visitors.MutatingVisitor; -import org.apache.cassandra.distributed.Cluster; -import org.apache.cassandra.distributed.api.Feature; - -public class InJVMTokenAwareExecutorTest extends IntegrationTestBase -{ - private static final Logger logger = LoggerFactory.getLogger(InJVMTokenAwareExecutorTest.class); - - @BeforeClass - public static void before() throws Throwable - { - cluster = Cluster.build() - .withNodes(5) - .withConfig((cfg) -> InJvmSutBase.defaultConfig().accept(cfg.with(Feature.GOSSIP, Feature.NETWORK))) - .createWithoutStarting(); - cluster.setUncaughtExceptionsFilter(t -> { - logger.error("Caught exception, reporting during shutdown. Ignoring.", t); - return true; - }); - cluster.startup(); - cluster = init(cluster); - sut = new InJvmSut(cluster); - } - - @Override - @Before - public void beforeEach() - { - cluster.schemaChange("DROP KEYSPACE IF EXISTS harry"); - cluster.schemaChange("CREATE KEYSPACE harry WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};"); - } - - @Test - public void testRepair() throws Throwable - { - Supplier<SchemaSpec> schemaGen = SchemaGenerators.progression(1); - for (int cnt = 0; cnt < SchemaGenerators.DEFAULT_RUNS; cnt++) - { - SchemaSpec schema = schemaGen.get(); - Configuration.ConfigurationBuilder builder = sharedConfiguration(cnt, schema); - - Configuration configuration = builder.build(); - Run run = configuration.createRun(); - run.sut.schemaChange(run.schemaSpec.compile().cql()); - - Runner.chain(configuration, - UpToLtsRunner.factory(MutatingVisitor.factory(InJVMTokenAwareVisitExecutor.factory(new Configuration.MutatingRowVisitorConfiguration(), - SystemUnderTest.ConsistencyLevel.NODE_LOCAL, - new TokenPlacementModel.SimpleReplicationFactor(3))), - 10_000, 2, TimeUnit.SECONDS), - Runner.single(RepairingLocalStateValidator.factoryForTests(5, QuiescentChecker::new))) - .run(); - } - } -} diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/IntegrationTestBase.java b/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/IntegrationTestBase.java deleted file mode 100644 index 7a6a926bedb2..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/IntegrationTestBase.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.integration.model; - -import java.util.function.Consumer; -import java.util.function.Supplier; - -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.distributed.api.IInstanceConfig; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.ddl.SchemaGenerators; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.clock.OffsetClock; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.sut.injvm.InJvmSut; -import org.apache.cassandra.harry.sut.injvm.InJvmSutBase; -import org.apache.cassandra.distributed.Cluster; -import org.apache.cassandra.distributed.test.TestBaseImpl; - -public class IntegrationTestBase extends TestBaseImpl -{ - protected static final Logger logger = LoggerFactory.getLogger(IntegrationTestBase.class); - protected static Cluster cluster; - protected static InJvmSut sut; - - @BeforeClass - public static void before() throws Throwable - { - init(1, InJvmSutBase.defaultConfig()); - } - - protected static void init(int nodes, Consumer<IInstanceConfig> cfg) throws Throwable - { - cluster = Cluster.build() - .withNodes(nodes) - .withConfig(cfg) - .createWithoutStarting(); - cluster.setUncaughtExceptionsFilter(t -> { - logger.error("Caught exception, reporting during shutdown. Ignoring.", t); - return true; - }); - cluster.startup(); - cluster = init(cluster); - sut = new InJvmSut(cluster); - } - @AfterClass - public static void afterClass() - { - sut.shutdown(); - } - - @Before - public void beforeEach() - { - cluster.schemaChange("DROP KEYSPACE IF EXISTS harry"); - cluster.schemaChange("CREATE KEYSPACE harry WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1};"); - } - - private static long seed = 0; - public static Supplier<Configuration.ConfigurationBuilder> sharedConfiguration() - { - Supplier<SchemaSpec> specGenerator = SchemaGenerators.progression(SchemaGenerators.DEFAULT_SWITCH_AFTER); - return () -> { - SchemaSpec schemaSpec = specGenerator.get(); - return sharedConfiguration(seed, schemaSpec); - }; - } - - public static Configuration.CDSelectorConfigurationBuilder sharedCDSelectorConfiguration() - { - return new Configuration.CDSelectorConfigurationBuilder() - .setOperationsPerLtsDistribution(new Configuration.ConstantDistributionConfig(2)) - .setMaxPartitionSize(100) - .setOperationKindWeights(new Configuration.OperationKindSelectorBuilder() - .addWeight(OpSelectors.OperationKind.DELETE_ROW, 1) - .addWeight(OpSelectors.OperationKind.DELETE_COLUMN, 1) - .addWeight(OpSelectors.OperationKind.DELETE_RANGE, 1) - .addWeight(OpSelectors.OperationKind.DELETE_SLICE, 1) - .addWeight(OpSelectors.OperationKind.DELETE_PARTITION, 1) - .addWeight(OpSelectors.OperationKind.DELETE_COLUMN_WITH_STATICS, 5) - .addWeight(OpSelectors.OperationKind.INSERT_WITH_STATICS, 20) - .addWeight(OpSelectors.OperationKind.INSERT, 20) - .addWeight(OpSelectors.OperationKind.UPDATE_WITH_STATICS, 25) - .addWeight(OpSelectors.OperationKind.UPDATE, 25) - .build()); - } - - public static Configuration.ConfigurationBuilder sharedConfiguration(long seed, SchemaSpec schema) - { - return new Configuration.ConfigurationBuilder().setSeed(seed) - .setClock(() -> new OffsetClock(100000)) - .setCreateSchema(true) - .setTruncateTable(false) - .setDropSchema(true) - .setSchemaProvider((seed1, sut) -> schema) - .setClusteringDescriptorSelector(sharedCDSelectorConfiguration().build()) - .setPartitionDescriptorSelector(new Configuration.DefaultPDSelectorConfiguration(2, 200)) - .setSUT(() -> sut); - } -} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/MockSchema.java b/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/MockSchema.java deleted file mode 100644 index 8d7475be184e..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/MockSchema.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.integration.model; - -import java.util.Arrays; - -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.ddl.SchemaGenerators; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.gen.Surjections; -import org.apache.cassandra.harry.util.BitSet; - -public class MockSchema -{ - public static final String KEYSPACE = "harry"; - public static final Surjections.Surjection<BitSet> columnMaskSelector1; - public static final Surjections.Surjection<BitSet> columnMaskSelector2; - public static final Surjections.Surjection<BitSet> compactColumnMaskSelector; - - public static final SchemaSpec tbl1; - public static final SchemaSpec tbl2; - public static final SchemaSpec compact_schema; - - static - { - columnMaskSelector1 = Surjections.pick(BitSet.create(0b0000111, 7), - BitSet.create(0b1110000, 7), - BitSet.create(0b1111111, 7)); - - columnMaskSelector2 = Surjections.pick(BitSet.create(0b11, 2), - BitSet.create(0b01, 2), - BitSet.create(0b10, 2)); - - compactColumnMaskSelector = Surjections.pick(BitSet.create(1, 1)); - - tbl1 = new SchemaSpec(KEYSPACE, "tbl1", - Arrays.asList(ColumnSpec.pk("pk1", ColumnSpec.asciiType), - ColumnSpec.pk("pk2", ColumnSpec.int64Type)), - Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.asciiType, false), - ColumnSpec.ck("ck2", ColumnSpec.int64Type, false)), - Arrays.asList(ColumnSpec.regularColumn("v1", ColumnSpec.int32Type), - ColumnSpec.regularColumn("v2", ColumnSpec.int64Type), - // TODO: test boolean values; before that - submit a PR to Cassandra where we add boolean to ByteBufferUtil#objectToBytes - ColumnSpec.regularColumn("v3", ColumnSpec.int32Type), - ColumnSpec.regularColumn("v4", ColumnSpec.asciiType), - ColumnSpec.regularColumn("v5", ColumnSpec.int64Type), - ColumnSpec.regularColumn("v6", ColumnSpec.floatType), - ColumnSpec.regularColumn("v7", ColumnSpec.doubleType)), - Arrays.asList(ColumnSpec.staticColumn("static1", ColumnSpec.asciiType), - ColumnSpec.staticColumn("static2", ColumnSpec.int64Type))); - - tbl2 = new SchemaSpec(KEYSPACE, "tbl2", - Arrays.asList(ColumnSpec.pk("pk1", ColumnSpec.asciiType), - ColumnSpec.pk("pk2", ColumnSpec.int64Type)), - Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.asciiType, true)), - Arrays.asList(ColumnSpec.regularColumn("v1", ColumnSpec.int32Type), - ColumnSpec.regularColumn("v2", ColumnSpec.asciiType)), - Arrays.asList(ColumnSpec.regularColumn("static1", ColumnSpec.int32Type), - ColumnSpec.regularColumn("static2", ColumnSpec.asciiType))); - - compact_schema = new SchemaSpec(KEYSPACE, "tbl3", - Arrays.asList(ColumnSpec.pk("pk1", ColumnSpec.asciiType), - ColumnSpec.pk("pk2", ColumnSpec.int64Type)), - Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.asciiType, true), - ColumnSpec.ck("ck2", ColumnSpec.int64Type, false)), - Arrays.asList(ColumnSpec.regularColumn("v1", ColumnSpec.asciiType)), - Arrays.asList(ColumnSpec.staticColumn("static1", ColumnSpec.asciiType))) - .withCompactStorage(); - } - - public static SchemaSpec randomSchema(String keyspace, String table, long seed) - { - return new SchemaGenerators.Builder(keyspace, () -> table) - .partitionKeySpec(1, 4, -// ColumnSpec.int8Type, -// ColumnSpec.int16Type, -// ColumnSpec.int32Type, - ColumnSpec.int64Type, - ColumnSpec.floatType, - ColumnSpec.doubleType, - ColumnSpec.asciiType(4, 10)) - .clusteringKeySpec(1, 4, -// ColumnSpec.int8Type, -// ColumnSpec.int16Type, -// ColumnSpec.int32Type, - ColumnSpec.int64Type, - ColumnSpec.floatType, - ColumnSpec.doubleType, - ColumnSpec.asciiType(4, 10)) - .regularColumnSpec(2, 10, - ColumnSpec.int8Type, - ColumnSpec.int16Type, - ColumnSpec.int32Type, - ColumnSpec.int64Type, - ColumnSpec.floatType, - ColumnSpec.doubleType, - ColumnSpec.asciiType(5, 10)) - .surjection() - .inflate(seed); - } -} diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/ModelTestBase.java b/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/ModelTestBase.java deleted file mode 100644 index 2776aea930e0..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/ModelTestBase.java +++ /dev/null @@ -1,138 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.integration.model; - -import java.util.Arrays; -import java.util.Collections; -import java.util.concurrent.TimeUnit; -import java.util.function.BiConsumer; -import java.util.function.Function; -import java.util.function.Supplier; - -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.ddl.SchemaGenerators; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.runner.Runner; -import org.apache.cassandra.harry.runner.UpToLtsRunner; -import org.apache.cassandra.harry.visitors.MutatingRowVisitor; -import org.apache.cassandra.harry.visitors.MutatingVisitor; -import org.apache.cassandra.harry.visitors.SingleValidator; - -public abstract class ModelTestBase extends IntegrationTestBase -{ - private final int ITERATIONS = 20_000; - - void negativeTest(Function<Run, Boolean> corrupt, BiConsumer<Throwable, Run> validate) throws Throwable - { - Supplier<SchemaSpec> supplier = SchemaGenerators.progression(SchemaGenerators.DEFAULT_SWITCH_AFTER); - for (int i = 0; i < SchemaGenerators.DEFAULT_RUNS; i++) - { - SchemaSpec schema = supplier.get(); - negativeTest(corrupt, validate, i, schema); - } - } - - void negativeIntegrationTest(Configuration.RunnerConfiguration runnerConfig) throws Throwable - { - Supplier<SchemaSpec> supplier = SchemaGenerators.progression(1); - for (int i = 0; i < SchemaGenerators.DEFAULT_RUNS; i++) - { - SchemaSpec schema = supplier.get(); - Configuration.ConfigurationBuilder builder = configuration(i, schema); - - builder.setClock(new Configuration.ApproximateClockConfiguration((int) TimeUnit.MINUTES.toMillis(10), - 1, TimeUnit.SECONDS)) - .setCreateSchema(false) - .setDropSchema(false) - .setRunner(runnerConfig); - - Configuration config = builder.build(); - Runner runner = config.createRunner(); - - Run run = runner.getRun(); - beforeEach(); - run.sut.schemaChange(run.schemaSpec.compile().cql()); - runner.run(); - } - } - - protected abstract Configuration.ModelConfiguration modelConfiguration(); - - protected SingleValidator validator(Run run) - { - return new SingleValidator(100, run , modelConfiguration()); - } - - public Configuration.ConfigurationBuilder configuration(long seed, SchemaSpec schema) - { - return sharedConfiguration(seed, schema); - } - - void negativeTest(Function<Run, Boolean> corrupt, BiConsumer<Throwable, Run> validate, int counter, SchemaSpec schemaSpec) throws Throwable - { - Configuration config = configuration(counter, schemaSpec) - .setCreateSchema(true) - .setKeyspaceDdl(String.format("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': %d};", - schemaSpec.keyspace, cluster.size())) - .setDropSchema(true) - .build(); - - Run run = config.createRun(); - - new Runner.ChainRunner(run, config, - Arrays.asList(writer(ITERATIONS, 2, TimeUnit.MINUTES), - (r, c) -> new Runner.SingleVisitRunner(r, c, Collections.singletonList(this::validator)) { - @Override - public void runInternal() - { - if (!corrupt.apply(run)) - { - System.out.println("Could not corrupt"); - return; - } - try - { - super.runInternal(); - throw new ShouldHaveThrownException(); - } - catch (Throwable t) - { - validate.accept(t, run); - } - } - })).run(); - } - - public static Configuration.RunnerConfiguration writer(long iterations, int runtime, TimeUnit timeUnit) - { - return (run, config) -> { - return new UpToLtsRunner(run, config, - Collections.singletonList((r_) -> new MutatingVisitor(r_, MutatingRowVisitor::new)), - iterations, - runtime, timeUnit); - }; - } - - public static class ShouldHaveThrownException extends AssertionError - { - - } -} - diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/QuerySelectorNegativeTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/QuerySelectorNegativeTest.java deleted file mode 100644 index 11c65e4ad91c..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/QuerySelectorNegativeTest.java +++ /dev/null @@ -1,157 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.integration.model; - -import java.util.Arrays; -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; -import java.util.Random; -import java.util.function.Supplier; - -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.corruptor.AddExtraRowCorruptor; -import org.apache.cassandra.harry.corruptor.ChangeValueCorruptor; -import org.apache.cassandra.harry.corruptor.HideRowCorruptor; -import org.apache.cassandra.harry.corruptor.HideValueCorruptor; -import org.apache.cassandra.harry.corruptor.QueryResponseCorruptor; -import org.apache.cassandra.harry.corruptor.ShowValueCorruptor; -import org.apache.cassandra.harry.ddl.SchemaGenerators; -import org.apache.cassandra.harry.model.Model; -import org.apache.cassandra.harry.model.QuiescentChecker; -import org.apache.cassandra.harry.visitors.MutatingVisitor; -import org.apache.cassandra.harry.visitors.MutatingRowVisitor; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.operations.QueryGenerator; -import org.apache.cassandra.harry.visitors.Visitor; - -import static org.apache.cassandra.harry.corruptor.QueryResponseCorruptor.SimpleQueryResponseCorruptor; - -@RunWith(Parameterized.class) -public class QuerySelectorNegativeTest extends IntegrationTestBase -{ - private final int CYCLES = 1000; - - private final Random rnd = new Random(); - - private final QueryResponseCorruptorFactory corruptorFactory; - - public QuerySelectorNegativeTest(QueryResponseCorruptorFactory corruptorFactory) - { - this.corruptorFactory = corruptorFactory; - } - - @Parameterized.Parameters - public static Collection<QueryResponseCorruptorFactory> source() - { - return Arrays.asList((run) -> new SimpleQueryResponseCorruptor(run.schemaSpec, - run.clock, - ChangeValueCorruptor::new), - (run) -> new SimpleQueryResponseCorruptor(run.schemaSpec, - run.clock, - HideValueCorruptor::new), - (run) -> new SimpleQueryResponseCorruptor(run.schemaSpec, - run.clock, - ShowValueCorruptor::new), - (run) -> new SimpleQueryResponseCorruptor(run.schemaSpec, - run.clock, - HideRowCorruptor::new), - (run) -> new AddExtraRowCorruptor(run.schemaSpec, - run.clock, - run.tracker, - run.descriptorSelector) - ); - } - - interface QueryResponseCorruptorFactory - { - QueryResponseCorruptor create(Run run); - } - - @Test - public void selectRows() - { - Map<Query.QueryKind, Integer> stats = new HashMap<>(); - Supplier<Configuration.ConfigurationBuilder> gen = sharedConfiguration(); - - int rounds = SchemaGenerators.DEFAULT_RUNS; - int failureCounter = 0; - outer: - for (int counter = 0; counter < rounds; counter++) - { - beforeEach(); - Configuration config = gen.get() - .setClusteringDescriptorSelector(sharedCDSelectorConfiguration() - .setOperationsPerLtsDistribution(new Configuration.ConstantDistributionConfig(2)) - .setMaxPartitionSize(2000) - .build()) - .build(); - Run run = config.createRun(); - run.sut.schemaChange(run.schemaSpec.compile().cql()); - System.out.println(run.schemaSpec.compile().cql()); - - Visitor visitor = new MutatingVisitor(run, MutatingRowVisitor::new); - Model model = new QuiescentChecker(run); - - QueryResponseCorruptor corruptor = this.corruptorFactory.create(run); - - for (int i = 0; i < CYCLES; i++) - visitor.visit(); - - while (true) - { - long verificationLts = rnd.nextInt(1000); - QueryGenerator queryGen = new QueryGenerator(run.schemaSpec, - run.pdSelector, - run.descriptorSelector, - run.rng); - - QueryGenerator.TypedQueryGenerator querySelector = new QueryGenerator.TypedQueryGenerator(run.rng, queryGen); - Query query = querySelector.inflate(verificationLts, counter); - - model.validate(query); - - if (!corruptor.maybeCorrupt(query, run.sut)) - continue; - - try - { - model.validate(query); - Assert.fail("Should've failed"); - } - catch (Throwable t) - { - // expected - failureCounter++; - stats.compute(query.queryKind, (Query.QueryKind kind, Integer cnt) -> cnt == null ? 1 : (cnt + 1)); - continue outer; - } - } - } - - Assert.assertTrue(String.format("Seen only %d failures", failureCounter), - failureCounter > (rounds * 0.8)); - } -} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/QuerySelectorTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/QuerySelectorTest.java deleted file mode 100644 index 39fe2c89c5cd..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/QuerySelectorTest.java +++ /dev/null @@ -1,163 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.integration.model; - -import java.util.HashSet; -import java.util.Set; -import java.util.function.Supplier; - -import org.junit.Assert; -import org.junit.Test; - -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.ddl.SchemaGenerators; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.Model; -import org.apache.cassandra.harry.model.QuiescentChecker; -import org.apache.cassandra.harry.model.SelectHelper; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.operations.CompiledStatement; -import org.apache.cassandra.harry.visitors.MutatingVisitor; -import org.apache.cassandra.harry.visitors.MutatingRowVisitor; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.operations.QueryGenerator; -import org.apache.cassandra.harry.visitors.Visitor; - -import static org.apache.cassandra.harry.gen.DataGenerators.NIL_DESCR; - -public class QuerySelectorTest extends IntegrationTestBase -{ - private static int CYCLES = 300; - - @Test - public void basicQuerySelectorTest() - { - Supplier<SchemaSpec> schemaGen = SchemaGenerators.progression(SchemaGenerators.DEFAULT_SWITCH_AFTER); - for (int cnt = 0; cnt < SchemaGenerators.DEFAULT_RUNS; cnt++) - { - beforeEach(); - SchemaSpec schemaSpec = schemaGen.get(); - int partitionSize = 200; - - int[] fractions = new int[schemaSpec.clusteringKeys.size()]; - int last = partitionSize; - for (int i = fractions.length - 1; i >= 0; i--) - { - fractions[i] = last; - last = last / 2; - } - - Configuration config = sharedConfiguration(cnt, schemaSpec) - .setClusteringDescriptorSelector(sharedCDSelectorConfiguration() - .setMaxPartitionSize(partitionSize) - .setFractions(fractions) - .build()) - .build(); - - Run run = config.createRun(); - run.sut.schemaChange(run.schemaSpec.compile().cql()); - - Visitor visitor = new MutatingVisitor(run, MutatingRowVisitor::new); - - for (int i = 0; i < CYCLES; i++) - visitor.visit(); - - QueryGenerator.TypedQueryGenerator querySelector = new QueryGenerator.TypedQueryGenerator(run); - - for (int i = 0; i < CYCLES; i++) - { - Query query = querySelector.inflate(i, i); - - Object[][] results = run.sut.execute(query.toSelectStatement(), SystemUnderTest.ConsistencyLevel.QUORUM); - Set<Long> matchingClusterings = new HashSet<>(); - for (Object[] row : results) - { - long cd = SelectHelper.resultSetToRow(run.schemaSpec, - run.clock, - row).cd; - matchingClusterings.add(cd); - } - - // the simplest test there can be: every row that is in the partition and was returned by the query, - // has to "match", every other row has to be a non-match - CompiledStatement selectPartition = SelectHelper.select(run.schemaSpec, run.pdSelector.pd(i, schemaSpec)); - Object[][] partition = run.sut.execute(selectPartition, SystemUnderTest.ConsistencyLevel.QUORUM); - for (Object[] row : partition) - { - long cd = SelectHelper.resultSetToRow(run.schemaSpec, - run.clock, - row).cd; - - // Skip static clustering - if (cd == NIL_DESCR) - continue; - - boolean expected = matchingClusterings.contains(cd); - boolean actual = query.matchCd(cd); - Assert.assertEquals(String.format("Mismatch for clustering: %d. Expected: %s. Actual: %s.\nQuery: %s", - cd, expected, actual, query.toSelectStatement()), - expected, - actual); - } - } - } - } - - @Test - public void querySelectorModelTest() - { - Supplier<SchemaSpec> gen = SchemaGenerators.progression(SchemaGenerators.DEFAULT_SWITCH_AFTER); - for (int cnt = 0; cnt < SchemaGenerators.DEFAULT_RUNS; cnt++) - { - SchemaSpec schemaSpec = gen.get(); - int[] fractions = new int[schemaSpec.clusteringKeys.size()]; - int partitionSize = 200; - int last = partitionSize; - for (int i = fractions.length - 1; i >= 0; i--) - { - fractions[i] = last; - last = last / 2; - } - - Configuration config = sharedConfiguration(cnt, schemaSpec) - .setClusteringDescriptorSelector(sharedCDSelectorConfiguration() - .setMaxPartitionSize(partitionSize) - .setFractions(fractions) - .build()) - .build(); - Run run = config.createRun(); - run.sut.schemaChange(run.schemaSpec.compile().cql()); - Visitor visitor = new MutatingVisitor(run, MutatingRowVisitor::new); - - for (int i = 0; i < CYCLES; i++) - visitor.visit(); - - QueryGenerator.TypedQueryGenerator querySelector = new QueryGenerator.TypedQueryGenerator(run); - Model model = new QuiescentChecker(run); - - long verificationLts = 10; - for (int i = 0; i < CYCLES; i++) - { - Query query = querySelector.inflate(verificationLts, i); - model.validate(query); - } - } - } -} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/QuiescentCheckerIntegrationTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/QuiescentCheckerIntegrationTest.java deleted file mode 100644 index e88dd30d7e2d..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/QuiescentCheckerIntegrationTest.java +++ /dev/null @@ -1,224 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.integration.model; - -import java.util.Arrays; -import java.util.Collections; -import java.util.concurrent.TimeUnit; - -import org.junit.Test; - -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Configuration.AllPartitionsValidatorConfiguration; -import org.apache.cassandra.harry.core.Configuration.ConcurrentRunnerConfig; -import org.apache.cassandra.harry.core.Configuration.LoggingVisitorConfiguration; -import org.apache.cassandra.harry.core.Configuration.RecentPartitionsValidatorConfiguration; -import org.apache.cassandra.harry.core.Configuration.SequentialRunnerConfig; -import org.apache.cassandra.harry.core.Configuration.SingleVisitRunnerConfig; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.corruptor.AddExtraRowCorruptor; -import org.apache.cassandra.harry.corruptor.ChangeValueCorruptor; -import org.apache.cassandra.harry.corruptor.HideRowCorruptor; -import org.apache.cassandra.harry.corruptor.HideValueCorruptor; -import org.apache.cassandra.harry.corruptor.QueryResponseCorruptor; -import org.apache.cassandra.harry.corruptor.QueryResponseCorruptor.SimpleQueryResponseCorruptor; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.Model; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.runner.StagedRunner.StagedRunnerConfig; -import org.apache.cassandra.harry.visitors.QueryLogger; -import org.apache.cassandra.harry.visitors.SingleValidator; - -public class QuiescentCheckerIntegrationTest extends ModelTestBase -{ - private final long CORRUPT_LTS = 0L; - - @Override - protected SingleValidator validator(Run run) - { - return new SingleValidator(100, run, modelConfiguration()) { - public void visit() - { - visit(CORRUPT_LTS); - } - }; - } - - @Test - public void testNormalCondition() throws Throwable - { - negativeTest((run) -> true, - (t, run) -> { - if (!(t instanceof ShouldHaveThrownException)) - throw new AssertionError(String.format("Throwable was supposed to be null. Schema: %s", - run.schemaSpec.compile().cql()), - t); - }); - } - - @Test - public void normalConditionIntegrationTest() throws Throwable - { - Model.ModelFactory factory = modelConfiguration(); - - SequentialRunnerConfig sequential = - new SequentialRunnerConfig(Arrays.asList(new LoggingVisitorConfiguration(new Configuration.MutatingRowVisitorConfiguration()), - new RecentPartitionsValidatorConfiguration(10, 10, factory::make, () -> QueryLogger.NO_OP), - new AllPartitionsValidatorConfiguration(10, factory::make, () -> QueryLogger.NO_OP)), - 1, TimeUnit.MINUTES); - negativeIntegrationTest(sequential); - } - - @Test - public void normalConditionStagedIntegrationTest() throws Throwable - { - Model.ModelFactory factory = modelConfiguration(); - - ConcurrentRunnerConfig concurrent = - new ConcurrentRunnerConfig(Arrays.asList(new Configuration.VisitorPoolConfiguration("Writer", 4, new Configuration.MutatingVisitorConfiguation(new Configuration.MutatingRowVisitorConfiguration()))), - 30, TimeUnit.SECONDS); - SingleVisitRunnerConfig sequential = - new SingleVisitRunnerConfig(Collections.singletonList(new RecentPartitionsValidatorConfiguration(1024, 0, factory::make, () -> QueryLogger.NO_OP))); - - StagedRunnerConfig staged = new StagedRunnerConfig(Arrays.asList(concurrent, sequential), 2, TimeUnit.MINUTES); - - negativeIntegrationTest(staged); - } - - @Test - public void testDetectsMissingRow() throws Throwable - { - negativeTest((run) -> { - SimpleQueryResponseCorruptor corruptor = new SimpleQueryResponseCorruptor(run.schemaSpec, - run.clock, - HideRowCorruptor::new); - - Query query = Query.selectAllColumns(run.schemaSpec, - run.pdSelector.pd(CORRUPT_LTS, run.schemaSpec), - false); - - return corruptor.maybeCorrupt(query, run.sut); - }, - (t, run) -> { - // TODO: We can actually pinpoint the difference - String expected = "Expected results to have the same number of results, but expected result iterator has more results"; - String expected2 = "Found a row in the model that is not present in the resultset"; - - if (t.getMessage().contains(expected) || t.getMessage().contains(expected2)) - return; - - throw new AssertionError(String.format("Exception string mismatch.\nExpected error: %s.\nActual error: %s", expected, t.getMessage()), - t); - }); - } - - @Test - public void testDetectsExtraRow() throws Throwable - { - negativeTest((run) -> { - QueryResponseCorruptor corruptor = new AddExtraRowCorruptor(run.schemaSpec, - run.clock, - run.tracker, - run.descriptorSelector); - - return corruptor.maybeCorrupt(Query.selectAllColumns(run.schemaSpec, - run.pdSelector.pd(CORRUPT_LTS, run.schemaSpec), - false), - run.sut); - }, - (t, run) -> { - String expected = "Found a row in the model that is not present in the resultset"; - String expected2 = "Expected results to have the same number of results, but actual result iterator has more results"; - String expected3 = "Found a row while model predicts statics only"; - if (t.getMessage().contains(expected) || t.getMessage().contains(expected2) || t.getMessage().contains(expected3)) - return; - - throw new AssertionError(String.format("Exception string mismatch.\nExpected error: %s.\nActual error: %s", expected, t.getMessage()), - t); - }); - } - - - @Test - public void testDetectsRemovedColumn() throws Throwable - { - negativeTest((run) -> { - SimpleQueryResponseCorruptor corruptor = new SimpleQueryResponseCorruptor(run.schemaSpec, - run.clock, - HideValueCorruptor::new); - - return corruptor.maybeCorrupt(Query.selectAllColumns(run.schemaSpec, - run.pdSelector.pd(CORRUPT_LTS, run.schemaSpec), - false), - run.sut); - }, - (t, run) -> { - String expected = "doesn't match the one predicted by the model"; - String expected2 = "don't match ones predicted by the model"; - String expected3 = "Found a row in the model that is not present in the resultset"; - if (t.getMessage().contains(expected) || t.getMessage().contains(expected2) || t.getMessage().contains(expected3)) - return; - - throw new AssertionError(String.format("Exception string mismatch.\nExpected error: %s.\nActual error: %s", expected, t.getMessage()), - t); - }); - } - - - @Test - public void testDetectsOverwrittenRow() throws Throwable - { - negativeTest((run) -> { - SimpleQueryResponseCorruptor corruptor = new SimpleQueryResponseCorruptor(run.schemaSpec, - run.clock, - ChangeValueCorruptor::new); - - return corruptor.maybeCorrupt(Query.selectAllColumns(run.schemaSpec, - run.pdSelector.pd(CORRUPT_LTS, run.schemaSpec), - false), - run.sut); - }, - (t, run) -> { - String expected = "Returned row state doesn't match the one predicted by the model"; - String expected2 = "Timestamps in the row state don't match ones predicted by the model"; - - if (t.getMessage() != null && - (t.getMessage().contains(expected) || t.getMessage().contains(expected2))) - return; - - throw new AssertionError(String.format("Exception string mismatch.\nExpected error: %s.\nActual error: %s", expected, t.getMessage()), - t); - }); - } - - @Override - protected Configuration.ModelConfiguration modelConfiguration() - { - return new Configuration.QuiescentCheckerConfig(); - } - - public Configuration.ConfigurationBuilder configuration(long seed, SchemaSpec schema) - { - return super.configuration(seed, schema) - .setClusteringDescriptorSelector(sharedCDSelectorConfiguration() - .setOperationsPerLtsDistribution(new Configuration.ConstantDistributionConfig(2)) - .setMaxPartitionSize(100) - .build()); - } -} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/QuiescentLocalStateCheckerIntegrationTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/QuiescentLocalStateCheckerIntegrationTest.java deleted file mode 100644 index 2ebfaa55cd5f..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/QuiescentLocalStateCheckerIntegrationTest.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.integration.model; - -import java.util.concurrent.TimeUnit; -import java.util.function.Supplier; - -import org.junit.BeforeClass; -import org.junit.Test; - -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.ddl.SchemaGenerators; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.sut.injvm.InJvmSut; -import org.apache.cassandra.harry.sut.injvm.InJvmSutBase; -import org.apache.cassandra.harry.sut.injvm.QuiescentLocalStateChecker; -import org.apache.cassandra.harry.runner.Runner; -import org.apache.cassandra.harry.visitors.AllPartitionsValidator; -import org.apache.cassandra.harry.visitors.MutatingVisitor; -import org.apache.cassandra.distributed.Cluster; -import org.apache.cassandra.distributed.api.Feature; - -public class QuiescentLocalStateCheckerIntegrationTest extends ModelTestBase -{ - @BeforeClass - public static void before() throws Throwable - { - cluster = init(Cluster.build() - .withNodes(2) - .withConfig(InJvmSutBase.defaultConfig().andThen((cfg) -> cfg.with(Feature.GOSSIP))) - .start()); - sut = new InJvmSut(cluster); - } - - @Test - public void testQuiescentLocalStateChecker() throws Throwable - { - Supplier<SchemaSpec> supplier = SchemaGenerators.progression(1); - for (int i = 0; i < SchemaGenerators.GENERATORS_COUNT; i++) - { - SchemaSpec schema = supplier.get(); - - Configuration config = configuration(i, schema) - .setKeyspaceDdl("CREATE KEYSPACE IF NOT EXISTS " + schema.keyspace + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1};") - .setCreateSchema(true) - .setDropSchema(true) - .build(); - - Runner.chain(config, - Runner.sequential(MutatingVisitor::new, 2, TimeUnit.SECONDS), - Runner.single(AllPartitionsValidator.factoryForTests(1, QuiescentLocalStateChecker.factory(new TokenPlacementModel.SimpleReplicationFactor(1))))) - .run(); - break; - } - } - - @Override - protected Configuration.ModelConfiguration modelConfiguration() - { - return new Configuration.QuiescentCheckerConfig(); - } -} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/ReconcilerIntegrationTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/ReconcilerIntegrationTest.java deleted file mode 100644 index 707fc70ca095..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/ReconcilerIntegrationTest.java +++ /dev/null @@ -1,93 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.integration.model; - -import java.util.Arrays; - -import org.junit.Test; - -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.dsl.ReplayingHistoryBuilder; -import org.apache.cassandra.harry.dsl.ValueDescriptorIndexGenerator; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.tracker.DefaultDataTracker; - -public class ReconcilerIntegrationTest extends IntegrationTestBase -{ - private static final long SEED = 1; - - @Test - public void testTrackingWithStatics() throws Throwable - { - SchemaSpec schema = new SchemaSpec(KEYSPACE, "tbl1", - Arrays.asList(ColumnSpec.pk("pk1", ColumnSpec.int64Type)), - Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.int64Type)), - Arrays.asList(ColumnSpec.regularColumn("v1", ColumnSpec.int64Type), - ColumnSpec.regularColumn("v2", ColumnSpec.int64Type)), - Arrays.asList(ColumnSpec.staticColumn("static1", ColumnSpec.int64Type), - ColumnSpec.staticColumn("static2", ColumnSpec.int64Type))) - .trackLts(); - - beforeEach(); - sut.schemaChange(schema.compile().cql()); - - TokenPlacementModel.ReplicationFactor rf = new TokenPlacementModel.SimpleReplicationFactor(1); - ReplayingHistoryBuilder historyBuilder = new ReplayingHistoryBuilder(SEED, 100, 1, new DefaultDataTracker(), sut, schema, rf, SystemUnderTest.ConsistencyLevel.QUORUM); - historyBuilder.visitPartition(1).insert(1, - new long[]{ ValueDescriptorIndexGenerator.UNSET, ValueDescriptorIndexGenerator.UNSET }, - new long[]{ 1, 1 }); - historyBuilder.validate(1); - - historyBuilder.visitPartition(2).insert(2, - new long[]{ 1, 1 }, - new long[]{ 1, 1 }); - historyBuilder.visitPartition(2).deleteRowRange(1, 3, true, true); - historyBuilder.validate(2); - } - - @Test - public void testTrackingWithoutStatics() throws Throwable - { - SchemaSpec schema = new SchemaSpec(KEYSPACE, "tbl1", - Arrays.asList(ColumnSpec.pk("pk1", ColumnSpec.int64Type)), - Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.int64Type)), - Arrays.asList(ColumnSpec.regularColumn("v1", ColumnSpec.int64Type), - ColumnSpec.regularColumn("v2", ColumnSpec.int64Type)), - Arrays.asList()) - .trackLts(); - - beforeEach(); - sut.schemaChange(schema.compile().cql()); - - TokenPlacementModel.ReplicationFactor rf = new TokenPlacementModel.SimpleReplicationFactor(1); - ReplayingHistoryBuilder historyBuilder = new ReplayingHistoryBuilder(SEED, 100, 1, new DefaultDataTracker(), sut, schema, rf, SystemUnderTest.ConsistencyLevel.QUORUM); - historyBuilder.visitPartition(2).insert(2, - new long[]{ 1, 1 }); - historyBuilder.visitPartition(2).deleteRowRange(1, 3, true, true); - historyBuilder.validate(2); - - historyBuilder = new ReplayingHistoryBuilder(SEED, 100, 1, new DefaultDataTracker(), sut, schema, rf, SystemUnderTest.ConsistencyLevel.QUORUM); - historyBuilder.visitPartition(2).insert(2, - new long[]{ 1, 1 }); - historyBuilder.visitPartition(2).deleteRowRange(1, 3, true, true); - historyBuilder.validate(2); - } -} diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/reconciler/SimpleReconcilerTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/reconciler/SimpleReconcilerTest.java deleted file mode 100644 index 3f3d4135e18b..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/integration/model/reconciler/SimpleReconcilerTest.java +++ /dev/null @@ -1,341 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.integration.model.reconciler; - -import java.util.*; - -import org.junit.Test; - -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.ddl.SchemaGenerators; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.gen.Surjections; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.model.QuiescentChecker; -import org.apache.cassandra.harry.model.SelectHelper; -import org.apache.cassandra.harry.model.reconciler.PartitionState; -import org.apache.cassandra.harry.sut.injvm.InJvmSut; -import org.apache.cassandra.harry.operations.CompiledStatement; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.operations.WriteHelper; -import org.apache.cassandra.harry.util.BitSet; -import org.apache.cassandra.distributed.api.ConsistencyLevel; -import org.apache.cassandra.fuzz.harry.integration.model.IntegrationTestBase; - -public class SimpleReconcilerTest extends IntegrationTestBase -{ - public static Surjections.Surjection<SchemaSpec> defaultSchemaSpecGen(String ks, String table) - { - return new SchemaGenerators.Builder(ks, () -> table) - .partitionKeySpec(1, 3, - ColumnSpec.int64Type, - ColumnSpec.asciiType(5, 256)) - .clusteringKeySpec(1, 3, - ColumnSpec.int64Type, - ColumnSpec.asciiType(2, 3), - ColumnSpec.ReversedType.getInstance(ColumnSpec.int64Type), - ColumnSpec.ReversedType.getInstance(ColumnSpec.asciiType(2, 3))) - .regularColumnSpec(50, 50, - ColumnSpec.int64Type, - ColumnSpec.asciiType(5, 256)) - .staticColumnSpec(50, 50, - ColumnSpec.int64Type, - ColumnSpec.asciiType(4, 256)) - .surjection(); - } - - @Test - public void testStatics() throws Throwable - { - int rowsPerPartition = 50; - SchemaSpec schema = defaultSchemaSpecGen("harry", "tbl").inflate(1); - Configuration config = sharedConfiguration(1, schema).build(); - Run run = config.createRun(); - SyntheticTest test = new SyntheticTest(run.rng, schema); - beforeEach(); - cluster.schemaChange(schema.compile().cql()); - - ModelState state = new ModelState(new HashMap<>()); - InJvmSut sut = (InJvmSut) run.sut; - Random rng = new Random(1); - - int partitionIdx = 0; - - for (int i = 0; i < 100; i++) - { - BitSet subset = BitSet.allUnset(schema.allColumns.size()); - for (int j = 0; j < subset.size(); j++) - { - if (rng.nextBoolean()) - subset.set(j); - } - if (!isValidSubset(schema.allColumns, subset)) - continue; - int pdIdx = partitionIdx++; - long pd = test.pd(pdIdx); - - for (int j = 0; j < 10; j++) - { - int cdIdx = rng.nextInt(rowsPerPartition); - long cd = test.cd(pdIdx, cdIdx); - - long[] vds = run.descriptorSelector.descriptors(pd, cd, state.lts, 0, schema.regularColumns, - schema.regularColumnsMask(), - subset, - schema.regularColumnsOffset); - long[] sds = run.descriptorSelector.descriptors(pd, cd, state.lts, 0, schema.staticColumns, - schema.staticColumnsMask, - subset, - schema.staticColumnsOffset); - - CompiledStatement statement = WriteHelper.inflateUpdate(schema, pd, cd, vds, sds, run.clock.rts(state.lts)); - sut.cluster.coordinator(1).execute(statement.cql(), ConsistencyLevel.QUORUM, statement.bindings()); - - PartitionState partitionState = state.state.get(pd); - if (partitionState == null) - { - partitionState = new PartitionState(pd, -1, schema); - state.state.put(pd, partitionState); - } - - partitionState.writeStaticRow(sds, state.lts); - partitionState.write(cd, vds, state.lts, true); - - state.lts++; - } - } - - // Validate that all partitions correspond to our expectations - for (Long pd : state.state.keySet()) - { - ArrayList<Long> clusteringDescriptors = new ArrayList<>(state.state.get(pd).rows().keySet()); - - // TODO: allow sub-selection - // Try different column subsets - for (int i = 0; i < 10; i++) - { - BitSet bitset = BitSet.allUnset(schema.allColumns.size()); - for (int j = 0; j < bitset.size(); j++) - { - if (rng.nextBoolean()) - bitset.set(j); - } - Set<ColumnSpec<?>> subset = i == 0 ? null : subset(schema.allColumns, bitset); - if (subset != null && !isValidSubset(schema.allColumns, bitset)) - continue; - - int a = rng.nextInt(clusteringDescriptors.size()); - long cd1tmp = clusteringDescriptors.get(a); - long cd2tmp; - int b; - while (true) - { - b = rng.nextInt(clusteringDescriptors.size()); - long tmp = clusteringDescriptors.get(b); - if (tmp != cd1tmp) - { - cd2tmp = tmp; - break; - } - } - - long cd1 = Math.min(cd1tmp, cd2tmp); - long cd2 = Math.max(cd1tmp, cd2tmp); - - for (boolean reverse : new boolean[]{ true, false }) - { - Query query; - - query = Query.selectAllColumns(schema, pd, reverse); - - QuiescentChecker.validate(schema, - run.tracker, - subset, - state.state.get(pd), - SelectHelper.execute(sut, run.clock, query, subset), - query); - - query = Query.singleClustering(schema, pd, cd1, false); - QuiescentChecker.validate(schema, - run.tracker, - subset, - state.state.get(pd).apply(query), - SelectHelper.execute(sut, run.clock, query, subset), - query); - - for (boolean isGt : new boolean[]{ true, false }) - { - for (boolean isEquals : new boolean[]{ true, false }) - { - try - { - query = Query.clusteringSliceQuery(schema, pd, cd1, rng.nextLong(), isGt, isEquals, reverse); - } - catch (IllegalArgumentException impossibleQuery) - { - continue; - } - - QuiescentChecker.validate(schema, - run.tracker, - subset, - state.state.get(pd).apply(query), - SelectHelper.execute(sut, run.clock, query, subset), - query); - } - } - - for (boolean isMinEq : new boolean[]{ true, false }) - { - for (boolean isMaxEq : new boolean[]{ true, false }) - { - try - { - query = Query.clusteringRangeQuery(schema, pd, cd1, cd2, rng.nextLong(), isMinEq, isMaxEq, reverse); - } - catch (IllegalArgumentException impossibleQuery) - { - continue; - } - QuiescentChecker.validate(schema, - run.tracker, - subset, - state.state.get(pd).apply(query), - SelectHelper.execute(sut, run.clock, query, subset), - query); - } - } - } - } - } - } - - public static boolean isValidSubset(List<ColumnSpec<?>> superset, BitSet bitSet) - { - boolean hasRegular = false; - for (int i = 0; i < superset.size(); i++) - { - ColumnSpec<?> column = superset.get(i); - if (column.kind == ColumnSpec.Kind.PARTITION_KEY && !bitSet.isSet(i)) - return false; - if (column.kind == ColumnSpec.Kind.CLUSTERING && !bitSet.isSet(i)) - return false; - if (column.kind == ColumnSpec.Kind.REGULAR && bitSet.isSet(i)) - hasRegular = true; - } - - return hasRegular; - } - - public static Set<ColumnSpec<?>> subset(List<ColumnSpec<?>> superset, BitSet bitSet) - { - Set<ColumnSpec<?>> subset = new HashSet<>(); - for (int i = 0; i < superset.size(); i++) - { - if (bitSet.isSet(i)) - subset.add(superset.get(i)); - } - - return subset; - } - - public static Set<ColumnSpec<?>> randomSubset(List<ColumnSpec<?>> superset, Random e) - { - Set<ColumnSpec<?>> set = new HashSet<>(); - boolean hadRegular = false; - for (ColumnSpec<?> v : superset) - { - // TODO: allow selecting without partition and clustering key, too - if (e.nextBoolean() || v.kind == ColumnSpec.Kind.CLUSTERING || v.kind == ColumnSpec.Kind.PARTITION_KEY) - { - set.add(v); - hadRegular |= v.kind == ColumnSpec.Kind.REGULAR; - } - } - - // TODO: this is an oversimplification and a workaround for "Invalid restrictions on clustering columns since the UPDATE statement modifies only static columns" - if (!hadRegular) - return randomSubset(superset, e); - - return set; - } - - public static <T> BitSet subsetToBitset(List<T> superset, Set<T> subset) - { - BitSet bitSet = new BitSet.BitSet64Bit(superset.size()); - for (int i = 0; i < superset.size(); i++) - { - if (subset.contains(superset.get(i))) - bitSet.set(i); - } - return bitSet; - } - - public static class ModelState - { - public long lts = 0; - public final Map<Long, PartitionState> state; - - public ModelState(Map<Long, PartitionState> state) - { - this.state = state; - } - } - - public static class SyntheticTest // TODO: horrible name - { - private static long PD_STREAM = System.nanoTime(); - private final OpSelectors.PureRng rng; - private final SchemaSpec schema; - - public SyntheticTest(OpSelectors.PureRng rng, SchemaSpec schema) - { - this.schema = schema; - this.rng = rng; - } - - public long pd(int pdIdx) - { - long pd = this.rng.randomNumber(pdIdx + 1, PD_STREAM); - long adjusted = schema.adjustPdEntropy(pd); - assert adjusted == pd : "Partition descriptors not utilising all entropy bits are not supported."; - return pd; - } - - public long pdIdx(long pd) - { - return this.rng.sequenceNumber(pd, PD_STREAM) - 1; - } - - public long cd(int pdIdx, int cdIdx) - { - long cd = this.rng.randomNumber(cdIdx + 1, pd(pdIdx)); - long adjusted = schema.adjustCdEntropy(cd); - assert adjusted == cd : "Clustering descriptors not utilising all entropy bits are not supported."; - return cd; - } - - public long cdIdx(long pd) - { - return this.rng.sequenceNumber(pd, PD_STREAM) - 1; - } - } -} diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/integration/op/RowVisitorTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/integration/op/RowVisitorTest.java deleted file mode 100644 index 5978f1e87732..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/integration/op/RowVisitorTest.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.integration.op; - -import java.util.function.Supplier; - -import org.junit.Before; -import org.junit.Test; - -import org.apache.cassandra.harry.core.MetricReporter; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.ddl.SchemaGenerators; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.gen.EntropySource; -import org.apache.cassandra.harry.gen.distribution.Distribution; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.clock.OffsetClock; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.tracker.DataTracker; -import org.apache.cassandra.harry.visitors.GeneratingVisitor; -import org.apache.cassandra.harry.visitors.MutatingRowVisitor; -import org.apache.cassandra.harry.visitors.MutatingVisitor; -import org.apache.cassandra.cql3.CQLTester; - -import static org.apache.cassandra.harry.model.OpSelectors.DefaultDescriptorSelector.DEFAULT_OP_SELECTOR; - -public class RowVisitorTest extends CQLTester -{ - @Before - public void beforeTest() throws Throwable { - super.beforeTest(); - schemaChange(String.format("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", SchemaGenerators.DEFAULT_KEYSPACE_NAME)); - } - - @Test - public void rowWriteGeneratorTest() - { - Supplier<SchemaSpec> specGenerator = SchemaGenerators.progression(SchemaGenerators.DEFAULT_SWITCH_AFTER); - EntropySource rand = EntropySource.forTests(6371747244598697093L); - - OpSelectors.PureRng rng = new OpSelectors.PCGFast(1); - - OpSelectors.PdSelector pdSelector = new OpSelectors.DefaultPdSelector(rng, 10, 10); - - for (int i = 0; i < SchemaGenerators.DEFAULT_RUNS; i++) - { - SchemaSpec schema = specGenerator.get(); - createTable(schema.compile().cql()); - - OpSelectors.DescriptorSelector descriptorSelector = new OpSelectors.DefaultDescriptorSelector(rng, - new OpSelectors.ColumnSelectorBuilder().forAll(schema) - .build(), - DEFAULT_OP_SELECTOR, - new Distribution.ScaledDistribution(1, 30), - 100); - - Run run = new Run(rng, - new OffsetClock(10000), - pdSelector, - descriptorSelector, - schema, - DataTracker.NO_OP, - SystemUnderTest.NO_OP, - MetricReporter.NO_OP); - - long[] descriptors = new long[4]; - for (int j = 0; j < descriptors.length; j++) - descriptors[j] = rand.next(); - - // some improvement wont hurt here - new GeneratingVisitor(run, new MutatingVisitor(run, MutatingRowVisitor::new)).visit(); - } - } -} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/model/ApproximateClockTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/model/ApproximateClockTest.java deleted file mode 100644 index 4751b4789cdf..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/model/ApproximateClockTest.java +++ /dev/null @@ -1,180 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.model; - -import java.util.Iterator; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.LockSupport; -import java.util.concurrent.locks.ReentrantLock; - -import org.junit.Assert; -import org.junit.Test; - -import org.apache.cassandra.harry.clock.ApproximateClock; - -public class ApproximateClockTest -{ - @Test - public void approximateClockTest() throws InterruptedException - { - ConcurrentHashMap<Long, Long> m = new ConcurrentHashMap<>(); - ConcurrentHashMap<Long, Long> inverse = new ConcurrentHashMap<>(); - TimeUnit timeUnit = TimeUnit.MILLISECONDS; - int duration = 1000; - int concurrency = 5; - long maxTicks = timeUnit.toMicros(duration) / (4 * concurrency); - - ApproximateClock clock = new ApproximateClock(duration, timeUnit); - - ScheduledExecutorService scheduledExecutor = Executors.newScheduledThreadPool(1); - ExecutorService executor = Executors.newFixedThreadPool(concurrency); - final Lock lock = new ReentrantLock(); - AtomicReference<Throwable> throwable = new AtomicReference(); - final Condition signalError = lock.newCondition(); - lock.lock(); - for (int i = 0; i < concurrency; i++) - { - executor.submit(() -> { - try - { - int sleepCnt = 0; - - while (!executor.isShutdown() && !Thread.currentThread().isInterrupted()) - { - sleepCnt++; - if (sleepCnt >= maxTicks) - { - LockSupport.parkNanos(timeUnit.toNanos(duration)); - sleepCnt = 0; - } - - if (executor.isShutdown() || Thread.currentThread().isInterrupted()) - return; - - long lts = clock.nextLts(); - - // Make sure to test "history" path - if (lts % 10000 == 0) - { - scheduledExecutor.schedule(() -> { - try - { - long rts = clock.rts(lts); - Assert.assertNull(m.put(lts, rts)); - Assert.assertNull(inverse.put(rts, lts)); - } - catch (Throwable t) - { - throwable.set(t); - signalError.signalAll(); - t.printStackTrace(); - } - }, 2 * duration, timeUnit); - continue; - } - - try - { - long rts = clock.rts(lts); - Assert.assertNull(m.put(lts, rts)); - Assert.assertNull(inverse.put(rts, lts)); - } - catch (Throwable t) - { - throwable.set(t); - signalError.signalAll(); - } - } - } - catch (Throwable t) - { - throwable.set(t); - signalError.signalAll(); - t.printStackTrace(); - } - }); - } - signalError.await(10, TimeUnit.SECONDS); - lock.unlock(); - executor.shutdown(); - Assert.assertTrue(executor.awaitTermination(30, TimeUnit.SECONDS)); - scheduledExecutor.shutdown(); - Assert.assertTrue(scheduledExecutor.awaitTermination(10, TimeUnit.SECONDS)); - Throwable t = throwable.get(); - if (t != null) - throw new AssertionError("Caught an exception while executing", t); - - Assert.assertEquals(m.size(), inverse.size()); - Iterator<Map.Entry<Long, Long>> iter = m.entrySet().iterator(); - - Map.Entry<Long, Long> previous = iter.next(); - while (iter.hasNext()) - { - if (previous == null) - { - previous = iter.next(); - continue; - } - - Map.Entry<Long, Long> current = iter.next(); - long lts = current.getKey(); - long rts = current.getValue(); - Assert.assertEquals(String.format("%s and %s sort wrong", previous, current), - Long.compare(previous.getKey(), current.getKey()), - Long.compare(previous.getValue(), current.getValue())); - - Assert.assertEquals(clock.rts(lts), rts); - Assert.assertEquals(clock.lts(rts), lts); - previous = current; - } - } - - @Test - public void approximateClockInvertibilityTest() - { - ConcurrentHashMap<Long, Long> m = new ConcurrentHashMap<>(); - TimeUnit timeUnit = TimeUnit.MILLISECONDS; - int duration = 100; - int cycles = 10_000; - ApproximateClock clock = new ApproximateClock(duration, timeUnit); - - for (long i = 0; i < cycles; i++) - { - long lts = clock.nextLts(); - Assert.assertEquals(lts, i); - long rts = clock.rts(lts); - Assert.assertNull(m.put(lts, rts)); - } - - for (Map.Entry<Long, Long> entry : m.entrySet()) - { - Assert.assertEquals(entry.getKey(), - Long.valueOf(clock.lts(entry.getValue()))); - } - } - -} diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/model/OpSelectorsTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/model/OpSelectorsTest.java deleted file mode 100644 index e9984c1ef316..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/model/OpSelectorsTest.java +++ /dev/null @@ -1,418 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.model; - -import java.util.*; -import java.util.function.BiConsumer; -import java.util.function.Supplier; - -import org.junit.Assert; -import org.junit.Test; - -import org.apache.cassandra.harry.core.MetricReporter; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.ddl.SchemaGenerators; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.gen.Surjections; -import org.apache.cassandra.harry.gen.distribution.Distribution; -import org.apache.cassandra.harry.clock.OffsetClock; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.operations.CompiledStatement; -import org.apache.cassandra.harry.tracker.DataTracker; -import org.apache.cassandra.harry.visitors.LtsVisitor; -import org.apache.cassandra.harry.visitors.MutatingVisitor; -import org.apache.cassandra.harry.visitors.OperationExecutor; -import org.apache.cassandra.harry.util.BitSet; -import org.apache.cassandra.harry.visitors.VisitExecutor; - -public class OpSelectorsTest -{ - private static int RUNS = 1000; - private static SchemaSpec SCHEMA = new SchemaSpec("ks", "tbl", - Collections.singletonList(ColumnSpec.pk("pk", ColumnSpec.int64Type)), - Collections.singletonList(ColumnSpec.pk("ck", ColumnSpec.int64Type)), - Collections.emptyList(), Collections.emptyList()); - - @Test - public void testRowDataDescriptorSupplier() - { - OpSelectors.PureRng rng = new OpSelectors.PCGFast(1); - SchemaSpec schema = new SchemaSpec("ks", "tbl1", - Arrays.asList(ColumnSpec.pk("pk1", ColumnSpec.asciiType), - ColumnSpec.pk("pk2", ColumnSpec.int64Type)), - Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.asciiType, false), - ColumnSpec.ck("ck2", ColumnSpec.int64Type, false)), - Arrays.asList(ColumnSpec.regularColumn("v1", ColumnSpec.int32Type), - ColumnSpec.regularColumn("v2", ColumnSpec.int64Type)), - Arrays.asList(ColumnSpec.staticColumn("static1", ColumnSpec.asciiType), - ColumnSpec.staticColumn("static2", ColumnSpec.int64Type))); - OpSelectors.DefaultDescriptorSelector descriptorSelector = new OpSelectors.DefaultDescriptorSelector(rng, - new OpSelectors.ColumnSelectorBuilder().forAll(schema) - .build(), - OpSelectors.DefaultDescriptorSelector.DEFAULT_OP_SELECTOR, - new Distribution.ScaledDistribution(2, 10), - 50); - - OpSelectors.PdSelector pdSupplier = new OpSelectors.DefaultPdSelector(rng, - 100, - 100); - - for (int lts = 0; lts < RUNS; lts++) - { - long pd = pdSupplier.pd(lts, schema); - - int opsPerLts = descriptorSelector.operationsPerLts(lts); - for (int opId = 0; opId < opsPerLts; opId++) - { - long cd = descriptorSelector.cd(pd, lts, opId, schema); - Assert.assertEquals(opId, descriptorSelector.opId(pd, lts, cd)); - Assert.assertTrue(descriptorSelector.isCdVisitedBy(pd, lts, cd)); - for (int col = 0; col < 10; col++) - { - long vd = descriptorSelector.vd(pd, cd, lts, opId, col); - } - } - } - } - - @Test - public void pdSelectorSymmetryTest() - { - OpSelectors.PureRng rng = new OpSelectors.PCGFast(1); - Supplier<SchemaSpec> gen = SchemaGenerators.progression(SchemaGenerators.DEFAULT_SWITCH_AFTER); - SchemaSpec schema = gen.get(); - - for (long[] positions : new long[][]{ { 0, Long.MAX_VALUE }, { 100, Long.MAX_VALUE }, { 1000, Long.MAX_VALUE } }) - { - for (int repeats = 2; repeats <= 1000; repeats++) - { - for (int windowSize = 2; windowSize <= 10; windowSize++) - { - OpSelectors.DefaultPdSelector pdSelector = new OpSelectors.DefaultPdSelector(rng, windowSize, repeats, positions[0], positions[1]); - - Map<Long, List<Long>> m = new HashMap<>(); - final long maxLts = 10_000; - for (long lts = 0; lts <= maxLts; lts++) - { - long pd = pdSelector.pd(lts, schema); - m.computeIfAbsent(pd, (k) -> new ArrayList<>()).add(lts); - } - - for (Long pd : m.keySet()) - { - long currentLts = pdSelector.minLtsFor(pd); - List<Long> predicted = new ArrayList<>(); - while (currentLts <= maxLts && currentLts >= 0) - { - predicted.add(currentLts); - currentLts = pdSelector.nextLts(currentLts); - } - Assert.assertEquals(m.get(pd), predicted); - } - - - } - } - } - } - - @Test - public void pdSelectorTest() - { - OpSelectors.PureRng rng = new OpSelectors.PCGFast(1); - int cycles = 10000; - for (long[] positions : new long[][]{ { 0, Long.MAX_VALUE }, { 100, Long.MAX_VALUE }, { 1000, Long.MAX_VALUE } }) - { - for (int repeats = 2; repeats <= 1000; repeats++) - { - for (int windowSize = 2; windowSize <= 10; windowSize++) - { - OpSelectors.DefaultPdSelector pdSupplier = new OpSelectors.DefaultPdSelector(rng, windowSize, repeats, positions[0], positions[1]); - long[] pds = new long[cycles]; - for (int i = 0; i < cycles; i++) - { - long pd = pdSupplier.pd(i, SCHEMA); - pds[i] = pd; - Assert.assertEquals(pdSupplier.positionFor(i), pdSupplier.positionForPd(pd)); - } - - Set<Long> noNext = new HashSet<>(); - for (int i = 0; i < cycles; i++) - { - long nextLts = pdSupplier.nextLts(i); - Assert.assertFalse(noNext.contains(pds[i])); - if (nextLts == -1) - { - noNext.add(nextLts); - } - else if (nextLts < cycles) - { - Assert.assertEquals(pds[(int) nextLts], pdSupplier.pd(i, SCHEMA)); - } - } - - Set<Long> noPrev = new HashSet<>(); - for (int i = cycles - 1; i >= 0; i--) - { - long prevLts = pdSupplier.prevLts(i); - Assert.assertFalse(noPrev.contains(pds[i])); - if (prevLts == -1) - { - noPrev.add(prevLts); - } - else if (prevLts >= 0) - { - Assert.assertEquals(pds[(int) prevLts], pdSupplier.pd(i, SCHEMA)); - } - } - - Set<Long> seen = new HashSet<>(); - for (int i = 0; i < cycles; i++) - { - long pd = pdSupplier.pd(i, SCHEMA); - if (!seen.contains(pd)) - { - Assert.assertEquals(i, pdSupplier.minLtsAt(pdSupplier.positionFor(i))); - seen.add(pd); - } - } - - for (int i = 0; i < cycles; i++) - { - long pd = pdSupplier.pd(i, SCHEMA); - long maxLts = pdSupplier.maxLtsFor(pd); - Assert.assertEquals(-1, pdSupplier.nextLts(maxLts)); - Assert.assertEquals(pdSupplier.pd(i, SCHEMA), pdSupplier.pd(maxLts, SCHEMA)); - } - } - } - } - } - - @Test - public void ckSelectorTest() - { - Supplier<SchemaSpec> gen = SchemaGenerators.progression(SchemaGenerators.DEFAULT_SWITCH_AFTER); - for (int i = 0; i < SchemaGenerators.DEFAULT_RUNS; i++) - ckSelectorTest(gen.get()); - } - - public void ckSelectorTest(SchemaSpec schema) - { - OpSelectors.PureRng rng = new OpSelectors.PCGFast(1); - OpSelectors.PdSelector pdSelector = new OpSelectors.DefaultPdSelector(rng, 10, 10); - OpSelectors.DescriptorSelector ckSelector = new OpSelectors.DefaultDescriptorSelector(rng, - new OpSelectors.ColumnSelectorBuilder().forAll(schema, Surjections.pick(schema.regularColumnsMask())).build(), - OpSelectors.OperationSelector.weighted(Surjections.weights(10, 10, 40, 40), - OpSelectors.OperationKind.DELETE_ROW, - OpSelectors.OperationKind.DELETE_COLUMN, - OpSelectors.OperationKind.INSERT, - OpSelectors.OperationKind.UPDATE), - new Distribution.ConstantDistribution(10), - 10); - - Map<Long, Set<Long>> partitionMap = new HashMap<>(); - CompiledStatement compiledStatement = new CompiledStatement(""); - BiConsumer<Long, Long> consumer = (pd, cd) -> { - partitionMap.compute(pd, (pk, list) -> { - if (list == null) - list = new HashSet<>(); - list.add(cd); - return list; - }); - }; - - Run run = new Run(rng, - new OffsetClock(0), - pdSelector, - ckSelector, - schema, - DataTracker.NO_OP, - SystemUnderTest.NO_OP, - MetricReporter.NO_OP); - - LtsVisitor visitor = new MutatingVisitor(run, - (r) -> new OperationExecutor() - { - public CompiledStatement insert(VisitExecutor.WriteOp op) - { - consumer.accept(op.pd(), op.cd()); - return compiledStatement; - } - - public CompiledStatement update(VisitExecutor.WriteOp op) - { - consumer.accept(op.pd(), op.cd()); - return compiledStatement; - } - - public CompiledStatement insertWithStatics(VisitExecutor.WriteStaticOp op) - { - consumer.accept(op.pd(), op.cd()); - return compiledStatement; - } - - public CompiledStatement updateWithStatics(VisitExecutor.WriteStaticOp op) - { - consumer.accept(op.pd(), op.cd()); - return compiledStatement; - } - - public CompiledStatement deleteColumn(VisitExecutor.DeleteColumnsOp op) - { - consumer.accept(op.pd(), op.cd()); - return compiledStatement; - } - - public CompiledStatement deleteColumnWithStatics(VisitExecutor.DeleteColumnsOp op) - { - consumer.accept(op.pd(), op.cd()); - return compiledStatement; - } - - public CompiledStatement deleteRow(VisitExecutor.DeleteRowOp op) - { - consumer.accept(op.pd(), op.cd()); - return compiledStatement; - } - - public CompiledStatement deletePartition(VisitExecutor.DeleteOp op) - { - // ignore - return compiledStatement; - } - - public CompiledStatement deleteRange(VisitExecutor.DeleteOp op) - { - // ignore - return compiledStatement; - } - - public CompiledStatement deleteSlice(VisitExecutor.DeleteOp op) - { - // ignore - return compiledStatement; - } - }); - - for (int lts = 0; lts < 1000; lts++) - visitor.visit(); - - for (Collection<Long> value : partitionMap.values()) - Assert.assertEquals(10, value.size()); - } - - @Test - public void hierarchicalDescriptorSelector() - { - SchemaSpec schema = new SchemaSpec("ks", "tbl1", - Collections.singletonList(ColumnSpec.pk("pk1", ColumnSpec.asciiType)), - Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.asciiType), - ColumnSpec.ck("ck2", ColumnSpec.asciiType), - ColumnSpec.ck("ck3", ColumnSpec.asciiType)), - Collections.singletonList(ColumnSpec.regularColumn("v1", ColumnSpec.asciiType)), - Collections.emptyList()); - - OpSelectors.PureRng rng = new OpSelectors.PCGFast(1); - OpSelectors.DescriptorSelector ckSelector = new OpSelectors.HierarchicalDescriptorSelector(rng, - new int[] {10, 20}, - OpSelectors.columnSelectorBuilder().forAll(schema, Surjections.pick(BitSet.allUnset(0))).build(), - OpSelectors.OperationSelector.weighted(Surjections.weights(10, 10, 40, 40), - OpSelectors.OperationKind.DELETE_ROW, - OpSelectors.OperationKind.DELETE_COLUMN, - OpSelectors.OperationKind.INSERT, - OpSelectors.OperationKind.UPDATE), - new Distribution.ConstantDistribution(10), - 100); - - Set<Long> ck1 = new TreeSet<>(); - Set<Long> ck2 = new TreeSet<>(); - Set<Long> ck3 = new TreeSet<>(); - for (int i = 0; i < 1000; i++) - { - long[] part = schema.ckGenerator.slice(ckSelector.cd(0, i, 0, schema)); - ck1.add(part[0]); - ck2.add(part[1]); - ck3.add(part[2]); - } - Assert.assertEquals(10, ck1.size()); - Assert.assertEquals(20, ck2.size()); - Assert.assertEquals(100, ck3.size()); - } - - @Test - public void testWeights() - { - Map<OpSelectors.OperationKind, Integer> config = new EnumMap<>(OpSelectors.OperationKind.class); - config.put(OpSelectors.OperationKind.DELETE_RANGE, 1); - config.put(OpSelectors.OperationKind.DELETE_SLICE, 1); - config.put(OpSelectors.OperationKind.DELETE_ROW, 1); - config.put(OpSelectors.OperationKind.DELETE_COLUMN, 1); - config.put(OpSelectors.OperationKind.DELETE_PARTITION, 1); - config.put(OpSelectors.OperationKind.DELETE_COLUMN_WITH_STATICS, 1); - config.put(OpSelectors.OperationKind.UPDATE, 500); - config.put(OpSelectors.OperationKind.INSERT, 500); - config.put(OpSelectors.OperationKind.UPDATE_WITH_STATICS, 500); - config.put(OpSelectors.OperationKind.INSERT_WITH_STATICS, 500); - - int[] weights = new int[config.size()]; - for (int i = 0; i < config.values().size(); i++) - weights[i] = config.get(OpSelectors.OperationKind.values()[i]); - OpSelectors.OperationSelector selector = OpSelectors.OperationSelector.weighted(Surjections.weights(weights), - OpSelectors.OperationKind.values()); - - OpSelectors.PureRng rng = new OpSelectors.PCGFast(1); - OpSelectors.PdSelector pdSelector = new OpSelectors.DefaultPdSelector(rng, 10, 10); - OpSelectors.DescriptorSelector descriptorSelector = new OpSelectors.DefaultDescriptorSelector(rng, - null, - selector, - new Distribution.ConstantDistribution(10), - 100); - - EnumMap<OpSelectors.OperationKind, Integer> m = new EnumMap<OpSelectors.OperationKind, Integer>(OpSelectors.OperationKind.class); - for (int lts = 0; lts < 1000000; lts++) - { - int total = descriptorSelector.operationsPerLts(lts); - long pd = pdSelector.pd(lts, SCHEMA); - for (int opId = 0; opId < total; opId++) - { - m.compute(descriptorSelector.operationType(pd, lts, opId), - (OpSelectors.OperationKind k, Integer old) -> { - if (old == null) return 1; - else return old + 1; - }); - } - } - - for (OpSelectors.OperationKind l : OpSelectors.OperationKind.values()) - { - for (OpSelectors.OperationKind r : OpSelectors.OperationKind.values()) - { - if (l != r) - { - Assert.assertEquals(m.get(l) * 1.0 / m.get(r), - config.get(l) * 1.0 / config.get(r), - (config.get(l) * 1.0 / config.get(r)) * 0.10); - } - } - } - } -} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/operations/RelationTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/operations/RelationTest.java deleted file mode 100644 index 975a87a13e55..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/operations/RelationTest.java +++ /dev/null @@ -1,204 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.operations; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; - -import org.junit.Assert; -import org.junit.Test; - -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.fuzz.harry.gen.DataGeneratorsTest; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.operations.QueryGenerator; -import org.apache.cassandra.harry.util.BitSet; - -public class RelationTest -{ - private static int RUNS = 50; - - @Test - public void testKeyGenerators() - { - for (int size = 1; size < 5; size++) - { - Iterator<ColumnSpec.DataType[]> iter = DataGeneratorsTest.permutations(size, - ColumnSpec.DataType.class, - ColumnSpec.int8Type, - ColumnSpec.asciiType, - ColumnSpec.int16Type, - ColumnSpec.int32Type, - ColumnSpec.int64Type, - ColumnSpec.floatType, - ColumnSpec.doubleType - ); - while (iter.hasNext()) - { - ColumnSpec.DataType[] types = iter.next(); - List<ColumnSpec<?>> spec = new ArrayList<>(types.length); - for (int i = 0; i < types.length; i++) - spec.add(ColumnSpec.ck("r" + i, types[i], false)); - - SchemaSpec schemaSpec = new SchemaSpec("ks", - "tbl", - Collections.singletonList(ColumnSpec.pk("pk", ColumnSpec.int64Type)), - spec, - Collections.emptyList(), - Collections.emptyList()); - - long[] cds = new long[RUNS]; - - int[] fractions = new int[schemaSpec.clusteringKeys.size()]; - int last = cds.length; - for (int i = fractions.length - 1; i >= 0; i--) - { - fractions[i] = last; - last = last / 2; - } - - for (int i = 0; i < cds.length; i++) - { - long cd = OpSelectors.HierarchicalDescriptorSelector.cd(i, fractions, schemaSpec, new OpSelectors.PCGFast(1L), 1L); - cds[i] = schemaSpec.adjustPdEntropy(cd); - } - Arrays.sort(cds); - - OpSelectors.PureRng rng = new OpSelectors.PCGFast(1L); - - // TODO: replace with mocks? - QueryGenerator querySelector = new QueryGenerator(schemaSpec, - new OpSelectors.PdSelector() - { - protected long pd(long lts) - { - return lts; - } - - public long nextLts(long lts) - { - throw new RuntimeException("not implemented"); - } - - public long prevLts(long lts) - { - throw new RuntimeException("not implemented"); - } - - public long maxLtsFor(long pd) - { - throw new RuntimeException("not implemented"); - } - - public long minLtsAt(long position) - { - throw new RuntimeException("not implemented"); - } - - - public long minLtsFor(long pd) - { - throw new RuntimeException("not implemented"); - } - - public long maxPosition(long maxLts) - { - throw new RuntimeException("not implemented"); - } - }, - new OpSelectors.DescriptorSelector() - { - public int operationsPerLts(long lts) - { - throw new RuntimeException("not implemented"); - } - - public int maxPartitionSize() - { - throw new RuntimeException("not implemented"); - } - - public boolean isCdVisitedBy(long pd, long lts, long cd) - { - throw new RuntimeException("not implemented"); - } - - protected long cd(long pd, long lts, long opId) - { - throw new RuntimeException("not implemented"); - } - - public long randomCd(long pd, long entropy) - { - return Math.abs(rng.prev(entropy)) % cds.length; - } - - protected long vd(long pd, long cd, long lts, long opId, int col) - { - throw new RuntimeException("not implemented"); - } - - public OpSelectors.OperationKind operationType(long pd, long lts, long opId) - { - throw new RuntimeException("not implemented"); - } - - public BitSet columnMask(long pd, long lts, long opId, OpSelectors.OperationKind opType) - { - throw new RuntimeException("not implemented"); - } - - public long opId(long pd, long lts, long cd) - { - return 0; - } - }, - rng); - - QueryGenerator.TypedQueryGenerator gen = new QueryGenerator.TypedQueryGenerator(rng, querySelector); - - try - { - for (int i = 0; i < RUNS; i++) - { - Query query = gen.inflate(i, 0); - for (int j = 0; j < cds.length; j++) - { - long cd = schemaSpec.ckGenerator.adjustEntropyDomain(cds[i]); - // the only thing we care about here is that query - Assert.assertEquals(String.format("Error caught while running a query %s with cd %d", - query, cd), - Query.simpleMatch(query, cd), - query.matchCd(cd)); - } - } - } - catch (Throwable t) - { - throw new AssertionError("Caught error for the type combination " + Arrays.toString(types), t); - } - } - } - } -} diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/runner/LockingDataTrackerTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/runner/LockingDataTrackerTest.java deleted file mode 100644 index 94b9f2e1312e..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/runner/LockingDataTrackerTest.java +++ /dev/null @@ -1,133 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.runner; - -import java.util.List; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; - -import org.junit.Assert; -import org.junit.Test; - -import org.apache.cassandra.concurrent.ExecutorFactory; -import org.apache.cassandra.concurrent.Interruptible; -import org.apache.cassandra.harry.ddl.SchemaGenerators; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.runner.Runner; -import org.apache.cassandra.harry.tracker.LockingDataTracker; -import org.apache.cassandra.utils.concurrent.WaitQueue; - -import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.Daemon.NON_DAEMON; -import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.Interrupts.UNSYNCHRONIZED; -import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.SimulatorSafe.SAFE; - -public class LockingDataTrackerTest -{ - @Test - public void testDataTracker() throws Throwable - { - SchemaSpec schemaSpec = SchemaGenerators.defaultSchemaSpecGen("test").inflate(1L); - OpSelectors.PureRng rng = new OpSelectors.PCGFast(1L); - OpSelectors.PdSelector pdSelector = new OpSelectors.DefaultPdSelector(rng, 5, 2); - LockingDataTracker tracker = new LockingDataTracker(pdSelector, schemaSpec); - - AtomicReference<State> excluded = new AtomicReference<>(State.UNLOCKED); - AtomicInteger readers = new AtomicInteger(0); - AtomicInteger writers = new AtomicInteger(0); - - WaitQueue queue = WaitQueue.newWaitQueue(); - WaitQueue.Signal interrupt = queue.register(); - List<Throwable> errors = new CopyOnWriteArrayList<>(); - - long lts = 1; - long pd = pdSelector.pd(lts, schemaSpec); - int parallelism = 2; - for (int i = 0; i < parallelism; i++) - { - ExecutorFactory.Global.executorFactory().infiniteLoop("write-" + i, Runner.wrapInterrupt(state -> { - try - { - if (state == Interruptible.State.NORMAL) - { - tracker.beginModification(lts); - Assert.assertEquals(0, readers.get()); - writers.incrementAndGet(); - excluded.updateAndGet((prev) -> { - assert (prev == State.UNLOCKED || prev == State.LOCKED_FOR_WRITE) : prev; - return State.LOCKED_FOR_WRITE; - }); - Assert.assertEquals(0, readers.get()); - excluded.updateAndGet((prev) -> { - assert (prev == State.UNLOCKED || prev == State.LOCKED_FOR_WRITE) : prev; - return State.UNLOCKED; - }); - Assert.assertEquals(0, readers.get()); - writers.decrementAndGet(); - tracker.endModification(lts); - } - } - catch (Throwable t) - { - t.printStackTrace(); - throw t; - } - }, interrupt::signal, errors::add), SAFE, NON_DAEMON, UNSYNCHRONIZED); - } - - for (int i = 0; i < parallelism; i++) - { - ExecutorFactory.Global.executorFactory().infiniteLoop("read-" + i, Runner.wrapInterrupt(state -> { - try - { - if (state == Interruptible.State.NORMAL) - { - tracker.beginValidation(pd); - Assert.assertEquals(0, writers.get()); - readers.incrementAndGet(); - excluded.updateAndGet((prev) -> { - assert (prev == State.UNLOCKED || prev == State.LOCKED_FOR_READ) : prev; - return State.LOCKED_FOR_READ; - }); - Assert.assertEquals(0, writers.get()); - excluded.updateAndGet((prev) -> { - assert (prev == State.UNLOCKED || prev == State.LOCKED_FOR_READ) : prev; - return State.UNLOCKED; - }); - Assert.assertEquals(0, writers.get()); - readers.decrementAndGet(); - tracker.endValidation(pd); - } - } - catch (Throwable t) - { - t.printStackTrace(); - throw t; - } - }, interrupt::signal, errors::add), SAFE, NON_DAEMON, UNSYNCHRONIZED); - } - - interrupt.await(1, TimeUnit.MINUTES); - if (!errors.isEmpty()) - Runner.mergeAndThrow(errors); - } - enum State { UNLOCKED, LOCKED_FOR_READ, LOCKED_FOR_WRITE } -} diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/util/RangesTest.java b/test/distributed/org/apache/cassandra/fuzz/harry/util/RangesTest.java deleted file mode 100644 index 38231ee2490f..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/harry/util/RangesTest.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.harry.util; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Random; - -import org.junit.Assert; -import org.junit.Test; - -import org.apache.cassandra.harry.util.DescriptorRanges; - -public class RangesTest -{ - - @Test - public void simpleRangesTest() - { - List<DescriptorRanges.DescriptorRange> list = Arrays.asList(inclusiveRange(10, 20, 10), - inclusiveRange(40, 50, 10), - inclusiveRange(60, 70, 10), - inclusiveRange(80, 90, 10)); - Collections.shuffle(list); - DescriptorRanges ranges = new DescriptorRanges(list); - - Assert.assertTrue(ranges.isShadowed(10, 5)); - Assert.assertFalse(ranges.isShadowed(10, 20)); - Assert.assertFalse(ranges.isShadowed(15, 20)); - Assert.assertTrue(ranges.isShadowed(49, 5)); - Assert.assertFalse(ranges.isShadowed(55, 5)); - Assert.assertFalse(ranges.isShadowed(50, 20)); - Assert.assertTrue(ranges.isShadowed(90, 9)); - } - - @Test - public void randomizedRangesTest() - { - for (int i = 0; i < 1000; i++) - _randomizedRangesTest(); - } - - private void _randomizedRangesTest() - { - List<DescriptorRanges.DescriptorRange> rangesList = new ArrayList<>(); - Random rnd = new Random(); - for (int i = 0; i < 100; i++) - { - long a = rnd.nextInt(1000); - long b = rnd.nextInt(1000); - DescriptorRanges.DescriptorRange range = new DescriptorRanges.DescriptorRange(Math.min(a, b), - Math.max(a,b), - rnd.nextBoolean(), - rnd.nextBoolean(), - rnd.nextInt(1000)); - rangesList.add(range); - } - DescriptorRanges ranges = new DescriptorRanges(rangesList); - - for (int i = 0; i < 10000; i++) - { - long descriptor = rnd.nextLong(); - long ts = rnd.nextInt(1000); - Assert.assertEquals(matchLinear(rangesList, descriptor, ts), - ranges.isShadowed(descriptor, ts)); - - } - } - - public boolean matchLinear(List<DescriptorRanges.DescriptorRange> ranges, long descriptor, long ts) - { - for (DescriptorRanges.DescriptorRange range : ranges) - { - if (range.contains(descriptor, ts)) - return true; - } - return false; - } - - public DescriptorRanges.DescriptorRange inclusiveRange(long start, long end, long ts) - { - return new DescriptorRanges.DescriptorRange(start, end, true, true, 10); - } -} diff --git a/test/distributed/org/apache/cassandra/fuzz/ring/ConsistentBootstrapTest.java b/test/distributed/org/apache/cassandra/fuzz/ring/ConsistentBootstrapTest.java index f5f5ef04888c..c6e8db78b18c 100644 --- a/test/distributed/org/apache/cassandra/fuzz/ring/ConsistentBootstrapTest.java +++ b/test/distributed/org/apache/cassandra/fuzz/ring/ConsistentBootstrapTest.java @@ -22,6 +22,8 @@ import org.junit.Assert; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.Constants; @@ -30,13 +32,19 @@ import org.apache.cassandra.distributed.api.IInstanceConfig; import org.apache.cassandra.distributed.api.IInvokableInstance; import org.apache.cassandra.distributed.api.TokenSupplier; -import org.apache.cassandra.harry.HarryHelper; import org.apache.cassandra.distributed.shared.NetworkTopology; import org.apache.cassandra.distributed.test.log.FuzzTestBase; -import org.apache.cassandra.harry.dsl.ReplayingHistoryBuilder; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.sut.injvm.InJvmSut; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.op.Visit; +import org.apache.cassandra.harry.dsl.HistoryBuilder; +import org.apache.cassandra.harry.dsl.HistoryBuilderHelper; +import org.apache.cassandra.harry.execution.DataTracker; +import org.apache.cassandra.harry.execution.RingAwareInJvmDTestVisitExecutor; +import org.apache.cassandra.harry.gen.Generator; +import org.apache.cassandra.harry.gen.Generators; +import org.apache.cassandra.harry.gen.SchemaGenerators; +import org.apache.cassandra.harry.model.QuiescentChecker; +import org.apache.cassandra.harry.model.TokenPlacementModel; import org.apache.cassandra.metrics.TCMMetrics; import org.apache.cassandra.net.Verb; import org.apache.cassandra.tcm.Epoch; @@ -46,15 +54,18 @@ import static org.apache.cassandra.distributed.shared.ClusterUtils.pauseBeforeCommit; import static org.apache.cassandra.distributed.shared.ClusterUtils.unpauseCommits; import static org.apache.cassandra.distributed.shared.ClusterUtils.waitForCMSToQuiesce; +import static org.apache.cassandra.harry.checker.TestHelper.withRandom; public class ConsistentBootstrapTest extends FuzzTestBase { + private static final Logger logger = LoggerFactory.getLogger(ConsistentBootstrapTest.class); private static int WRITES = 500; @Test public void bootstrapFuzzTest() throws Throwable { - IInvokableInstance cmsInstance = null; + Generator<SchemaSpec> schemaGen = SchemaGenerators.schemaSpecGen(KEYSPACE, "bootstrap_fuzz", 1000); + IInvokableInstance forShutdown = null; try (Cluster cluster = builder().withNodes(3) .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(4)) .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(4, "dc0", "rack0")) @@ -63,53 +74,69 @@ public void bootstrapFuzzTest() throws Throwable .set("metadata_snapshot_frequency", 5)) .start()) { - cmsInstance = cluster.get(1); + IInvokableInstance cmsInstance = cluster.get(1); + forShutdown = cmsInstance; waitForCMSToQuiesce(cluster, cmsInstance); - ReplayingHistoryBuilder harry = HarryHelper.dataGen(new InJvmSut(cluster), - new TokenPlacementModel.SimpleReplicationFactor(3), - SystemUnderTest.ConsistencyLevel.ALL); - cluster.coordinator(1).execute(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 3};", HarryHelper.KEYSPACE), - ConsistencyLevel.ALL); - cluster.coordinator(1).execute(harry.schema().compile().cql(), ConsistencyLevel.ALL); - waitForCMSToQuiesce(cluster, cluster.get(1)); - Runnable writeAndValidate = () -> { - System.out.println("Starting write phase..."); - for (int i = 0; i < WRITES; i++) - harry.insert(); - - System.out.println("Starting validate phase..."); - harry.validateAll(harry.quiescentLocalChecker()); - }; - writeAndValidate.run(); - - IInstanceConfig config = cluster.newInstanceConfig() - .set("auto_bootstrap", true) - .set(Constants.KEY_DTEST_FULL_STARTUP, true); - IInvokableInstance newInstance = cluster.bootstrap(config); - - // Prime the CMS node to pause before the finish join event is committed - Callable<?> pending = pauseBeforeCommit(cmsInstance, (e) -> e instanceof PrepareJoin.FinishJoin); - new Thread(() -> newInstance.startup()).start(); - pending.call(); - - writeAndValidate.run(); - - // Make sure there can be only one FinishJoin in flight - waitForCMSToQuiesce(cluster, cmsInstance); - // set expectation of finish join & retrieve the sequence when it gets committed - Callable<Epoch> bootstrapVisible = getSequenceAfterCommit(cmsInstance, (e, r) -> e instanceof PrepareJoin.FinishJoin && r.isSuccess()); + withRandom(rng -> { + SchemaSpec schema = schemaGen.generate(rng); + Generators.TrackingGenerator<Integer> pkGen = Generators.tracking(Generators.int32(0, Math.min(schema.valueGenerators.pkPopulation(), 1000))); + Generator<Integer> ckGen = Generators.int32(0, Math.min(schema.valueGenerators.ckPopulation(), 1000)); + + HistoryBuilder history = new HistoryBuilder(schema.valueGenerators); + Runnable writeAndValidate = () -> { + for (int i = 0; i < WRITES; i++) + HistoryBuilderHelper.insertRandomData(schema, pkGen, ckGen, rng, history); + + for (int pk : pkGen.generated()) + history.selectPartition(pk); + }; + + history.customThrowing(() -> { + cluster.schemaChange(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 3};", KEYSPACE)); + cluster.schemaChange(schema.compile()); + waitForCMSToQuiesce(cluster, cmsInstance); + }, "Setup"); + + writeAndValidate.run(); + + history.customThrowing(() -> { + IInstanceConfig config = cluster.newInstanceConfig() + .set("auto_bootstrap", true) + .set(Constants.KEY_DTEST_FULL_STARTUP, true); + IInvokableInstance newInstance = cluster.bootstrap(config); + + // Prime the CMS node to pause before the finish join event is committed + Callable<?> pending = pauseBeforeCommit(cmsInstance, (e) -> e instanceof PrepareJoin.FinishJoin); + new Thread(() -> newInstance.startup()).start(); + pending.call(); + }, "Start boostrap"); + + writeAndValidate.run(); + + history.customThrowing(() -> { + // Make sure there can be only one FinishJoin in flight + waitForCMSToQuiesce(cluster, cmsInstance); + // set expectation of finish join & retrieve the sequence when it gets committed + Callable<Epoch> bootstrapVisible = getSequenceAfterCommit(cmsInstance, (e, r) -> e instanceof PrepareJoin.FinishJoin && r.isSuccess()); - // wait for the cluster to all witness the finish join event - unpauseCommits(cmsInstance); - waitForCMSToQuiesce(cluster, bootstrapVisible.call()); + // wait for the cluster to all witness the finish join event + unpauseCommits(cmsInstance); + waitForCMSToQuiesce(cluster, bootstrapVisible.call()); + }, "Finish bootstrap"); + writeAndValidate.run(); - writeAndValidate.run(); + RingAwareInJvmDTestVisitExecutor.replay(RingAwareInJvmDTestVisitExecutor.builder() + .replicationFactor(new TokenPlacementModel.SimpleReplicationFactor(2)) + .consistencyLevel(ConsistencyLevel.ALL) + .build(schema, history, cluster), + history); + }); } catch (Throwable t) { - if (cmsInstance != null) - unpauseCommits(cmsInstance); + if (forShutdown != null) + unpauseCommits(forShutdown); throw t; } } @@ -117,7 +144,9 @@ public void bootstrapFuzzTest() throws Throwable @Test public void coordinatorIsBehindTest() throws Throwable { - IInvokableInstance cmsInstance = null; + Generator<SchemaSpec> schemaGen = SchemaGenerators.schemaSpecGen(KEYSPACE, "coordinator_is_behind", 1000); + + IInvokableInstance forShutdown = null; try (Cluster cluster = builder().withNodes(3) .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(4)) .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(4, "dc0", "rack0")) @@ -126,109 +155,122 @@ public void coordinatorIsBehindTest() throws Throwable .set("metadata_snapshot_frequency", 5)) .start()) { - cmsInstance = cluster.get(1); + IInvokableInstance cmsInstance = cluster.get(1); + forShutdown = cmsInstance; waitForCMSToQuiesce(cluster, cmsInstance); - ReplayingHistoryBuilder harry = HarryHelper.dataGen(new InJvmSut(cluster, () -> 2, (t) -> false) - { - public Object[][] execute(String statement, ConsistencyLevel cl, int coordinator, int pagesize, Object... bindings) - { - try - { - return super.execute(statement, cl, coordinator, pagesize, bindings); - } - catch (Throwable t) - { - // Avoid retries - return new Object[][]{}; - } - } - }, - new TokenPlacementModel.SimpleReplicationFactor(3), - SystemUnderTest.ConsistencyLevel.ALL); - - cluster.coordinator(1).execute(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 3};", HarryHelper.KEYSPACE), - ConsistencyLevel.ALL); - cluster.coordinator(1).execute(harry.schema().compile().cql(), ConsistencyLevel.ALL); - waitForCMSToQuiesce(cluster, cluster.get(1)); - - cluster.filters().verbs(Verb.TCM_REPLICATION.id, - Verb.TCM_FETCH_CMS_LOG_RSP.id, - Verb.TCM_FETCH_PEER_LOG_RSP.id, - Verb.TCM_CURRENT_EPOCH_REQ.id) - .to(2) - .drop() - .on(); - - IInstanceConfig config = cluster.newInstanceConfig() - .set("auto_bootstrap", true) - .set(Constants.KEY_DTEST_FULL_STARTUP, true) - .set("progress_barrier_default_consistency_level", "NODE_LOCAL"); - IInvokableInstance newInstance = cluster.bootstrap(config); - - // Prime the CMS node to pause before the finish join event is committed - Callable<?> pending = pauseBeforeCommit(cmsInstance, (e) -> e instanceof PrepareJoin.MidJoin); - long[] metricCounts = new long[4]; - for (int i = 1; i <= 4; i++) - metricCounts[i - 1] = cluster.get(i).callOnInstance(() -> TCMMetrics.instance.coordinatorBehindPlacements.getCount()); - Thread thread = new Thread(() -> newInstance.startup()); - thread.start(); - pending.call(); - - boolean triggered = false; - long[] markers = new long[4]; - outer: - for (int i = 0; i < 20; i++) - { - for (int n = 0; n < 4; n++) - markers[n] = cluster.get(n + 1).logs().mark(); - - try - { - harry.insert(); - } - catch (Throwable t) - { - // ignore - } - for (int n = 0; n < markers.length; n++) + withRandom(rng -> { + SchemaSpec schema = schemaGen.generate(rng); + Generator<Integer> ckGen = Generators.int32(0, Math.min(schema.valueGenerators.ckPopulation(), 1000)); + + HistoryBuilder history = new HistoryBuilder(schema.valueGenerators); + + cluster.schemaChange(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 3};", KEYSPACE)); + cluster.schemaChange(schema.compile()); + waitForCMSToQuiesce(cluster, cluster.get(1)); + + cluster.filters().verbs(Verb.TCM_REPLICATION.id, + Verb.TCM_FETCH_CMS_LOG_RSP.id, + Verb.TCM_FETCH_PEER_LOG_RSP.id, + Verb.TCM_CURRENT_EPOCH_REQ.id) + .to(2) + .drop() + .on(); + + IInstanceConfig config = cluster.newInstanceConfig() + .set("auto_bootstrap", true) + .set(Constants.KEY_DTEST_FULL_STARTUP, true) + .set("progress_barrier_default_consistency_level", "NODE_LOCAL"); + IInvokableInstance newInstance = cluster.bootstrap(config); + // Prime the CMS node to pause before the finish join event is committed + long[] metricCounts = new long[4]; + for (int i = 1; i <= 4; i++) + metricCounts[i - 1] = cluster.get(i).callOnInstance(() -> TCMMetrics.instance.coordinatorBehindPlacements.getCount()); + + DataTracker tracker = new DataTracker.SequentialDataTracker(); + RingAwareInJvmDTestVisitExecutor executor = RingAwareInJvmDTestVisitExecutor.builder() + .replicationFactor(new TokenPlacementModel.SimpleReplicationFactor(2)) + .nodeSelector(i -> 2) + .consistencyLevel(ConsistencyLevel.ALL) + .build(schema, + tracker, + new QuiescentChecker(schema.valueGenerators, tracker, history), + cluster); + + Thread startup = new Thread(() -> newInstance.startup()); + + history.customThrowing(() -> { + Callable<?> pending = pauseBeforeCommit(cmsInstance, (e) -> e instanceof PrepareJoin.MidJoin); + startup.start(); + pending.call(); + }, "Startup"); + + long[] markers = new long[4]; + history.custom(() -> { + for (int n = 0; n < 4; n++) + markers[n] = cluster.get(n + 1).logs().mark(); + }, "Start grep"); + + outer: + for (int i = 0; i < history.valueGenerators().pkPopulation(); i++) { - if ((n + 1) == 2) // skip 2nd node - continue; - - if (!cluster.get(n + 1) - .logs() - .grep(markers[n], "Routing is correct, but coordinator needs to catch-up") - .getResult() - .isEmpty()) + long pd = history.valueGenerators().pkGen().descriptorAt(i); + for (TokenPlacementModel.Replica replica : executor.getReplicasFor(pd)) { - triggered = true; - break outer; + if (cluster.get(1).config().broadcastAddress().toString().contains(replica.node().id())) + { + HistoryBuilderHelper.insertRandomData(schema, i, ckGen.generate(rng), rng, history); + break outer; + } } } - } - Assert.assertTrue("Should have triggered routing exception on the replica", triggered); - boolean metricTriggered = false; - for (int i = 1; i <= 4; i++) - { - long prevMetric = metricCounts[i - 1]; - long newMetric = cluster.get(i).callOnInstance(() -> TCMMetrics.instance.coordinatorBehindPlacements.getCount()); - if (newMetric - prevMetric > 0) - { - metricTriggered = true; - break; - } - } - Assert.assertTrue("Metric CoordinatorBehindRing should have been bumped by at least one replica", metricTriggered); - cluster.filters().reset(); - unpauseCommits(cmsInstance); - thread.join(); + + history.customThrowing(() -> { + boolean triggered = false; + for (int n = 0; n < markers.length; n++) + { + if ((n + 1) == 2) // skip 2nd node + continue; + + if (!cluster.get(n + 1) + .logs() + .grep(markers[n], "Routing is correct, but coordinator needs to catch-up") + .getResult() + .isEmpty()) + { + triggered = true; + break; + } + } + + Assert.assertTrue("Should have triggered routing exception on the replica", triggered); + boolean metricTriggered = false; + for (int i = 1; i <= 4; i++) + { + long prevMetric = metricCounts[i - 1]; + long newMetric = cluster.get(i).callOnInstance(() -> TCMMetrics.instance.coordinatorBehindPlacements.getCount()); + if (newMetric - prevMetric > 0) + { + metricTriggered = true; + break; + } + } + Assert.assertTrue("Metric CoordinatorBehindRing should have been bumped by at least one replica", metricTriggered); + + cluster.filters().reset(); + unpauseCommits(cmsInstance); + startup.join(); + }, "Validate triggered"); + + for (Visit visit : history) + executor.execute(visit); + }); } catch (Throwable t) { - if (cmsInstance != null) - unpauseCommits(cmsInstance); + if (forShutdown != null) + unpauseCommits(forShutdown); throw t; } } diff --git a/test/distributed/org/apache/cassandra/fuzz/ring/ConsistentLeaveTest.java b/test/distributed/org/apache/cassandra/fuzz/ring/ConsistentLeaveTest.java index 8c98eb2e2f5e..218575e7bf91 100644 --- a/test/distributed/org/apache/cassandra/fuzz/ring/ConsistentLeaveTest.java +++ b/test/distributed/org/apache/cassandra/fuzz/ring/ConsistentLeaveTest.java @@ -32,16 +32,19 @@ import org.apache.cassandra.distributed.api.Feature; import org.apache.cassandra.distributed.api.IInvokableInstance; import org.apache.cassandra.distributed.api.TokenSupplier; -import org.apache.cassandra.harry.HarryHelper; import org.apache.cassandra.distributed.shared.NetworkTopology; import org.apache.cassandra.distributed.test.log.FuzzTestBase; import org.apache.cassandra.gms.ApplicationState; import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.dsl.HistoryBuilder; import org.apache.cassandra.harry.dsl.ReplayingHistoryBuilder; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.sut.injvm.InJvmSut; -import org.apache.cassandra.harry.sut.injvm.InJvmSutBase; +import org.apache.cassandra.harry.execution.InJvmDTestVisitExecutor; +import org.apache.cassandra.harry.execution.RingAwareInJvmDTestVisitExecutor; +import org.apache.cassandra.harry.gen.Generator; +import org.apache.cassandra.harry.gen.Generators; +import org.apache.cassandra.harry.gen.SchemaGenerators; +import org.apache.cassandra.harry.model.TokenPlacementModel; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tcm.Epoch; @@ -52,6 +55,7 @@ import static org.apache.cassandra.distributed.shared.ClusterUtils.pauseBeforeCommit; import static org.apache.cassandra.distributed.shared.ClusterUtils.unpauseCommits; import static org.apache.cassandra.distributed.shared.ClusterUtils.waitForCMSToQuiesce; +import static org.apache.cassandra.harry.checker.TestHelper.withRandom; import static org.junit.Assert.assertFalse; public class ConsistentLeaveTest extends FuzzTestBase @@ -61,67 +65,82 @@ public class ConsistentLeaveTest extends FuzzTestBase @Test public void decommissionTest() throws Throwable { - IInvokableInstance cmsInstance = null; + Generator<SchemaSpec> schemaGen = SchemaGenerators.schemaSpecGen(KEYSPACE, "decomission", 1000); + IInvokableInstance forShutdown = null; try (Cluster cluster = builder().withNodes(3) .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(3)) .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(3, "dc0", "rack0")) .appendConfig(c -> c.with(Feature.NETWORK)) .start()) { - cmsInstance = cluster.get(1); - IInvokableInstance leavingInstance = cluster.get(2); - waitForCMSToQuiesce(cluster, cmsInstance); - - - ReplayingHistoryBuilder harry = HarryHelper.dataGen(new InJvmSut(cluster, () -> 1, InJvmSutBase.retryOnTimeout()), - new TokenPlacementModel.SimpleReplicationFactor(2), - SystemUnderTest.ConsistencyLevel.ALL); - cluster.coordinator(1).execute(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 2};", HarryHelper.KEYSPACE), - ConsistencyLevel.ALL); - cluster.coordinator(1).execute(harry.schema().compile().cql(), ConsistencyLevel.ALL); - waitForCMSToQuiesce(cluster, cmsInstance); - - Runnable writeAndValidate = () -> { - System.out.println("Starting write phase..."); - for (int i = 0; i < WRITES; i++) - harry.insert(); - - System.out.println("Starting validate phase..."); - harry.validateAll(harry.quiescentLocalChecker()); - }; - writeAndValidate.run(); + IInvokableInstance cmsInstance = cluster.get(1); + forShutdown = cmsInstance; - // Prime the CMS node to pause before the finish leave event is committed - Callable<?> pending = pauseBeforeCommit(cmsInstance, (e) -> e instanceof PrepareLeave.FinishLeave); - new Thread(() -> leavingInstance.runOnInstance(() -> StorageService.instance.decommission(true))).start(); - pending.call(); - - waitForCMSToQuiesce(cluster, cmsInstance); - assertGossipStatus(cluster, leavingInstance.config().num(), "LEAVING"); - - writeAndValidate.run(); - - // Make sure there can be only one FinishLeave in flight + IInvokableInstance leavingInstance = cluster.get(2); waitForCMSToQuiesce(cluster, cmsInstance); - // set expectation of finish leave & retrieve the sequence when it gets committed - Epoch currentEpoch = getClusterMetadataVersion(cmsInstance); - Callable<Epoch> finishedLeaving = getSequenceAfterCommit(cmsInstance, (e, r) -> e instanceof PrepareLeave.FinishLeave && r.isSuccess()); - unpauseCommits(cmsInstance); - Epoch nextEpoch = finishedLeaving.call(); - Assert.assertEquals(String.format("Epoch %s should have immediately superseded epoch %s.", nextEpoch, currentEpoch), - nextEpoch.getEpoch(), currentEpoch.getEpoch() + 1); - - // wait for the cluster to all witness the finish join event - waitForCMSToQuiesce(cluster, nextEpoch); - - assertGossipStatus(cluster, leavingInstance.config().num(), "LEFT"); - writeAndValidate.run(); + withRandom(rng -> { + SchemaSpec schema = schemaGen.generate(rng); + Generators.TrackingGenerator<Integer> pkGen = Generators.tracking(Generators.int32(0, Math.min(schema.valueGenerators.pkPopulation(), 1000))); + Generator<Integer> ckGen = Generators.int32(0, Math.min(schema.valueGenerators.ckPopulation(), 1000)); + + HistoryBuilder history = new ReplayingHistoryBuilder(schema.valueGenerators, + (hb) -> RingAwareInJvmDTestVisitExecutor.builder() + .replicationFactor(new TokenPlacementModel.SimpleReplicationFactor(2)) + .consistencyLevel(ConsistencyLevel.ALL) + .retryPolicy(InJvmDTestVisitExecutor.RetryPolicy.RETRY_ON_TIMEOUT) + .nodeSelector(lts -> 1) + .build(schema, hb, cluster)); + Runnable writeAndValidate = () -> { + for (int i = 0; i < WRITES; i++) + history.insert(pkGen.generate(rng), ckGen.generate(rng)); + + for (int pk : pkGen.generated()) + history.selectPartition(pk); + }; + + cluster.schemaChange(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 2};", KEYSPACE)); + cluster.schemaChange(schema.compile()); + waitForCMSToQuiesce(cluster, cmsInstance); + + writeAndValidate.run(); + + history.customThrowing(() -> { + // Prime the CMS node to pause before the finish leave event is committed + Callable<?> pending = pauseBeforeCommit(cmsInstance, (e) -> e instanceof PrepareLeave.FinishLeave); + new Thread(() -> leavingInstance.runOnInstance(() -> StorageService.instance.decommission(true))).start(); + pending.call(); + + waitForCMSToQuiesce(cluster, cmsInstance); + assertGossipStatus(cluster, leavingInstance.config().num(), "LEAVING"); + }, "Start Leave"); + + writeAndValidate.run(); + + history.customThrowing(() -> { + // Make sure there can be only one FinishLeave in flight + waitForCMSToQuiesce(cluster, cmsInstance); + // set expectation of finish leave & retrieve the sequence when it gets committed + Epoch currentEpoch = getClusterMetadataVersion(cmsInstance); + Callable<Epoch> finishedLeaving = getSequenceAfterCommit(cmsInstance, (e, r) -> e instanceof PrepareLeave.FinishLeave && r.isSuccess()); + unpauseCommits(cmsInstance); + Epoch nextEpoch = finishedLeaving.call(); + Assert.assertEquals(String.format("Epoch %s should have immediately superseded epoch %s.", nextEpoch, currentEpoch), + nextEpoch.getEpoch(), currentEpoch.getEpoch() + 1); + + // wait for the cluster to all witness the finish join event + waitForCMSToQuiesce(cluster, nextEpoch); + + assertGossipStatus(cluster, leavingInstance.config().num(), "LEFT"); + }, "Finish leave"); + + writeAndValidate.run(); + }); } catch (Throwable t) { - if (cmsInstance != null) - unpauseCommits(cmsInstance); + if (forShutdown != null) + unpauseCommits(forShutdown); throw t; } } diff --git a/test/distributed/org/apache/cassandra/fuzz/ring/ConsistentMoveTest.java b/test/distributed/org/apache/cassandra/fuzz/ring/ConsistentMoveTest.java index 4ee7bc6487a2..8fcac18f045d 100644 --- a/test/distributed/org/apache/cassandra/fuzz/ring/ConsistentMoveTest.java +++ b/test/distributed/org/apache/cassandra/fuzz/ring/ConsistentMoveTest.java @@ -19,28 +19,32 @@ package org.apache.cassandra.fuzz.ring; import java.util.List; -import java.util.Random; import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import com.google.common.util.concurrent.Uninterruptibles; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.api.ConsistencyLevel; import org.apache.cassandra.distributed.api.Feature; import org.apache.cassandra.distributed.api.IInvokableInstance; import org.apache.cassandra.distributed.api.TokenSupplier; -import org.apache.cassandra.harry.HarryHelper; import org.apache.cassandra.distributed.shared.NetworkTopology; import org.apache.cassandra.distributed.test.log.FuzzTestBase; import org.apache.cassandra.gms.ApplicationState; import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.dsl.HistoryBuilder; import org.apache.cassandra.harry.dsl.ReplayingHistoryBuilder; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.sut.injvm.InJvmSut; +import org.apache.cassandra.harry.execution.RingAwareInJvmDTestVisitExecutor; +import org.apache.cassandra.harry.gen.Generator; +import org.apache.cassandra.harry.gen.Generators; +import org.apache.cassandra.harry.gen.SchemaGenerators; +import org.apache.cassandra.harry.model.TokenPlacementModel; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tcm.Epoch; @@ -50,82 +54,101 @@ import static org.apache.cassandra.distributed.shared.ClusterUtils.pauseBeforeCommit; import static org.apache.cassandra.distributed.shared.ClusterUtils.unpauseCommits; import static org.apache.cassandra.distributed.shared.ClusterUtils.waitForCMSToQuiesce; +import static org.apache.cassandra.harry.checker.TestHelper.withRandom; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; public class ConsistentMoveTest extends FuzzTestBase { + private static final Logger logger = LoggerFactory.getLogger(ConsistentMoveTest.class); private static int WRITES = 500; @Test public void moveTest() throws Throwable { - IInvokableInstance cmsInstance = null; + Generator<SchemaSpec> schemaGen = SchemaGenerators.schemaSpecGen(KEYSPACE, "move", 1000); + + IInvokableInstance forShutdown = null; try (Cluster cluster = builder().withNodes(3) .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(3)) .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(3, "dc0", "rack0")) .appendConfig(c -> c.with(Feature.NETWORK)) .start()) { - cmsInstance = cluster.get(1); + IInvokableInstance cmsInstance = cluster.get(1); + forShutdown = cmsInstance; IInvokableInstance movingInstance = cluster.get(2); waitForCMSToQuiesce(cluster, cmsInstance); - ReplayingHistoryBuilder harry = HarryHelper.dataGen(new InJvmSut(cluster), - new TokenPlacementModel.SimpleReplicationFactor(2), - SystemUnderTest.ConsistencyLevel.ALL); - cluster.coordinator(1).execute(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 2};", HarryHelper.KEYSPACE), - ConsistencyLevel.ALL); - cluster.coordinator(1).execute(harry.schema().compile().cql(), ConsistencyLevel.ALL); - waitForCMSToQuiesce(cluster, cmsInstance); - - Runnable writeAndValidate = () -> { - System.out.println("Starting write phase..."); - for (int i = 0; i < WRITES; i++) - harry.insert(); - - System.out.println("Starting validate phase..."); - harry.validateAll(harry.quiescentLocalChecker()); - }; - writeAndValidate.run(); - - - // Make sure there can be only one FinishLeave in flight - waitForCMSToQuiesce(cluster, cmsInstance); - - Callable<?> pending = pauseBeforeCommit(cmsInstance, (e) -> e instanceof PrepareMove.FinishMove); - new Thread(() -> { - Random rng = new Random(1); - movingInstance.runOnInstance(() -> StorageService.instance.move(Long.toString(rng.nextLong()))); - }).start(); - pending.call(); - - assertGossipStatus(cluster, movingInstance.config().num(), "MOVING"); - - // wait for the cluster to all witness the finish join event - Callable<Epoch> finishedMoving = getSequenceAfterCommit(cmsInstance, (e, r) -> e instanceof PrepareMove.FinishMove && r.isSuccess()); - unpauseCommits(cmsInstance); - Epoch nextEpoch = finishedMoving.call(); - waitForCMSToQuiesce(cluster, nextEpoch); - - // TODO: rewrite the test to check only PENDING ranges. - writeAndValidate.run(); - - int clusterSize = cluster.size(); - List<InetAddressAndPort> endpoints = cluster.stream().map(i -> InetAddressAndPort.getByAddress(i.config().broadcastAddress())).collect(Collectors.toList()); - cluster.forEach(inst -> inst.runOnInstance(() -> { - for (int i = 1; i <= clusterSize; i++) - { - String gossipStatus = Gossiper.instance.getApplicationState(endpoints.get(i - 1), ApplicationState.STATUS_WITH_PORT); - assertTrue(endpoints.get(i - 1) + ": " + gossipStatus, - gossipStatus.contains("NORMAL")); - } - })); + withRandom(rng -> { + SchemaSpec schema = schemaGen.generate(rng); + Generators.TrackingGenerator<Integer > pkGen = Generators.tracking(Generators.int32(0, Math.min(schema.valueGenerators.pkPopulation(), 1000))); + Generator<Integer> ckGen = Generators.int32(0, Math.min(schema.valueGenerators.ckPopulation(), 1000)); + + HistoryBuilder history = new ReplayingHistoryBuilder(schema.valueGenerators, + (hb) -> RingAwareInJvmDTestVisitExecutor.builder() + .replicationFactor(new TokenPlacementModel.SimpleReplicationFactor(2)) + .consistencyLevel(ConsistencyLevel.ALL) + .build(schema, hb, cluster)); + Runnable writeAndValidate = () -> { + for (int i = 0; i < WRITES; i++) + history.insert(pkGen.generate(rng), ckGen.generate(rng)); + + for (int pk : pkGen.generated()) + history.selectPartition(pk); + }; + + history.custom(() -> { + cluster.schemaChange(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 2};", KEYSPACE)); + cluster.schemaChange(schema.compile()); + waitForCMSToQuiesce(cluster, cmsInstance); + }, "Setup"); + + writeAndValidate.run(); + + history.customThrowing(() -> { + // Make sure there can be only one FinishLeave in flight + waitForCMSToQuiesce(cluster, cmsInstance); + + Callable<?> pending = pauseBeforeCommit(cmsInstance, (e) -> e instanceof PrepareMove.FinishMove); + new Thread(() -> { + logger.info("Executing move..."); + movingInstance.acceptsOnInstance((Long token) -> { + StorageService.instance.move(Long.toString(token)); + }).accept(rng.next()); + }).start(); + pending.call(); + + assertGossipStatus(cluster, movingInstance.config().num(), "MOVING"); + + // wait for the cluster to all witness the finish join event + Callable<Epoch> finishedMoving = getSequenceAfterCommit(cmsInstance, (e, r) -> e instanceof PrepareMove.FinishMove && r.isSuccess()); + unpauseCommits(cmsInstance); + Epoch nextEpoch = finishedMoving.call(); + waitForCMSToQuiesce(cluster, nextEpoch); + }, "Move"); + + // TODO: rewrite the test to check only PENDING ranges. + writeAndValidate.run(); + + history.custom(() -> { + int clusterSize = cluster.size(); + List<InetAddressAndPort> endpoints = cluster.stream().map(i -> InetAddressAndPort.getByAddress(i.config().broadcastAddress())).collect(Collectors.toList()); + cluster.forEach(inst -> inst.runOnInstance(() -> { + for (int i = 1; i <= clusterSize; i++) + { + String gossipStatus = Gossiper.instance.getApplicationState(endpoints.get(i - 1), ApplicationState.STATUS_WITH_PORT); + assertTrue(endpoints.get(i - 1) + ": " + gossipStatus, + gossipStatus.contains("NORMAL")); + } + })); + }, "Finish"); + }); } catch (Throwable t) { - if (cmsInstance != null) - unpauseCommits(cmsInstance); + if (forShutdown != null) + unpauseCommits(forShutdown); throw t; } } diff --git a/test/distributed/org/apache/cassandra/fuzz/sai/MultiNodeSAITest.java b/test/distributed/org/apache/cassandra/fuzz/sai/MultiNodeSAITest.java index a86472b99aed..9ca536921d92 100644 --- a/test/distributed/org/apache/cassandra/fuzz/sai/MultiNodeSAITest.java +++ b/test/distributed/org/apache/cassandra/fuzz/sai/MultiNodeSAITest.java @@ -18,88 +18,10 @@ package org.apache.cassandra.fuzz.sai; -import org.junit.Before; -import org.junit.BeforeClass; - -import org.apache.cassandra.distributed.Cluster; -import org.apache.cassandra.distributed.test.sai.SAIUtil; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.sut.injvm.InJvmSut; -import org.apache.cassandra.harry.sut.injvm.InJvmSutBase; - -import static org.apache.cassandra.distributed.api.Feature.GOSSIP; -import static org.apache.cassandra.distributed.api.Feature.NETWORK; - -public class MultiNodeSAITest extends SingleNodeSAITest +public class MultiNodeSAITest extends MultiNodeSAITestBase { - /** - * Chosing a fetch size has implications for how well this test will excercise paging, short-read protection, and - * other important parts of the distributed query apparatus. This should be set low enough to ensure a significant - * number of queries during validation page, but not too low that more expesive queries time out and fail the test. - */ - private static final int FETCH_SIZE = 10; - - @BeforeClass - public static void before() throws Throwable - { - cluster = Cluster.build() - .withNodes(2) - // At lower fetch sizes, queries w/ hundreds or thousands of matches can take a very long time. - .withConfig(InJvmSutBase.defaultConfig().andThen(c -> c.set("range_request_timeout", "180s") - .set("read_request_timeout", "180s") - .set("native_transport_timeout", "180s") - .set("slow_query_log_timeout", "180s") - .with(GOSSIP).with(NETWORK))) - .createWithoutStarting(); - cluster.setUncaughtExceptionsFilter(t -> { - logger.error("Caught exception, reporting during shutdown. Ignoring.", t); - return true; - }); - cluster.startup(); - cluster = init(cluster); - sut = new InJvmSut(cluster) { - @Override - public Object[][] execute(String cql, ConsistencyLevel cl, Object[] bindings) - { - // The goal here is to make replicas as out of date as possible, modulo the efforts of repair - // and read-repair in the test itself. - if (cql.contains("SELECT")) - return super.execute(cql, ConsistencyLevel.ALL, FETCH_SIZE, bindings); - return super.execute(cql, ConsistencyLevel.NODE_LOCAL, bindings); - } - }; - } - - @Before - public void beforeEach() - { - cluster.schemaChange("DROP KEYSPACE IF EXISTS harry"); - cluster.schemaChange("CREATE KEYSPACE harry WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};"); - } - - @Override - protected void flush(SchemaSpec schema) - { - cluster.get(1).nodetool("flush", schema.keyspace, schema.table); - cluster.get(2).nodetool("flush", schema.keyspace, schema.table); - } - - @Override - protected void repair(SchemaSpec schema) - { - cluster.get(1).nodetool("repair", schema.keyspace); - } - - @Override - protected void compact(SchemaSpec schema) - { - cluster.get(1).nodetool("compact", schema.keyspace); - cluster.get(2).nodetool("compact", schema.keyspace); - } - - @Override - protected void waitForIndexesQueryable(SchemaSpec schema) + public MultiNodeSAITest() { - SAIUtil.waitForIndexQueryable(cluster, schema.keyspace); + super(); } } \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/sai/MultiNodeSAITestBase.java b/test/distributed/org/apache/cassandra/fuzz/sai/MultiNodeSAITestBase.java new file mode 100644 index 000000000000..045cfc81d992 --- /dev/null +++ b/test/distributed/org/apache/cassandra/fuzz/sai/MultiNodeSAITestBase.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.fuzz.sai; + +import org.junit.Before; +import org.junit.BeforeClass; + +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.test.sai.SAIUtil; +import org.apache.cassandra.harry.SchemaSpec; + +import static org.apache.cassandra.distributed.api.Feature.GOSSIP; +import static org.apache.cassandra.distributed.api.Feature.NETWORK; + +public abstract class MultiNodeSAITestBase extends SingleNodeSAITestBase +{ + public MultiNodeSAITestBase() + { + super(); + } + + @BeforeClass + public static void before() throws Throwable + { + cluster = Cluster.build() + .withNodes(2) + // At lower fetch sizes, queries w/ hundreds or thousands of matches can take a very long time. + .withConfig(defaultConfig().andThen(c -> c.set("range_request_timeout", "180s") + .set("read_request_timeout", "180s") + .set("write_request_timeout", "180s") + .set("native_transport_timeout", "180s") + .set("slow_query_log_timeout", "180s") + .with(GOSSIP).with(NETWORK))) + .createWithoutStarting(); + cluster.startup(); + cluster = init(cluster); + } + + @Before + public void beforeEach() + { + cluster.schemaChange("DROP KEYSPACE IF EXISTS harry"); + cluster.schemaChange("CREATE KEYSPACE harry WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};"); + } + + @Override + protected void flush(SchemaSpec schema) + { + cluster.forEach(i -> i.nodetool("flush", schema.keyspace)); + } + + @Override + protected void repair(SchemaSpec schema) + { + cluster.get(1).nodetool("repair", schema.keyspace); + } + + @Override + protected void compact(SchemaSpec schema) + { + cluster.forEach(i -> i.nodetool("compact", schema.keyspace)); + } + + @Override + protected void waitForIndexesQueryable(SchemaSpec schema) + { + SAIUtil.waitForIndexQueryable(cluster, schema.keyspace); + } +} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/sai/PagingSingleNodeSAITest.java b/test/distributed/org/apache/cassandra/fuzz/sai/PagingSingleNodeSAITest.java deleted file mode 100644 index f11a7cc3b663..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/sai/PagingSingleNodeSAITest.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.sai; - -import org.junit.BeforeClass; - -import org.apache.cassandra.distributed.Cluster; -import org.apache.cassandra.harry.sut.injvm.InJvmSut; -import org.apache.cassandra.harry.sut.injvm.InJvmSutBase; - -public class PagingSingleNodeSAITest extends SingleNodeSAITest -{ - private static final int FETCH_SIZE = 10; - - @BeforeClass - public static void before() throws Throwable - { - cluster = Cluster.build() - .withNodes(1) - .withConfig(InJvmSutBase.defaultConfig()) - .createWithoutStarting(); - cluster.setUncaughtExceptionsFilter(t -> { - logger.error("Caught exception, reporting during shutdown. Ignoring.", t); - return true; - }); - cluster.startup(); - cluster = init(cluster); - sut = new InJvmSut(cluster) { - @Override - public Object[][] execute(String cql, ConsistencyLevel cl, Object[] bindings) - { - if (cql.contains("SELECT")) - return super.execute(cql, cl, FETCH_SIZE, bindings); - return super.execute(cql, cl, bindings); - } - }; - } -} diff --git a/test/distributed/org/apache/cassandra/fuzz/sai/PagingStaticsTortureTest.java b/test/distributed/org/apache/cassandra/fuzz/sai/PagingStaticsTortureTest.java deleted file mode 100644 index dc81a277cdc0..000000000000 --- a/test/distributed/org/apache/cassandra/fuzz/sai/PagingStaticsTortureTest.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.fuzz.sai; - -import org.junit.BeforeClass; - -import org.apache.cassandra.distributed.Cluster; -import org.apache.cassandra.harry.sut.injvm.InJvmSut; -import org.apache.cassandra.harry.sut.injvm.InJvmSutBase; - -public class PagingStaticsTortureTest extends StaticsTortureTest -{ - private static final int FETCH_SIZE = 10; - - @BeforeClass - public static void before() throws Throwable - { - cluster = Cluster.build() - .withNodes(1) - .withConfig(InJvmSutBase.defaultConfig()) - .createWithoutStarting(); - cluster.setUncaughtExceptionsFilter(t -> { - logger.error("Caught exception, reporting during shutdown. Ignoring.", t); - return true; - }); - cluster.startup(); - cluster = init(cluster); - sut = new InJvmSut(cluster) { - @Override - public Object[][] execute(String cql, ConsistencyLevel cl, Object[] bindings) - { - if (cql.contains("SELECT")) - return super.execute(cql, cl, FETCH_SIZE, bindings); - return super.execute(cql, cl, bindings); - } - }; - } -} diff --git a/test/distributed/org/apache/cassandra/fuzz/sai/SingleNodeSAITest.java b/test/distributed/org/apache/cassandra/fuzz/sai/SingleNodeSAITest.java index bbe4ce83987f..37a9b1c184b9 100644 --- a/test/distributed/org/apache/cassandra/fuzz/sai/SingleNodeSAITest.java +++ b/test/distributed/org/apache/cassandra/fuzz/sai/SingleNodeSAITest.java @@ -18,290 +18,10 @@ package org.apache.cassandra.fuzz.sai; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -import org.junit.Test; - -import org.apache.cassandra.config.CassandraRelevantProperties; -import org.apache.cassandra.fuzz.harry.integration.model.IntegrationTestBase; -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.dsl.ReplayingHistoryBuilder; -import org.apache.cassandra.harry.gen.DataGenerators; -import org.apache.cassandra.harry.gen.EntropySource; -import org.apache.cassandra.harry.gen.rng.JdkRandomEntropySource; -import org.apache.cassandra.harry.model.QuiescentChecker; -import org.apache.cassandra.harry.model.SelectHelper; -import org.apache.cassandra.harry.model.reconciler.PartitionState; -import org.apache.cassandra.harry.model.reconciler.Reconciler; -import org.apache.cassandra.harry.operations.FilteringQuery; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.operations.Relation; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.tracker.DataTracker; -import org.apache.cassandra.harry.tracker.DefaultDataTracker; - -public class SingleNodeSAITest extends IntegrationTestBase +public class SingleNodeSAITest extends SingleNodeSAITestBase { - private static final int RUNS = 1; - - private static final int OPERATIONS_PER_RUN = 30_000; - private static final int REPAIR_SKIP = OPERATIONS_PER_RUN / 2; - private static final int FLUSH_SKIP = OPERATIONS_PER_RUN / 7; - private static final int VALIDATION_SKIP = OPERATIONS_PER_RUN / 100; - - private static final int NUM_PARTITIONS = OPERATIONS_PER_RUN / 1000; - protected static final int MAX_PARTITION_SIZE = 10_000; - private static final int UNIQUE_CELL_VALUES = 5; - - long seed = 1; - - @Test - public void basicSaiTest() - { - CassandraRelevantProperties.SAI_INTERSECTION_CLAUSE_LIMIT.setInt(6); - SchemaSpec schema = new SchemaSpec(KEYSPACE, "tbl1", - Arrays.asList(ColumnSpec.ck("pk1", ColumnSpec.int64Type), - ColumnSpec.ck("pk2", ColumnSpec.asciiType(4, 100)), - ColumnSpec.ck("pk3", ColumnSpec.int64Type)), - Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.asciiType(4, 100)), - ColumnSpec.ck("ck2", ColumnSpec.asciiType, true), - ColumnSpec.ck("ck3", ColumnSpec.int64Type)), - Arrays.asList(ColumnSpec.regularColumn("v1", ColumnSpec.asciiType(40, 100)), - ColumnSpec.regularColumn("v2", ColumnSpec.int64Type), - ColumnSpec.regularColumn("v3", ColumnSpec.int64Type)), - List.of(ColumnSpec.staticColumn("s1", ColumnSpec.asciiType(40, 100)))) - .withCompactionStrategy("LeveledCompactionStrategy"); - - sut.schemaChange(schema.compile().cql()); - sut.schemaChange(schema.cloneWithName(schema.keyspace, schema.table + "_debug").compile().cql()); - sut.schemaChange(String.format("CREATE INDEX %s_sai_idx ON %s.%s (%s) USING 'sai' ", - schema.regularColumns.get(0).name, - schema.keyspace, - schema.table, - schema.regularColumns.get(0).name)); - sut.schemaChange(String.format("CREATE INDEX %s_sai_idx ON %s.%s (%s) USING 'sai';", - schema.regularColumns.get(1).name, - schema.keyspace, - schema.table, - schema.regularColumns.get(1).name)); - sut.schemaChange(String.format("CREATE INDEX %s_sai_idx ON %s.%s (%s) USING 'sai';", - schema.regularColumns.get(2).name, - schema.keyspace, - schema.table, - schema.regularColumns.get(2).name)); - sut.schemaChange(String.format("CREATE INDEX %s_sai_idx ON %s.%s (%s) USING 'sai';", - schema.staticColumns.get(0).name, - schema.keyspace, - schema.table, - schema.staticColumns.get(0).name)); - - waitForIndexesQueryable(schema); - - DataTracker tracker = new DefaultDataTracker(); - TokenPlacementModel.ReplicationFactor rf = new TokenPlacementModel.SimpleReplicationFactor(cluster.size()); - ReplayingHistoryBuilder history = new ReplayingHistoryBuilder(seed, - MAX_PARTITION_SIZE, - MAX_PARTITION_SIZE, - tracker, - sut, - schema, - rf, - SystemUnderTest.ConsistencyLevel.QUORUM); - - for (int run = 0; run < RUNS; run++) - { - logger.info("Starting run {}/{}...", run + 1, RUNS); - EntropySource random = new JdkRandomEntropySource(run); - - // Populate the array of possible values for all operations in the run: - long[] values = new long[UNIQUE_CELL_VALUES]; - for (int i = 0; i < values.length; i++) - values[i] = random.next(); - - for (int i = 0; i < OPERATIONS_PER_RUN; i++) - { - int partitionIndex = random.nextInt(0, NUM_PARTITIONS); - - history.visitPartition(partitionIndex) - .insert(random.nextInt(MAX_PARTITION_SIZE), - new long[] { random.nextBoolean() ? DataGenerators.UNSET_DESCR : values[random.nextInt(values.length)], - random.nextBoolean() ? DataGenerators.UNSET_DESCR : values[random.nextInt(values.length)], - random.nextBoolean() ? DataGenerators.UNSET_DESCR : values[random.nextInt(values.length)] }, - new long[] { random.nextBoolean() ? DataGenerators.UNSET_DESCR : values[random.nextInt(values.length)] }); - - if (random.nextFloat() > 0.99f) - { - int row1 = random.nextInt(MAX_PARTITION_SIZE); - int row2 = random.nextInt(MAX_PARTITION_SIZE); - history.visitPartition(partitionIndex).deleteRowRange(Math.min(row1, row2), Math.max(row1, row2), - random.nextBoolean(), random.nextBoolean()); - } - else if (random.nextFloat() > 0.999f) - { - history.visitPartition(partitionIndex).deleteRowSlice(); - } - - if (random.nextFloat() > 0.995f) - { - history.visitPartition(partitionIndex).deleteColumns(); - } - - if (random.nextFloat() > 0.9995f) - { - history.visitPartition(partitionIndex).deletePartition(); - } - - if (i % REPAIR_SKIP == 0) - { - logger.debug("Repairing/flushing after operation {}...", i); - repair(schema); - } - else if (i % FLUSH_SKIP == 0) - { - logger.debug("Flushing after operation {}...", i); - flush(schema); - } - - if (i % VALIDATION_SKIP != 0) - continue; - - logger.debug("Validating partition at index {} after operation {} in run {}...", partitionIndex, i, run + 1); - - for (int j = 0; j < 10; j++) - { - List<Relation> relations = new ArrayList<>(); - - // For one text column and 2 numeric columns, we can use between 1 and 5 total relations. - int num = random.nextInt(1, 5); - - List<List<Relation.RelationKind>> pick = new ArrayList<>(); - //noinspection ArraysAsListWithZeroOrOneArgument - pick.add(new ArrayList<>(Arrays.asList(Relation.RelationKind.EQ))); // text column supports only EQ - pick.add(new ArrayList<>(Arrays.asList(Relation.RelationKind.EQ, Relation.RelationKind.GT, Relation.RelationKind.LT))); - pick.add(new ArrayList<>(Arrays.asList(Relation.RelationKind.EQ, Relation.RelationKind.GT, Relation.RelationKind.LT))); - - if (random.nextFloat() > 0.75f) - { - relations.addAll(Query.clusteringSliceQuery(schema, - partitionIndex, - random.next(), - random.next(), - random.nextBoolean(), - random.nextBoolean(), - false).relations); - } - - for (int k = 0; k < num; k++) - { - int column = random.nextInt(schema.regularColumns.size()); - Relation.RelationKind relationKind = pickKind(random, pick, column); - - if (relationKind != null) - relations.add(Relation.relation(relationKind, - schema.regularColumns.get(column), - values[random.nextInt(values.length)])); - } - - if (random.nextFloat() > 0.7f) - { - relations.add(Relation.relation(Relation.RelationKind.EQ, - schema.staticColumns.get(0), - values[random.nextInt(values.length)])); - } - - long pd = history.pdSelector().pdAtPosition(partitionIndex); - FilteringQuery query = new FilteringQuery(pd, false, relations, schema); - Reconciler reconciler = new Reconciler(history.pdSelector(), schema, history::visitor); - Set<ColumnSpec<?>> columns = new HashSet<>(schema.allColumns); - - PartitionState modelState = reconciler.inflatePartitionState(pd, tracker, query).filter(query); - - if (modelState.rows().size() > 0) - logger.debug("Model contains {} matching rows for query {}.", modelState.rows().size(), query); - - try - { - QuiescentChecker.validate(schema, - tracker, - columns, - modelState, - SelectHelper.execute(sut, history.clock(), query), - query); - - // Run the query again to see if the first execution caused an issue via read-repair: - QuiescentChecker.validate(schema, - tracker, - columns, - modelState, - SelectHelper.execute(sut, history.clock(), query), - query); - } - catch (Throwable t) - { - logger.debug("Partition index = {}, run = {}, j = {}, i = {}", partitionIndex, run, j, i); - - Query partitionQuery = Query.selectAllColumns(schema, pd, false); - QuiescentChecker.validate(schema, - tracker, - columns, - reconciler.inflatePartitionState(pd, tracker, partitionQuery), - SelectHelper.execute(sut, history.clock(), partitionQuery), - partitionQuery); - logger.debug("Partition state agrees. Throwing original error..."); - - throw t; - } - } - } - - if (run + 1 < RUNS) - { - logger.debug("Forcing compaction at the end of run {}...", run + 1); - compact(schema); - } - } - } - - protected void flush(SchemaSpec schema) + public SingleNodeSAITest() { - cluster.get(1).nodetool("flush", schema.keyspace, schema.table); - } - - protected void compact(SchemaSpec schema) - { - cluster.get(1).nodetool("compact", schema.keyspace); - } - - protected void repair(SchemaSpec schema) - { - // Repair is nonsensical for a single node, but a repair does flush first, so do that at least. - cluster.get(1).nodetool("flush", schema.keyspace, schema.table); - } - - protected void waitForIndexesQueryable(SchemaSpec schema) {} - - private static Relation.RelationKind pickKind(EntropySource random, List<List<Relation.RelationKind>> options, int column) - { - Relation.RelationKind kind = null; - - if (!options.get(column).isEmpty()) - { - List<Relation.RelationKind> possible = options.get(column); - int chosen = random.nextInt(possible.size()); - kind = possible.remove(chosen); - - if (kind == Relation.RelationKind.EQ) - possible.clear(); // EQ precludes LT and GT - else - possible.remove(Relation.RelationKind.EQ); // LT GT preclude EQ - } - - return kind; + super(); } -} \ No newline at end of file +} diff --git a/test/distributed/org/apache/cassandra/fuzz/sai/SingleNodeSAITestBase.java b/test/distributed/org/apache/cassandra/fuzz/sai/SingleNodeSAITestBase.java new file mode 100644 index 000000000000..b14fa6908ac2 --- /dev/null +++ b/test/distributed/org/apache/cassandra/fuzz/sai/SingleNodeSAITestBase.java @@ -0,0 +1,284 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.fuzz.sai; + + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.function.Consumer; + +import com.google.common.collect.Streams; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.IInstanceConfig; +import org.apache.cassandra.distributed.test.TestBaseImpl; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.dsl.HistoryBuilder; +import org.apache.cassandra.harry.dsl.HistoryBuilderHelper; +import org.apache.cassandra.harry.dsl.ReplayingHistoryBuilder; +import org.apache.cassandra.harry.execution.InJvmDTestVisitExecutor; +import org.apache.cassandra.harry.gen.EntropySource; +import org.apache.cassandra.harry.gen.Generator; +import org.apache.cassandra.harry.gen.Generators; +import org.apache.cassandra.harry.gen.SchemaGenerators; + +import static org.apache.cassandra.distributed.api.Feature.GOSSIP; +import static org.apache.cassandra.distributed.api.Feature.NETWORK; +import static org.apache.cassandra.harry.checker.TestHelper.withRandom; +import static org.apache.cassandra.harry.dsl.SingleOperationBuilder.IdxRelation; + +// TODO: "WITH OPTIONS = {'case_sensitive': 'false', 'normalize': 'true', 'ascii': 'true'};", +public abstract class SingleNodeSAITestBase extends TestBaseImpl +{ + private static final int OPERATIONS_PER_RUN = 30_000; + private static final int REPAIR_SKIP = OPERATIONS_PER_RUN / 2; + private static final int FLUSH_SKIP = OPERATIONS_PER_RUN / 7; + private static final int COMPACTION_SKIP = OPERATIONS_PER_RUN / 11; + + private static final int NUM_PARTITIONS = OPERATIONS_PER_RUN / 1000; + protected static final int MAX_PARTITION_SIZE = 10_000; + private static final int UNIQUE_CELL_VALUES = 5; + + protected static final Logger logger = LoggerFactory.getLogger(SingleNodeSAITest.class); + protected static Cluster cluster; + + protected SingleNodeSAITestBase() + { + } + + @BeforeClass + public static void before() throws Throwable + { + init(1, + // At lower fetch sizes, queries w/ hundreds or thousands of matches can take a very long time. + defaultConfig().andThen(c -> c.set("range_request_timeout", "180s") + .set("read_request_timeout", "180s") + .set("write_request_timeout", "180s") + .set("native_transport_timeout", "180s") + .set("slow_query_log_timeout", "180s") + .with(GOSSIP).with(NETWORK)) + ); + } + + protected static void init(int nodes, Consumer<IInstanceConfig> cfg) throws Throwable + { + cluster = Cluster.build() + .withNodes(nodes) + .withConfig(cfg) + .createWithoutStarting(); + cluster.startup(); + cluster = init(cluster); + } + @AfterClass + public static void afterClass() + { + cluster.close(); + } + + @Before + public void beforeEach() + { + cluster.schemaChange("DROP KEYSPACE IF EXISTS harry"); + cluster.schemaChange("CREATE KEYSPACE harry WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1};"); + } + + @Test + public void simplifiedSaiTest() + { + withRandom(rng -> basicSaiTest(rng, SchemaGenerators.trivialSchema("harry", "simplified", 1000).generate(rng))); + } + + @Test + public void basicSaiTest() + { + Generator<SchemaSpec> schemaGen = schemaGenerator(); + withRandom(rng -> { + basicSaiTest(rng, schemaGen.generate(rng)); + }); + } + + private void basicSaiTest(EntropySource rng, SchemaSpec schema) + { + Set<Integer> usedPartitions = new HashSet<>(); + logger.info(schema.compile()); + + Generator<Integer> globalPkGen = Generators.int32(0, Math.min(NUM_PARTITIONS, schema.valueGenerators.pkPopulation())); + Generator<Integer> ckGen = Generators.int32(0, schema.valueGenerators.ckPopulation()); + + CassandraRelevantProperties.SAI_INTERSECTION_CLAUSE_LIMIT.setInt(100); + beforeEach(); + cluster.forEach(i -> i.nodetool("disableautocompaction")); + + cluster.schemaChange(schema.compile()); + cluster.schemaChange(schema.compile().replace(schema.keyspace + "." + schema.table, + schema.keyspace + ".debug_table")); + Streams.concat(schema.clusteringKeys.stream(), + schema.regularColumns.stream(), + schema.staticColumns.stream()) + .forEach(column -> { + cluster.schemaChange(String.format("CREATE INDEX %s_sai_idx ON %s.%s (%s) USING 'sai' ", + column.name, + schema.keyspace, + schema.table, + column.name)); + }); + + waitForIndexesQueryable(schema); + + HistoryBuilder history = new ReplayingHistoryBuilder(schema.valueGenerators, + (hb) -> InJvmDTestVisitExecutor.builder() + .pageSizeSelector(pageSizeSelector(rng)) + .consistencyLevel(consistencyLevelSelector()) + .doubleWriting(schema, hb, cluster, "debug_table")); + List<Integer> partitions = new ArrayList<>(); + for (int j = 0; j < 5; j++) + { + int picked = globalPkGen.generate(rng); + if (usedPartitions.contains(picked)) + continue; + partitions.add(picked); + } + + usedPartitions.addAll(partitions); + if (partitions.isEmpty()) + return; + + Generator<Integer> pkGen = Generators.pick(partitions); + for (int i = 0; i < OPERATIONS_PER_RUN; i++) + { + int partitionIndex = pkGen.generate(rng); + HistoryBuilderHelper.insertRandomData(schema, partitionIndex, ckGen.generate(rng), rng, 0.5d, history); + + if (rng.nextFloat() > 0.99f) + { + int row1 = ckGen.generate(rng); + int row2 = ckGen.generate(rng); + history.deleteRowRange(partitionIndex, + Math.min(row1, row2), + Math.max(row1, row2), + rng.nextInt(schema.clusteringKeys.size()), + rng.nextBoolean(), + rng.nextBoolean()); + } + + if (rng.nextFloat() > 0.995f) + HistoryBuilderHelper.deleteRandomColumns(schema, partitionIndex, ckGen.generate(rng), rng, history); + + if (rng.nextFloat() > 0.9995f) + history.deletePartition(partitionIndex); + + if (i % REPAIR_SKIP == 0) + history.custom(() -> repair(schema), "Repair"); + else if (i % FLUSH_SKIP == 0) + history.custom(() -> flush(schema), "Flush"); + else if (i % COMPACTION_SKIP == 0) + history.custom(() -> compact(schema), "Compact"); + + if (i > 0 && i % 1000 == 0) + { + for (int j = 0; j < 5; j++) + { + List<IdxRelation> regularRelations = HistoryBuilderHelper.generateValueRelations(rng, schema.regularColumns.size(), + column -> Math.min(schema.valueGenerators.regularPopulation(column), UNIQUE_CELL_VALUES)); + List<IdxRelation> staticRelations = HistoryBuilderHelper.generateValueRelations(rng, schema.staticColumns.size(), + column -> Math.min(schema.valueGenerators.staticPopulation(column), UNIQUE_CELL_VALUES)); + history.select(pkGen.generate(rng), + HistoryBuilderHelper.generateClusteringRelations(rng, schema.clusteringKeys.size(), ckGen).toArray(new IdxRelation[0]), + regularRelations.toArray(new IdxRelation[regularRelations.size()]), + staticRelations.toArray(new IdxRelation[staticRelations.size()])); + } + } + } + } + + protected Generator<SchemaSpec> schemaGenerator() + { + SchemaSpec.OptionsBuilder builder = SchemaSpec.optionsBuilder().compactionStrategy("LeveledCompactionStrategy"); + return SchemaGenerators.schemaSpecGen(KEYSPACE, "basic_sai", MAX_PARTITION_SIZE, builder); + } + + protected void flush(SchemaSpec schema) + { + cluster.get(1).nodetool("flush", schema.keyspace, schema.table); + } + + protected void compact(SchemaSpec schema) + { + cluster.get(1).nodetool("compact", schema.keyspace); + } + + protected void repair(SchemaSpec schema) + { + // Repair is nonsensical for a single node, but a repair does flush first, so do that at least. + cluster.get(1).nodetool("flush", schema.keyspace, schema.table); + } + + protected void waitForIndexesQueryable(SchemaSpec schema) {} + + public static Consumer<IInstanceConfig> defaultConfig() + { + return (cfg) -> { + cfg.set("row_cache_size", "50MiB") + .set("index_summary_capacity", "50MiB") + .set("counter_cache_size", "50MiB") + .set("key_cache_size", "50MiB") + .set("file_cache_size", "50MiB") + .set("index_summary_capacity", "50MiB") + .set("memtable_heap_space", "128MiB") + .set("memtable_offheap_space", "128MiB") + .set("memtable_flush_writers", 1) + .set("concurrent_compactors", 1) + .set("concurrent_reads", 5) + .set("concurrent_writes", 5) + .set("compaction_throughput_mb_per_sec", 10) + .set("hinted_handoff_enabled", false); + }; + } + + protected InJvmDTestVisitExecutor.ConsistencyLevelSelector consistencyLevelSelector() + { + return visit -> { + if (visit.selectOnly) + return ConsistencyLevel.ALL; + + // The goal here is to make replicas as out of date as possible, modulo the efforts of repair + // and read-repair in the test itself. + return ConsistencyLevel.NODE_LOCAL; + + }; + } + + protected InJvmDTestVisitExecutor.PageSizeSelector pageSizeSelector(EntropySource rng) + { + // Chosing a fetch size has implications for how well this test will excercise paging, short-read protection, and + // other important parts of the distributed query apparatus. This should be set low enough to ensure a significant + // number of queries during validation page, but not too low that more expesive queries time out and fail the test. + return lts -> rng.nextInt(1, 20); + } +} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/sai/StaticsTortureTest.java b/test/distributed/org/apache/cassandra/fuzz/sai/StaticsTortureTest.java index e68c204d2368..07844b045e31 100644 --- a/test/distributed/org/apache/cassandra/fuzz/sai/StaticsTortureTest.java +++ b/test/distributed/org/apache/cassandra/fuzz/sai/StaticsTortureTest.java @@ -18,40 +18,30 @@ package org.apache.cassandra.fuzz.sai; -import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.function.Consumer; -import java.util.stream.Stream; import org.junit.Test; import org.apache.cassandra.config.CassandraRelevantProperties; -import org.apache.cassandra.fuzz.harry.integration.model.IntegrationTestBase; -import org.apache.cassandra.harry.data.ResultSetRow; -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.ddl.SchemaSpec; +import org.apache.cassandra.distributed.test.IntegrationTestBase; +import org.apache.cassandra.harry.ColumnSpec; +import org.apache.cassandra.harry.MagicConstants; +import org.apache.cassandra.harry.SchemaSpec; import org.apache.cassandra.harry.dsl.ReplayingHistoryBuilder; -import org.apache.cassandra.harry.gen.DataGenerators; +import org.apache.cassandra.harry.execution.InJvmDTestVisitExecutor; import org.apache.cassandra.harry.gen.EntropySource; +import org.apache.cassandra.harry.gen.Generator; +import org.apache.cassandra.harry.gen.Generators; import org.apache.cassandra.harry.gen.rng.JdkRandomEntropySource; -import org.apache.cassandra.harry.model.AgainstSutChecker; -import org.apache.cassandra.harry.model.Model; -import org.apache.cassandra.harry.model.SelectHelper; -import org.apache.cassandra.harry.operations.CompiledStatement; -import org.apache.cassandra.harry.operations.FilteringQuery; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.operations.Relation; -import org.apache.cassandra.harry.sut.DoubleWritingSut; -import org.apache.cassandra.harry.sut.QueryModifyingSut; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.tracker.DataTracker; -import org.apache.cassandra.harry.tracker.DefaultDataTracker; +import org.apache.cassandra.harry.util.BitSet; + +import static org.apache.cassandra.harry.dsl.HistoryBuilderHelper.generateClusteringRelations; +import static org.apache.cassandra.harry.dsl.HistoryBuilderHelper.generateValueRelations; +import static org.apache.cassandra.harry.dsl.SingleOperationBuilder.IdxRelation; public class StaticsTortureTest extends IntegrationTestBase { - private static final long SEED = 1; private static final int MAX_PARTITION_SIZE = 10_000; private static final int NUM_PARTITIONS = 100; private static final int UNIQUE_CELL_VALUES = 5; @@ -61,206 +51,139 @@ public void staticsTortureTest() { CassandraRelevantProperties.SAI_INTERSECTION_CLAUSE_LIMIT.setInt(6); int idx = 0; - staticsTortureTest(Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.asciiType(4, 100)), - ColumnSpec.ck("ck2", ColumnSpec.asciiType), - ColumnSpec.ck("ck3", ColumnSpec.int64Type)), - idx++); - - for (boolean b1 : new boolean[]{ true, false }) - for (boolean b2 : new boolean[]{ true, false }) - for (boolean b3 : new boolean[]{ true, false }) + for (boolean b1 : new boolean[]{ false, true }) + for (boolean b2 : new boolean[]{ false, true }) + for (boolean b3 : new boolean[]{ false, true }) { - staticsTortureTest(Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.asciiType(4, 100), b1), - ColumnSpec.ck("ck2", ColumnSpec.asciiType, b2), - ColumnSpec.ck("ck3", ColumnSpec.int64Type, b3)), + staticsTortureTest(Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.asciiType, Generators.ascii(4, 1000), b1), + ColumnSpec.ck("ck2", ColumnSpec.asciiType, Generators.ascii(4, 1000), b2), + ColumnSpec.ck("ck3", ColumnSpec.int64Type, Generators.int64(), b3)), idx++); } } public void staticsTortureTest(List<ColumnSpec<?>> cks, int idx) { - SchemaSpec schema = new SchemaSpec(KEYSPACE, "tbl" + idx, - Arrays.asList(ColumnSpec.ck("pk1", ColumnSpec.int64Type), - ColumnSpec.ck("pk2", ColumnSpec.asciiType(4, 100)), - ColumnSpec.ck("pk3", ColumnSpec.int64Type)), + SchemaSpec schema = new SchemaSpec(idx, + 10_000, + KEYSPACE, + "tbl" + idx, + Arrays.asList(ColumnSpec.pk("pk1", ColumnSpec.int64Type, Generators.int64()), + ColumnSpec.pk("pk2", ColumnSpec.asciiType, Generators.ascii(4, 1000)), + ColumnSpec.pk("pk3", ColumnSpec.int64Type, Generators.int64())), cks, - Arrays.asList(ColumnSpec.regularColumn("v1", ColumnSpec.asciiType(40, 100)), + Arrays.asList(ColumnSpec.regularColumn("v1", ColumnSpec.asciiType, Generators.ascii(4, 1000)), ColumnSpec.regularColumn("v2", ColumnSpec.int64Type), ColumnSpec.regularColumn("v3", ColumnSpec.int64Type)), - List.of(ColumnSpec.staticColumn("s1", ColumnSpec.asciiType(40, 100)), - ColumnSpec.staticColumn("s2", ColumnSpec.int64Type), - ColumnSpec.staticColumn("s3", ColumnSpec.asciiType(40, 100)) - )); + Arrays.asList(ColumnSpec.staticColumn("s1", ColumnSpec.asciiType, Generators.ascii(4, 1000)), + ColumnSpec.staticColumn("s2", ColumnSpec.int64Type), + ColumnSpec.staticColumn("s3", ColumnSpec.asciiType, Generators.ascii(4, 1000)))); - sut.schemaChange(schema.compile().cql()); - SchemaSpec debugSchema = schema.cloneWithName(schema.keyspace, schema.table + "_debug"); - sut.schemaChange(schema.cloneWithName(schema.keyspace, schema.table + "_debug").compile().cql()); - sut.schemaChange(String.format("CREATE INDEX %s_%s_sai_idx ON %s.%s (%s) USING 'sai' " + + + cluster.schemaChange(schema.compile()); + cluster.get(1).nodetool("disableautocompaction"); + cluster.schemaChange(String.format("CREATE INDEX %s_%s_sai_idx ON %s.%s (%s) USING 'sai' " + "WITH OPTIONS = {'case_sensitive': 'false', 'normalize': 'true', 'ascii': 'true'};", schema.table, schema.regularColumns.get(0).name, schema.keyspace, schema.table, schema.regularColumns.get(0).name)); - sut.schemaChange(String.format("CREATE INDEX %s_%s_sai_idx ON %s.%s (%s) USING 'sai';", + cluster.schemaChange(String.format("CREATE INDEX %s_%s_sai_idx ON %s.%s (%s) USING 'sai';", schema.table, schema.regularColumns.get(1).name, schema.keyspace, schema.table, schema.regularColumns.get(1).name)); - sut.schemaChange(String.format("CREATE INDEX %s_%s_sai_idx ON %s.%s (%s) USING 'sai';", + cluster.schemaChange(String.format("CREATE INDEX %s_%s_sai_idx ON %s.%s (%s) USING 'sai';", schema.table, schema.regularColumns.get(2).name, schema.keyspace, schema.table, schema.regularColumns.get(2).name)); - sut.schemaChange(String.format("CREATE INDEX %s_%s_sai_idx ON %s.%s (%s) USING 'sai';", + cluster.schemaChange(String.format("CREATE INDEX %s_%s_sai_idx ON %s.%s (%s) USING 'sai';", schema.table, schema.staticColumns.get(0).name, schema.keyspace, schema.table, schema.staticColumns.get(0).name)); - sut.schemaChange(String.format("CREATE INDEX %s_%s_sai_idx ON %s.%s (%s) USING 'sai';", + cluster.schemaChange(String.format("CREATE INDEX %s_%s_sai_idx ON %s.%s (%s) USING 'sai';", schema.table, schema.staticColumns.get(1).name, schema.keyspace, schema.table, schema.staticColumns.get(1).name)); - sut.schemaChange(String.format("CREATE INDEX %s_%s_sai_idx ON %s.%s (%s) USING 'sai';", + cluster.schemaChange(String.format("CREATE INDEX %s_%s_sai_idx ON %s.%s (%s) USING 'sai';", schema.table, schema.staticColumns.get(2).name, schema.keyspace, schema.table, schema.staticColumns.get(2).name)); - DataTracker tracker = new DefaultDataTracker(); - TokenPlacementModel.ReplicationFactor rf = new TokenPlacementModel.SimpleReplicationFactor(cluster.size()); - ReplayingHistoryBuilder history = new ReplayingHistoryBuilder(SEED + idx, - MAX_PARTITION_SIZE, - 100, - tracker, - new DoubleWritingSut(sut, - new QueryModifyingSut(sut, - schema.keyspace + "." + schema.table, - debugSchema.keyspace + "." + debugSchema.table)), - schema, - rf, - SystemUnderTest.ConsistencyLevel.QUORUM); + Generator<BitSet> regularColumnBitSet = Generators.bitSet(schema.regularColumns.size()); + Generator<BitSet> staticColumnBitSet = Generators.bitSet(schema.staticColumns.size()); EntropySource rng = new JdkRandomEntropySource(1l); - long[] values = new long[UNIQUE_CELL_VALUES]; - for (int i = 0; i < values.length; i++) - values[i] = rng.next(); + + ReplayingHistoryBuilder history = new ReplayingHistoryBuilder(schema.valueGenerators, hb -> { + return InJvmDTestVisitExecutor.builder().pageSizeSelector(i -> rng.nextInt(1, 10)).build(schema, hb, cluster); + }); + for (int i = 0; i < NUM_PARTITIONS; i++) { - history.visitPartition(i) - .insert(1, - new long[]{ rng.nextBoolean() ? DataGenerators.UNSET_DESCR : values[rng.nextInt(values.length)], - rng.nextBoolean() ? DataGenerators.UNSET_DESCR : values[rng.nextInt(values.length)], - rng.nextBoolean() ? DataGenerators.UNSET_DESCR : values[rng.nextInt(values.length)] + history.insert(i, rng.nextInt(5), + new int[]{ rng.nextBoolean() ? MagicConstants.UNSET_IDX : rng.nextInt(UNIQUE_CELL_VALUES), + rng.nextBoolean() ? MagicConstants.UNSET_IDX : rng.nextInt(UNIQUE_CELL_VALUES), + rng.nextBoolean() ? MagicConstants.UNSET_IDX : rng.nextInt(UNIQUE_CELL_VALUES) }, - new long[]{ rng.nextBoolean() ? DataGenerators.UNSET_DESCR : values[rng.nextInt(values.length)], - rng.nextBoolean() ? DataGenerators.UNSET_DESCR : values[rng.nextInt(values.length)], - rng.nextBoolean() ? DataGenerators.UNSET_DESCR : values[rng.nextInt(values.length)] - }); - history.visitPartition(i) - .insert(5, - new long[]{ rng.nextBoolean() ? DataGenerators.UNSET_DESCR : values[rng.nextInt(values.length)], - rng.nextBoolean() ? DataGenerators.UNSET_DESCR : values[rng.nextInt(values.length)], - rng.nextBoolean() ? DataGenerators.UNSET_DESCR : values[rng.nextInt(values.length)] + new int[]{ rng.nextBoolean() ? MagicConstants.UNSET_IDX : rng.nextInt(UNIQUE_CELL_VALUES), + rng.nextBoolean() ? MagicConstants.UNSET_IDX : rng.nextInt(UNIQUE_CELL_VALUES), + rng.nextBoolean() ? MagicConstants.UNSET_IDX : rng.nextInt(UNIQUE_CELL_VALUES) + }); + history.insert(i,rng.nextInt(5), + new int[]{ rng.nextBoolean() ? MagicConstants.UNSET_IDX : rng.nextInt(UNIQUE_CELL_VALUES), + rng.nextBoolean() ? MagicConstants.UNSET_IDX : rng.nextInt(UNIQUE_CELL_VALUES), + rng.nextBoolean() ? MagicConstants.UNSET_IDX : rng.nextInt(UNIQUE_CELL_VALUES) }, - new long[]{ rng.nextBoolean() ? DataGenerators.UNSET_DESCR : values[rng.nextInt(values.length)], - rng.nextBoolean() ? DataGenerators.UNSET_DESCR : values[rng.nextInt(values.length)], - rng.nextBoolean() ? DataGenerators.UNSET_DESCR : values[rng.nextInt(values.length)] - }); + new int[]{ rng.nextBoolean() ? MagicConstants.UNSET_IDX : rng.nextInt(UNIQUE_CELL_VALUES), + rng.nextBoolean() ? MagicConstants.UNSET_IDX : rng.nextInt(UNIQUE_CELL_VALUES), + rng.nextBoolean() ? MagicConstants.UNSET_IDX : rng.nextInt(UNIQUE_CELL_VALUES) + }); if (rng.nextFloat() > 0.9f) { - history.visitPartition(i) - .deleteRowRange(rng.nextInt(5), rng.nextInt(5), rng.nextBoolean(), rng.nextBoolean()); + history.deleteRowRange(i, rng.nextInt(5), rng.nextInt(5), + rng.nextInt(1, schema.clusteringKeys.size()), + rng.nextBoolean(), rng.nextBoolean()); } if (rng.nextFloat() > 0.9f) { - history.visitPartition(i) - .deleteColumns(); + history.deleteColumns(i, rng.nextInt(5), regularColumnBitSet.generate(rng), staticColumnBitSet.generate(rng)); } if (i % 50 == 0) cluster.get(1).nodetool("flush", schema.keyspace, schema.table); - } - Model model = new AgainstSutChecker(tracker, history.clock(), sut, schema, schema.cloneWithName(schema.keyspace, debugSchema.table)) { - @Override - protected List<ResultSetRow> executeOnDebugSchema(Query query) - { - CompiledStatement s2 = query.toSelectStatement(doubleWriteTable.allColumnsSet, true) - .withSchema(schema.keyspace, schema.table, doubleWriteTable.keyspace, doubleWriteTable.table) - .withFiltering(); - return SelectHelper.execute(sut, clock, s2, schema); - } - }; + if (i % 100 == 0) + cluster.get(1).nodetool("compact", schema.keyspace, schema.table); + } + Generator<Integer> ckIdxGen = Generators.int32(0, MAX_PARTITION_SIZE); for (int pdx = 0; pdx < NUM_PARTITIONS; pdx++) { - long pd = history.pdSelector().pdAtPosition(pdx); - history.pdSelector().pdAtPosition(1); - for (int i1 = 0; i1 < values.length; i1++) - for (int i2 = 0; i2 < values.length; i2++) - for (int i3 = 0; i3 < values.length; i3++) - { - long[] descriptors = new long[]{ values[i1], values[i2], values[i3], - values[i1], values[i2], values[i3] }; - List<Relation> relations = new ArrayList<>(); - Stream.concat(schema.regularColumns.stream(), - schema.staticColumns.stream()) - .forEach(new Consumer<>() - { - int counter = 0; - - @Override - public void accept(ColumnSpec<?> column) - { - if (rng.nextBoolean()) - return; - - if (column.type.toString().equals(ColumnSpec.int64Type.toString())) - { - if (rng.nextBoolean()) - { - relations.add(Relation.relation(Relation.RelationKind.EQ, - column, - descriptors[counter])); - } - else - { - Relation.relation(rng.nextBoolean() ? Relation.RelationKind.LT : Relation.RelationKind.GT, - column, - descriptors[counter]); - } - } - else - { - Relation.relation(Relation.RelationKind.EQ, - column, - descriptors[counter]); - } - - counter++; - } - }); - - // Without partition key - model.validate(new FilteringQuery(-1, false, relations, schema) - { - @Override - public CompiledStatement toSelectStatement() - { - return SelectHelper.select(schemaSpec, null, schemaSpec.allColumnsSet, relations, reverse, true); - } - }); - model.validate(new FilteringQuery(pd, false, relations, schema)); - } + for (int i = 0; i < 10; i++) + { + List<IdxRelation> ckRelations = generateClusteringRelations(rng, schema.clusteringKeys.size(), ckIdxGen); + List<IdxRelation> regularRelations = generateValueRelations(rng, schema.regularColumns.size(), + column -> Math.min(schema.valueGenerators.regularPopulation(column), MAX_PARTITION_SIZE)); + List<IdxRelation> staticRelations = generateValueRelations(rng, schema.staticColumns.size(), + column -> Math.min(schema.valueGenerators.staticPopulation(column), MAX_PARTITION_SIZE)); + history.select(pdx, + ckRelations.toArray(new IdxRelation[0]), + regularRelations.toArray(new IdxRelation[0]), + staticRelations.toArray(new IdxRelation[0])); + } } } } \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/topology/HarryTopologyMixupTest.java b/test/distributed/org/apache/cassandra/fuzz/topology/HarryTopologyMixupTest.java index 11fce664a1fa..6f83ae00136c 100644 --- a/test/distributed/org/apache/cassandra/fuzz/topology/HarryTopologyMixupTest.java +++ b/test/distributed/org/apache/cassandra/fuzz/topology/HarryTopologyMixupTest.java @@ -18,12 +18,18 @@ package org.apache.cassandra.fuzz.topology; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.BiFunction; import java.util.function.Consumer; import java.util.function.Function; - import javax.annotation.Nullable; +import com.google.common.base.Throwables; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import accord.utils.Gen; import accord.utils.Property; import accord.utils.Property.Command; @@ -32,16 +38,30 @@ import accord.utils.RandomSource; import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.api.IInstanceConfig; -import org.apache.cassandra.harry.HarryHelper; +import org.apache.cassandra.exceptions.RequestTimeoutException; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.dsl.HistoryBuilder; import org.apache.cassandra.harry.dsl.ReplayingHistoryBuilder; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.sut.injvm.InJvmSut; +import org.apache.cassandra.harry.execution.InJvmDTestVisitExecutor; +import org.apache.cassandra.harry.gen.EntropySource; +import org.apache.cassandra.harry.gen.Generator; +import org.apache.cassandra.harry.gen.Generators; +import org.apache.cassandra.harry.gen.SchemaGenerators; +import org.apache.cassandra.harry.gen.rng.JdkRandomEntropySource; +import org.apache.cassandra.utils.AssertionUtils; +import org.assertj.core.api.Condition; import static org.apache.cassandra.distributed.shared.ClusterUtils.waitForCMSToQuiesce; public class HarryTopologyMixupTest extends TopologyMixupTestBase<HarryTopologyMixupTest.Spec> { + protected static final Condition<Object> TIMEOUT_CHECKER = AssertionUtils.isInstanceof(RequestTimeoutException.class); + private static final Logger logger = LoggerFactory.getLogger(HarryTopologyMixupTest.class); + + public HarryTopologyMixupTest() + { + } + @Override protected Gen<State<Spec>> stateGen() { @@ -61,49 +81,73 @@ protected void destroyState(State<Spec> state, @Nullable Throwable cause) if (cause != null) return; if (((HarryState) state).numInserts > 0) { - // do one last read just to make sure we validate the data... - var harry = state.schemaSpec.harry; - harry.validateAll(harry.quiescentLocalChecker()); + for (Integer pkIdx : state.schema.pkGen.generated()) + state.schema.harry.selectPartition(pkIdx); } } - private static Spec createSchemaSpec(RandomSource rs, Cluster cluster) + private static BiFunction<RandomSource, Cluster, Spec> createSchemaSpec() { - ReplayingHistoryBuilder harry = HarryHelper.dataGen(rs.nextLong(), - new InJvmSut(cluster), - new TokenPlacementModel.SimpleReplicationFactor(3), - SystemUnderTest.ConsistencyLevel.ALL); - cluster.schemaChange(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 3};", HarryHelper.KEYSPACE)); - var schema = harry.schema(); - cluster.schemaChange(schema.compile().cql()); - waitForCMSToQuiesce(cluster, cluster.get(1)); - return new Spec(harry); + return (rs, cluster) -> { + EntropySource rng = new JdkRandomEntropySource(rs.nextLong()); + Generator<SchemaSpec> schemaGen = SchemaGenerators.schemaSpecGen("harry", "table", 1000);; + SchemaSpec schema = schemaGen.generate(rng); + + HistoryBuilder harry = new ReplayingHistoryBuilder(schema.valueGenerators, + hb -> { + InJvmDTestVisitExecutor.Builder builder = InJvmDTestVisitExecutor.builder(); + return builder.nodeSelector(new InJvmDTestVisitExecutor.NodeSelector() + { + private final AtomicLong cnt = new AtomicLong(); + + @Override + public int select(long lts) + { + for (int i = 0; i < 42; i++) + { + int selected = (int) (cnt.getAndIncrement() % cluster.size() + 1); + if (!cluster.get(selected).isShutdown()) + return selected; + } + throw new IllegalStateException("Unable to find an alive instance"); + } + }) + .retryPolicy(t -> { + t = Throwables.getRootCause(t); + if (!TIMEOUT_CHECKER.matches(t)) + return false; + return false; + }) + .build(schema, hb, cluster); + }); + cluster.schemaChange(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 3};", schema.keyspace)); + cluster.schemaChange(schema.compile()); + waitForCMSToQuiesce(cluster, cluster.get(1)); + return new Spec(harry, schema); + }; } - private static BiFunction<RandomSource, State<Spec>, Command<State<Spec>, Void, ?>> cqlOperations(Spec spec) + private static class HarryCommand extends SimpleCommand<State<Spec>> { - class HarryCommand extends SimpleCommand<State<Spec>> + HarryCommand(Function<State<Spec>, String> name, Consumer<State<Spec>> fn) { - HarryCommand(Function<State<Spec>, String> name, Consumer<State<Spec>> fn) - { - super(name, fn); - } + super(name, fn); + } - @Override - public PreCheckResult checkPreconditions(State<Spec> state) - { - int clusterSize = state.topologyHistory.up().length; - return clusterSize >= 3 ? PreCheckResult.Ok : PreCheckResult.Ignore; - } + @Override + public PreCheckResult checkPreconditions(State<Spec> state) + { + int clusterSize = state.topologyHistory.up().length; + return clusterSize >= 3 ? PreCheckResult.Ok : PreCheckResult.Ignore; } + } + + private static CommandGen<Spec> cqlOperations(Spec spec) + { Command<State<Spec>, Void, ?> insert = new HarryCommand(state -> "Harry Insert" + state.commandNamePostfix(), state -> { spec.harry.insert(); ((HarryState) state).numInserts++; }); - Command<State<Spec>, Void, ?> validateAll = new HarryCommand(state -> "Harry Validate All" + state.commandNamePostfix(), state -> { - spec.harry.validateAll(spec.harry.quiescentLocalChecker()); - ((HarryState) state).numInserts = 0; - }); return (rs, state) -> { HarryState harryState = (HarryState) state; TopologyHistory history = state.topologyHistory; @@ -111,43 +155,68 @@ public PreCheckResult checkPreconditions(State<Spec> state) if (harryState.generation != history.generation()) { harryState.generation = history.generation(); - return validateAll; + return validateAll(state); } if ((harryState.numInserts > 0 && rs.decide(0.2))) // 20% of the time do reads - return validateAll; + return validateAll(state); return insert; }; } - public static class Spec implements TopologyMixupTestBase.SchemaSpec + private static Command<State<Spec>, Void, ?> validateAll(State<Spec> state) + { + Spec spec = state.schema; + List<Command<State<Spec>, Void, ?>> reads = new ArrayList<>(); + + for (Integer pkIdx : spec.pkGen.generated()) + { + long pd = spec.harry.valueGenerators().pkGen().descriptorAt(pkIdx); + reads.add(new HarryCommand(s -> String.format("Harry Validate pd=%d%s", pd, state.commandNamePostfix()), s -> spec.harry.selectPartition(pkIdx))); + } + reads.add(new HarryCommand(s -> "Reset Harry Write State" + state.commandNamePostfix(), s -> ((HarryState) s).numInserts = 0)); + return Property.multistep(reads); + } + + public static class Spec implements Schema { - private final ReplayingHistoryBuilder harry; + private final Generators.TrackingGenerator<Integer> pkGen; + private final HistoryBuilder harry; + private final SchemaSpec schema; - public Spec(ReplayingHistoryBuilder harry) + public Spec(HistoryBuilder harry, SchemaSpec schema) { this.harry = harry; + this.schema = schema; + this.pkGen = Generators.tracking(Generators.int32(0, schema.valueGenerators.pkPopulation())); } @Override - public String name() + public String table() { - return harry.schema().table; + return schema.table; } @Override - public String keyspaceName() + public String keyspace() { - return HarryHelper.KEYSPACE; + return schema.keyspace; + } + + @Override + public String createSchema() + { + return schema.compile(); } } - public static class HarryState extends State<Spec> + public class HarryState extends State<Spec> { private long generation; private int numInserts = 0; + public HarryState(RandomSource rs) { - super(rs, HarryTopologyMixupTest::createSchemaSpec, HarryTopologyMixupTest::cqlOperations); + super(rs, createSchemaSpec(), HarryTopologyMixupTest::cqlOperations); } @Override @@ -156,4 +225,4 @@ protected void onConfigure(IInstanceConfig config) config.set("metadata_snapshot_frequency", 5); } } -} +} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/topology/TopologyMixupTestBase.java b/test/distributed/org/apache/cassandra/fuzz/topology/TopologyMixupTestBase.java index 8e8c57b13237..192713d361c3 100644 --- a/test/distributed/org/apache/cassandra/fuzz/topology/TopologyMixupTestBase.java +++ b/test/distributed/org/apache/cassandra/fuzz/topology/TopologyMixupTestBase.java @@ -18,6 +18,7 @@ package org.apache.cassandra.fuzz.topology; +import javax.annotation.Nullable; import java.io.IOException; import java.io.UncheckedIOException; import java.net.InetSocketAddress; @@ -25,7 +26,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Comparator; import java.util.EnumSet; import java.util.HashSet; import java.util.LinkedHashMap; @@ -33,15 +33,27 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.Objects; +import java.util.Optional; import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BiConsumer; import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.Collectors; -import javax.annotation.Nullable; +import java.util.stream.Stream; +import com.google.common.base.Throwables; +import com.google.common.collect.Iterables; import com.google.common.collect.Sets; +import org.agrona.collections.Int2ObjectHashMap; +import org.agrona.collections.IntArrayList; +import org.agrona.collections.IntHashSet; +import org.apache.cassandra.distributed.Constants; +import org.apache.cassandra.distributed.api.ICoordinator; +import org.apache.cassandra.distributed.api.Row; +import org.apache.cassandra.distributed.api.SimpleQueryResult; + import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,9 +65,6 @@ import accord.utils.Property.Command; import accord.utils.Property.SimpleCommand; import accord.utils.RandomSource; -import org.agrona.collections.Int2ObjectHashMap; -import org.agrona.collections.IntArrayList; -import org.agrona.collections.IntHashSet; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.YamlConfigurationLoader; import org.apache.cassandra.distributed.Cluster; @@ -68,23 +77,33 @@ import org.apache.cassandra.distributed.impl.InstanceConfig; import org.apache.cassandra.distributed.shared.ClusterUtils; import org.apache.cassandra.distributed.test.TestBaseImpl; +import org.apache.cassandra.harry.model.TokenPlacementModelHelper; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.schema.ReplicationParams; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; +import org.apache.cassandra.tcm.Epoch; +import org.apache.cassandra.tools.nodetool.formatter.TableBuilder; +import org.apache.cassandra.utils.Clock; import org.apache.cassandra.utils.ConfigGenBuilder; +import org.apache.cassandra.utils.Retry; import static accord.utils.Property.commands; import static accord.utils.Property.ignoreCommand; import static accord.utils.Property.multistep; import static accord.utils.Property.stateful; +import static org.apache.cassandra.harry.model.TokenPlacementModel.Range; +import static org.apache.cassandra.harry.model.TokenPlacementModel.Replica; +import static org.apache.cassandra.harry.model.TokenPlacementModel.ReplicatedRanges; +import static org.apache.cassandra.harry.model.TokenPlacementModel.ReplicationFactor; +import static org.apache.cassandra.harry.model.TokenPlacementModel.SimpleReplicationFactor; /** * These tests can create many instances, so mac users may need to run the following to avoid address bind failures * <p> * {@code for id in $(seq 0 15); do sudo ifconfig lo0 alias "127.0.0.$id"; done;} */ -public abstract class TopologyMixupTestBase<S extends TopologyMixupTestBase.SchemaSpec> extends TestBaseImpl +public abstract class TopologyMixupTestBase<S extends TopologyMixupTestBase.Schema> extends TestBaseImpl { private static final Logger logger = LoggerFactory.getLogger(TopologyMixupTestBase.class); @@ -98,9 +117,9 @@ private enum TopologyChange AddNode, RemoveNode, HostReplace, + StopNode, + StartNode, //TODO (coverage): add the following states once supported -// StopNode, -// StartNode, // MoveToken //TODO (coverage): node migrate to another rack or dc (unsupported on trunk as of this writing, but planned work for TCM) // MoveNodeToNewRack, @@ -114,48 +133,130 @@ private enum RemoveType private static final int TARGET_RF = 3; private static final Gen<Gen<RemoveType>> REMOVE_TYPE_DISTRIBUTION = Gens.enums().allMixedDistribution(RemoveType.class); private static final Gen<Map<String, Object>> CONF_GEN = new ConfigGenBuilder() - // jvm-dtest hard codes this partitioner in its APIs, so overriding will break the test - .withPartitionerGen(null) - .build(); + // jvm-dtest hard codes this partitioner in its APIs, so overriding will break the test + .withPartitionerGen(null) + .build(); // common commands private Command<State<S>, Void, ?> repairCommand(int toCoordinate) { - return new SimpleCommand<>(state -> "nodetool repair " + state.schemaSpec.keyspaceName() + ' ' + state.schemaSpec.name() + " from node" + toCoordinate + state.commandNamePostfix(), - state -> state.cluster.get(toCoordinate).nodetoolResult("repair", state.schemaSpec.keyspaceName(), state.schemaSpec.name()).asserts().success()); + return new SimpleCommand<>(state -> "nodetool repair " + state.schema.keyspace() + ' ' + state.schema.table() + " from node" + toCoordinate + state.commandNamePostfix(), + state -> state.cluster.get(toCoordinate).nodetoolResult("repair", state.schema.keyspace(), state.schema.table(), "--force").asserts().success()); + } + + private static <S extends Schema> Command<State<S>, Void, ?> repairCommand(int toCoordinate, String ks, String... tables) { + return new SimpleCommand<>(state -> "nodetool repair " + ks + (tables.length == 0 ? "" : " " + Arrays.asList(tables)) + " from node" + toCoordinate + state.commandNamePostfix(), + state -> { + if (tables.length == 0) { + state.cluster.get(toCoordinate).nodetoolResult("repair", ks, "--force").asserts().success(); + return; + } + List<String> args = new ArrayList<>(3 + tables.length); + args.add("repair"); + args.add(ks); + args.addAll(Arrays.asList(tables)); + args.add("--force"); + state.cluster.get(toCoordinate).nodetoolResult(args.toArray(String[]::new)).asserts().success(); + }); } private Command<State<S>, Void, ?> waitForCMSToQuiesce() { - return new SimpleCommand<>(state -> "Waiting for CMS to Quiesce" + state.commandNamePostfix(), - state -> ClusterUtils.waitForCMSToQuiesce(state.cluster, state.cmsGroup)); + return new Property.StateOnlyCommand<>() + { + private Epoch maxEpoch = null; + @Override + public String detailed(State<S> state) + { + if (maxEpoch == null) + maxEpoch = ClusterUtils.maxEpoch(state.cluster, state.topologyHistory.up()); + return "Waiting for CMS to Quiesce on epoch " + maxEpoch.getEpoch() + state.commandNamePostfix(); + } + + @Override + public void applyUnit(State<S> state) + { + Invariants.nonNull(maxEpoch, "detailed was not called before calling apply"); + ClusterUtils.waitForCMSToQuiesce(state.cluster, maxEpoch, true); + } + }; } - private Command<State<S>, Void, ?> stopInstance(int toRemove) + private Command<State<S>, Void, ?> waitForGossipToSettle() { - return new SimpleCommand<>(state -> "Stop Node" + toRemove + " for Assassinate" + state.commandNamePostfix(), - state -> { - IInvokableInstance inst = state.cluster.get(toRemove); - TopologyHistory.Node node = state.topologyHistory.node(toRemove); - ClusterUtils.stopUnchecked(inst); - node.down(); - }); + return new SimpleCommand<>(state -> "Waiting for Ring to Settle" + state.commandNamePostfix(), + state -> { + int[] up = state.topologyHistory.up(); + for (int node : up) + { + IInvokableInstance instance = state.cluster.get(node); + ClusterUtils.awaitRingJoin(state.cluster, up, instance); + } + }); + } + + private Command<State<S>, Void, ?> waitAllNodesInPeers() + { + return new SimpleCommand<>(state -> "Waiting for all alive nodes to be in peers" + state.commandNamePostfix(), + state -> { + int[] up = state.topologyHistory.up(); + for (int node : up) + { + IInvokableInstance instance = state.cluster.get(node); + ClusterUtils.awaitInPeers(state.cluster, up, instance); + } + }); + } + + private Command<State<S>, Void, ?> stopInstance(RandomSource rs, State<S> state) + { + int toStop = rs.pickInt(state.upAndSafe()); + return stopInstance(toStop, "Normal Stop"); + } + + private Command<State<S>, Void, ?> startInstance(RandomSource rs, State<S> state) + { + int toStop = rs.pickInt(state.topologyHistory.down()); + return startInstance(toStop); + } + + private Command<State<S>, Void, ?> startInstance(int toStart) + { + return new SimpleCommand<>(state -> "Start Node" + toStart + state.commandNamePostfix(), + state -> { + IInvokableInstance inst = state.cluster.get(toStart); + TopologyHistory.Node node = state.topologyHistory.node(toStart); + inst.startup(); + node.up(); + }); + } + + private Command<State<S>, Void, ?> stopInstance(int toRemove, String why) + { + return new SimpleCommand<>(state -> "Stop Node" + toRemove + " for " + why + state.commandNamePostfix(), + state -> { + IInvokableInstance inst = state.cluster.get(toRemove); + TopologyHistory.Node node = state.topologyHistory.node(toRemove); + ClusterUtils.stopUnchecked(inst); + node.down(); + }); } private Command<State<S>, Void, ?> addNode() { return new SimpleCommand<>(state -> "Add Node" + (state.topologyHistory.uniqueInstances + 1) + state.commandNamePostfix(), - state -> { - TopologyHistory.Node n = state.topologyHistory.addNode(); - IInvokableInstance newInstance = ClusterUtils.addInstance(state.cluster, n.dc, n.rack, c -> c.set("auto_bootstrap", true)); - newInstance.startup(state.cluster); - n.up(); - }); + state -> { + TopologyHistory.Node n = state.topologyHistory.addNode(); + IInvokableInstance newInstance = ClusterUtils.addInstance(state.cluster, n.dc, n.rack, c -> c.set("auto_bootstrap", true)); + newInstance.startup(state.cluster); + ClusterUtils.assertModeJoined(newInstance); + n.up(); + }); } private Command<State<S>, Void, ?> removeNodeDecommission(RandomSource rs, State<S> state) { - int toRemove = rs.pickInt(state.topologyHistory.up()); + int toRemove = rs.pickInt(state.upAndSafe()); return new SimpleCommand<>("nodetool decommission node" + toRemove + state.commandNamePostfix(), s2 -> { IInvokableInstance inst = s2.cluster.get(toRemove); TopologyHistory.Node node = s2.topologyHistory.node(toRemove); @@ -169,7 +270,7 @@ private enum RemoveType private Command<State<S>, Void, ?> removeNode(RandomSource rs, State<S> state) { int[] up = state.topologyHistory.up(); - int toRemove = rs.pickInt(up); + int toRemove = rs.pickInt(state.upAndSafe()); int toCoordinate; { int picked; @@ -180,28 +281,21 @@ private enum RemoveType while (picked == toRemove); toCoordinate = picked; } - return multistep(stopInstance(toRemove), - new SimpleCommand<>("nodetool removenode node" + toRemove + " from node" + toCoordinate + state.commandNamePostfix(), s2 -> { - TopologyHistory.Node node = s2.topologyHistory.node(toRemove); - node.status = TopologyHistory.Node.Status.BeingRemoved; - IInvokableInstance coordinator = s2.cluster.get(toCoordinate); - coordinator.nodetoolResult("removenode", Integer.toString(toRemove), "--force").asserts().success(); - node.removed(); - s2.currentEpoch.set(HackSerialization.tcmEpoch(coordinator)); - }), - repairCommand(toCoordinate)); + return multistep(stopInstance(toRemove, "nodetool removenode"), + new SimpleCommand<>("nodetool removenode node" + toRemove + " from node" + toCoordinate + state.commandNamePostfix(), s2 -> { + TopologyHistory.Node node = s2.topologyHistory.node(toRemove); + node.status = TopologyHistory.Node.Status.BeingRemoved; + IInvokableInstance coordinator = s2.cluster.get(toCoordinate); + coordinator.nodetoolResult("removenode", Integer.toString(toRemove), "--force").asserts().success(); + node.removed(); + s2.currentEpoch.set(HackSerialization.tcmEpoch(coordinator)); + }), + repairCommand(toCoordinate)); } private Command<State<S>, Void, ?> removeNodeAssassinate(RandomSource rs, State<S> state) { - //TODO (correctness): assassinate CMS member isn't allowed - IntHashSet up = asSet(state.topologyHistory.up()); - IntHashSet cmsGroup = asSet(state.cmsGroup); - Sets.SetView<Integer> upAndNotInCMS = Sets.difference(up, cmsGroup); - if (upAndNotInCMS.isEmpty()) throw new AssertionError("Every node is a CMS member"); - List<Integer> allowed = new ArrayList<>(upAndNotInCMS); - allowed.sort(Comparator.naturalOrder()); - int toRemove = rs.pick(allowed); + int toRemove = rs.pickInt(state.upAndSafe()); int toCoordinate; { int[] upInt = state.topologyHistory.up(); @@ -213,17 +307,17 @@ private enum RemoveType while (picked == toRemove); toCoordinate = picked; } - return multistep(stopInstance(toRemove), - new SimpleCommand<>("nodetool assassinate node" + toRemove + " from node" + toCoordinate + state.commandNamePostfix(), s2 -> { - TopologyHistory.Node node = s2.topologyHistory.node(toRemove); - node.status = TopologyHistory.Node.Status.BeingAssassinated; - IInvokableInstance coordinator = s2.cluster.get(toCoordinate); - InetSocketAddress address = s2.cluster.get(toRemove).config().broadcastAddress(); - coordinator.nodetoolResult("assassinate", address.getAddress().getHostAddress() + ":" + address.getPort()).asserts().success(); - node.removed(); - s2.currentEpoch.set(HackSerialization.tcmEpoch(coordinator)); - }), - repairCommand(toCoordinate) + return multistep(stopInstance(toRemove, "nodetool assassinate"), + new SimpleCommand<>("nodetool assassinate node" + toRemove + " from node" + toCoordinate + state.commandNamePostfix(), s2 -> { + TopologyHistory.Node node = s2.topologyHistory.node(toRemove); + node.status = TopologyHistory.Node.Status.BeingAssassinated; + IInvokableInstance coordinator = s2.cluster.get(toCoordinate); + InetSocketAddress address = s2.cluster.get(toRemove).config().broadcastAddress(); + coordinator.nodetoolResult("assassinate", address.getAddress().getHostAddress() + ":" + address.getPort()).asserts().success(); + node.removed(); + s2.currentEpoch.set(HackSerialization.tcmEpoch(coordinator)); + }), + repairCommand(toCoordinate) ); } @@ -245,26 +339,24 @@ private enum RemoveType private Command<State<S>, Void, ?> hostReplace(RandomSource rs, State<S> state) { - int nodeToReplace = rs.pickInt(state.topologyHistory.up()); + int nodeToReplace = rs.pickInt(state.upAndSafe()); IInvokableInstance toReplace = state.cluster.get(nodeToReplace); TopologyHistory.Node adding = state.topologyHistory.replace(nodeToReplace); TopologyHistory.Node removing = state.topologyHistory.nodes.get(nodeToReplace); - return multistep(new SimpleCommand<>("Stop Node" + nodeToReplace + " for HostReplace; Node" + adding.id + state.commandNamePostfix(), s2 -> { - ClusterUtils.stopUnchecked(toReplace); - removing.down(); - }), - new SimpleCommand<>("Host Replace Node" + nodeToReplace + "; Node" + adding.id + state.commandNamePostfix(), s2 -> { - logger.info("node{} starting host replacement; epoch={}", adding.id, HackSerialization.tcmEpochAndSync(s2.cluster.getFirstRunningInstance())); - removing.status = TopologyHistory.Node.Status.BeingReplaced; - IInvokableInstance inst = ClusterUtils.replaceHostAndStart(s2.cluster, toReplace); - s2.topologyHistory.replaced(removing, adding); - long epoch = HackSerialization.tcmEpoch(inst); - s2.currentEpoch.set(epoch); - logger.info("{} completed host replacement in epoch={}", inst, epoch); - }), - //TODO (remove after rebase to trunk): https://issues.apache.org/jira/browse/CASSANDRA-19705 After the rebase to trunk this is not needed. The issue is that the CMS placement removes the node, it does not promote another node, this cases rf=3 to become rf=2 - new SimpleCommand<>("CMS reconfigure on Node" + adding.id + state.commandNamePostfix(), s2 -> s2.cluster.get(adding.id).nodetoolResult("cms", "reconfigure", Integer.toString(TARGET_RF)).asserts().success()) + return multistep(stopInstance(nodeToReplace, "HostReplace; Node" + adding.id), + new SimpleCommand<>("Host Replace Node" + nodeToReplace + "; Node" + adding.id + state.commandNamePostfix(), s2 -> { + logger.info("node{} starting host replacement; epoch={}", adding.id, HackSerialization.tcmEpochAndSync(s2.cluster.getFirstRunningInstance())); + removing.status = TopologyHistory.Node.Status.BeingReplaced; + IInvokableInstance inst = ClusterUtils.replaceHostAndStart(s2.cluster, toReplace); + ClusterUtils.assertModeJoined(inst); + s2.topologyHistory.replaced(removing, adding); + long epoch = HackSerialization.tcmEpoch(inst); + s2.currentEpoch.set(epoch); + logger.info("{} completed host replacement in epoch={}", inst, epoch); + }), + //TODO (remove after rebase to trunk): https://issues.apache.org/jira/browse/CASSANDRA-19705 After the rebase to trunk this is not needed. The issue is that the CMS placement removes the node, it does not promote another node, this cases rf=3 to become rf=2 + new SimpleCommand<>("CMS reconfigure on Node" + adding.id + state.commandNamePostfix(), s2 -> s2.cluster.get(adding.id).nodetoolResult("cms", "reconfigure", Integer.toString(TARGET_RF)).asserts().success()) ); } @@ -283,24 +375,30 @@ protected void destroyState(State<S> state, @Nullable Throwable cause) throws Th @Test public void test() { - Property.StatefulBuilder statefulBuilder = stateful().withSteps(20).withStepTimeout(Duration.ofMinutes(2)).withExamples(1); + Property.StatefulBuilder statefulBuilder = stateful().withSteps(20).withStepTimeout(Duration.ofMinutes(3)).withExamples(1); preCheck(statefulBuilder); statefulBuilder.check(commands(this::stateGen) - .preCommands(state -> state.preActions.forEach(Runnable::run)) - .add(2, (rs, state) -> { - EnumSet<TopologyChange> possibleTopologyChanges = possibleTopologyChanges(state); - if (possibleTopologyChanges.isEmpty()) return ignoreCommand(); - return topologyCommand(state, possibleTopologyChanges).next(rs); - }) - .add(1, (rs, state) -> repairCommand(rs.pickInt(state.topologyHistory.up()))) - .add(7, (rs, state) -> state.statementGen.apply(rs, state)) - .destroyState((state, cause) -> { - try (state) - { - TopologyMixupTestBase.this.destroyState(state, cause); - } - }) - .build()); + .preCommands(state -> state.preActions.forEach(Runnable::run)) + .add(2, (rs, state) -> { + EnumSet<TopologyChange> possibleTopologyChanges = possibleTopologyChanges(state); + if (possibleTopologyChanges.isEmpty()) return ignoreCommand(); + return topologyCommand(state, possibleTopologyChanges).next(rs); + }) + .add(1, (rs, state) -> repairCommand(rs.pickInt(state.topologyHistory.up()))) + .add(7, (rs, state) -> state.statementGen.apply(rs, state)) + .destroyState((state, cause) -> { + try (state) + { + TopologyMixupTestBase.this.destroyState(state, cause); + } + }) + .commandsTransformer((state, gen) -> { + for (BiFunction<State<S>, Gen<Command<State<S>, Void, ?>>, Gen<Command<State<S>, Void, ?>>> fn : state.commandsTransformers) + gen = fn.apply(state, gen); + return gen; + }) + .onSuccess((state, sut, history) -> logger.info("Successful for the following:\nState {}\nHistory:\n{}", state, Property.formatList("\t\t", history))) + .build()); } private EnumSet<TopologyChange> possibleTopologyChanges(State<S> state) @@ -308,18 +406,32 @@ private EnumSet<TopologyChange> possibleTopologyChanges(State<S> state) EnumSet<TopologyChange> possibleTopologyChanges = EnumSet.noneOf(TopologyChange.class); // up or down is logically more correct, but since this runs sequentially and after the topology changes are complete, we don't have downed nodes at this point // so up is enough to know the topology size - int size = state.topologyHistory.up().length; - if (size < state.topologyHistory.maxNodes) + int up = state.topologyHistory.up().length; + int down = state.topologyHistory.down().length; + int[] upAndSafe = state.upAndSafe(); + int total = up + down; + if (total < state.topologyHistory.maxNodes) possibleTopologyChanges.add(TopologyChange.AddNode); - if (size > state.topologyHistory.quorum()) + if (upAndSafe.length > 0) { - if (size > TARGET_RF) + // can't remove the node if all nodes are CMS nodes + if (!Sets.difference(asSet(upAndSafe), asSet(state.cmsGroup)).isEmpty()) possibleTopologyChanges.add(TopologyChange.RemoveNode); possibleTopologyChanges.add(TopologyChange.HostReplace); + possibleTopologyChanges.add(TopologyChange.StopNode); } + if (down > 0) + possibleTopologyChanges.add(TopologyChange.StartNode); return possibleTopologyChanges; } + private Command<State<S>, Void, ?> awaitClusterStable() + { + return multistep(waitForCMSToQuiesce(), + waitForGossipToSettle(), + waitAllNodesInPeers()); + } + private Gen<Command<State<S>, Void, ?>> topologyCommand(State<S> state, EnumSet<TopologyChange> possibleTopologyChanges) { Map<Gen<Command<State<S>, Void, ?>>, Integer> possible = new LinkedHashMap<>(); @@ -328,13 +440,19 @@ private EnumSet<TopologyChange> possibleTopologyChanges(State<S> state) switch (task) { case AddNode: - possible.put(ignore -> multistep(addNode(), waitForCMSToQuiesce()), 1); + possible.put(ignore -> multistep(addNode(), awaitClusterStable()), 1); break; case RemoveNode: - possible.put(rs -> multistep(removeNodeRandomizedDispatch(rs, state), waitForCMSToQuiesce()), 1); + possible.put(rs -> multistep(removeNodeRandomizedDispatch(rs, state), awaitClusterStable()), 1); break; case HostReplace: - possible.put(rs -> multistep(hostReplace(rs, state), waitForCMSToQuiesce()), 1); + possible.put(rs -> multistep(hostReplace(rs, state), awaitClusterStable()), 1); + break; + case StartNode: + possible.put(rs -> startInstance(rs, state), 1); + break; + case StopNode: + possible.put(rs -> stopInstance(rs, state), 1); break; default: throw new UnsupportedOperationException(task.name()); @@ -351,74 +469,142 @@ private static IntHashSet asSet(int[] array) return set; } - public interface SchemaSpec + public interface Schema { - String name(); + String table(); + String keyspace(); + String createSchema(); + } - String keyspaceName(); + protected interface CommandGen<S extends Schema> + { + Command<State<S>, Void, ?> apply(RandomSource rs, State<S> state); } - protected static class State<S extends SchemaSpec> implements AutoCloseable + private static class LoggingCommand<State, SystemUnderTest, Result> extends Property.ForwardingCommand<State, SystemUnderTest, Result> + { + private static final Logger logger = LoggerFactory.getLogger(LoggingCommand.class); + + private LoggingCommand(Command<State, SystemUnderTest, Result> delegate) + { + super(delegate); + } + + @Override + public Result apply(State s) throws Throwable + { + String name = detailed(s); + long startNanos = Clock.Global.nanoTime(); + try + { + logger.info("Starting command: {}", name); + Result o = super.apply(s); + logger.info("Command {} was success after {}", name, Duration.ofNanos(Clock.Global.nanoTime() - startNanos)); + return o; + } + catch (Throwable t) + { + logger.warn("Command {} failed after {}: {}", name, Duration.ofNanos(Clock.Global.nanoTime() - startNanos), t.toString()); // don't want stack trace, just type/msg + throw t; + } + } + } + + protected static class State<S extends Schema> implements AutoCloseable { final TopologyHistory topologyHistory; final Cluster cluster; - final S schemaSpec; + final S schema; + final List<BiFunction<State<S>, Gen<Command<State<S>, Void, ?>>, Gen<Command<State<S>, Void, ?>>>> commandsTransformers = new ArrayList<>(); final List<Runnable> preActions = new CopyOnWriteArrayList<>(); final AtomicLong currentEpoch = new AtomicLong(); - final BiFunction<RandomSource, State<S>, Command<State<S>, Void, ?>> statementGen; + final CommandGen<S> statementGen; final Gen<RemoveType> removeTypeGen; private final Map<String, Object> yamlConfigOverrides; int[] cmsGroup = new int[0]; + private ReplicationFactor rf; + private final RingModel ring = new RingModel(); - public State(RandomSource rs, BiFunction<RandomSource, Cluster, S> schemaSpecGen, Function<S, BiFunction<RandomSource, State<S>, Command<State<S>, Void, ?>>> cqlOperationsGen) + public State(RandomSource rs, BiFunction<RandomSource, Cluster, S> schemaSpecGen, Function<S, CommandGen<S>> cqlOperationsGen) { this.topologyHistory = new TopologyHistory(rs.fork(), 2, 4); + rf = new SimpleReplicationFactor(2); try { this.yamlConfigOverrides = CONF_GEN.next(rs); cluster = Cluster.build(topologyHistory.minNodes) - .withTokenSupplier(topologyHistory) - .withConfig(c -> { - c.with(Feature.values()) - .set("write_request_timeout", "10s"); - //TODO (maintenance): where to put this? Anything touching ConfigGenBuilder with jvm-dtest needs this... - ((InstanceConfig) c).remove("commitlog_sync_period_in_ms"); - for (Map.Entry<String, Object> e : yamlConfigOverrides.entrySet()) - c.set(e.getKey(), e.getValue()); - onConfigure(c); - }) - //TODO (maintenance): should TopologyHistory also be a INodeProvisionStrategy.Factory so address information is stored in the Node? - //TODO (maintenance): AbstractCluster's Map<Integer, NetworkTopology.DcAndRack> nodeIdTopology makes playing with dc/rack annoying, if this becomes an interface then TopologyHistory could own - .withNodeProvisionStrategy((subnet, portMap) -> new INodeProvisionStrategy.AbstractNodeProvisionStrategy(portMap) - { - { - Invariants.checkArgument(subnet == 0, "Unexpected subnet detected: %d", subnet); - } - - private final String ipPrefix = "127.0." + subnet + '.'; - - @Override - public int seedNodeNum() - { - int[] up = topologyHistory.up(); - if (Arrays.equals(up, new int[]{ 1, 2 })) - return 1; - return rs.pickInt(up); - } - - @Override - public String ipAddress(int nodeNum) - { - return ipPrefix + nodeNum; - } - }) - .start(); + .withTokenSupplier(topologyHistory) + .withConfig(c -> { + c.with(Feature.values()) + .set("write_request_timeout", "10s") + .set("read_request_timeout", "10s") + .set("range_request_timeout", "20s") + .set("request_timeout", "20s") + .set("native_transport_timeout", "30s") + // bound startup to some value larger than the task timeout, this is to allow the + // tests to stop blocking when a startup issue is detected. The main reason for + // this is that startup blocks forever, waiting for accord and streaming to + // complete... but if there are bugs at these layers then the startup will never + // exit, blocking the JVM from giving the needed information (logs/seed) to debug. + .set(Constants.KEY_DTEST_STARTUP_TIMEOUT, "4m") + .set(Constants.KEY_DTEST_API_STARTUP_FAILURE_AS_SHUTDOWN, false); + //TODO (maintenance): where to put this? Anything touching ConfigGenBuilder with jvm-dtest needs this... + ((InstanceConfig) c).remove("commitlog_sync_period_in_ms"); + for (Map.Entry<String, Object> e : yamlConfigOverrides.entrySet()) + c.set(e.getKey(), e.getValue()); + onConfigure(c); + }) + //TODO (maintenance): should TopologyHistory also be a INodeProvisionStrategy.Factory so address information is stored in the Node? + //TODO (maintenance): AbstractCluster's Map<Integer, NetworkTopology.DcAndRack> nodeIdTopology makes playing with dc/rack annoying, if this becomes an interface then TopologyHistory could own + .withNodeProvisionStrategy((subnet, portMap) -> new INodeProvisionStrategy.AbstractNodeProvisionStrategy(portMap) + { + { + Invariants.checkArgument(subnet == 0, "Unexpected subnet detected: %d", subnet); + } + + private final String ipPrefix = "127.0." + subnet + '.'; + + @Override + public int seedNodeNum() + { + int[] up = topologyHistory.up(); + if (Arrays.equals(up, new int[]{ 1, 2 })) + return 1; + return rs.pickInt(up); + } + + @Override + public String ipAddress(int nodeNum) + { + return ipPrefix + nodeNum; + } + }) + .start(); } catch (IOException e) { throw new UncheckedIOException(e); } + cluster.setUncaughtExceptionsFilter((node, t) -> { + // api is "ignore" so false means include, + var rootCause = Throwables.getRootCause(t); + if (rootCause.getMessage() != null) + { + if (rootCause.getMessage().startsWith("Queried for epoch") && rootCause.getMessage().contains("but could not catch up. Current epoch:")) + return true; + if (rootCause.getMessage().startsWith("Operation timed out")) + { + // is this due to TCM fetching epochs? PaxosBackedProcessor.getLogState is costly and more likely to timeout... so ignore those + Optional<StackTraceElement> match = Stream.of(rootCause.getStackTrace()) + .filter(s -> s.getClassName().equals("org.apache.cassandra.tcm.PaxosBackedProcessor") && s.getMethodName().equals("getLogState")) + .findFirst(); + if (match.isPresent()) + return true; + } + } + return false; + }); fixDistributedSchemas(cluster); init(cluster, TARGET_RF); // fix TCM @@ -427,31 +613,71 @@ public String ipAddress(int nodeNum) result.asserts().success(); logger.info("CMS reconfigure: {}", result.getStdout()); } - preActions.add(new Runnable() - { + commandsTransformers.add(new BiFunction<State<S>, Gen<Command<State<S>, Void, ?>>, Gen<Command<State<S>, Void, ?>>>() { // in order to remove this action, an anonymous class is needed so "this" works, lambda "this" is the parent class @Override - public void run() - { - if (topologyHistory.up().length == TARGET_RF) - { + public Gen<Command<State<S>, Void, ?>> apply(State<S> state, Gen<Command<State<S>, Void, ?>> commandGen) { + if (topologyHistory.up().length < TARGET_RF) + return commandGen; + SimpleCommand<State<S>> reconfig = new SimpleCommand<>("nodetool cms reconfigure " + TARGET_RF, ignore -> { NodeToolResult result = cluster.get(1).nodetoolResult("cms", "reconfigure", Integer.toString(TARGET_RF)); result.asserts().success(); logger.info("CMS reconfigure: {}", result.getStdout()); - preActions.remove(this); - } + }); + SimpleCommand<State<S>> fixDistributedSchemas = new SimpleCommand<>("Set system distributed keyspaces to RF=" + TARGET_RF, ignore -> + fixDistributedSchemas(cluster)); + SimpleCommand<State<S>> fixTestKeyspace = new SimpleCommand<>("Set " + KEYSPACE + " keyspace to RF=" + TARGET_RF, s -> { + cluster.schemaChange("ALTER KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': " + TARGET_RF + "}"); + rf = new SimpleReplicationFactor(TARGET_RF); + }); + var self = this; + return rs -> { + Command<State<S>, Void, ?> next = commandGen.next(rs); + if (next.checkPreconditions(state) == Property.PreCheckResult.Ignore) + return next; + commandsTransformers.remove(self); + int[] up = state.topologyHistory.up(); + List<Command<State<S>, Void, ?>> commands = new ArrayList<>(); + commands.add(fixDistributedSchemas); + for (String ks : Arrays.asList("system_auth", "system_traces")) + { + int coordinator = rs.pickInt(up); + commands.add(repairCommand(coordinator, ks)); + } + commands.add(fixTestKeyspace); + { + int coordinator = rs.pickInt(up); + commands.add(repairCommand(coordinator, KEYSPACE)); + } + commands.add(reconfig); + commands.add(next); + return multistep(commands); + }; } }); + commandsTransformers.add((state, commandGen) -> rs2 -> { + Command<State<S>, Void, ?> c = commandGen.next(rs2); + if (!(c instanceof Property.MultistepCommand)) + return new LoggingCommand<>(c); + Property.MultistepCommand<State<S>, Void> multistep = (Property.MultistepCommand<State<S>, Void>) c; + List<Command<State<S>, Void, ?>> subcommands = new ArrayList<>(); + for (var sub : multistep) + subcommands.add(new LoggingCommand<>(sub)); + return multistep(subcommands); + }); preActions.add(() -> { int[] up = topologyHistory.up(); // use the most recent node just in case the cluster isn't in-sync IInvokableInstance node = cluster.get(up[up.length - 1]); cmsGroup = HackSerialization.cmsGroup(node); currentEpoch.set(HackSerialization.tcmEpoch(node)); + + ring.rebuild(cluster.coordinator(up[0]), rf, up); + // ring must know about the up nodes }); preActions.add(() -> cluster.checkAndResetUncaughtExceptions()); - this.schemaSpec = schemaSpecGen.apply(rs, cluster); - statementGen = cqlOperationsGen.apply(schemaSpec); + this.schema = schemaSpecGen.apply(rs, cluster); + statementGen = cqlOperationsGen.apply(schema); removeTypeGen = REMOVE_TYPE_DISTRIBUTION.next(rs); @@ -472,7 +698,38 @@ protected void onConfigure(IInstanceConfig config) protected String commandNamePostfix() { - return "; epoch=" + currentEpoch.get() + ", cms=" + Arrays.toString(cmsGroup); + return "; epoch=" + currentEpoch.get() + ", cms=" + Arrays.toString(cmsGroup) + ", up=" + Arrays.toString(topologyHistory.up()) + ", down=" + Arrays.toString(topologyHistory.down()); + } + + public int[] upAndSafe() + { + IntHashSet up = asSet(topologyHistory.up()); + int quorum = topologyHistory.quorum(); + // find what ranges are able to handle 1 node loss + Set<Range> safeRanges = new HashSet<>(); + ring.rangesToReplicas((range, replicas) -> { + IntHashSet alive = new IntHashSet(); + for (int peer : replicas) + { + if (up.contains(peer)) + alive.add(peer); + } + if (quorum < alive.size()) + safeRanges.add(range); + }); + + // filter nodes where 100% of their ranges are "safe" + IntArrayList safeNodes = new IntArrayList(); + for (int id : up) + { + List<Range> ranges = ring.ranges(id); + if (ranges.stream().allMatch(safeRanges::contains)) + safeNodes.add(id); + } + + int[] upAndSafe = safeNodes.toIntArray(); + Arrays.sort(upAndSafe); + return upAndSafe; } @Override @@ -480,14 +737,42 @@ public String toString() { StringBuilder sb = new StringBuilder(); sb.append("Yaml Config:\n").append(YamlConfigurationLoader.toYaml(this.yamlConfigOverrides)); + String cql = schema.createSchema(); + sb.append("\n-- Setup Schema\n").append(cql); sb.append("\nTopology:\n").append(topologyHistory); sb.append("\nCMS Voting Group: ").append(Arrays.toString(cmsGroup)); + if (epochHistory != null) + sb.append("\n").append(epochHistory); return sb.toString(); } + private String epochHistory = null; + @Override public void close() throws Exception { + var cmsNodesUp = Sets.intersection(asSet(cmsGroup), asSet(topologyHistory.up())); + int cmsNode = Iterables.getFirst(cmsNodesUp, null); + try + { + SimpleQueryResult qr = Retry.retryWithBackoffBlocking(5, () -> cluster.get(cmsNode).executeInternalWithResult("SELECT epoch, kind, transformation FROM system_views.cluster_metadata_log")); + TableBuilder builder = new TableBuilder(" | "); + builder.add(qr.names()); + while (qr.hasNext()) + { + Row next = qr.next(); + builder.add(Stream.of(next.toObjectArray()) + .map(Objects::toString) + .map(s -> s.length() > 100 ? s.substring(0, 100) + "..." : s) + .collect(Collectors.toList())); + } + epochHistory = "Epochs:\n" + builder; + } + catch (Throwable t) + { + logger.warn("Unable to fetch epoch history on node{}", cmsNode, t); + } + logger.info("Shutting down clusters"); cluster.close(); } } @@ -550,11 +835,21 @@ public Collection<String> tokens(int i) } public int[] up() + { + return nodes(Node.Status.Up); + } + + public int[] down() + { + return nodes(Node.Status.Down); + } + + private int[] nodes(Node.Status target) { IntArrayList up = new IntArrayList(nodes.size(), -1); for (Map.Entry<Integer, Node> n : nodes.entrySet()) { - if (n.getValue().status == Node.Status.Up) + if (n.getValue().status == target) up.add(n.getKey()); } int[] ints = up.toIntArray(); @@ -571,12 +866,12 @@ public Node addNode() { int id = ++uniqueInstances; List<String> instTokens = Gens.lists(MURMUR_TOKEN_GEN - .filterAsInt(t -> !activeTokens.contains(Integer.toString(t)))) - .unique() - .ofSize(tokensPerNode) - .next(rs).stream() - .map(Object::toString) - .collect(Collectors.toList()); + .filterAsInt(t -> !activeTokens.contains(Integer.toString(t)))) + .unique() + .ofSize(tokensPerNode) + .next(rs).stream() + .map(Object::toString) + .collect(Collectors.toList()); activeTokens.addAll(instTokens); Node node = new Node(this, id, instTokens, "datacenter0", "rack0"); node.status = Node.Status.Down; @@ -652,10 +947,10 @@ public void removed() public String toString() { return "Node{" + - "status=" + status + - (replacing == null ? "" : (", replacing=" + replacing)) + - ", tokens=" + tokens + - '}'; + "status=" + status + + (replacing == null ? "" : (", replacing=" + replacing)) + + ", tokens=" + tokens + + '}'; } } } @@ -694,4 +989,63 @@ private static int addressToNodeId(InetAddressAndPort addressAndPort) return Integer.parseInt(parts[3]); } } + + private static class RingModel + { + ReplicatedRanges ring = null; + Int2ObjectHashMap<Replica> idToReplica = null; + + private void rebuild(ICoordinator coordinator, ReplicationFactor rf, int[] up) + { + ring = TokenPlacementModelHelper.getRing(coordinator, rf); + + Int2ObjectHashMap<Replica> idToReplica = new Int2ObjectHashMap<>(); + for (Map.Entry<Range, List<Replica>> e : ring.asMap().entrySet()) + { + for (var replica : e.getValue()) + idToReplica.put(toNodeId(replica), replica); + } + this.idToReplica = idToReplica; + + IntHashSet upSet = asSet(up); + if (!idToReplica.keySet().containsAll(upSet)) + { + int coordinatorNode = coordinator.instance().config().num(); + Sets.SetView<Integer> diff = Sets.difference(upSet, idToReplica.keySet()); + throw new AssertionError("Unable to find nodes " + diff + " in the ring on node" + coordinatorNode); + } + } + + private static int toNodeId(Replica replica) + { + //TODO (fix test api): NodeId is in the API but is always null. Cheapest way to get the id is to assume the address has it + // same issue with address... + // /127.0.0.2 + String harryId = replica.node().id(); + int index = harryId.lastIndexOf('.'); + int peer = Integer.parseInt(harryId.substring(index + 1)); + return peer; + } + + List<Range> ranges(int node) + { + Replica replica = idToReplica.get(node); + if (replica == null) + throw new AssertionError("Unknown node" + node); + List<Range> ranges = ring.ranges(replica); + if (ranges == null) + throw new AssertionError("node" + node + " some how does not have ranges..."); + return ranges; + } + + private void rangesToReplicas(BiConsumer<Range, int[]> fn) + { + for (Map.Entry<Range, List<Replica>> e : ring.asMap().entrySet()) + { + int[] replicas = e.getValue().stream().mapToInt(RingModel::toNodeId).toArray(); + Arrays.sort(replicas); + fn.accept(e.getKey(), replicas); + } + } + } } diff --git a/test/harry/main/README.md b/test/harry/main/README.md deleted file mode 100644 index a2f2cfacb95f..000000000000 --- a/test/harry/main/README.md +++ /dev/null @@ -1,647 +0,0 @@ -# Harry, a fuzz testing tool for Apache Cassandra - -The project aims to generate _reproducible_ workloads that are as close to real-life as possible, while being able to -_efficiently_ verify the cluster state against the model without pausing the workload itself. - -## Getting Started in under 5 minutes - -Harry can operate as a straightforward read/write "correctness stress tool" that will check to ensure reads on a cluster -are consistent with what it knows it wrote to the cluster. You have a couple options for this. - -### Option 2: Running things manually lower in the stack: - -To start a workload that performs a concurrent read/write workload, 2 read and 2 write threads for 60 seconds -against a in-jvm cluster you can use the following code: - -``` -try (Cluster cluster = builder().withNodes(3) - .start()) -{ - SchemaSpec schema = new SchemaSpec("harry", "test_table", - asList(pk("pk1", asciiType), pk("pk1", int64Type)), - asList(ck("ck1", asciiType), ck("ck1", int64Type)), - asList(regularColumn("regular1", asciiType), regularColumn("regular1", int64Type)), - asList(staticColumn("static1", asciiType), staticColumn("static1", int64Type))); - - Configuration config = HarryHelper.defaultConfiguration() - .setKeyspaceDdl(String.format("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': %d};", schema.keyspace, 3)) - .setSUT(() -> new InJvmSut(cluster)) - .build(); - - Run run = config.createRun(); - - concurrent(run, config, - asList(pool("Writer", 2, MutatingVisitor::new), - pool("Reader", 2, RandomPartitionValidator::new)), - 2, TimeUnit.MINUTES) - .run(); -} -``` - -# I've found a falsification. What now? - -There is no one-size-fits-all solution for debugging a falsification. We did try to create a shrinker, but unfortunately -without Simulator, shrinker only works for issues that are non-concurrent in nature, since there's no way to create a -stable repro otherwise. That said, there are several things that might get you started and inspire further ideas about -how to debug the issue. - -First of all, understand whether or not the issue is likely to be concurrent in nature. If you re-run your test with the -same seed, but see no falsification, and it fails only sporadically, and often on different logical timestamp, it is -likely that the issue is, in fact concurrent. Here, it is important to note that when you are running concurrent -read/write workload, you will get different interleaving of reads and writes every time you do this. - -If you can get a stable repro with a sequential runner, you're in luck. Now all you need to do is to add logs everywhere -and understand what might be causing it. But even if you do not have a stable repro, you are still likely to follow the -same steps: - -* Inspect the error itself. Do Cassandra-returned results make sense? Is anything out of order? Are there any duplicates - or gaps? -* Switch to logging mutating visitor and closely inspect its output. Closely inspect the output of the model. Do the - values make sense? -* Check the output of data tracker. Does the model or Cassandra have missing columns or rows? Do these outputs contain - latest logical timestamps for each of the operations from the log? How about in-flight operations? -* Filter out relevant operation log entries and inspect them closely. Given these operations, does the output of the - model, or output of the database make most sense? - -Next, you might want to try to narrow down the scope of the problem. Depending on what the falsification looks like, use -your Cassandra knowledge to see what might apply in your situation: - -* Try checking if changing schema to use different column types does anything. -* Try disabling range deletes, regular deletes, or column deletes. -* Try changing the size of partition and see if the issue still reproduces. -* Try disabling static columns. - -To avoid listing every feature in Harry, it suffices to say you should try to enable/disable features that make sense -in the given context, and try to find the combination that avoids the failure, or a minimal combination that still -reproduces the issue. Your first goal should be to find a _stable repro_, even if it involves modifying Cassandra or -Harry, or taking the operations, and composing the repro manually. Having a stable repro will make finding a cause much -simpler. Sometimes you will find the cause before you have a stable repro, in which case, you _still_ have to produce a -stable repro to make things simpler for the reviewer, and to include it into the test suite of your patch. - -Lastly, *be patient*. Debugging falsifications is often a multi-hour endeavour, and things do not always jump out at you, -so you might have to spend a significant amount of time tracking the problem down. Once you have found it, it is very -rewarding. - -## Further Reading -* [Harry: An open-source fuzz testing and verification tool for Apache Cassandra](https://cassandra.apache.org/_/blog/Harry-an-Open-Source-Fuzz-Testing-and-Verification-Tool-for-Apache-Cassandra.html) - ---- -# Technical and Implementation Details - -## System Under Test implementations - -* `in_jvm/InJvmSut` - simple in-JVM-dtest system under test. -* `println/PrintlnSut` - system under test that prints to sdtout instead of executing queries on the cluster; useful for - debugging. -* `mixed_in_jvm/MixedVersionInJvmSut` - in-JVM-dtest system under test that works with mixed version clusters. -* `external/ExternalClusterSut` - system under test that works with CCM, Docker, Kubernetes, or cluster you may. have - deployed elsewhere - -Both in-JVM SUTs have fault-injecting functionality available. - -## Visitors - -* `single/SingleValidator` - visitor that runs several different read queries against a single partition that is - associated with current logical timestamp, and validates their results using given model. -* `all_partitions/AllPartitionsValidator` - concurrently validates all partitions that were visited during this run. -* `repair_and_validate_local_states/RepairingLocalStateValidator` - similar to `AllPartitionsValidator`, but performs - repair before checking node states. -* `mutating/MutatingVisitor` - visitor that performs all sorts of mutations. -* `logging/LoggingVisitor` - similar to `MutatingVisitor`, but also logs all operations to a file; useful for debug - purposes. -* `corrupting/CorruptingVisitor` - visitor that will deliberately change data in the partition it visits. Useful for - negative tests (i.e. to ensure that your model actually detects data inconsistencies). - -And more. - -## Models - -* `querying_no_op/QueryingNoOpValidator` - a model that can be used to "simply" run random queries. -* `quiescent_checker/QuiescentChecker` - a model that can be used to verify results of any read that has no writes to - the same partition_ concurrent to it. Should be used in conjunction with locking data tracker. -* `quiescent_local_state_checker/QuiescentLocalStateChecker` - a model that can check local states of each replica that - has to own - -## Runners - -* `sequential/SequentialRunner` - runs all visitors sequentially, in the loop, for a specified amount of time; useful - for simple tests that do not have to exercise concurrent read/write path. -* `concurrent/ConcurrentRunner` - runs all visitors concurrently, each visitor in its own thread, looped, for a - specified amount of time; useful for things like concurrent read/write workloads. -* `chain/ChainRunner` - receives other runners as input, and runs them one after another once. Useful for both simple - and complex scenarios that involve both read/write workloads, validating all partitions, exercising other node-local - or cluster-wide operations. -* `staged/StagedRunner` - receives other runners (stages) as input, and runs them one after another in a loop; useful - for implementing complex scenarios, such as read/write workloads followed by some cluster changing operations. - -## Clock - -* `approximate_monotonic/ApproximateMonotonicClock` - a timestamp supplier implementation that tries to keep as close to - real time as possible, while preserving mapping from real-time to logical timestamps. -* `offset/OffsetClock` - a (monotonic) clock that supplies timestamps that do not have any relation to real time. - -# Introduction - -Harry has two primary modes of functionality: - -* Unit test mode: in which you define specific sequences of - operations and let Harry test these operations using different - schemas and conditions. -* Exploratory/fuzz mode: in which you define distributions of events - rather rather than sequences themselves, and let Harry try out - different things. - -Usually, in unit-test mode, we’re applying several write operations to the cluster state and then run different read -queries and validate their results. To learn more about writing unit tests, refer to the "Writing Unit Tests" section. - -In exploratory mode, we continuously apply write operations to the cluster and validate their state, allowing data size -to grow and simulating real-life behaviour. To learn more about implementing test cases using fuzz mode, refer to the " -Implementing Tests" section of this guide, but it's likely you'll have to read the rest of this document to implement -more complex scenarios. - -# Writing Unit Tests - -To write unit tests with Harry, there's no special knowledge required. Usually, unit tests are written by simply -hardcoding the schema and then writing several modification statements one after the other, and then manually validating -results of a `SELECT` query. This might work for simple scenarios, but there’s still a chance that for some other schema -or some combination of values the tested feature may not work. - -To improve the situation, we can express the test in more abstract terms and, instead of writing it using specific -statements, we can describe which statement _types_ are to be used: - -``` -test(new SchemaGenerators.Builder("harry") - .partitionKeySpec(1, 5) - .clusteringKeySpec(1, 5) - .regularColumnSpec(1, 10) - .generator(), - historyBuilder -> { - historyBuilder.insert(); - historyBuilder.deletePartition(); - historyBuilder.deleteRowSlice(); - }); -``` - -This spec can be used to generate clusters of different sizes, configured with different schemas, executing the given a -sequence of actions both in isolation, and combined with other randomly generated ones, with failure-injection. - -Best of all is that this test will _not only_ ensure that such a sequence of actions does not produce an exception, but -also would ensure that cluster will respond with correct results to _any_ allowed read query. - -To begin specifying operations for a new partition, either start calling methods on the `HistoryBuilder`, or, if you -would like to specify the partition which Harry needs to visit use `#visitPartition` or `#beginBatch` have to be called, -for starting a visit to a particular partition with a single or multiple actions. - -After that, the actions are self-explanatory: `#insert`, `#update`, `#deleteRow`, `#deleteColumns`, `#deleteRowRange`, `#deleteRowSlice` -`#deletePartition`. - -After history generated by `HistoryBuilder` is replayed using `ReplayingVisitor` (or by using a `ReplayingHistoryBuilder` -which combines the two for your convenience), you can use any model (`QuiescentChecker` by default) to validate queries. -Queries can be provided manually or generated using `QueryGenerator` or `TypedQueryGenerator`. - -# Basic Terminology - -* Inflate / inflatable: a process of producing a value (for example, string, or a blob) from a `long` descriptor that - uniquely identifies the value. See data generation section of this guide for more details. -* Deflate / deflatable: a process of producing the descriptor the value was inflated from during verification. - See model section of this guide for more details. - -For definitions of logical timestamp, descriptor, and other entities used during inflation and deflation, refer -to formal relationships section. - -# Features - -Currently, Harry can exercise the following Cassandra functionality: - -* Supported data types: `int8`, `int16`, `int32`, `int64`, `boolean`, `float`, `double`, `ascii`, `uuid`, `timestamp`. - Collections are only _inflatable_. -* Random schema generation, with an arbitrary number of partition and clustering keys. -* Schemas with arbitrary `CLUSTERING ORDER BY` -* Randomly generated `INSERT` and `UPDATE` queries with all columns or arbitrary column subset -* Randomly generated `DELETE` queries: for a single column, single row, or a range of rows -* Inflating and validating entire partitions (with allowed in-flight queries) -* Inflating and validating random `SELECT` queries: single row, slices (with single open end), and ranges (with both - ends of clusterings specified) - -Inflating partitions is done using `Reconciler`. Validating partitions and random queries can be done using `QuiescentChecker`. - -## Outstanding Work - -#### The following have not yet been implemented: - -* Some types (such as collections) are not deflatable -* 2i queries are not implemented -* Fault injection is not implemented (available via Cassandra Simulator) -* TTL is not supported -* Some SELECT queries are not supported: `LIMIT`, `IN`, `GROUP BY`, token range queries - -#### Some features can be improved upon or further optimized: - -* Pagination is currently implemented but hard-coded to a page size of 1 -* RNG should be able to yield less than 64 bits of entropy per step -* State tracking should be done in a compact off-heap data structure -* Inflated partition state and per-row operation log should be done in a compact - off-heap data structure -* Decision-making about _when_ we visit partitions and/or rows should be improved - -While this list of improvements is incomplete, t should give the reader a rough idea about the state of the project. -The original goal of the project was to drive to stability after the significant storage engine rewrite in -CASSANDRA-8099 and help remove data loss bugs from the codebase before they got out into the wild. Next steps are to -integrate it into both CI and into regular daily dev workflows. - -# Goals: Reproducibility and Efficiency - -_Reproducibility_ is achieved by using the PCG family of random number generators and generating schema, configuration, -and every step of the workload from the repeatable sequence of random numbers. Schema and configuration are generated -from the _seed_. Each operation is assigned its own monotonically increasing _logical timestamp_, which preserves -logical operation order between different runs. - -_Efficiency_ is achieved by employing the features of the PCG random number generator (walking the sequence of random -numbers back and forth), and writing value generators in a way that preserves properties of the descriptor it was -generated from. - -Given a `long` _descriptor_ can be _inflated_ into some value: - -* value can be _deflated_ back to the descriptor it was generated from (in other words, generation is *invertible*) -* two inflated values will sort the same way as two descriptors they were generated from (in other words, generation is - *order-preserving*) - -These properties are also preserved for the composite values, such as clustering and partition keys. - -# Components - -Every Harry run starts from Configuration. You can find an example configuration under `conf/example.yml`. - -*Clock* is a component responsible for mapping _logical_ timestamps to _real-time_ ones. When reproducing test failures, -and for validation purposes, a snapshot of such be taken to map a real-time timestamp from the value retrieved from the -database to map it back to the logical timestamp of the operation that wrote this value. Given a real-time timestamp, -the clock can return a logical timestamp, and vice versa. - -*Runner* is a component that schedules operations that change the cluster (system under test) and model state. - -*System under test*: a Cassandra node or cluster. Default implementation is in_jvm (in-JVM DTest cluster). Harry also -supports external clusters. - -*Model* is responsible for tracking logical timestamps that system under test was notified about. - -*Partition descriptor selector* controls how partitions are selected based on the current logical timestamp. The default -implementation is a sliding window of partition descriptors that will visit one partition after the other in the -window `slide_after_repeats` times. After that, it will retire one partition descriptor, and pick a new one. - -*Clustering descriptor selector* controls how clustering keys are picked within the partition: how many rows there can -be in a partition, how many rows are visited for a logical timestamp, how many operations there will be in batch, what -kind of operations there will and how often each kind of operation is going to occur. - -# Implementing Tests - -All Harry components are pluggable and can be redefined. However, many of the default implementations will cover most of -the use-cases, so in this guide we’ll focus on ones that are most often used to implement different use cases: - -* System Under Test: defines how Harry can communicate with Cassandra instances and issue common queries. Examples of a - system under test can be a CCM cluster, a “real” Cassandra cluster, or an in-JVM dtest cluster. -* Visitor: defines behaviour that gets triggered at a specific logical timestamp. One of the default implementations is - MutatingVisitor, which executes write workload against SystemUnderTest. Examples of a visitor, besides a mutating - visitor, could be a validator that uses the model to validate results of different queries, a repair runner, or a - fault injector. -* Model: validates results of read queries by comparing its own internal representation against the results returned by - system under test. You can find three simplified implementations of model in this document: Visible Rows Checker, - Quiescent Checker, and an Exhaustive Checker. -* Runner: defines how operations defined by visitors are executed. Harry includes two default implementations: a - sequential and a concurrent runner. Sequential runner allows no overlap between different visitors or logical - timestamps. Concurrent runner allows visitors for different timestamps to overlap. - -System under test is the simplest one to implement: you only need a way to execute Cassandra queries. At the moment of -writing, all custom things, such as nodetool commands, failure injection, etc, are implemented using a SUT / visitor -combo: visitor knows about internals of the cluster it is dealing with. - -Generally, visitor has to follow the rules specified by DescriptorSelector and PdSelector: (it can only visit issue -mutations against the partition that PdSelector has picked for this LTS), and DescriptorSelector (it can visit exactly -DescriptorSelector#numberOfModifications rows within this partition, operations have to have a type specified by -#operationKind, clustering and value descriptors have to be in accordance with DescriptorSelector#cd and -DescriptorSelector#vds). The reason for these limitations is because model has to be able to reproduce the exact -sequence of events that was applied to system under test. - -Default implementations of partition and clustering descriptors, used in fuzz mode allow to optimise verification. For -example, it is possible to go find logical timestamps that are visiting the same partition as any given logical -timestamp. When running Harry in unit-test mode, we use a special generating visitor that keeps an entire given sequence -of events in memory rather than producing it on the fly. For reasons of efficiency, we do not use generating visitors -for generating and verifying large datasets. - -# Formal Relations Between Entities - -To be able to implement efficient models, we had to reduce the amount of state required for validation to a minimum and -try to operate on primitive data values most of the time. Any Harry run starts with a `seed`. Given the same -configuration, and the same seed, we're able to make runs deterministic (in other words, records in two clusters created -from the same seed are going to have different real-time timestamps, but will be otherwise identical; logical time -stamps will also be identical). - -Since it's clear how to generate things like random schemas, cluster configurations, etc., let's discuss how we're -generating data, and why this type of generation makes validation efficient. - -First, we're using PCG family of random number generators, which, besides having nice characteristics that any RNG -should have, have two important features: - -* Streams: for single seed, we can have several independent _different_ streams of random numbers. -* Walkability: PCG generators generate a stream of numbers you can walk _back_ and _forth_. That is, for any number _n_ - that represents a _position_ of the random number in the stream of random numbers, we can get the random number at - this position. Conversely, given a random number, we can determine what is its position in the stream. Moreover, - knowing a random number, we can determine which number precedes it in the stream of random numbers, and, finally, we - can determine how many numbers there are in a stream between the two random numbers. - -Out of these operations, determining the _next_ random number in the sequence can be done in constant time, `O(1)`. -Advancing generator by _n_ steps can be done in `O(log(n))` steps. Since generation is cyclical, advancing the iterator -backward is equivalent to advancing it by `cardinality - 1` steps. If we're generating 64 bits of entropy, advancing -by `-1` can be done in 64 steps. - -Let's introduce some definitions: - -* `lts` is a *logical timestamp*, an entity (number in our case), given by the clock, on which some action occurs -* `m` is a *modification id*, a sequential number of the modification that occurs on `lts` -* `rts` is an approximate real-time as of clock for this run -* `pid` is a partition position, a number between `0` and N, for `N` unique generated partitions -* `pd` is a partition descriptor, a unique descriptor identifying the partition -* `cd` is a clustering descriptor, a unique descriptor identifying row within some partition - -A small introduction that can help to understand the relation between these -entities. Hierarchically, the generation process looks as follows: - -* `lts` is an entry point, from which the decision process starts -* `pd` is picked from `lts`, and determines which partition is going to be visited -* for `(pd, lts)` combination, `#mods` (the number of modification batches) and `#rows` (the number of rows per - modification batch) is determined. `m` is an index of the modification batch, and `i` is an index of the operation in - the modification batch. -* `cd` is picked based on `(pd, lts)`, and `n`, a sequential number of the operation among all modification batches -* operation type (whether we're going to perform a write, delete, range delete, etc), columns involved in this - operation, and values for the modification are picked depending on the `pd`, `lts`, `m`, and `i` - -Most of this formalization is implemented in `OpSelectors`, and is relied upon in`PartitionVisitor` and any -implementation of a `Model`. - -Random number generation (see `OpSelectors#Rng`): - -* `rng(i, stream[, e])`: returns i'th number drawn from random sequence `stream` producing values with `e` bits of - entropy (64 bits for now). -* `rng'(s, stream[, e])`: returns `i` of the random number `s` drawn from the random sequence `stream`. This function is - an inverse of `rng`. -* `next(rnd, stream[, e])` and `prev(rnd, stream[, e])`: the next/previous number relative to `rnd` drawn from random - sequence `stream`. - -A simple example of a partition descriptor selector is one that is based on a sliding window of a size `s`, that slides -every `n` iterations. First, we determine _where_ the window should start for a given `lts` (in other words, how many -times it has already slid). After that, we determine which `pd` we pick out of `s` available ones. After picking each -one of the `s` descriptors `n` times, we retire the oldest descriptor and pick a new one to the window. Window start and -offset are then used as input for the `rng(start + offset, stream)` to make sure descriptors are uniformly distributed. - -We can build a clustering descriptor selector in a similar manner. Each partition will use its `pd` as a stream id, and -pick `cd` from a universe of possible `cds` of size `#cds`. On each `lts`, we pick a random `offset`, and start -picking `#ops` clusterings from this `offset < #cds`, and wrap around to index 0 after that. This way, each operation -maps to a unique `cd`, and `#op` can be determined from `cd` deterministically. - -# Data Generation - -So far, we have established how to generate partition, clustering, and value _descriptors_. Now, we need to understand -how we can generate data modification statements out of these descriptors in a way that helps us to validate data later. - -Since every run has a predefined schema, and by the time we visit a partition we have a logical timestamp, we can make -the rest of the decisions: pick a number of batches we're about to perform, determine what kind of operations each one -of the batches is going to contain, which rows we're going to visit (clustering for each modification operation). - -To generate a write, we need to know _which partition_ we're going to visit (in other words, partition descriptor), -_which row_ we'd like to modify (in other words, clustering descriptor), _which columns_ we're modifying (in other -words, a column mask), and, for each modified column - its value. By the time we're ready to make an actual query to the -database, we already know `pd`, `cd`, `rts`, and `vds[]`, which is all we need to "inflate" a write. - -To inflate each value descriptor, we take a generator for its datatype, and turn its descriptor into the object. This -generation process has the following important properties: - -* it is invertible: for every `inflate(vd) -> value`, there's `deflate(value) -> vd` -* it is order-preserving: `compare(vd1, vd2) == compare(inflate(vd1), inflate(vd2))` - -Inflating `pd` and `cd` is slightly more involved than inflating `vds`, since partition and clustering keys are often -composite. This means that `inflate(pd)` returns an array of objects, rather just a single -object: `inflate(pd) -> value[]`, and `deflate(value[]) -> pd`. Just like inflating value descriptors, inflating keys -preserves order. - -It is easy to see that, given two modifications: `Update(pd1, cd1, [vd1_1, vd2_1, vd3_1], lts1)` -and `Update(pd1, cd1, [vd1_2, vd3_2], lts2)`, we will end up with a resultset that contains effects of both -operations: `ResultSetRow(pd1, cd1, [vd1_2@rts2, vd2_1@rts1, vd3_2@rts2])`. - -# Model - -`Model` in Harry ties the rest of the components together and allows us to check whether or not data returned by the -cluster actually makes sense. The model relies on the clock, since we have to convert real-time timestamps of the -returned values back to logical timestamps, and on descriptor selectors to pick the right partition and rows. - -## Visible Rows Checker - -Let's try to put it all together and build a simple model. The simplest one is a visible row checker. It can check if -any row in the response returned from the database could have been produced by one of the operations. However, it won't -be able to find errors related to missing rows, and will only notice some cases of erroneously overwritten rows. - -In the model, we can see a response from the database in its deflated state. In other words, instead of the actual -values returned, we see their descriptors. Every resultset row consists of `pd`, `cd`, `vds[]` (value descriptors), -and `lts[]` (logical timestamps at which these values were written). - -To validate, we need to iterate through all operations for this partition, starting with the latest one the model is -aware of. This model has no internal state, and validates entire partitions: - -``` -void validatePartitionState(long validationLts, List<ResultSetRow> rows) { - long pd = pdSelector.pd(validationLts, schema); - - for (ResultSetRow row : rows) { - // iterator that gives us unique lts from the row in descending order - LongIterator rowLtsIter = descendingIterator(row.lts); - // iterator that gives us unique lts from the model in descending order - LongIterator modelLtsIter = descendingIterator(pdSelector, validationLts); - - outer: - while (rowLtsIter.hasNext()) { - long rowLts = rowLtsIter.nextLong(); - - // this model can not check columns whose values were never written or were deleted - if (rowLts == NO_TIMESTAMP) - continue outer; - - if (!modelLtsIter.hasNext()) - throw new ValidationException(String.format("Model iterator is exhausted, could not verify %d lts for the row: \n%s %s", - rowLts, row)); - - while (modelLtsIter.hasNext()) { - long modelLts = modelLtsIter.nextLong(); - // column was written by the operation that has a lower lts than the current one from the model - if (modelLts > rowLts) - continue; - // column was written by the operation that has a higher lts, which contradicts to the model, since otherwise we'd validate it by now - if (modelLts < rowLts) - throw new RuntimeException("Can't find a corresponding event id in the model for: " + rowLts + " " + modelLts); - - // Compare values for columns that were supposed to be written with this lts - for (int col = 0; col < row.lts.length; col++) { - if (row.lts[col] != rowLts) - continue; - - long m = descriptorSelector.modificationId(pd, row.cd, rowLts, row.vds[col], col); - long vd = descriptorSelector.vd(pd, row.cd, rowLts, m, col); - - // If the value model predicts doesn't match the one received from the database, throw an exception - if (vd != row.vds[col]) - throw new RuntimeException("Returned value doesn't match the model"); - } - continue outer; - } - } - } -} -``` - -As you can see, all validation is done using deflated `ResultSetRows`, which contain enough data to say which logical -timestamp each value was written with, and which value descriptor each value has. This model can also validate data -concurrently to the ongoing data modification operations. - -## Quiescent Checker - -Let's consider one more checker. It'll be more powerful than the visible rows checker in one way since it can find any -inconsistency in data (incorrect timestamp, missing or additional row, rows coming in the wrong order, etc), but it'll -also have one limitation: it won't be able to run concurrently with data modification statements. This means that for -this model to be used, we should have no _in-flight_ queries, and all queries have to be in a deterministic state by the -time we're validating their results. - -For this checker, we assume that we have a component that is called `Reconciler`, which can inflate partition state _up -to some_ `lts`. `Reconciler` works by simply applying each modification in the same order they were applied to the -cluster, and using standard Cassandra data reconciliation rules (last write wins / DELETE wins over INSERT in case of a -timestamp collision). - -With this component, and knowing that there can be no in-fight queries, we can validate data in the following way: - -``` -public void validatePartitionState(Iterator<ResultSetRow> actual, Query query) { - // find out up the highest completed logical timestamp - long maxCompleteLts = tracker.maxComplete(); - - // get the expected state from reconciler - Iterator<Reconciler.RowState> expected = reconciler.inflatePartitionState(query.pd, maxCompleteLts, query).iterator(query.reverse); - - // compare actual and expected rows one-by-one in-order - while (actual.hasNext() && expected.hasNext()) { - ResultSetRow actualRowState = actual.next(); - Reconciler.RowState expectedRowState = expected.next(); - - if (actualRowState.cd != expectedRowState.cd) - throw new ValidationException("Found a row in the model that is not present in the resultset:\nExpected: %s\nActual: %s", - expectedRowState, actualRowState); - - if (!Arrays.equals(actualRowState.vds, expectedRowState.vds)) - throw new ValidationException("Returned row state doesn't match the one predicted by the model:\nExpected: %s (%s)\nActual: %s (%s).", - Arrays.toString(expectedRowState.vds), expectedRowState, - Arrays.toString(actualRowState.vds), actualRowState); - - if (!Arrays.equals(actualRowState.lts, expectedRowState.lts)) - throw new ValidationException("Timestamps in the row state don't match ones predicted by the model:\nExpected: %s (%s)\nActual: %s (%s).", - Arrays.toString(expectedRowState.lts), expectedRowState, - Arrays.toString(actualRowState.lts), actualRowState); - } - - if (actual.hasNext() || expected.hasNext()) { - throw new ValidationException("Expected results to have the same number of results, but %s result iterator has more results", - actual.hasNext() ? "actual" : "expected"); - } -} -``` - -If there's any mismatch, it'll be caught right away: if there's an extra row -(for example, there were issues in Cassandra that caused it to have duplicate -rows), or if some row or even value in the row is missing. - -## Exhaustive Checker - -To be able to both run validation concurrently to modifications and be able to -catch all kinds of inconsistencies, we need a more involved checker. - -In this checker, we rely on inflating partition state. However, we're most -interested in `lts`, `opId`, and visibility (whether or not it is still -in-flight) of each modification operation. To be able to give a reliable result, -we need to make sure we follow these rules: - -* every operation model _thinks_ should be visible, has to be visible -* every operation model _thinks_ should be invisible, has to be invisible -* every operation model doesn't know the state of (i.e., it is still - in-flight) can be _either_ visible _invisible_ -* there can be no state in the database that model is not aware of (in other words, - we either can _explain_ how a row came to be, or we conclude that the row is - erroneous) - -A naive way to do this would be to inflate every possible partition state, where -every in-flight operation would be either visible or invisible, but this gets -costly very quickly since the number of possible combinations grows -exponentially. A better (and simpler) way to do this is to iterate all -operations and keep the state of "explained" operations: - -``` -public class RowValidationState { - // every column starts in UNOBSERVED, and has to move to either REMOVED, or OBSERVED state - private final ColumnState[] columnStates; - // keep track of operations related to each column state - private final Operation[] causingOperations; -} -``` - -Now, we move through all operations for a given row, starting from the _newest_ ones, towards -the oldest ones: - -``` -public void validatePartitionState(long verificationLts, PeekingIterator<ResultSetRow> actual_, Query query) { - // get a list of operations for each cd - NavigableMap<Long, List<Operation>> operations = inflatePartitionState(query); - - for (Map.Entry<Long, List<Operation>> entry : operations.entrySet()) { - long cd = entry.getKey(); - List<Operation> ops = entry.getValue(); - - // Found a row that is present both in the model and in the resultset - if (actual.hasNext() && actual.peek().cd == cd) { - validateRow(new RowValidationState(actual.next), operations); - } else { - validateNoRow(cd, operations); - - // Row is not present in the resultset, and we currently look at modifications with a clustering past it - if (actual.hasNext() && cmp.compare(actual.peek().cd, cd) < 0) - throw new ValidationException("Couldn't find a corresponding explanation for the row in the model"); - } - } - - // if there are more rows in the resultset, and we don't have model explanation for them, we've found an issue - if (actual.hasNext()) - throw new ValidationException("Observed unvalidated rows"); -} -``` - -Now, we have to implement `validateRow` and `validateNoRow`. `validateNoRow` is easy: we only need to make sure that a -set of operations results in an invisible row. Since we're iterating operations in reverse order, if we encounter a -delete not followed by any writes, we can conclude that the row is invisible and exit early. If there's a write that is -not followed by a delete, and the row isn't covered by a range tombstone, we know it's an error. - -`validateRow` only has to iterate operations in reverse order until it can explain the value in every column. For -example, if a value is `UNOBSERVED`, and the first thing we encounter is a `DELETE` that removes this column, we only -need to make sure that the value is actually `null`, in which case we can conclude that the value can be explained -as `REMOVED`. - -Similarly, if we encounter an operation that has written the expected value, we conclude that the value is `OBSERVED`. -If there are any seeming inconsistencies between the model and resultset, we have to check whether or not the operation -in question is still in flight. If it is, its results may still not be visible, so we can't reliably say it's an error. - -To summarize, in order for us to implement an exhaustive checker, we have to iterate operations for each of the rows -present in the model in reverse order until we either detect inconsistency that can't be explained by an in-flight -operation or until we explain every value in the row. - -## Conclusion - -As you can see, all checkers up till now are almost entirely stateless. Exhaustive and quiescent models rely -on `DataTracker` component that is aware of the in-flight and completed `lts`, but don't need any other state apart from -that, since we can always inflate a complete partition from scratch every time we validate. - -While not relying on the state is a useful feature, at least _some_ state is useful to have. For example, if we're -validating just a few rows in the partition, right now we have to iterate through each and every `lts` that has visited -this partition and filter out only modifications that have visited it. However, since the model is notified of each -_started_, and, later, finished modification via `recordEvent`, we can keep track of `pd -> (cd -> lts)` map. You can -check out `VisibleRowsChecker` as an example of that. diff --git a/test/harry/main/org/apache/cassandra/harry/ColumnSpec.java b/test/harry/main/org/apache/cassandra/harry/ColumnSpec.java new file mode 100644 index 000000000000..1bad3652408f --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/ColumnSpec.java @@ -0,0 +1,610 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.net.InetAddress; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.UUID; + +import accord.utils.Invariants; +import org.apache.cassandra.cql3.ast.Symbol; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.AsciiType; +import org.apache.cassandra.db.marshal.BooleanType; +import org.apache.cassandra.db.marshal.ByteType; +import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.marshal.DecimalType; +import org.apache.cassandra.db.marshal.DoubleType; +import org.apache.cassandra.db.marshal.FloatType; +import org.apache.cassandra.db.marshal.InetAddressType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.IntegerType; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.db.marshal.ShortType; +import org.apache.cassandra.db.marshal.TimeType; +import org.apache.cassandra.db.marshal.TimeUUIDType; +import org.apache.cassandra.db.marshal.TimestampType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.marshal.UUIDType; +import org.apache.cassandra.harry.gen.Generator; +import org.apache.cassandra.harry.gen.Generators; +import org.apache.cassandra.harry.gen.TypeAdapters; +import org.apache.cassandra.utils.ByteArrayUtil; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.TimeUUID; + +// TODO: counters +// TODO: UDTs +// TODO: collections: frozen/unfrozen +// TODO: empty / 0 / min / max values if present +public class ColumnSpec<T> +{ + public final String name; + public final DataType<T> type; + public final Generator<T> gen; + public final Kind kind; + + public ColumnSpec(String name, + DataType<T> type, + Generator<T> gen, + Kind kind) + { + + this.name = name; + this.type = Invariants.nonNull(type); + this.gen = Invariants.nonNull(gen); + this.kind = kind; + } + + public String toCQL() + { + return String.format("%s %s%s", + Symbol.maybeQuote(name), + type, + kind == Kind.STATIC ? " static" : ""); + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ColumnSpec<?> that = (ColumnSpec<?>) o; + return Objects.equals(name, that.name) && + Objects.equals(type.cqlName, that.type.cqlName) && + kind == that.kind; + } + + @Override + public int hashCode() + { + return Objects.hash(name, type.cqlName, kind); + } + + public String name() + { + return name; + } + + public boolean isReversed() + { + return type.isReversed(); + } + + public String toString() + { + return name + '(' + type.toString() + ")"; + } + + public Generator<T> gen() + { + return gen; + } + + public static <T> ColumnSpec<T> pk(String name, DataType<T> type, Generator<T> gen) + { + return new ColumnSpec<>(name, type, gen, Kind.PARTITION_KEY); + } + + public static <T> ColumnSpec<T> pk(String name, DataType<T> type) + { + return new ColumnSpec<>(name, type, (Generator<T>) TypeAdapters.forValues(type.asServerType()), Kind.PARTITION_KEY); + } + + @SuppressWarnings("unchecked") + public static <T> ColumnSpec<T> ck(String name, DataType<T> type, Generator<T> gen, boolean isReversed) + { + return new ColumnSpec(name, isReversed ? ReversedType.getInstance(type) : type, gen, Kind.CLUSTERING); + } + + public static <T> ColumnSpec<T> ck(String name, DataType<T> type) + { + return ck(name, type, false); + } + + public static <T> ColumnSpec<T> ck(String name, DataType<T> type, boolean isReversed) + { + return new ColumnSpec(name, isReversed ? ReversedType.getInstance(type) : type, + TypeAdapters.forValues(type.asServerType()), + Kind.CLUSTERING); + } + + + public static <T> ColumnSpec<T> regularColumn(String name, DataType<T> type, Generator<T> gen) + { + return new ColumnSpec<>(name, type, gen, Kind.REGULAR); + } + + public static <T> ColumnSpec<T> regularColumn(String name, DataType<T> type) + { + return new ColumnSpec(name, type, TypeAdapters.forValues(type.asServerType()), Kind.REGULAR); + } + + public static <T> ColumnSpec<T> staticColumn(String name, DataType<T> type, Generator<T> gen) + { + return new ColumnSpec<>(name, type, gen, Kind.STATIC); + } + + public static <T> ColumnSpec<T> staticColumn(String name, DataType<T> type) + { + return new ColumnSpec(name, type, TypeAdapters.forValues(type.asServerType()), Kind.STATIC); + } + + public enum Kind + { + CLUSTERING, REGULAR, STATIC, PARTITION_KEY + } + + public static abstract class DataType<T> + { + protected final String cqlName; + + protected DataType(String cqlName) + { + this.cqlName = cqlName; + } + + public abstract /* unsigned */ long typeEntropy(); + + public boolean isReversed() + { + return false; + } + + public abstract AbstractType<?> asServerType(); + + public final String toString() + { + return cqlName; + } + + public final boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + DataType<?> dataType = (DataType<?>) o; + return Objects.equals(cqlName, dataType.cqlName); + } + + public final int hashCode() + { + return Objects.hash(cqlName); + } + + public abstract Comparator<T> comparator(); + } + + public static abstract class ComparableDataType<T extends Comparable<? super T>> extends DataType<T> + { + protected ComparableDataType(String cqlName) + { + super(cqlName); + } + + @Override + public Comparator<T> comparator() + { + return Comparable::compareTo; + } + } + + public static final DataType<Byte> int8Type = new ComparableDataType<>("tinyint") + { + @Override + public /* unsigned */ long typeEntropy() + { + return 1L << (8 * Byte.BYTES); + } + + @Override + public AbstractType<?> asServerType() + { + return ByteType.instance; + } + }; + + public static final DataType<Short> int16Type = new ComparableDataType<>("smallint") + { + @Override + public /* unsigned */ long typeEntropy() + { + return 1L << Byte.SIZE; + } + + @Override + public AbstractType<?> asServerType() + { + return ShortType.instance; + } + }; + + public static final DataType<Integer> int32Type = new ComparableDataType<>("int") + { + @Override + public /* unsigned */ long typeEntropy() + { + return 1L << Integer.SIZE; + } + + @Override + public AbstractType<?> asServerType() + { + return Int32Type.instance; + } + }; + + public static final DataType<Long> int64Type = new ComparableDataType<Long>("bigint") + { + @Override + public /* unsigned */ long typeEntropy() + { + return 1L << (8 * Long.BYTES - 1); + } + + @Override + public AbstractType<?> asServerType() + { + return LongType.instance; + } + }; + + public static final DataType<Boolean> booleanType = new ComparableDataType<Boolean>("boolean") + { + @Override + public /* unsigned */ long typeEntropy() + { + return 2; + } + + @Override + public AbstractType<?> asServerType() + { + return BooleanType.instance; + } + }; + + public static final DataType<Float> floatType = new ComparableDataType<Float>("float") + { + @Override + public /* unsigned */ long typeEntropy() + { + return 1L << (4 * Float.BYTES); + } + + @Override + public AbstractType<?> asServerType() + { + return FloatType.instance; + } + }; + + public static final DataType<Double> doubleType = new ComparableDataType<Double>("double") + { + @Override + public /* unsigned */ long typeEntropy() + { + return 1L << 63; + } + + @Override + public AbstractType<?> asServerType() + { + return DoubleType.instance; + } + }; + + public static final DataType<ByteBuffer> blobType = new DataType<>("blob") + { + @Override + public /* unsigned */ long typeEntropy() + { + return 1L << 63; + } + + @Override + public AbstractType<?> asServerType() + { + return BytesType.instance; + } + + public Comparator<ByteBuffer> comparator() + { + return ByteBufferUtil::compareUnsigned; + } + }; + + public static final DataType<String> asciiType = new ComparableDataType<>("ascii") + { + @Override + public /* unsigned */ long typeEntropy() + { + return 1L << 63; + } + + @Override + public AbstractType<?> asServerType() + { + return AsciiType.instance; + } + }; + + // utf8 + public static final DataType<String> textType = new DataType<>("text") + { + @Override + public /* unsigned */ long typeEntropy() + { + return 1L << 63; + } + + @Override + public AbstractType<?> asServerType() + { + return UTF8Type.instance; + } + + @Override + public Comparator<String> comparator() + { + return (o1, o2) -> ByteArrayUtil.compareUnsigned(o1.getBytes(), o2.getBytes()); + } + }; + + public static final DataType<UUID> uuidType = new DataType<>("uuid") + { + @Override + public /* unsigned */ long typeEntropy() + { + return 1L << 63; + } + + @Override + public AbstractType<?> asServerType() + { + return UUIDType.instance; + } + + public Comparator<UUID> comparator() + { + // TODO: avoid serialization to match C* order + return (o1, o2) -> UUIDType.instance.compare(UUIDType.instance.decompose(o1), + UUIDType.instance.decompose(o2)); + } + }; + + public static final DataType<TimeUUID> timeUuidType = new ComparableDataType<>("timeuuid") + { + @Override + public /* unsigned */ long typeEntropy() + { + return 1L << 63; + } + + @Override + public AbstractType<?> asServerType() + { + return TimeUUIDType.instance; + } + }; + + public static final DataType<Date> timestampType = new ComparableDataType<>("timestamp") + { + @Override + public /* unsigned */ long typeEntropy() + { + return 1L << 63; + } + + @Override + public AbstractType<?> asServerType() + { + return TimestampType.instance; + } + }; + + public static final DataType<BigInteger> varintType = new ComparableDataType<>("varint") + { + @Override + public /* unsigned */ long typeEntropy() + { + return 1L << 63; + } + + @Override + public AbstractType<?> asServerType() + { + return IntegerType.instance; + } + }; + + public static final DataType<Long> timeType = new ComparableDataType<>("time") + { + @Override + public /* unsigned */ long typeEntropy() + { + return 1L << 63; + } + + @Override + public AbstractType<?> asServerType() + { + return TimeType.instance; + } + }; + + public static final DataType<BigDecimal> decimalType = new ComparableDataType<>("decimal") + { + @Override + public /* unsigned */ long typeEntropy() + { + return 1L << 63; + } + + @Override + public AbstractType<?> asServerType() + { + return DecimalType.instance; + } + }; + + public static final DataType<InetAddress> inetType = new DataType<>("inet") + { + @Override + public /* unsigned */ long typeEntropy() + { + return 1L << 63; + } + + @Override + public AbstractType<?> asServerType() + { + return InetAddressType.instance; + } + + @Override + public Comparator<InetAddress> comparator() + { + return (o1, o2) -> { + byte[] b1 = o1.getAddress(); + byte[] b2 = o2.getAddress(); + return ByteArrayUtil.compareUnsigned(b1, b2); + }; + } + }; + + public static final List<DataType<?>> TYPES; + + static + { + List<DataType<?>> types = new ArrayList<>() + {{ + add(int8Type); + add(int16Type); + add(int32Type); + add(int64Type); + add(floatType); + add(doubleType); + // TODO: SAI tests seem to fail these types + // add(booleanType); + // add(inetType); + // add(varintType); + // add(decimalType); + add(asciiType); + add(textType); + // TODO: blob is not supported in SAI + // add(blobType); + add(uuidType); + add(timestampType); + // TODO: SAI test fails due to TimeSerializer#toString in tracing + // add(timeType); + // TODO: compose proper value + // add(timeUuidType); + }}; + TYPES = Collections.unmodifiableList(types); + } + + public static Generator<DataType<?>> regularColumnTypeGen() + { + return Generators.pick(TYPES); + } + + public static Generator<DataType<?>> clusteringColumnTypeGen() + { + return Generators.pick(new ArrayList<>(ReversedType.cache.keySet())); + } + + public static class ReversedType<T> extends DataType<T> + { + public static final Map<DataType<?>, ReversedType<?>> cache = new HashMap<>() + {{ + for (DataType<?> type : TYPES) + put(type, new ReversedType<>(type)); + }}; + + private final DataType<T> baseType; + + public ReversedType(DataType<T> baseType) + { + super(baseType.cqlName); + this.baseType = baseType; + } + + @Override + public /* unsigned */ long typeEntropy() + { + return baseType.typeEntropy(); + } + + public boolean isReversed() + { + return true; + } + + @Override + public AbstractType<?> asServerType() + { + return org.apache.cassandra.db.marshal.ReversedType.getInstance(baseType.asServerType()); + } + + public static <T> DataType<T> getInstance(DataType<T> type) + { + ReversedType<T> t = (ReversedType<T>) cache.get(type); + if (t == null) + t = new ReversedType<>(type); + assert t.baseType == type : String.format("Type mismatch %s != %s", t.baseType, type); + return t; + } + + @Override + public Comparator<T> comparator() + { + return baseType.comparator(); + } + } +} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/HarryHelper.java b/test/harry/main/org/apache/cassandra/harry/HarryHelper.java deleted file mode 100644 index 9dc0fcfaf415..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/HarryHelper.java +++ /dev/null @@ -1,172 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry; - -import java.io.File; -import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.cassandra.harry.clock.OffsetClock; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.ddl.SchemaGenerators; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.dsl.ReplayingHistoryBuilder; -import org.apache.cassandra.harry.gen.Surjections; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.runner.HarryRunner; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.tracker.DefaultDataTracker; - -import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_ALLOW_SIMPLE_STRATEGY; -import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_MINIMUM_REPLICATION_FACTOR; -import static org.apache.cassandra.config.CassandraRelevantProperties.DISABLE_TCACTIVE_OPENSSL; -import static org.apache.cassandra.config.CassandraRelevantProperties.IO_NETTY_TRANSPORT_NONATIVE; -import static org.apache.cassandra.config.CassandraRelevantProperties.LOG4J2_DISABLE_JMX; -import static org.apache.cassandra.config.CassandraRelevantProperties.LOG4J2_DISABLE_JMX_LEGACY; -import static org.apache.cassandra.config.CassandraRelevantProperties.LOG4J_SHUTDOWN_HOOK_ENABLED; -import static org.apache.cassandra.config.CassandraRelevantProperties.ORG_APACHE_CASSANDRA_DISABLE_MBEAN_REGISTRATION; - -public class HarryHelper -{ - public static final String KEYSPACE = "harry"; - - public static void init() - { - // setting both ways as changes between versions - LOG4J2_DISABLE_JMX.setBoolean(true); - LOG4J2_DISABLE_JMX_LEGACY.setBoolean(true); - LOG4J_SHUTDOWN_HOOK_ENABLED.setBoolean(false); - CASSANDRA_ALLOW_SIMPLE_STRATEGY.setBoolean(true); // makes easier to share OSS tests without RF limits - CASSANDRA_MINIMUM_REPLICATION_FACTOR.setInt(0); // makes easier to share OSS tests without RF limits - DISABLE_TCACTIVE_OPENSSL.setBoolean(true); - IO_NETTY_TRANSPORT_NONATIVE.setBoolean(true); - ORG_APACHE_CASSANDRA_DISABLE_MBEAN_REGISTRATION.setBoolean(true); - } - - public static Configuration configuration(String... args) throws Exception - { - File configFile = HarryRunner.loadConfig(args); - Configuration configuration = Configuration.fromFile(configFile); - System.out.println("Using configuration generated from: " + configFile); - return configuration; - } - - private static AtomicInteger counter = new AtomicInteger(); - - public static Surjections.Surjection<SchemaSpec> schemaSpecGen(String keyspace, String prefix) - { - return new SchemaGenerators.Builder(keyspace, () -> prefix + counter.getAndIncrement()) - .partitionKeySpec(1, 2, - ColumnSpec.int8Type, - ColumnSpec.int16Type, - ColumnSpec.int32Type, - ColumnSpec.int64Type, - ColumnSpec.doubleType, - ColumnSpec.asciiType, - ColumnSpec.textType) - .clusteringKeySpec(1, 2, - ColumnSpec.int8Type, - ColumnSpec.int16Type, - ColumnSpec.int32Type, - ColumnSpec.int64Type, - ColumnSpec.doubleType, - ColumnSpec.asciiType, - ColumnSpec.textType, - ColumnSpec.ReversedType.getInstance(ColumnSpec.int64Type), - ColumnSpec.ReversedType.getInstance(ColumnSpec.doubleType), - ColumnSpec.ReversedType.getInstance(ColumnSpec.asciiType), - ColumnSpec.ReversedType.getInstance(ColumnSpec.textType)) - .regularColumnSpec(1, 5, - ColumnSpec.int8Type, - ColumnSpec.int16Type, - ColumnSpec.int32Type, - ColumnSpec.int64Type, - ColumnSpec.floatType, - ColumnSpec.doubleType, - ColumnSpec.asciiType(4, 128)) - .staticColumnSpec(0, 5, - ColumnSpec.int8Type, - ColumnSpec.int16Type, - ColumnSpec.int32Type, - ColumnSpec.int64Type, - ColumnSpec.floatType, - ColumnSpec.doubleType, - ColumnSpec.asciiType(4, 128)) - .surjection(); - } - - public static Configuration.ConfigurationBuilder defaultConfiguration() throws Exception - { - return new Configuration.ConfigurationBuilder() - .setClock(() -> new OffsetClock(100000)) - .setCreateSchema(true) - .setTruncateTable(false) - .setDropSchema(false) - .setSchemaProvider((seed, sut) -> schemaSpecGen("harry", "tbl_").inflate(seed)) - .setClusteringDescriptorSelector(defaultClusteringDescriptorSelectorConfiguration().build()) - .setPartitionDescriptorSelector(new Configuration.DefaultPDSelectorConfiguration(1, 1)) - .setDataTracker(new Configuration.DefaultDataTrackerConfiguration()) - .setRunner((run, configuration) -> { - throw new IllegalArgumentException("Runner is not configured by default."); - }) - .setMetricReporter(new Configuration.NoOpMetricReporterConfiguration()); - } - - public static ReplayingHistoryBuilder dataGen(SystemUnderTest sut, TokenPlacementModel.ReplicationFactor rf, SystemUnderTest.ConsistencyLevel writeCl) - { - return dataGen(1, sut, rf, writeCl); - } - - public static ReplayingHistoryBuilder dataGen(long seed, SystemUnderTest sut, TokenPlacementModel.ReplicationFactor rf, SystemUnderTest.ConsistencyLevel writeCl) - { - SchemaSpec schema = schemaSpecGen("harry", "tbl_").inflate(seed); - return new ReplayingHistoryBuilder(seed, 100, 1, new DefaultDataTracker(), sut, schema, rf, writeCl); - } - - public static Configuration.CDSelectorConfigurationBuilder defaultClusteringDescriptorSelectorConfiguration() - { - return new Configuration.CDSelectorConfigurationBuilder() - .setOperationsPerLtsDistribution(new Configuration.ConstantDistributionConfig(1)) - .setMaxPartitionSize(100) - .setOperationKindWeights(new Configuration.OperationKindSelectorBuilder() - .addWeight(OpSelectors.OperationKind.DELETE_ROW, 1) - .addWeight(OpSelectors.OperationKind.DELETE_COLUMN, 1) - .addWeight(OpSelectors.OperationKind.DELETE_RANGE, 1) - .addWeight(OpSelectors.OperationKind.DELETE_SLICE, 1) - .addWeight(OpSelectors.OperationKind.DELETE_PARTITION, 1) - .addWeight(OpSelectors.OperationKind.DELETE_COLUMN_WITH_STATICS, 1) - .addWeight(OpSelectors.OperationKind.INSERT_WITH_STATICS, 20) - .addWeight(OpSelectors.OperationKind.INSERT, 20) - .addWeight(OpSelectors.OperationKind.UPDATE_WITH_STATICS, 20) - .addWeight(OpSelectors.OperationKind.UPDATE, 20) - .build()); - } - - public static Configuration.CDSelectorConfigurationBuilder singleRowPerModification() - { - return new Configuration.CDSelectorConfigurationBuilder() - .setOperationsPerLtsDistribution(new Configuration.ConstantDistributionConfig(1)) - .setMaxPartitionSize(100) - .setOperationKindWeights(new Configuration.OperationKindSelectorBuilder() - .addWeight(OpSelectors.OperationKind.INSERT_WITH_STATICS, 100) - .build()); - } -} - diff --git a/test/harry/main/org/apache/cassandra/harry/MagicConstants.java b/test/harry/main/org/apache/cassandra/harry/MagicConstants.java new file mode 100644 index 000000000000..9e7137fccc37 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/MagicConstants.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry; + +import java.util.Set; + +import org.apache.cassandra.harry.util.BitSet; + +public class MagicConstants +{ + public static final BitSet ALL_COLUMNS = BitSet.allSet(64); + /** + * For keys + */ + public static final Object[] UNKNOWN_KEY = new Object[]{}; + public static final Object[] NIL_KEY = new Object[]{}; + + /** + * For values + */ + public static final Object UNKNOWN_VALUE = new Object() { + public String toString() + { + return "UNKNOWN"; + } + }; + public static final Object UNSET_VALUE = new Object() { + public String toString() + { + return "UNSET"; + } + }; + /** + * For Descriptors + */ + public static final long UNKNOWN_DESCR = Long.MIN_VALUE + 2; + // TODO: Empty value, for the types that support it + public static final long EMPTY_VALUE_DESCR = Long.MIN_VALUE + 1; + public static final long UNSET_DESCR = Long.MIN_VALUE + 3; + public static final long NIL_DESCR = Long.MIN_VALUE; + public static final Set<Long> MAGIC_DESCRIPTOR_VALS = Set.of(UNKNOWN_DESCR, EMPTY_VALUE_DESCR, UNSET_DESCR, NIL_DESCR); + /** + * For LTS + */ + public static final long[] LTS_UNKNOWN = new long[]{}; + public static final long NO_TIMESTAMP = Long.MIN_VALUE; + + /** + * For indices + */ + public static final int UNKNOWN_IDX = Integer.MIN_VALUE + 2; + public static final int UNSET_IDX = Integer.MIN_VALUE + 1; + public static final int NIL_IDX = Integer.MIN_VALUE; +} diff --git a/test/harry/main/org/apache/cassandra/harry/Relations.java b/test/harry/main/org/apache/cassandra/harry/Relations.java new file mode 100644 index 000000000000..91f64ded48b3 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/Relations.java @@ -0,0 +1,205 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry; + +import java.util.Arrays; +import java.util.Comparator; +import java.util.function.IntFunction; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.harry.gen.Bijections; + +public class Relations +{ + private static final Logger logger = LoggerFactory.getLogger(Relations.class); + + @SuppressWarnings({ "rawtypes", "unchecked" }) + public static boolean matchRange(Bijections.Bijection<Object[]> ckGen, + IntFunction<Comparator<Object>> comparators, + int ckCoulmnCount, + long lowBoundDescr, long highBoundDescr, + Relations.RelationKind[] lowBoundRelations, Relations.RelationKind[] highBoundRelations, + long matchDescr) + { + Object[] lowBoundValue = lowBoundDescr == MagicConstants.UNSET_DESCR ? null : ckGen.inflate(lowBoundDescr); + Object[] highBoundValue = highBoundDescr == MagicConstants.UNSET_DESCR ? null : ckGen.inflate(highBoundDescr); + Object[] matchValue = ckGen.inflate(matchDescr); + // TODO: assert that all equals + null checks + for (int i = 0; i < ckCoulmnCount; i++) + { + Object matched = matchValue[i]; + + if (lowBoundValue != null) + { + Object l = lowBoundValue[i]; + Relations.RelationKind lr = lowBoundRelations[i]; + + if (lr != null && !lr.match(comparators.apply(i), matched, l)) + { + if (logger.isTraceEnabled()) + logger.trace("Low Bound {} {} {} did match {}", lowBoundValue[i], lr, matchValue[i], i); + return false; + } + } + + if (highBoundValue != null) + { + Object h = highBoundValue[i]; + Relations.RelationKind hr = highBoundRelations[i]; + + if (hr != null && !hr.match(comparators.apply(i), matched, h)) + { + if (logger.isTraceEnabled()) + logger.trace("High Bound {} {} {} did match {}", highBoundValue[i], hr, matchValue[i], i); + return false; + } + } + } + + if (logger.isTraceEnabled()) + logger.trace("{} is between {} and {} fully matched match", Arrays.toString(matchValue), Arrays.toString(lowBoundValue), Arrays.toString(highBoundValue)); + return true; + } + + public static class Relation + { + public final Relations.RelationKind kind; + public final long descriptor; + public final int column; + public Relation(RelationKind kind, long descriptor, int column) + { + this.kind = kind; + this.descriptor = descriptor; + this.column = column; + } + } + + @SuppressWarnings({ "rawtypes", "unchecked" }) + public enum RelationKind + { + LT + { + @Override + public boolean match(Comparator<Object> comparator, Object l, Object r) + { + return comparator.compare(l, r) < 0; + } + + @Override + public String symbol() + { + return "<"; + } + + @Override + public RelationKind reverse() + { + return GT; + } + }, + GT + { + @Override + public boolean match(Comparator<Object> comparator, Object l, Object r) + { + return comparator.compare(l, r) > 0; + } + + @Override + public String symbol() + { + return ">"; + } + + @Override + public RelationKind reverse() + { + return LT; + } + }, + LTE + { + @Override + public boolean match(Comparator<Object> comparator, Object l, Object r) + { + return comparator.compare(l, r) <= 0; + } + + @Override + public String symbol() + { + return "<="; + } + + @Override + public RelationKind reverse() + { + return GTE; + } + }, + GTE + { + @Override + public boolean match(Comparator<Object> comparator, Object l, Object r) + { + return comparator.compare(l, r) >= 0; + } + + @Override + public String symbol() + { + return ">="; + } + + @Override + public RelationKind reverse() + { + return LTE; + } + }, + EQ + { + @Override + public boolean match(Comparator<Object> comparator, Object l, Object r) + { + return comparator.compare(l, r) == 0; + } + + @Override + public String symbol() + { + return "="; + } + + @Override + public RelationKind reverse() + { + return EQ; + } + }; + + public abstract boolean match(Comparator<Object> comparator, Object l, Object r); + + public abstract String symbol(); + + public abstract RelationKind reverse(); + } +} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/SchemaSpec.java b/test/harry/main/org/apache/cassandra/harry/SchemaSpec.java new file mode 100644 index 000000000000..5ab6862994ac --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/SchemaSpec.java @@ -0,0 +1,440 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.function.Consumer; + +import org.apache.cassandra.cql3.ast.Symbol; +import org.apache.cassandra.harry.dsl.HistoryBuilder; +import org.apache.cassandra.harry.gen.Generator; +import org.apache.cassandra.harry.gen.Generators; +import org.apache.cassandra.harry.gen.ValueGenerators; +import org.apache.cassandra.harry.util.IteratorsUtil; +import org.apache.cassandra.utils.ByteArrayUtil; + +import static org.apache.cassandra.harry.gen.InvertibleGenerator.MAX_ENTROPY; + +public class SchemaSpec +{ + public final String keyspace; + public final String table; + + public final List<ColumnSpec<?>> partitionKeys; + public final List<ColumnSpec<?>> clusteringKeys; + public final List<ColumnSpec<?>> regularColumns; + public final List<ColumnSpec<?>> staticColumns; + + public final List<ColumnSpec<?>> allColumnInSelectOrder; + public final ValueGenerators valueGenerators; + public final Options options; + + public SchemaSpec(long seed, + int populationPerColumn, + String keyspace, + String table, + List<ColumnSpec<?>> partitionKeys, + List<ColumnSpec<?>> clusteringKeys, + List<ColumnSpec<?>> regularColumns, + List<ColumnSpec<?>> staticColumns) + { + this(seed, populationPerColumn, keyspace, table, partitionKeys, clusteringKeys, regularColumns, staticColumns, optionsBuilder()); + } + + @SuppressWarnings({ "unchecked" }) + public SchemaSpec(long seed, + int populationPerColumn, + String keyspace, + String table, + List<ColumnSpec<?>> partitionKeys, + List<ColumnSpec<?>> clusteringKeys, + List<ColumnSpec<?>> regularColumns, + List<ColumnSpec<?>> staticColumns, + Options options) + { + this.keyspace = keyspace; + this.table = table; + this.options = options; + + this.partitionKeys = Collections.unmodifiableList(new ArrayList<>(partitionKeys)); + this.clusteringKeys = Collections.unmodifiableList(new ArrayList<>(clusteringKeys)); + this.staticColumns = Collections.unmodifiableList(new ArrayList<>(staticColumns)); + this.regularColumns = Collections.unmodifiableList(new ArrayList<>(regularColumns)); + + List<ColumnSpec<?>> staticSelectOrder = new ArrayList<>(staticColumns); + staticSelectOrder.sort((s1, s2) -> ByteArrayUtil.compareUnsigned(s1.name.getBytes(), s2.name.getBytes())); + List<ColumnSpec<?>> regularSelectOrder = new ArrayList<>(regularColumns); + regularSelectOrder.sort((s1, s2) -> ByteArrayUtil.compareUnsigned(s1.name.getBytes(), s2.name.getBytes())); + + List<ColumnSpec<?>> selectOrder = new ArrayList<>(); + for (ColumnSpec<?> column : IteratorsUtil.concat(partitionKeys, + clusteringKeys, + staticSelectOrder, + regularSelectOrder)) + selectOrder.add(column); + this.allColumnInSelectOrder = Collections.unmodifiableList(selectOrder); + + // TODO: empty gen + this.valueGenerators = HistoryBuilder.valueGenerators(this, seed, populationPerColumn); + } + + public static /* unsigned */ long cumulativeEntropy(List<ColumnSpec<?>> columns) + { + if (columns.isEmpty()) + return 0; + + long entropy = 1; + for (ColumnSpec<?> column : columns) + { + if (Long.compareUnsigned(column.type.typeEntropy(), MAX_ENTROPY) == 0) + return MAX_ENTROPY; + + long next = entropy * column.type.typeEntropy(); + if (Long.compareUnsigned(next, entropy) < 0 || Long.compareUnsigned(next, column.type.typeEntropy()) < 0) + return MAX_ENTROPY; + + entropy = next; + } + + return entropy; + } + + public static Generator<Object[]> forKeys(List<ColumnSpec<?>> columns) + { + Generator<?>[] gens = new Generator[columns.size()]; + for (int i = 0; i < gens.length; i++) + gens[i] = columns.get(i).gen; + return Generators.zipArray(gens); + } + + public String compile() + { + StringBuilder sb = new StringBuilder(); + + sb.append("CREATE TABLE "); + if (options.ifNotExists()) + sb.append("IF NOT EXISTS "); + + sb.append(Symbol.maybeQuote(keyspace)) + .append(".") + .append(Symbol.maybeQuote(table)) + .append(" ("); + + SeparatorAppender commaAppender = new SeparatorAppender(); + for (ColumnSpec<?> cd : partitionKeys) + { + commaAppender.accept(sb); + sb.append(cd.toCQL()); + if (partitionKeys.size() == 1 && clusteringKeys.isEmpty()) + sb.append(" PRIMARY KEY"); + } + + for (ColumnSpec<?> cd : IteratorsUtil.concat(clusteringKeys, + staticColumns, + regularColumns)) + { + commaAppender.accept(sb); + sb.append(cd.toCQL()); + } + + if (!clusteringKeys.isEmpty() || partitionKeys.size() > 1) + { + sb.append(", ").append(getPrimaryKeyCql()); + } + + // TODO: test + if (options.trackLts()) + sb.append(", ").append("visited_lts list<bigint> static"); + + sb.append(')'); + + Runnable appendWith = doOnce(() -> sb.append(" WITH")); + boolean shouldAppendAnd = false; + + if (options.compactStorage()) + { + appendWith.run(); + sb.append(" COMPACT STORAGE"); + shouldAppendAnd = true; + } + + if (options.disableReadRepair()) + { + appendWith.run(); + if (shouldAppendAnd) + sb.append(" AND"); + sb.append(" read_repair = 'NONE' "); + shouldAppendAnd = true; + } + + if (options.compactionStrategy() != null) + { + appendWith.run(); + if (shouldAppendAnd) + sb.append(" AND"); + sb.append(" compaction = {'class': '").append(options.compactionStrategy()).append("'}"); + shouldAppendAnd = true; + } + + if (!clusteringKeys.isEmpty()) + { + appendWith.run(); + if (shouldAppendAnd) + { + sb.append(" AND"); + shouldAppendAnd = false; + } + sb.append(getClusteringOrderCql()); + } + + if (shouldAppendAnd) + { + sb.append(" AND"); + shouldAppendAnd = false; + } + + sb.append(';'); + return sb.toString(); + } + + private String getClusteringOrderCql() + { + StringBuilder sb = new StringBuilder(); + if (!clusteringKeys.isEmpty()) + { + sb.append(" CLUSTERING ORDER BY ("); + + SeparatorAppender commaAppender = new SeparatorAppender(); + for (ColumnSpec<?> column : clusteringKeys) + { + commaAppender.accept(sb); + sb.append(column.name).append(' ').append(column.isReversed() ? "DESC" : "ASC"); + } + + sb.append(')'); + } + + return sb.toString(); + } + + private String getPrimaryKeyCql() + { + StringBuilder sb = new StringBuilder(); + sb.append("PRIMARY KEY ("); + if (partitionKeys.size() > 1) + { + sb.append('('); + SeparatorAppender commaAppender = new SeparatorAppender(); + for (ColumnSpec<?> cd : partitionKeys) + { + commaAppender.accept(sb); + sb.append(cd.name); + } + sb.append(')'); + } + else + { + sb.append(partitionKeys.get(0).name); + } + + for (ColumnSpec<?> cd : clusteringKeys) + sb.append(", ").append(cd.name); + + return sb.append(')').toString(); + } + + public String toString() + { + return String.format("schema {cql=%s}", compile()); + } + + private static Runnable doOnce(Runnable r) + { + return new Runnable() + { + boolean executed = false; + + public void run() + { + if (executed) + return; + + executed = true; + r.run(); + } + }; + } + + public static class SeparatorAppender implements Consumer<StringBuilder> + { + boolean isFirst = true; + private final String separator; + + public SeparatorAppender() + { + this(","); + } + + public SeparatorAppender(String separator) + { + this.separator = separator; + } + + public void accept(StringBuilder stringBuilder) + { + if (isFirst) + isFirst = false; + else + stringBuilder.append(separator); + } + + public void accept(StringBuilder stringBuilder, String s) + { + accept(stringBuilder); + stringBuilder.append(s); + } + + + public void reset() + { + isFirst = true; + } + } + + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + SchemaSpec that = (SchemaSpec) o; + return Objects.equals(keyspace, that.keyspace) && + Objects.equals(table, that.table) && + Objects.equals(partitionKeys, that.partitionKeys) && + Objects.equals(clusteringKeys, that.clusteringKeys) && + Objects.equals(regularColumns, that.regularColumns); + } + + public int hashCode() + { + return Objects.hash(keyspace, table, partitionKeys, clusteringKeys, regularColumns); + } + + public interface Options + { + boolean addWriteTimestamps(); + boolean disableReadRepair(); + String compactionStrategy(); + boolean compactStorage(); + boolean ifNotExists(); + boolean trackLts(); + } + + public static OptionsBuilder optionsBuilder() + { + return new OptionsBuilder(); + } + + public static class OptionsBuilder implements Options + { + private boolean addWriteTimestamps = true; + private boolean disableReadRepair = false; + private String compactionStrategy = null; + private boolean ifNotExists = false; + private boolean trackLts = false; + private boolean compactStorage = false; + + private OptionsBuilder() + { + } + + public OptionsBuilder addWriteTimestamps(boolean newValue) + { + this.addWriteTimestamps = newValue; + return this; + } + + @Override + public boolean addWriteTimestamps() + { + return addWriteTimestamps; + } + + public OptionsBuilder disableReadRepair(boolean newValue) + { + this.disableReadRepair = newValue; + return this; + } + + @Override + public boolean disableReadRepair() + { + return disableReadRepair; + } + + public OptionsBuilder compactionStrategy(String compactionStrategy) + { + this.compactionStrategy = compactionStrategy; + return this; + } + + @Override + public String compactionStrategy() + { + return compactionStrategy; + } + + public OptionsBuilder withCompactStorage() + { + this.compactStorage = true; + return this; + } + + @Override + public boolean compactStorage() + { + return compactStorage; + } + + public OptionsBuilder ifNotExists(boolean v) + { + this.ifNotExists = v; + return this; + } + + @Override + public boolean ifNotExists() + { + return ifNotExists; + } + + public OptionsBuilder trackLts(boolean v) + { + this.trackLts = v; + return this; + } + + @Override + public boolean trackLts() + { + return trackLts; + } + } +} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/harry/gen/ExtensionsTest.java b/test/harry/main/org/apache/cassandra/harry/ValueGeneratorHelper.java similarity index 58% rename from test/distributed/org/apache/cassandra/fuzz/harry/gen/ExtensionsTest.java rename to test/harry/main/org/apache/cassandra/harry/ValueGeneratorHelper.java index d6d331eed55a..c4bbababc01b 100644 --- a/test/distributed/org/apache/cassandra/fuzz/harry/gen/ExtensionsTest.java +++ b/test/harry/main/org/apache/cassandra/harry/ValueGeneratorHelper.java @@ -16,29 +16,26 @@ * limitations under the License. */ -package org.apache.cassandra.fuzz.harry.gen; +package org.apache.cassandra.harry; -import java.util.function.Supplier; - -import org.junit.Assert; -import org.junit.Test; +import java.util.function.IntFunction; +import org.apache.cassandra.harry.dsl.HistoryBuilder; import org.apache.cassandra.harry.gen.EntropySource; -import org.apache.cassandra.harry.gen.Generators; -public class ExtensionsTest +public class ValueGeneratorHelper { - - @Test - public void testPick() + public static long[] randomDescriptors(EntropySource rng, IntFunction<HistoryBuilder.IndexedBijection<Object>> valueGens, int count) { - Supplier<Integer> gen = Generators.pick(101, 102, 103, 104, 105).bind(EntropySource.forTests()); - - int[] counts = new int[5]; - for (int i = 0; i < 1000; i++) - counts[gen.get() - 101] += 1; - - // It is possible, however very improbable we won't hit each one at least once - for (int count: counts) Assert.assertTrue(count > 0); + long[] vds = new long[count]; + for (int i = 0; i < count; i++) + { + if (rng.nextBoolean()) + vds[i] = MagicConstants.UNSET_DESCR; + else + vds[i] = valueGens.apply(i).descriptorAt(rng.nextInt(count)); + } + + return vds; } } diff --git a/test/harry/main/org/apache/cassandra/harry/checker/ModelChecker.java b/test/harry/main/org/apache/cassandra/harry/checker/ModelChecker.java index a5dc2ffd25a4..9b4ad21317d8 100644 --- a/test/harry/main/org/apache/cassandra/harry/checker/ModelChecker.java +++ b/test/harry/main/org/apache/cassandra/harry/checker/ModelChecker.java @@ -53,13 +53,13 @@ public void run(int minSteps, long maxSteps) throws Throwable run(minSteps, maxSteps, new JdkRandomEntropySource(System.currentTimeMillis())); } - public void run(int minSteps, long maxSteps, EntropySource entropySource) throws Throwable + public void run(int minSteps, long maxSteps, EntropySource rng) throws Throwable { assert init != null : "Initial condition is not specified"; Ref<Pair<STATE, SUT>> state = new Ref<>(init, Pair.unchanged()); if (beforeAll != null) - state.map((s) -> beforeAll.next(s.l, s.r, entropySource)); + state.map((s) -> beforeAll.next(s.l, s.r, rng)); for (int i = 0; i < maxSteps; i++) { @@ -67,13 +67,13 @@ public void run(int minSteps, long maxSteps, EntropySource entropySource) throws return; // TODO: add randomisation / probability for triggering a specific step - steps.get(entropySource.nextInt(steps.size())).execute(state, entropySource.derive()); + steps.get(rng.nextInt(steps.size())).execute(state, rng.derive()); for (Precondition<STATE, SUT> invariant : invariants) invariant.test(state.get()); } if (afterAll != null) - state.map((s) -> afterAll.next(s.l, s.r, entropySource)); + state.map((s) -> afterAll.next(s.l, s.r, rng)); } public ModelChecker<STATE, SUT> init(STATE state, SUT sut) @@ -111,12 +111,12 @@ public ModelChecker<STATE, SUT> exitCondition(Precondition<STATE, SUT> precondit public ModelChecker<STATE, SUT> step(Precondition<STATE, SUT> precondition, Step<STATE, SUT> step) { - steps.add((ref, entropySource) -> { + steps.add((ref, rng) -> { ref.map(state -> { if (!precondition.test(state)) return state; - return step.next(state.l, state.r, entropySource); + return step.next(state.l, state.r, rng); }); }); @@ -136,7 +136,7 @@ public ModelChecker<STATE, SUT> step(Step<STATE, SUT> step) public ModelChecker<STATE, SUT> step(StatePrecondition<STATE> precondition, ThrowingFunction<STATE, STATE> step) { - steps.add((ref, entropySource) -> { + steps.add((ref, rng) -> { ref.map(state -> { if (!precondition.test(state.l)) return state; @@ -154,7 +154,7 @@ public ModelChecker<STATE, SUT> step(StatePrecondition<STATE> precondition, Thro public ModelChecker<STATE, SUT> step(StatePrecondition<STATE> precondition, ThrowingBiFunction<STATE, SUT, STATE> step) { - steps.add((ref, entropySource) -> { + steps.add((ref, rng) -> { ref.map(state -> { if (!precondition.test(state.l)) return state; @@ -172,12 +172,12 @@ public ModelChecker<STATE, SUT> step(StatePrecondition<STATE> precondition, Thro public ModelChecker<STATE, SUT> step(ThrowingFunction<STATE, STATE> step) { - return step((t, sut, entropySource) -> new Pair<>(step.apply(t), sut)); + return step((t, sut, rng) -> new Pair<>(step.apply(t), sut)); } interface StepExecutor<STATE, SUT> { - void execute(Ref<Pair<STATE, SUT>> state, EntropySource entropySource) throws Throwable; + void execute(Ref<Pair<STATE, SUT>> state, EntropySource rng) throws Throwable; } public interface StatePrecondition<STATE> @@ -202,7 +202,7 @@ static <STATE, SUT> Precondition<STATE, SUT> alwaysTrue() public interface Step<STATE, SUT> { - Pair<STATE, SUT> next(STATE t, SUT sut, EntropySource entropySource) throws Throwable; + Pair<STATE, SUT> next(STATE t, SUT sut, EntropySource rng) throws Throwable; } public interface ThrowingConsumer<I> @@ -289,7 +289,7 @@ public Simple init(STATE state) @SuppressWarnings("unused") public Simple beforeAll(ThrowingConsumer<STATE> beforeAll) { - ModelChecker.this.beforeAll = (t, sut, entropySource) -> { + ModelChecker.this.beforeAll = (t, sut, rng) -> { beforeAll.accept(t); return Pair.unchanged(); }; @@ -298,8 +298,8 @@ public Simple beforeAll(ThrowingConsumer<STATE> beforeAll) public Simple beforeAll(ThrowingBiConsumer<STATE, EntropySource> beforeAll) { - ModelChecker.this.beforeAll = (t, sut, entropySource) -> { - beforeAll.accept(t, entropySource); + ModelChecker.this.beforeAll = (t, sut, rng) -> { + beforeAll.accept(t, rng); return Pair.unchanged(); }; return this; @@ -308,13 +308,13 @@ public Simple beforeAll(ThrowingBiConsumer<STATE, EntropySource> beforeAll) @SuppressWarnings("unused") public Simple beforeAll(ThrowingFunction<STATE, STATE> beforeAll) { - ModelChecker.this.beforeAll = (t, sut, entropySource) -> new Pair<>(beforeAll.apply(t), sut); + ModelChecker.this.beforeAll = (t, sut, rng) -> new Pair<>(beforeAll.apply(t), sut); return this; } public Simple afterAll(ThrowingConsumer<STATE> afterAll) { - ModelChecker.this.afterAll = (t, sut, entropySource) -> { + ModelChecker.this.afterAll = (t, sut, rng) -> { afterAll.accept(t); return Pair.unchanged(); }; @@ -324,7 +324,7 @@ public Simple afterAll(ThrowingConsumer<STATE> afterAll) @SuppressWarnings("unused") public Simple afterAll(ThrowingFunction<STATE, STATE> afterAll) { - ModelChecker.this.afterAll = (t, sut, entropySource) -> new Pair(afterAll.apply(t), sut); + ModelChecker.this.afterAll = (t, sut, rng) -> new Pair(afterAll.apply(t), sut); return this; } @@ -343,13 +343,13 @@ public Simple invariant(Predicate<STATE> invariant) public Simple step(ThrowingFunction<STATE, STATE> step) { - ModelChecker.this.step((state, sut, entropySource) -> new Pair<>(step.apply(state), sut)); + ModelChecker.this.step((state, sut, rng) -> new Pair<>(step.apply(state), sut)); return this; } public Simple step(ThrowingConsumer<STATE> step) { - ModelChecker.this.step((state, sut, entropySource) -> { + ModelChecker.this.step((state, sut, rng) -> { step.accept(state); return Pair.unchanged(); }); @@ -358,8 +358,8 @@ public Simple step(ThrowingConsumer<STATE> step) public Simple step(ThrowingBiConsumer<STATE, EntropySource> step) { - ModelChecker.this.step((state, sut, entropySource) -> { - step.accept(state, entropySource); + ModelChecker.this.step((state, sut, rng) -> { + step.accept(state, rng); return Pair.unchanged(); }); return this; @@ -367,11 +367,11 @@ public Simple step(ThrowingBiConsumer<STATE, EntropySource> step) public Simple step(BiPredicate<STATE, EntropySource> precondition, BiConsumer<STATE, EntropySource> step) { - ModelChecker.this.step((state, sut, entropySource) -> { - if (!precondition.test(state, entropySource)) + ModelChecker.this.step((state, sut, rng) -> { + if (!precondition.test(state, rng)) return Pair.unchanged(); - step.accept(state, entropySource); + step.accept(state, rng); return Pair.unchanged(); }); @@ -381,7 +381,7 @@ public Simple step(BiPredicate<STATE, EntropySource> precondition, BiConsumer<ST public Simple step(Predicate<STATE> precondition, Consumer<STATE> step) { ModelChecker.this.step((state, ignore) -> precondition.test(state), - (t, sut, entropySource) -> { + (t, sut, rng) -> { step.accept(t); return Pair.unchanged(); }); @@ -394,9 +394,15 @@ public void run() throws Throwable ModelChecker.this.run(); } - public void run(int minSteps, long maxSteps, EntropySource entropySource) throws Throwable + public void run(EntropySource rng) throws Throwable { - ModelChecker.this.run(minSteps, maxSteps, entropySource); + ModelChecker.this.run(0, Integer.MAX_VALUE, rng); + } + + public void run(int minSteps, long maxSteps, EntropySource rng) throws Throwable + { + ModelChecker.this.run(minSteps, maxSteps, rng); } } + } diff --git a/test/harry/main/org/apache/cassandra/harry/checker/TestHelper.java b/test/harry/main/org/apache/cassandra/harry/checker/TestHelper.java new file mode 100644 index 000000000000..f44b6e5fa2c0 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/checker/TestHelper.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.checker; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.distributed.test.ExecUtil; +import org.apache.cassandra.harry.gen.EntropySource; +import org.apache.cassandra.harry.gen.rng.JdkRandomEntropySource; + +public class TestHelper +{ + private static final Logger logger = LoggerFactory.getLogger(TestHelper.class); + + public static void withRandom(ModelChecker.ThrowingConsumer<EntropySource> rng) + { + withRandom(System.nanoTime(), rng); + } + + public static void withRandom(long seed, ModelChecker.ThrowingConsumer<EntropySource> rng) + { + try + { + logger.info("Seed: {}", seed); + rng.accept(new JdkRandomEntropySource(seed)); + } + catch (Throwable t) + { + throw new AssertionError(String.format("Caught an exception at seed:%dL", seed), t); + } + } + + public static void repeat(int num, ExecUtil.ThrowingSerializableRunnable<?> r) + { + for (int i = 0; i < num; i++) + { + try + { + r.run(); + } + catch (Throwable throwable) + { + throw new AssertionError(throwable); + } + } + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/clock/ApproximateClock.java b/test/harry/main/org/apache/cassandra/harry/clock/ApproximateClock.java deleted file mode 100644 index 5bb4aa00ef86..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/clock/ApproximateClock.java +++ /dev/null @@ -1,279 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.clock; - -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.LockSupport; - -import com.google.common.annotations.VisibleForTesting; - -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.model.OpSelectors; - -/** - * Monotonic clock, that guarantees that any LTS can be converted to a unique RTS, given - * the number of LTS does not exceed the number of RTS (e.g., we do not draw LTS more frequently - * than once per microsecond). - * <p> - * This conversion works as follows: - * <p> - * * every `period`, we record the current timestamp and maximum seen LTS, and keep history of up to - * `historySize` LTS/timestamp combinations - * * when queried to retrieve the LTS, we find a timestamp, relative to which we can calculate RTS. - * After that, we calculate a difference between the largest LTS that is still smaller than the converted - * one, and add this difference to the timestamp. - * <p> - * This way, later LTS can only be mapped to later RTS, and any LTS that was drawn previously, will be - * uniquely mapped relative to some timestamp, with the order matching the LTS order. - */ -public class ApproximateClock implements OpSelectors.Clock -{ - public static final long START_VALUE = 0; - public static final long DEFUNCT = Long.MIN_VALUE; - public static final long REBASE_IN_PROGRESS = Long.MIN_VALUE + 1; - - // TODO: there's a theoretical possibility of a bug; when we have several consecutive epochs without - // change in LTS, current implementation will return the latest epoch instead of the earliest one. - // This is not a big deal in terms of monotonicity but can cause some problems when validating TTL. - // The simples fix would be to find the smallest matching epoch. - private final ScheduledExecutorService executor; - private final int historySize; - private final CopyOnWriteArrayList<Long> ltsHistory; - private final long startTimeMicros; - private volatile int idx; - private final AtomicLong lts; - - private final long periodMicros; - - private final long epoch; - private final TimeUnit epochTimeUnit; - - public ApproximateClock(long period, TimeUnit timeUnit) - { - this(10000, period, timeUnit); - } - - public ApproximateClock(int historySize, long epoch, TimeUnit epochTimeUnit) - { - this(TimeUnit.MILLISECONDS.toMicros(System.currentTimeMillis()), - historySize, new CopyOnWriteArrayList<>(), START_VALUE, 0, epoch, epochTimeUnit); - rebase(); - } - - ApproximateClock(long startTimeMicros, - int historySize, - CopyOnWriteArrayList<Long> history, - long lts, - int idx, - long epoch, - TimeUnit epochTimeUnit) - { - this.startTimeMicros = startTimeMicros; - this.historySize = historySize; - this.ltsHistory = history; - this.lts = new AtomicLong(lts); - this.idx = idx; - this.periodMicros = epochTimeUnit.toMicros(epoch); - this.executor = Executors.newSingleThreadScheduledExecutor(r -> { - Thread t = new Thread(r); - t.setName("ApproximateMonotonicClock-ScheduledTasks"); - t.setDaemon(true); - return t; - }); - this.executor.scheduleAtFixedRate(this::rebase, epoch, epoch, epochTimeUnit); - this.epoch = epoch; - this.epochTimeUnit = epochTimeUnit; - } - - @VisibleForTesting - public static ApproximateClock forDebug(long startTimeMicros, int historySize, long lts, int idx, long period, TimeUnit timeUnit, long... values) - { - CopyOnWriteArrayList<Long> history = new CopyOnWriteArrayList<>(); - for (int i = 0; i < values.length; i++) - history.set(i, values[i]); - - assert values.length == idx; // sanity check - return new ApproximateClock(startTimeMicros, historySize, history, lts, idx, period, timeUnit); - } - - public long get(long idx) - { - return ltsHistory.get((int) (idx % historySize)); - } - - private void rebase() - { - int arrayIdx = idx % historySize; - long rebaseLts = lts.get(); - if (rebaseLts == DEFUNCT) - throw new IllegalStateException(); - - while (!lts.compareAndSet(rebaseLts, REBASE_IN_PROGRESS)) - rebaseLts = lts.get(); - - ltsHistory.add(arrayIdx, rebaseLts == START_VALUE ? START_VALUE : (rebaseLts + 1)); - - // If we happen to exhaust counter, we just need to make operations "wider". - // It is unsafe to proceed, so we defunct the clock. - // - // We could make a clock implementation that would sleep on `get`, but it will - // be more expensive, since we'll have to check for overflow each time before - // returning anything. - if (idx > 1 && get(idx) - get(idx - 1) > periodMicros) - { - lts.set(DEFUNCT); - executor.shutdown(); - throwCounterExhaustedException(); - } - - idx = idx + 1; - if (!lts.compareAndSet(REBASE_IN_PROGRESS, rebaseLts)) - throw new IllegalStateException("No thread should have changed LTS during rebase. " + lts.get()); - } - - @Override - public long nextLts() - { - long current = lts.get(); - while (true) - { - if (current >= 0) - { - if (lts.compareAndSet(current, current + 1)) - return current; - - current = lts.get(); - continue; - } - - if (current == REBASE_IN_PROGRESS) - { - LockSupport.parkNanos(1); - current = lts.get(); - continue; - } - - if (current == DEFUNCT) - throwCounterExhaustedException(); - - throw new IllegalStateException("This should have been unreachable: " + current); - } - } - - public long peek() - { - while (true) - { - long ret = lts.get(); - - if (ret == REBASE_IN_PROGRESS) - { - LockSupport.parkNanos(1); - continue; - } - - if (ret == DEFUNCT) - throwCounterExhaustedException(); - - return ret; - } - } - - public Configuration.ClockConfiguration toConfig() - { - int idx = this.idx; - long[] history = new long[Math.min(idx, historySize)]; - for (int i = 0; i < history.length; i++) - history[i] = ltsHistory.get(i); - return new Configuration.DebugApproximateClockConfiguration(startTimeMicros, - ltsHistory.size(), - history, - lts.get(), - idx, - epoch, - epochTimeUnit); - } - - public long lts(final long rts) - { - final int historyIdx = idx - 1; - for (int i = 0; i < historySize - 1 && historyIdx - i >= 0; i++) - { - long periodStartRts = startTimeMicros + periodMicros * (historyIdx - i); - if (rts >= periodStartRts) - { - long periodStartLts = get(historyIdx - i); - return periodStartLts + rts - periodStartRts; - } - } - throw new IllegalStateException("RTS is too old to convert to LTS: " + rts + "\n " + ltsHistory); - } - - public long rts(final long lts) - { - assert lts <= peek() : String.format("Queried for LTS we haven't yet issued %d. Max is %d.", lts, peek()); - - final int historyIdx = idx - 1; - for (int i = 0; i < historySize - 1 && historyIdx - i >= 0; i++) - { - long periodStartLts = get(historyIdx - i); - if (lts >= periodStartLts) - { - long periodStartRts = startTimeMicros + periodMicros * (historyIdx - i); - return periodStartRts + lts - periodStartLts; - } - } - - throw new IllegalStateException("LTS is too old to convert to RTS: " + lts + "\n " + dumpHistory()); - } - - private String dumpHistory() - { - String s = ""; - int idx = this.idx; - for (int i = 0; i < Math.min(idx, historySize); i++) - { - s += ltsHistory.get(i) + ","; - } - return s.substring(0, Math.max(0, s.length() - 1)); - } - - public String toString() - { - return String.format("withDebugClock(%dL,\n\t%d,\n\t%d,\n\t%d,\n\t%d,\n\t%s,\n\t%s)", - startTimeMicros, - historySize, - lts.get(), - idx, - epoch, - epochTimeUnit, - dumpHistory()); - } - - private void throwCounterExhaustedException() - { - long diff = get(idx) - get(idx - 1); - throw new RuntimeException(String.format("Counter was exhausted. Drawn %d out of %d lts during the period.", - diff, periodMicros)); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/clock/OffsetClock.java b/test/harry/main/org/apache/cassandra/harry/clock/OffsetClock.java deleted file mode 100644 index d12948cf1e42..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/clock/OffsetClock.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.clock; - -import java.util.concurrent.atomic.AtomicLong; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.model.OpSelectors; - -/** - * A trivial implemementation of the clock, one that does not attempt to follow the wall clock. - * This clock simply offsets LTS by a preset value. - */ -public class OffsetClock implements OpSelectors.Clock -{ - final AtomicLong lts; - - private final long base; - - public OffsetClock(long base) - { - this(ApproximateClock.START_VALUE, base); - } - - public OffsetClock(long startValue, long base) - { - this.lts = new AtomicLong(startValue); - this.base = base; - } - - public long rts(long lts) - { - return base + lts; - } - - public long lts(long rts) - { - return rts - base; - } - - public long nextLts() - { - return lts.getAndIncrement(); - } - - public long peek() - { - return lts.get(); - } - - public Configuration.ClockConfiguration toConfig() - { - return new OffsetClockConfiguration(lts.get(), base); - } - - @JsonTypeName("offset") - public static class OffsetClockConfiguration implements Configuration.ClockConfiguration - { - public final long offset; - public final long base; - - @JsonCreator - public OffsetClockConfiguration(@JsonProperty("offset") long offset, - @JsonProperty(value = "base", defaultValue = "0") long base) - { - this.offset = offset; - this.base = base; - } - - public OpSelectors.Clock make() - { - return new OffsetClock(base, offset); - } - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/core/Configuration.java b/test/harry/main/org/apache/cassandra/harry/core/Configuration.java deleted file mode 100644 index 47e1d3bb5d7b..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/core/Configuration.java +++ /dev/null @@ -1,1255 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.core; - -import java.io.BufferedWriter; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStreamWriter; -import java.lang.annotation.Annotation; -import java.util.*; -import java.util.concurrent.TimeUnit; -import java.util.function.Consumer; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.fasterxml.jackson.annotation.JsonTypeName; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; -import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator; -import org.apache.cassandra.harry.ddl.SchemaGenerators; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.gen.Surjections; -import org.apache.cassandra.harry.gen.distribution.Distribution; -import org.apache.cassandra.harry.model.Model; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.model.NoOpChecker; -import org.apache.cassandra.harry.model.QuiescentChecker; -import org.apache.cassandra.harry.clock.ApproximateClock; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.tracker.LockingDataTracker; -import org.apache.cassandra.harry.tracker.DataTracker; -import org.apache.cassandra.harry.tracker.DefaultDataTracker; -import org.apache.cassandra.harry.runner.Runner; -import org.apache.cassandra.harry.util.BitSet; -import org.apache.cassandra.harry.visitors.AllPartitionsValidator; -import org.apache.cassandra.harry.visitors.CorruptingVisitor; -import org.apache.cassandra.harry.visitors.LoggingVisitor; -import org.apache.cassandra.harry.visitors.MutatingRowVisitor; -import org.apache.cassandra.harry.visitors.MutatingVisitor; -import org.apache.cassandra.harry.visitors.OperationExecutor; -import org.apache.cassandra.harry.visitors.QueryLogger; -import org.apache.cassandra.harry.visitors.RandomValidator; -import org.apache.cassandra.harry.visitors.RecentValidator; -import org.apache.cassandra.harry.visitors.Visitor; -import org.reflections.Reflections; -import org.reflections.scanners.Scanners; -import org.reflections.util.NameHelper; - -public class Configuration -{ - private enum NameUtils implements NameHelper - { - INSTANCE; - } - - private static final ObjectMapper mapper; - - private static <A extends Annotation> Set<Class<?>> findClassesMarkedWith(Class<A> annotation) - { - Reflections reflections = new Reflections(org.reflections.util.ConfigurationBuilder.build("harry").setExpandSuperTypes(false)); - Collection<Class<?>> classes = NameUtils.INSTANCE.forNames(reflections.get(Scanners.TypesAnnotated.get(annotation.getName())), - reflections.getConfiguration().getClassLoaders()); - return new HashSet<>(classes); - } - - static - { - mapper = new ObjectMapper(new YAMLFactory() // checkstyle: permit this instantiation - .disable(YAMLGenerator.Feature.USE_NATIVE_TYPE_ID) - .disable(YAMLGenerator.Feature.WRITE_DOC_START_MARKER) - .disable(YAMLGenerator.Feature.CANONICAL_OUTPUT) - .enable(YAMLGenerator.Feature.INDENT_ARRAYS)); - - findClassesMarkedWith(JsonTypeName.class) - .forEach(mapper::registerSubtypes); - } - - public final long seed; - public final SchemaProviderConfiguration schema_provider; - - public final boolean drop_schema; - public final String keyspace_ddl; - public final boolean create_schema; - public final boolean truncate_table; - - public final MetricReporterConfiguration metric_reporter; - public final ClockConfiguration clock; - public final SutConfiguration system_under_test; - public final DataTrackerConfiguration data_tracker; - public final RunnerConfiguration runner; - public final PDSelectorConfiguration partition_descriptor_selector; - public final CDSelectorConfiguration clustering_descriptor_selector; - - @JsonCreator - public Configuration(@JsonProperty("seed") long seed, - @JsonProperty("schema_provider") SchemaProviderConfiguration schema_provider, - @JsonProperty("drop_schema") boolean drop_schema, - @JsonProperty("create_keyspace") String keyspace_ddl, - @JsonProperty("create_schema") boolean create_schema, - @JsonProperty("truncate_schema") boolean truncate_table, - @JsonProperty("metric_reporter") MetricReporterConfiguration metric_reporter, - @JsonProperty("clock") ClockConfiguration clock, - @JsonProperty("runner") RunnerConfiguration runner, - @JsonProperty("system_under_test") SutConfiguration system_under_test, - @JsonProperty("data_tracker") DataTrackerConfiguration data_tracker, - @JsonProperty("partition_descriptor_selector") PDSelectorConfiguration partition_descriptor_selector, - @JsonProperty("clustering_descriptor_selector") CDSelectorConfiguration clustering_descriptor_selector) - { - this.seed = seed; - this.schema_provider = schema_provider; - this.keyspace_ddl = keyspace_ddl; - this.drop_schema = drop_schema; - this.create_schema = create_schema; - this.truncate_table = truncate_table; - this.metric_reporter = metric_reporter; - this.clock = clock; - this.system_under_test = system_under_test; - this.data_tracker = data_tracker; - this.partition_descriptor_selector = partition_descriptor_selector; - this.clustering_descriptor_selector = clustering_descriptor_selector; - this.runner = runner; - } - - public static void registerSubtypes(Class<?>... classes) - { - mapper.registerSubtypes(classes); - } - - public static void toFile(File file, Configuration config) - { - try (BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(new FileOutputStream(file)))) - { - bw.write(Configuration.toYamlString(config)); - bw.flush(); - } - catch (IOException e) - { - throw new RuntimeException(e); - } - } - public static String toYamlString(Configuration config) - { - try - { - return mapper.writeValueAsString(config); - } - catch (Throwable t) - { - throw new RuntimeException(t); - } - } - - public static Configuration fromYamlString(String config) - { - try - { - return mapper.readValue(config, Configuration.class); - } - catch (Throwable t) - { - throw new RuntimeException(t); - } - } - - public static Configuration fromFile(String path) - { - return fromFile(new File(path)); - } - - public static Configuration fromFile(File file) - { - try - { - return mapper.readValue(file, Configuration.class); - } - catch (Throwable t) - { - throw new RuntimeException(t); - } - } - - public static void validate(Configuration config) - { - Objects.requireNonNull(config.schema_provider, "Schema provider should not be null"); - Objects.requireNonNull(config.metric_reporter, "Metric reporter should not be null"); - Objects.requireNonNull(config.clock, "Clock should not be null"); - Objects.requireNonNull(config.system_under_test, "System under test should not be null"); - Objects.requireNonNull(config.partition_descriptor_selector, "Partition descriptor selector should not be null"); - Objects.requireNonNull(config.clustering_descriptor_selector, "Clustering descriptor selector should not be null"); - } - - public Runner createRunner() - { - return createRunner(this); - } - - public Run createRun() - { - return createRun(this); - } - - public static Run createRun(Configuration snapshot) - { - SystemUnderTest sut = null; - try - { - validate(snapshot); - - long seed = snapshot.seed; - - OpSelectors.PureRng rng = new OpSelectors.PCGFast(seed); - - // TODO: validate that operation kind is compatible with schema, due to statics etc - sut = snapshot.system_under_test.make(); - SchemaSpec schemaSpec = snapshot.schema_provider.make(seed, sut); - schemaSpec.validate(); - - OpSelectors.PdSelector pdSelector = snapshot.partition_descriptor_selector.make(rng); - DataTrackerConfiguration dataTrackerConfiguration = snapshot.data_tracker == null ? new DefaultDataTrackerConfiguration() : snapshot.data_tracker; - DataTracker tracker = dataTrackerConfiguration.make(pdSelector, schemaSpec); - - OpSelectors.DescriptorSelector descriptorSelector = snapshot.clustering_descriptor_selector.make(rng, schemaSpec); - OpSelectors.Clock clock = snapshot.clock.make(); - - MetricReporter metricReporter = snapshot.metric_reporter.make(); - - return new Run(rng, - clock, - pdSelector, - descriptorSelector, - schemaSpec, - tracker, - sut, - metricReporter); - } - catch (Throwable t) - { - // Make sure to shut down all SUT threads if it has been started - if (sut != null) - { - sut.shutdown(); - } - throw t; - } - } - - public static Runner createRunner(Configuration config) - { - Run run = createRun(config); - return config.runner.make(run, config); - } - - public static class ConfigurationBuilder - { - long seed; - SchemaProviderConfiguration schema_provider = new DefaultSchemaProviderConfiguration(); - - String keyspace_ddl; - boolean drop_schema; - boolean create_schema; - boolean truncate_table; - - ClockConfiguration clock; - MetricReporterConfiguration metric_reporter = new NoOpMetricReporterConfiguration(); - DataTrackerConfiguration data_tracker = new DefaultDataTrackerConfiguration(); - RunnerConfiguration runner; - SutConfiguration system_under_test; - PDSelectorConfiguration partition_descriptor_selector = new Configuration.DefaultPDSelectorConfiguration(10, 100); - CDSelectorConfiguration clustering_descriptor_selector; // TODO: sensible default value - - public ConfigurationBuilder setSeed(long seed) - { - this.seed = seed; - return this; - } - - public ConfigurationBuilder setSchemaProvider(SchemaProviderConfiguration schema_provider) - { - this.schema_provider = schema_provider; - return this; - } - - public ConfigurationBuilder setDataTracker(DataTrackerConfiguration tracker) - { - this.data_tracker = tracker; - return this; - } - - public ConfigurationBuilder setKeyspaceDdl(String keyspace_ddl) - { - this.keyspace_ddl = keyspace_ddl; - return this; - } - - - public ConfigurationBuilder setClock(ClockConfiguration clock) - { - this.clock = clock; - return this; - } - - public ConfigurationBuilder setSUT(SutConfiguration system_under_test) - { - this.system_under_test = system_under_test; - return this; - } - - public ConfigurationBuilder setDropSchema(boolean drop_schema) - { - this.drop_schema = drop_schema; - return this; - } - - public ConfigurationBuilder setCreateSchema(boolean create_schema) - { - this.create_schema = create_schema; - return this; - } - - public ConfigurationBuilder setTruncateTable(boolean truncate_table) - { - this.truncate_table = truncate_table; - return this; - } - - public ConfigurationBuilder setRunner(RunnerConfiguration runner) - { - this.runner = runner; - return this; - } - - public ConfigurationBuilder setPartitionDescriptorSelector(PDSelectorConfiguration partition_descriptor_selector) - { - this.partition_descriptor_selector = partition_descriptor_selector; - return this; - } - - public ConfigurationBuilder setClusteringDescriptorSelector(CDSelectorConfiguration builder) - { - this.clustering_descriptor_selector = builder; - return this; - } - - public ConfigurationBuilder setClusteringDescriptorSelector(Consumer<CDSelectorConfigurationBuilder> build) - { - CDSelectorConfigurationBuilder builder = new CDSelectorConfigurationBuilder(); - build.accept(builder); - return setClusteringDescriptorSelector(builder.build()); - } - - public ConfigurationBuilder setMetricReporter(MetricReporterConfiguration metric_reporter) - { - this.metric_reporter = metric_reporter; - return this; - } - - public Configuration build() - { - return new Configuration(seed, - schema_provider, - drop_schema, - keyspace_ddl, - create_schema, - truncate_table, - metric_reporter, - clock, - runner, - system_under_test, - data_tracker, - partition_descriptor_selector, - clustering_descriptor_selector); - } - } - - public ConfigurationBuilder unbuild() - { - ConfigurationBuilder builder = new ConfigurationBuilder(); - builder.seed = seed; - - builder.schema_provider = schema_provider; - builder.keyspace_ddl = keyspace_ddl; - builder.drop_schema = drop_schema; - builder.create_schema = create_schema; - builder.truncate_table = truncate_table; - - builder.data_tracker = data_tracker; - builder.clock = clock; - builder.runner = runner; - builder.system_under_test = system_under_test; - builder.metric_reporter = metric_reporter; - - builder.partition_descriptor_selector = partition_descriptor_selector; - builder.clustering_descriptor_selector = clustering_descriptor_selector; - - return builder; - } - - - @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.WRAPPER_OBJECT) - public interface DataTrackerConfiguration extends DataTracker.DataTrackerFactory - { - - } - - @JsonTypeName("no_op") - public static class NoOpDataTrackerConfiguration implements DataTrackerConfiguration - { - @JsonCreator - public NoOpDataTrackerConfiguration() - { - } - - public DataTracker make(OpSelectors.PdSelector pdSelector, SchemaSpec schemaSpec) - { - return DataTracker.NO_OP; - } - } - - @JsonTypeName("default") - public static class DefaultDataTrackerConfiguration implements DataTrackerConfiguration - { - public final long max_seen_lts; - public final long max_complete_lts; - public final List<Long> reorder_buffer; - - public DefaultDataTrackerConfiguration() - { - this(-1, -1, null); - } - - @JsonCreator - public DefaultDataTrackerConfiguration(@JsonProperty(value = "max_seen_lts", defaultValue = "-1") long max_seen_lts, - @JsonProperty(value = "max_complete_lts", defaultValue = "-1") long max_complete_lts, - @JsonProperty(value = "reorder_buffer", defaultValue = "null") List<Long> reorder_buffer) - { - this.max_seen_lts = max_seen_lts; - this.max_complete_lts = max_complete_lts; - this.reorder_buffer = reorder_buffer; - } - - @Override - public DataTracker make(OpSelectors.PdSelector pdSelector, SchemaSpec schemaSpec) - { - DefaultDataTracker tracker = new DefaultDataTracker(); - tracker.forceLts(max_seen_lts, max_complete_lts, reorder_buffer); - return tracker; - } - } - - @JsonTypeName("locking") - public static class LockingDataTrackerConfiguration implements DataTrackerConfiguration - { - public final long max_seen_lts; - public final long max_complete_lts; - public final List<Long> reorder_buffer; - - @JsonCreator - public LockingDataTrackerConfiguration(@JsonProperty(value = "max_seen_lts", defaultValue = "-1") long max_seen_lts, - @JsonProperty(value = "max_complete_lts", defaultValue = "-1") long max_complete_lts, - @JsonProperty(value = "reorder_buffer", defaultValue = "null") List<Long> reorder_buffer) - { - this.max_seen_lts = max_seen_lts; - this.max_complete_lts = max_complete_lts; - this.reorder_buffer = reorder_buffer; - } - - @Override - public DataTracker make(OpSelectors.PdSelector pdSelector, SchemaSpec schemaSpec) - { - LockingDataTracker tracker = new LockingDataTracker(pdSelector, schemaSpec); - tracker.forceLts(max_seen_lts, max_complete_lts, reorder_buffer); - return tracker; - } - } - - public static class DefaultLockingDataTrackerConfiguration implements DataTrackerConfiguration - { - @Override - public DataTracker make(OpSelectors.PdSelector pdSelector, SchemaSpec schemaSpec) - { - LockingDataTracker tracker = new LockingDataTracker(pdSelector, schemaSpec); - tracker.forceLts(-1, -1, null); - return tracker; - } - } - - @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.WRAPPER_OBJECT) - public interface ClockConfiguration extends OpSelectors.ClockFactory - { - } - - @JsonTypeName("approximate_monotonic") - public static class ApproximateClockConfiguration implements ClockConfiguration - { - public final int history_size; - public final int epoch_length; - public final TimeUnit epoch_time_unit; - - @JsonCreator - public ApproximateClockConfiguration(@JsonProperty("history_size") int history_size, - @JsonProperty("epoch_length") int epoch_length, - @JsonProperty("epoch_time_unit") TimeUnit epoch_time_unit) - { - this.history_size = history_size; - this.epoch_length = epoch_length; - this.epoch_time_unit = epoch_time_unit; - } - - public OpSelectors.Clock make() - { - return new ApproximateClock(history_size, - epoch_length, - epoch_time_unit); - } - } - - @JsonTypeName("debug_approximate_monotonic") - public static class DebugApproximateClockConfiguration implements ClockConfiguration - { - public final long start_time_micros; - public final int history_size; - public final long[] history; - public final long lts; - public final int idx; - public final long epoch_period; - public final TimeUnit epoch_time_unit; - - @JsonCreator - public DebugApproximateClockConfiguration(@JsonProperty("start_time_micros") long start_time_micros, - @JsonProperty("history_size") int history_size, - @JsonProperty("history") long[] history, - @JsonProperty("lts") long lts, - @JsonProperty("idx") int idx, - @JsonProperty("epoch_period") long epoch_period, - @JsonProperty("epoch_time_unit") TimeUnit epoch_time_unit) - { - this.start_time_micros = start_time_micros; - this.history_size = history_size; - this.history = history; - this.lts = lts; - this.idx = idx; - this.epoch_period = epoch_period; - this.epoch_time_unit = epoch_time_unit; - } - - public OpSelectors.Clock make() - { - return ApproximateClock.forDebug(start_time_micros, - history_size, - lts, - idx, - epoch_period, - epoch_time_unit, - history); - } - } - - @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.WRAPPER_OBJECT) - public interface RunnerConfiguration extends Runner.RunnerFactory - { - } - - @JsonTypeName("concurrent") - public static class ConcurrentRunnerConfig implements RunnerConfiguration - { - @JsonProperty(value = "visitor_pools") - public final List<VisitorPoolConfiguration> visitor_pools; - - public final long run_time; - public final TimeUnit run_time_unit; - - @JsonCreator - public ConcurrentRunnerConfig(@JsonProperty(value = "visitor_pools") List<VisitorPoolConfiguration> visitor_pools, - @JsonProperty(value = "run_time", defaultValue = "2") long runtime, - @JsonProperty(value = "run_time_unit", defaultValue = "HOURS") TimeUnit runtimeUnit) - { - this.visitor_pools = visitor_pools; - this.run_time = runtime; - this.run_time_unit = runtimeUnit; - } - - @Override - public Runner make(Run run, Configuration config) - { - return new Runner.ConcurrentRunner(run, config, visitor_pools, run_time, run_time_unit); - } - } - - public static class VisitorPoolConfiguration - { - public final String prefix; - public final int concurrency; - public final VisitorConfiguration visitor; - - @JsonCreator - public VisitorPoolConfiguration(@JsonProperty(value = "prefix") String prefix, - @JsonProperty(value = "concurrency") int concurrency, - @JsonProperty(value = "visitor") VisitorConfiguration visitor) - { - this.prefix = prefix; - this.concurrency = concurrency; - this.visitor = visitor; - } - - public static VisitorPoolConfiguration pool(String prefix, int concurrency, VisitorConfiguration visitor) - { - return new VisitorPoolConfiguration(prefix, concurrency, visitor); - } - } - - @JsonTypeName("sequential") - public static class SequentialRunnerConfig implements RunnerConfiguration - { - @JsonProperty(value = "visitors") - public final List<VisitorConfiguration> visitorFactories; - - public final long run_time; - public final TimeUnit run_time_unit; - - @JsonCreator - public SequentialRunnerConfig(@JsonProperty(value = "visitors") List<VisitorConfiguration> visitors, - @JsonProperty(value = "run_time", defaultValue = "2") long runtime, - @JsonProperty(value = "run_time_unit", defaultValue = "HOURS") TimeUnit runtimeUnit) - { - this.visitorFactories = visitors; - this.run_time = runtime; - this.run_time_unit = runtimeUnit; - } - - @Override - public Runner make(Run run, Configuration config) - { - return new Runner.SequentialRunner(run, config, visitorFactories, run_time, run_time_unit); - } - } - - @JsonTypeName("single") - public static class SingleVisitRunnerConfig implements RunnerConfiguration - { - @JsonProperty(value = "visitors") - public final List<VisitorConfiguration> visitorFactories; - - @JsonCreator - public SingleVisitRunnerConfig(@JsonProperty(value = "visitors") List<VisitorConfiguration> visitors) - { - this.visitorFactories = visitors; - } - - @Override - public Runner make(Run run, Configuration config) - { - return new Runner.SingleVisitRunner(run, config, visitorFactories); - } - } - - @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.WRAPPER_OBJECT) - public interface SutConfiguration extends SystemUnderTest.SUTFactory - { - } - - @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.WRAPPER_OBJECT) - public interface ModelConfiguration extends Model.ModelFactory - { - } - - @JsonTypeName("quiescent_checker") - public static class QuiescentCheckerConfig implements ModelConfiguration - { - @JsonCreator - public QuiescentCheckerConfig() - { - } - - public Model make(Run run) - { - return new QuiescentChecker(run); - } - } - - @JsonTypeName("no_op") - public static class NoOpCheckerConfig implements ModelConfiguration - { - @JsonCreator - public NoOpCheckerConfig() - { - } - - public Model make(Run run) - { - return new NoOpChecker(run); - } - } - - - @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.WRAPPER_OBJECT) - public interface PDSelectorConfiguration extends OpSelectors.PdSelectorFactory - { - } - - @JsonTypeName("default") - public static class DefaultPDSelectorConfiguration implements PDSelectorConfiguration - { - public final int window_size; - public final int slide_after_repeats; - public final long position_offset; - public final long position_window_size; - - public DefaultPDSelectorConfiguration(int window_size, - int slide_after_repeats) - { - this.window_size = window_size; - this.slide_after_repeats = slide_after_repeats; - this.position_offset = 0L; - this.position_window_size = Long.MAX_VALUE; - } - - @JsonCreator - public DefaultPDSelectorConfiguration(@JsonProperty(value = "window_size", defaultValue = "10") int window_size, - @JsonProperty(value = "slide_after_repeats", defaultValue = "100") int slide_after_repeats, - @JsonProperty(value = "runner_index") Long runner_index, - @JsonProperty(value = "total_runners") Long total_runners, - @JsonProperty(value = "position_offset") Long position_offset, - @JsonProperty(value = "position_window_size") Long position_window_size) - { - this.window_size = window_size; - this.slide_after_repeats = slide_after_repeats; - if (runner_index != null || total_runners != null) - { - assert runner_index != null && total_runners != null : "Both runner_index and total_runners are required"; - assert position_offset == null && position_window_size == null : "Please use either runner_index/total_runners or position_offset/position_window_size combinations."; - this.position_window_size = Long.MAX_VALUE / total_runners; - this.position_offset = this.position_window_size * runner_index; - } - else - { - assert runner_index == null && total_runners == null : "Please use either runner_index/total_runners or position_offset/position_window_size combinations."; - this.position_offset = position_offset == null ? 0 : position_offset; - if (position_window_size == null) - this.position_window_size = Long.MAX_VALUE - this.position_offset; - else - this.position_window_size = position_window_size; - } - } - - public OpSelectors.PdSelector make(OpSelectors.PureRng rng) - { - return new OpSelectors.DefaultPdSelector(rng, window_size, slide_after_repeats, position_offset, position_window_size); - } - } - - @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.WRAPPER_OBJECT) - public interface CDSelectorConfiguration extends OpSelectors.DescriptorSelectorFactory - { - } - - public static class WeightedSelectorBuilder<T> - { - private final Map<T, Integer> operation_kind_weights; - - public WeightedSelectorBuilder() - { - operation_kind_weights = new HashMap<>(); - } - - public WeightedSelectorBuilder<T> addWeight(T v, int weight) - { - operation_kind_weights.put(v, weight); - return this; - } - - public Map<T, Integer> build() - { - return operation_kind_weights; - } - } - - public static class OperationKindSelectorBuilder extends WeightedSelectorBuilder<OpSelectors.OperationKind> - { - } - - // TODO: configure fractions/fractional builder - public static class CDSelectorConfigurationBuilder - { - private DistributionConfig operations_per_lts = new ConstantDistributionConfig(10); - private int max_partition_size = 100; - private Map<OpSelectors.OperationKind, Integer> operation_kind_weights = new OperationKindSelectorBuilder() - .addWeight(OpSelectors.OperationKind.DELETE_ROW, 1) - .addWeight(OpSelectors.OperationKind.DELETE_COLUMN, 1) - .addWeight(OpSelectors.OperationKind.INSERT, 98) - .build(); - private Map<OpSelectors.OperationKind, long[]> column_mask_bitsets; - private int[] fractions; - - public CDSelectorConfigurationBuilder setOperationsPerLtsDistribution(DistributionConfig operations_per_lts) - { - this.operations_per_lts = operations_per_lts; - return this; - } - - public CDSelectorConfigurationBuilder setMaxPartitionSize(int max_partition_size) - { - if (max_partition_size <= 0) - throw new IllegalArgumentException("Max partition size should be positive"); - this.max_partition_size = max_partition_size; - return this; - } - - public CDSelectorConfigurationBuilder setOperationKindWeights(Map<OpSelectors.OperationKind, Integer> operation_kind_weights) - { - this.operation_kind_weights = operation_kind_weights; - return this; - } - - public CDSelectorConfigurationBuilder setColumnMasks(Map<OpSelectors.OperationKind, long[]> column_mask_bitsets) - { - this.column_mask_bitsets = column_mask_bitsets; - return this; - } - - public CDSelectorConfigurationBuilder setFractions(int[] fractions) - { - this.fractions = fractions; - return this; - } - - public DefaultCDSelectorConfiguration build() - { - if (fractions == null) - { - return new DefaultCDSelectorConfiguration(operations_per_lts, - max_partition_size, - operation_kind_weights, - column_mask_bitsets); - } - else - { - return new HierarchicalCDSelectorConfiguration(operations_per_lts, - max_partition_size, - operation_kind_weights, - column_mask_bitsets, - fractions); - } - } - } - - @JsonTypeName("default") - public static class DefaultCDSelectorConfiguration implements CDSelectorConfiguration - { - public final DistributionConfig operations_per_lts; - public final int max_partition_size; - public final Map<OpSelectors.OperationKind, Integer> operation_kind_weights; - public final Map<OpSelectors.OperationKind, long[]> column_mask_bitsets; - - @JsonCreator - public DefaultCDSelectorConfiguration(@JsonProperty("operations_per_lts") DistributionConfig operations_per_lts, - @JsonProperty(value = "window_size", defaultValue = "100") int max_partition_size, - @JsonProperty("operation_kind_weights") Map<OpSelectors.OperationKind, Integer> operation_kind_weights, - @JsonProperty("column_mask_bitsets") Map<OpSelectors.OperationKind, long[]> column_mask_bitsets) - { - this.operations_per_lts = operations_per_lts; - this.max_partition_size = max_partition_size; - this.operation_kind_weights = operation_kind_weights; - this.column_mask_bitsets = column_mask_bitsets; - } - - protected OpSelectors.ColumnSelector columnSelector(SchemaSpec schemaSpec) - { - OpSelectors.ColumnSelector columnSelector; - if (column_mask_bitsets == null) - { - columnSelector = OpSelectors.columnSelectorBuilder().forAll(schemaSpec).build(); - } - else - { - Map<OpSelectors.OperationKind, Surjections.Surjection<BitSet>> m = new EnumMap<>(OpSelectors.OperationKind.class); - for (Map.Entry<OpSelectors.OperationKind, long[]> entry : column_mask_bitsets.entrySet()) - { - List<BitSet> bitSets = new ArrayList<>(entry.getValue().length); - for (long raw_bitset : entry.getValue()) - { - bitSets.add(BitSet.create(raw_bitset, schemaSpec.allColumns.size())); - } - Surjections.Surjection<BitSet> selector = Surjections.pick(bitSets); - m.put(entry.getKey(), selector); - } - columnSelector = (opKind, descr) -> m.get(opKind).inflate(descr); - } - - return columnSelector; - } - - public OpSelectors.DescriptorSelector make(OpSelectors.PureRng rng, SchemaSpec schemaSpec) - { - return new OpSelectors.DefaultDescriptorSelector(rng, - columnSelector(schemaSpec), - OpSelectors.OperationSelector.weighted(operation_kind_weights), - operations_per_lts.make(), - max_partition_size); - } - } - - public static class HierarchicalCDSelectorConfiguration extends DefaultCDSelectorConfiguration - { - private final int[] fractions; - - public HierarchicalCDSelectorConfiguration(DistributionConfig operations_per_lts, - int max_partition_size, - Map<OpSelectors.OperationKind, Integer> operation_kind_weights, - Map<OpSelectors.OperationKind, long[]> column_mask_bitsets, - int[] fractions) - { - super(operations_per_lts, max_partition_size, operation_kind_weights, column_mask_bitsets); - this.fractions = fractions; - } - - public OpSelectors.DescriptorSelector make(OpSelectors.PureRng rng, SchemaSpec schemaSpec) - { - return new OpSelectors.HierarchicalDescriptorSelector(rng, - fractions, - columnSelector(schemaSpec), - OpSelectors.OperationSelector.weighted(operation_kind_weights), - operations_per_lts.make(), - max_partition_size); - } - } - - @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") - public interface DistributionConfig extends Distribution.DistributionFactory - { - } - - @JsonTypeName("identity") - public static class IdentityDistributionConfig implements DistributionConfig - { - @JsonCreator - public IdentityDistributionConfig() - { - } - - public Distribution make() - { - return new Distribution.IdentityDistribution(); - } - } - - @JsonTypeName("normal") - public static class NormalDistributionConfig implements DistributionConfig - { - @JsonCreator - public NormalDistributionConfig() - { - } - - public Distribution make() - { - return new Distribution.NormalDistribution(); - } - } - - @JsonTypeName("constant") - public static class ConstantDistributionConfig implements DistributionConfig - { - public final long constant; - - @JsonCreator - public ConstantDistributionConfig(@JsonProperty("constant") long constant) - { - this.constant = constant; - } - - public Distribution make() - { - return new Distribution.ConstantDistribution(constant); - } - } - - @JsonTypeName("scaled") - public static class ScaledDistributionConfig implements DistributionConfig - { - private final long min; - private final long max; - - @JsonCreator - public ScaledDistributionConfig(long min, long max) - { - this.min = min; - this.max = max; - } - - public Distribution make() - { - return new Distribution.ScaledDistribution(min, max); - } - } - - - @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.WRAPPER_OBJECT) - public interface VisitorConfiguration extends Visitor.VisitorFactory - { - } - - - @JsonTypeName("mutating") - public static class MutatingVisitorConfiguation implements VisitorConfiguration - { - public final RowVisitorConfiguration row_visitor; - - @JsonCreator - public MutatingVisitorConfiguation(@JsonProperty("row_visitor") RowVisitorConfiguration row_visitor) - { - this.row_visitor = row_visitor; - } - - @Override - public Visitor make(Run run) - { - return new MutatingVisitor(run, row_visitor::make); - } - } - - @JsonTypeName("logging") - public static class LoggingVisitorConfiguration implements VisitorConfiguration - { - protected final RowVisitorConfiguration row_visitor; - - @JsonCreator - public LoggingVisitorConfiguration(@JsonProperty("row_visitor") RowVisitorConfiguration row_visitor) - { - this.row_visitor = row_visitor; - } - - @Override - public Visitor make(Run run) - { - return new LoggingVisitor(run, row_visitor::make); - } - } - - @JsonTypeName("validate_all_partitions") - public static class AllPartitionsValidatorConfiguration implements VisitorConfiguration - { - public final int concurrency; - public final QueryLoggerConfiguration query_logger; - public final Configuration.ModelConfiguration model; - - @JsonCreator - public AllPartitionsValidatorConfiguration(@JsonProperty("concurrency") int concurrency, - @JsonProperty("model") Configuration.ModelConfiguration model, - @JsonProperty("query_logger") QueryLoggerConfiguration query_logger) - { - this.concurrency = concurrency; - this.model = model; - this.query_logger = QueryLogger.thisOrDefault(query_logger); - } - - public Visitor make(Run run) - { - return new AllPartitionsValidator(run, concurrency, model, query_logger.make()); - } - } - - @JsonTypeName("corrupt") - public static class CorruptingVisitorConfiguration implements VisitorConfiguration - { - public final int trigger_after; - - @JsonCreator - public CorruptingVisitorConfiguration(@JsonProperty("trigger_after") int trigger_after) - { - this.trigger_after = trigger_after; - } - - public Visitor make(Run run) - { - return new CorruptingVisitor(trigger_after, run); - } - } - - @JsonTypeName("validate_recent_partitions") - public static class RecentPartitionsValidatorConfiguration implements VisitorConfiguration - { - public final int partition_count; - public final int queries; - public final Configuration.ModelConfiguration modelConfiguration; - public final QueryLoggerConfiguration query_logger; - - // TODO: make query selector configurable - @JsonCreator - public RecentPartitionsValidatorConfiguration(@JsonProperty("partition_count") int partition_count, - @JsonProperty("queries_per_partition") int queries, - @JsonProperty("model") Configuration.ModelConfiguration model, - @JsonProperty("logger") QueryLoggerConfiguration query_logger) - { - this.partition_count = partition_count; - this.queries = queries; - this.modelConfiguration = model; - this.query_logger = QueryLogger.thisOrDefault(query_logger); - } - - @Override - public Visitor make(Run run) - { - return new RecentValidator(partition_count, queries, run, modelConfiguration, query_logger.make()); - } - } - - @JsonTypeName("validate_random_partitions") - public static class RandomPartitionValidatorConfiguration implements VisitorConfiguration - { - public final int partition_count; - public final int queries; - public final Configuration.ModelConfiguration model_configuration; - public final QueryLoggerConfiguration query_logger; - - // TODO: make query selector configurable - @JsonCreator - public RandomPartitionValidatorConfiguration(@JsonProperty("partition_count") int partition_count, - @JsonProperty("queries_per_partition") int queries, - @JsonProperty("model") Configuration.ModelConfiguration model, - @JsonProperty("logger") QueryLoggerConfiguration query_logger) - { - this.partition_count = partition_count; - this.queries = queries; - this.model_configuration = model; - this.query_logger = QueryLogger.thisOrDefault(query_logger); - } - - @Override - public Visitor make(Run run) - { - return new RandomValidator(partition_count, queries, run, model_configuration, query_logger.make()); - } - } - - @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.WRAPPER_OBJECT) - public interface QueryLoggerConfiguration extends QueryLogger.QueryLoggerFactory - { - } - - @JsonTypeName("no_op") - public static class NoOpQueryLoggerConfiguration implements QueryLoggerConfiguration - { - public QueryLogger make() - { - return QueryLogger.NO_OP; - } - } - - @JsonTypeName("file") - public static class FileQueryLoggerConfiguration implements QueryLoggerConfiguration - { - public final String filename; - - @JsonCreator - public FileQueryLoggerConfiguration(@JsonProperty("filename") String filename) - { - this.filename = filename; - } - - public QueryLogger make() - { - return new QueryLogger.FileQueryLogger(filename); - } - } - - - @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.WRAPPER_OBJECT) - public interface RowVisitorConfiguration extends OperationExecutor.RowVisitorFactory - { - } - - @JsonTypeName("mutating") - public static class MutatingRowVisitorConfiguration implements RowVisitorConfiguration - { - @Override - public OperationExecutor make(Run run) - { - return new MutatingRowVisitor(run); - } - } - - @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.WRAPPER_OBJECT) - public interface SchemaProviderConfiguration extends SchemaSpec.SchemaSpecFactory - { - } - - @JsonTypeName("default") - public static class DefaultSchemaProviderConfiguration implements SchemaProviderConfiguration - { - public SchemaSpec make(long seed, SystemUnderTest sut) - { - return SchemaGenerators.defaultSchemaSpecGen("table0") - .inflate(seed); - } - } - - @JsonTypeName("fixed") - public static class FixedSchemaProviderConfiguration implements SchemaProviderConfiguration - { - private final SchemaSpec schemaSpec; - - @JsonCreator - public FixedSchemaProviderConfiguration(@JsonProperty("keyspace") String keyspace, - @JsonProperty("table") String table, - @JsonProperty("partition_keys") Map<String, String> pks, - @JsonProperty("clustering_keys") Map<String, String> cks, - @JsonProperty("regular_columns") Map<String, String> regulars, - @JsonProperty("static_columns") Map<String, String> statics) - { - this(SchemaGenerators.parse(keyspace, table, - pks, cks, regulars, statics)); - } - - public FixedSchemaProviderConfiguration(SchemaSpec schemaSpec) - { - this.schemaSpec = schemaSpec; - } - public SchemaSpec make(long seed, SystemUnderTest sut) - { - return schemaSpec; - } - } - - @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.WRAPPER_OBJECT) - public interface MetricReporterConfiguration extends MetricReporter.MetricReporterFactory - { - } - - @JsonTypeName("no_op") - public static class NoOpMetricReporterConfiguration implements MetricReporterConfiguration - { - public MetricReporter make() - { - return MetricReporter.NO_OP; - } - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/core/MetricReporter.java b/test/harry/main/org/apache/cassandra/harry/core/MetricReporter.java deleted file mode 100644 index 4034eb97c567..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/core/MetricReporter.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.core; - -public interface MetricReporter -{ - void columnDelete(); - void rowDelete(); - void partitionDelete(); - void insert(); - void rangeDelete(); - - void validatePartition(); - void validateRandomQuery(); - - interface MetricReporterFactory - { - MetricReporter make(); - } - - MetricReporter NO_OP = new NoOpMetricReporter(); - - class NoOpMetricReporter implements MetricReporter - { - private NoOpMetricReporter() {} - - public void columnDelete(){} - public void rowDelete(){} - public void partitionDelete(){} - public void insert(){} - public void rangeDelete(){} - public void validatePartition(){} - public void validateRandomQuery(){} - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/core/Run.java b/test/harry/main/org/apache/cassandra/harry/core/Run.java deleted file mode 100644 index 0399a34072d2..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/core/Run.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.core; - -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.tracker.DataTracker; -import org.apache.cassandra.harry.operations.QueryGenerator; - -public class Run -{ - public final OpSelectors.PureRng rng; - public final OpSelectors.Clock clock; - public final OpSelectors.PdSelector pdSelector; - public final OpSelectors.DescriptorSelector descriptorSelector; - public final QueryGenerator rangeSelector; - - public final SchemaSpec schemaSpec; - public final DataTracker tracker; - public final SystemUnderTest sut; - - public final MetricReporter metricReporter; - - public Run(OpSelectors.PureRng rng, - OpSelectors.Clock clock, - OpSelectors.PdSelector pdSelector, - OpSelectors.DescriptorSelector descriptorSelector, - SchemaSpec schemaSpec, - DataTracker tracker, - SystemUnderTest sut, - MetricReporter metricReporter) - { - this(rng, clock, pdSelector, descriptorSelector, - new QueryGenerator(schemaSpec, pdSelector, descriptorSelector, rng), - schemaSpec, tracker, sut, metricReporter); - } - - private Run(OpSelectors.PureRng rng, - OpSelectors.Clock clock, - OpSelectors.PdSelector pdSelector, - OpSelectors.DescriptorSelector descriptorSelector, - QueryGenerator rangeSelector, - SchemaSpec schemaSpec, - DataTracker tracker, - SystemUnderTest sut, - MetricReporter metricReporter) - { - this.rng = rng; - this.clock = clock; - this.pdSelector = pdSelector; - this.descriptorSelector = descriptorSelector; - this.rangeSelector = rangeSelector; - this.schemaSpec = schemaSpec; - this.tracker = tracker; - this.sut = sut; - this.metricReporter = metricReporter; - } -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/corruptor/AddExtraRowCorruptor.java b/test/harry/main/org/apache/cassandra/harry/corruptor/AddExtraRowCorruptor.java deleted file mode 100644 index 11de57ba4ae1..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/corruptor/AddExtraRowCorruptor.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.corruptor; - -import java.util.HashSet; -import java.util.Set; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.harry.data.ResultSetRow; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.model.SelectHelper; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.operations.WriteHelper; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.tracker.DataTracker; - -public class AddExtraRowCorruptor implements QueryResponseCorruptor -{ - private static final Logger logger = LoggerFactory.getLogger(AddExtraRowCorruptor.class); - - private final SchemaSpec schema; - private final OpSelectors.Clock clock; - private final DataTracker tracker; - private final OpSelectors.DescriptorSelector descriptorSelector; - - public AddExtraRowCorruptor(SchemaSpec schema, - OpSelectors.Clock clock, - DataTracker tracker, - OpSelectors.DescriptorSelector descriptorSelector) - { - this.schema = schema; - this.clock = clock; - this.tracker = tracker; - this.descriptorSelector = descriptorSelector; - } - - public boolean maybeCorrupt(Query query, SystemUnderTest sut) - { - Set<Long> cds = new HashSet<>(); - long maxLts = tracker.maxStarted(); - for (Object[] obj : sut.execute(query.toSelectStatement(), SystemUnderTest.ConsistencyLevel.ALL)) - { - ResultSetRow row = SelectHelper.resultSetToRow(schema, clock, obj); - cds.add(row.cd); - } - boolean partitionIsFull = cds.size() >= descriptorSelector.maxPartitionSize(); - - long attempt = 0; - long cd = descriptorSelector.randomCd(query.pd, attempt, schema); - while (!query.matchCd(cd) || cds.contains(cd)) - { - if (partitionIsFull) - // We can't pick from the existing CDs, so let's try to come up with a new one that would match the query - cd += descriptorSelector.randomCd(query.pd, attempt, schema); - else - cd = descriptorSelector.randomCd(query.pd, attempt, schema); - if (attempt++ == 1000) - return false; - } - - long[] vds = descriptorSelector.vds(query.pd, cd, maxLts, 0, OpSelectors.OperationKind.INSERT, schema); - - // We do not know if the row was deleted. We could try inferring it, but that - // still won't help since we can't use it anyways, since collisions between a - // written value and tombstone are resolved in favour of tombstone, so we're - // just going to take the next lts. - logger.info("Corrupting the resultset by writing a row with cd {}", cd); - sut.execute(WriteHelper.inflateInsert(schema, query.pd, cd, vds, null, clock.rts(maxLts) + 1), SystemUnderTest.ConsistencyLevel.ALL); - return true; - } -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/corruptor/ChangeValueCorruptor.java b/test/harry/main/org/apache/cassandra/harry/corruptor/ChangeValueCorruptor.java deleted file mode 100644 index 1e9189a93bf8..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/corruptor/ChangeValueCorruptor.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.corruptor; - -import java.util.Arrays; - -import org.apache.cassandra.harry.data.ResultSetRow; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.gen.DataGenerators; -import org.apache.cassandra.harry.gen.rng.PcgRSUFast; -import org.apache.cassandra.harry.gen.EntropySource; -import org.apache.cassandra.harry.model.Model; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.operations.CompiledStatement; -import org.apache.cassandra.harry.operations.WriteHelper; - -/** - * Corrupts a single value written value in the row by writing a valid, invertible value with an incorrect - * descriptor, if row has any values written. - */ -public class ChangeValueCorruptor implements RowCorruptor -{ - private final SchemaSpec schema; - private final OpSelectors.Clock clock; - private final EntropySource rng; - - public ChangeValueCorruptor(SchemaSpec schemaSpec, - OpSelectors.Clock clock) - { - this.schema = schemaSpec; - this.clock = clock; - this.rng = new PcgRSUFast(1, 1); - } - - // Can corrupt any row that has at least one written non-null value - public boolean canCorrupt(ResultSetRow row) - { - for (int idx = 0; idx < row.lts.length; idx++) - { - // TODO: in addition to this, we should check if the value equals to the largest possible - // value, since otherwise it won't sort correctly. - if (row.lts[idx] != Model.NO_TIMESTAMP) - return true; - } - return false; - } - - public CompiledStatement corrupt(ResultSetRow row) - { - long[] corruptedVds = new long[row.vds.length]; - Arrays.fill(corruptedVds, DataGenerators.UNSET_DESCR); - - int idx; - do - { - idx = rng.nextInt(corruptedVds.length - 1); - } while (row.lts[idx] == Model.NO_TIMESTAMP); - - final long oldV = row.vds[idx]; - do - { - corruptedVds[idx] = schema.regularColumns.get(idx).type.generator().adjustEntropyDomain(rng.next()); - } - // we need to find a value that sorts strictly greater than the current one - while (schema.regularColumns.get(idx).type.compareLexicographically(corruptedVds[idx], oldV) <= 0); - - return WriteHelper.inflateInsert(schema, row.pd, row.cd, corruptedVds, null, clock.rts(row.lts[idx])); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/corruptor/HideRowCorruptor.java b/test/harry/main/org/apache/cassandra/harry/corruptor/HideRowCorruptor.java deleted file mode 100644 index d8b8ed971c4a..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/corruptor/HideRowCorruptor.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.corruptor; - -import org.apache.cassandra.harry.data.ResultSetRow; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.operations.CompiledStatement; -import org.apache.cassandra.harry.operations.DeleteHelper; - -public class HideRowCorruptor implements RowCorruptor -{ - private final SchemaSpec schema; - private final OpSelectors.Clock clock; - - public HideRowCorruptor(SchemaSpec schemaSpec, - OpSelectors.Clock clock) - { - this.schema = schemaSpec; - this.clock = clock; - } - - // Can corrupt any row that has at least one written non-null value - public boolean canCorrupt(ResultSetRow row) - { - return row != null; - } - - public CompiledStatement corrupt(ResultSetRow row) - { - return DeleteHelper.deleteRow(schema, row.pd, row.cd, clock.rts(clock.peek())); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/corruptor/HideValueCorruptor.java b/test/harry/main/org/apache/cassandra/harry/corruptor/HideValueCorruptor.java deleted file mode 100644 index 7b2d1fbfe7aa..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/corruptor/HideValueCorruptor.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.corruptor; - -import java.util.HashSet; -import java.util.Set; - -import org.apache.cassandra.harry.data.ResultSetRow; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.gen.rng.JdkRandomEntropySource; -import org.apache.cassandra.harry.gen.EntropySource; -import org.apache.cassandra.harry.model.Model; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.operations.CompiledStatement; -import org.apache.cassandra.harry.operations.DeleteHelper; -import org.apache.cassandra.harry.util.BitSet; - -// removes/hides the value of one of the columns that was previously set -public class HideValueCorruptor implements RowCorruptor -{ - private final SchemaSpec schema; - private final OpSelectors.Clock clock; - private final EntropySource rng; - - public HideValueCorruptor(SchemaSpec schemaSpec, - OpSelectors.Clock clock) - { - this.schema = schemaSpec; - this.clock = clock; - this.rng = new JdkRandomEntropySource(1L); - } - - // Can corrupt any row that has at least one written non-null value - public boolean canCorrupt(ResultSetRow row) - { - for (int idx = 0; idx < row.lts.length; idx++) - { - if (row.lts[idx] != Model.NO_TIMESTAMP) - return true; - } - return false; - } - - public CompiledStatement corrupt(ResultSetRow row) - { - BitSet mask; - // Corrupt a static row, if it is available and if RNG says so - if (row.hasStaticColumns() && rng.nextBoolean()) - { - int cnt = 0; - int idx; - do - { - idx = rng.nextInt(row.slts.length); - cnt++; - } - while (row.slts[idx] == Model.NO_TIMESTAMP && cnt < 10); - - if (row.slts[idx] != Model.NO_TIMESTAMP) - { - mask = BitSet.allUnset(schema.allColumns.size()); - mask.set(schema.staticColumnsOffset + idx); - - return DeleteHelper.deleteColumn(schema, - row.pd, - mask, - schema.regularAndStaticColumnsMask(), - clock.rts(clock.peek())); - } - } - - Set<Integer> tried = new HashSet<>(); - int idx; - do - { - if (tried.size() == row.lts.length) - throw new IllegalStateException(String.format("Could not corrupt after trying all %s indexes", tried)); - idx = rng.nextInt(row.lts.length); - tried.add(idx); - } - while (row.lts[idx] == Model.NO_TIMESTAMP); - - mask = BitSet.allUnset(schema.allColumns.size()); - mask.set(schema.regularColumnsOffset + idx); - - return DeleteHelper.deleteColumn(schema, - row.pd, - row.cd, - mask, - schema.regularAndStaticColumnsMask(), - clock.rts(clock.peek())); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/corruptor/QueryResponseCorruptor.java b/test/harry/main/org/apache/cassandra/harry/corruptor/QueryResponseCorruptor.java deleted file mode 100644 index 6495bd4d1143..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/corruptor/QueryResponseCorruptor.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.corruptor; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.harry.data.ResultSetRow; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.model.SelectHelper; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.operations.CompiledStatement; -import org.apache.cassandra.harry.operations.Query; - -public interface QueryResponseCorruptor -{ - Logger logger = LoggerFactory.getLogger(QueryResponseCorruptor.class); - - boolean maybeCorrupt(Query query, SystemUnderTest sut); - - class SimpleQueryResponseCorruptor implements QueryResponseCorruptor - { - private final RowCorruptor rowCorruptor; - private final SchemaSpec schema; - private final OpSelectors.Clock clock; - - public SimpleQueryResponseCorruptor(SchemaSpec schema, - OpSelectors.Clock clock, - RowCorruptor.RowCorruptorFactory factory) - { - this.rowCorruptor = factory.create(schema, clock); - this.schema = schema; - this.clock = clock; - } - - public boolean maybeCorrupt(Query query, SystemUnderTest sut) - { - List<ResultSetRow> result = new ArrayList<>(); - CompiledStatement statement = query.toSelectStatement(); - Object[][] before = sut.execute(statement.cql(), SystemUnderTest.ConsistencyLevel.ALL, statement.bindings()); - for (Object[] obj : before) - result.add(SelectHelper.resultSetToRow(schema, clock, obj)); - - // Technically, we can do this just depends on corruption strategy, - // we just need to corrupt results of the current query. - if (result.isEmpty()) - return false; - - for (ResultSetRow row : result) - { - if (rowCorruptor.maybeCorrupt(row, sut)) - { - Object[][] after = sut.execute(statement.cql(), SystemUnderTest.ConsistencyLevel.ALL, statement.bindings()); - boolean mismatch = false; - for (int i = 0; i < before.length && i < after.length; i++) - { - if (!Arrays.equals(before[i], after[i])) - { - logger.info("Corrupted: \nBefore: {}\n" + - "After: {}\n", - Arrays.toString(before[i]), - Arrays.toString(after[i])); - mismatch = true; - } - } - assert mismatch || before.length != after.length : String.format("Could not corrupt.\n" + - "Before\n%s\n" + - "After\n%s\nkma", - toString(before), - toString(after)); - return true; - } - } - return false; - } - - private static String toString(Object[][] obj) - { - StringBuilder sb = new StringBuilder(); - for (Object[] objects : obj) - { - sb.append(Arrays.toString(objects)).append("\n"); - } - return sb.toString(); - } - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/corruptor/RowCorruptor.java b/test/harry/main/org/apache/cassandra/harry/corruptor/RowCorruptor.java deleted file mode 100644 index 969f49ce4601..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/corruptor/RowCorruptor.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.corruptor; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.harry.data.ResultSetRow; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.operations.CompiledStatement; - -public interface RowCorruptor -{ - final Logger logger = LoggerFactory.getLogger(QueryResponseCorruptor.class); - - boolean canCorrupt(ResultSetRow row); - - CompiledStatement corrupt(ResultSetRow row); - - // Returns true if it could corrupt a row, false otherwise - default boolean maybeCorrupt(ResultSetRow row, SystemUnderTest sut) - { - if (canCorrupt(row)) - { - CompiledStatement statement = corrupt(row); - sut.execute(statement.cql(), SystemUnderTest.ConsistencyLevel.ALL, statement.bindings()); - logger.info("Corrupting with: {} ({})", statement.cql(), CompiledStatement.bindingsToString(statement.bindings())); - return true; - } - return false; - } - - interface RowCorruptorFactory - { - RowCorruptor create(SchemaSpec schemaSpec, - OpSelectors.Clock clock); - } -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/corruptor/ShowValueCorruptor.java b/test/harry/main/org/apache/cassandra/harry/corruptor/ShowValueCorruptor.java deleted file mode 100644 index 79c6c737e708..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/corruptor/ShowValueCorruptor.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.corruptor; - -import java.util.Arrays; - -import org.apache.cassandra.harry.data.ResultSetRow; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.gen.DataGenerators; -import org.apache.cassandra.harry.gen.rng.PcgRSUFast; -import org.apache.cassandra.harry.gen.EntropySource; -import org.apache.cassandra.harry.model.Model; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.operations.CompiledStatement; -import org.apache.cassandra.harry.operations.WriteHelper; - -public class ShowValueCorruptor implements RowCorruptor -{ - private final SchemaSpec schema; - private final OpSelectors.Clock clock; - private final EntropySource rng; - - public ShowValueCorruptor(SchemaSpec schemaSpec, - OpSelectors.Clock clock) - { - this.schema = schemaSpec; - this.clock = clock; - this.rng = new PcgRSUFast(1, 1); - } - - // Can corrupt any row that has at least one written non-null value - public boolean canCorrupt(ResultSetRow row) - { - for (int idx = 0; idx < row.lts.length; idx++) - { - if (row.lts[idx] == Model.NO_TIMESTAMP) - return true; - } - return false; - } - - public CompiledStatement corrupt(ResultSetRow row) - { - long[] corruptedVds = new long[row.lts.length]; - Arrays.fill(corruptedVds, DataGenerators.UNSET_DESCR); - - int idx; - do - { - idx = rng.nextInt(corruptedVds.length - 1); - } - while (row.lts[idx] != Model.NO_TIMESTAMP); - - corruptedVds[idx] = rng.next(); - - // We do not know LTS of the deleted row. We could try inferring it, but that - // still won't help since we can't use it anyways, since collisions between a - // written value and tombstone are resolved in favour of tombstone. - return WriteHelper.inflateInsert(schema, row.pd, row.cd, corruptedVds, null, clock.rts(clock.peek())); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/cql/DeleteHelper.java b/test/harry/main/org/apache/cassandra/harry/cql/DeleteHelper.java new file mode 100644 index 000000000000..1a237f58287a --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/cql/DeleteHelper.java @@ -0,0 +1,245 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.cql; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.Consumer; +import java.util.function.IntConsumer; + +import org.apache.cassandra.cql3.ast.Symbol; +import org.apache.cassandra.harry.ColumnSpec; +import org.apache.cassandra.harry.Relations; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.op.Operations; +import org.apache.cassandra.harry.execution.CompiledStatement; +import org.apache.cassandra.harry.util.BitSet; + +public class DeleteHelper +{ + public static CompiledStatement inflateDelete(Operations.DeletePartition delete, + SchemaSpec schema, + long timestamp) + { + StringBuilder b = new StringBuilder(); + b.append("DELETE FROM ") + .append(Symbol.maybeQuote(schema.keyspace)) + .append(".") + .append(Symbol.maybeQuote(schema.table)); + + if (timestamp != -1 && schema.options.addWriteTimestamps()) + b.append(" USING TIMESTAMP ").append(timestamp); + + b.append(" WHERE "); + + List<Object> bindings = new ArrayList<>(); + + Object[] pk = schema.valueGenerators.pkGen().inflate(delete.pd()); + + RelationWriter writer = new RelationWriter(b, bindings::add) ; + + for (int i = 0; i < pk.length; i++) + writer.write(schema.partitionKeys.get(i), Relations.RelationKind.EQ, pk[i]); + + b.append(";"); + + Object[] bindingsArr = bindings.toArray(new Object[bindings.size()]); + + return new CompiledStatement(b.toString(), bindingsArr); + } + + public static CompiledStatement inflateDelete(Operations.DeleteRow delete, + SchemaSpec schema, + long timestamp) + { + StringBuilder b = new StringBuilder(); + b.append("DELETE FROM ") + .append(Symbol.maybeQuote(schema.keyspace)) + .append(".") + .append(Symbol.maybeQuote(schema.table)); + + if (timestamp != -1 && schema.options.addWriteTimestamps()) + b.append(" USING TIMESTAMP ").append(timestamp); + + b.append(" WHERE "); + + List<Object> bindings = new ArrayList<>(); + + Object[] pk = schema.valueGenerators.pkGen().inflate(delete.pd()); + Object[] ck = schema.valueGenerators.ckGen().inflate(delete.cd()); + + RelationWriter writer = new RelationWriter(b, bindings::add); + + for (int i = 0; i < pk.length; i++) + writer.write(schema.partitionKeys.get(i), Relations.RelationKind.EQ, pk[i]); + for (int i = 0; i < ck.length; i++) + writer.write(schema.clusteringKeys.get(i), Relations.RelationKind.EQ, ck[i]); + + b.append(";"); + + Object[] bindingsArr = bindings.toArray(new Object[bindings.size()]); + + return new CompiledStatement(b.toString(), bindingsArr); + } + + public static CompiledStatement inflateDelete(Operations.DeleteColumns delete, + SchemaSpec schema, + long timestamp) + { + StringBuilder b = new StringBuilder(); + b.append("DELETE "); + + { + String[] names = columnNames(schema.regularColumns, delete.regularColumns()); + for (int i = 0; i < names.length; i++) + { + if (i > 0) + b.append(", "); + b.append(Symbol.maybeQuote(names[i])); + } + b.append(" "); + } + + { + String[] names = columnNames(schema.staticColumns, delete.staticColumns()); + for (int i = 0; i < names.length; i++) + { + if (i > 0) + b.append(", "); + b.append(Symbol.maybeQuote(names[i])); + } + b.append(" "); + } + + b.append("FROM ") + .append(Symbol.maybeQuote(schema.keyspace)) + .append(".") + .append(Symbol.maybeQuote(schema.table)); + + if (timestamp != -1 && schema.options.addWriteTimestamps()) + b.append(" USING TIMESTAMP ").append(timestamp); + + b.append(" WHERE "); + + List<Object> bindings = new ArrayList<>(); + + Object[] pk = schema.valueGenerators.pkGen().inflate(delete.pd()); + Object[] ck = schema.valueGenerators.ckGen().inflate(delete.cd()); + + RelationWriter writer = new RelationWriter(b, bindings::add); + + for (int i = 0; i < pk.length; i++) + writer.write(schema.partitionKeys.get(i), Relations.RelationKind.EQ, pk[i]); + for (int i = 0; i < ck.length; i++) + writer.write(schema.clusteringKeys.get(i), Relations.RelationKind.EQ, ck[i]); + + b.append(";"); + + Object[] bindingsArr = bindings.toArray(new Object[bindings.size()]); + + return new CompiledStatement(b.toString(), bindingsArr); + } + + public static CompiledStatement inflateDelete(Operations.DeleteRange delete, + SchemaSpec schema, + long timestamp) + { + StringBuilder b = new StringBuilder(); + b.append("DELETE FROM ") + .append(Symbol.maybeQuote(schema.keyspace)) + .append(".") + .append(Symbol.maybeQuote(schema.table)); + + if (timestamp != -1 && schema.options.addWriteTimestamps()) + b.append(" USING TIMESTAMP ").append(timestamp); + + b.append(" WHERE "); + + List<Object> bindings = new ArrayList<>(); + + Object[] pk = schema.valueGenerators.pkGen().inflate(delete.pd()); + Object[] lowBound = schema.valueGenerators.ckGen().inflate(delete.lowerBound()); + Object[] highBound = schema.valueGenerators.ckGen().inflate(delete.upperBound()); + + RelationWriter writer = new RelationWriter(b, bindings::add); + + for (int i = 0; i < pk.length; i++) + writer.write(schema.partitionKeys.get(i), Relations.RelationKind.EQ, pk[i]); + for (int i = 0; i < delete.lowerBoundRelation().length; i++) + { + Relations.RelationKind kind; + kind = delete.lowerBoundRelation()[i]; + if (kind != null) + writer.write(schema.clusteringKeys.get(i), kind, lowBound[i]); + kind = delete.upperBoundRelation()[i]; + if (kind != null) + writer.write(schema.clusteringKeys.get(i), kind, highBound[i]); + } + + b.append(";"); + + Object[] bindingsArr = bindings.toArray(new Object[bindings.size()]); + + return new CompiledStatement(b.toString(), bindingsArr); + } + + private static final class RelationWriter + { + boolean isFirst = true; + final StringBuilder builder; + final Consumer<Object> collectBindings; + + private RelationWriter(StringBuilder builder, Consumer<Object> bindings) + { + this.builder = builder; + this.collectBindings = bindings; + } + + public void write(ColumnSpec<?> column, Relations.RelationKind kind, Object value) + { + if (isFirst) + isFirst = false; + else + builder.append(" AND "); + + builder.append(Symbol.maybeQuote(column.name)) + .append(" ") + .append(kind.symbol()) + .append(" ") + .append("?"); + collectBindings.accept(value); + } + } + + + private static String[] columnNames(List<ColumnSpec<?>> columns, BitSet selectedColumns) + { + String[] columnNames = new String[selectedColumns.size()]; + selectedColumns.eachSetBit(new IntConsumer() + { + int i = 0; + + public void accept(int idx) + { + columnNames[i++] = columns.get(idx).name; + } + }); + return columnNames; + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/cql/SelectHelper.java b/test/harry/main/org/apache/cassandra/harry/cql/SelectHelper.java new file mode 100644 index 000000000000..148711173c69 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/cql/SelectHelper.java @@ -0,0 +1,241 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.cql; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.cassandra.cql3.ast.Bind; +import org.apache.cassandra.cql3.ast.CQLFormatter; +import org.apache.cassandra.cql3.ast.Conditional.Where; +import org.apache.cassandra.cql3.ast.FunctionCall; +import org.apache.cassandra.cql3.ast.Select; +import org.apache.cassandra.cql3.ast.Symbol; +import org.apache.cassandra.harry.ColumnSpec; +import org.apache.cassandra.harry.op.Operations; +import org.apache.cassandra.harry.Relations; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.execution.CompiledStatement; + +public class SelectHelper +{ + public static CompiledStatement select(Operations.SelectPartition select, SchemaSpec schema) + { + Select.Builder builder = commmonPart(select, schema); + + if (select.orderBy() == Operations.ClusteringOrderBy.DESC) + { + for (int i = 0; i < schema.clusteringKeys.size(); i++) + { + ColumnSpec<?> c = schema.clusteringKeys.get(i); + builder.orderByColumn(c.name, c.type.asServerType(), c.isReversed() ? Select.OrderBy.Ordering.ASC : Select.OrderBy.Ordering.DESC); + } + } + + return toCompiled(builder.build()); + } + + public static CompiledStatement select(Operations.SelectRow select, SchemaSpec schema) + { + Select.Builder builder = commmonPart(select, schema); + + Object[] ck = schema.valueGenerators.ckGen().inflate(select.cd()); + + for (int i = 0; i < schema.clusteringKeys.size(); i++) + { + ColumnSpec<?> column = schema.clusteringKeys.get(i); + builder.where(new Symbol(column.name, column.type.asServerType()), + toInequality(Relations.RelationKind.EQ), + new Bind(ck[i], column.type.asServerType())); + } + + return toCompiled(builder.build()); + } + + public static CompiledStatement select(Operations.SelectRange select, SchemaSpec schema) + { + Select.Builder builder = commmonPart(select, schema); + + Object[] lowBound = schema.valueGenerators.ckGen().inflate(select.lowerBound()); + Object[] highBound = schema.valueGenerators.ckGen().inflate(select.upperBound()); + + for (int i = 0; i < schema.clusteringKeys.size(); i++) + { + ColumnSpec<?> column = schema.clusteringKeys.get(i); + if (select.lowerBoundRelation()[i] != null) + { + builder.where(new Symbol(column.name, column.type.asServerType()), + toInequality(select.lowerBoundRelation()[i]), + new Bind(lowBound[i], column.type.asServerType())); + } + + if (select.upperBoundRelation()[i] != null) + { + builder.where(new Symbol(column.name, column.type.asServerType()), + toInequality(select.upperBoundRelation()[i]), + new Bind(highBound[i], column.type.asServerType())); + } + } + + if (select.orderBy() == Operations.ClusteringOrderBy.DESC) + { + for (int i = 0; i < schema.clusteringKeys.size(); i++) + { + ColumnSpec<?> c = schema.clusteringKeys.get(i); + builder.orderByColumn(c.name, c.type.asServerType(), c.isReversed() ? Select.OrderBy.Ordering.ASC : Select.OrderBy.Ordering.DESC); + } + } + + return toCompiled(builder.build()); + } + + public static CompiledStatement select(Operations.SelectCustom select, SchemaSpec schema) + { + Select.Builder builder = commmonPart(select, schema); + + Map<Long, Object[]> cache = new HashMap<>(); + for (Relations.Relation relation : select.ckRelations()) + { + Object[] query = cache.computeIfAbsent(relation.descriptor, schema.valueGenerators.ckGen()::inflate); + ColumnSpec<?> column = schema.clusteringKeys.get(relation.column); + builder.where(new Symbol(column.name, column.type.asServerType()), + toInequality(relation.kind), + new Bind(query[relation.column], column.type.asServerType())); + } + + for (Relations.Relation relation : select.regularRelations()) + { + ColumnSpec<?> column = schema.regularColumns.get(relation.column); + Object query = schema.valueGenerators.regularColumnGen(relation.column).inflate(relation.descriptor); + builder.where(new Symbol(column.name, column.type.asServerType()), + toInequality(relation.kind), + new Bind(query, column.type.asServerType())); + } + + for (Relations.Relation relation : select.staticRelations()) + { + Object query = schema.valueGenerators.staticColumnGen(relation.column).inflate(relation.descriptor); + ColumnSpec<?> column = schema.staticColumns.get(relation.column); + builder.where(new Symbol(column.name, column.type.asServerType()), + toInequality(relation.kind), + new Bind(query, column.type.asServerType())); + } + + if (select.orderBy() == Operations.ClusteringOrderBy.DESC) + { + for (int i = 0; i < schema.clusteringKeys.size(); i++) + { + ColumnSpec<?> c = schema.clusteringKeys.get(i); + builder.orderByColumn(c.name, c.type.asServerType(), c.isReversed() ? Select.OrderBy.Ordering.ASC : Select.OrderBy.Ordering.DESC); + } + } + + builder.allowFiltering(); + + return toCompiled(builder.build()); + } + + public static Select.Builder commmonPart(Operations.SelectStatement select, SchemaSpec schema) + { + Select.Builder builder = new Select.Builder(); + + Operations.Selection selection = Operations.Selection.fromBitSet(select.selection(), schema); + if (selection.isWildcard()) + { + builder.wildcard(); + } + else + { + for (int i = 0; i < schema.allColumnInSelectOrder.size(); i++) + { + ColumnSpec<?> spec = schema.allColumnInSelectOrder.get(i); + if (!selection.columns().contains(spec)) + continue; + + builder.columnSelection(spec.name, spec.type.asServerType()); + } + + if (selection.includeTimestamps()) + { + for (ColumnSpec<?> spec : schema.staticColumns) + { + if (!selection.columns().contains(spec)) + continue; + builder.selection(FunctionCall.writetime(spec.name, spec.type.asServerType())); + } + + for (ColumnSpec<?> spec : schema.regularColumns) + { + if (!selection.columns().contains(spec)) + continue; + builder.selection(FunctionCall.writetime(spec.name, spec.type.asServerType())); + } + } + } + + builder.table(schema.keyspace, schema.table); + + Object[] pk = schema.valueGenerators.pkGen().inflate(select.pd()); + for (int i = 0; i < schema.partitionKeys.size(); i++) + { + ColumnSpec<?> column = schema.partitionKeys.get(i); + Object value = pk[i]; + builder.where(new Symbol(column.name, column.type.asServerType()), + Where.Inequality.EQUAL, + new Bind(value, column.type.asServerType())); + } + + return builder; + } + + private static Where.Inequality toInequality(Relations.RelationKind kind) + { + Where.Inequality inequalities; + switch (kind) + { + case LT: + inequalities = Where.Inequality.LESS_THAN; + break; + case LTE: + inequalities = Where.Inequality.LESS_THAN_EQ; + break; + case GT: + inequalities = Where.Inequality.GREATER_THAN; + break; + case GTE: + inequalities = Where.Inequality.GREATER_THAN_EQ; + break; + case EQ: + inequalities = Where.Inequality.EQUAL; + break; + default: + throw new UnsupportedOperationException("Unknown kind: " + kind); + } + return inequalities; + } + + private static CompiledStatement toCompiled(Select select) + { + // Select does not add ';' by default, but CompiledStatement expects this + String cql = select.toCQL(CQLFormatter.None.instance) + ';'; + Object[] bindingsArr = select.binds(); + return new CompiledStatement(cql, bindingsArr); + } + +} diff --git a/test/harry/main/org/apache/cassandra/harry/cql/WriteHelper.java b/test/harry/main/org/apache/cassandra/harry/cql/WriteHelper.java new file mode 100644 index 000000000000..b3143a7043f4 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/cql/WriteHelper.java @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.cql; + +import java.util.List; + +import accord.utils.Invariants; +import org.apache.cassandra.harry.ColumnSpec; +import org.apache.cassandra.harry.MagicConstants; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.execution.CompiledStatement; +import org.apache.cassandra.harry.op.Operations; + +public class WriteHelper +{ + public static CompiledStatement inflateInsert(Operations.WriteOp op, + SchemaSpec schema, + long timestamp) + { + assert op.vds().length == schema.regularColumns.size(); + assert op.sds().length == schema.staticColumns.size(); + assert op.vds().length == schema.valueGenerators.regularColumnCount(); + assert op.sds().length == schema.valueGenerators.staticColumnCount(); + + Object[] partitionKey = schema.valueGenerators.pkGen().inflate(op.pd()); + assert partitionKey.length == schema.partitionKeys.size(); + Object[] clusteringKey = schema.valueGenerators.ckGen().inflate(op.cd()); + assert clusteringKey.length == schema.clusteringKeys.size(); + Object[] regularColumns = new Object[op.vds().length]; + Object[] staticColumns = new Object[op.sds().length]; + + for (int i = 0; i < op.vds().length; i++) + { + long descriptor = op.vds()[i]; + if (descriptor == MagicConstants.UNSET_DESCR) + regularColumns[i] = MagicConstants.UNSET_VALUE; + else + regularColumns[i] = schema.valueGenerators.regularColumnGen(i).inflate(descriptor); + } + + for (int i = 0; i < op.sds().length; i++) + { + long descriptor = op.sds()[i]; + if (descriptor == MagicConstants.UNSET_DESCR) + staticColumns[i] = MagicConstants.UNSET_VALUE; + else + staticColumns[i] = schema.valueGenerators.staticColumnGen(i).inflate(descriptor); + } + + Object[] bindings = new Object[schema.allColumnInSelectOrder.size()]; + + StringBuilder b = new StringBuilder(); + b.append("INSERT INTO ") + .append(schema.keyspace) + .append('.') + .append(schema.table) + .append(" ("); + + int bindingsCount = 0; + bindingsCount += appendStatements(b, bindings, schema.partitionKeys, partitionKey, bindingsCount, true, ",", "%s"); + bindingsCount += appendStatements(b, bindings, schema.clusteringKeys, clusteringKey, bindingsCount, false, ",", "%s"); + bindingsCount += appendStatements(b, bindings, schema.regularColumns, regularColumns, bindingsCount, false, ",", "%s"); + bindingsCount += appendStatements(b, bindings, schema.staticColumns, staticColumns, bindingsCount, false, ",", "%s"); + + b.append(") VALUES ("); + + for (int i = 0; i < bindingsCount; i++) + { + if (i > 0) + b.append(", "); + b.append("?"); + } + + b.append(")"); + if (timestamp != -1 && schema.options.addWriteTimestamps()) + { + b.append(" USING TIMESTAMP ") + .append(timestamp); + } + + b.append(";"); + return new CompiledStatement(b.toString(), adjustArraySize(bindings, bindingsCount)); + } + + public static Object[] adjustArraySize(Object[] bindings, int bindingsCount) + { + if (bindingsCount != bindings.length) + { + Object[] tmp = new Object[bindingsCount]; + System.arraycopy(bindings, 0, tmp, 0, bindingsCount); + bindings = tmp; + } + return bindings; + } + + public static CompiledStatement inflateUpdate(Operations.WriteOp op, + SchemaSpec schema, + long timestamp) + { + assert op.vds().length == schema.regularColumns.size(); + assert op.sds().length == schema.staticColumns.size(); + assert op.vds().length == schema.valueGenerators.regularColumnCount(); + assert op.sds().length == schema.valueGenerators.staticColumnCount(); + + Object[] partitionKey = schema.valueGenerators.pkGen().inflate(op.pd); + assert partitionKey.length == schema.partitionKeys.size(); + Object[] clusteringKey = schema.valueGenerators.ckGen().inflate(op.cd()); + assert clusteringKey.length == schema.clusteringKeys.size(); + Object[] regularColumns = new Object[op.vds().length]; + Object[] staticColumns = new Object[op.sds().length]; + + for (int i = 0; i < op.vds().length; i++) + regularColumns[i] = schema.valueGenerators.regularColumnGen(i).inflate(op.vds()[i]); + + for (int i = 0; i < op.sds().length; i++) + staticColumns[i] = schema.valueGenerators.staticColumnGen(i).inflate(op.sds()[i]); + + Object[] bindings = new Object[schema.allColumnInSelectOrder.size()]; + + StringBuilder b = new StringBuilder(); + b.append("UPDATE ") + .append(schema.keyspace) + .append('.') + .append(schema.table); + + if (timestamp != -1 && schema.options.addWriteTimestamps()) + { + b.append(" USING TIMESTAMP ") + .append(timestamp) + .append(" SET "); + } + + int bindingsCount = 0; + bindingsCount += addSetStatements(b, bindings, schema.regularColumns, regularColumns, bindingsCount); + if (staticColumns.length != 0) + bindingsCount += addSetStatements(b, bindings, schema.staticColumns, staticColumns, bindingsCount); + + assert bindingsCount > 0 : "Can not have an UPDATE statement without any updates"; + b.append(" WHERE "); + + bindingsCount += addWhereStatements(b, bindings, schema.partitionKeys, partitionKey, bindingsCount, true); + bindingsCount += addWhereStatements(b, bindings, schema.clusteringKeys, clusteringKey, bindingsCount, false); + b.append(";"); + return new CompiledStatement(b.toString(), adjustArraySize(bindings, bindingsCount)); + } + + private static int addSetStatements(StringBuilder b, + Object[] bindings, + List<ColumnSpec<?>> columns, + Object[] values, + int bound) + { + return appendStatements(b, bindings, columns, values, bound, bound == 0, ", ", "%s = ?"); + } + + private static int addWhereStatements(StringBuilder b, + Object[] bindings, + List<ColumnSpec<?>> columns, + Object[] values, + int bound, + boolean firstStatement) + { + return appendStatements(b, bindings, columns, values, bound, firstStatement, " AND ", "%s = ?"); + } + + private static int appendStatements(StringBuilder b, + Object[] allBindings, + List<ColumnSpec<?>> columns, + Object[] values, + int bound, + boolean firstStatement, + String separator, + String nameFormatter) + { + int bindingsCount = 0; + for (int i = 0; i < values.length; i++) + { + Object value = values[i]; + if (value == MagicConstants.UNSET_VALUE) + continue; + Invariants.nonNull(value); + ColumnSpec<?> column = columns.get(i); + if (bindingsCount > 0 || !firstStatement) + b.append(separator); + + b.append(String.format(nameFormatter, column.name)); + allBindings[bound + bindingsCount] = value; + bindingsCount++; + } + return bindingsCount; + } +} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/data/ResultSetRow.java b/test/harry/main/org/apache/cassandra/harry/data/ResultSetRow.java deleted file mode 100644 index 7ee1e06e0c44..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/data/ResultSetRow.java +++ /dev/null @@ -1,137 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.data; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Objects; - -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.util.StringUtils; - -public class ResultSetRow -{ - public final long pd; - public final long cd; - public final long[] vds; - public final long[] lts; - - public final long[] sds; - public final long[] slts; - public final List<Long> visited_lts; - - private ResultSetRow(long pd, - long cd, - long[] sds, - long[] slts, - long[] vds, - long[] lts) - { - this(pd, cd, sds, slts, vds, lts, Collections.emptyList()); - } - - public ResultSetRow(long pd, - long cd, - long[] sds, - long[] slts, - long[] vds, - long[] lts, - List<Long> visited_lts) - { - // There should be as many timestamps as value descriptors, both for regular and static columns - assert slts.length == sds.length : String.format("Descriptors: %s, timestamps: %s", Arrays.toString(sds), Arrays.toString(slts)); - assert lts.length == vds.length : String.format("Descriptors: %s, timestamps: %s", Arrays.toString(vds), Arrays.toString(lts)); - this.pd = pd; - this.cd = cd; - this.vds = vds; - this.lts = lts; - this.sds = sds; - this.slts = slts; - this.visited_lts = visited_lts; - } - - public boolean hasStaticColumns() - { - return slts.length > 0; - } - - @Override - public ResultSetRow clone() - { - return new ResultSetRow(pd, cd, - Arrays.copyOf(sds, sds.length), Arrays.copyOf(slts, slts.length), - Arrays.copyOf(vds, vds.length), Arrays.copyOf(lts, lts.length), - visited_lts); - } - - @Override - public int hashCode() - { - int result = Objects.hash(pd, cd, visited_lts); - result = 31 * result + Arrays.hashCode(vds); - result = 31 * result + Arrays.hashCode(lts); - result = 31 * result + Arrays.hashCode(sds); - result = 31 * result + Arrays.hashCode(slts); - return result; - } - - @Override - public boolean equals(Object o) - { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - ResultSetRow that = (ResultSetRow) o; - return pd == that.pd && - cd == that.cd && - Arrays.equals(vds, that.vds) && - Arrays.equals(lts, that.lts) && - Arrays.equals(sds, that.sds) && - Arrays.equals(slts, that.slts) && - Objects.equals(visited_lts, that.visited_lts); - } - - @Override - public String toString() - { - return "resultSetRow(" - + pd + - "L, " + cd + - (sds == null ? "" : "L, statics(" + StringUtils.toString(sds) + ")") + - (slts == null ? "" : ", lts(" + StringUtils.toString(slts) + ")") + - ", values(" + StringUtils.toString(vds) + ")" + - ", lts(" + StringUtils.toString(lts) + ")" + - ")"; - } - - public String toString(SchemaSpec schema) - { - return "resultSetRow(" - + pd + - "L, " + cd + - (sds == null ? "" : "L, staticValues(" + StringUtils.toString(sds) + ")") + - (slts == null ? "" : ", slts(" + StringUtils.toString(slts) + ")") + - ", values(" + StringUtils.toString(vds) + ")" + - ", lts(" + StringUtils.toString(lts) + ")" + - ", clustering=" + Arrays.toString(schema.inflateClusteringKey(cd)) + - ", values=" + Arrays.toString(schema.inflateRegularColumns(vds)) + - (sds == null ? "" : ", statics=" + Arrays.toString(schema.inflateStaticColumns(sds))) + - ")"; - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/ddl/ColumnSpec.java b/test/harry/main/org/apache/cassandra/harry/ddl/ColumnSpec.java deleted file mode 100644 index 88bd95d6a957..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/ddl/ColumnSpec.java +++ /dev/null @@ -1,460 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.ddl; - -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.Date; -import java.util.HashMap; -import java.util.Map; -import java.util.Objects; -import java.util.UUID; - -import org.apache.cassandra.harry.gen.Bijections; -import org.apache.cassandra.harry.gen.StringBijection; - -public class ColumnSpec<T> -{ - public final String name; - public final DataType<T> type; - public final Kind kind; - int columnIndex; - - public ColumnSpec(String name, - DataType<T> type, - Kind kind) - { - this.name = name; - this.type = type; - this.kind = kind; - } - - - public ColumnSpec<T> override(Bijections.Bijection<T> override) - { - return new ColumnSpec<>(name, - new DataType<>(type.cqlName) { - @Override - public int compareLexicographically(long l, long r) - { - return type.compareLexicographically(l, r); - } - - @Override - public boolean isReversed() - { - return type.isReversed(); - } - - @Override - public Bijections.Bijection<T> generator() - { - return override; - } - }, - kind); - } - - - void setColumnIndex(int idx) - { - this.columnIndex = idx; - } - - public int getColumnIndex() - { - return columnIndex; - } - - public String toCQL() - { - return String.format("%s %s%s", - name, - type.toString(), - kind == Kind.STATIC ? " static" : ""); - } - - @Override - public boolean equals(Object o) - { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - ColumnSpec<?> that = (ColumnSpec<?>) o; - return Objects.equals(name, that.name) && - Objects.equals(type.cqlName, that.type.cqlName) && - kind == that.kind; - } - - @Override - public int hashCode() - { - return Objects.hash(name, type.cqlName, kind); - } - - public String name() - { - return name; - } - - public boolean isReversed() - { - return type.isReversed(); - } - - public String toString() - { - return name + '(' + type.toString() + ")"; - } - - public Bijections.Bijection<T> generator() - { - return type.generator(); - } - - public T inflate(long current) - { - return type.generator().inflate(current); - } - - public long deflate(T value) - { - return type.generator().deflate(value); - } - - public static ColumnSpec<?> pk(String name, DataType<?> type) - { - return new ColumnSpec<>(name, type, Kind.PARTITION_KEY); - } - - @SuppressWarnings("unchecked") - public static ColumnSpec<?> ck(String name, DataType<?> type, boolean isReversed) - { - return new ColumnSpec(name, isReversed ? ReversedType.getInstance(type) : type, Kind.CLUSTERING); - } - - @SuppressWarnings("unchecked") - public static ColumnSpec<?> ck(String name, DataType<?> type) - { - return new ColumnSpec(name, type, Kind.CLUSTERING); - } - - public static ColumnSpec<?> regularColumn(String name, DataType<?> type) - { - return new ColumnSpec<>(name, type, Kind.REGULAR); - } - - public static ColumnSpec<?> staticColumn(String name, DataType<?> type) - { - return new ColumnSpec<>(name, type, Kind.STATIC); - } - - public enum Kind - { - CLUSTERING, REGULAR, STATIC, PARTITION_KEY - } - - public static abstract class DataType<T> - { - protected final String cqlName; - - protected DataType(String cqlName) - { - this.cqlName = cqlName; - } - - public boolean isReversed() - { - return false; - } - - /** - * Cassandra uses lexicographical oder for resolving timestamp ties - */ - public int compareLexicographically(long l, long r) - { - for (int i = Long.BYTES - 1; i >= 0; i--) - { - int cmp = Integer.compare((int) ((l >> (i * 8)) & 0xffL), - (int) ((r >> (i * 8)) & 0xffL)); - if (cmp != 0) - return cmp; - } - return 0; - } - - public abstract Bijections.Bijection<T> generator(); - - public int maxSize() - { - return generator().byteSize(); - } - - public final String toString() - { - return cqlName; - } - - public String nameForParser() - { - return cqlName; - } - - public final boolean equals(Object o) - { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - DataType<?> dataType = (DataType<?>) o; - return Objects.equals(cqlName, dataType.cqlName); - } - - public final int hashCode() - { - return Objects.hash(cqlName); - } - } - - public static final DataType<Byte> int8Type = new DataType<Byte>("tinyint") - { - public Bijections.Bijection<Byte> generator() - { - return Bijections.INT8_GENERATOR; - } - }; - - public static final DataType<Short> int16Type = new DataType<Short>("smallint") - { - public Bijections.Bijection<Short> generator() - { - return Bijections.INT16_GENERATOR; - } - }; - - public static final DataType<Integer> int32Type = new DataType<Integer>("int") - { - public Bijections.Bijection<Integer> generator() - { - return Bijections.INT32_GENERATOR; - } - }; - - public static final DataType<Long> int64Type = new DataType<Long>("bigint") - { - public Bijections.Bijection<Long> generator() - { - return Bijections.INT64_GENERATOR; - } - }; - - public static final DataType<Boolean> booleanType = new DataType<Boolean>("boolean") - { - public Bijections.Bijection<Boolean> generator() - { - return Bijections.BOOLEAN_GENERATOR; - } - - public int compareLexicographically(long l, long r) - { - throw new RuntimeException("Boolean does not support custom comparators"); - } - }; - - public static final DataType<Float> floatType = new DataType<Float>("float") - { - public Bijections.Bijection<Float> generator() - { - return Bijections.FLOAT_GENERATOR; - } - }; - - public static final DataType<Double> doubleType = new DataType<Double>("double") - { - public Bijections.Bijection<Double> generator() - { - return Bijections.DOUBLE_GENERATOR; - } - }; - - public static final DataType<String> asciiType = new DataType<String>("ascii") - { - private final Bijections.Bijection<String> gen = new StringBijection(); - - public Bijections.Bijection<String> generator() - { - return gen; - } - - public int compareLexicographically(long l, long r) - { - return Long.compare(l, r); - } - }; - - public static final DataType<String> textType = new DataType<String>("text") - { - private final Bijections.Bijection<String> gen = new StringBijection(); - - public Bijections.Bijection<String> generator() - { - return gen; - } - - public int compareLexicographically(long l, long r) - { - return Long.compare(l, r); - } - }; - - public static DataType<String> asciiType(int nibbleSize, int maxRandomBytes) - { - Bijections.Bijection<String> gen = new StringBijection(nibbleSize, maxRandomBytes); - - return new DataType<String>("ascii") - { - public Bijections.Bijection<String> generator() - { - return gen; - } - - public int compareLexicographically(long l, long r) - { - return Long.compare(l, r); - } - - public String nameForParser() - { - return String.format("%s(%d,%d)", - super.nameForParser(), - nibbleSize, - maxRandomBytes); - } - }; - } - - public static final DataType<UUID> uuidType = new DataType<UUID>("uuid") - { - public Bijections.Bijection<UUID> generator() - { - return Bijections.UUID_GENERATOR; - } - - public int compareLexicographically(long l, long r) - { - throw new RuntimeException("UUID does not support custom comparators"); - } - }; - - public static final DataType<UUID> timeUuidType = new DataType<UUID>("timeuuid") - { - public Bijections.Bijection<UUID> generator() - { - return Bijections.TIME_UUID_GENERATOR; - } - - public int compareLexicographically(long l, long r) - { - throw new RuntimeException("UUID does not support custom comparators"); - } - }; - - public static final DataType<Date> timestampType = new DataType<Date>("timestamp") - { - public Bijections.Bijection<Date> generator() - { - return Bijections.TIMESTAMP_GENERATOR; - } - - public int compareLexicographically(long l, long r) - { - throw new RuntimeException("Date does not support custom comparators"); - } - }; - - public static final Collection<DataType<?>> DATA_TYPES = Collections.unmodifiableList( - Arrays.asList(ColumnSpec.int8Type, - ColumnSpec.int16Type, - ColumnSpec.int32Type, - ColumnSpec.int64Type, - ColumnSpec.booleanType, - ColumnSpec.floatType, - ColumnSpec.doubleType, - ColumnSpec.asciiType, - ColumnSpec.textType, - ColumnSpec.uuidType, - ColumnSpec.timeUuidType, - ColumnSpec.timestampType)); - - public static class ReversedType<T> extends DataType<T> - { - public static final Map<DataType<?>, ReversedType<?>> cache = new HashMap() - {{ - put(int8Type, new ReversedType<>(int8Type)); - put(int16Type, new ReversedType<>(int16Type)); - put(int32Type, new ReversedType<>(int32Type)); - put(int64Type, new ReversedType<>(int64Type)); - put(booleanType, new ReversedType<>(booleanType)); - put(floatType, new ReversedType<>(floatType, new Bijections.ReverseFloatGenerator())); - put(doubleType, new ReversedType<>(doubleType, new Bijections.ReverseDoubleGenerator())); - put(asciiType, new ReversedType<>(asciiType)); - put(uuidType, new ReversedType<>(uuidType)); - put(timeUuidType, new ReversedType<>(timeUuidType)); - }}; - - private final DataType<T> baseType; - private final Bijections.Bijection<T> generator; - - public ReversedType(DataType<T> baseType) - { - super(baseType.cqlName); - this.baseType = baseType; - this.generator = new Bijections.ReverseBijection<>(baseType.generator()); - } - - public ReversedType(DataType<T> baseType, Bijections.Bijection<T> generator) - { - super(baseType.cqlName); - this.baseType = baseType; - this.generator = generator; - } - - public boolean isReversed() - { - return true; - } - - public Bijections.Bijection<T> generator() - { - return generator; - } - - public int maxSize() - { - return baseType.maxSize(); - } - - public static <T> DataType<T> getInstance(DataType<T> type) - { - ReversedType<T> t = (ReversedType<T>) cache.get(type); - if (t == null) - t = new ReversedType<>(type); - assert t.baseType == type : String.format("Type mismatch %s != %s", t.baseType, type); - return t; - } - } -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/ddl/SchemaGenerators.java b/test/harry/main/org/apache/cassandra/harry/ddl/SchemaGenerators.java deleted file mode 100644 index 7f1a8790bd45..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/ddl/SchemaGenerators.java +++ /dev/null @@ -1,512 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.ddl; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Function; -import java.util.function.Supplier; - -import org.apache.cassandra.config.CassandraRelevantProperties; -import org.apache.cassandra.harry.gen.Generator; -import org.apache.cassandra.harry.gen.Surjections; - -public class SchemaGenerators -{ - private final static long SCHEMAGEN_STREAM_ID = 0x6264593273L; - - public static Builder schema(String ks) - { - return new Builder(ks); - } - - public static final Map<String, ColumnSpec.DataType<?>> nameToTypeMap; - public static final Collection<ColumnSpec.DataType<?>> columnTypes; - public static final Collection<ColumnSpec.DataType<?>> partitionKeyTypes; - public static final Collection<ColumnSpec.DataType<?>> clusteringKeyTypes; - - static - { - partitionKeyTypes = Collections.unmodifiableList(Arrays.asList(ColumnSpec.int8Type, - ColumnSpec.int16Type, - ColumnSpec.int32Type, - ColumnSpec.int64Type, - ColumnSpec.floatType, - ColumnSpec.doubleType, - ColumnSpec.asciiType, - ColumnSpec.textType)); - - columnTypes = Collections.unmodifiableList(Arrays.asList(ColumnSpec.int8Type, - ColumnSpec.int16Type, - ColumnSpec.int32Type, - ColumnSpec.int64Type, - ColumnSpec.floatType, - ColumnSpec.doubleType, - ColumnSpec.asciiType, - ColumnSpec.textType)); - - - List<ColumnSpec.DataType<?>> builder = new ArrayList<>(partitionKeyTypes); - Map<String, ColumnSpec.DataType<?>> mapBuilder = new HashMap<>(); - - for (ColumnSpec.DataType<?> columnType : partitionKeyTypes) - { - ColumnSpec.DataType<?> reversedType = ColumnSpec.ReversedType.getInstance(columnType); - builder.add(reversedType); - - mapBuilder.put(columnType.nameForParser(), columnType); - mapBuilder.put(String.format("desc(%s)", columnType.nameForParser()), columnType); - } - - builder.add(ColumnSpec.floatType); - builder.add(ColumnSpec.doubleType); - - clusteringKeyTypes = Collections.unmodifiableList(builder); - nameToTypeMap = Collections.unmodifiableMap(mapBuilder); - } - - @SuppressWarnings("unchecked") - public static <T> Generator<T> fromValues(Collection<T> allValues) - { - return fromValues((T[]) allValues.toArray()); - } - - public static <T> Generator<T> fromValues(T[] allValues) - { - return (rng) -> { - return allValues[rng.nextInt(allValues.length - 1)]; - }; - } - - @SuppressWarnings("unchecked") - public static Generator<ColumnSpec<?>> columnSpecGenerator(String prefix, ColumnSpec.Kind kind) - { - return fromValues(columnTypes) - .map(new Function<ColumnSpec.DataType<?>, ColumnSpec<?>>() - { - private int counter = 0; - - public ColumnSpec<?> apply(ColumnSpec.DataType<?> type) - { - return new ColumnSpec<>(prefix + (counter++), - type, - kind); - } - }); - } - - @SuppressWarnings("unchecked") - public static Generator<ColumnSpec<?>> columnSpecGenerator(Collection<ColumnSpec.DataType<?>> columnTypes, String prefix, ColumnSpec.Kind kind) - { - return fromValues(columnTypes) - .map(new Function<ColumnSpec.DataType<?>, ColumnSpec<?>>() - { - private int counter = 0; - - public ColumnSpec<?> apply(ColumnSpec.DataType<?> type) - { - return new ColumnSpec<>(String.format("%s%04d", prefix, counter++), - type, - kind); - } - }); - } - - @SuppressWarnings("unchecked") - public static Generator<ColumnSpec<?>> clusteringColumnSpecGenerator(String prefix) - { - return fromValues(clusteringKeyTypes) - .map(new Function<ColumnSpec.DataType<?>, ColumnSpec<?>>() - { - private int counter = 0; - - public ColumnSpec<?> apply(ColumnSpec.DataType<?> type) - { - return ColumnSpec.ck(String.format("%s%04d", prefix, counter++), type); - } - }); - } - - @SuppressWarnings("unchecked") - public static Generator<ColumnSpec<?>> partitionColumnSpecGenerator(String prefix) - { - return fromValues(partitionKeyTypes) - .map(new Function<ColumnSpec.DataType<?>, ColumnSpec<?>>() - { - private int counter = 0; - - public ColumnSpec<?> apply(ColumnSpec.DataType<?> type) - { - - return ColumnSpec.pk(String.format("%s%04d", prefix, counter++), - type); - } - }); - } - - private static AtomicInteger tableCounter = new AtomicInteger(1); - - public static class Builder - { - private final String keyspace; - private final Supplier<String> tableNameSupplier; - - private Generator<ColumnSpec<?>> pkGenerator = partitionColumnSpecGenerator("pk"); - private Generator<ColumnSpec<?>> ckGenerator = clusteringColumnSpecGenerator("ck"); - private Generator<ColumnSpec<?>> regularGenerator = columnSpecGenerator("regular", ColumnSpec.Kind.REGULAR); - private Generator<ColumnSpec<?>> staticGenerator = columnSpecGenerator("static", ColumnSpec.Kind.STATIC); - - private int minPks = 1; - private int maxPks = 1; - private int minCks = 0; - private int maxCks = 0; - private int minRegular = 0; - private int maxRegular = 0; - private int minStatic = 0; - private int maxStatic = 0; - - public Builder(String keyspace) - { - this(keyspace, () -> "table_" + tableCounter.getAndIncrement()); - } - - public Builder(String keyspace, Supplier<String> tableNameSupplier) - { - this.keyspace = keyspace; - this.tableNameSupplier = tableNameSupplier; - } - - public Builder partitionKeyColumnCount(int numCols) - { - return partitionKeyColumnCount(numCols, numCols); - } - - public Builder partitionKeyColumnCount(int minCols, int maxCols) - { - this.minPks = minCols; - this.maxPks = maxCols; - return this; - } - - public Builder partitionKeySpec(int minCols, int maxCols, ColumnSpec.DataType<?>... columnTypes) - { - return partitionKeySpec(minCols, maxCols, Arrays.asList(columnTypes)); - } - - public Builder partitionKeySpec(int minCols, int maxCols, Collection<ColumnSpec.DataType<?>> columnTypes) - { - this.minPks = minCols; - this.maxPks = maxCols; - this.pkGenerator = columnSpecGenerator(columnTypes, "pk", ColumnSpec.Kind.PARTITION_KEY); - return this; - } - - public Builder clusteringColumnCount(int numCols) - { - return clusteringColumnCount(numCols, numCols); - } - - public Builder clusteringColumnCount(int minCols, int maxCols) - { - this.minCks = minCols; - this.maxCks = maxCols; - return this; - } - - public Builder clusteringKeySpec(int minCols, int maxCols, ColumnSpec.DataType<?>... columnTypes) - { - return clusteringKeySpec(minCols, maxCols, Arrays.asList(columnTypes)); - } - - public Builder clusteringKeySpec(int minCols, int maxCols, Collection<ColumnSpec.DataType<?>> columnTypes) - { - this.minCks = minCols; - this.maxCks = maxCols; - this.ckGenerator = columnSpecGenerator(columnTypes, "ck", ColumnSpec.Kind.CLUSTERING); - return this; - } - - public Builder regularColumnCount(int minCols, int maxCols) - { - this.minRegular = minCols; - this.maxRegular = maxCols; - return this; - } - - public Builder regularColumnCount(int numCols) - { - return regularColumnCount(numCols, numCols); - } - - public Builder regularColumnSpec(int minCols, int maxCols, ColumnSpec.DataType<?>... columnTypes) - { - return this.regularColumnSpec(minCols, maxCols, Arrays.asList(columnTypes)); - } - - public Builder regularColumnSpec(int minCols, int maxCols, Collection<ColumnSpec.DataType<?>> columnTypes) - { - this.minRegular = minCols; - this.maxRegular = maxCols; - this.regularGenerator = columnSpecGenerator(columnTypes, "regular", ColumnSpec.Kind.REGULAR); - return this; - } - - public Builder staticColumnCount(int minCols, int maxCols) - { - this.minStatic = minCols; - this.maxStatic = maxCols; - return this; - } - - public Builder staticColumnCount(int numCols) - { - return staticColumnCount(numCols, numCols); - } - - public Builder staticColumnSpec(int minCols, int maxCols, ColumnSpec.DataType<?>... columnTypes) - { - return this.staticColumnSpec(minCols, maxCols, Arrays.asList(columnTypes)); - } - - public Builder staticColumnSpec(int minCols, int maxCols, Collection<ColumnSpec.DataType<?>> columnTypes) - { - this.minStatic = minCols; - this.maxStatic = maxCols; - this.staticGenerator = columnSpecGenerator(columnTypes, "static", ColumnSpec.Kind.STATIC); - return this; - } - - private static class ColumnCounts - { - private final int pks; - private final int cks; - private final int regulars; - private final int statics; - - private ColumnCounts(int pks, int cks, int regulars, int statics) - { - this.pks = pks; - this.cks = cks; - this.regulars = regulars; - this.statics = statics; - } - } - - public Generator<ColumnCounts> columnCountsGenerator() - { - return (rand) -> { - int pks = rand.nextInt(minPks, maxPks); - int cks = rand.nextInt(minCks, maxCks); - int regulars = rand.nextInt(minRegular, maxRegular); - int statics = rand.nextInt(minStatic, maxStatic); - - return new ColumnCounts(pks, cks, regulars, statics); - }; - } - - public Generator<SchemaSpec> generator() - { - Generator<ColumnCounts> columnCountsGenerator = columnCountsGenerator(); - - return columnCountsGenerator.flatMap(counts -> { - return rand -> { - List<ColumnSpec<?>> pk = pkGenerator.generate(rand, counts.pks); - List<ColumnSpec<?>> ck = ckGenerator.generate(rand, counts.cks); - return new SchemaSpec(keyspace, - tableNameSupplier.get(), - pk, - ck, - regularGenerator.generate(rand, counts.regulars), - staticGenerator.generate(rand, counts.statics)); - }; - }); - } - - public Surjections.Surjection<SchemaSpec> surjection() - { - return generator().toSurjection(SCHEMAGEN_STREAM_ID); - } - } - - public static Surjections.Surjection<SchemaSpec> defaultSchemaSpecGen(String table) - { - return new SchemaGenerators.Builder(DEFAULT_KEYSPACE_NAME, () -> table) - .partitionKeySpec(1, 3, - partitionKeyTypes) - .clusteringKeySpec(1, 3, - clusteringKeyTypes) - .regularColumnSpec(3, 5, - ColumnSpec.int8Type, - ColumnSpec.int16Type, - ColumnSpec.int32Type, - ColumnSpec.int64Type, - ColumnSpec.floatType, - ColumnSpec.doubleType, - ColumnSpec.asciiType(5, 256)) - .staticColumnSpec(3, 5, - ColumnSpec.int8Type, - ColumnSpec.int16Type, - ColumnSpec.int32Type, - ColumnSpec.int64Type, - ColumnSpec.floatType, - ColumnSpec.doubleType, - ColumnSpec.asciiType(4, 512), - ColumnSpec.asciiType(4, 2048)) - .surjection(); - } - - public static String DEFAULT_KEYSPACE_NAME = "harry"; - - private static final String DEFAULT_PREFIX = "table_"; - private static final AtomicInteger counter = new AtomicInteger(); - private static final Supplier<String> tableNameSupplier = () -> DEFAULT_PREFIX + counter.getAndIncrement(); - - // simplest schema gen, nothing can go wrong with it - public static final Surjections.Surjection<SchemaSpec> longOnlySpecBuilder = new Builder(DEFAULT_KEYSPACE_NAME, tableNameSupplier) - .partitionKeySpec(1, 1, ColumnSpec.int64Type) - .clusteringKeySpec(1, 1, ColumnSpec.int64Type) - .regularColumnSpec(1, 10, ColumnSpec.int64Type) - .staticColumnSpec(1, 10, ColumnSpec.int64Type) - .surjection(); - - private static final ColumnSpec.DataType<String> simpleStringType = ColumnSpec.asciiType(4, 10); - private static final Surjections.Surjection<SchemaSpec> longAndStringSpecBuilder = new SchemaGenerators.Builder(DEFAULT_KEYSPACE_NAME, tableNameSupplier) - .partitionKeySpec(2, 2, ColumnSpec.int64Type, simpleStringType) - .clusteringKeySpec(2, 2, ColumnSpec.int64Type, simpleStringType) - .regularColumnSpec(1, 10, ColumnSpec.int64Type, simpleStringType) - .staticColumnSpec(1, 10, ColumnSpec.int64Type) - .surjection(); - - public static final Surjections.Surjection<SchemaSpec> longOnlyWithReverseSpecBuilder = new SchemaGenerators.Builder(DEFAULT_KEYSPACE_NAME, tableNameSupplier) - .partitionKeySpec(1, 1, ColumnSpec.int64Type) - .clusteringKeySpec(1, 1, ColumnSpec.ReversedType.getInstance(ColumnSpec.int64Type)) - .regularColumnSpec(1, 10, ColumnSpec.int64Type) - .staticColumnSpec(1, 10, ColumnSpec.int64Type) - .surjection(); - - public static final Surjections.Surjection<SchemaSpec> longAndStringSpecWithReversedLongBuilder = new SchemaGenerators.Builder(DEFAULT_KEYSPACE_NAME, tableNameSupplier) - .partitionKeySpec(2, 2, ColumnSpec.int64Type, simpleStringType) - .clusteringKeySpec(2, 2, ColumnSpec.ReversedType.getInstance(ColumnSpec.int64Type), simpleStringType) - .regularColumnSpec(1, 10, ColumnSpec.int64Type, simpleStringType) - .staticColumnSpec(1, 10, ColumnSpec.int64Type) - .surjection(); - - public static final Surjections.Surjection<SchemaSpec> longAndStringSpecWithReversedStringBuilder = new SchemaGenerators.Builder(DEFAULT_KEYSPACE_NAME, tableNameSupplier) - .partitionKeySpec(2, 2, ColumnSpec.int64Type, simpleStringType) - .clusteringKeySpec(2, 2, ColumnSpec.int64Type, ColumnSpec.ReversedType.getInstance(simpleStringType)) - .regularColumnSpec(1, 10, ColumnSpec.int64Type, simpleStringType) - .staticColumnSpec(1, 10, ColumnSpec.int64Type) - .surjection(); - - public static final Surjections.Surjection<SchemaSpec> longAndStringSpecWithReversedBothBuilder = new SchemaGenerators.Builder(DEFAULT_KEYSPACE_NAME, tableNameSupplier) - .partitionKeySpec(2, 2, ColumnSpec.int64Type, simpleStringType) - .clusteringKeySpec(2, 2, ColumnSpec.ReversedType.getInstance(ColumnSpec.int64Type), ColumnSpec.ReversedType.getInstance(simpleStringType)) - .regularColumnSpec(1, 10, ColumnSpec.int64Type, simpleStringType) - .staticColumnSpec(1, 10, ColumnSpec.int64Type) - .surjection(); - - public static final Surjections.Surjection<SchemaSpec> withAllFeaturesEnabled = new SchemaGenerators.Builder(DEFAULT_KEYSPACE_NAME, tableNameSupplier) - .partitionKeySpec(1, 4, columnTypes) - .clusteringKeySpec(1, 4, clusteringKeyTypes) - .regularColumnSpec(1, 10, columnTypes) - .surjection(); - - public static final Surjections.Surjection<SchemaSpec>[] PROGRESSIVE_GENERATORS = new Surjections.Surjection[]{ - longOnlySpecBuilder, - longAndStringSpecBuilder, - longOnlyWithReverseSpecBuilder, - longAndStringSpecWithReversedLongBuilder, - longAndStringSpecWithReversedStringBuilder, - longAndStringSpecWithReversedBothBuilder, - withAllFeaturesEnabled - }; - - // Create schema generators that would produce tables starting with just a few features, progressing to use more - public static Supplier<SchemaSpec> progression(int switchAfter) - { - Supplier<SchemaSpec>[] generators = new Supplier[PROGRESSIVE_GENERATORS.length]; - for (int i = 0; i < generators.length; i++) - generators[i] = PROGRESSIVE_GENERATORS[i].toSupplier(); - - return new Supplier<SchemaSpec>() - { - private int counter = 0; - public SchemaSpec get() - { - int idx = (counter / switchAfter) % generators.length; - counter++; - SchemaSpec spec = generators[idx].get(); - int tries = 100; - while ((spec.pkGenerator.byteSize() != Long.BYTES) && tries > 0) - { - System.out.println("Skipping schema, since it doesn't have enough entropy bits available: " + spec.compile().cql()); - spec = generators[idx].get(); - tries--; - } - - spec.validate(); - - assert tries > 0 : String.format("Max number of tries exceeded on generator %d, can't generate a needed schema", idx); - return spec; - } - }; - } - - public static List<ColumnSpec<?>> toColumns(Map<String, String> config, ColumnSpec.Kind kind, boolean allowReverse) - { - if (config == null) - return Collections.EMPTY_LIST; - - List<ColumnSpec<?>> columns = new ArrayList<>(config.size()); - - for (Map.Entry<String, String> e : config.entrySet()) - { - ColumnSpec.DataType<?> type = nameToTypeMap.get(e.getValue()); - assert type != null : "Can't parse the type"; - assert allowReverse || !type.isReversed() : String.format("%s columns aren't allowed to be reversed", type); - columns.add(new ColumnSpec<>(e.getKey(), type, kind)); - } - - return columns; - } - - public static SchemaSpec parse(String keyspace, - String table, - Map<String, String> pks, - Map<String, String> cks, - Map<String, String> regulars, - Map<String, String> statics) - { - return new SchemaSpec(keyspace, table, - toColumns(pks, ColumnSpec.Kind.PARTITION_KEY, false), - toColumns(cks, ColumnSpec.Kind.CLUSTERING, false), - toColumns(regulars, ColumnSpec.Kind.REGULAR, false), - toColumns(statics, ColumnSpec.Kind.STATIC, false)); - } - - public static int DEFAULT_SWITCH_AFTER = CassandraRelevantProperties.TEST_HARRY_SWITCH_AFTER.getInt(); - public static int GENERATORS_COUNT = PROGRESSIVE_GENERATORS.length; - public static int DEFAULT_RUNS = DEFAULT_SWITCH_AFTER * GENERATORS_COUNT; -} diff --git a/test/harry/main/org/apache/cassandra/harry/ddl/SchemaSpec.java b/test/harry/main/org/apache/cassandra/harry/ddl/SchemaSpec.java deleted file mode 100644 index 407059dfce20..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/ddl/SchemaSpec.java +++ /dev/null @@ -1,531 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.ddl; - -import java.util.*; -import java.util.function.Consumer; - -import org.apache.cassandra.harry.gen.DataGenerators; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.operations.CompiledStatement; -import org.apache.cassandra.harry.operations.Relation; -import org.apache.cassandra.harry.util.BitSet; - -public class SchemaSpec -{ - public interface SchemaSpecFactory - { - SchemaSpec make(long seed, SystemUnderTest sut); - } - - public final DataGenerators.KeyGenerator pkGenerator; - public final DataGenerators.KeyGenerator ckGenerator; - - private final boolean isCompactStorage; - private final boolean disableReadRepair; - private final String compactionStrategy; - public final boolean trackLts; - - // These fields are immutable, and are safe as public - public final String keyspace; - public final String table; - - public final List<ColumnSpec<?>> partitionKeys; - public final List<ColumnSpec<?>> clusteringKeys; - public final List<ColumnSpec<?>> regularColumns; - public final List<ColumnSpec<?>> staticColumns; - public final List<ColumnSpec<?>> allColumns; - public final Set<ColumnSpec<?>> allColumnsSet; - - public final BitSet ALL_COLUMNS_BITSET; - public final int regularColumnsOffset; - public final int staticColumnsOffset; - public final BitSet regularColumnsMask; - public final BitSet regularAndStaticColumnsMask; - public final BitSet staticColumnsMask; - - public SchemaSpec(String keyspace, - String table, - List<ColumnSpec<?>> partitionKeys, - List<ColumnSpec<?>> clusteringKeys, - List<ColumnSpec<?>> regularColumns, - List<ColumnSpec<?>> staticColumns) - { - this(keyspace, table, partitionKeys, clusteringKeys, regularColumns, staticColumns, DataGenerators.createKeyGenerator(clusteringKeys), false, false, null, false); - } - - public SchemaSpec cloneWithName(String ks, - String table) - { - return new SchemaSpec(ks, table, partitionKeys, clusteringKeys, regularColumns, staticColumns, ckGenerator, isCompactStorage, disableReadRepair, compactionStrategy, trackLts); - } - - public SchemaSpec trackLts() - { - return new SchemaSpec(keyspace, table, partitionKeys, clusteringKeys, regularColumns, staticColumns, ckGenerator, isCompactStorage, disableReadRepair, compactionStrategy, true); - } - - public SchemaSpec withCompactStorage() - { - return new SchemaSpec(keyspace, table, partitionKeys, clusteringKeys, regularColumns, staticColumns, ckGenerator, true, disableReadRepair, compactionStrategy, trackLts); - } - - public SchemaSpec withCompactionStrategy(String compactionStrategy) - { - return new SchemaSpec(keyspace, table, partitionKeys, clusteringKeys, regularColumns, staticColumns, ckGenerator, false, disableReadRepair, compactionStrategy, trackLts); - } - - public SchemaSpec withCkGenerator(DataGenerators.KeyGenerator ckGeneratorOverride, List<ColumnSpec<?>> clusteringKeys) - { - return new SchemaSpec(keyspace, table, partitionKeys, clusteringKeys, regularColumns, staticColumns, ckGeneratorOverride, isCompactStorage, disableReadRepair, compactionStrategy, trackLts); - } - - public SchemaSpec withColumns(List<ColumnSpec<?>> regularColumns, List<ColumnSpec<?>> staticColumns) - { - return new SchemaSpec(keyspace, table, partitionKeys, clusteringKeys, regularColumns, staticColumns, ckGenerator, isCompactStorage, disableReadRepair, compactionStrategy, trackLts); - } - - public SchemaSpec(String keyspace, - String table, - List<ColumnSpec<?>> partitionKeys, - List<ColumnSpec<?>> clusteringKeys, - List<ColumnSpec<?>> regularColumns, - List<ColumnSpec<?>> staticColumns, - DataGenerators.KeyGenerator ckGenerator, - boolean isCompactStorage, - boolean disableReadRepair, - String compactionStrategy, - boolean trackLts) - { - assert !isCompactStorage || clusteringKeys.isEmpty() || regularColumns.size() <= 1 : - String.format("Compact storage %s. Clustering keys: %d. Regular columns: %d", isCompactStorage, clusteringKeys.size(), regularColumns.size()); - - this.keyspace = keyspace; - this.table = table; - this.isCompactStorage = isCompactStorage; - this.disableReadRepair = disableReadRepair; - this.compactionStrategy = compactionStrategy; - - this.partitionKeys = Collections.unmodifiableList(new ArrayList<>(partitionKeys)); - for (int i = 0; i < partitionKeys.size(); i++) - partitionKeys.get(i).setColumnIndex(i); - this.clusteringKeys = Collections.unmodifiableList(new ArrayList<>(clusteringKeys)); - for (int i = 0; i < clusteringKeys.size(); i++) - clusteringKeys.get(i).setColumnIndex(i); - this.staticColumns = Collections.unmodifiableList(new ArrayList<>(staticColumns)); - for (int i = 0; i < staticColumns.size(); i++) - staticColumns.get(i).setColumnIndex(i); - this.regularColumns = Collections.unmodifiableList(new ArrayList<>(regularColumns)); - for (int i = 0; i < regularColumns.size(); i++) - regularColumns.get(i).setColumnIndex(i); - - List<ColumnSpec<?>> all = new ArrayList<>(); - for (ColumnSpec<?> columnSpec : concat(partitionKeys, - clusteringKeys, - staticColumns, - regularColumns)) - { - all.add(columnSpec); - } - this.allColumns = Collections.unmodifiableList(all); - this.allColumnsSet = Collections.unmodifiableSet(new LinkedHashSet<>(all)); - - this.pkGenerator = DataGenerators.createKeyGenerator(partitionKeys); - if (ckGenerator == null) - ckGenerator = DataGenerators.createKeyGenerator(clusteringKeys); - this.ckGenerator = ckGenerator; - - this.ALL_COLUMNS_BITSET = BitSet.allSet(regularColumns.size()); - - this.staticColumnsOffset = partitionKeys.size() + clusteringKeys.size(); - this.regularColumnsOffset = staticColumnsOffset + staticColumns.size(); - - this.regularColumnsMask = regularColumnsMask(this); - this.regularAndStaticColumnsMask = regularAndStaticColumnsMask(this); - this.staticColumnsMask = staticColumnsMask(this); - this.trackLts = trackLts; - } - - - - public static BitSet allColumnsMask(SchemaSpec schema) - { - return BitSet.allSet(schema.allColumns.size()); - } - - public BitSet regularColumnsMask() - { - return this.regularColumnsMask; - } - - public BitSet regularAndStaticColumnsMask() - { - return this.regularAndStaticColumnsMask; - } - - public BitSet staticColumnsMask() - { - return this.staticColumnsMask; - } - - private static BitSet regularColumnsMask(SchemaSpec schema) - { - BitSet mask = BitSet.allUnset(schema.allColumns.size()); - for (int i = 0; i < schema.regularColumns.size(); i++) - mask.set(schema.regularColumnsOffset + i); - return mask; - } - - private static BitSet regularAndStaticColumnsMask(SchemaSpec schema) - { - BitSet mask = BitSet.allUnset(schema.allColumns.size()); - for (int i = 0; i < schema.staticColumns.size() + schema.regularColumns.size(); i++) - mask.set(schema.staticColumnsOffset + i); - return mask; - } - - private static BitSet staticColumnsMask(SchemaSpec schema) - { - BitSet mask = BitSet.allUnset(schema.allColumns.size()); - for (int i = 0; i < schema.staticColumns.size(); i++) - mask.set(schema.staticColumnsOffset + i); - return mask; - } - - public void validate() - { - assert pkGenerator.byteSize() == Long.BYTES : partitionKeys.toString(); - } - - public interface AddRelationCallback - { - void accept(ColumnSpec<?> spec, Relation.RelationKind kind, Object value); - } - - public void inflateRelations(long pd, - List<Relation> clusteringRelations, - AddRelationCallback consumer) - { - Object[] pk = inflatePartitionKey(pd); - for (int i = 0; i < pk.length; i++) - consumer.accept(partitionKeys.get(i), Relation.RelationKind.EQ, pk[i]); - - inflateRelations(clusteringRelations, consumer); - } - - public void inflateRelations(List<Relation> clusteringRelations, - AddRelationCallback consumer) - { - for (Relation r : clusteringRelations) - consumer.accept(r.columnSpec, r.kind, r.value()); - } - - public Object[] inflatePartitionKey(long pd) - { - return pkGenerator.inflate(pd); - } - - public Object[] inflateClusteringKey(long cd) - { - return ckGenerator.inflate(cd); - } - - public Object[] inflateRegularColumns(long[] vds) - { - return DataGenerators.inflateData(regularColumns, vds); - } - - public Object[] inflateStaticColumns(long[] sds) - { - return DataGenerators.inflateData(staticColumns, sds); - } - - public long adjustPdEntropy(long descriptor) - { - return pkGenerator.adjustEntropyDomain(descriptor); - } - - public long adjustCdEntropy(long descriptor) - { - return ckGenerator.adjustEntropyDomain(descriptor); - } - - public long deflatePartitionKey(Object[] pk) - { - return pkGenerator.deflate(pk); - } - - public long deflateClusteringKey(Object[] ck) - { - return ckGenerator.deflate(ck); - } - - public long[] deflateStaticColumns(Object[] statics) - { - return DataGenerators.deflateData(staticColumns, statics); - } - - public long[] deflateRegularColumns(Object[] regulars) - { - return DataGenerators.deflateData(regularColumns, regulars); - } - - public CompiledStatement compile() - { - StringBuilder sb = new StringBuilder(); - - sb.append("CREATE TABLE IF NOT EXISTS "); - sb.append(keyspace) - .append(".") - .append(table) - .append(" ("); - - SeparatorAppender commaAppender = new SeparatorAppender(); - for (ColumnSpec<?> cd : partitionKeys) - { - commaAppender.accept(sb); - sb.append(cd.toCQL()); - if (partitionKeys.size() == 1 && clusteringKeys.size() == 0) - sb.append(" PRIMARY KEY"); - } - - for (ColumnSpec<?> cd : concat(clusteringKeys, - staticColumns, - regularColumns)) - { - commaAppender.accept(sb); - sb.append(cd.toCQL()); - } - - if (clusteringKeys.size() > 0 || partitionKeys.size() > 1) - { - sb.append(", ").append(getPrimaryKeyCql()); - } - - if (trackLts) - sb.append(", ").append("visited_lts list<bigint> static"); - - sb.append(')'); - - Runnable appendWith = doOnce(() -> sb.append(" WITH")); - - if (isCompactStorage) - { - appendWith.run(); - sb.append(" COMPACT STORAGE AND"); - } - - if (disableReadRepair) - { - appendWith.run(); - sb.append(" read_repair = 'NONE' AND"); - } - - if (compactionStrategy != null) - { - appendWith.run(); - sb.append(" compaction = {'class': '").append(compactionStrategy).append("'} AND"); - } - - if (clusteringKeys.size() > 0) - { - appendWith.run(); - sb.append(getClusteringOrderCql()) - .append(';'); - } - - return new CompiledStatement(sb.toString()); - } - - private String getClusteringOrderCql() - { - StringBuilder sb = new StringBuilder(); - if (clusteringKeys.size() > 0) - { - sb.append(" CLUSTERING ORDER BY ("); - - SeparatorAppender commaAppender = new SeparatorAppender(); - for (ColumnSpec<?> column : clusteringKeys) - { - commaAppender.accept(sb); - sb.append(column.name).append(' ').append(column.isReversed() ? "DESC" : "ASC"); - } - - sb.append(")"); - } - - return sb.toString(); - } - - private String getPrimaryKeyCql() - { - StringBuilder sb = new StringBuilder(); - sb.append("PRIMARY KEY ("); - if (partitionKeys.size() > 1) - { - sb.append('('); - SeparatorAppender commaAppender = new SeparatorAppender(); - for (ColumnSpec<?> cd : partitionKeys) - { - commaAppender.accept(sb); - sb.append(cd.name); - } - sb.append(')'); - } - else - { - sb.append(partitionKeys.get(0).name); - } - - for (ColumnSpec<?> cd : clusteringKeys) - sb.append(", ").append(cd.name); - - return sb.append(')').toString(); - } - - public String toString() - { - return String.format("schema {cql=%s, columns=%s}", compile().toString(), allColumns); - } - - private static Runnable doOnce(Runnable r) - { - return new Runnable() - { - boolean executed = false; - - public void run() - { - if (executed) - return; - - executed = true; - r.run(); - } - }; - } - - public static class SeparatorAppender implements Consumer<StringBuilder> - { - boolean isFirst = true; - private final String separator; - - public SeparatorAppender() - { - this(","); - } - - public SeparatorAppender(String separator) - { - this.separator = separator; - } - - public void accept(StringBuilder stringBuilder) - { - if (isFirst) - isFirst = false; - else - stringBuilder.append(separator); - } - - public void accept(StringBuilder stringBuilder, String s) - { - accept(stringBuilder); - stringBuilder.append(s); - } - - - public void reset() - { - isFirst = true; - } - } - - public boolean equals(Object o) - { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - SchemaSpec that = (SchemaSpec) o; - return Objects.equals(keyspace, that.keyspace) && - Objects.equals(table, that.table) && - Objects.equals(partitionKeys, that.partitionKeys) && - Objects.equals(clusteringKeys, that.clusteringKeys) && - Objects.equals(regularColumns, that.regularColumns); - } - - public int hashCode() - { - return Objects.hash(keyspace, table, partitionKeys, clusteringKeys, regularColumns); - } - - public static <T> Iterable<T> concat(Iterable<T>... iterables) - { - assert iterables != null && iterables.length > 0; - if (iterables.length == 1) - return iterables[0]; - - return () -> { - return new Iterator<T>() - { - int idx; - Iterator<T> current; - boolean hasNext; - - { - idx = 0; - prepareNext(); - } - - private void prepareNext() - { - if (current != null && current.hasNext()) - { - hasNext = true; - return; - } - - while (idx < iterables.length) - { - current = iterables[idx].iterator(); - idx++; - if (current.hasNext()) - { - hasNext = true; - return; - } - } - - hasNext = false; - } - - public boolean hasNext() - { - return hasNext; - } - - public T next() - { - T next = current.next(); - prepareNext(); - return next; - } - }; - }; - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/dsl/BatchOperationBuilder.java b/test/harry/main/org/apache/cassandra/harry/dsl/BatchOperationBuilder.java deleted file mode 100644 index 50321fb39808..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/dsl/BatchOperationBuilder.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.dsl; - -public interface BatchOperationBuilder -{ - SingleOperationBuilder beginBatch(); - - /** - * Begin batch for a partition descriptor at a specific index. - * - * Imagine all partition descriptors were longs in an array. Index of a descriptor - * is a sequential number of the descriptor in this imaginary array. - */ - SingleOperationBuilder beginBatch(long pdIdx); -} diff --git a/test/harry/main/org/apache/cassandra/harry/dsl/BatchVisitBuilder.java b/test/harry/main/org/apache/cassandra/harry/dsl/BatchVisitBuilder.java deleted file mode 100644 index 7de5ccf31e79..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/dsl/BatchVisitBuilder.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.dsl; - -import java.io.Closeable; -import java.util.function.Consumer; - -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.visitors.ReplayingVisitor; - -public class BatchVisitBuilder extends SingleOperationVisitBuilder implements Closeable -{ - private final HistoryBuilder historyBuilder; - - public BatchVisitBuilder(HistoryBuilder historyBuilder, - PartitionVisitStateImpl partitionState, - long lts, - OpSelectors.PureRng rng, - OpSelectors.DescriptorSelector descriptorSelector, - SchemaSpec schema, - ValueHelper valueHelper, - Consumer<ReplayingVisitor.Visit> appendToLog) - { - super(partitionState, lts, rng, descriptorSelector, schema, valueHelper, appendToLog); - this.historyBuilder = historyBuilder; - } - - @Override - public int size() - { - return super.size(); - } - - @Override - public BatchVisitBuilder insert() - { - super.insert(); - return this; - } - - public BatchVisitBuilder inserts(int n) - { - assert n > 0; - for (int i = 0; i < n; i++) - insert(); - return this; - } - - @Override - public BatchVisitBuilder insert(int rowIdx) - { - super.insert(rowIdx); - return this; - } - - @Override - public BatchVisitBuilder insert(int rowIdx, long[] valueIdxs) - { - super.insert(rowIdx, valueIdxs); - return this; - } - - @Override - public BatchVisitBuilder deletePartition() - { - super.deletePartition(); - return this; - } - - @Override - public BatchVisitBuilder deleteRow() - { - super.deleteRow(); - return this; - } - - @Override - public BatchVisitBuilder deleteColumns() - { - super.deleteColumns(); - return this; - } - - @Override - public BatchVisitBuilder deleteRowRange() - { - super.deleteRowRange(); - return this; - } - - @Override - public BatchVisitBuilder deleteRowRange(int lowBoundRowIdx, int highBoundRowIdx, boolean isMinEq, boolean isMaxEq) - { - super.deleteRowRange(lowBoundRowIdx, highBoundRowIdx, isMinEq, isMaxEq); - return this; - } - - @Override - public BatchVisitBuilder deleteRowSlice() - { - super.deleteRowSlice(); - return this; - } - - // TODO: prevent from closing more than once - public HistoryBuilder endBatch() - { - super.end(); - return this.historyBuilder; - } - - /** - * Implements closeable to instruct users to end batch before using. - * - * Non-finished batches are _not_ appended to the history and will appear as gaps in history. - */ - @Override - public void close() - { - endBatch(); - } -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/dsl/HistoryBuilder.java b/test/harry/main/org/apache/cassandra/harry/dsl/HistoryBuilder.java index a5d5fdafe219..35631d55a0fa 100644 --- a/test/harry/main/org/apache/cassandra/harry/dsl/HistoryBuilder.java +++ b/test/harry/main/org/apache/cassandra/harry/dsl/HistoryBuilder.java @@ -1,191 +1,84 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.cassandra.harry.dsl; import java.util.ArrayList; import java.util.Arrays; +import java.util.Comparator; import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.NavigableSet; -import java.util.Set; -import java.util.TreeSet; -import java.util.function.Consumer; -import java.util.function.Function; -import java.util.function.LongSupplier; - -import org.apache.cassandra.harry.clock.ApproximateClock; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.MetricReporter; -import org.apache.cassandra.harry.ddl.SchemaSpec; +import java.util.stream.Collectors; + +import org.apache.cassandra.harry.ColumnSpec; +import org.apache.cassandra.harry.MagicConstants; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.gen.Bijections; import org.apache.cassandra.harry.gen.EntropySource; +import org.apache.cassandra.harry.gen.IndexGenerators; +import org.apache.cassandra.harry.gen.InvertibleGenerator; +import org.apache.cassandra.harry.gen.ValueGenerators; import org.apache.cassandra.harry.gen.rng.JdkRandomEntropySource; +import org.apache.cassandra.harry.op.Operations; +import org.apache.cassandra.harry.op.Visit; import org.apache.cassandra.harry.model.Model; -import org.apache.cassandra.harry.model.NoOpChecker; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.model.QuiescentChecker; -import org.apache.cassandra.harry.model.reconciler.Reconciler; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.sut.injvm.QuiescentLocalStateChecker; -import org.apache.cassandra.harry.tracker.DataTracker; -import org.apache.cassandra.harry.visitors.MutatingRowVisitor; -import org.apache.cassandra.harry.visitors.MutatingVisitor; -import org.apache.cassandra.harry.visitors.ReplayingVisitor; -import org.apache.cassandra.harry.visitors.VisitExecutor; - -/** - * History builder is a component for a simple yet flexible generation of arbitrary data. You can write queries - * _as if_ you were writing them with primitive values (such as 0,1, and using for loops, and alike). - * - * You can create a history builder like: - * - * HistoryBuilder historyBuilder = new HistoryBuilder(seed, maxPartitionSize, 10, schema); - * - * The core idea is that you use simple-to-remember numbers as placeholders for your values. For partition key, - * the value (such as 0,1,2,...) would signify a distinct partition key for a given schema. You can not know in - * advance the relative order of two generated partition keys (i.e. how they'd sort). - * - * For clustering keys, the value 0 signifies the smallest possible-to-generate clustering _for this partition_ - * (i.e. there may be other values that would sort LT relative to it, but they will never be generated in this - * context. Similarly, `maxPartitionSize - 1` is going to be the largest possible-to-generate clustering _for this - * partition_). All other values (i.e. between 0 and maxPartitionSize - 1) that will be generated are ordered in - * the same way as the numbers you used to generate them. This is done for your convenience and being able to create - * complex/interesting RT queries. - * - * You can also go arbitrarily deep into specifying details of your query. For example, calling - * - * historyBuilder.insert(); - * - * Will generate an INSERT query, according to the given schema, for a random partition, random clustering, with - * random values. At the same time, calling: - * - * historyBuilder.visitPartition(1).insert(); - * - * Will generate an insert for a partition whose partition key is under index 1 (generating other writes prefixed - * with `visitPartition(1)` will ensure operations are executed against the same partition). Clustering and - * values are still going to be random. Calling: - * - * historyBuilder.visitPartition(1).insert(2); - * - * Will generate an insert for a partition whose partition key is under index 1, and the clustering will be third- - * largest possible clustering for this partition (remember, 0 is smallest, so 0,1,2 - third). Values inserted into - * this row are still going to be random. - * - * Lastly, calling - * - * historyBuilder.visitPartition(1).insert(2, new long[] { 1, 2 }); - * - * Will generate an insert to 1st partition, 2nd row, and the values are going to be taken from the random - * streams for the values for corresponding columns. - * - * Other possible operations are deleteRow, deleteColumns, deleteRowRange, deleteRowSlide, and deletePartition. - * - * HistoryBuilder also allows hardcoding/overriding clustering keys, regular, and static values, but _not_ for - * partition keys as of now. - * - * Since clusterings are ordered according to their value, it is only possible to instruct generator to ensure - * such value is going to be present. This is done by: - * - * history.forPartition(1).ensureClustering(new Object[]{ "", "b", -1L, "c", "d" }); - * - * For regular and static columns, overrides are done on the top level, not per-partition, so you can simply do: - * - * history.valueOverrides().override(column.name, 1245, "overriden value"); - * - * history.visitPartition(1) - * .insert(1, new long[]{ 12345, 12345 }); - * - * This will insert "overriden value" for the 1st row of 1st partition, for two columns. In other words, the index - * 12345 will now be associated with this overriden value. But all other / random values will still be, random. - */ -public class HistoryBuilder implements Iterable<ReplayingVisitor.Visit>, SingleOperationBuilder, BatchOperationBuilder -{ - protected final OverridingCkGenerator ckGenerator; +import org.apache.cassandra.harry.util.BitSet; +import org.apache.cassandra.harry.util.IteratorsUtil; - protected final SchemaSpec schema; - protected final TokenPlacementModel.ReplicationFactor rf; +import static org.apache.cassandra.harry.SchemaSpec.cumulativeEntropy; +import static org.apache.cassandra.harry.SchemaSpec.forKeys; +import static org.apache.cassandra.harry.gen.InvertibleGenerator.fromType; - protected final OpSelectors.PureRng pureRng; - protected final OpSelectors.DescriptorSelector descriptorSelector; - protected final ValueHelper valueHelper; - // TODO: would be great to have a very simple B-Tree here - protected final Map<Long, ReplayingVisitor.Visit> log; +// TODO: either create or replay timestamps out of order +public class HistoryBuilder implements SingleOperationBuilder, Model.Replay +{ + protected final IndexedValueGenerators valueGenerators; + protected final IndexGenerators indexGenerators; - // TODO: primitive array with a custom/noncopying growth strat - protected final Map<Long, PartitionVisitStateImpl> partitionStates = new HashMap<>(); - protected final Set<Long> visitedPartitions = new HashSet<>(); + protected int nextOpIdx = 0; - /** - * A selector that is going to be used by the model checker. - */ - protected final PresetPdSelector presetSelector; - - /** - * Default selector will select every partition exactly once. - */ - protected final OpSelectors.DefaultPdSelector defaultSelector; - protected final OffsetClock clock; - protected final int maxPartitionSize; + // TODO: would be great to have a very simple B-Tree here + protected final Map<Long, Visit> log; - public HistoryBuilder(long seed, - int maxPartitionSize, - int interleaveWindowSize, - SchemaSpec schema, - TokenPlacementModel.ReplicationFactor rf) + public static HistoryBuilder fromSchema(SchemaSpec schemaSpec, long seed, int population) { - this.maxPartitionSize = maxPartitionSize; - this.log = new HashMap<>(); - this.pureRng = new OpSelectors.PCGFast(seed); - - this.presetSelector = new PresetPdSelector(); - this.ckGenerator = OverridingCkGenerator.make(schema.ckGenerator); - this.valueHelper = new ValueHelper(schema, pureRng); - this.schema = schema.withCkGenerator(this.ckGenerator, this.ckGenerator.columns) - .withColumns(valueHelper.regularColumns, valueHelper.staticColumns); - this.rf = rf; - - // TODO: make clock pluggable - this.clock = new OffsetClock(ApproximateClock.START_VALUE, - interleaveWindowSize, - new JdkRandomEntropySource(seed)); - - this.defaultSelector = new OpSelectors.DefaultPdSelector(pureRng, 1, 1); + IndexedValueGenerators generators = valueGenerators(schemaSpec, seed, population); + return new HistoryBuilder(generators, IndexGenerators.withDefaults(generators)); + } - this.descriptorSelector = new Configuration.CDSelectorConfigurationBuilder() - .setOperationsPerLtsDistribution(new Configuration.ConstantDistributionConfig(Integer.MAX_VALUE)) - .setMaxPartitionSize(maxPartitionSize) - .build() - .make(pureRng, schema); + public HistoryBuilder(ValueGenerators generators) + { + this((IndexedValueGenerators) generators, IndexGenerators.withDefaults(generators)); } - public ValueOverrides valueOverrides() + public HistoryBuilder(IndexedValueGenerators valueGenerators, + IndexGenerators indexGenerators) { - return valueHelper; + this.log = new HashMap<>(); + this.valueGenerators = valueGenerators; + this.indexGenerators = indexGenerators; } - public SchemaSpec schema() + public IndexedValueGenerators valueGenerators() { - return schema; + return valueGenerators; } public int size() @@ -193,483 +86,356 @@ public int size() return log.size(); } - public OpSelectors.Clock clock() + @Override + public Iterator<Visit> iterator() { - return clock; + return new Iterator<>() + { + long replayed = 0; + + public boolean hasNext() + { + return replayed < nextOpIdx; + } + + public Visit next() + { + return log.get(replayed++); + } + }; } - /** - * Visited partition descriptors _not_ in the order they were visited - */ - public List<Long> visitedPds() + @Override + public Visit replay(long lts) { - return new ArrayList<>(visitedPartitions); + return log.get(lts); } @Override - public Iterator<ReplayingVisitor.Visit> iterator() + public Operations.Operation replay(long lts, int opId) { - return log.values().iterator(); + return replay(lts).operations[opId]; } - protected SingleOperationVisitBuilder singleOpVisitBuilder() + SingleOperationVisitBuilder singleOpVisitBuilder() { - long visitLts = clock.nextLts(); - return singleOpVisitBuilder(defaultSelector.pd(visitLts, schema), visitLts, (ps) -> {}); + long visitLts = nextOpIdx++; + return new SingleOperationVisitBuilder(visitLts, + valueGenerators, + indexGenerators, + (visit) -> log.put(visit.lts, visit)); } - protected SingleOperationVisitBuilder singleOpVisitBuilder(long pd, long lts, Consumer<PartitionVisitState> setupPs) + ; + + public MultiOperationVisitBuilder multistep() { - PartitionVisitStateImpl partitionState = presetSelector.register(lts, pd, setupPs); - return new SingleOperationVisitBuilder(partitionState, lts, pureRng, descriptorSelector, schema, valueHelper, (visit) -> { - visitedPartitions.add(pd); - log.put(visit.lts, visit); - }); + long visitLts = nextOpIdx++; + return new MultiOperationVisitBuilder(visitLts, + valueGenerators, + indexGenerators, + visit -> log.put(visit.lts, visit)); } @Override - public HistoryBuilder insert() + public SingleOperationBuilder custom(Runnable runnable, String tag) { - singleOpVisitBuilder().insert(); + singleOpVisitBuilder().custom(runnable, tag); return this; } @Override - public HistoryBuilder insert(int rowId) + public SingleOperationBuilder custom(OperationFactory factory) { - singleOpVisitBuilder().insert(rowId); + singleOpVisitBuilder().custom(factory); return this; } @Override - public HistoryBuilder insert(int rowId, long[] valueIdxs) + public SingleOperationBuilder update() { - singleOpVisitBuilder().insert(rowId, valueIdxs); + singleOpVisitBuilder().update(); return this; } - public SingleOperationBuilder insert(int rowIdx, long[] valueIdxs, long[] sValueIdxs) + @Override + public SingleOperationBuilder update(int pdIdx) { - singleOpVisitBuilder().insert(rowIdx, valueIdxs, sValueIdxs); + singleOpVisitBuilder().update(pdIdx); return this; } @Override - public HistoryBuilder deletePartition() + public SingleOperationBuilder update(int pdIdx, int cdIdx) { - singleOpVisitBuilder().deletePartition(); + singleOpVisitBuilder().update(pdIdx, cdIdx); return this; } @Override - public HistoryBuilder deleteRow() + public SingleOperationBuilder update(int pdIdx, int rowIdx, int[] valueIdxs, int[] sValueIdxs) { - singleOpVisitBuilder().deleteRow(); + singleOpVisitBuilder().update(pdIdx, rowIdx, valueIdxs, sValueIdxs); return this; } @Override - public HistoryBuilder deleteRow(int rowIdx) + public SingleOperationBuilder insert() { - singleOpVisitBuilder().deleteRow(rowIdx); + singleOpVisitBuilder().insert(); return this; } @Override - public HistoryBuilder deleteColumns() + public SingleOperationBuilder insert(int pdIdx) { - singleOpVisitBuilder().deleteColumns(); + singleOpVisitBuilder().insert(pdIdx); return this; } @Override - public HistoryBuilder deleteRowRange() + public SingleOperationBuilder insert(int pdIdx, int cdIdx) { - singleOpVisitBuilder().deleteRowRange(); + singleOpVisitBuilder().insert(pdIdx, cdIdx); return this; } @Override - public HistoryBuilder deleteRowRange(int lowBoundRowIdx, int highBoundRowIdx, boolean isMinEq, boolean isMaxEq) + public SingleOperationBuilder insert(int pdIdx, int rowIdx, int[] valueIdxs, int[] sValueIdxs) { - singleOpVisitBuilder().deleteRowRange(lowBoundRowIdx, highBoundRowIdx, isMinEq, isMaxEq); + singleOpVisitBuilder().insert(pdIdx, rowIdx, valueIdxs, sValueIdxs); return this; } @Override - public HistoryBuilder deleteRowSlice() + public SingleOperationBuilder deleteRowRange(int pdIdx, int lowerBoundRowIdx, int upperBoundRowIdx, + int nonEqFrom, boolean includeLowerBound, boolean includeUpperBound) { - singleOpVisitBuilder().deleteRowSlice(); + singleOpVisitBuilder().deleteRowRange(pdIdx, lowerBoundRowIdx, upperBoundRowIdx, + nonEqFrom, includeLowerBound, includeUpperBound); return this; } @Override - public BatchVisitBuilder beginBatch() + public SingleOperationBuilder select(int pdIdx, IdxRelation[] ckRelations, IdxRelation[] regularRelations, IdxRelation[] staticRelations) { - long visitLts = clock.nextLts(); - return batchVisitBuilder(defaultSelector.pd(visitLts, schema), visitLts); + singleOpVisitBuilder().select(pdIdx, ckRelations, regularRelations, staticRelations); + return this; } - /** - * Begin batch for a partition descriptor at a specific index. - * - * Imagine all partition descriptors were longs in an array. Index of a descriptor - * is a sequential number of the descriptor in this imaginary array. - */ @Override - public BatchVisitBuilder beginBatch(long pdIdx) - { - long visitLts = clock.nextLts(); - return batchVisitBuilder(presetSelector.pdAtPosition(pdIdx), visitLts); - } - - protected BatchVisitBuilder batchVisitBuilder(long pd, long lts) - { - PartitionVisitStateImpl partitionState = presetSelector.register(lts, pd, (ps) -> {}); - return new BatchVisitBuilder(this, partitionState, lts, pureRng, descriptorSelector, schema, valueHelper, (visit) -> { - visitedPartitions.add(pd); - log.put(visit.lts, visit); - }); - } - - public SingleOperationBuilder visitPartition(long pdIdx) - { - long visitLts = clock.nextLts(); - long pd = presetSelector.pdAtPosition(pdIdx); - return singleOpVisitBuilder(pd, visitLts, (ps) -> {}); - } - - public SingleOperationBuilder visitPartition(long pdIdx, Consumer<PartitionVisitState> setupPs) + public SingleOperationBuilder selectRowRange(int pdIdx, int lowerBoundRowIdx, int upperBoundRowIdx, + int nonEqFrom, boolean includeLowerBound, boolean includeUpperBound) { - long visitLts = clock.nextLts(); - long pd = presetSelector.pdAtPosition(pdIdx); - return singleOpVisitBuilder(pd, visitLts, setupPs); + singleOpVisitBuilder().selectRowRange(pdIdx, lowerBoundRowIdx, upperBoundRowIdx, + nonEqFrom, includeLowerBound, includeUpperBound); + return this; } - public PartitionVisitState forPartition(long pdIdx) + @Override + public SingleOperationBuilder selectPartition(int pdIdx) { - long pd = defaultSelector.pdAtPosition(pdIdx, schema); - return partitionStates.computeIfAbsent(pd, (pd_) -> makePartitionVisitState(pd)); + singleOpVisitBuilder().selectPartition(pdIdx); + return this; } - private PartitionVisitStateImpl makePartitionVisitState(long pd) + @Override + public SingleOperationBuilder selectPartition(int pdIdx, Operations.ClusteringOrderBy orderBy) { - Long[] possibleCds = new Long[maxPartitionSize]; - for (int cdIdx = 0; cdIdx < possibleCds.length; cdIdx++) - { - long cd = descriptorSelector.cd(pd, 0, cdIdx, schema); - possibleCds[cdIdx] = cd; - } - Arrays.sort(possibleCds, Long::compare); - - long[] primitiveArray = new long[maxPartitionSize]; - for (int i = 0; i < possibleCds.length; i++) - primitiveArray[i] = possibleCds[i]; - - // TODO: can we have something more efficient than a tree set here? - return new PartitionVisitStateImpl(pd, primitiveArray, new TreeSet<>(), schema); + singleOpVisitBuilder().selectPartition(pdIdx, orderBy); + return this; } - public PresetPdSelector pdSelector() + @Override + public SingleOperationBuilder selectRow(int pdIdx, int rowIdx) { - return presetSelector; + singleOpVisitBuilder().selectRow(pdIdx, rowIdx); + return this; } - /** - * This is an adapter HistoryBuilder is using to reproduce state for the reconciler. - * - * This class is inherently not thread-safe. The thinking behind this is that you should generate - * operations in advance, and only after you have generated them, should you start execution. - * If you would like to generate on the fly, you should use default Harry machinery and pure generators, - * that walk LTS space without intermediate state. This set of primitives is intended to be used for much - * smaller scale testing. - */ - public class PresetPdSelector extends OpSelectors.PdSelector + @Override + public SingleOperationBuilder selectRowSliceByLowerBound(int pdIdx, int lowerBoundRowIdx, int nonEqFrom, boolean isEq) { - // TODO: implement a primitive long map? - private final Map<Long, Long> ltsToPd = new HashMap<>(); - - public PartitionVisitStateImpl register(long lts, long pd, Consumer<PartitionVisitState> setup) - { - Long prev = ltsToPd.put(lts, pd); - if (prev != null) - throw new IllegalStateException(String.format("LTS %d. Was registered twice, first with %d, and then with %d", lts, prev, pd)); - - PartitionVisitStateImpl partitionState = partitionStates.computeIfAbsent(pd, (pd_) -> { - PartitionVisitStateImpl partitionVisitState = makePartitionVisitState(pd); - setup.accept(partitionVisitState); - return partitionVisitState; - }); - partitionState.visitedLts.add(lts); - return partitionState; - } - - protected long pd(long lts) - { - return ltsToPd.get(lts); - } - - public long nextLts(long lts) - { - long pd = pd(lts); - PartitionVisitStateImpl partitionState = partitionStates.get(pd); - NavigableSet<Long> visitedLts = partitionState.visitedLts.subSet(lts, false, Long.MAX_VALUE, false); - if (visitedLts.isEmpty()) - return -1; - else - return visitedLts.first(); - } - - public long prevLts(long lts) - { - long pd = pd(lts); - PartitionVisitStateImpl partitionState = partitionStates.get(pd); - NavigableSet<Long> visitedLts = partitionState.visitedLts.descendingSet().subSet(lts, false, 0L, false); - if (visitedLts.isEmpty()) - return -1; - else - return visitedLts.first(); - } - - public long maxLtsFor(long pd) - { - PartitionVisitStateImpl partitionState = partitionStates.get(pd); - if (partitionState == null) - return -1; - return partitionState.visitedLts.last(); - } - - public long minLtsFor(long pd) - { - PartitionVisitStateImpl partitionState = partitionStates.get(pd); - if (partitionState == null) - return -1; - return partitionState.visitedLts.first(); - } - - public long pdAtPosition(long pdIdx) - { - return defaultSelector.pdAtPosition(pdIdx, schema); - } - - public long minLtsAt(long position) - { - throw new IllegalArgumentException("not implemented"); - } - - public long maxPosition(long maxLts) - { - // since, unlike other PdSelectors, this one is not computational, we can answer which position is the largest just - // by tracking the largest position - return 0; - } + singleOpVisitBuilder().selectRowSliceByLowerBound(pdIdx, lowerBoundRowIdx, nonEqFrom, isEq); + return this; } - public ReplayingVisitor visitor(DataTracker tracker, SystemUnderTest sut, SystemUnderTest.ConsistencyLevel cl) + @Override + public SingleOperationBuilder selectRowSliceByUpperBound(int pdIdx, int upperBoundRowIdx, int nonEqFrom, boolean isEq) { - if (schema.trackLts) - { - return visitor(new MutatingVisitor.LtsTrackingVisitExecutor(descriptorSelector, - tracker, - sut, - schema, - new MutatingRowVisitor(schema, clock, MetricReporter.NO_OP), - cl)); - } - else - { - return visitor(new MutatingVisitor.MutatingVisitExecutor(descriptorSelector, - tracker, - sut, - schema, - new MutatingRowVisitor(schema, clock, MetricReporter.NO_OP), - cl)); - } + singleOpVisitBuilder().selectRowSliceByUpperBound(pdIdx, upperBoundRowIdx, nonEqFrom, isEq); + return this; } - public Model noOpChecker(SystemUnderTest.ConsistencyLevel cl, SystemUnderTest sut) + @Override + public SingleOperationBuilder deletePartition(int pdIdx) { - return new NoOpChecker(sut, cl); + singleOpVisitBuilder().deletePartition(pdIdx); + return this; } - public Model quiescentChecker(DataTracker tracker, SystemUnderTest sut) + @Override + public SingleOperationBuilder deleteRow(int pdIdx, int rowIdx) { - // TODO: CL for quiescent checker - return new QuiescentChecker(clock, sut, tracker, schema, - new Reconciler(presetSelector, - schema, - this::visitor)); + singleOpVisitBuilder().deleteRow(pdIdx, rowIdx); + return this; } - public Model quiescentLocalChecker(DataTracker tracker, SystemUnderTest sut) + @Override + public SingleOperationBuilder deleteColumns(int pdIdx, int rowIdx, BitSet regularSelection, BitSet staticSelection) { - return new QuiescentLocalStateChecker(clock, presetSelector, sut, tracker, schema, - new Reconciler(presetSelector, - schema, - this::visitor), - rf); + singleOpVisitBuilder().deleteRow(pdIdx, rowIdx); + return this; } - public void validate(DataTracker tracker, SystemUnderTest sut, int... partitionIdxs) + @Override + public SingleOperationBuilder deleteRowSliceByLowerBound(int pdIdx, int lowerBoundRowIdx, int nonEqFrom, boolean isEq) { - validate(quiescentChecker(tracker, sut), partitionIdxs); + singleOpVisitBuilder().deleteRowSliceByLowerBound(pdIdx, lowerBoundRowIdx, nonEqFrom, isEq); + return this; } - public void validate(Model model, int... partitionIdxs) + @Override + public SingleOperationBuilder deleteRowSliceByUpperBound(int pdIdx, int upperBoundRowIdx, int nonEqFrom, boolean isEq) { - for (int partitionIdx : partitionIdxs) - { - long pd = presetSelector.pdAtPosition(partitionIdx); - if (presetSelector.minLtsFor(pd) < 0) - continue; - model.validate(Query.selectAllColumns(schema, pd, false)); - model.validate(Query.selectAllColumns(schema, pd, true)); - } + singleOpVisitBuilder().deleteRowSliceByUpperBound(pdIdx, upperBoundRowIdx, nonEqFrom, isEq); + return this; } - public void validateAll(DataTracker tracker, SystemUnderTest sut) - { - validateAll(quiescentChecker(tracker, sut)); - } - public void validateAll(Model model) + /** + * Indexed bijection allows to decouple descriptor order from value order, which makes data generation simpler. + * <p> + * For regular Harry bijections, this is done at no cost, since values are inflated in a way that preserves + * descriptor order, which means that idx order is consistent with descriptor order and consistent with value order. + * <p> + * An indexed bijection allows order to be established via index, and use descriptor simply as a seed for random values. + */ + public interface IndexedBijection<T> extends Bijections.Bijection<T> { - for (Long pd : partitionStates.keySet()) - { - model.validate(Query.selectAllColumns(schema, pd, false)); - model.validate(Query.selectAllColumns(schema, pd, true)); - } - } + int idxFor(long descriptor); - public void validateAll(Model model, Function<Long, List<Query>> queries) - { - for (Long pd : partitionStates.keySet()) - { - for (Query query : queries.apply(pd)) - model.validate(query); - } - } + long descriptorAt(int idx); - public ReplayingVisitor visitor(VisitExecutor executor) - { - LongIterator replay = clock.replayAll(); - return new ReplayingVisitor(executor, replay) + @Override + default String toString(long pd) { - public Visit getVisit(long lts) - { - long idx = lts - clock.base; - Visit visit = log.get(idx); - assert visit != null : String.format("Could not find a visit for LTS %d", lts); - return visit; - } + if (pd == MagicConstants.UNSET_DESCR) + return Integer.toString(MagicConstants.UNSET_IDX); - public void replayAll() - { - while (replay.hasNext()) - visit(); - } - }; - } + if (pd == MagicConstants.NIL_DESCR) + return Integer.toString(MagicConstants.NIL_IDX); - public interface LongIterator extends LongSupplier - { - boolean hasNext(); - long getAsLong(); + return Integer.toString(idxFor(pd)); + } } - /** - * Non-monotonic version of OffsetClock. - */ - public class OffsetClock implements OpSelectors.Clock - { - private long lowerBound; - private long current; - - private final long base; - private final long batchSize; - private final Set<Long> returned; - private final EntropySource entropySource; - - private final List<Long> visitOrder; - - public OffsetClock(long base, long batchSize, EntropySource entropySource) + public static IndexedValueGenerators valueGenerators(SchemaSpec schema, long seed) + { + return valueGenerators(schema, seed, 1000); + } + + @SuppressWarnings({ "unchecked" }) + public static IndexedValueGenerators valueGenerators(SchemaSpec schema, long seed, int populationPerColumn) + { + List<Comparator<Object>> pkComparators = new ArrayList<>(); + List<Comparator<Object>> ckComparators = new ArrayList<>(); + List<Comparator<Object>> regularComparators = new ArrayList<>(); + List<Comparator<Object>> staticComparators = new ArrayList<>(); + + EntropySource rng = new JdkRandomEntropySource(seed); + for (int i = 0; i < schema.partitionKeys.size(); i++) + pkComparators.add((Comparator<Object>) schema.partitionKeys.get(i).type.comparator()); + for (int i = 0; i < schema.clusteringKeys.size(); i++) + ckComparators.add((Comparator<Object>) schema.clusteringKeys.get(i).type.comparator()); + for (int i = 0; i < schema.regularColumns.size(); i++) + regularComparators.add((Comparator<Object>) schema.regularColumns.get(i).type.comparator()); + for (int i = 0; i < schema.staticColumns.size(); i++) + staticComparators.add((Comparator<Object>) schema.staticColumns.get(i).type.comparator()); + + Map<ColumnSpec<?>, InvertibleGenerator<Object>> map = new HashMap<>(); + for (ColumnSpec<?> column : IteratorsUtil.concat(schema.regularColumns, schema.staticColumns)) + map.computeIfAbsent(column, (a) -> (InvertibleGenerator<Object>) fromType(rng, populationPerColumn, column)); + + // TODO: empty gen + return new IndexedValueGenerators(new InvertibleGenerator<>(rng, cumulativeEntropy(schema.partitionKeys), populationPerColumn, forKeys(schema.partitionKeys), keyComparator(schema.partitionKeys)), + new InvertibleGenerator<>(rng, cumulativeEntropy(schema.clusteringKeys), populationPerColumn, forKeys(schema.clusteringKeys), keyComparator(schema.clusteringKeys)), + schema.regularColumns.stream() + .map(map::get) + .collect(Collectors.toList()), + schema.staticColumns.stream() + .map(map::get) + .collect(Collectors.toList()), + pkComparators, + ckComparators, + regularComparators, + staticComparators); + } + + public static class IndexedValueGenerators extends ValueGenerators + { + public IndexedValueGenerators(IndexedBijection<Object[]> pkGen, + IndexedBijection<Object[]> ckGen, + List<IndexedBijection<Object>> regularColumnGens, + List<IndexedBijection<Object>> staticColumnGens, + List<Comparator<Object>> pkComparators, + List<Comparator<Object>> ckComparators, + List<Comparator<Object>> regularComparators, + List<Comparator<Object>> staticComparators) { - this.lowerBound = base; - this.base = base; - this.batchSize = batchSize; - this.returned = new HashSet<>(); - this.entropySource = entropySource; - this.visitOrder = new ArrayList<>(); - this.current = computeNext(); + super(pkGen, ckGen, + (List<Bijections.Bijection<Object>>) (List<?>) regularColumnGens, + (List<Bijections.Bijection<Object>>) (List<?>) staticColumnGens, + pkComparators, ckComparators, regularComparators, staticComparators); } - /** - * Visit Order - related methods - */ - public LongIterator replayAll() + @Override + public IndexedBijection<Object[]> pkGen() { - return new LongIterator() - { - private int visitedUpTo; - - public boolean hasNext() - { - return visitedUpTo < visitOrder.size(); - } - - public long getAsLong() - { - return visitOrder.get(visitedUpTo++); - } - }; + return (IndexedBijection<Object[]>) super.pkGen(); } - private long computeNext() + @Override + public IndexedBijection<Object[]> ckGen() { - if (returned.size() == batchSize) - { - returned.clear(); - lowerBound += batchSize; - } - - long generated = entropySource.nextLong(lowerBound, lowerBound + batchSize); - while (returned.contains(generated)) - generated = entropySource.nextLong(lowerBound, lowerBound + batchSize); - - returned.add(generated); - return generated; + return (IndexedBijection<Object[]>) super.ckGen(); } @Override - public long rts(long lts) + public IndexedBijection<Object> regularColumnGen(int idx) { - return base + lts; + return (IndexedBijection<Object>) super.regularColumnGen(idx); } @Override - public long lts(long rts) + public IndexedBijection<Object> staticColumnGen(int idx) { - return rts - base; + return (IndexedBijection<Object>) super.staticColumnGen(idx); } + } - @Override - public long nextLts() - { - long ret = current; - current = computeNext(); - visitOrder.add(ret); - return ret; - } + private static Comparator<Object[]> keyComparator(List<ColumnSpec<?>> columns) + { + return (o1, o2) -> compareKeys(columns, o1, o2); + } - public long peek() - { - return current; - } + public static int compareKeys(List<ColumnSpec<?>> columns, Object[] v1, Object[] v2) + { + assert v1.length == v2.length : String.format("Values should be of same length: %d != %d\n%s\n%s", + v1.length, v2.length, Arrays.toString(v1), Arrays.toString(v2)); - public Configuration.ClockConfiguration toConfig() + for (int i = 0; i < v1.length; i++) { - throw new RuntimeException("Not implemented"); + int res; + ColumnSpec column = columns.get(i); + if (column.type.isReversed()) + res = column.type.comparator().reversed().compare(v1[i], v2[i]); + else + res = column.type.comparator().compare(v1[i], v2[i]); + if (res != 0) + return res; } + return 0; } -} +} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/dsl/HistoryBuilderHelper.java b/test/harry/main/org/apache/cassandra/harry/dsl/HistoryBuilderHelper.java new file mode 100644 index 000000000000..281f62cc87f6 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/dsl/HistoryBuilderHelper.java @@ -0,0 +1,195 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.dsl; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; + +import org.apache.cassandra.harry.gen.EntropySource; +import org.apache.cassandra.harry.gen.Generator; +import org.apache.cassandra.harry.gen.Generators; +import org.apache.cassandra.harry.MagicConstants; +import org.apache.cassandra.harry.Relations; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.util.BitSet; + +import static org.apache.cassandra.harry.Relations.RelationKind.EQ; +import static org.apache.cassandra.harry.Relations.RelationKind.GT; +import static org.apache.cassandra.harry.Relations.RelationKind.GTE; +import static org.apache.cassandra.harry.Relations.RelationKind.LT; + +/** + * Things that seemed like a good idea, but ultimately were not a good fit for the HistoryBuilder API + */ +public class HistoryBuilderHelper +{ + /** + * Perform a random insert to any row + */ + public static void insertRandomData(SchemaSpec schema, Generator<Integer> pkGen, Generator<Integer> ckGen, EntropySource rng, HistoryBuilder history) + { + insertRandomData(schema, pkGen.generate(rng), ckGen.generate(rng), rng, history); + } + + public static void insertRandomData(SchemaSpec schema, int partitionIdx, int rowIdx, EntropySource rng, HistoryBuilder history) + { + int[] vIdxs = new int[schema.regularColumns.size()]; + for (int i = 0; i < schema.regularColumns.size(); i++) + vIdxs[i] = rng.nextInt(history.valueGenerators().regularPopulation(i)); + int[] sIdxs = new int[schema.staticColumns.size()]; + for (int i = 0; i < schema.staticColumns.size(); i++) + sIdxs[i] = rng.nextInt(history.valueGenerators().staticPopulation(i)); + history.insert(partitionIdx, rowIdx, vIdxs, sIdxs); + } + + public static void insertRandomData(SchemaSpec schema, int pkIdx, EntropySource rng, HistoryBuilder history) + { + insertRandomData(schema, + pkIdx, + rng.nextInt(0, history.valueGenerators().ckPopulation()), + rng, + 0, + history); + } + + public static void insertRandomData(SchemaSpec schema, int partitionIdx, int rowIdx, EntropySource rng, double chanceOfUnset, HistoryBuilder history) + { + int[] vIdxs = new int[schema.regularColumns.size()]; + for (int i = 0; i < schema.regularColumns.size(); i++) + vIdxs[i] = rng.nextDouble() <= chanceOfUnset ? MagicConstants.UNSET_IDX : rng.nextInt(history.valueGenerators().regularPopulation(i)); + int[] sIdxs = new int[schema.staticColumns.size()]; + for (int i = 0; i < schema.staticColumns.size(); i++) + sIdxs[i] = rng.nextDouble() <= chanceOfUnset ? MagicConstants.UNSET_IDX : rng.nextInt(history.valueGenerators().staticPopulation(i)); + history.insert(partitionIdx, rowIdx, vIdxs, sIdxs); + } + + + public static void deleteRandomColumns(SchemaSpec schema, int partitionIdx, int rowIdx, EntropySource rng, SingleOperationBuilder history) + { + Generator<BitSet> regularMask = Generators.bitSet(schema.regularColumns.size()); + Generator<BitSet> staticMask = Generators.bitSet(schema.staticColumns.size()); + + history.deleteColumns(partitionIdx, + rowIdx, + regularMask.generate(rng), + staticMask.generate(rng)); + } + + private static final Generator<Relations.RelationKind> relationKindGen = Generators.pick(LT, GT, EQ); + private static final Set<Relations.RelationKind> lowBoundRelations = Set.of(GT, GTE, EQ); + + /** + * Generates random relations for regular and static columns for FILTERING and SAI queries. + * + * Will generate at most 2 relations per column: + * * generates a random relation + * * if this relation is EQ, that's the only relation that will lock this column + * * if relation is GT, next bound, if generated, will be LT + * * if relation is LT, next bound, if generated, will be GT + * + * @param rng - random number generator + * @param numColumns - number of columns in the generated set of relationships + * @param population - expected population / number of possible values for a given column + * @return a list of relations + */ + public static List<SingleOperationBuilder.IdxRelation> generateValueRelations(EntropySource rng, int numColumns, Function<Integer, Integer> population) + { + List<SingleOperationBuilder.IdxRelation> relations = new ArrayList<>(); + Map<Integer, Set<Relations.RelationKind>> kindsMap = new HashMap<>(); + int remainingColumns = numColumns; + while (remainingColumns > 0) + { + int column = rng.nextInt(numColumns); + Set<Relations.RelationKind> kinds = kindsMap.computeIfAbsent(column, c -> new HashSet<>()); + if (kinds.size() > 1 || kinds.contains(EQ)) + continue; + Relations.RelationKind kind; + if (kinds.size() == 1) + { + if (kinds.contains(LT)) kind = GT; + else kind = LT; + remainingColumns--; + } + else + // TODO: weights per relation? + kind = relationKindGen.generate(rng); + + if (kind == EQ) + remainingColumns--; + + kinds.add(kind); + + int regularIdx = rng.nextInt(population.apply(column)); + relations.add(new SingleOperationBuilder.IdxRelation(kind, regularIdx, column)); + if (rng.nextBoolean()) + break; + } + return relations; + } + + /** + * Generates random relations for regular and static columns for FILTERING and SAI queries. + * + * Will generate at most 2 relations per column. Low bound will always use values from low bound clustering, + * high bound will always use values from high bound. + * + * @param rng - random number generator + * @param numColumns - number of columns in the generated set of relationships + * @return a list of relations + */ + public static List<SingleOperationBuilder.IdxRelation> generateClusteringRelations(EntropySource rng, int numColumns, Generator<Integer> ckGen) + { + List<SingleOperationBuilder.IdxRelation> relations = new ArrayList<>(); + Map<Integer, Set<Relations.RelationKind>> kindsMap = new HashMap<>(); + int remainingColumns = numColumns; + int lowBoundIdx = ckGen.generate(rng); + int highBoundIdx = ckGen.generate(rng); + while (remainingColumns > 0) + { + int column = rng.nextInt(numColumns); + Set<Relations.RelationKind> kinds = kindsMap.computeIfAbsent(column, c -> new HashSet<>()); + if (kinds.size() > 1 || kinds.contains(EQ)) + continue; + Relations.RelationKind kind; + if (kinds.size() == 1) + { + if (kinds.contains(LT)) kind = GT; + else kind = LT; + remainingColumns--; + } + else + kind = relationKindGen.generate(rng); + + if (kind == EQ) + remainingColumns--; + + kinds.add(kind); + + relations.add(new SingleOperationBuilder.IdxRelation(kind, lowBoundRelations.contains(kind) ? lowBoundIdx : highBoundIdx, column)); + if (rng.nextBoolean()) + break; + } + return relations; + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/dsl/MultiOperationVisitBuilder.java b/test/harry/main/org/apache/cassandra/harry/dsl/MultiOperationVisitBuilder.java new file mode 100644 index 000000000000..7dbff3db2e14 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/dsl/MultiOperationVisitBuilder.java @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.dsl; + +import java.io.Closeable; +import java.util.function.Consumer; + +import org.apache.cassandra.harry.gen.IndexGenerators; +import org.apache.cassandra.harry.op.Visit; +import org.apache.cassandra.harry.util.BitSet; + +import static org.apache.cassandra.harry.dsl.HistoryBuilder.IndexedValueGenerators; + +public class MultiOperationVisitBuilder extends SingleOperationVisitBuilder implements Closeable +{ + MultiOperationVisitBuilder(long lts, IndexedValueGenerators valueGenerators, IndexGenerators indexGenerators, Consumer<Visit> appendToLog) + { + super(lts, valueGenerators, indexGenerators, appendToLog); + } + + @Override + public MultiOperationVisitBuilder custom(Runnable runnable, String tag) + { + super.custom(runnable, tag); + return this; + } + + @Override + public MultiOperationVisitBuilder update(int pdIdx, int rowIdx, int[] valueIdxs, int[] sValueIdxs) + { + super.update(pdIdx, rowIdx, valueIdxs, sValueIdxs); + return this; + } + + @Override + public MultiOperationVisitBuilder insert(int partitionIdx, int rowIdx, int[] valueIdxs, int[] sValueIdxs) + { + super.insert(partitionIdx, rowIdx, valueIdxs, sValueIdxs); + return this; + } + + @Override + public MultiOperationVisitBuilder selectRowRange(int partitionIdx, int lowBoundRowIdx, int highBoundRowIdx, + int nonEqFrom, boolean includeLowerBound, boolean includeHighBound) + { + super.selectRowRange(partitionIdx, lowBoundRowIdx, highBoundRowIdx, nonEqFrom, includeLowerBound, includeHighBound); + return this; + } + + @Override + public MultiOperationVisitBuilder selectPartition(int partitionIdx) + { + super.selectPartition(partitionIdx); + return this; + } + + @Override + public MultiOperationVisitBuilder selectRow(int partitionIdx, int rowIdx) + { + super.selectRow(partitionIdx, rowIdx); + return this; + } + + @Override + public MultiOperationVisitBuilder selectRowSliceByLowerBound(int partitionIdx, int lowBoundRowIdx, int nonEqFrom, boolean isEq) + { + super.selectRowSliceByLowerBound(partitionIdx, lowBoundRowIdx, nonEqFrom, isEq); + return this; + } + + @Override + public MultiOperationVisitBuilder selectRowSliceByUpperBound(int partitionIdx, int highBoundRowIdx, int nonEqFrom, boolean isEq) + { + super.selectRowSliceByUpperBound(partitionIdx, highBoundRowIdx, nonEqFrom, isEq); + return this; + } + + @Override + public MultiOperationVisitBuilder deletePartition(int partitionIdx) + { + super.deletePartition(partitionIdx); + return this; + } + + @Override + public MultiOperationVisitBuilder deleteRow(int partitionIdx, int rowIdx) + { + super.deleteRow(partitionIdx, rowIdx); + return this; + } + + @Override + public MultiOperationVisitBuilder deleteColumns(int partitionIdx, int rowIdx, BitSet regularSelection, BitSet staticSelection) + { + super.deleteColumns(partitionIdx, rowIdx, regularSelection, staticSelection); + return this; + } + + @Override + public MultiOperationVisitBuilder deleteRowSliceByLowerBound(int partitionIdx, int lowBoundRowIdx, int nonEqFrom, boolean isEq) + { + super.deleteRowSliceByLowerBound(partitionIdx, lowBoundRowIdx, nonEqFrom, isEq); + return this; + } + + @Override + public MultiOperationVisitBuilder deleteRowSliceByUpperBound(int partitionIdx, int highBoundRowIdx, int nonEqFrom, boolean isEq) + { + super.deleteRowSliceByUpperBound(partitionIdx, highBoundRowIdx, nonEqFrom, isEq); + return this; + } + + @Override + public MultiOperationVisitBuilder deleteRowRange(int pdIdx, int lowerBoundRowIdx, int upperBoundRowIdx, + int nonEqFrom, boolean includeLowerBound, boolean includeUpperBound) + { + super.deleteRowRange(pdIdx, lowerBoundRowIdx, upperBoundRowIdx, + nonEqFrom, includeLowerBound, includeUpperBound); + return this; + } + + @Override + public MultiOperationVisitBuilder select(int partitionIdx, + IdxRelation[] ckRelations, + IdxRelation[] regularRelations, + IdxRelation[] staticRelations) + { + super.select(partitionIdx, ckRelations, regularRelations, staticRelations); + return this; + } + + @Override + Visit build() + { + throw new IllegalStateException("Shuold not be called directly, use auto-close"); + } + + Visit buildInternal() + { + return super.build(); + } + + + @Override + int size() + { + return super.size(); + } + + public void close() + { + buildInternal(); + } +} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/dsl/OverridingBijection.java b/test/harry/main/org/apache/cassandra/harry/dsl/OverridingBijection.java deleted file mode 100644 index fe645d298010..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/dsl/OverridingBijection.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.dsl; - -import java.util.HashMap; -import java.util.Map; - -import org.apache.cassandra.harry.gen.Bijections; - -class OverridingBijection<T> implements Bijections.Bijection<T> -{ - protected final Bijections.Bijection<T> delegate; - protected final Map<Long, T> descriptorToValue; - protected final Map<T, Long> valueToDescriptor; - - public OverridingBijection(Bijections.Bijection<T> delegate) - { - this.delegate = delegate; - descriptorToValue = new HashMap<>(); - valueToDescriptor = new HashMap<>(); - } - - public void override(long descriptor, T value) - { - T old = descriptorToValue.get(descriptor); - if (old != null) - throw new IllegalStateException(String.format("Can't override %d twice. Was already overriden to %s", descriptor, old)); - - T orig = delegate.inflate(descriptor); - if (!orig.equals(value)) - { - descriptorToValue.put(descriptor, value); - valueToDescriptor.put(value, descriptor); - } - } - - @Override - public T inflate(long descriptor) - { - Object v = descriptorToValue.get(descriptor); - if (v != null) - { - return (T) v; - } - return delegate.inflate(descriptor); - } - - @Override - public long deflate(T value) - { - Long descriptor = valueToDescriptor.get(value); - if (descriptor != null) - return descriptor; - return delegate.deflate(value); - } - - @Override - public int byteSize() - { - return delegate.byteSize(); - } - - @Override - public int compare(long l, long r) - { - return delegate.compare(l, r); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/dsl/OverridingCkGenerator.java b/test/harry/main/org/apache/cassandra/harry/dsl/OverridingCkGenerator.java deleted file mode 100644 index 92aa00f69508..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/dsl/OverridingCkGenerator.java +++ /dev/null @@ -1,153 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.dsl; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.gen.Bijections; -import org.apache.cassandra.harry.gen.DataGenerators; - -/** - * A class that helps to override parts of clustering key. The tricky part about CK overrides is that Harry model makes - * an assumption about the ordering of clustering keys, which means clusterings have to be sorted in the same way their - * descriptors are. This, combined with reverse types, makes managing this state somewhat tricky at times. - * - * Additionally, Relation in delete/select query receives individual CK descriptors (i.e. after they have been sliced), - * while most other queries usually operate on inflated clustering key. All of this is required for efficient _stateless_ - * validation, but makes overrides a bit less intuitive. - * - * To summarise: overrides for inflating are done for individual clustering key columns. Overrides for deflating a clustering - * operate on an entire key. Main reason for this is to allow having same string in several rows for the same column. - */ - -@SuppressWarnings({"rawtypes", "unchecked"}) -public class OverridingCkGenerator extends DataGenerators.KeyGenerator -{ - private final DataGenerators.KeyGenerator delegate; - private final KeyPartOverride[] columnValueOverrides; - private final List<ColumnSpec<?>> columnSpecOverrides; - private final Map<ArrayWrapper, Long> valueToDescriptor; - - // Had to be a static method because you can not call super after you have initialised any fields - public static OverridingCkGenerator make(DataGenerators.KeyGenerator delegate) - { - KeyPartOverride[] columnValueOverrides = new KeyPartOverride[delegate.columns.size()]; - List<ColumnSpec<?>> columnSpecOverrides = new ArrayList<>(); - for (int i = 0; i < delegate.columns.size(); i++) - { - columnValueOverrides[i] = new KeyPartOverride<>(delegate.columns.get(i).generator()); - columnSpecOverrides.add(delegate.columns.get(i).override(columnValueOverrides[i])); - } - assert columnValueOverrides.length == columnSpecOverrides.size(); - return new OverridingCkGenerator(delegate, columnValueOverrides, columnSpecOverrides); - } - - private OverridingCkGenerator(DataGenerators.KeyGenerator delegate, - KeyPartOverride[] columnValueOverrides, - List<ColumnSpec<?>> columnSpecOverrides) - { - super(columnSpecOverrides); - this.columnValueOverrides = columnValueOverrides; - this.columnSpecOverrides = columnSpecOverrides; - this.delegate = delegate; - this.valueToDescriptor = new HashMap<>(); - } - - public void override(long descriptor, Object[] value) - { - long[] slices = delegate.slice(descriptor); - for (int i = 0; i < slices.length; i++) - columnValueOverrides[i].override(slices[i], value[i]); - - // We _always_ deflate clustering key as a package, since we can not validate a clustering key without all components anyways. - valueToDescriptor.put(new ArrayWrapper(value), descriptor); - } - - @Override - public Object[] inflate(long descriptor) - { - return DataGenerators.inflateKey(columnSpecOverrides, descriptor, slice(descriptor)); - } - - @Override - public long deflate(Object[] value) - { - Long descriptor = valueToDescriptor.get(new ArrayWrapper(value)); - if (descriptor != null) - return descriptor; - - return delegate.deflate(value); - } - - @Override - public int byteSize() - { - return delegate.byteSize(); - } - - @Override - public int compare(long l, long r) - { - return delegate.byteSize(); - } - - @Override - public long[] slice(long descriptor) - { - return delegate.slice(descriptor); - } - - @Override - public long stitch(long[] parts) - { - return delegate.stitch(parts); - } - - @Override - public long minValue(int idx) - { - return delegate.minValue(idx); - } - - @Override - public long maxValue(int idx) - { - return delegate.maxValue(idx); - } - - public static class KeyPartOverride<T> extends OverridingBijection<T> - { - public KeyPartOverride(Bijections.Bijection<T> delegate) - { - super(delegate); - } - - // We do not use deflate for key part overrides - @Override - public long deflate(T value) - { - throw new IllegalStateException(); - } - } - -} diff --git a/test/harry/main/org/apache/cassandra/harry/dsl/PartitionVisitState.java b/test/harry/main/org/apache/cassandra/harry/dsl/PartitionVisitState.java deleted file mode 100644 index 37558274b10f..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/dsl/PartitionVisitState.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.dsl; - -public interface PartitionVisitState -{ - /** - * Informs a generator that a specific clustering key has to be generated. - * - * Since Harry model has a few constraints, we can not override to an arbitrary clustering to an arbitrary value, since - * Harry ensures that clustering descriptors are sorted the same way clusterings themselves are sorted. - * - * Gladly, most of the time we need to override just one or a couple of values to trigger some edge condition, - * which means that we can simply instruct Harry to produce a given handcrafted value. - * - * When using `ensureClustering`, you can not reliably know in advance where specifically in the row this value is - * going to sort. - * - * If you need arbitrary overrides, you will have to produce _all_ clusterings possible for the given partition. - */ - void ensureClustering(Object[] overrides); - void overrideClusterings(Object[][] overrides); - long pd(); -} diff --git a/test/harry/main/org/apache/cassandra/harry/dsl/PartitionVisitStateImpl.java b/test/harry/main/org/apache/cassandra/harry/dsl/PartitionVisitStateImpl.java deleted file mode 100644 index d61210515b6c..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/dsl/PartitionVisitStateImpl.java +++ /dev/null @@ -1,115 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.dsl; - -import java.util.Arrays; -import java.util.NavigableSet; - -import org.apache.cassandra.harry.ddl.SchemaSpec; - -public class PartitionVisitStateImpl implements PartitionVisitState -{ - final long pd; - final long[] possibleCds; - final NavigableSet<Long> visitedLts; - final SchemaSpec schema; - private final OverridingCkGenerator ckGenerator; - - PartitionVisitStateImpl(long pd, long[] possibleCds, NavigableSet<Long> visitedLts, SchemaSpec schema) - { - this.pd = pd; - this.possibleCds = possibleCds; - this.visitedLts = visitedLts; - this.schema = schema; - this.ckGenerator = (OverridingCkGenerator) schema.ckGenerator; - } - - - /** - * Ensures that exactly one of the clustering keys will have given values. - */ - @Override - public void ensureClustering(Object[] overrides) - { - long cd = findCdForOverride(overrides); - ckGenerator.override(cd, overrides); - } - - @Override - public void overrideClusterings(Object[][] overrides) - { - assert possibleCds.length == overrides.length; - Arrays.sort(overrides, this::compareCds); - for (int i = 0; i < overrides.length; i++) - ckGenerator.override(possibleCds[i], overrides[i]); - } - - @Override - public long pd() - { - return pd; - } - - long findCdForOverride(Object[] ck) - { - int low = 0; - int high = possibleCds.length - 1; - - while (low <= high) - { - int mid = (low + high) >>> 1; - long midEl = possibleCds[mid]; - int cmp = compareCds(ck, midEl); - - if (cmp < 0) - low = mid + 1; - else if (cmp > 0) - high = mid - 1; - else - throw new IllegalStateException("This value is already present"); - } - - return possibleCds[Math.min(possibleCds.length - 1, low)]; - } - - private int compareCds(Object[] v1, long cd2) - { - Object[] v2 = schema.ckGenerator.inflate(cd2); - return compareCds(v1, v2); - } - - private int compareCds(Object[] v1, Object[] v2) - { - assert v1.length == v2.length : String.format("Values should be of same length: %d != %d\n%s\n%s", - v1.length, v2.length, Arrays.toString(v1), Arrays.toString(v2)); - - for (int i = 0; i < v1.length; i++) - { - int res = ((Comparable) v2[i]).compareTo(v1[i]); - if (res != 0) - { - if (schema.clusteringKeys.get(i).type.isReversed()) - res = res * -1; - - return res; - } - } - return 0; - } -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/dsl/ReplayingHistoryBuilder.java b/test/harry/main/org/apache/cassandra/harry/dsl/ReplayingHistoryBuilder.java index 60d8e4ea5326..830c3ad4efcd 100644 --- a/test/harry/main/org/apache/cassandra/harry/dsl/ReplayingHistoryBuilder.java +++ b/test/harry/main/org/apache/cassandra/harry/dsl/ReplayingHistoryBuilder.java @@ -1,120 +1,73 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.cassandra.harry.dsl; -import java.util.function.Consumer; +import java.util.function.Function; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.Model; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.tracker.DataTracker; -import org.apache.cassandra.harry.visitors.ReplayingVisitor; +import org.apache.cassandra.harry.execution.CQLVisitExecutor; +import org.apache.cassandra.harry.gen.ValueGenerators; +import org.apache.cassandra.harry.op.Visit; public class ReplayingHistoryBuilder extends HistoryBuilder { - private final ReplayingVisitor visitor; - private final SystemUnderTest sut; - public final DataTracker tracker; + private final CQLVisitExecutor executor; - public ReplayingHistoryBuilder(long seed, - int maxPartitionSize, - int interleaveWindowSize, - DataTracker tracker, - SystemUnderTest sut, - SchemaSpec schema, - TokenPlacementModel.ReplicationFactor rf, - SystemUnderTest.ConsistencyLevel writeCl) + public ReplayingHistoryBuilder(ValueGenerators generators, Function<HistoryBuilder, CQLVisitExecutor> executorFactory) { - super(seed, maxPartitionSize, interleaveWindowSize, schema, rf); - this.visitor = visitor(tracker, sut, writeCl); - this.tracker = tracker; - this.sut = sut; - } + super((IndexedValueGenerators) generators); + this.executor = executorFactory.apply(this); + } - @Override - protected SingleOperationVisitBuilder singleOpVisitBuilder(long pd, long lts, Consumer<PartitionVisitState> setupPs) + SingleOperationVisitBuilder singleOpVisitBuilder() { - PartitionVisitStateImpl partitionState = presetSelector.register(lts, pd, setupPs); - return new SingleOperationVisitBuilder(partitionState, lts, pureRng, descriptorSelector, schema, valueHelper, (visit) -> { - log.put(lts, visit); - }) { + long visitLts = nextOpIdx++; + HistoryBuilder this_ = this; + return new SingleOperationVisitBuilder(visitLts, + valueGenerators, + indexGenerators, + (visit) -> log.put(visit.lts, visit)) { @Override - void end() + Visit build() { - super.end(); - visitor.replayAll(); + Visit visit = super.build(); + CQLVisitExecutor.executeVisit(visit, executor, this_); + return visit; } }; } @Override - public BatchVisitBuilder beginBatch() - { - long visitLts = clock.nextLts(); - return batchVisitBuilder(defaultSelector.pd(visitLts, schema), visitLts); - } - - /** - * Begin batch for a partition descriptor at a specific index. - * - * Imagine all partition descriptors were longs in an array. Index of a descriptor - * is a sequential number of the descriptor in this imaginary array. - */ - @Override - public BatchVisitBuilder beginBatch(long pdIdx) - { - long visitLts = clock.nextLts(); - return batchVisitBuilder(presetSelector.pdAtPosition(pdIdx), visitLts); - } - - @Override - protected BatchVisitBuilder batchVisitBuilder(long pd, long lts) + public MultiOperationVisitBuilder multistep() { - PartitionVisitStateImpl partitionState = presetSelector.register(lts, pd, (ps) -> {}); - return new BatchVisitBuilder(this, partitionState, lts, pureRng, descriptorSelector, schema, valueHelper, (visit) -> { - log.put(lts, visit); - }) { + long visitLts = nextOpIdx++; + HistoryBuilder this_ = this; + return new MultiOperationVisitBuilder(visitLts, + valueGenerators, + indexGenerators, + visit -> log.put(visit.lts, visit)) { @Override - public HistoryBuilder endBatch() + Visit buildInternal() { - super.endBatch(); - visitor.replayAll(); - return ReplayingHistoryBuilder.this; + Visit visit = super.buildInternal(); + CQLVisitExecutor.executeVisit(visit, executor, this_); + return visit; } }; } - - public ReplayingHistoryBuilder validate(int... partitions) - { - validate(tracker, sut, partitions); - return this; - } - - public Model quiescentChecker() - { - return quiescentChecker(tracker, sut); - } - - public Model quiescentLocalChecker() - { - return quiescentLocalChecker(tracker, sut); - } - -} +} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/dsl/SingleOperationBuilder.java b/test/harry/main/org/apache/cassandra/harry/dsl/SingleOperationBuilder.java index a9b98b69ca15..49a73268a302 100644 --- a/test/harry/main/org/apache/cassandra/harry/dsl/SingleOperationBuilder.java +++ b/test/harry/main/org/apache/cassandra/harry/dsl/SingleOperationBuilder.java @@ -1,52 +1,85 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.cassandra.harry.dsl; +import org.apache.cassandra.harry.Relations; +import org.apache.cassandra.harry.op.Operations; +import org.apache.cassandra.harry.util.BitSet; +import org.apache.cassandra.harry.util.ThrowingRunnable; + public interface SingleOperationBuilder { - /** - * Perform an insert operation to _some_ row - */ - SingleOperationBuilder insert(); + default SingleOperationBuilder customThrowing(ThrowingRunnable runnable, String tag) + { + return custom(runnable.toRunnable(), tag); + } + SingleOperationBuilder custom(Runnable runnable, String tag); + SingleOperationBuilder custom(OperationFactory factory); - /** - * Perform an insert operation to a _specific_ row. Rows are ordered by clustering key and - * numbered from 0 to maxRows - */ - SingleOperationBuilder insert(int rowIdx); + SingleOperationBuilder update(); + SingleOperationBuilder update(int pdIdx); + SingleOperationBuilder update(int pdIdx, int cdIdx); + SingleOperationBuilder update(int pdIdx, int cdIdx, int[] valueIdxs, int[] sValueIdxs); + + SingleOperationBuilder insert(); + SingleOperationBuilder insert(int pdIdx); + SingleOperationBuilder insert(int pdIdx, int cdIdx); + SingleOperationBuilder insert(int pdIdx, int cdIdx, int[] valueIdxs, int[] sValueIdxs); - /** - * Insert _specific values_ into _specific_ row. Rows are ordered by clustering key and - * numbered from 0 to maxRows - */ - SingleOperationBuilder insert(int rowIdx, long[] valueIdxs); - SingleOperationBuilder insert(int rowIdx, long[] valueIdxs, long[] sValueIdxs); + // TODO: selection + SingleOperationBuilder selectRowRange(int pdIdx, int lowerBoundRowIdx, int upperBoundRowIdx, + int nonEqFrom, boolean includeLowBound, boolean includeHighBound); + SingleOperationBuilder selectPartition(int pdIdx); + SingleOperationBuilder selectPartition(int pdIdx, Operations.ClusteringOrderBy orderBy); + SingleOperationBuilder selectRow(int pdIdx, int cdIdx); + SingleOperationBuilder selectRowSliceByLowerBound(int pdIdx, int lowerBoundRowIdx, int nonEqFrom, boolean isEq); + SingleOperationBuilder selectRowSliceByUpperBound(int pdIdx, int upperBoundRowIdx, int nonEqFrom, boolean isEq); - SingleOperationBuilder deletePartition(); + SingleOperationBuilder deletePartition(int pdIdx); + SingleOperationBuilder deleteRow(int pdIdx, int cdIdx); + SingleOperationBuilder deleteColumns(int pdIdx, int cdIdx, BitSet regularSelection, BitSet staticSelection); + SingleOperationBuilder deleteRowSliceByLowerBound(int pdIdx, int lowerBoundRowIdx, int nonEqFrom, boolean isEq); + SingleOperationBuilder deleteRowSliceByUpperBound(int pdIdx, int upperBoundRowIdx, int nonEqFrom, boolean isEq); + SingleOperationBuilder deleteRowRange(int pdIdx, int lowerBoundRowIdx, int upperBoundRowIdx, + int nonEqFrom, boolean includeLowBound, boolean includeHighBound); - SingleOperationBuilder deleteRow(); - SingleOperationBuilder deleteRow(int rowIdx); + SingleOperationBuilder select(int pdIdx, + IdxRelation[] ckRelations, + IdxRelation[] regularRelations, + IdxRelation[] staticRelations); - SingleOperationBuilder deleteColumns(); + class IdxRelation + { + public final Relations.RelationKind kind; + public final int idx; + public final int column; - SingleOperationBuilder deleteRowRange(); - SingleOperationBuilder deleteRowRange(int lowBoundRowIdx, int highBoundRowIdx, boolean isMinEq, boolean isMaxEq); + public IdxRelation(Relations.RelationKind kind, int idx, int column) + { + this.kind = kind; + this.idx = idx; + this.column = column; + } + } - SingleOperationBuilder deleteRowSlice(); + interface OperationFactory + { + Operations.Operation make(long lts, long opId); + } } diff --git a/test/harry/main/org/apache/cassandra/harry/dsl/SingleOperationVisitBuilder.java b/test/harry/main/org/apache/cassandra/harry/dsl/SingleOperationVisitBuilder.java index 33eca3238c01..b4f2ff908bf2 100644 --- a/test/harry/main/org/apache/cassandra/harry/dsl/SingleOperationVisitBuilder.java +++ b/test/harry/main/org/apache/cassandra/harry/dsl/SingleOperationVisitBuilder.java @@ -1,295 +1,577 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.cassandra.harry.dsl; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.function.Consumer; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.gen.EntropySource; -import org.apache.cassandra.harry.gen.rng.JdkRandomEntropySource; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.operations.Query; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import accord.utils.Invariants; +import org.apache.cassandra.harry.gen.IndexGenerators; +import org.apache.cassandra.harry.gen.rng.PCGFastPure; +import org.apache.cassandra.harry.gen.rng.PureRng; +import org.apache.cassandra.harry.gen.rng.SeedableEntropySource; +import org.apache.cassandra.harry.MagicConstants; +import org.apache.cassandra.harry.Relations; +import org.apache.cassandra.harry.op.Operations; +import org.apache.cassandra.harry.op.Visit; import org.apache.cassandra.harry.util.BitSet; -import org.apache.cassandra.harry.visitors.GeneratingVisitor; -import org.apache.cassandra.harry.visitors.ReplayingVisitor; -import org.apache.cassandra.harry.visitors.VisitExecutor; + +import static org.apache.cassandra.harry.dsl.HistoryBuilder.*; +import static org.apache.cassandra.harry.op.Operations.Kind; +import static org.apache.cassandra.harry.op.Operations.Operation; +import static org.apache.cassandra.harry.op.Operations.WriteOp; class SingleOperationVisitBuilder implements SingleOperationBuilder { - // TODO: singleton collection for this op class - private final List<VisitExecutor.BaseOperation> operations; - private final PartitionVisitStateImpl partitionState; + private static final Logger logger = LoggerFactory.getLogger(SingleOperationVisitBuilder.class); - private final long lts; - private final long pd; + // TODO: singleton collection for this op class + protected final List<Operation> operations; - private final OpSelectors.PureRng rng; + protected final long lts; - private final OpSelectors.DescriptorSelector descriptorSelector; - private final ValueHelper valueHelper; - private final SchemaSpec schema; + protected final Consumer<Visit> appendToLog; + protected final SeedableEntropySource rngSupplier = new SeedableEntropySource(); + protected final PureRng seedSelector; - private final Consumer<ReplayingVisitor.Visit> appendToLog; - private final WithEntropySource rngSupplier = new WithEntropySource(); + protected int opIdCounter; - private int opIdCounter; + protected final IndexedValueGenerators valueGenerators; + protected final IndexGenerators indexGenerators; - public SingleOperationVisitBuilder(PartitionVisitStateImpl partitionState, - long lts, - OpSelectors.PureRng rng, - OpSelectors.DescriptorSelector descriptorSelector, - SchemaSpec schema, - ValueHelper valueHelper, - Consumer<ReplayingVisitor.Visit> appendToLog) + SingleOperationVisitBuilder(long lts, + IndexedValueGenerators valueGenerators, + IndexGenerators indexGenerators, + Consumer<Visit> appendToLog) { this.operations = new ArrayList<>(); - this.partitionState = partitionState; - - this.pd = partitionState.pd; this.lts = lts; - this.rng = rng; - - this.descriptorSelector = descriptorSelector; - this.valueHelper = valueHelper; - this.schema = schema; - this.appendToLog = appendToLog; this.opIdCounter = 0; + + this.valueGenerators = valueGenerators; + this.indexGenerators = indexGenerators; + + this.seedSelector = new PureRng.PCGFast(lts); } @Override - public SingleOperationVisitBuilder insert() + public SingleOperationBuilder insert() { - int clusteringOffset = rngSupplier.withSeed(lts).nextInt(0, partitionState.possibleCds.length - 1); - return insert(clusteringOffset); + int opId = opIdCounter++; + long seed = PCGFastPure.shuffle(PCGFastPure.advanceState(lts, opId, lts)); + return rngSupplier.computeWithSeed(seed, rng -> { + int pdIdx = indexGenerators.pkIdxGen.generate(rng); + int cdIdx = indexGenerators.ckIdxGen.generate(rng); + + int[] valueIdxs = new int[indexGenerators.regularIdxGens.length]; + for (int i = 0; i < valueIdxs.length; i++) + valueIdxs[i] = indexGenerators.regularIdxGens[i].generate(rng); + int[] sValueIdxs = new int[indexGenerators.staticIdxGens.length]; + for (int i = 0; i < sValueIdxs.length; i++) + sValueIdxs[i] = indexGenerators.staticIdxGens[i].generate(rng); + return insert(pdIdx, cdIdx, valueIdxs, sValueIdxs); + }); } @Override - public SingleOperationVisitBuilder insert(int rowIdx) + public SingleOperationBuilder insert(int pdIdx) { int opId = opIdCounter++; - long cd = partitionState.possibleCds[rowIdx]; - operations.add(new GeneratingVisitor.GeneratedWriteOp(lts, pd, cd, opId, - OpSelectors.OperationKind.INSERT) - { - public long[] vds() + long seed = PCGFastPure.shuffle(PCGFastPure.advanceState(lts, opId, lts)); + return rngSupplier.computeWithSeed(seed, rng -> { + int cdIdx = indexGenerators.ckIdxGen.generate(rng); + + int[] valueIdxs = new int[indexGenerators.regularIdxGens.length]; + for (int i = 0; i < valueIdxs.length; i++) + valueIdxs[i] = indexGenerators.regularIdxGens[i].generate(rng); + int[] sValueIdxs = new int[indexGenerators.staticIdxGens.length]; + for (int i = 0; i < sValueIdxs.length; i++) + sValueIdxs[i] = indexGenerators.staticIdxGens[i].generate(rng); + return insert(pdIdx, cdIdx, valueIdxs, sValueIdxs); + }); + } + + @Override + public SingleOperationBuilder insert(int pdIdx, int cdIdx) + { + int opId = opIdCounter++; + long seed = PCGFastPure.shuffle(PCGFastPure.advanceState(lts, opId, lts)); + return rngSupplier.computeWithSeed(seed, rng -> { + int[] valueIdxs = new int[indexGenerators.regularIdxGens.length]; + for (int i = 0; i < valueIdxs.length; i++) + valueIdxs[i] = indexGenerators.regularIdxGens[i].generate(rng); + int[] sValueIdxs = new int[indexGenerators.staticIdxGens.length]; + for (int i = 0; i < sValueIdxs.length; i++) + sValueIdxs[i] = indexGenerators.staticIdxGens[i].generate(rng); + return insert(pdIdx, cdIdx, valueIdxs, sValueIdxs); + }); + } + + @Override + public SingleOperationBuilder insert(int pdIdx, int cdIdx, int[] valueIdxs, int[] sValueIdxs) + { + return write(pdIdx, cdIdx, valueIdxs, sValueIdxs, Kind.INSERT); + } + + @Override + public SingleOperationBuilder custom(Runnable runnable, String tag) + { + // TODO: assert that custom op is always alone in visit + operations.add(new Operations.CustomRunnableOperation(lts, opIdCounter, runnable) { + @Override + public String toString() { - return descriptorSelector.vds(pd, cd, lts, opId, kind(), schema); + return String.format("%s (%s)", Kind.CUSTOM, tag); } }); - end(); + build(); + return this; + } + + @Override + public SingleOperationBuilder custom(OperationFactory factory) + { + operations.add(factory.make(lts, opIdCounter++)); + build(); return this; } @Override - public SingleOperationVisitBuilder insert(int rowIdx, long[] valueIdxs) + public SingleOperationBuilder update() + { + int opId = opIdCounter++; + long seed = PCGFastPure.shuffle(PCGFastPure.advanceState(lts, opId, lts)); + return rngSupplier.computeWithSeed(seed, rng -> { + int pdIdx = indexGenerators.pkIdxGen.generate(rng); + int cdIdx = indexGenerators.ckIdxGen.generate(rng); + + int[] valueIdxs = new int[indexGenerators.regularIdxGens.length]; + for (int i = 0; i < valueIdxs.length; i++) + valueIdxs[i] = indexGenerators.regularIdxGens[i].generate(rng); + int[] sValueIdxs = new int[indexGenerators.staticIdxGens.length]; + for (int i = 0; i < sValueIdxs.length; i++) + sValueIdxs[i] = indexGenerators.staticIdxGens[i].generate(rng); + return update(pdIdx, cdIdx, valueIdxs, sValueIdxs); + }); + } + + @Override + public SingleOperationBuilder update(int pdIdx) + { + int opId = opIdCounter++; + long seed = PCGFastPure.shuffle(PCGFastPure.advanceState(lts, opId, lts)); + return rngSupplier.computeWithSeed(seed, rng -> { + int cdIdx = indexGenerators.ckIdxGen.generate(rng); + + int[] valueIdxs = new int[indexGenerators.regularIdxGens.length]; + for (int i = 0; i < valueIdxs.length; i++) + valueIdxs[i] = indexGenerators.regularIdxGens[i].generate(rng); + int[] sValueIdxs = new int[indexGenerators.staticIdxGens.length]; + for (int i = 0; i < sValueIdxs.length; i++) + sValueIdxs[i] = indexGenerators.staticIdxGens[i].generate(rng); + return update(pdIdx, cdIdx, valueIdxs, sValueIdxs); + }); + } + + @Override + public SingleOperationBuilder update(int pdIdx, int cdIdx) { - assert valueIdxs.length == valueHelper.regularColumns.size(); int opId = opIdCounter++; - long cd = partitionState.possibleCds[rowIdx]; - operations.add(new GeneratingVisitor.GeneratedWriteOp(lts, pd, cd, opId, - OpSelectors.OperationKind.INSERT) + long seed = PCGFastPure.shuffle(PCGFastPure.advanceState(lts, opId, lts)); + return rngSupplier.computeWithSeed(seed, rng -> { + int[] valueIdxs = new int[indexGenerators.regularIdxGens.length]; + for (int i = 0; i < valueIdxs.length; i++) + valueIdxs[i] = indexGenerators.regularIdxGens[i].generate(rng); + int[] sValueIdxs = new int[indexGenerators.staticIdxGens.length]; + for (int i = 0; i < sValueIdxs.length; i++) + sValueIdxs[i] = indexGenerators.staticIdxGens[i].generate(rng); + return update(pdIdx, cdIdx, valueIdxs, sValueIdxs); + }); + } + + @Override + public SingleOperationBuilder update(int pdIdx, int cdIdx, int[] valueIdxs, int[] sValueIdxs) + { + return write(pdIdx, cdIdx, valueIdxs, sValueIdxs, Kind.UPDATE); + } + + private SingleOperationBuilder write(int pdIdx, int cdIdx, int[] valueIdxs, int[] sValueIdxs, Kind kind) + { + assert valueIdxs.length == valueGenerators.regularColumnCount(); + assert sValueIdxs.length == valueGenerators.staticColumnCount(); + + long pd = valueGenerators.pkGen().descriptorAt(pdIdx); + long cd = valueGenerators.ckGen().descriptorAt(cdIdx); + + opIdCounter++; + long[] vds = new long[valueIdxs.length]; + for (int i = 0; i < valueGenerators.regularColumnCount(); i++) { - public long[] vds() + int valueIdx = valueIdxs[i]; + if (valueIdx == MagicConstants.UNSET_IDX) + vds[i] = MagicConstants.UNSET_DESCR; + else + vds[i] = valueGenerators.regularColumnGen(i).descriptorAt(valueIdx); + } + + long[] sds = new long[sValueIdxs.length]; + for (int i = 0; i < sValueIdxs.length; i++) + { + int valueIdx = sValueIdxs[i]; + if (valueIdx == MagicConstants.UNSET_IDX) + sds[i] = MagicConstants.UNSET_DESCR; + else + sds[i] = valueGenerators.staticColumnGen(i).descriptorAt(valueIdx); + } + + operations.add(new WriteOp(lts, pd, cd, vds, sds, kind) { + @Override + public String toString() { - long[] vds = new long[valueIdxs.length]; - for (int i = 0; i < valueHelper.regularColumns.size(); i++) - { - vds[i] = valueHelper.descriptorGenerators - .get(valueHelper.regularColumns.get(i).name) - .inflate(valueIdxs[i]); - } - return vds; + return String.format("%s (%d, %d, %s, %s)", + kind, pdIdx, cdIdx, Arrays.toString(valueIdxs), Arrays.toString(sValueIdxs)); } }); - end(); + build(); return this; } @Override - public SingleOperationBuilder insert(int rowIdx, long[] valueIdxs, long[] sValueIdxs) + public SingleOperationVisitBuilder deleteRowRange(int pdIdx, int lowerBoundRowIdx, int upperBoundRowIdx, + int nonEqFrom, boolean includeLowerBound, boolean includeUpperBound) { - assert valueIdxs.length == valueHelper.regularColumns.size(); - assert sValueIdxs.length == valueHelper.staticColumns.size(); + long pd = valueGenerators.pkGen().descriptorAt(pdIdx); + + long lowerBoundCd = valueGenerators.ckGen().descriptorAt(lowerBoundRowIdx); + long upperBoundCd = valueGenerators.ckGen().descriptorAt(upperBoundRowIdx); + + Relations.RelationKind[] lowerBoundRelations = new Relations.RelationKind[valueGenerators.ckColumnCount()]; + Relations.RelationKind[] upperBoundRelations = new Relations.RelationKind[valueGenerators.ckColumnCount()]; + int opId = opIdCounter++; - long cd = partitionState.possibleCds[rowIdx]; - operations.add(new GeneratingVisitor.GeneratedWriteWithStaticOp(lts, pd, cd, opId, - OpSelectors.OperationKind.INSERT_WITH_STATICS) - { - @Override - public long[] vds() + rngSupplier.doWithSeed(opId, rng -> { + for (int i = 0; i < Math.min(nonEqFrom + 1, valueGenerators.ckColumnCount()); i++) { - long[] vds = new long[valueIdxs.length]; - for (int i = 0; i < valueHelper.regularColumns.size(); i++) + if (i < nonEqFrom) + lowerBoundRelations[i] = Relations.RelationKind.EQ; + else { - vds[i] = valueHelper.descriptorGenerators - .get(valueHelper.regularColumns.get(i).name) - .inflate(valueIdxs[i]); + lowerBoundRelations[i] = includeLowerBound ? Relations.RelationKind.GTE : Relations.RelationKind.GT; + upperBoundRelations[i] = includeUpperBound ? Relations.RelationKind.LTE : Relations.RelationKind.LT; } - return vds; } + }); + operations.add(new Operations.DeleteRange(lts, pd, + lowerBoundCd, upperBoundCd, + lowerBoundRelations, upperBoundRelations) { @Override - public long[] sds() + public String toString() { - long[] sds = new long[sValueIdxs.length]; - for (int i = 0; i < valueHelper.staticColumns.size(); i++) - { - sds[i] = valueHelper.descriptorGenerators - .get(valueHelper.staticColumns.get(i).name) - .inflate(sValueIdxs[i]); - } - return sds; + return String.format("DELETE (%d, >%s%d, <%s%d) - (%d)", + pdIdx, + includeLowerBound ? "=" : "", lowerBoundRowIdx, + includeUpperBound ? "=" : "", upperBoundRowIdx, + nonEqFrom); } }); - end(); + build(); return this; } @Override - public SingleOperationVisitBuilder deletePartition() + public SingleOperationBuilder deletePartition(int pdIdx) { - int opId = opIdCounter++; - operations.add(new GeneratingVisitor.GeneratedDeleteOp(lts, pd, opId, OpSelectors.OperationKind.DELETE_PARTITION, - Query.selectAllColumns(schema, pd, false))); - end(); + long pd = valueGenerators.pkGen().descriptorAt(pdIdx); + opIdCounter++; + + operations.add(new Operations.DeletePartition(lts, pd) { + @Override + public String toString() + { + return String.format("DELETE_PARTITION (%d)", pdIdx); + } + }); + build(); return this; } @Override - public SingleOperationVisitBuilder deleteRow() + public SingleOperationBuilder deleteRow(int pdIdx, int rowIdx) { - int opId = opIdCounter++; - long queryDescriptor = rng.next(opId, lts); - rngSupplier.withSeed(queryDescriptor, (rng) -> { - int cdIdx = rngSupplier.withSeed(queryDescriptor).nextInt(partitionState.possibleCds.length); - long cd = partitionState.possibleCds[cdIdx]; - operations.add(new GeneratingVisitor.GeneratedDeleteRowOp(lts, pd, cd, opId, - OpSelectors.OperationKind.DELETE_ROW)); + long pd = valueGenerators.pkGen().descriptorAt(pdIdx); + opIdCounter++; + long cd = valueGenerators.ckGen().descriptorAt(rowIdx); + operations.add(new Operations.DeleteRow(lts, pd, cd) { + @Override + public String toString() + { + return String.format("DELETE_ROW (%d, %s)", pdIdx, rowIdx); + } + }); + build(); + return this; + } + + @Override + public SingleOperationBuilder deleteColumns(int pdIdx, int rowIdx, BitSet regularSelection, BitSet staticSelection) + { + long pd = valueGenerators.pkGen().descriptorAt(pdIdx); + opIdCounter++; + long cd = valueGenerators.ckGen().descriptorAt(rowIdx); + operations.add(new Operations.DeleteColumns(lts, pd, cd, regularSelection, staticSelection) { + @Override + public String toString() + { + return String.format("DELETE_COLUMNS (%d, %d, %s, %s)", pdIdx, rowIdx, regularSelection, staticSelection); + } }); - end(); + build(); return this; } @Override - public SingleOperationVisitBuilder deleteRow(int cdIdx) + public SingleOperationBuilder deleteRowSliceByLowerBound(int pdIdx, int lowerBoundRowIdx, int nonEqFrom, boolean includeBound) { + long pd = valueGenerators.pkGen().descriptorAt(pdIdx); + long lowerBoundCd = valueGenerators.ckGen().descriptorAt(lowerBoundRowIdx); + + Relations.RelationKind[] lowerBoundRelations = new Relations.RelationKind[valueGenerators.ckColumnCount()]; + int opId = opIdCounter++; - long cd = partitionState.possibleCds[cdIdx]; - operations.add(new GeneratingVisitor.GeneratedDeleteRowOp(lts, pd, cd, opId, - OpSelectors.OperationKind.DELETE_ROW)); - end(); + rngSupplier.doWithSeed(opId, rng -> { + for (int i = 0; i < Math.min(nonEqFrom + 1, valueGenerators.ckColumnCount()); i++) + { + if (i < nonEqFrom) + lowerBoundRelations[i] = Relations.RelationKind.EQ; + else + lowerBoundRelations[i] = includeBound ? Relations.RelationKind.GTE : Relations.RelationKind.GT; + } + }); + + operations.add(new Operations.DeleteRange(lts, pd, + lowerBoundCd, MagicConstants.UNSET_DESCR, + lowerBoundRelations, null) { + @Override + public String toString() + { + return String.format("DELETE (%d, >%s%d) - (%d)", + pdIdx, includeBound ? "=" : "", lowerBoundRowIdx, nonEqFrom); + } + }); + build(); return this; } @Override - public SingleOperationVisitBuilder deleteColumns() + public SingleOperationBuilder deleteRowSliceByUpperBound(int pdIdx, int upperBoundRowIdx, int nonEqFrom, boolean includeBound) { + long pd = valueGenerators.pkGen().descriptorAt(pdIdx); + long upperBoundCd = valueGenerators.ckGen().descriptorAt(upperBoundRowIdx); + + Relations.RelationKind[] upperBoundRelations = new Relations.RelationKind[valueGenerators.ckColumnCount()]; + int opId = opIdCounter++; - long queryDescriptor = rng.next(opId, lts); - rngSupplier.withSeed(queryDescriptor, (rng) -> { - int cdIdx = rng.nextInt(partitionState.possibleCds.length); - long cd = partitionState.possibleCds[cdIdx]; - BitSet columns = descriptorSelector.columnMask(pd, lts, opId, OpSelectors.OperationKind.DELETE_COLUMN); - operations.add(new GeneratingVisitor.GeneratedDeleteColumnsOp(lts, pd, cd, opId, - OpSelectors.OperationKind.DELETE_COLUMN, columns)); + rngSupplier.doWithSeed(opId, rng -> { + for (int i = 0; i < Math.min(nonEqFrom + 1, valueGenerators.ckColumnCount()); i++) + { + if (i < nonEqFrom) + upperBoundRelations[i] = Relations.RelationKind.EQ; + else + upperBoundRelations[i] = includeBound ? Relations.RelationKind.LTE : Relations.RelationKind.LT; + } + }); + + operations.add(new Operations.DeleteRange(lts, pd, + MagicConstants.UNSET_DESCR, upperBoundCd, + null, upperBoundRelations) { + @Override + public String toString() + { + return String.format("DELETE (%d, <%s%d) - (%d)", + pdIdx, includeBound ? "=" : "", upperBoundRowIdx, nonEqFrom); + } }); - end(); + build(); return this; } + @Override - public SingleOperationVisitBuilder deleteRowRange() + public SingleOperationVisitBuilder selectRowRange(int pdIdx, int lowerBoundRowIdx, int upperBoundRowIdx, + int nonEqFrom, boolean includeLowerBound, boolean includeUpperBound) { + long pd = valueGenerators.pkGen().descriptorAt(pdIdx); + long lowerBoundCd = valueGenerators.ckGen().descriptorAt(lowerBoundRowIdx); + long upperBoundCd = valueGenerators.ckGen().descriptorAt(upperBoundRowIdx); + + Relations.RelationKind[] lowerBoundRelations = new Relations.RelationKind[valueGenerators.ckColumnCount()]; + Relations.RelationKind[] upperBoundRelations = new Relations.RelationKind[valueGenerators.ckColumnCount()]; + int opId = opIdCounter++; - long queryDescriptor = rng.next(opId, lts); - rngSupplier.withSeed(queryDescriptor, (rng) -> { - Query query = null; - while (query == null) + rngSupplier.doWithSeed(opId, rng -> { + for (int i = 0; i < Math.min(nonEqFrom + 1, valueGenerators.ckColumnCount()); i++) { - try + if (i < nonEqFrom) + lowerBoundRelations[i] = Relations.RelationKind.EQ; + else { - long cd1 = partitionState.possibleCds[rng.nextInt(partitionState.possibleCds.length)]; - long cd2 = partitionState.possibleCds[rng.nextInt(partitionState.possibleCds.length)]; - while (cd2 == cd1) - cd2 = partitionState.possibleCds[rng.nextInt(partitionState.possibleCds.length)]; - - boolean isMinEq = rng.nextBoolean(); - boolean isMaxEq = rng.nextBoolean(); - query = Query.clusteringRangeQuery(schema, pd, cd1, cd2, queryDescriptor, isMinEq, isMaxEq, false); - break; - } - catch (IllegalArgumentException retry) - { - continue; + lowerBoundRelations[i] = includeLowerBound ? Relations.RelationKind.GTE : Relations.RelationKind.GT; + upperBoundRelations[i] = includeUpperBound ? Relations.RelationKind.LTE : Relations.RelationKind.LT; } } - operations.add(new GeneratingVisitor.GeneratedDeleteOp(lts, pd, opId, OpSelectors.OperationKind.DELETE_SLICE, query)); }); - end(); + + operations.add(new Operations.SelectRange(lts, pd, + lowerBoundCd, upperBoundCd, + lowerBoundRelations, upperBoundRelations)); + build(); return this; } @Override - public SingleOperationVisitBuilder deleteRowRange(int lowBoundRowIdx, int highBoundRowIdx, boolean isMinEq, boolean isMaxEq) + public SingleOperationBuilder select(int pdIdx, IdxRelation[] ckIdxRelations, IdxRelation[] regularIdxRelations, IdxRelation[] staticIdxRelations) { - int opId = opIdCounter++; - long queryDescriptor = rng.next(opId, lts); + long pd = valueGenerators.pkGen().descriptorAt(pdIdx); + opIdCounter++; + + Relations.Relation[] ckRelations = new Relations.Relation[ckIdxRelations.length]; + for (int i = 0; i < ckRelations.length; i++) + { + Invariants.checkState(ckIdxRelations[i].column < valueGenerators.ckColumnCount()); + ckRelations[i] = new Relations.Relation(ckIdxRelations[i].kind, + valueGenerators.ckGen().descriptorAt(ckIdxRelations[i].idx), + ckIdxRelations[i].column); + } + + Relations.Relation[] regularRelations = new Relations.Relation[regularIdxRelations.length]; + for (int i = 0; i < regularRelations.length; i++) + { + Invariants.checkState(regularIdxRelations[i].column < valueGenerators.regularColumnCount()); + regularRelations[i] = new Relations.Relation(regularIdxRelations[i].kind, + valueGenerators.regularColumnGen(regularIdxRelations[i].column).descriptorAt(regularIdxRelations[i].idx), + regularIdxRelations[i].column); + } + + Relations.Relation[] staticRelations = new Relations.Relation[staticIdxRelations.length]; + for (int i = 0; i < staticRelations.length; i++) + { + Invariants.checkState(staticIdxRelations[i].column < valueGenerators.staticColumnCount()); + staticRelations[i] = new Relations.Relation(staticIdxRelations[i].kind, + valueGenerators.staticColumnGen(staticIdxRelations[i].column).descriptorAt(staticIdxRelations[i].idx), + staticIdxRelations[i].column); + } + + operations.add(new Operations.SelectCustom(lts, pd, ckRelations, regularRelations, staticRelations)); + build(); + return this; + } + + @Override + public SingleOperationBuilder selectPartition(int pdIdx) + { + long pd = valueGenerators.pkGen().descriptorAt(pdIdx); + opIdCounter++; - long cd1 = partitionState.possibleCds[lowBoundRowIdx]; - long cd2 = partitionState.possibleCds[highBoundRowIdx]; - Query query = Query.clusteringRangeQuery(schema, pd, cd1, cd2, queryDescriptor, isMinEq, isMaxEq, false); - operations.add(new GeneratingVisitor.GeneratedDeleteOp(lts, pd, opId, OpSelectors.OperationKind.DELETE_SLICE, query)); - end(); + operations.add(new Operations.SelectPartition(lts, pd)); + build(); return this; } @Override - public SingleOperationVisitBuilder deleteRowSlice() + public SingleOperationBuilder selectPartition(int pdIdx, Operations.ClusteringOrderBy orderBy) { + long pd = valueGenerators.pkGen().descriptorAt(pdIdx); + opIdCounter++; + + operations.add(new Operations.SelectPartition(lts, pd, orderBy)); + build(); + return this; + } + + @Override + public SingleOperationBuilder selectRow(int pdIdx, int rowIdx) + { + long pd = valueGenerators.pkGen().descriptorAt(pdIdx); + opIdCounter++; + long cd = valueGenerators.ckGen().descriptorAt(rowIdx); + operations.add(new Operations.SelectRow(lts, pd, cd)); + build(); + return this; + } + + @Override + public SingleOperationBuilder selectRowSliceByLowerBound(int pdIdx, int lowerBoundRowIdx, int nonEqFrom, boolean includeBound) + { + long pd = valueGenerators.pkGen().descriptorAt(pdIdx); + long lowerBoundCd = valueGenerators.ckGen().descriptorAt(lowerBoundRowIdx); + + Relations.RelationKind[] lowerBoundRelations = new Relations.RelationKind[valueGenerators.ckColumnCount()]; + int opId = opIdCounter++; - long queryDescriptor = rng.next(opId, lts); - rngSupplier.withSeed(queryDescriptor, (rng) -> { - Query query = null; - while (query == null) + rngSupplier.doWithSeed(opId, rng -> { + for (int i = 0; i < Math.min(nonEqFrom + 1, valueGenerators.ckColumnCount()); i++) { - try - { - int cdIdx = rng.nextInt(partitionState.possibleCds.length); - long cd = partitionState.possibleCds[cdIdx]; + if (i < nonEqFrom) + lowerBoundRelations[i] = Relations.RelationKind.EQ; + else + lowerBoundRelations[i] = includeBound ? Relations.RelationKind.GTE : Relations.RelationKind.GT; + } + }); - boolean isGt = rng.nextBoolean(); - boolean isEquals = rng.nextBoolean(); - query = Query.clusteringSliceQuery(schema, pd, cd, queryDescriptor, isGt, isEquals, false); - break; - } - catch (IllegalArgumentException retry) - { - continue; - } + operations.add(new Operations.SelectRange(lts, pd, + lowerBoundCd, MagicConstants.UNSET_DESCR, + lowerBoundRelations, null)); + build(); + return this; + } + + @Override + public SingleOperationBuilder selectRowSliceByUpperBound(int pdIdx, int upperBoundRowIdx, int nonEqFrom, boolean includeBound) + { + long pd = valueGenerators.pkGen().descriptorAt(pdIdx); + long upperBoundCd = valueGenerators.ckGen().descriptorAt(upperBoundRowIdx); + + Relations.RelationKind[] upperBoundRelations = new Relations.RelationKind[valueGenerators.ckColumnCount()]; + + int opId = opIdCounter++; + rngSupplier.doWithSeed(opId, rng -> { + for (int i = 0; i < Math.min(nonEqFrom + 1, valueGenerators.ckColumnCount()); i++) + { + if (i < nonEqFrom) + upperBoundRelations[i] = Relations.RelationKind.EQ; + else + upperBoundRelations[i] = includeBound ? Relations.RelationKind.LTE : Relations.RelationKind.LT; } - operations.add(new GeneratingVisitor.GeneratedDeleteOp(lts, pd, opId, OpSelectors.OperationKind.DELETE_SLICE, query)); }); - end(); + + operations.add(new Operations.SelectRange(lts, pd, + MagicConstants.UNSET_DESCR, upperBoundCd, + null, upperBoundRelations)); + build(); return this; } @@ -298,28 +580,12 @@ int size() return this.operations.size(); } - void end() + Visit build() { - VisitExecutor.Operation[] ops = new VisitExecutor.Operation[operations.size()]; + Operation[] ops = new Operation[operations.size()]; operations.toArray(ops); - ReplayingVisitor.Visit visit = new ReplayingVisitor.Visit(lts, pd, ops); + Visit visit = new Visit(lts, ops); appendToLog.accept(visit); - } - - private static class WithEntropySource - { - private final EntropySource entropySource = new JdkRandomEntropySource(0); - - public void withSeed(long seed, Consumer<EntropySource> rng) - { - entropySource.seed(seed); - rng.accept(entropySource); - } - - public EntropySource withSeed(long seed) - { - entropySource.seed(seed); - return entropySource; - } + return visit; } } diff --git a/test/harry/main/org/apache/cassandra/harry/util/TestRunner.java b/test/harry/main/org/apache/cassandra/harry/dsl/TestRunner.java similarity index 77% rename from test/harry/main/org/apache/cassandra/harry/util/TestRunner.java rename to test/harry/main/org/apache/cassandra/harry/dsl/TestRunner.java index 1941137436e4..1bbf6b56c81c 100644 --- a/test/harry/main/org/apache/cassandra/harry/util/TestRunner.java +++ b/test/harry/main/org/apache/cassandra/harry/dsl/TestRunner.java @@ -1,22 +1,22 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ -package org.apache.cassandra.harry.util; +package org.apache.cassandra.harry.dsl; import java.util.function.BiConsumer; import java.util.function.BiFunction; diff --git a/test/harry/main/org/apache/cassandra/harry/dsl/ValueDescriptorIndexGenerator.java b/test/harry/main/org/apache/cassandra/harry/dsl/ValueDescriptorIndexGenerator.java deleted file mode 100644 index fd100f869c8e..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/dsl/ValueDescriptorIndexGenerator.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.dsl; - -import java.util.function.LongSupplier; - -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.gen.Bytes; -import org.apache.cassandra.harry.gen.EntropySource; -import org.apache.cassandra.harry.gen.DataGenerators; -import org.apache.cassandra.harry.gen.Surjections; -import org.apache.cassandra.harry.model.OpSelectors; - -/** - * By default, descriptors in Harry are chosen as items in the stream of `pd ^ cd ^ lts ^ col`, - * but for purpose of some tests one may want to have more fine-grained control over values, - * which is particularly useful for 2i testing. - * - * Imagine all possible value descriptors as items in some array (each column would have its own - * array with different descriptors). This generator will pick an item from this array by given index. - */ -public class ValueDescriptorIndexGenerator implements Surjections.Surjection<Long> -{ - public static int UNSET = Integer.MIN_VALUE; - - private final OpSelectors.PureRng rng; - private final long columnHash; - private final long mask; - - public ValueDescriptorIndexGenerator(ColumnSpec<?> columnSpec, long seed) - { - this(columnSpec, new OpSelectors.PCGFast(seed)); - } - - public ValueDescriptorIndexGenerator(ColumnSpec<?> columnSpec, OpSelectors.PureRng rng) - { - this.rng = rng; - this.columnHash = columnSpec.hashCode(); - this.mask = Bytes.bytePatternFor(columnSpec.type.maxSize()); - } - - @Override - public Long inflate(long idx) - { - if (idx == UNSET) - return DataGenerators.UNSET_DESCR; - - return rng.randomNumber(idx, columnHash) & mask; - } - - /** - * Returns a supplier that would uniformly pick from at most {@param values} values. - * - * @param values number of possible values - */ - public LongSupplier toSupplier(EntropySource orig, int values, float chanceOfUnset) - { - EntropySource derived = orig.derive(); - if (chanceOfUnset > 0) - { - assert chanceOfUnset < 1.0; - return () -> { - if (orig.nextFloat() < chanceOfUnset) - return DataGenerators.UNSET_DESCR; - return inflate(derived.nextInt(values)); - }; - } - - return () -> inflate(derived.nextInt(values)); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/dsl/ValueHelper.java b/test/harry/main/org/apache/cassandra/harry/dsl/ValueHelper.java deleted file mode 100644 index fcce22254343..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/dsl/ValueHelper.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.dsl; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.OpSelectors; - -class ValueHelper implements ValueOverrides -{ - protected Map<String, OverridingBijection<?>> overrides = new HashMap<>(); - protected Map<String, ValueDescriptorIndexGenerator> descriptorGenerators = new HashMap<>(); - - protected final List<ColumnSpec<?>> regularColumns; - protected final List<ColumnSpec<?>> staticColumns; - - @SuppressWarnings("rawtypes,unchecked") - public ValueHelper(SchemaSpec orig, OpSelectors.PureRng rng) - { - this.regularColumns = new ArrayList<>(); - for (ColumnSpec<?> regular : orig.regularColumns) - { - OverridingBijection override = new OverridingBijection<>(regular.generator()); - regular = regular.override(override); - this.regularColumns.add(regular); - this.overrides.put(regular.name, override); - this.descriptorGenerators.put(regular.name, new ValueDescriptorIndexGenerator(regular, rng)); - } - - this.staticColumns = new ArrayList<>(); - for (ColumnSpec<?> static_ : orig.staticColumns) - { - OverridingBijection override = new OverridingBijection<>(static_.generator()); - static_ = static_.override(override); - this.staticColumns.add(static_); - this.overrides.put(static_.name, override); - this.descriptorGenerators.put(static_.name, new ValueDescriptorIndexGenerator(static_, rng)); - } - } - - @Override - @SuppressWarnings("unchecked,rawtypes") - public void override(String columnName, int idx, Object override) - { - OverridingBijection gen = overrides.get(columnName); - if (gen == null) - throw new IllegalStateException(String.format("Overrides for %s are not supported", columnName)); - if (idx == ValueDescriptorIndexGenerator.UNSET) - throw new IllegalStateException("Can't override an UNSET value"); - - gen.override(descriptorGenerators.get(columnName).inflate(idx), override); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/execution/CQLTesterVisitExecutor.java b/test/harry/main/org/apache/cassandra/harry/execution/CQLTesterVisitExecutor.java new file mode 100644 index 000000000000..0f301d8cb8f0 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/execution/CQLTesterVisitExecutor.java @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.execution; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.Function; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import accord.utils.Invariants; +import org.apache.cassandra.cql3.UntypedResultSet; +import org.apache.cassandra.harry.ColumnSpec; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.op.Visit; +import org.apache.cassandra.harry.op.Operations; +import org.apache.cassandra.harry.model.Model; + +import static org.apache.cassandra.harry.MagicConstants.*; + +public class CQLTesterVisitExecutor extends CQLVisitExecutor +{ + private static final Logger logger = LoggerFactory.getLogger(CQLTesterVisitExecutor.class); + private final Function<CompiledStatement, UntypedResultSet> execute; + + public CQLTesterVisitExecutor(SchemaSpec schema, + DataTracker dataTracker, + Model model, + Function<CompiledStatement, UntypedResultSet> execute) + { + super(schema, dataTracker, model, new QueryBuildingVisitExecutor(schema, QueryBuildingVisitExecutor.WrapQueries.UNLOGGED_BATCH)); + this.execute = execute; + } + + @Override + public List<ResultSetRow> executeWithResult(Visit visit, CompiledStatement statement) + { + List<ResultSetRow> actual = new ArrayList<>(); + // TODO: Have never tested with multiple + Invariants.checkState(visit.operations.length == 1); + for (UntypedResultSet.Row row : execute.apply(statement)) + actual.add(resultSetToRow(schema, (Operations.SelectStatement) visit.operations[0], row)); + return actual; + } + + protected void executeWithoutResult(Visit visit, CompiledStatement statement) + { + execute.apply(statement); + } + + public static ResultSetRow resultSetToRow(SchemaSpec schema, Operations.SelectStatement select, UntypedResultSet.Row row) + { + // TODO: do we want to use selection? + Operations.Selection selection = Operations.Selection.fromBitSet(select.selection(), schema); + + long pd = UNKNOWN_DESCR; + if (selection.selectsAllOf(schema.partitionKeys)) + { + Object[] partitionKey = new Object[schema.partitionKeys.size()]; + for (int i = 0; i < schema.partitionKeys.size(); i++) + { + ColumnSpec<?> column = schema.partitionKeys.get(i); + partitionKey[i] = column.type.asServerType().compose(row.getBytes(column.name)); + } + + pd = schema.valueGenerators.pkGen().deflate(partitionKey); + } + + + long cd = UNKNOWN_DESCR; + if (selection.selectsAllOf(schema.clusteringKeys)) + { + Object[] clusteringKey = new Object[schema.clusteringKeys.size()]; + for (int i = 0; i < schema.clusteringKeys.size(); i++) + { + ColumnSpec<?> column = schema.clusteringKeys.get(i); + if (row.has(column.name)) + { + clusteringKey[i] = column.type.asServerType().compose(row.getBytes(column.name)); + } + else + { + for (int j = 0; j < schema.clusteringKeys.size(); j++) + { + Invariants.checkState(!row.has(schema.clusteringKeys.get(j).name), + "All elements of clustering key should have been null"); + } + clusteringKey = NIL_KEY; + break; + } + } + + // Clusterings can not be set to nil, so if we do not see, we assume it is unset + if (clusteringKey == NIL_KEY) + cd = UNSET_DESCR; + else + cd = schema.valueGenerators.ckGen().deflate(clusteringKey); + } + + long[] regularColumns = new long[schema.regularColumns.size()]; + for (int i = 0; i < schema.regularColumns.size(); i++) + { + ColumnSpec<?> column = schema.regularColumns.get(i); + if (selection.selects(column)) + { + if (row.has(column.name)) + { + Object value = column.type.asServerType().compose(row.getBytes(column.name)); + regularColumns[i] = schema.valueGenerators.regularColumnGen(i).deflate(value); + } + else + { + regularColumns[i] = NIL_DESCR; + } + } + else + { + regularColumns[i] = UNKNOWN_DESCR; + } + } + + long[] staticColumns = new long[schema.staticColumns.size()]; + for (int i = 0; i < schema.staticColumns.size(); i++) + { + ColumnSpec<?> column = schema.staticColumns.get(i); + if (selection.selects(column)) + { + if (row.has(column.name)) + { + Object value = column.type.asServerType().compose(row.getBytes(column.name)); + staticColumns[i] = schema.valueGenerators.staticColumnGen(i).deflate(value); + } + else + { + staticColumns[i] = NIL_DESCR; + } + } + else + { + staticColumns[i] = UNKNOWN_DESCR; + } + } + + long[] regularLts = LTS_UNKNOWN; + if (selection.includeTimestamps()) + throw new IllegalStateException("not implemented for CQL Tester"); + long[] staticLts = LTS_UNKNOWN; + if (selection.includeTimestamps()) + throw new IllegalStateException("not implemented for CQL Tester"); + + return new ResultSetRow(pd, cd, staticColumns, staticLts, regularColumns, regularLts); + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/execution/CQLVisitExecutor.java b/test/harry/main/org/apache/cassandra/harry/execution/CQLVisitExecutor.java new file mode 100644 index 000000000000..1922f0011a7e --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/execution/CQLVisitExecutor.java @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.execution; + +import java.util.Arrays; +import java.util.List; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import accord.utils.Invariants; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.model.Model; +import org.apache.cassandra.harry.op.Operations; +import org.apache.cassandra.harry.op.Visit; + +/** + * + * TODO: Transactional results ; LET + */ +public abstract class CQLVisitExecutor +{ + private static final Logger logger = LoggerFactory.getLogger(QueryBuildingVisitExecutor.class); + protected final SchemaSpec schema; + + protected final DataTracker dataTracker; + protected final Model model; + private final QueryBuildingVisitExecutor queryBuilder; + + public CQLVisitExecutor(SchemaSpec schema, DataTracker dataTracker, Model model, QueryBuildingVisitExecutor queryBuilder) + { + this.schema = schema; + this.dataTracker = dataTracker; + this.model = model; + this.queryBuilder = queryBuilder; + } + + public static void replay(CQLVisitExecutor executor, Model.Replay replay) + { + for (Visit visit : replay) + { + if (visit.lts % 1000 == 0) + logger.debug("Replaying visit {}", visit.lts); + executeVisit(visit, executor, replay); + } + } + + public static void executeVisit(Visit visit, CQLVisitExecutor executor, Model.Replay replay) + { + try + { + executor.execute(visit); + } + catch (Throwable t) + { + // TODO (required): remove when CASSANDRA-19869 is fixed + if (t.getMessage() != null && t.getMessage().contains("class org.apache.cassandra.db.ReadQuery$1 cannot be cast to class org.apache.cassandra.db.SinglePartitionReadQuery$Group")) + return; + + replayAfterFailure(visit, executor, replay); + + throw t; + } + } + + public enum ResultDumpMode + { + PARTITION, + LAST_50 + } + + public static void replayAfterFailure(Visit visit, CQLVisitExecutor executor, Model.Replay replay) + { + QueryBuildingVisitExecutor queryBuilder = executor.queryBuilder; + logger.error("Caught an exception at {} while replaying {}\ncluster.schemaChange(\"{}\");\nOperations _for this partition_ up to this visit:", + visit, queryBuilder.compile(visit), + queryBuilder.schema.compile()); + + // Configurable yet hardcoded for a person who is trying to generate repro + ResultDumpMode mode = ResultDumpMode.PARTITION; + long minLts = 0; + if (mode == ResultDumpMode.LAST_50) + minLts = Math.max(0, visit.lts - 20); + for (Visit rereplay : replay) + { + if (rereplay.lts < minLts) + continue; + + // Skip if this visit was for a different partition + if (mode == ResultDumpMode.PARTITION && !intersects(visit.visitedPartitions, rereplay.visitedPartitions)) + continue; + + String str = ""; + CompiledStatement cs = queryBuilder.compile(rereplay); + if (cs != null) + str = cs.dump(ConsistencyLevel.QUORUM); + logger.info("/*{}*/ {}", rereplay, str); + + if (rereplay.lts > visit.lts) + return; + } + } + + private static boolean intersects(Set<?> set1, Set<?> set2) + { + for (Object o : set1) + { + if (set2.contains(o)) + return true; + } + return false; + } + + public void execute(Visit visit) + { + dataTracker.begin(visit); + QueryBuildingVisitExecutor.BuiltQuery compiledStatement = queryBuilder.compile(visit); + // All operations are not touching any data + if (compiledStatement == null) + { + Invariants.checkArgument(Arrays.stream(visit.operations).allMatch(op -> op.kind() == Operations.Kind.CUSTOM)); + return; + } + + List<Operations.SelectStatement> selects = compiledStatement.selects; + if (selects.isEmpty()) + { + executeMutatingVisit(visit, compiledStatement); + } + else + { + Invariants.checkState(selects.size() == 1); + executeValidatingVisit(visit, selects, compiledStatement); + } + dataTracker.end(visit); + } + + // Lives in a separate method so that it is easier to override it + protected void executeMutatingVisit(Visit visit, CompiledStatement statement) + { + executeWithoutResult(visit, statement); + } + + // Lives in a separate method so that it is easier to override it + protected void executeValidatingVisit(Visit visit, List<Operations.SelectStatement> selects, CompiledStatement compiledStatement) + { + // TODO: Have not tested with multiple + List<ResultSetRow> resultSetRow = executeWithResult(visit, compiledStatement); + try + { + model.validate(selects.get(0), resultSetRow); + } + catch (Throwable t) + { + throw new AssertionError(String.format("Caught an exception while validating %s:\n%s", selects.get(0), compiledStatement), + t); + } + } + + protected abstract List<ResultSetRow> executeWithResult(Visit visit, CompiledStatement statement); + protected abstract void executeWithoutResult(Visit visit, CompiledStatement statement); + +} diff --git a/test/harry/main/org/apache/cassandra/harry/execution/CompiledStatement.java b/test/harry/main/org/apache/cassandra/harry/execution/CompiledStatement.java new file mode 100644 index 000000000000..8e84a64b6188 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/execution/CompiledStatement.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.execution; + +import java.net.InetAddress; +import java.util.UUID; + +import org.apache.cassandra.distributed.api.ConsistencyLevel; + +public class CompiledStatement +{ + private final String cql; + private final Object[] bindings; + + public CompiledStatement(String cql, Object... bindings) + { + this.cql = cql; + this.bindings = bindings; + } + + public String cql() + { + return cql; + } + + public CompiledStatement withSchema(String oldKs, String oldTable, String newKs, String newTable) + { + return new CompiledStatement(cql.replace(oldKs + "." + oldTable, + newKs + "." + newTable), + bindings); + } + + public CompiledStatement withFiltering() + { + return new CompiledStatement(cql.replace(";", + " ALLOW FILTERING;"), + bindings); + } + + public Object[] bindings() + { + return bindings; + } + + public static CompiledStatement create(String cql, Object... bindings) + { + return new CompiledStatement(cql, bindings); + } + + public String toString() + { + return "CompiledStatement{" + + "cql=execute(\"" + cql.replace("\n", "\" + \n\"") + '\"' + + ", " + bindingsToString(bindings) + + '}'; + } + + public String dump(ConsistencyLevel cl) + { + return String.format("cluster.coordinator(1).execute(\"%s\", ConsistencyLevel.%s, %s);", + cql.replace("\n", "\" + \n\""), + cl.toString(), + bindingsToString(bindings)); + } + + public static String bindingsToString(Object... bindings) + { + StringBuilder sb = new StringBuilder(); + boolean isFirst = true; + for (Object binding : bindings) + { + if (isFirst) + isFirst = false; + else + sb.append(","); + + if (binding instanceof String) + sb.append("\"").append(binding).append("\""); + else if (binding instanceof Short) + sb.append("(short)").append(binding); + else if (binding instanceof Byte) + sb.append("(byte)").append(binding); + else if (binding instanceof Float) + sb.append("(float)").append(binding); + else if (binding instanceof Double) + sb.append("(double)").append(binding); + else if (binding instanceof Long) + sb.append(binding).append("L"); + else if (binding instanceof Integer) + sb.append("(int)").append(binding); + else if (binding instanceof Boolean) + sb.append(binding); + else if (binding instanceof UUID) + sb.append("java.util.UUID.fromString(\"").append(binding).append("\")"); + else if (binding instanceof java.sql.Timestamp) + sb.append("new java.sql.Timestamp(").append(((java.sql.Timestamp) binding).getTime()).append("L)"); + else if (binding instanceof java.sql.Time) + sb.append("new java.sql.Time(").append(((java.sql.Time) binding).getTime()).append("L)"); + else if (binding instanceof java.util.Date) + { + sb.append("new java.util.Date(") + .append(((java.util.Date) binding).getTime()) + .append("L)"); + } + else if (binding instanceof java.math.BigInteger) + sb.append("new java.math.BigInteger(\"").append(binding).append("\")"); + else if (binding instanceof java.math.BigDecimal) + sb.append("new java.math.BigDecimal(\"").append(binding).append("\")"); + else if (binding instanceof java.net.InetAddress) + { + byte[] address = ((InetAddress) binding).getAddress(); + sb.append("java.net.InetAddress.getByAddress(new byte[]{"); + for (int i = 0; i < address.length; i++) { + sb.append(address[i]); + if (i < address.length - 1) sb.append(", "); + } + sb.append("})"); + } + else + sb.append(binding); + // TODO: byte arrays + } + return sb.toString(); + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/execution/DataTracker.java b/test/harry/main/org/apache/cassandra/harry/execution/DataTracker.java new file mode 100644 index 000000000000..f2a8c9332fdc --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/execution/DataTracker.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.execution; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; + +import accord.utils.Invariants; +import org.apache.cassandra.harry.op.Visit; +import org.apache.cassandra.harry.op.Operations; +import org.apache.cassandra.harry.model.Model; + +import static org.apache.cassandra.harry.op.Operations.Kind.CUSTOM; +import static org.apache.cassandra.harry.op.Operations.Kind.SELECT_CUSTOM; +import static org.apache.cassandra.harry.op.Operations.Kind.SELECT_PARTITION; +import static org.apache.cassandra.harry.op.Operations.Kind.SELECT_RANGE; +import static org.apache.cassandra.harry.op.Operations.Kind.SELECT_ROW; + +/** + * Data tracker tracks every operation that was started and finished. + * + * In principle, it should allow operation to get timed out and remain in the "undecided" state. + * However, there is no implementation as of now that supports this. Right now, every operation + * should be started or should be reliably failed / invisible. + */ +public interface DataTracker +{ + void begin(Visit visit); + void end(Visit visit); + + Iterable<Model.LtsOperationPair> potentialVisits(long pd); + boolean isFinished(long lts); + boolean allFinished(); + + Set<Operations.Kind> OPS_WITHOUT_EFFECT = Set.of(SELECT_CUSTOM, SELECT_PARTITION, SELECT_ROW, SELECT_RANGE, CUSTOM); + + /** + * Data tracker that only allows partition visits to be done _in sequence_ + */ + class SequentialDataTracker implements DataTracker + { + private final AtomicLong started = new AtomicLong(); + private final AtomicLong finished = new AtomicLong(); + + private Map<Long, List<Model.LtsOperationPair>> partitionVisits = new HashMap<>(); + + public void begin(Visit visit) + { + long prev = started.get(); + Invariants.checkState(prev == 0 || visit.lts == (prev + 1)); + started.set(visit.lts); + for (int i = 0; i < visit.operations.length; i++) + { + Operations.Operation operation = visit.operations[i]; + + // SELECT statements have no effect on the model + if (OPS_WITHOUT_EFFECT.contains(operation.kind())) + continue; + + Operations.PartitionOperation partitionOp = (Operations.PartitionOperation) operation; + partitionVisits.computeIfAbsent(partitionOp.pd, pd_ -> new ArrayList<>()) + .add(new Model.LtsOperationPair(visit.lts, i)); + } + } + + public void end(Visit visit) + { + long current = started.get(); + Invariants.checkState(current == visit.lts, + "Current stated %d, current visit: %d", current, visit.lts); + finished.set(visit.lts); + } + + public Iterable<Model.LtsOperationPair> potentialVisits(long pd) + { + Iterable<Model.LtsOperationPair> res = partitionVisits.get(pd); + if (res != null) + return res; + + return Collections.emptyList(); + } + + public boolean isFinished(long lts) + { + return finished.get() >= lts; + } + + @Override + public boolean allFinished() + { + return started.get() == finished.get(); + } + } + + // TODO: optimize for sequential accesses + + /** + * Data tracker able to track LTS out of order + */ + class SimpleDataTracker implements DataTracker + { + private final Set<Long> started = new HashSet<>(); + private final Set<Long> finished = new HashSet<>(); + + private Map<Long, List<Model.LtsOperationPair>> partitionVisits = new HashMap<>(); + + public void begin(Visit visit) + { + started.add(visit.lts); + for (int i = 0; i < visit.operations.length; i++) + { + Operations.Operation operation = visit.operations[i]; + + // SELECT statements have no effect on the model + if (OPS_WITHOUT_EFFECT.contains(operation.kind())) + continue; + + Operations.PartitionOperation partitionOp = (Operations.PartitionOperation) operation; + partitionVisits.computeIfAbsent(partitionOp.pd, pd_ -> new ArrayList<>()) + .add(new Model.LtsOperationPair(visit.lts, i)); + } + } + + public void end(Visit visit) + { + finished.add(visit.lts); + } + + public Iterable<Model.LtsOperationPair> potentialVisits(long pd) + { + Iterable<Model.LtsOperationPair> res = partitionVisits.get(pd); + if (res != null) + return res; + + return Collections.emptyList(); + } + + public boolean isFinished(long lts) + { + return finished.contains(lts); + } + + @Override + public boolean allFinished() + { + return started.size() == finished.size(); + } + } +} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/execution/InJvmDTestVisitExecutor.java b/test/harry/main/org/apache/cassandra/harry/execution/InJvmDTestVisitExecutor.java new file mode 100644 index 000000000000..8362a3111bc6 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/execution/InJvmDTestVisitExecutor.java @@ -0,0 +1,424 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.execution; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.function.Function; + +import com.google.common.base.Throwables; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import accord.utils.Invariants; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.ICluster; +import org.apache.cassandra.exceptions.RequestTimeoutException; +import org.apache.cassandra.harry.ColumnSpec; +import org.apache.cassandra.harry.MagicConstants; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.model.Model; +import org.apache.cassandra.harry.model.QuiescentChecker; +import org.apache.cassandra.harry.op.Operations; +import org.apache.cassandra.harry.op.Visit; +import org.apache.cassandra.utils.AssertionUtils; + +import static org.apache.cassandra.harry.MagicConstants.LTS_UNKNOWN; +import static org.apache.cassandra.harry.MagicConstants.NIL_DESCR; +import static org.apache.cassandra.harry.MagicConstants.NIL_KEY; +import static org.apache.cassandra.harry.MagicConstants.UNKNOWN_DESCR; +import static org.apache.cassandra.harry.MagicConstants.UNSET_DESCR; +import static org.apache.cassandra.harry.execution.QueryBuildingVisitExecutor.*; + +public class InJvmDTestVisitExecutor extends CQLVisitExecutor +{ + private static final Logger logger = LoggerFactory.getLogger(InJvmDTestVisitExecutor.class); + + protected final ICluster<?> cluster; + protected final ConsistencyLevelSelector consistencyLevel; + + protected final NodeSelector nodeSelector; + protected final PageSizeSelector pageSizeSelector; + protected final RetryPolicy retryPolicy; + + protected InJvmDTestVisitExecutor(SchemaSpec schema, + DataTracker dataTracker, + Model model, + ICluster<?> cluster, + + NodeSelector nodeSelector, + PageSizeSelector pageSizeSelector, + RetryPolicy retryPolicy, + ConsistencyLevelSelector consistencyLevel, + WrapQueries wrapQueries) + { + super(schema, dataTracker, model, new QueryBuildingVisitExecutor(schema, wrapQueries)); + this.cluster = cluster; + this.consistencyLevel = consistencyLevel; + + this.nodeSelector = nodeSelector; + this.pageSizeSelector = pageSizeSelector; + this.retryPolicy = retryPolicy; + } + + @Override + protected List<ResultSetRow> executeWithResult(Visit visit, CompiledStatement statement) + { + while (true) + { + try + { + ConsistencyLevel consistencyLevel = this.consistencyLevel.consistencyLevel(visit); + int pageSize = PageSizeSelector.NO_PAGING; + if (consistencyLevel != ConsistencyLevel.NODE_LOCAL) + pageSize = pageSizeSelector.pages(visit); + return executeWithResult(visit, nodeSelector.select(visit.lts), pageSize, statement, consistencyLevel); + } + catch (Throwable t) + { + if (retryPolicy.retry(t)) + continue; + throw t; + } + } + } + + protected List<ResultSetRow> executeWithResult(Visit visit, int node, int pageSize, CompiledStatement statement, ConsistencyLevel consistencyLevel) + { + Invariants.checkState(visit.operations.length == 1); + Object[][] rows; + if (consistencyLevel == ConsistencyLevel.NODE_LOCAL) + rows = cluster.get(node).executeInternal(statement.cql(), statement.bindings()); + else + { + if (pageSize == PageSizeSelector.NO_PAGING) + rows = cluster.coordinator(node).execute(statement.cql(), consistencyLevel, statement.bindings()); + else + rows = iterToArr(cluster.coordinator(node) + .executeWithPaging(statement.cql(), consistencyLevel, pageSize, statement.bindings())); + } + + if (logger.isTraceEnabled()) + logger.trace("{} returned {} results", statement, rows.length); + + return rowsToResultSet(schema, (Operations.SelectStatement) visit.operations[0], rows); + } + + protected static Object[][] iterToArr(Iterator<Object[]> iter) + { + List<Object[]> tmp = new ArrayList<>(); + while (iter.hasNext()) + tmp.add(iter.next()); + return tmp.toArray(new Object[tmp.size()][]); + } + + @Override + protected void executeWithoutResult(Visit visit, CompiledStatement statement) + { + while (true) + { + try + { + ConsistencyLevel consistencyLevel = this.consistencyLevel.consistencyLevel(visit); + executeWithoutResult(visit, nodeSelector.select(visit.lts), statement, consistencyLevel); + return; + } + catch (Throwable t) + { + if (retryPolicy.retry(t)) + continue; + throw t; + } + } + } + + protected void executeWithoutResult(Visit visit, int node, CompiledStatement statement, ConsistencyLevel consistencyLevel) + { + if (consistencyLevel == ConsistencyLevel.NODE_LOCAL) + cluster.get(node).executeInternal(statement.cql(), statement.bindings()); + else + cluster.coordinator(node).execute(statement.cql(), consistencyLevel, statement.bindings()); + } + + public static List<ResultSetRow> rowsToResultSet(SchemaSpec schema, Operations.SelectStatement select, Object[][] result) + { + List<ResultSetRow> rs = new ArrayList<>(); + for (Object[] res : result) + rs.add(rowToResultSet(schema, select, res)); + return rs; + } + + public static ResultSetRow rowToResultSet(SchemaSpec schema, Operations.SelectStatement select, Object[] result) + { + long[] staticColumns = new long[schema.staticColumns.size()]; + long[] regularColumns = new long[schema.regularColumns.size()]; + Arrays.fill(staticColumns, UNKNOWN_DESCR); + Arrays.fill(regularColumns, UNKNOWN_DESCR); + // TODO: with timestamps + long[] staticLts = LTS_UNKNOWN; + long[] regularLts = LTS_UNKNOWN; + + long pd = UNKNOWN_DESCR; + Operations.Selection selection = Operations.Selection.fromBitSet(select.selection(), schema); + if (selection.selectsAllOf(schema.partitionKeys)) + { + Object[] partitionKey = new Object[schema.partitionKeys.size()]; + for (int i = 0; i < schema.partitionKeys.size(); i++) + partitionKey[i] = result[selection.indexOf(schema.partitionKeys.get(i))]; + + pd = schema.valueGenerators.pkGen().deflate(partitionKey); + } + + // Deflate logic for clustering key is a bit more involved, since CK can be nil in case of a single static row. + long cd = UNKNOWN_DESCR; + if (selection.selectsAllOf(schema.clusteringKeys)) + { + Object[] clusteringKey = new Object[schema.clusteringKeys.size()]; + for (int i = 0; i < schema.clusteringKeys.size(); i++) + { + Object v = result[selection.indexOf(schema.clusteringKeys.get(i))]; + if (v == null) + { + for (int j = 0; j < schema.clusteringKeys.size(); j++) + { + Invariants.checkState(result[selection.indexOf(schema.clusteringKeys.get(j))] == null, + "All elements of clustering key should have been null"); + } + clusteringKey = NIL_KEY; + break; + } + clusteringKey[i] = v; + } + + // Clusterings can not be set to nil, so if we do not see, we assume it is unset + if (clusteringKey == NIL_KEY) + cd = UNSET_DESCR; + else + cd = schema.valueGenerators.ckGen().deflate(clusteringKey); + } + + for (int i = 0; i < schema.regularColumns.size(); i++) + { + ColumnSpec<?> column = schema.regularColumns.get(i); + if (selection.selects(column)) + { + Object v = result[selection.indexOf(schema.regularColumns.get(i))]; + if (v == null) + regularColumns[i] = NIL_DESCR; + else + regularColumns[i] = schema.valueGenerators.regularColumnGen(i).deflate(v); + } + else + { + regularLts[i] = UNKNOWN_DESCR; + } + } + + for (int i = 0; i < schema.staticColumns.size(); i++) + { + ColumnSpec<?> column = schema.staticColumns.get(i); + if (selection.selects(column)) + { + Object v = result[selection.indexOf(schema.staticColumns.get(i))]; + if (v == null) + staticColumns[i] = NIL_DESCR; + else + staticColumns[i] = schema.valueGenerators.staticColumnGen(i).deflate(v); + } + else + { + staticColumns[i] = UNKNOWN_DESCR; + } + } + + if (selection.includeTimestamps()) + { + long[] slts = new long[schema.staticColumns.size()]; + Arrays.fill(slts, MagicConstants.NO_TIMESTAMP); + for (int i = 0, sltsBase = schema.allColumnInSelectOrder.size(); i < slts.length && sltsBase + i < result.length; i++) + { + Object v = result[schema.allColumnInSelectOrder.size() + i]; + if (v != null) + slts[i] = (long) v; + } + + long[] lts = new long[schema.regularColumns.size()]; + Arrays.fill(lts, MagicConstants.NO_TIMESTAMP); + for (int i = 0, ltsBase = schema.allColumnInSelectOrder.size() + slts.length; i < lts.length && ltsBase + i < result.length; i++) + { + Object v = result[ltsBase + i]; + if (v != null) + lts[i] = (long) v; + } + } + + return new ResultSetRow(pd, cd, staticColumns, staticLts, regularColumns, regularLts); + } + + public interface NodeSelector + { + int select(long lts); + } + + public interface PageSizeSelector + { + int NO_PAGING = -1; + int pages(Visit visit); + } + + public interface ConsistencyLevelSelector + { + ConsistencyLevel consistencyLevel(Visit visit); + } + + public interface RetryPolicy + { + RetryPolicy RETRY_ON_TIMEOUT = (t) -> { + return t.getMessage().contains("timed out") || + AssertionUtils.isInstanceof(RequestTimeoutException.class) + .matches(Throwables.getRootCause(t)); + }; + boolean retry(Throwable t); + } + + public static Builder builder() + { + return new Builder(); + } + + public static class Builder + { + protected ConsistencyLevelSelector consistencyLevel = v -> ConsistencyLevel.QUORUM; + + protected NodeSelector nodeSelector = null; + protected PageSizeSelector pageSizeSelector = (lts) -> 1; + protected RetryPolicy retryPolicy = RetryPolicy.RETRY_ON_TIMEOUT; + protected WrapQueries wrapQueries = WrapQueries.UNLOGGED_BATCH; + + public Builder wrapQueries(WrapQueries wrapQueries) + { + this.wrapQueries = wrapQueries; + return this; + } + + public Builder consistencyLevel(ConsistencyLevel consistencyLevel) + { + this.consistencyLevel = v -> consistencyLevel; + return this; + } + + public Builder consistencyLevel(ConsistencyLevelSelector consistencyLevel) + { + this.consistencyLevel = consistencyLevel; + return this; + } + + public Builder nodeSelector(NodeSelector nodeSelector) + { + this.nodeSelector = nodeSelector; + return this; + } + + public Builder pageSizeSelector(PageSizeSelector pageSizeSelector) + { + this.pageSizeSelector = pageSizeSelector; + return this; + } + + public Builder retryPolicy(RetryPolicy retryPolicy) + { + this.retryPolicy = retryPolicy; + return this; + } + + protected void setDefaults(SchemaSpec schema, ICluster<?> cluster) + { + if (nodeSelector == null) + { + this.nodeSelector = new NodeSelector() + { + final int nodes = cluster.size(); + long counter = 0; + + @Override + public int select(long lts) + { + return (int) (counter++ % nodes + 1); + } + }; + } + } + + public InJvmDTestVisitExecutor build(SchemaSpec schema, Model.Replay replay, ICluster<?> cluster) + { + setDefaults(schema, cluster); + DataTracker tracker = new DataTracker.SequentialDataTracker(); + Model model = new QuiescentChecker(schema.valueGenerators, tracker, replay); + return new InJvmDTestVisitExecutor(schema, tracker, model, cluster, + nodeSelector, pageSizeSelector, retryPolicy, consistencyLevel, wrapQueries); + } + + public InJvmDTestVisitExecutor build(SchemaSpec schema, ICluster<?> cluster, Function<Builder, InJvmDTestVisitExecutor> overrides) + { + setDefaults(schema, cluster); + return overrides.apply(this); + } + + /** + * WARNING: highly experimental + */ + public InJvmDTestVisitExecutor doubleWriting(SchemaSpec schema, Model.Replay replay, ICluster<?> cluster, String secondTable) + { + setDefaults(schema, cluster); + DataTracker tracker = new DataTracker.SequentialDataTracker(); + Model model = new QuiescentChecker(schema.valueGenerators, tracker, replay); + return new InJvmDTestVisitExecutor(schema, tracker, model, cluster, + nodeSelector, pageSizeSelector, retryPolicy, consistencyLevel, wrapQueries) + { + @Override + protected List<ResultSetRow> executeWithResult(Visit visit, int node, int pageSize, CompiledStatement statement, ConsistencyLevel consistencyLevel) + { + List<ResultSetRow> rows = super.executeWithResult(visit, node, pageSize, statement, consistencyLevel); + List<ResultSetRow> secondOpinion = super.executeWithResult(visit, node, pageSize, statement.withSchema(schema.keyspace, schema.table, + schema.keyspace, secondTable), + consistencyLevel); + if (!rows.equals(secondOpinion)) + { + logger.debug("Second opinion: "); + for (ResultSetRow resultSetRow : secondOpinion) + logger.debug(resultSetRow.toString(schema.valueGenerators)); + } + return rows; + } + + @Override + protected void executeWithoutResult(Visit visit, int node, CompiledStatement statement, ConsistencyLevel consistencyLevel) + { + super.executeWithoutResult(visit, node, statement, consistencyLevel); + super.executeWithoutResult(visit, node, statement.withSchema(schema.keyspace, schema.table, + schema.keyspace, secondTable), + consistencyLevel); + } + }; + } + } + +} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/execution/QueryBuildingVisitExecutor.java b/test/harry/main/org/apache/cassandra/harry/execution/QueryBuildingVisitExecutor.java new file mode 100644 index 000000000000..a330007a2a4d --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/execution/QueryBuildingVisitExecutor.java @@ -0,0 +1,198 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.execution; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import accord.utils.Invariants; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.cql.DeleteHelper; +import org.apache.cassandra.harry.cql.SelectHelper; +import org.apache.cassandra.harry.cql.WriteHelper; +import org.apache.cassandra.harry.op.Operations; +import org.apache.cassandra.harry.op.Visit; + +// TODO: this class can be substantially improved by removing internal mutable state +public class QueryBuildingVisitExecutor extends VisitExecutor +{ + private static final Logger logger = LoggerFactory.getLogger(QueryBuildingVisitExecutor.class); + protected final SchemaSpec schema; + protected final WrapQueries wrapQueries; + + public QueryBuildingVisitExecutor(SchemaSpec schema, WrapQueries wrapQueries) + { + this.schema = schema; + this.wrapQueries = wrapQueries; + } + + public BuiltQuery compile(Visit visit) + { + beginLts(visit.lts); + for (Operations.Operation op : visit.operations) + { + if (logger.isTraceEnabled()) + logger.trace("{} {}", visit.lts, op); + operation(op); + } + + // TODO: try inducing timeouts and checking non-propagation or discovery + endLts(visit.lts); + + if (!statements.isEmpty()) + { + Object[] bindingsArray = new Object[bindings.size()]; + bindings.toArray(bindingsArray); + BuiltQuery query = new BuiltQuery(selects, + wrapQueries.wrap(visit, String.join("\n ", statements)), + bindingsArray); + clear(); + return query; + } + + Invariants.checkState(bindings.isEmpty() && visitedPds.isEmpty() && selects.isEmpty()); + return null; + } + + public static class BuiltQuery extends CompiledStatement + { + protected final List<Operations.SelectStatement> selects; + public BuiltQuery(List<Operations.SelectStatement> selects, String cql, Object... bindings) + { + super(cql, bindings); + this.selects = selects; + } + } + + /** + * Per-LTS state + */ + private final List<String> statements = new ArrayList<>(); + private final List<Object> bindings = new ArrayList<>(); + private final Set<Long> visitedPds = new HashSet<>(); + + private List<Operations.SelectStatement> selects = null; + + protected void beginLts(long lts) + { + statements.clear(); + bindings.clear(); + visitedPds.clear(); + selects = new ArrayList<>(); + } + + protected void endLts(long lts) + { + if (statements.isEmpty()) + { + Invariants.checkState(bindings.isEmpty() && visitedPds.isEmpty() && selects.isEmpty()); + return; + } + + assert visitedPds.size() == 1 : String.format("Token aware only works with a single value per token, but got %s", visitedPds); + } + + private void clear() + { + statements.clear(); + bindings.clear(); + visitedPds.clear(); + selects = null; + } + + protected void operation(Operations.Operation operation) + { + if (operation instanceof Operations.PartitionOperation) + visitedPds.add(((Operations.PartitionOperation) operation).pd()); + CompiledStatement statement; + switch (operation.kind()) + { + case UPDATE: + statement = WriteHelper.inflateUpdate((Operations.WriteOp) operation, schema, operation.lts()); + break; + case INSERT: + statement = WriteHelper.inflateInsert((Operations.WriteOp) operation, schema, operation.lts()); + break; + case DELETE_RANGE: + statement = DeleteHelper.inflateDelete((Operations.DeleteRange) operation, schema, operation.lts()); + break; + case DELETE_PARTITION: + statement = DeleteHelper.inflateDelete((Operations.DeletePartition) operation, schema, operation.lts()); + break; + case DELETE_ROW: + statement = DeleteHelper.inflateDelete((Operations.DeleteRow) operation, schema, operation.lts()); + break; + case DELETE_COLUMNS: + statement = DeleteHelper.inflateDelete((Operations.DeleteColumns) operation, schema, operation.lts()); + break; + case SELECT_PARTITION: + statement = SelectHelper.select((Operations.SelectPartition) operation, schema); + selects.add((Operations.SelectStatement) operation); + break; + case SELECT_ROW: + statement = SelectHelper.select((Operations.SelectRow) operation, schema); + selects.add((Operations.SelectStatement) operation); + break; + case SELECT_RANGE: + statement = SelectHelper.select((Operations.SelectRange) operation, schema); + selects.add((Operations.SelectStatement) operation); + break; + case SELECT_CUSTOM: + statement = SelectHelper.select((Operations.SelectCustom) operation, schema); + selects.add((Operations.SelectStatement) operation); + break; + + case CUSTOM: + ((Operations.CustomRunnableOperation) operation).execute(); + return; + default: + throw new IllegalArgumentException(); + } + statements.add(statement.cql()); + Collections.addAll(bindings, statement.bindings()); + } + + private static final String wrapInUnloggedBatchFormat = "BEGIN UNLOGGED BATCH\n" + + " %s\n" + + "APPLY BATCH;"; + + private static final String wrapInTxnFormat = "BEGIN TRANSACTION\n" + + " %s\n" + + "COMMIT TRANSACTION;"; + + public interface WrapQueries + { + WrapQueries UNLOGGED_BATCH = (visit, compiled) -> { + if (visit.operations.length == 1) + return compiled; + return String.format(wrapInUnloggedBatchFormat, compiled); + }; + + WrapQueries TRANSACTION = (visit, compiled) -> String.format(wrapInTxnFormat, compiled); + + + String wrap(Visit visit, String compiled); + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/execution/ResultSetRow.java b/test/harry/main/org/apache/cassandra/harry/execution/ResultSetRow.java new file mode 100644 index 000000000000..3bc40dbb1e85 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/execution/ResultSetRow.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.execution; + +import org.apache.cassandra.harry.gen.Bijections; +import org.apache.cassandra.harry.gen.ValueGenerators; +import org.apache.cassandra.harry.util.StringUtils; + +import java.util.Arrays; +import java.util.Objects; +import java.util.function.IntFunction; + +public class ResultSetRow +{ + public final long pd; + public final long cd; + public final long[] vds; + public final long[] lts; + + public final long[] sds; + public final long[] slts; + + public ResultSetRow(long pd, + long cd, + long[] sds, + long[] slts, + long[] vds, + long[] lts) + { + this.pd = pd; + this.cd = cd; + this.vds = vds; + this.lts = lts; + this.sds = sds; + this.slts = slts; + } + + public boolean hasStaticColumns() + { + return slts.length > 0; + } + + @Override + public ResultSetRow clone() + { + return new ResultSetRow(pd, cd, + Arrays.copyOf(sds, sds.length), Arrays.copyOf(slts, slts.length), + Arrays.copyOf(vds, vds.length), Arrays.copyOf(lts, lts.length)); + } + + @Override + public int hashCode() + { + int result = Objects.hash(pd, cd); + result = 31 * result + Arrays.hashCode(vds); + result = 31 * result + Arrays.hashCode(lts); + result = 31 * result + Arrays.hashCode(sds); + result = 31 * result + Arrays.hashCode(slts); + return result; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ResultSetRow that = (ResultSetRow) o; + return pd == that.pd && + cd == that.cd && + Arrays.equals(vds, that.vds) && + Arrays.equals(lts, that.lts) && + Arrays.equals(sds, that.sds) && + Arrays.equals(slts, that.slts); + } + + public String toString() + { + return "resultSetRow(" + + pd + + "L, " + cd + + (sds == null ? "" : "L, statics(" + StringUtils.toString(sds) + ")") + + (slts == null ? "" : ", lts(" + StringUtils.toString(slts) + ")") + + ", values(" + StringUtils.toString(vds) + ")" + + ", lts(" + StringUtils.toString(lts) + ")" + + ")"; + } + + private static String toString(long[] descriptors, IntFunction<Bijections.Bijection<Object>> gens) + { + String[] idxs = new String[descriptors.length]; + for (int i = 0; i < descriptors.length; i++) + idxs[i] = descrToIdx(gens.apply(i), descriptors[i]); + return String.join(",", idxs); + } + + private static String descrToIdx(Bijections.Bijection<?> gen, long descr) + { + return gen.toString(descr); + } + + public String toString(ValueGenerators valueGenerators) + { + return "resultSetRow(" + + valueGenerators.pkGen().toString(pd) + + ", " + descrToIdx(valueGenerators.ckGen(), cd) + + (sds == null ? "" : ", statics(" + toString(sds, valueGenerators::staticColumnGen) + ")") + + (slts == null ? "" : ", slts(" + StringUtils.toString(slts) + ")") + + ", values(" + toString(vds, valueGenerators::regularColumnGen) + ")" + + ", lts(" + StringUtils.toString(lts) + ")" + + ")"; + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/execution/RingAwareInJvmDTestVisitExecutor.java b/test/harry/main/org/apache/cassandra/harry/execution/RingAwareInJvmDTestVisitExecutor.java new file mode 100644 index 000000000000..b9a6e0565d56 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/execution/RingAwareInJvmDTestVisitExecutor.java @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.execution; + + +import java.util.ArrayList; +import java.util.List; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import accord.utils.Invariants; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.ICluster; +import org.apache.cassandra.distributed.api.ICoordinator; +import org.apache.cassandra.distributed.api.IInstance; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.op.Visit; +import org.apache.cassandra.harry.op.Operations; +import org.apache.cassandra.harry.model.Model; +import org.apache.cassandra.harry.model.QuiescentChecker; +import org.apache.cassandra.harry.model.TokenPlacementModel; +import org.apache.cassandra.harry.util.ByteUtils; +import org.apache.cassandra.harry.util.TokenUtil; + +/** + * Executes all modifications queries with NODE_LOCAL. Executes all validation queries + * with NODE_LOCAL _on each node_. + */ +public class RingAwareInJvmDTestVisitExecutor extends InJvmDTestVisitExecutor +{ + private static final Logger logger = LoggerFactory.getLogger(InJvmDTestVisitExecutor.class); + + private final TokenPlacementModel.ReplicationFactor rf; + + private RingAwareInJvmDTestVisitExecutor(SchemaSpec schema, + DataTracker dataTracker, + Model model, + ICluster<?> cluster, + NodeSelector nodeSelector, + PageSizeSelector pageSizeSelector, + RetryPolicy retryPolicy, + ConsistencyLevelSelector consistencyLevel, + TokenPlacementModel.ReplicationFactor rf, + QueryBuildingVisitExecutor.WrapQueries wrapQueries) + { + super(schema, dataTracker, model, cluster, nodeSelector, pageSizeSelector, retryPolicy, consistencyLevel, QueryBuildingVisitExecutor.WrapQueries.UNLOGGED_BATCH); + this.rf = rf; + } + + protected TokenPlacementModel.ReplicatedRanges getRing() + { + IInstance node = ((Cluster)cluster).firstAlive(); + ICoordinator coordinator = node.coordinator(); + List<TokenPlacementModel.Node> other = TokenPlacementModel.peerStateToNodes(coordinator.execute("select peer, tokens, data_center, rack from system.peers", ConsistencyLevel.ONE)); + List<TokenPlacementModel.Node> self = TokenPlacementModel.peerStateToNodes(coordinator.execute("select broadcast_address, tokens, data_center, rack from system.local", ConsistencyLevel.ONE)); + List<TokenPlacementModel.Node> all = new ArrayList<>(); + all.addAll(self); + all.addAll(other); + all.sort(TokenPlacementModel.Node::compareTo); + return rf.replicate(all); + } + + public List<TokenPlacementModel.Replica> getReplicasFor(long pd) + { + return getRing().replicasFor(token(pd)); + } + + protected long token(long pd) + { + return TokenUtil.token(ByteUtils.compose(ByteUtils.objectsToBytes(schema.valueGenerators.pkGen().inflate(pd)))); + } + + @Override + protected void executeValidatingVisit(Visit visit, List<Operations.SelectStatement> selects, CompiledStatement statement) + { + try + { + for (TokenPlacementModel.Replica replica : getReplicasFor(selects.get(0).pd)) + { + IInstance instance = cluster + .stream() + .filter((n) -> n.config().broadcastAddress().toString().contains(replica.node().id())) + .findFirst() + .get(); + ConsistencyLevel consistencyLevel = this.consistencyLevel.consistencyLevel(visit); + int pageSize = PageSizeSelector.NO_PAGING; + if (consistencyLevel != ConsistencyLevel.NODE_LOCAL) + pageSize = pageSizeSelector.pages(visit); + List<ResultSetRow> resultSetRow = executeWithResult(visit, instance.config().num(), pageSize, statement, consistencyLevel); + model.validate(selects.get(0), resultSetRow); + } + } + catch (Throwable t) + { + throw new AssertionError(String.format("Caught an exception while validating %s:\n%s", selects.get(0), statement), t); + } + } + + @Override + protected void executeWithoutResult(Visit visit, CompiledStatement statement) + { + try + { + Invariants.checkState(visit.visitedPartitions.size() == 1, + "Ring aware executor can only read and write one partition at a time"); + for (TokenPlacementModel.Replica replica : getReplicasFor(visit.visitedPartitions.iterator().next().longValue())) + { + IInstance instance = cluster + .stream() + .filter((n) -> n.config().broadcastAddress().toString().contains(replica.node().id())) + .findFirst() + .get(); + executeWithoutResult(visit, instance.config().num(), statement, consistencyLevel.consistencyLevel(visit)); + } + } + catch (Throwable t) + { + throw new AssertionError(String.format("Caught an exception while validating %s", statement), t); + } + } + + public static Builder builder() + { + return new Builder(); + } + + public static class Builder extends InJvmDTestVisitExecutor.Builder + { + protected TokenPlacementModel.ReplicationFactor rf; + + public Builder replicationFactor(TokenPlacementModel.ReplicationFactor rf) + { + this.rf = rf; + return this; + } + + @Override + public Builder wrapQueries(QueryBuildingVisitExecutor.WrapQueries wrapQueries) + { + super.wrapQueries(wrapQueries); + return this; + } + + @Override + public Builder consistencyLevel(ConsistencyLevel consistencyLevel) + { + super.consistencyLevel(consistencyLevel); + return this; + } + + @Override + public Builder nodeSelector(NodeSelector nodeSelector) + { + super.nodeSelector(nodeSelector); + return this; + } + + @Override + public Builder pageSizeSelector(PageSizeSelector pageSizeSelector) + { + super.pageSizeSelector(pageSizeSelector); + return this; + } + + @Override + public Builder retryPolicy(RetryPolicy retryPolicy) + { + super.retryPolicy(retryPolicy); + return this; + } + + @Override + protected void setDefaults(SchemaSpec schema, ICluster<?> cluster) + { + super.setDefaults(schema, cluster); + if (this.rf == null) + { + this.rf = new TokenPlacementModel.SimpleReplicationFactor(Math.min(3, cluster.size())); + } + } + + public RingAwareInJvmDTestVisitExecutor build(SchemaSpec schema, Model.Replay replay, ICluster<?> cluster) + { + DataTracker tracker = new DataTracker.SequentialDataTracker(); + Model model = new QuiescentChecker(schema.valueGenerators, tracker, replay); + return build(schema, tracker, model, cluster); + } + + public RingAwareInJvmDTestVisitExecutor build(SchemaSpec schema, DataTracker tracker, Model model, ICluster<?> cluster) + { + setDefaults(schema, cluster); + return new RingAwareInJvmDTestVisitExecutor(schema, tracker, model, cluster, + nodeSelector, pageSizeSelector, retryPolicy, consistencyLevel, rf, wrapQueries); + } + } +} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/dsl/ValueOverrides.java b/test/harry/main/org/apache/cassandra/harry/execution/VisitExecutor.java similarity index 69% rename from test/harry/main/org/apache/cassandra/harry/dsl/ValueOverrides.java rename to test/harry/main/org/apache/cassandra/harry/execution/VisitExecutor.java index df1e61e0f3d7..5d19a29ff708 100644 --- a/test/harry/main/org/apache/cassandra/harry/dsl/ValueOverrides.java +++ b/test/harry/main/org/apache/cassandra/harry/execution/VisitExecutor.java @@ -16,9 +16,14 @@ * limitations under the License. */ -package org.apache.cassandra.harry.dsl; +package org.apache.cassandra.harry.execution; -public interface ValueOverrides +import org.apache.cassandra.harry.op.Operations; + +// Abstract class and not interface becasue methods are protected +public abstract class VisitExecutor { - void override(String column, int idx, Object override); + protected abstract void beginLts(long lts); + protected abstract void endLts(long lts); + protected abstract void operation(Operations.Operation operation); } diff --git a/test/harry/main/org/apache/cassandra/harry/gen/Bijections.java b/test/harry/main/org/apache/cassandra/harry/gen/Bijections.java index 9b8d8ada2658..b41487303b9a 100644 --- a/test/harry/main/org/apache/cassandra/harry/gen/Bijections.java +++ b/test/harry/main/org/apache/cassandra/harry/gen/Bijections.java @@ -18,6 +18,7 @@ package org.apache.cassandra.harry.gen; +import java.util.Comparator; import java.util.Date; import java.util.UUID; @@ -52,7 +53,10 @@ public interface Bijection<T> // TODO: byteSize is great, but you know what's better? Bit size! For example, for `boolean`, we only need a single bit. int byteSize(); - + default int population() + { + return byteSize() * Byte.SIZE; + } /** * Compare as if we were comparing the values in question */ @@ -77,9 +81,19 @@ default boolean unsigned() { return false; } + + default Comparator<Long> descriptorsComparator() + { + return Long::compare; + } + + default String toString(long pd) + { + return Long.toString(pd); + } } - protected static long minForSize(int size) + public static long minForSize(int size) { long min = 1L << (size * Byte.SIZE - 1); @@ -89,7 +103,7 @@ protected static long minForSize(int size) return min; } - protected static long maxForSize(int size) + public static long maxForSize(int size) { long max = Bytes.bytePatternFor(size) >>> 1; @@ -454,4 +468,4 @@ public int byteSize() return Long.BYTES; } } -} \ No newline at end of file +} diff --git a/test/harry/main/org/apache/cassandra/harry/gen/BooleanGenerator.java b/test/harry/main/org/apache/cassandra/harry/gen/BooleanGenerator.java deleted file mode 100644 index 53a1529ac5a5..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/gen/BooleanGenerator.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.gen; - -public class BooleanGenerator implements Generator<Boolean> -{ - public static BooleanGenerator INSTANCE = new BooleanGenerator(); - - public Boolean generate(EntropySource rng) - { - return rng.nextBoolean(); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/gen/Collections.java b/test/harry/main/org/apache/cassandra/harry/gen/Collections.java deleted file mode 100644 index 22ff5b65652a..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/gen/Collections.java +++ /dev/null @@ -1,249 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.gen; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.gen.rng.RngUtils; - -// TODO: collections are currently not deflatable and/or checkable with a model -public class Collections -{ - public static <K, V> ColumnSpec.DataType<Map<K, V>> mapColumn(ColumnSpec.DataType<K> k, - ColumnSpec.DataType<V> v, - int maxSize) - { - return new ColumnSpec.DataType<Map<K, V>>(String.format("map<%s,%s>", k.toString(), v.toString())) - { - private final Bijections.Bijection<Map<K, V>> gen = mapGen(k.generator(), v.generator(), maxSize); - - public Bijections.Bijection<Map<K, V>> generator() - { - return gen; - } - - public int maxSize() - { - return Long.BYTES; - } - }; - } - - public static <V> ColumnSpec.DataType<List<V>> listColumn(ColumnSpec.DataType<V> v, - int maxSize) - { - return new ColumnSpec.DataType<List<V>>(String.format("set<%s>", v.toString())) - { - private final Bijections.Bijection<List<V>> gen = listGen(v.generator(), maxSize); - - public Bijections.Bijection<List<V>> generator() - { - return gen; - } - - public int maxSize() - { - return Long.BYTES; - } - }; - } - - - public static <V> ColumnSpec.DataType<Set<V>> setColumn(ColumnSpec.DataType<V> v, - int maxSize) - { - return new ColumnSpec.DataType<Set<V>>(String.format("set<%s>", v.toString())) - { - private final Bijections.Bijection<Set<V>> gen = setGen(v.generator(), maxSize); - - public Bijections.Bijection<Set<V>> generator() - { - return gen; - } - - public int maxSize() - { - return Long.BYTES; - } - }; - } - - - public static <K, V> Bijections.Bijection<Map<K, V>> mapGen(Bijections.Bijection<K> keyGen, - Bijections.Bijection<V> valueGen, - int maxSize) - { - return new MapGenerator<>(keyGen, valueGen, maxSize); - } - - public static <V> Bijections.Bijection<List<V>> listGen(Bijections.Bijection<V> valueGen, - int maxSize) - { - return new ListGenerator<>(valueGen, maxSize); - } - - public static <V> Bijections.Bijection<Set<V>> setGen(Bijections.Bijection<V> valueGen, - int maxSize) - { - return new SetGenerator<>(valueGen, maxSize); - } - - public static class MapGenerator<K, V> implements Bijections.Bijection<Map<K, V>> - { - public final Bijections.Bijection<K> keyGen; - public final Bijections.Bijection<V> valueGen; - public int maxSize; - - public MapGenerator(Bijections.Bijection<K> keyGen, - Bijections.Bijection<V> valueGen, - int maxSize) - { - this.keyGen = keyGen; - this.valueGen = valueGen; - this.maxSize = maxSize; - } - - public Map<K, V> inflate(long descriptor) - { - long rnd = RngUtils.next(descriptor); - int count = RngUtils.asInt(rnd, 0, maxSize); - Map<K, V> m = new HashMap<>(); - for (int i = 0; i < count; i++) - { - rnd = RngUtils.next(rnd); - K key = keyGen.inflate(rnd); - rnd = RngUtils.next(rnd); - V value = valueGen.inflate(rnd); - m.put(key, value); - } - - return m; - } - - // At least for non-frozen ones - public long deflate(Map<K, V> value) - { - throw new UnsupportedOperationException(); - } - - public int byteSize() - { - return Long.BYTES; - } - - public int compare(long l, long r) - { - throw new UnsupportedOperationException(); - } - } - - public static class ListGenerator<V> implements Bijections.Bijection<List<V>> - { - public final Bijections.Bijection<V> valueGen; - public int maxSize; - - public ListGenerator(Bijections.Bijection<V> valueGen, - int maxSize) - { - this.valueGen = valueGen; - this.maxSize = maxSize; - } - - public List<V> inflate(long descriptor) - { - long rnd = RngUtils.next(descriptor); - int count = RngUtils.asInt(rnd, 0, maxSize); - List<V> m = new ArrayList<>(); - for (int i = 0; i < count; i++) - { - rnd = RngUtils.next(rnd); - V value = valueGen.inflate(rnd); - m.add(value); - } - - return m; - } - - // At least for non-frozen ones - public long deflate(List<V> value) - { - throw new UnsupportedOperationException(); - } - - public int byteSize() - { - return Long.BYTES; - } - - public int compare(long l, long r) - { - throw new UnsupportedOperationException(); - } - } - - public static class SetGenerator<V> implements Bijections.Bijection<Set<V>> - { - public final Bijections.Bijection<V> valueGen; - public int maxSize; - - public SetGenerator(Bijections.Bijection<V> valueGen, - int maxSize) - { - this.valueGen = valueGen; - this.maxSize = maxSize; - } - - public Set<V> inflate(long descriptor) - { - long rnd = RngUtils.next(descriptor); - int count = RngUtils.asInt(rnd, 0, maxSize); - Set<V> m = new HashSet<>(); - for (int i = 0; i < count; i++) - { - rnd = RngUtils.next(rnd); - V value = valueGen.inflate(rnd); - m.add(value); - } - - return m; - } - - // At least for non-frozen ones - public long deflate(Set<V> value) - { - throw new UnsupportedOperationException(); - } - - public int byteSize() - { - return Long.BYTES; - } - - public int compare(long l, long r) - { - throw new UnsupportedOperationException(); - } - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/gen/DataGenerators.java b/test/harry/main/org/apache/cassandra/harry/gen/DataGenerators.java deleted file mode 100644 index 39c0c80bafe7..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/gen/DataGenerators.java +++ /dev/null @@ -1,511 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.gen; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.gen.rng.RngUtils; - -public class DataGenerators -{ - public static final Object UNSET_VALUE = new Object() { - public String toString() - { - return "UNSET"; - } - }; - - // There is still a slim chance that we're going to produce either of these values by chance, but we'll catch this - // during value generation - public static long UNSET_DESCR = Long.MAX_VALUE; - public static long NIL_DESCR = Long.MIN_VALUE; - // Empty value, for the types that support it - public static long EMPTY_VALUE = Long.MIN_VALUE + 1; - - public static Object[] inflateData(List<ColumnSpec<?>> columns, long[] descriptors) - { - // This can be not true depending on how we implement subselections - assert columns.size() == descriptors.length; - Object[] data = new Object[descriptors.length]; - for (int i = 0; i < descriptors.length; i++) - { - ColumnSpec columnSpec = columns.get(i); - if (descriptors[i] == UNSET_DESCR) - data[i] = UNSET_VALUE; - else if (descriptors[i] == NIL_DESCR) - data[i] = null; - else - data[i] = columnSpec.inflate(descriptors[i]); - } - return data; - } - - public static long[] deflateData(List<ColumnSpec<?>> columns, Object[] data) - { - // This can be not true depending on how we implement subselections - assert columns.size() == data.length; - long[] descriptors = new long[data.length]; - for (int i = 0; i < descriptors.length; i++) - { - ColumnSpec columnSpec = columns.get(i); - if (data[i] == null) - descriptors[i] = NIL_DESCR; - else if (data[i] == UNSET_VALUE) - descriptors[i] = UNSET_DESCR; - else - descriptors[i] = columnSpec.deflate(data[i]); - } - return descriptors; - } - - public static int[] requiredBytes(List<ColumnSpec<?>> columns) - { - switch (columns.size()) - { - case 0: - throw new RuntimeException("Can't inflate empty data column set as it is not inversible"); - case 1: - return new int[]{ Math.min(columns.get(0).type.maxSize(), Long.SIZE) }; - default: - class Pair - { - final int idx, maxSize; - - Pair(int idx, int maxSize) - { - this.idx = idx; - this.maxSize = maxSize; - } - } - int[] bytes = new int[Math.min(KeyGenerator.MAX_UNIQUE_PREFIX_COLUMNS, columns.size())]; - Pair[] sorted = new Pair[bytes.length]; - for (int i = 0; i < sorted.length; i++) - sorted[i] = new Pair(i, columns.get(i).type.maxSize()); - - int remainingBytes = Long.BYTES; - int slotSize = remainingBytes / bytes.length; - // first pass: give it at most a slot number of bytes - for (int i = 0; i < sorted.length; i++) - { - int size = sorted[i].maxSize; - int allotedSize = Math.min(size, slotSize); - remainingBytes -= allotedSize; - bytes[sorted[i].idx] = allotedSize; - } - - // sliced evenly - if (remainingBytes == 0) - return bytes; - - // second pass: try to occupy remaining bytes - // it is possible to improve the second pass and separate additional bytes evenly, but it is - // questionable how much it'll bring since it does not change the total amount of entropy. - for (int i = 0; i < sorted.length; i++) - { - if (remainingBytes == 0) - break; - Pair p = sorted[i]; - if (bytes[p.idx] < p.maxSize) - { - int allotedSize = Math.min(p.maxSize - bytes[p.idx], remainingBytes); - remainingBytes -= allotedSize; - bytes[p.idx] += allotedSize; - } - } - - return bytes; - } - } - - public static Object[] inflateKey(List<ColumnSpec<?>> columns, long descriptor, long[] slices) - { - assert columns.size() >= slices.length : String.format("Columns: %s. Slices: %s", columns, Arrays.toString(slices)); - assert columns.size() > 0 : "Can't deflate from empty columnset"; - - Object[] res = new Object[columns.size()]; - for (int i = 0; i < slices.length; i++) - { - ColumnSpec spec = columns.get(i); - res[i] = spec.inflate(slices[i]); - } - - // The rest can be random, since prefix is always fixed - long current = descriptor; - for (int i = slices.length; i < columns.size(); i++) - { - current = RngUtils.next(current); - res[i] = columns.get(i).inflate(current); - } - - return res; - } - - public static long[] deflateKey(List<ColumnSpec<?>> columns, Object[] values) - { - assert columns.size() == values.length : String.format("%s != %s", columns.size(), values.length); - assert columns.size() > 0 : "Can't deflate from empty columnset"; - - int fixedPart = Math.min(KeyGenerator.MAX_UNIQUE_PREFIX_COLUMNS, columns.size()); - - long[] slices = new long[fixedPart]; - boolean allNulls = true; - for (int i = 0; i < fixedPart; i++) - { - ColumnSpec spec = columns.get(i); - Object value = values[i]; - if (value != null) - allNulls = false; - - slices[i] = value == null ? NIL_DESCR : spec.deflate(value); - } - - if (allNulls) - return null; - - return slices; - } - - public static KeyGenerator createKeyGenerator(List<ColumnSpec<?>> columns) - { - switch (columns.size()) - { - case 0: - return EMPTY_KEY_GEN; - case 1: - return new SinglePartKeyGenerator(columns); - default: - return new MultiPartKeyGenerator(columns); - } - } - - private static final KeyGenerator EMPTY_KEY_GEN = new KeyGenerator(Collections.emptyList()) - { - private final long[] EMPTY_SLICED = new long[0]; - private final Object[] EMPTY_INFLATED = new Object[0]; - - public long[] slice(long descriptor) - { - return EMPTY_SLICED; - } - - public long stitch(long[] parts) - { - return 0; - } - - public long minValue(int idx) - { - return 0; - } - - public long maxValue(int idx) - { - return 0; - } - - @Override - public Object[] inflate(long descriptor) - { - return EMPTY_INFLATED; - } - - @Override - public long deflate(Object[] value) - { - return 0; - } - - public long adjustEntropyDomain(long descriptor) - { - return 0; - } - - public int byteSize() - { - return 0; - } - - public int compare(long l, long r) - { - return 0; - } - }; - - public static abstract class KeyGenerator implements Bijections.Bijection<Object[]> - { - // Maximum number of columns that uniquely identify the value (i.e. use entropy bits). - // Subsequent columns will have random data in them. - public static final int MAX_UNIQUE_PREFIX_COLUMNS = 4; - @VisibleForTesting - public final List<ColumnSpec<?>> columns; - - protected KeyGenerator(List<ColumnSpec<?>> columns) - { - this.columns = columns; - } - - public abstract long[] slice(long descriptor); - - public abstract long stitch(long[] parts); - - public long minValue() - { - return Bijections.minForSize(byteSize()); - } - - public long maxValue() - { - return Bijections.maxForSize(byteSize()); - } - - /** - * Min value for a segment: 0, possibly with an inverted 0 sign for stitching. - */ - public abstract long minValue(int idx); - public abstract long maxValue(int idx); - } - - public static class SinglePartKeyGenerator extends KeyGenerator - { - private final Bijections.Bijection keyGen; - private final int totalSize; - - public SinglePartKeyGenerator(List<ColumnSpec<?>> columns) - { - super(columns); - assert columns.size() == 1; - this.keyGen = columns.get(0).generator(); - this.totalSize = keyGen.byteSize(); - } - - public long[] slice(long descriptor) - { - if (shouldInvertSign()) - descriptor ^= Bytes.signMaskFor(byteSize()); - - descriptor = adjustEntropyDomain(descriptor); - return new long[]{ descriptor }; - } - - public long stitch(long[] parts) - { - long descriptor = parts[0]; - - if (shouldInvertSign()) - descriptor ^= Bytes.signMaskFor(byteSize()); - - return adjustEntropyDomain(descriptor); - } - - public long minValue(int idx) - { - assert idx == 0; - return keyGen.minValue(); - } - - public long maxValue(int idx) - { - assert idx == 0; - return keyGen.maxValue(); - } - - public Object[] inflate(long descriptor) - { - long[] sliced = slice(descriptor); - return new Object[]{ keyGen.inflate(sliced[0]) }; - } - - public boolean shouldInvertSign() - { - return totalSize != Long.BYTES && !keyGen.unsigned(); - } - - public long deflate(Object[] value) - { - Object v = value[0]; - if (v == null) - return NIL_DESCR; - long descriptor = keyGen.deflate(v); - return stitch(new long[] { descriptor }); - } - - public int byteSize() - { - return totalSize; - } - - public int compare(long l, long r) - { - return Long.compare(l, r); - } - } - - public static class MultiPartKeyGenerator extends KeyGenerator - { - @VisibleForTesting - public final int[] sizes; - protected final int totalSize; - - public MultiPartKeyGenerator(List<ColumnSpec<?>> columns) - { - super(columns); - assert columns.size() > 1 : "It makes sense to use a multipart generator if you have more than one column, but you have " + columns.size(); - - this.sizes = requiredBytes(columns); - int total = 0; - for (int size : sizes) - total += size; - - this.totalSize = total; - } - - public long deflate(Object[] values) - { - long[] stiched = DataGenerators.deflateKey(columns, values); - if (stiched == null) - return NIL_DESCR; - return stitch(stiched); - } - - public Object[] inflate(long descriptor) - { - return DataGenerators.inflateKey(columns, descriptor, slice(descriptor)); - } - - // Checks whether we need to invert a slice sign to preserve order of the sliced descriptor - public boolean shouldInvertSign(int idx) - { - Bijections.Bijection<?> gen = columns.get(idx).generator(); - - int maxSliceSize = gen.byteSize(); - int actualSliceSize = sizes[idx]; - - if (idx == 0) - { - // We consume a sign of a descriptor (long, long), (int, int), etc. - if (totalSize == Long.BYTES) - { - // If we use only 3 bytes for a 4-byte int, or 4 bytes for a 8-byte int, - // they're effectively unsigned/byte-ordered, so their order won't match - if (maxSliceSize > actualSliceSize) - return true; - // Sign of the current descriptor should match the sign of the slice. - // For example, (tinyint, double) or (double, tinyint). In the first case (tinyint first), - // sign of the first component is going to match the sign of the descriptor. - // In the second case (double first), double is 7-bit, but its most significant bit - // does not hold a sign, so we have to invert it to match sign of the descriptor. - else - return gen.unsigned(); - } - // We do not consume a sign of a descriptor (float, tinyint), (int, tinyint), etc, - // so we have to only invert signs of the values, since their order doesn't match. - else - { - assert maxSliceSize == actualSliceSize; - return !gen.unsigned(); - } - } - else if (gen.unsigned()) - return false; - else - // We invert sign of all subsequent chunks if they have enough entropy to have a sign bit set - return maxSliceSize == actualSliceSize; - } - - public long[] slice(long descriptor) - { - long[] pieces = new long[columns.size()]; - long pos = totalSize; - for (int i = 0; i < sizes.length; i++) - { - final int size = sizes[i]; - long piece = descriptor >> ((pos - size) * Byte.SIZE); - - if (shouldInvertSign(i)) - piece ^= Bytes.signMaskFor(size); - - piece &= Bytes.bytePatternFor(size); - - pieces[i] = piece; - pos -= size; - } - - // The rest can be random, since prefix is always fixed - long current = descriptor; - for (int i = sizes.length; i < columns.size(); i++) - { - current = RngUtils.next(current); - pieces[i] = columns.get(i).generator().adjustEntropyDomain(current); - } - - return pieces; - } - - public long stitch(long[] parts) - { - long stitched = 0; - int consumed = 0; - for (int i = sizes.length - 1; i >= 0; i--) - { - int size = sizes[i]; - long piece = parts[i]; - - if (shouldInvertSign(i)) - piece ^= Bytes.signMaskFor(size); - - piece &= Bytes.bytePatternFor(size); - stitched |= piece << (consumed * Byte.SIZE); - consumed += size; - } - return stitched; - } - - public long minValue(int idx) - { - long res = columns.get(idx).generator().minValue(); - // Inverting sign is important for range queries and RTs, since we're - // making boundaries that'll be stitched later. - if (shouldInvertSign(idx)) - res ^= Bytes.signMaskFor(sizes[idx]); - return res; - } - - public long maxValue(int idx) - { - long res = columns.get(idx).generator().maxValue(); - if (shouldInvertSign(idx)) - res ^= Bytes.signMaskFor(sizes[idx]); - return res; - } - - public int byteSize() - { - return totalSize; - } - - public int compare(long l, long r) - { - return Long.compare(l, r); - } - } -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/gen/EntropySource.java b/test/harry/main/org/apache/cassandra/harry/gen/EntropySource.java index d6087893bc62..0f9308516f68 100644 --- a/test/harry/main/org/apache/cassandra/harry/gen/EntropySource.java +++ b/test/harry/main/org/apache/cassandra/harry/gen/EntropySource.java @@ -42,6 +42,7 @@ public interface EntropySource int nextInt(int max); int nextInt(int min, int max); float nextFloat(); + double nextDouble(); /** * Code is adopted from a similar method in JDK 17, and has to be removed as soon as we migrate to JDK 17. diff --git a/test/harry/main/org/apache/cassandra/harry/gen/Generators.java b/test/harry/main/org/apache/cassandra/harry/gen/Generators.java index ed966a96ed6f..ca2e5f274982 100644 --- a/test/harry/main/org/apache/cassandra/harry/gen/Generators.java +++ b/test/harry/main/org/apache/cassandra/harry/gen/Generators.java @@ -18,19 +18,316 @@ package org.apache.cassandra.harry.gen; +import java.math.BigDecimal; +import java.math.BigInteger; import java.net.InetAddress; import java.net.UnknownHostException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.UUID; import java.util.function.Supplier; +import accord.utils.Invariants; +import org.apache.cassandra.harry.util.BitSet; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.utils.TimeUUID; public class Generators { + public static Generator<BitSet> bitSet(int size) + { + return rng -> { + BitSet bitSet = BitSet.allUnset(size); + for (int i = 0; i < size; i++) + if (rng.nextBoolean()) + bitSet.set(i); + return bitSet; + }; + } + + public static Generator<String> ascii(int minLength, int maxLength) + { + return new StringGenerator(minLength, maxLength, 0, 127); + } + + public static Generator<String> utf8(int minLength, int maxLength) + { + return rng -> { + int length = rng.nextInt(minLength, maxLength); + int[] codePoints = new int[length]; + for (int i = 0; i < length; i++) + { + int next; + // Exclude surrogate range, generate values before and after it + if (rng.nextBoolean()) + next = rng.nextInt(0x0000, 0xD800); + else + next = rng.nextInt(0xD801, 0xDFFF); + codePoints[i] = next; + } + + return new String(codePoints, 0, codePoints.length); + }; + } + + public static Generator<String> englishAlphabet(int minLength, int maxLength) + { + return new StringGenerator(minLength, maxLength, 97, 122); + } + + public static Generator<Byte> int8() + { + return rng -> (byte) rng.nextInt(); + } + + public static Generator<Short> int16() + { + return rng -> (short) rng.nextInt(); + } + + public static Generator<Integer> int32() + { + return EntropySource::nextInt; + } + + public static Generator<Integer> int32(int min, int max) + { + return rng -> rng.nextInt(min, max); + } + + public static Generator<Long> int64() + { + return new LongGenerator(); + } + + public static Generator<Long> int64(long min, long max) + { + return rng -> rng.nextLong(min, max); + } + + public static Generator<Boolean> bool() + { + return EntropySource::nextBoolean; + } + + public static Generator<Double> doubles() + { + return EntropySource::nextDouble; + } + + public static Generator<Float> floats() + { + return EntropySource::nextFloat; + } + + public static Generator<InetAddress> inetAddr() + { + return new InetAddressGenerator(); + } + + public static <T> Generator<T> inetAddr(Generator<T> delegate) + { + return new UniqueGenerator<>(delegate, 10); + } + + public static Generator<ByteBuffer> bytes(int minSize, int maxSize) + { + return rng -> { + int size = rng.nextInt(minSize, maxSize); + byte[] bytes = new byte[size]; + for (int i = 0; i < size; ) + for (long v = rng.next(), + n = Math.min(size - i, Long.SIZE / Byte.SIZE); + n-- > 0; v >>= Byte.SIZE) + bytes[i++] = (byte) v; + return ByteBuffer.wrap(bytes); + }; + } + + public static Generator<UUID> uuidGen() + { + return rng -> { + long msb = rng.next(); + // Adopted from JDK code, UUID#randomUUID + // randomBytes[6] &= 0x0f; /* clear version */ + msb &= ~(0xFL << 12); + // randomBytes[6] |= 0x40; /* set to version 4 */ + msb |= (0x40L << 8); + long lsb = rng.next(); + // randomBytes[8] &= 0x3f; /* clear variant */ + lsb &= ~(0x3L << 62); + // randomBytes[8] |= 0x80; /* set to IETF variant */ + lsb |= (0x2L << 62); + return new UUID(msb, lsb); + }; + } + + public static Generator<BigInteger> bigInt() + { + return rng -> BigInteger.valueOf(rng.next()); + } + + public static Generator<BigDecimal> bigDecimal() + { + return rng -> BigDecimal.valueOf(rng.next()); + } + + public static Generator<TimeUUID> timeuuid() + { + return new Generator<TimeUUID>() + { + public TimeUUID generate(EntropySource rng) + { + return TimeUUID.atUnixMicrosWithLsb(rng.nextLong(0, Long.MAX_VALUE), + makeClockSeqAndNode(rng)); + } + + private long makeClockSeqAndNode(EntropySource rng) + { + long clock = rng.nextInt(); + + long lsb = 0; + lsb |= 0x8000000000000000L; // variant (2 bits) + lsb |= (clock & 0x0000000000003FFFL) << 48; // clock sequence (14 bits) + lsb |= makeNode(rng); // 6 bytes + return lsb; + } + + private long makeNode(EntropySource rng) + { + // ideally, we'd use the MAC address, but java doesn't expose that. + long v = rng.next(); + byte[] hash = new byte[] { (byte) (0xff & v), + (byte) (0xff & (v << 8)), + (byte) (0xff & (v << 16)), + (byte) (0xff & (v << 24)), + (byte) (0xff & (v << 32)), + (byte) (0xff & (v << 40)) + }; + long node = 0; + for (int i = 0; i < 6; i++) + node |= (0x00000000000000ff & (long)hash[i]) << (5-i)*8; + assert (0xff00000000000000L & node) == 0; + + return node | 0x0000010000000000L; + } + }; + } + + public static <T> TrackingGenerator<T> tracking(Generator<T> delegate) + { + return new TrackingGenerator<>(delegate); + } + + public static class TrackingGenerator<T> implements Generator<T> + { + private final Set<T> generated; + private final Generator<T> delegate; + public TrackingGenerator(Generator<T> delegate) + { + this.generated = new HashSet<>(); + this.delegate = delegate; + } + + public Iterable<T> generated() + { + return generated; + } + + @Override + public T generate(EntropySource rng) + { + T next = delegate.generate(rng); + generated.add(next); + return next; + } + } + + public static <T> Generator<T> unique(Generator<T> delegate) + { + return new UniqueGenerator<>(delegate, 100); + } + + + /** + * WARNING: uses hash code as a proxy for equality + */ + public static class UniqueGenerator<T> implements Generator<T> + { + private final Set<Integer> hashCodes = new HashSet<>(); + private final Generator<T> delegate; + private final int maxSteps; + + public UniqueGenerator(Generator<T> delegate, int maxSteps) + { + this.delegate = delegate; + this.maxSteps = maxSteps; + } + + /** + * + */ + public void clear() + { + hashCodes.clear(); + } + + public T generate(EntropySource rng) + { + for (int i = 0; i < maxSteps; i++) + { + T v = delegate.generate(rng); + int hashCode = v.hashCode(); + Invariants.checkState(hashCode != System.identityHashCode(v), "hashCode was not overridden for type %s", v.getClass()); + if (hashCodes.contains(hashCode)) + continue; + hashCodes.add(hashCode); + return v; + } + + throw new IllegalStateException(String.format("Could not generate a unique value within %d from %s", maxSteps, delegate)); + } + } + + public static final class StringGenerator implements Generator<String> + { + private final int minLength; + private final int maxLength; + private final int minChar; + private final int maxChar; + + public StringGenerator(int minLength, int maxLength, int minChar, int maxChar) + { + this.minLength = minLength; + this.maxLength = maxLength; + this.minChar = minChar; + this.maxChar = maxChar; + } + + @Override + public String generate(EntropySource rng) + { + int length = rng.nextInt(minLength, maxLength); + int[] codePoints = new int[length]; + for (int i = 0; i < length; i++) + codePoints[i] = rng.nextInt(minChar, maxChar); + return new String(codePoints, 0, codePoints.length); + } + } + + public static final class LongGenerator implements Generator<Long> + { + @Override + public Long generate(EntropySource rng) + { + return rng.next(); + } + } + public static class InetAddrAndPortGenerator implements Generator<InetAddressAndPort> { private final int port; @@ -49,9 +346,9 @@ public InetAddressAndPort generate(EntropySource rng) { int orig = rng.nextInt(); byte[] bytes = new byte[]{ (byte) (orig & 0xff), - (byte) (orig << 8 & 0xff), - (byte) (orig << 16 & 0xff), - (byte) (orig << 24 & 0xff) }; + (byte) ((orig >> 8) & 0xff), + (byte) ((orig >> 16) & 0xff), + (byte) ((orig >> 24) & 0xff) }; try { return InetAddressAndPort.getByAddressOverrideDefaults(InetAddress.getByAddress(bytes), bytes, port); @@ -63,6 +360,27 @@ public InetAddressAndPort generate(EntropySource rng) } } + public static class InetAddressGenerator implements Generator<InetAddress> + { + @Override + public InetAddress generate(EntropySource rng) + { + int orig = rng.nextInt(); + byte[] bytes = new byte[]{ (byte) (orig & 0xff), + (byte) ((orig >> 8) & 0xff), + (byte) ((orig >> 16) & 0xff), + (byte) ((orig >> 24) & 0xff) }; + try + { + return InetAddress.getByAddress(bytes); + } + catch (UnknownHostException e) + { + throw new RuntimeException(e); + } + } + } + public static <T> Generator<T> pick(List<T> ts) { if (ts.isEmpty()) @@ -75,6 +393,29 @@ public static <T> Generator<T> pick(T... ts) return pick(Arrays.asList(ts)); } + public static <T> Generator<List<T>> list(int minSize, int maxSize, Generator<T> gen) + { + return rng -> { + List<T> objects = new ArrayList<>(); + int size = rng.nextInt(minSize, maxSize); + for (int i = 0; i < size; i++) + objects.add(gen.generate(rng)); + + return objects; + }; + } + + public static Generator<Object[]> zipArray(Generator<?>... gens) + { + return rng -> { + Object[] objects = new Object[gens.length]; + for (int i = 0; i < objects.length; i++) + objects[i] = gens[i].generate(rng); + + return objects; + }; + } + public static <T> Generator<List<T>> subsetGenerator(List<T> list) { return subsetGenerator(list, 0, list.size() - 1); diff --git a/test/harry/main/org/apache/cassandra/harry/gen/IndexGenerators.java b/test/harry/main/org/apache/cassandra/harry/gen/IndexGenerators.java new file mode 100644 index 000000000000..9aeadf357392 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/gen/IndexGenerators.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.gen; + +import org.apache.cassandra.harry.MagicConstants; + +public class IndexGenerators +{ + private final ValueGenerators valueGenerators; + public final Generator<Integer> pkIdxGen; + public final Generator<Integer> ckIdxGen; + public final Generator<Integer>[] regularIdxGens; + public final Generator<Integer>[] staticIdxGens; + + public static IndexGenerators withDefaults(ValueGenerators valueGenerators) + { + Generator<Integer>[] regularIdxGens = new Generator[valueGenerators.regularColumnGens.size()]; + Generator<Integer>[] staticIdxGens = new Generator[valueGenerators.staticColumnGens.size()]; + + for (int i = 0; i < regularIdxGens.length; i++) + { + int column = i; + regularIdxGens[i] = (rng) -> rng.nextInt(valueGenerators.regularPopulation(column)); + } + + for (int i = 0; i < staticIdxGens.length; i++) + { + int column = i; + staticIdxGens[i] = (rng) -> rng.nextInt(valueGenerators.staticPopulation(column)); + } + + return new IndexGenerators(valueGenerators, + // TODO: distribution for visits + Generators.int32(0, valueGenerators.pkPopulation()), + Generators.int32(0, valueGenerators.ckPopulation()), + regularIdxGens, + staticIdxGens); + } + + public IndexGenerators(ValueGenerators valueGenerators, + Generator<Integer> pkIdxGen, + Generator<Integer> ckIdxGen, + Generator<Integer>[] regularIdxGens, + Generator<Integer>[] staticIdxGens) + { + this.valueGenerators = valueGenerators; + this.pkIdxGen = pkIdxGen; + this.ckIdxGen = ckIdxGen; + this.regularIdxGens = regularIdxGens; + this.staticIdxGens = staticIdxGens; + + } + + public IndexGenerators roundRobinPk() + { + Generator<Integer> pkIdxGen = new Generator<Integer>() + { + int offset = 0; + @Override + public Integer generate(EntropySource rng) + { + int next = offset++; + + if (offset < 0) + offset = 0; + + return next % valueGenerators.pkPopulation(); + } + }; + + return new IndexGenerators(valueGenerators, + pkIdxGen, + ckIdxGen, + regularIdxGens, + staticIdxGens); + } + + public IndexGenerators trackPk() + { + if (pkIdxGen instanceof Generators.TrackingGenerator<?>) + return this; + + return new IndexGenerators(valueGenerators, + Generators.tracking(pkIdxGen), + ckIdxGen, + regularIdxGens, + staticIdxGens); + } + + public IndexGenerators withChanceOfUnset(double chanceOfUnset) + { + Generator<Integer>[] regularIdxGens = new Generator[valueGenerators.regularColumnGens.size()]; + Generator<Integer>[] staticIdxGens = new Generator[valueGenerators.staticColumnGens.size()]; + + for (int i = 0; i < regularIdxGens.length; i++) + { + int column = i; + regularIdxGens[i] = (rng) -> rng.nextDouble() <= chanceOfUnset ? MagicConstants.UNSET_IDX : rng.nextInt(valueGenerators.regularPopulation(column)); + } + + for (int i = 0; i < staticIdxGens.length; i++) + { + int column = i; + staticIdxGens[i] = (rng) -> rng.nextDouble() <= chanceOfUnset ? MagicConstants.UNSET_IDX : rng.nextInt(valueGenerators.staticPopulation(column)); + } + + return new IndexGenerators(valueGenerators, + // TODO: distribution for visits + Generators.int32(0, valueGenerators.pkPopulation()), + Generators.int32(0, valueGenerators.ckPopulation()), + regularIdxGens, + staticIdxGens); + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/gen/InvertibleGenerator.java b/test/harry/main/org/apache/cassandra/harry/gen/InvertibleGenerator.java new file mode 100644 index 000000000000..8c023e5c8cbd --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/gen/InvertibleGenerator.java @@ -0,0 +1,261 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.gen; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import accord.utils.Invariants; +import org.agrona.collections.IntHashSet; +import org.apache.cassandra.harry.ColumnSpec; +import org.apache.cassandra.harry.MagicConstants; +import org.apache.cassandra.harry.dsl.HistoryBuilder; +import org.apache.cassandra.harry.gen.rng.SeedableEntropySource; +import org.apache.cassandra.utils.ArrayUtils; + +/** + * Invertible generator allows you to provide _any_ data type. Harry is based on the idea that descriptors + * can be inflated into values, and values can be turned back into descriptors. Descriptors follow the sorting + * order of the values they were generated from. This makes _writing_ these generators a bit more complex. + * There is a library of lightweight generators available for simple cases. + * + * InvertibleGenerator decouples descriptor order from value order, and allows descriptor to be used simply as + * a seed for generating values. Since it tracks all descriptors it generated values from in a sorted order, + * it can always turn the given value back into a descriptor by inflating log(population) values and comparing them + * to the searched value. In other words, it trades memory required for storing map of values to CPU required + * to re-compute the value order. + * + * TODO (expected): custom invertible generator for bool, u8, u16, u32, etc, for efficiency. + * TODO (expected): implement support for tuple/vector/udt, and other multi-cell types. + */ +public class InvertibleGenerator<T> implements HistoryBuilder.IndexedBijection<T> +{ + public static long MAX_ENTROPY = 1L << 63; + + private static final boolean PARANOIA = true; + + // TODO (required): switch to use a primitive array; will need to implement a sort comparator for primitive types + private final List<Long> allocatedDescriptors; + + private final Generator<T> gen; + private final Comparator<T> comparator; + + // To avoid <?> erased types + public static <T> InvertibleGenerator<T> fromType(EntropySource rng, int population, ColumnSpec<T> spec) + { + return new InvertibleGenerator<>(rng, spec.type.typeEntropy(), population, spec.gen, spec.type.comparator()); + } + + public InvertibleGenerator(EntropySource rng, + /* unsigned */ long typeEntropy, + int population, + Generator<T> gen, + Comparator<T> comparator) + { + Invariants.checkState(population > 0, + "Population should be strictly positive %d", population); + Invariants.checkState(Long.compareUnsigned(typeEntropy, 0) > 0, + "Type entropy should be strictly positive, but was %d: %s", typeEntropy, gen); + + // We can / will generate at most that many values + if (Long.compareUnsigned(typeEntropy, Integer.MAX_VALUE) > 0) + typeEntropy = Integer.MAX_VALUE; + + population = (int) Math.min(typeEntropy, population); + + this.gen = gen; + this.comparator = comparator; + this.allocatedDescriptors = new ArrayList<>(); + + // Generate a population of _unique_ values. We do not want to store all values, only their hashes. + IntHashSet hashes = new IntHashSet(population); + while (allocatedDescriptors.size() < population) + { + long candidate = rng.next(); + + // Should never allocate these, however improbable that is + if (MagicConstants.MAGIC_DESCRIPTOR_VALS.contains(candidate)) + continue; + + Object inflated = inflate(candidate); + int hash = ArrayUtils.hashCode(inflated); + Invariants.checkState(hash != System.identityHashCode(inflated), "hashCode was not overridden for type %s", inflated.getClass()); + + if (hashes.add(hash)) + allocatedDescriptors.add(candidate); + } + hashes.clear(); + + allocatedDescriptors.sort(this::compare); + + // Check there are no duplicates, and items are properly sorted. + if (PARANOIA) + { + T prev = inflate(allocatedDescriptors.get(0)); + for (int i = 1; i < allocatedDescriptors.size(); i++) + { + T current = inflate(allocatedDescriptors.get(i)); + Invariants.checkState( comparator.compare(current, prev) > 0, + () -> String.format("%s should be strictly after %s", prev, current)); + } + } + } + + @Override + public int idxFor(long descriptor) + { + return Collections.binarySearch(allocatedDescriptors, descriptor, this.descriptorsComparator()); + } + + @Override + public long descriptorAt(int idx) + { + return allocatedDescriptors.get(idx); + } + + @Override + public T inflate(long descriptor) + { + Invariants.checkState(!MagicConstants.MAGIC_DESCRIPTOR_VALS.contains(descriptor), + String.format("Should not be able to inflate %d, as it's magic value", descriptor)); + return SeedableEntropySource.computeWithSeed(descriptor, gen::generate); + } + + @Override + public long deflate(T value) + { + final int idx = binarySearch(value); + if (PARANOIA) + { + if (idx < 0) + { + for (long descriptor : allocatedDescriptors) + { + Object expected = inflate(descriptor); + if (value.getClass().isArray()) + { + Object[] valueArr = (Object[]) value; + Object[] expectedArr = (Object[]) expected; + Invariants.checkState(comparator.compare((T) expected, value) != 0, + "%s was found: %s", Arrays.toString(expectedArr), Arrays.toString(valueArr)); + + } + else + { + Invariants.checkState(comparator.compare((T) expected, value) != 0, + "%s was found: %s", expected, value); + } + + } + } + else + { + long res = allocatedDescriptors.get(idx); + Object expected = inflate(res); + if (value.getClass().isArray()) + { + Object[] valueArr = (Object[]) value; + Object[] expectedArr = (Object[]) expected; + + Invariants.checkState(comparator.compare((T) expected, value) == 0, + "%s != %s", Arrays.toString(expectedArr), Arrays.toString(valueArr)); + + } + else + { + Invariants.checkState(comparator.compare((T) expected, value) == 0, + "%s != %s", expected, value); + } + + return res; + } + } + + if (idx < 0) + { + int start = Math.max(0, idx - 2); + List<Object> nearby = new ArrayList<>(); + for (int i = start; i < start + 2; i++) + nearby.add(inflate(allocatedDescriptors.get(i))); + throw new IllegalStateException(String.format("Could not find: %s\nNearby objects: %s", + ArrayUtils.toString(value), nearby.stream().map(ArrayUtils::toString).collect(Collectors.toList()))); + } + + return allocatedDescriptors.get(idx); + } + + + @Override + public int byteSize() + { + return Long.BYTES; + } + + private int binarySearch(T key) + { + int low = 0, mid = allocatedDescriptors.size(), high = mid - 1, result = -1; + while (low <= high) + { + mid = (low + high) >>> 1; + result = comparator.compare(key, inflate(allocatedDescriptors.get(mid))); + if (result > 0) + low = mid + 1; + else if (result == 0) + return mid; + else + high = mid - 1; + } + return -mid - (result < 0 ? 1 : 2); + } + + + @Override + public int compare(long d1, long d2) + { + if (d1 == d2) + return 0; + T v1 = inflate(d1); + T v2 = inflate(d2); + return comparator.compare(v1, v2); + } + + /** + * Returns a number of allocated descriptors + */ + @Override + public int population() + { + return allocatedDescriptors.size(); + } + + public Comparator<Long> descriptorsComparator() + { + // TODO: this can be cached + Map<Long, Integer> descriptorToIdx = new HashMap<>(); + for (int i = 0; i < allocatedDescriptors.size(); i++) + descriptorToIdx.put(allocatedDescriptors.get(i), i); + return Comparator.comparingInt(descriptorToIdx::get); + } +} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/gen/OperationsGenerators.java b/test/harry/main/org/apache/cassandra/harry/gen/OperationsGenerators.java new file mode 100644 index 000000000000..33b831bfb6f5 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/gen/OperationsGenerators.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.gen; + +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.dsl.HistoryBuilder; +import org.apache.cassandra.harry.op.Operations; + +public class OperationsGenerators +{ + public static Generator<Long> lts() + { + return new Generator<>() + { + long counter = 0; + + @Override + public Long generate(EntropySource rng) + { + return counter++; + } + }; + } + + // TODO: distributions + public static Generator<Long> sequentialPd(SchemaSpec schema) + { + // TODO: switch away from Indexed generators here + HistoryBuilder.IndexedValueGenerators valueGenerators = (HistoryBuilder.IndexedValueGenerators) schema.valueGenerators; + int population = valueGenerators.pkPopulation(); + + return new Generator<>() + { + int counter = 0; + + @Override + public Long generate(EntropySource rng) + { + return valueGenerators.pkGen().descriptorAt(counter++ % population); + } + }; + } + + public static Generator<Long> sequentialCd(SchemaSpec schema) + { + // TODO: switch away from Indexed generators here + HistoryBuilder.IndexedValueGenerators valueGenerators = (HistoryBuilder.IndexedValueGenerators) schema.valueGenerators; + int population = valueGenerators.ckPopulation(); + + return new Generator<>() + { + int counter = 0; + + @Override + public Long generate(EntropySource rng) + { + return valueGenerators.ckGen().descriptorAt(counter++ % population); + } + }; + } + + public static Generator<ToOp> writeOp(SchemaSpec schema) + { + return writeOp(schema, sequentialPd(schema), sequentialCd(schema)); + } + + // TODO: chance of unset + public static Generator<ToOp> writeOp(SchemaSpec schema, + Generator<Long> pdGen, + Generator<Long> cdGen) + { + // TODO: switch away from Indexed generators here + HistoryBuilder.IndexedValueGenerators valueGenerators = (HistoryBuilder.IndexedValueGenerators) schema.valueGenerators; + + return (rng) -> { + long pd = pdGen.generate(rng); + long cd = cdGen.generate(rng); + long[] vds = new long[schema.regularColumns.size()]; + for (int i = 0; i < schema.regularColumns.size(); i++) + { + int idx = rng.nextInt(valueGenerators.regularPopulation(i)); + vds[i] = valueGenerators.regularColumnGen(i).descriptorAt(idx); + } + long[] sds = new long[schema.staticColumns.size()]; + for (int i = 0; i < schema.staticColumns.size(); i++) + { + int idx = rng.nextInt(valueGenerators.staticPopulation(i)); + sds[i] = valueGenerators.staticColumnGen(i).descriptorAt(idx); + } + return lts -> new Operations.WriteOp(lts, pd, cd, vds, sds, Operations.Kind.INSERT); + }; + } + + public interface ToOp + { + Operations.Operation toOp(long lts); + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/gen/SchemaGenerators.java b/test/harry/main/org/apache/cassandra/harry/gen/SchemaGenerators.java new file mode 100644 index 000000000000..c0e0c6e5696b --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/gen/SchemaGenerators.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.gen; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.apache.cassandra.harry.ColumnSpec; +import org.apache.cassandra.harry.SchemaSpec; + +public class SchemaGenerators +{ + public static Generator<SchemaSpec> schemaSpecGen(String ks, String prefix, int expectedValues) + { + return schemaSpecGen(ks, prefix, expectedValues, SchemaSpec.optionsBuilder()); + } + + public static Generator<SchemaSpec> schemaSpecGen(String ks, String prefix, int expectedValues, SchemaSpec.Options options) + { + return new Generator<>() + { + final Generator<List<ColumnSpec<?>>> regularGen = Generators.list(1, 10, regularColumnSpecGen()); + final Generator<List<ColumnSpec<?>>> staticGen = Generators.list(1, 10, staticColumnSpecGen()); + final Generator<List<ColumnSpec<?>>> ckGen = Generators.list(1, 10, ckColumnSpecGen()); + final Generator<List<ColumnSpec<?>>> pkGen = Generators.list(1, 10, pkColumnSpecGen()); + + int counter = 0; + public SchemaSpec generate(EntropySource rng) + { + int idx = counter++; + return new SchemaSpec(rng.next(), + expectedValues, + ks, + prefix + idx, + pkGen.generate(rng), + ckGen.generate(rng), + regularGen.generate(rng), + staticGen.generate(rng), + options); + }; + }; + } + + public static Generator<ColumnSpec<?>> regularColumnSpecGen() + { + return columnSpecGen("regular", ColumnSpec.Kind.REGULAR, Generators.pick(ColumnSpec.TYPES)); + } + + public static Generator<ColumnSpec<?>> staticColumnSpecGen() + { + return columnSpecGen("static", ColumnSpec.Kind.STATIC, Generators.pick(ColumnSpec.TYPES)); + } + + public static Generator<ColumnSpec<?>> pkColumnSpecGen() + { + return columnSpecGen("pk", ColumnSpec.Kind.PARTITION_KEY, Generators.pick(ColumnSpec.TYPES)); + } + + public static Generator<ColumnSpec<?>> ckColumnSpecGen() + { + List<ColumnSpec.DataType<?>> forwardAndReverse = new ArrayList<>(); + forwardAndReverse.addAll(ColumnSpec.TYPES); + forwardAndReverse.addAll(ColumnSpec.ReversedType.cache.values()); + return columnSpecGen("ck", ColumnSpec.Kind.CLUSTERING, Generators.pick(forwardAndReverse)); + } + + public static Generator<ColumnSpec<?>> columnSpecGen(String prefix, ColumnSpec.Kind kind, Generator<ColumnSpec.DataType<?>> typeGen) + { + return new Generator<ColumnSpec<?>>() + { + int counter = 0; + public ColumnSpec<?> generate(EntropySource rng) + { + ColumnSpec.DataType<?> type = typeGen.generate(rng); + Generator<?> gen = TypeAdapters.defaults.get(type.asServerType()); + int idx = counter++; + return new ColumnSpec(prefix + idx, type, gen, kind); + } + }; + } + + public static Generator<SchemaSpec> trivialSchema(String ks, String table, int population) + { + return (rng) -> { + return new SchemaSpec(rng.next(), + population, + ks, table, + Arrays.asList(ColumnSpec.pk("pk1", ColumnSpec.int64Type, Generators.int64())), + Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.int64Type, Generators.int64(), false)), + Arrays.asList(ColumnSpec.regularColumn("v1", ColumnSpec.int64Type)), + List.of(ColumnSpec.staticColumn("s1", ColumnSpec.int64Type))); + }; + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/gen/TypeAdapters.java b/test/harry/main/org/apache/cassandra/harry/gen/TypeAdapters.java new file mode 100644 index 000000000000..808ff5e45ef3 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/gen/TypeAdapters.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.gen; + +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.Map; + +import com.google.common.collect.ImmutableList; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.AsciiType; +import org.apache.cassandra.db.marshal.BooleanType; +import org.apache.cassandra.db.marshal.ByteType; +import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.marshal.DecimalType; +import org.apache.cassandra.db.marshal.DoubleType; +import org.apache.cassandra.db.marshal.FloatType; +import org.apache.cassandra.db.marshal.InetAddressType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.IntegerType; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.db.marshal.ReversedType; +import org.apache.cassandra.db.marshal.ShortType; +import org.apache.cassandra.db.marshal.TimeType; +import org.apache.cassandra.db.marshal.TimeUUIDType; +import org.apache.cassandra.db.marshal.TimestampType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.marshal.UUIDType; +import org.apache.cassandra.schema.ColumnMetadata; + +/** + * A class that helps to translate Cassandra's AbstractType instances to Harry Generators + */ +public class TypeAdapters +{ + public static final Map<AbstractType<?>, Generator<?>> defaults = new HashMap<>() {{ + put(ByteType.instance, Generators.int8()); + put(ShortType.instance, Generators.int16()); + put(Int32Type.instance, Generators.int32()); + put(LongType.instance, Generators.int64()); + put(BooleanType.instance, Generators.bool()); // this type has extremely small entryop + put(FloatType.instance, Generators.floats()); + put(DoubleType.instance, Generators.doubles()); + put(BytesType.instance, Generators.bytes(10, 20)); + put(AsciiType.instance, Generators.englishAlphabet(5, 10)); + put(UTF8Type.instance, Generators.utf8(10, 20)); + put(UUIDType.instance, Generators.uuidGen()); + put(TimeUUIDType.instance, Generators.timeuuid()); + put(TimestampType.instance, Generators.int64(0, Long.MAX_VALUE).map(Date::new)); + put(TimeType.instance, Generators.int64(0, Long.MAX_VALUE)); + put(IntegerType.instance, Generators.bigInt()); + put(DecimalType.instance, Generators.bigDecimal()); + put(InetAddressType.instance, Generators.inetAddr()); + + for (AbstractType<?> type : new ArrayList<>(keySet())) + put(ReversedType.getInstance(type), get(type)); + }}; + + public static Generator<Object[]> forKeys(ImmutableList<ColumnMetadata> columns) + { + return forKeys(columns, defaults); + } + + @SuppressWarnings({ "rawtypes", "unchecked" }) + private static Generator<Object[]> forKeys(ImmutableList<ColumnMetadata> columns, Map<AbstractType<?>, Generator<?>> typeToGen) + { + Generator[] gens = new Generator[columns.size()]; + for (int i = 0; i < gens.length; i++) + { + gens[i] = forValues(columns.get(i), typeToGen); + } + return Generators.zipArray(gens); + } + + public static Generator<Object> forValues(ColumnMetadata column) + { + return forValues(column, defaults); + } + + private static Generator<Object> forValues(ColumnMetadata column, Map<AbstractType<?>, Generator<?>> typeToGen) + { + Generator<Object> gen = (Generator<Object>) typeToGen.get(column.type); + if (gen == null) + { + throw new IllegalArgumentException(String.format("Could not find generator for column %s of type %s in %s", + column, column.type, typeToGen)); + } + return gen; + } + + public static Generator<Object> forValues(AbstractType type) + { + return forValues(type, defaults); + } + + private static Generator<Object> forValues(AbstractType type, Map<AbstractType<?>, Generator<?>> typeToGen) + { + Generator<Object> gen = (Generator<Object>) typeToGen.get(type); + if (gen == null) + { + throw new IllegalArgumentException(String.format("Could not find generator for column type %s in %s", + type, typeToGen)); + } + return gen; + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/gen/ValueGenerators.java b/test/harry/main/org/apache/cassandra/harry/gen/ValueGenerators.java new file mode 100644 index 000000000000..28a671a09c88 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/gen/ValueGenerators.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.gen; + +import java.util.Comparator; +import java.util.List; + +public class ValueGenerators +{ + protected final Bijections.Bijection<Object[]> pkGen; + protected final Bijections.Bijection<Object[]> ckGen; + + protected final List<Bijections.Bijection<Object>> regularColumnGens; + protected final List<Bijections.Bijection<Object>> staticColumnGens; + + protected final List<Comparator<Object>> pkComparators; + protected final List<Comparator<Object>> ckComparators; + protected final List<Comparator<Object>> regularComparators; + protected final List<Comparator<Object>> staticComparators; + + public ValueGenerators(Bijections.Bijection<Object[]> pkGen, + Bijections.Bijection<Object[]> ckGen, + List<Bijections.Bijection<Object>> regularColumnGens, + List<Bijections.Bijection<Object>> staticColumnGens, + + List<Comparator<Object>> pkComparators, + List<Comparator<Object>> ckComparators, + List<Comparator<Object>> regularComparators, + List<Comparator<Object>> staticComparators) + { + this.pkGen = pkGen; + this.ckGen = ckGen; + this.regularColumnGens = regularColumnGens; + this.staticColumnGens = staticColumnGens; + this.pkComparators = pkComparators; + this.ckComparators = ckComparators; + this.regularComparators = regularComparators; + this.staticComparators = staticComparators; + } + + public Bijections.Bijection<Object[]> pkGen() + { + return pkGen; + } + + public Bijections.Bijection<Object[]> ckGen() + { + return ckGen; + } + + public Bijections.Bijection<Object> regularColumnGen(int idx) + { + return regularColumnGens.get(idx); + } + + public Bijections.Bijection<Object> staticColumnGen(int idx) + { + return staticColumnGens.get(idx); + } + + public int ckColumnCount() + { + return ckComparators.size(); + } + + public int regularColumnCount() + { + return regularColumnGens.size(); + } + + public int staticColumnCount() + { + return staticColumnGens.size(); + } + + public Comparator<Object> pkComparator(int idx) + { + return pkComparators.get(idx); + } + + public Comparator<Object> ckComparator(int idx) + { + return ckComparators.get(idx); + } + + public Comparator<Object> regularComparator(int idx) + { + return regularComparators.get(idx); + } + + public Comparator<Object> staticComparator(int idx) + { + return staticComparators.get(idx); + } + + public int pkPopulation() + { + return pkGen.population(); + } + + public int ckPopulation() + { + return ckGen.population(); + } + + public int regularPopulation(int i) + { + return regularColumnGens.get(i).population(); + } + + public int staticPopulation(int i) + { + return staticColumnGens.get(i).population(); + } +} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/gen/distribution/Distribution.java b/test/harry/main/org/apache/cassandra/harry/gen/distribution/Distribution.java deleted file mode 100644 index ac961f9ac736..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/gen/distribution/Distribution.java +++ /dev/null @@ -1,136 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.gen.distribution; - -import org.apache.cassandra.harry.core.Configuration; - -public interface Distribution -{ - Configuration.DistributionConfig toConfig(); - - interface DistributionFactory - { - Distribution make(); - } - - long skew(long i); - - class IdentityDistribution implements Distribution - { - public Configuration.DistributionConfig toConfig() - { - return new Configuration.IdentityDistributionConfig(); - } - - public long skew(long i) - { - return i; - } - } - - class ConstantDistribution implements Distribution - { - public final long constant; - - public ConstantDistribution(long constant) - { - this.constant = constant; - } - - public Configuration.DistributionConfig toConfig() - { - return new Configuration.ConstantDistributionConfig(constant); - } - - public long skew(long i) - { - return constant; - } - - public String toString() - { - return "ConstantDistribution{" + - "constant=" + constant + - '}'; - } - } - - class ScaledDistribution implements Distribution - { - private final long min; - private final long max; - - public ScaledDistribution(long min, long max) - { - this.min = min; - this.max = max; - } - - public Configuration.DistributionConfig toConfig() - { - return new Configuration.ScaledDistributionConfig(min, max); - } - - public long skew(long i) - { - return scale(i, min, max); - } - - public static long scale(long value, long min, long max) - { - if (value == 0) - return (max - min) / 2; - - double nomalized = (1.0 * Math.abs(value)) / Long.MAX_VALUE; - double diff = 0.5 * (max - min); - if (value > 0) - return (long) (min + nomalized * diff); - else - return (long) (max - nomalized * diff); - } - - public String toString() - { - return "ScaledDistribution{" + - "min=" + min + - ", max=" + max + - '}'; - } - } - - class NormalDistribution implements Distribution - { - private final org.apache.commons.math3.distribution.NormalDistribution delegate; - - public NormalDistribution() - { - delegate = new org.apache.commons.math3.distribution.NormalDistribution(); - } - - public Configuration.DistributionConfig toConfig() - { - return new Configuration.NormalDistributionConfig(); - } - - public long skew(long i) - { - return (long) delegate.cumulativeProbability((double) i); - } - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/gen/rng/JdkRandomEntropySource.java b/test/harry/main/org/apache/cassandra/harry/gen/rng/JdkRandomEntropySource.java index 5ddc88528396..068b6737851d 100644 --- a/test/harry/main/org/apache/cassandra/harry/gen/rng/JdkRandomEntropySource.java +++ b/test/harry/main/org/apache/cassandra/harry/gen/rng/JdkRandomEntropySource.java @@ -36,34 +36,48 @@ public JdkRandomEntropySource(Random rng) this.rng = rng; } + @Override public long next() { return rng.nextLong(); } + @Override public void seed(long seed) { rng.setSeed(seed); } + @Override public EntropySource derive() { return new JdkRandomEntropySource(new Random(rng.nextLong())); } + @Override public int nextInt() { return rng.nextInt(); } + /** + * Generates int between [0, max). + */ + @Override public int nextInt(int max) { return rng.nextInt(max); } + /** + * Generates a value between [min, max). + */ + @Override public int nextInt(int min, int max) { - return rng.nextInt(max) + min; + if (min == max) + return min; + return rng.nextInt(max - min) + min; } public long nextLong() @@ -71,11 +85,19 @@ public long nextLong() return rng.nextLong(); } + @Override public float nextFloat() { return rng.nextFloat(); } + @Override + public double nextDouble() + { + return rng.nextDouble(); + } + + @Override public boolean nextBoolean() { return rng.nextBoolean(); diff --git a/test/harry/main/org/apache/cassandra/harry/gen/rng/PcgRSUFast.java b/test/harry/main/org/apache/cassandra/harry/gen/rng/PcgRSUFast.java index 070fe24f67bd..913e61615c8a 100644 --- a/test/harry/main/org/apache/cassandra/harry/gen/rng/PcgRSUFast.java +++ b/test/harry/main/org/apache/cassandra/harry/gen/rng/PcgRSUFast.java @@ -108,6 +108,12 @@ public float nextFloat() return RngUtils.asFloat(next()); } + @Override + public double nextDouble() + { + return RngUtils.asDouble(next()); + } + @Override public boolean nextBoolean() { diff --git a/test/harry/main/org/apache/cassandra/harry/gen/rng/PureRng.java b/test/harry/main/org/apache/cassandra/harry/gen/rng/PureRng.java new file mode 100644 index 000000000000..6fc8aa276bfd --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/gen/rng/PureRng.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.gen.rng; + +public interface PureRng +{ + long randomNumber(long i, long stream); + + long sequenceNumber(long r, long stream); + + default long next(long r) + { + return next(r, 0); + } + + long next(long r, long stream); + + long prev(long r, long stream); + + default long prev(long r) + { + return next(r, 0); + } + + class PCGFast implements PureRng + { + private final long seed; + + public PCGFast(long seed) + { + this.seed = seed; + } + + public long randomNumber(long i, long stream) + { + return PCGFastPure.shuffle(PCGFastPure.advanceState(seed, i, stream)); + } + + public long sequenceNumber(long r, long stream) + { + return PCGFastPure.distance(seed, PCGFastPure.unshuffle(r), stream); + } + + public long next(long r, long stream) + { + return PCGFastPure.next(r, stream); + } + + public long prev(long r, long stream) + { + return PCGFastPure.previous(r, stream); + } + } +} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/gen/rng/SeedableEntropySource.java b/test/harry/main/org/apache/cassandra/harry/gen/rng/SeedableEntropySource.java new file mode 100644 index 000000000000..48afebe4c284 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/gen/rng/SeedableEntropySource.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.gen.rng; + +import java.util.function.Consumer; +import java.util.function.Function; + +import io.netty.util.concurrent.FastThreadLocal; +import org.apache.cassandra.harry.gen.EntropySource; + +public class SeedableEntropySource +{ + private static final FastThreadLocal<EntropySource> THREAD_LOCAL = new FastThreadLocal<>(); + + public static <T> T computeWithSeed(long seed, Function<EntropySource, T> fn) + { + return fn.apply(entropySource(seed)); + } + + public static void doWithSeed(long seed, Consumer<EntropySource> fn) + { + fn.accept(entropySource(seed)); + } + + private static EntropySource entropySource(long seed) + { + EntropySource entropySource = THREAD_LOCAL.get(); + if (entropySource == null) + { + entropySource = new JdkRandomEntropySource(0); + THREAD_LOCAL.set(entropySource); + } + entropySource.seed(seed); + return entropySource; + } +} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/gen/test/EntropySourceTest.java b/test/harry/main/org/apache/cassandra/harry/gen/test/EntropySourceTest.java new file mode 100644 index 000000000000..d04ad5a783f0 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/gen/test/EntropySourceTest.java @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.gen.test; + +import org.junit.Assert; +import org.junit.Test; + +import org.apache.cassandra.harry.gen.EntropySource; +import org.apache.cassandra.harry.gen.rng.PCGFastPure; +import org.apache.cassandra.harry.gen.rng.PcgRSUFast; +import org.apache.cassandra.harry.gen.rng.PureRng; + +import static org.apache.cassandra.harry.checker.TestHelper.repeat; +import static org.apache.cassandra.harry.checker.TestHelper.withRandom; + +public class EntropySourceTest +{ + private static int RUNS = 100000; + + @Test + public void testShuffleUnshuffle() + { + withRandom(rng -> { + repeat(RUNS, () -> { + long l = rng.next(); + Assert.assertEquals(l, PCGFastPure.unshuffle(PCGFastPure.shuffle(l))); + }); + }); + } + + @Test + public void testImmutableRng() + { + withRandom(entropy -> { + int size = 5; + PureRng rng = new PureRng.PCGFast(entropy.next()); + repeat(RUNS, () -> { + long stream = entropy.next(); + long[] generated = new long[size]; + for (int i = 0; i < size; i++) + generated[i] = rng.randomNumber(i, stream); + + Assert.assertEquals(0, rng.sequenceNumber(generated[0], stream)); + Assert.assertEquals(generated[1], rng.next(generated[0], stream)); + + for (int i = 1; i < size; i++) + { + Assert.assertEquals(generated[i], rng.next(generated[i - 1], stream)); + Assert.assertEquals(generated[i - 1], rng.prev(generated[i], stream)); + Assert.assertEquals(i, rng.sequenceNumber(generated[i], stream)); + } + }); + }); + } + + @Test + public void testSequenceNumber() + { + withRandom(entropy -> { + int size = 5; + PureRng rng = new PureRng.PCGFast(entropy.next()); + for (int stream = 1; stream < RUNS; stream++) + { + for (int i = 0; i < size; i++) + Assert.assertEquals(i, rng.sequenceNumber(rng.randomNumber(i, stream), stream)); + } + }); + } + + @Test + public void seekTest() + { + PcgRSUFast rand = new PcgRSUFast(1, 1); + long first = rand.next(); + long last = 0; + for (int i = 0; i < 10; i++) + last = rand.next(); + + rand.advance(-11); + Assert.assertEquals(first, rand.next()); + + rand.advance(9); + Assert.assertEquals(last, rand.next()); + Assert.assertEquals(first, rand.nextAt(0)); + Assert.assertEquals(last, rand.nextAt(10)); + Assert.assertEquals(-10, rand.distance(first)); + } + + @Test + public void shuffleUnshuffleTest() + { + withRandom(entropy -> { + repeat(RUNS, () -> { + long a = entropy.next(); + Assert.assertEquals(a, PCGFastPure.unshuffle(PCGFastPure.shuffle(a))); + }); + }); + } + + @Test + public void testIntBetween() + { + withRandom(entropy -> { + EntropySource rng = new PcgRSUFast(entropy.next(), entropy.next()); + int a = 0; + int b = 50; + int[] cardinality = new int[b - a]; + for (int i = 0; i < RUNS; i++) + { + int min = Math.min(a, b); + int max = Math.max(a, b); + cardinality[rng.nextInt(min, max - 1) - min]++; + } + + // Extremely improbable yet possible that some of the values won't be generated + for (int i = 0; i < cardinality.length; i++) + Assert.assertTrue(cardinality[i] > 0); + }); + } +} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/model/AgainstSutChecker.java b/test/harry/main/org/apache/cassandra/harry/model/AgainstSutChecker.java deleted file mode 100644 index e4bd009c8c77..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/model/AgainstSutChecker.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.model; - -import java.util.List; - -import org.apache.cassandra.harry.data.ResultSetRow; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.sut.QueryModifyingSut; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.operations.CompiledStatement; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.tracker.DataTracker; - -/** - * A simple way to verify if something might be a Harry issue: check against a different SUT. - * - * For example, if you are using `flush` in your primary SUT, avoid using it for the secondary SUT, - * and compare results. - * - * Usually used in combination with {@link QueryModifyingSut}, which writes to - * the second SUT. - * - * SchemaSpec doubleWriteSchema = schema.cloneWithName(schema.keyspace, schema.keyspace + "_debug"); - * - * sut.schemaChange(doubleWriteSchema.compile().cql()); - * - * QueryModifyingSut sut = new QueryModifyingSut(this.sut, - * schema.table, - * doubleWriteSchema.table); - * - * - * Model model = new AgainstSutChecker(tracker, history.clock(), sut, schema, doubleWriteSchema); - */ -public class AgainstSutChecker implements Model -{ - protected final OpSelectors.Clock clock; - protected final SystemUnderTest sut; - protected final SchemaSpec schema; - protected final SchemaSpec doubleWriteTable; - protected final DataTracker tracker; - - public AgainstSutChecker(DataTracker tracker, - OpSelectors.Clock clock, - SystemUnderTest sut, - SchemaSpec schema, - SchemaSpec doubleWriteTable) - { - this.clock = clock; - this.sut = sut; - this.schema = schema; - this.doubleWriteTable = doubleWriteTable; - this.tracker = tracker; - } - - public void validate(Query query) - { - tracker.beginValidation(query.pd); - - List<ResultSetRow> rows1 = executeOnMainSchema(query); - List<ResultSetRow> rows2 = executeOnDebugSchema(query); - - if (rows1.size() != rows2.size()) - throw new IllegalStateException(String.format("Sizes do not match %d %d\n%s\n%s\nQuery:%s\n", rows1.size(), rows2.size(), rows1, rows2, query.toSelectStatement())); - - for (int i = 0; i < rows1.size(); i++) - { - if (!rows1.get(i).equals(rows2.get(i))) - { - throw new IllegalStateException(String.format("Rows mismatch:\n" + - "%s\n" + - "%s\n", - rows1.get(i), - rows2.get(i))); - } - } - tracker.endValidation(query.pd); - } - - protected final List<ResultSetRow> executeOnMainSchema(Query query) - { - CompiledStatement s1 = query.toSelectStatement(schema.allColumnsSet, true); - return SelectHelper.execute(sut, clock, s1, schema); - } - - protected List<ResultSetRow> executeOnDebugSchema(Query query) - { - CompiledStatement s2 = query.toSelectStatement(doubleWriteTable.allColumnsSet, true) - .withSchema(schema.keyspace, schema.table, doubleWriteTable.keyspace, doubleWriteTable.table); - return SelectHelper.execute(sut, clock, s2, schema); - } - -} diff --git a/test/harry/main/org/apache/cassandra/harry/model/AlwaysSamePartitionSelector.java b/test/harry/main/org/apache/cassandra/harry/model/AlwaysSamePartitionSelector.java deleted file mode 100644 index daecd029fe43..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/model/AlwaysSamePartitionSelector.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.model; - -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import org.apache.cassandra.harry.core.Configuration; - -/** - * A simple test-only descriptor selector that can used for testing things where you only need one partition - */ -public class AlwaysSamePartitionSelector extends OpSelectors.PdSelector -{ - private final long pd; - - public AlwaysSamePartitionSelector(long pd) - { - this.pd = pd; - } - - protected long pd(long lts) - { - return 0; - } - - public long nextLts(long lts) - { - return lts + 1; - } - - public long prevLts(long lts) - { - return lts - 1; - } - - public long maxLtsFor(long pd) - { - return 1000; - } - - public long minLtsAt(long position) - { - return 0; - } - - public long minLtsFor(long pd) - { - return 0; - } - - public long positionFor(long lts) - { - return 0; - } - - public long maxPosition(long maxLts) - { - return 0; - } - - @JsonTypeName("always_same") - public static class AlwaysSamePartitionSelectorConfiguration implements Configuration.PDSelectorConfiguration - { - private final long pd; - - public AlwaysSamePartitionSelectorConfiguration(@JsonProperty("pd") long pd) - { - this.pd = pd; - } - - public OpSelectors.PdSelector make(OpSelectors.PureRng rng) - { - return new AlwaysSamePartitionSelector(pd); - } - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/model/Model.java b/test/harry/main/org/apache/cassandra/harry/model/Model.java index 0a85eaffeb6e..642488014d54 100644 --- a/test/harry/main/org/apache/cassandra/harry/model/Model.java +++ b/test/harry/main/org/apache/cassandra/harry/model/Model.java @@ -1,47 +1,49 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.cassandra.harry.model; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.operations.Query; +import java.util.List; + +import org.apache.cassandra.harry.op.Operations; +import org.apache.cassandra.harry.execution.ResultSetRow; +import org.apache.cassandra.harry.op.Visit; public interface Model { - long NO_TIMESTAMP = Long.MIN_VALUE; - - void validate(Query query); + void validate(Operations.SelectStatement select, List<ResultSetRow> actual); - interface ModelFactory + class LtsOperationPair { - Model make(Run run); - } + final long lts; + final int opId; - class ValidationException extends RuntimeException - { - public ValidationException(String trackerState, String partitionState, String observedState, String format, Object... objects) + public LtsOperationPair(long lts, int opId) { - super(String.format(format, objects) + - "\nTracker state:\n" + trackerState + - "\nPartition state:\n" + partitionState + - "\nObserved state:\n" + observedState); + this.lts = lts; + this.opId = opId; } } + interface Replay extends Iterable<Visit> + { + Visit replay(long lts); + Operations.Operation replay(long lts, int opId); + } -} \ No newline at end of file +} diff --git a/test/harry/main/org/apache/cassandra/harry/model/NoOpChecker.java b/test/harry/main/org/apache/cassandra/harry/model/NoOpChecker.java deleted file mode 100644 index 7683f0ef9516..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/model/NoOpChecker.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.model; - -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.operations.Query; - -public class NoOpChecker implements Model -{ - private final SystemUnderTest sut; - private final SystemUnderTest.ConsistencyLevel cl; - public NoOpChecker(Run run) - { - this(run.sut, SystemUnderTest.ConsistencyLevel.QUORUM); - } - - public NoOpChecker(SystemUnderTest sut, SystemUnderTest.ConsistencyLevel cl) - { - this.sut = sut; - this.cl = cl; - } - - public void validate(Query query) - { - sut.execute(query.toSelectStatement(), cl); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/model/OpSelectors.java b/test/harry/main/org/apache/cassandra/harry/model/OpSelectors.java deleted file mode 100644 index fe83a418a8c0..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/model/OpSelectors.java +++ /dev/null @@ -1,903 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.model; - -import java.util.EnumMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - -import org.apache.cassandra.harry.core.Configuration; -import com.google.common.annotations.VisibleForTesting; -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.gen.Bytes; -import org.apache.cassandra.harry.gen.rng.PCGFastPure; -import org.apache.cassandra.harry.gen.rng.RngUtils; -import org.apache.cassandra.harry.gen.Surjections; -import org.apache.cassandra.harry.gen.distribution.Distribution; -import org.apache.cassandra.harry.runner.EarlyExitException; -import org.apache.cassandra.harry.util.BitSet; - -import static org.apache.cassandra.harry.gen.DataGenerators.NIL_DESCR; -import static org.apache.cassandra.harry.gen.DataGenerators.UNSET_DESCR; - -/** - * Row (deflated) data selectors. Not calling them generators, since their output is entirely - * deterministic, and for each input they are able to produce a single output. - * <p> - * This is more or less a direct translation of the formalization. - * <p> - * All functions implemented by this interface have to _always_ produce same outputs for same inputs. - * Most of the functions, with the exception of real-time clock translations, should be pure. - * <p> - * Functions that are reverse of their counterparts are prefixed with "un" - */ -public interface OpSelectors -{ - interface PureRng - { - long randomNumber(long i, long stream); - - long sequenceNumber(long r, long stream); - - default long next(long r) - { - return next(r, 0); - } - - long next(long r, long stream); - - long prev(long r, long stream); - - default long prev(long r) - { - return next(r, 0); - } - } - - /** - * Clock is a component responsible for mapping _logical_ timestamps to _real-time_ ones. - * When reproducing test failures, and for validation purposes, a snapshot of such clock can - * be taken to map a real-time timestamp from the value retrieved from the database in order - * to map it back to the logical timestamp of the operation that wrote this value. - */ - interface Clock - { - /** - * Returns a _real-time_ time timestamp given a _logical_ timestamp - */ - long rts(long lts); - /** - * Returns a _logical_ time timestamp given a _real-time_ timestamp - */ - long lts(long rts); - - /** - * Returns the next/smallest logical timestamp that has not yet been visited, and - * moves internal state to indicate that his timestamp has been taken. - */ - long nextLts(); - - /** - * Same as #nextLts, but does not change internal state - */ - long peek(); - - Configuration.ClockConfiguration toConfig(); - } - - interface ClockFactory - { - Clock make(); - } - - /** - * *Partition descriptor selector* controls how partitions is selected based on the current logical - * timestamp. Default implementation is a sliding window of partition descriptors that will visit - * one partition after the other in the window `slide_after_repeats` times. After that will - * retire one partition descriptor, and pick one instead of it. - */ - abstract class PdSelector - { - @VisibleForTesting - protected abstract long pd(long lts); - - public long pd(long lts, SchemaSpec schema) - { - return schema.adjustPdEntropy(pd(lts)); - } - - // previous and next LTS with that will yield same pd - public abstract long nextLts(long lts); - - public abstract long prevLts(long lts); - - public abstract long maxLtsFor(long pd); - public abstract long minLtsAt(long position); - - public abstract long minLtsFor(long pd); - public abstract long maxPosition(long maxLts); - } - - interface PdSelectorFactory - { - PdSelector make(PureRng rng); - } - - interface DescriptorSelectorFactory - { - DescriptorSelector make(PureRng rng, SchemaSpec schemaSpec); - } - - /** - * DescriptorSelector controls how clustering descriptors are picked within the partition: - * how many rows there can be in a partition, how many rows will be visited for a logical timestamp, - * how many operations there will be in batch, what kind of operations there will and how often - * each kind of operation is going to occur. - */ - abstract class DescriptorSelector - { - public abstract int operationsPerLts(long lts); - - public abstract int maxPartitionSize(); - - public abstract boolean isCdVisitedBy(long pd, long lts, long cd); - - // clustering descriptor is calculated using operation id and not modification id, since - // value descriptors are calculated using modification ids. - public long cd(long pd, long lts, long opId, SchemaSpec schema) - { - return schema.adjustCdEntropy(cd(pd, lts, opId)); - } - - /** - * Currently, we do not allow visiting the same row more than once per lts, which means that: - * <p> - * * `max(opId)` returned `cds` have to be unique for any `lts/pd` pair - * * {@code max(opId) < maxPartitionSize} - */ - @VisibleForTesting - protected abstract long cd(long pd, long lts, long opId); - - public long randomCd(long pd, long entropy, SchemaSpec schema) - { - return schema.adjustCdEntropy(randomCd(pd, entropy)); - } - - public abstract long randomCd(long pd, long entropy); - - @VisibleForTesting - protected abstract long vd(long pd, long cd, long lts, long opId, int col); - - public long[] vds(long pd, long cd, long lts, long opId, OperationKind opType, SchemaSpec schema) - { - BitSet setColumns = columnMask(pd, lts, opId, opType); - return descriptors(pd, cd, lts, opId, schema.regularColumns, schema.regularColumnsMask(), setColumns, schema.regularColumnsOffset); - } - - public long[] sds(long pd, long cd, long lts, long opId, OperationKind opType, SchemaSpec schema) - { - BitSet setColumns = columnMask(pd, lts, opId, opType); - return descriptors(pd, cd, lts, opId, schema.staticColumns, schema.staticColumnsMask(), setColumns, schema.staticColumnsOffset); - } - - public long[] descriptors(long pd, long cd, long lts, long opId, List<ColumnSpec<?>> columns, BitSet mask, BitSet setColumns, int offset) - { - assert opId < operationsPerLts(lts) : String.format("Operation id %d exceeds the maximum expected number of operations per lts %d", - opId, operationsPerLts(lts)); - long[] descriptors = new long[columns.size()]; - - for (int i = 0; i < descriptors.length; i++) - { - int col = offset + i; - if (setColumns.isSet(col, mask)) - { - ColumnSpec<?> spec = columns.get(i); - long vd = vd(pd, cd, lts, opId, col) & Bytes.bytePatternFor(spec.type.maxSize()); - assert vd != UNSET_DESCR : "Ambiguous unset descriptor generated for the value"; - assert vd != NIL_DESCR : "Ambiguous nil descriptor generated for the value"; - - descriptors[i] = vd; - } - else - { - descriptors[i] = UNSET_DESCR; - } - } - return descriptors; - } - - public abstract OperationKind operationType(long pd, long lts, long opId); - - public abstract BitSet columnMask(long pd, long lts, long opId, OperationKind opType); - - public abstract long opId(long pd, long lts, long cd); - - } - - class PCGFast implements PureRng - { - private final long seed; - - public PCGFast(long seed) - { - this.seed = seed; - } - - public long randomNumber(long i, long stream) - { - return PCGFastPure.shuffle(PCGFastPure.advanceState(seed, i, stream)); - } - - public long sequenceNumber(long r, long stream) - { - return PCGFastPure.distance(seed, PCGFastPure.unshuffle(r), stream); - } - - public long next(long r, long stream) - { - return PCGFastPure.next(r, stream); - } - - public long prev(long r, long stream) - { - return PCGFastPure.previous(r, stream); - } - } - - /** - * Generates partition descriptors, based on LTS as if we had a sliding window. - * <p> - * Each {@code windowSize * switchAfter} steps, we move the window by one, effectively - * expiring one partition descriptor, and adding one partition descriptor to the window. - * <p> - * For any LTS, we can calculate previous and next LTS on which it will visit the same - * partition - */ - class DefaultPdSelector extends OpSelectors.PdSelector - { - public final static long PARTITION_DESCRIPTOR_STREAM_ID = 0x706b; - - private final PureRng rng; - - private final long slideAfterRepeats; - private final long switchAfter; - private final long windowSize; - - private final long positionOffset; - private final long positionWindowSize; - - public DefaultPdSelector(PureRng rng, long windowSize, long slideAfterRepeats) - { - this(rng, windowSize, slideAfterRepeats, 0L, Long.MAX_VALUE); - } - - public DefaultPdSelector(PureRng rng, long windowSize, long slideAfterRepeats, long positionOffset, long positionWindowSize) - { - this.rng = rng; - this.slideAfterRepeats = slideAfterRepeats; - this.windowSize = windowSize; - this.switchAfter = windowSize * slideAfterRepeats; - this.positionOffset = positionOffset; - this.positionWindowSize = positionWindowSize; - } - - public Iterator<Long> allPds(long maxLts) - { - final long maxPosition = maxPosition(maxLts); - return new Iterator<Long>() - { - private long position = 0; - - public boolean hasNext() - { - return position <= maxPosition; - } - - public Long next() - { - long pd = rng.randomNumber(adjustPosition(position), PARTITION_DESCRIPTOR_STREAM_ID); - position++; - return pd; - } - }; - } - - protected long pd(long lts) - { - return rng.randomNumber(adjustPosition(positionFor(lts)), PARTITION_DESCRIPTOR_STREAM_ID); - } - - public long minLtsAt(long position) - { - if (position < windowSize) - return position; - - long windowStart = (position - (windowSize - 1)) * slideAfterRepeats * windowSize; - return windowStart + windowSize - 1; - } - - public long minLtsFor(long pd) - { - long position = positionForPd(pd); - return minLtsAt(position); - } - - public long randomVisitedPd(long maxLts, long visitLts, SchemaSpec schema) - { - long maxPosition = maxPosition(maxLts); - if (maxPosition == 0) - return pdAtPosition(0, schema); - - int idx = RngUtils.asInt(rng.randomNumber(visitLts, maxPosition), 0, (int) (maxPosition - 1)); - return pdAtPosition(idx, schema); - } - - public long maxPosition(long maxLts) - { - long timesCycled = maxLts / switchAfter; - long windowStart = timesCycled * windowSize; - - // We have cycled through _current_ window at least once - if (maxLts > (windowStart * slideAfterRepeats) + windowSize) - return windowStart + windowSize; - return windowStart + maxLts % windowSize; - } - - public long positionFor(long lts) - { - long windowStart = lts / switchAfter; - return windowStart + lts % windowSize; - } - - /** - * We only adjust position right before we go to the PCG RNG to grab a partition id, since we want - * the fact that PCG is actually offset to be hidden from the user and even the rest of this class. - */ - private long adjustPosition(long position) - { - if (position > positionWindowSize) - throw new EarlyExitException(String.format("Partition position has wrapped around, so can not be safely used. " + - "This runner has been given %d partitions, and if we wrap back to " + - "position 0, partition state is not going to be inflatable, since" + - "nextLts will not jump to the lts that is about to be visited. " + - "Increase rows per visit, batch size, or slideAfter repeats.", - positionWindowSize)); - return positionOffset + position; - } - - /** - * When computing position from pd or lts, we need to translate the offset back to the original, since - * it is not aware (and should not be) of the fact that positions are offset. - */ - private long unadjustPosition(long position) - { - return position - positionOffset; - } - - public long positionForPd(long pd) - { - return unadjustPosition(rng.sequenceNumber(pd, PARTITION_DESCRIPTOR_STREAM_ID)); - } - - public long pdAtPosition(long position, SchemaSpec schemaSpec) - { - return schemaSpec.adjustCdEntropy(rng.randomNumber(adjustPosition(position), PARTITION_DESCRIPTOR_STREAM_ID)); - } - - public long nextLts(long lts) - { - long slideCount = lts / switchAfter; - long positionInCycle = lts - slideCount * switchAfter; - long nextRepeat = positionInCycle / windowSize + 1; - - if (nextRepeat > slideAfterRepeats || - (nextRepeat == slideAfterRepeats && (positionInCycle % windowSize) == 0)) - return -1; - - // last cycle before window slides; next window will have shifted by one - if (nextRepeat == slideAfterRepeats) - positionInCycle -= 1; - - return slideCount * switchAfter + windowSize + positionInCycle; - } - - public long prevLts(long lts) - { - long slideCount = lts / switchAfter; - long positionInCycle = lts - slideCount * switchAfter; - long prevRepeat = positionInCycle / windowSize - 1; - - if (lts < windowSize || - prevRepeat < -1 || - (prevRepeat == -1 && (positionInCycle % windowSize) == (windowSize - 1))) - return -1; - - // last cycle before window slides; next window will have shifted by one - if (prevRepeat == -1) - positionInCycle += 1; - - return slideCount * switchAfter - windowSize + positionInCycle; - } - - public long maxLtsFor(long pd) - { - long position = positionForPd(pd); - return position * switchAfter + (slideAfterRepeats - 1) * windowSize; - } - - public String toString() - { - return "DefaultPdSelector{" + - "slideAfterRepeats=" + slideAfterRepeats + - ", windowSize=" + windowSize + - '}'; - } - } - - static ColumnSelectorBuilder columnSelectorBuilder() - { - return new ColumnSelectorBuilder(); - } - - class ColumnSelectorBuilder - { - private Map<OperationKind, Surjections.Surjection<BitSet>> m; - - public ColumnSelectorBuilder() - { - this.m = new EnumMap<>(OperationKind.class); - } - - public ColumnSelectorBuilder forAll(SchemaSpec schema) - { - return forAll(schema, BitSet.surjection(schema.allColumns.size())); - } - - // TODO: change bitsets to take into account _all_ columns not only regulars - public ColumnSelectorBuilder forAll(SchemaSpec schema, Surjections.Surjection<BitSet> orig) - { - for (OperationKind type : OperationKind.values()) - { - Surjections.Surjection<BitSet> gen = orig; - - switch (type) - { - case UPDATE_WITH_STATICS: - case DELETE_COLUMN_WITH_STATICS: - gen = (descriptor) -> { - long counter = 0; - while (counter <= 100) - { - BitSet bitSet = orig.inflate(descriptor); - if ((schema.regularColumns.isEmpty() || !bitSet.allUnset(schema.regularColumnsMask)) - && (schema.staticColumns.isEmpty() || !bitSet.allUnset(schema.staticColumnsMask))) - return bitSet; - - descriptor = RngUtils.next(descriptor); - counter++; - } - throw new RuntimeException(String.format("Could not generate a value after %d attempts.", counter)); - }; - break; - // Can not have an UPDATE statement without anything to update - case UPDATE: - gen = descriptor -> { - long counter = 0; - while (counter <= 100) - { - BitSet bitSet = orig.inflate(descriptor); - - if (!bitSet.allUnset(schema.regularColumnsMask)) - return bitSet; - - descriptor = RngUtils.next(descriptor); - counter++; - } - throw new RuntimeException(String.format("Could not generate a value after %d attempts.", counter)); - }; - break; - case DELETE_COLUMN: - gen = (descriptor) -> { - long counter = 0; - while (counter <= 100) - { - BitSet bitSet = orig.inflate(descriptor); - BitSet mask = schema.regularColumnsMask; - - if (!bitSet.allUnset(mask)) - return bitSet; - - descriptor = RngUtils.next(descriptor); - counter++; - } - throw new RuntimeException(String.format("Could not generate a value after %d attempts.", counter)); - }; - break; - } - this.m.put(type, gen); - } - return this; - } - - public ColumnSelectorBuilder forWrite(Surjections.Surjection<BitSet> gen) - { - m.put(OperationKind.INSERT, gen); - return this; - } - - public ColumnSelectorBuilder forWrite(BitSet pickFrom) - { - return forWrite(Surjections.pick(pickFrom)); - } - - public ColumnSelectorBuilder forDelete(Surjections.Surjection<BitSet> gen) - { - m.put(OperationKind.DELETE_ROW, gen); - return this; - } - - public ColumnSelectorBuilder forDelete(BitSet pickFrom) - { - return forDelete(Surjections.pick(pickFrom)); - } - - public ColumnSelectorBuilder forColumnDelete(Surjections.Surjection<BitSet> gen) - { - m.put(OperationKind.DELETE_COLUMN, gen); - return this; - } - - public ColumnSelectorBuilder forColumnDelete(BitSet pickFrom) - { - return forColumnDelete(Surjections.pick(pickFrom)); - } - - public ColumnSelector build() - { - return (kind, descriptor) -> m.get(kind).inflate(descriptor); - } - } - - - /** - * ColumnSelector has to return BitSet specifying _all_ columns - */ - interface ColumnSelector - { - public BitSet columnMask(OperationKind operationKind, long descriptor); - } - - class HierarchicalDescriptorSelector extends DefaultDescriptorSelector - { - private final int[] fractions; - - public HierarchicalDescriptorSelector(PureRng rng, - // how many parts (at most) each subsequent "level" should contain - int[] fractions, - ColumnSelector columnSelector, - OperationSelector operationSelector, - Distribution operationsPerLtsDistribution, - int maxPartitionSize) - { - super(rng, - columnSelector, - operationSelector, - operationsPerLtsDistribution, - maxPartitionSize); - this.fractions = fractions; - } - - @Override - public long cd(long pd, long lts, long opId, SchemaSpec schema) - { - if (schema.clusteringKeys.size() <= 1) - return schema.adjustCdEntropy(super.cd(pd, lts, opId)); - - int partitionSize = maxPartitionSize(); - int clusteringOffset = clusteringOffset(lts); - long res; - if (clusteringOffset == 0) - { - res = rng.prev(opId, pd); - } - else - { - int positionInPartition = (int) ((clusteringOffset + opId) % partitionSize); - res = cd(positionInPartition, fractions, schema, rng, pd); - } - return schema.adjustCdEntropy(res); - } - - @VisibleForTesting - public static long cd(int positionInPartition, int[] fractions, SchemaSpec schema, PureRng rng, long pd) - { - long[] slices = new long[schema.clusteringKeys.size()]; - for (int i = 0; i < slices.length; i++) - { - int idx = i < fractions.length ? (positionInPartition % (fractions[i] - 1)) : positionInPartition; - slices[i] = rng.prev(idx, rng.next(pd, i + 1)); - } - - return schema.ckGenerator.stitch(slices); - } - - protected long cd(long pd, long lts, long opId) - { - throw new RuntimeException("Shouldn't be called"); - } - } - - class DefaultDescriptorSelector extends DescriptorSelector - { - protected final static long OPERATIONS_PER_LTS_STREAM = 0x5e03812e293L; - protected final static long BITSET_IDX_STREAM = 0x92eb607bef1L; - - public static OperationSelector DEFAULT_OP_SELECTOR = OperationSelector.weighted(Surjections.weights(45, 45, 3, 2, 2, 1, 1, 1), - OperationKind.INSERT, - OperationKind.INSERT_WITH_STATICS, - OperationKind.DELETE_ROW, - OperationKind.DELETE_COLUMN, - OperationKind.DELETE_COLUMN_WITH_STATICS, - OperationKind.DELETE_PARTITION, - OperationKind.DELETE_RANGE, - OperationKind.DELETE_SLICE); - - protected final PureRng rng; - protected final OperationSelector operationSelector; - protected final ColumnSelector columnSelector; - protected final Distribution operationsPerLtsDistribution; - protected final int maxPartitionSize; - - public DefaultDescriptorSelector(PureRng rng, - ColumnSelector columnMaskSelector, - OperationSelector operationSelector, - Distribution operationsPerLtsDistribution, - int maxPartitionSize) - { - this.rng = rng; - - this.operationSelector = operationSelector; - this.columnSelector = columnMaskSelector; - this.operationsPerLtsDistribution = operationsPerLtsDistribution; - this.maxPartitionSize = maxPartitionSize; - } - - public int operationsPerLts(long lts) - { - return (int) operationsPerLtsDistribution.skew(rng.randomNumber(lts, OPERATIONS_PER_LTS_STREAM)); - } - - public int maxPartitionSize() - { - return maxPartitionSize; - } - - protected int clusteringOffset(long lts) - { - return RngUtils.asInt(lts, 0, maxPartitionSize() - 1); - } - - // TODO: this won't work for entropy-adjusted CDs, at least the way they're implemented now - public boolean isCdVisitedBy(long pd, long lts, long cd) - { - return opId(pd, lts, cd) < operationsPerLts(lts); - } - - public long randomCd(long pd, long entropy) - { - long positionInPartition = Math.abs(rng.prev(entropy)) % maxPartitionSize(); - return rng.prev(positionInPartition, pd); - } - - protected long cd(long pd, long lts, long opId) - { - int partitionSize = maxPartitionSize(); - int clusteringOffset = clusteringOffset(lts); - if (clusteringOffset == 0) - return rng.prev(opId, pd); - - int positionInPartition = (int) ((clusteringOffset + opId) % partitionSize); - return rng.prev(positionInPartition, pd); - } - - public long opId(long pd, long lts, long cd) - { - int partitionSize = maxPartitionSize(); - int clusteringOffset = clusteringOffset(lts); - int positionInPartition = (int) rng.next(cd, pd); - - if (clusteringOffset == 0) - return positionInPartition; - - if (positionInPartition == 0) - return partitionSize - clusteringOffset; - if (positionInPartition == clusteringOffset) - return 0; - else if (positionInPartition < clusteringOffset) - return partitionSize - clusteringOffset + positionInPartition; - else - return positionInPartition - clusteringOffset; - } - - public OperationKind operationType(long pd, long lts, long opId) - { - return operationType(pd, lts, opId, partitionLevelOperationsMask(pd, lts)); - } - - public BitSet partitionLevelOperationsMask(long pd, long lts) - { - int totalOps = operationsPerLts(lts); - if (totalOps > 64) - { - throw new IllegalArgumentException("RngUtils#randomBits currently supports only up to 64 bits of entropy, so we can not " + - "split partition and row level operations for more than 64 operations at the moment."); - } - - long seed = rng.randomNumber(pd, lts); - - int partitionLevelOps = (int) Math.ceil(operationSelector.partitionLevelThreshold * totalOps); - long partitionLevelOpsMask = RngUtils.randomBits(partitionLevelOps, totalOps, seed); - - return BitSet.create(partitionLevelOpsMask, totalOps); - } - - private OperationKind operationType(long pd, long lts, long opId, BitSet partitionLevelOperationsMask) - { - try - { - long descriptor = rng.randomNumber(pd ^ lts ^ opId, BITSET_IDX_STREAM); - return operationSelector.inflate(descriptor, partitionLevelOperationsMask.isSet((int) opId)); - } - catch (Throwable t) - { - throw new RuntimeException(String.format("Can not generate a random number with the following inputs: " + - "pd=%d lts=%d opId=%d partitionLevelOperationsMask=%s", - pd, lts, opId, partitionLevelOperationsMask)); - } - } - - public BitSet columnMask(long pd, long lts, long opId, OperationKind opType) - { - long descriptor = rng.randomNumber(pd ^ lts ^ opId, BITSET_IDX_STREAM); - return columnSelector.columnMask(opType, descriptor); - } - - public long vd(long pd, long cd, long lts, long opId, int col) - { - return rng.randomNumber(opId + 1, pd ^ cd ^ lts ^ col); - } - } - - enum OperationKind - { - UPDATE(false), - INSERT(false), - UPDATE_WITH_STATICS(true), - INSERT_WITH_STATICS(true), - DELETE_PARTITION(true), - DELETE_ROW(false), - DELETE_COLUMN(false), - DELETE_COLUMN_WITH_STATICS(true), - DELETE_RANGE(false), - DELETE_SLICE(false); - - public final boolean partititonLevel; - - OperationKind(boolean partitionLevel) - { - this.partititonLevel = partitionLevel; - } - - /** - * All operations apart from partiton delition, including partition hist are visible since if there is a - * partition liveness marker, partition's static column is going to survive. We use this method to match - * computed LTS with tracked ones. - */ - public boolean hasVisibleVisit() - { - return this != OpSelectors.OperationKind.DELETE_PARTITION; - } - } - - public static class OperationSelector - { - public final Surjections.Surjection<OperationKind> partitionLevelOperationSelector; - public final Surjections.Surjection<OperationKind> rowLevelOperationSelector; - // TODO: start using partitionLevelThreshold - public final double partitionLevelThreshold; - - public OperationSelector(Surjections.Surjection<OperationKind> partitionLevelOperationSelector, - Surjections.Surjection<OperationKind> rowLevelOperationSelector, - double partitionLevelThreshold) - { - this.partitionLevelOperationSelector = partitionLevelOperationSelector; - this.rowLevelOperationSelector = rowLevelOperationSelector; - this.partitionLevelThreshold = partitionLevelThreshold; - } - - public OperationKind inflate(long descriptor, boolean partitionLevel) - { - OperationKind operationKind = partitionLevel ? partitionLevelOperationSelector.inflate(descriptor) : rowLevelOperationSelector.inflate(descriptor); - assert operationKind.partititonLevel == partitionLevel : "Generated operation with an incorrect partition level. Check your generators."; - return operationKind; - } - - public static OperationSelector weighted(Map<OperationKind, Integer> weightsMap) - { - int[] weights = new int[weightsMap.size()]; - OperationKind[] operationKinds = new OperationKind[weightsMap.size()]; - int i = 0; - for (Map.Entry<OperationKind, Integer> entry : weightsMap.entrySet()) - { - weights[i] = entry.getValue(); - operationKinds[i] = entry.getKey(); - i++; - } - return weighted(Surjections.weights(weights), operationKinds); - } - - public static OperationSelector weighted(long[] weights, OperationKind... operationKinds) - { - assert weights.length == operationKinds.length; - - Map<OperationKind, Integer> partitionLevel = new EnumMap<OperationKind, Integer>(OperationKind.class); - Map<OperationKind, Integer> rowLevel = new EnumMap<OperationKind, Integer>(OperationKind.class); - - int partitionLevelSum = 0; - int rowLevelSum = 0; - for (int i = 0; i < weights.length; i++) - { - int v = (int) (weights[i] >> 32); - if (operationKinds[i].partititonLevel) - { - partitionLevel.put(operationKinds[i], v); - partitionLevelSum += v; - } - else - { - rowLevel.put(operationKinds[i], v); - rowLevelSum += v; - } - } - int sum = (partitionLevelSum + rowLevelSum); - - return new OperationSelector(Surjections.weighted(normalize(partitionLevel)), - Surjections.weighted(normalize(rowLevel)), - (partitionLevelSum * 1.0) / sum); - } - - public static Map<OperationKind, Integer> normalize(Map<OperationKind, Integer> weights) - { - Map<OperationKind, Integer> normalized = new EnumMap<OperationKind, Integer>(OperationKind.class); - int sum = 0; - for (Integer value : weights.values()) - sum += value; - - for (OperationKind kind : weights.keySet()) - { - double dbl = (sum * ((double) weights.get(kind)) / sum); - normalized.put(kind, (int) Math.round(dbl)); - } - - return normalized; - } - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/model/PartitionState.java b/test/harry/main/org/apache/cassandra/harry/model/PartitionState.java new file mode 100644 index 000000000000..c47f733db6f8 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/model/PartitionState.java @@ -0,0 +1,403 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.model; + +import org.apache.cassandra.harry.MagicConstants; +import org.apache.cassandra.harry.Relations; +import org.apache.cassandra.harry.gen.Bijections; +import org.apache.cassandra.harry.gen.ValueGenerators; +import org.apache.cassandra.harry.op.Operations; +import org.apache.cassandra.harry.util.BitSet; +import org.apache.cassandra.harry.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.*; +import java.util.function.IntFunction; + +public class PartitionState implements Iterable<PartitionState.RowState> +{ + public static long STATIC_CLUSTERING = MagicConstants.NIL_DESCR; + + private static final Logger logger = LoggerFactory.getLogger(PartitionState.class); + + public final long pd; + + final List<Long> visitedLts = new ArrayList<>(); + final List<Long> skippedLts = new ArrayList<>(); + + RowState staticRow; + NavigableMap<Long, RowState> rows; + + final ValueGenerators valueGenerators; + + public PartitionState(long pd, ValueGenerators valueGenerators) + { + this.pd = pd; + this.rows = new TreeMap<>(valueGenerators.ckGen().descriptorsComparator()); + this.valueGenerators = valueGenerators; + this.staticRow = new RowState(this, + STATIC_CLUSTERING, + arr(valueGenerators.staticColumnCount(), MagicConstants.NIL_DESCR), + arr(valueGenerators.staticColumnCount(), MagicConstants.NO_TIMESTAMP)); + } + + /** + * Returns a navigable map of rows + */ + public NavigableMap<Long, RowState> rows() + { + return rows; + } + + public void writeStatic(long[] sds, long lts) + { + staticRow = updateRowState(staticRow, valueGenerators::staticColumnGen, STATIC_CLUSTERING, sds, lts, false); + } + + public void writeRegular(long cd, long[] vds, long lts, boolean writePrimaryKeyLiveness) + { + rows.compute(cd, (cd_, current) -> updateRowState(current, valueGenerators::regularColumnGen, cd, vds, lts, writePrimaryKeyLiveness)); + } + + public void delete(Operations.DeleteRange delete, long lts) + { + // TODO: inefficient; need to search for lower/higher bounds + rows.entrySet().removeIf(e -> Relations.matchRange(valueGenerators.ckGen(), + valueGenerators::ckComparator, + valueGenerators.ckColumnCount(), + delete.lowerBound(), + delete.upperBound(), + delete.lowerBoundRelation(), + delete.upperBoundRelation(), + e.getValue().cd)); + } + + public void reverse() + { + rows = rows.descendingMap(); + } + + public void filter(Operations.SelectStatement select) + { + switch (select.kind) + { + case SELECT_PARTITION: + // selecting everything + break; + case SELECT_ROW: + filterInternal((Operations.SelectRow) select); + break; + case SELECT_RANGE: + filterInternal((Operations.SelectRange) select); + break; + case SELECT_CUSTOM: + filterInternal((Operations.SelectCustom) select); + break; + default: + throw new IllegalStateException("Filtering not implemented for " + select); + } + } + + private void filterInternal(Operations.SelectRow select) + { + // TODO: inefficient; need to search for lower/higher bounds + rows.entrySet().removeIf(e -> e.getValue().cd != select.cd()); + } + + private void filterInternal(Operations.SelectRange select) + { + // TODO: inefficient; need to search for lower/higher bounds + rows.entrySet().removeIf(e -> !Relations.matchRange(valueGenerators.ckGen(), + valueGenerators::ckComparator, + valueGenerators.ckColumnCount(), + select.lowerBound(), + select.upperBound(), + select.lowerBoundRelation(), + select.upperBoundRelation(), + e.getValue().cd)); + } + + private void filterInternal(Operations.SelectCustom select) + { + // TODO: inefficient; need to search for lower/higher bounds + rows.entrySet().removeIf(e -> { + Map<Long, Object[]> cache = new HashMap<>(); + for (Relations.Relation relation : select.ckRelations()) + { + Object[] query = cache.computeIfAbsent(relation.descriptor, valueGenerators.ckGen()::inflate); + Object[] match = cache.computeIfAbsent(e.getValue().cd, valueGenerators.ckGen()::inflate); + if (!relation.kind.match(valueGenerators.ckComparator(relation.column), match[relation.column], query[relation.column])) + return true; // true means "no match", so remove from resultset + } + + for (Relations.Relation relation : select.regularRelations()) + { + Object query = valueGenerators.regularColumnGen(relation.column).inflate(relation.descriptor); + long descriptor = e.getValue().vds[relation.column]; + if (MagicConstants.MAGIC_DESCRIPTOR_VALS.contains(descriptor)) // TODO: do we allow UNSET queries? + return true; + Object match = valueGenerators.regularColumnGen(relation.column).inflate(e.getValue().vds[relation.column]); + if (!relation.kind.match(valueGenerators.regularComparator(relation.column), match, query)) + return true; + } + + for (Relations.Relation relation : select.staticRelations()) + { + Object query = valueGenerators.staticColumnGen(relation.column).inflate(relation.descriptor); + long descriptor = e.getValue().partitionState.staticRow.vds[relation.column]; + if (MagicConstants.MAGIC_DESCRIPTOR_VALS.contains(descriptor)) // TODO: do we allow UNSET queries? + return true; + Object match = valueGenerators.staticColumnGen(relation.column).inflate(e.getValue().partitionState.staticRow.vds[relation.column]); + if (!relation.kind.match(valueGenerators.staticComparator(relation.column), match, query)) + return true; + } + + return false; + }); + } + + public void delete(long cd, long lts) + { + RowState state = rows.remove(cd); + if (state != null) + { + for (long v : state.lts) + assert lts >= v : String.format("Attempted to remove a row with a tombstone that has older timestamp (%d): %s", lts, state); + } + } + + public boolean isEmpty() + { + return rows.isEmpty(); + } + + /** + * Method used to update row state of both static and regular rows. + */ + private RowState updateRowState(RowState currentState, IntFunction<Bijections.Bijection<Object>> columns, long cd, long[] vds, long lts, boolean writePrimaryKeyLiveness) + { + if (currentState == null) + { + long[] ltss = new long[vds.length]; + long[] vdsCopy = new long[vds.length]; + for (int i = 0; i < vds.length; i++) + { + if (vds[i] != MagicConstants.UNSET_DESCR) + { + ltss[i] = lts; + vdsCopy[i] = vds[i]; + } + else + { + ltss[i] = MagicConstants.NO_TIMESTAMP; + vdsCopy[i] = MagicConstants.NIL_DESCR; + } + } + + currentState = new RowState(this, cd, vdsCopy, ltss); + } + else + { + assert currentState.vds.length == vds.length : String.format("Vds: %d, sds: %d", currentState.vds.length, vds.length); + for (int i = 0; i < vds.length; i++) + { + if (vds[i] == MagicConstants.UNSET_DESCR) + continue; + + assert lts >= currentState.lts[i] : String.format("Out-of-order LTS: %d. Max seen: %s", lts, currentState.lts[i]); // sanity check; we're iterating in lts order + + if (currentState.lts[i] == lts) + { + // Timestamp collision case + Bijections.Bijection<?> column = columns.apply(i); + if (column.compare(vds[i], currentState.vds[i]) > 0) + currentState.vds[i] = vds[i]; + } + else + { + currentState.vds[i] = vds[i]; + assert lts > currentState.lts[i]; + currentState.lts[i] = lts; + } + } + } + + if (writePrimaryKeyLiveness) + currentState.hasPrimaryKeyLivenessInfo = true; + + return currentState; + } + + public void deleteRegularColumns(long lts, long cd, BitSet columns) + { + deleteColumns(lts, rows.get(cd), columns); + } + + public void deleteStaticColumns(long lts, BitSet columns) + { + deleteColumns(lts, staticRow, columns); + } + + public void deleteColumns(long lts, RowState state, BitSet columns) + { + if (state == null) + return; + + //TODO: optimise by iterating over the columns that were removed by this deletion + //TODO: optimise final decision to fully remove the column by counting a number of set/unset columns + boolean allNil = true; + for (int i = 0; i < state.vds.length; i++) + { + if (columns.isSet(i)) + { + state.vds[i] = MagicConstants.NIL_DESCR; + state.lts[i] = MagicConstants.NO_TIMESTAMP; + } + else if (state.vds[i] != MagicConstants.NIL_DESCR) + { + allNil = false; + } + } + + if (state.cd != STATIC_CLUSTERING && allNil & !state.hasPrimaryKeyLivenessInfo) + delete(state.cd, lts); + } + + public void deletePartition(long lts) + { + rows.clear(); + Arrays.fill(staticRow.vds, MagicConstants.NIL_DESCR); + Arrays.fill(staticRow.lts, MagicConstants.NO_TIMESTAMP); + } + + public Iterator<RowState> iterator() + { + return iterator(false); + } + + public Iterator<RowState> iterator(boolean reverse) + { + if (reverse) + return rows.descendingMap().values().iterator(); + + return rows.values().iterator(); + } + + public Collection<RowState> rows(boolean reverse) + { + if (reverse) + return rows.descendingMap().values(); + + return rows.values(); + } + + public RowState staticRow() + { + return staticRow; + } + + public String toString() + { + // TODO: display inices not LTS when doing tostring + StringBuilder sb = new StringBuilder(); + + sb.append("Visited LTS: " + visitedLts).append("\n"); + sb.append("Skipped LTS: " + skippedLts).append("\n"); + + if (staticRow != null) + { + sb.append("Static row:\n" + staticRow.toString(valueGenerators)).append("\n"); + sb.append("\n"); + } + + for (RowState row : rows.values()) + sb.append(row.toString(valueGenerators)).append("\n"); + + return sb.toString(); + } + + public static class RowState + { + public boolean hasPrimaryKeyLivenessInfo = false; + + public final PartitionState partitionState; + public final long cd; + public final long[] vds; + public final long[] lts; + + public RowState(PartitionState partitionState, + long cd, + long[] vds, + long[] lts) + { + this.partitionState = partitionState; + this.cd = cd; + this.vds = vds; + this.lts = lts; + } + + public RowState clone() + { + RowState rowState = new RowState(partitionState, cd, Arrays.copyOf(vds, vds.length), Arrays.copyOf(lts, lts.length)); + rowState.hasPrimaryKeyLivenessInfo = hasPrimaryKeyLivenessInfo; + return rowState; + } + + private static String toString(long[] descriptors, IntFunction<Bijections.Bijection<Object>> gens) + { + String[] idxs = new String[descriptors.length]; + for (int i = 0; i < descriptors.length; i++) + idxs[i] = descrToIdxForToString(gens.apply(i), descriptors[i]); + return String.join(",", idxs); + } + + public static String descrToIdxForToString(Bijections.Bijection<?> gen, long descr) + { + return gen.toString(descr); + } + + public String toString(ValueGenerators valueGenerators) + { + if (cd == STATIC_CLUSTERING) + { + return " rowStateRow(" + + valueGenerators.pkGen().toString(partitionState.pd) + + ", STATIC" + + ", statics(" + toString(partitionState.staticRow.vds, valueGenerators::staticColumnGen) + ")" + + ", lts(" + StringUtils.toString(partitionState.staticRow.lts) + ")"; + } + else + { + return " rowStateRow(" + + valueGenerators.pkGen().toString(partitionState.pd) + + ", " + descrToIdxForToString(valueGenerators.ckGen(), cd) + + ", vds(" + toString(vds, valueGenerators::regularColumnGen) + ")" + + ", lts(" + StringUtils.toString(lts) + ")"; + } + } + } + + public static long[] arr(int length, long fill) + { + long[] arr = new long[length]; + Arrays.fill(arr, fill); + return arr; + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/model/PartitionStateBuilder.java b/test/harry/main/org/apache/cassandra/harry/model/PartitionStateBuilder.java new file mode 100644 index 000000000000..d67a9ed21517 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/model/PartitionStateBuilder.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.model; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.harry.MagicConstants; +import org.apache.cassandra.harry.gen.ValueGenerators; +import org.apache.cassandra.harry.execution.VisitExecutor; + +import static org.apache.cassandra.harry.op.Operations.*; +import static org.apache.cassandra.harry.op.Operations.Kind.INSERT; + +class PartitionStateBuilder extends VisitExecutor +{ + private static final Logger logger = LoggerFactory.getLogger(PartitionStateBuilder.class); + + private final PartitionState partitionState; + private boolean hadPartitionDeletion; + private boolean hadTrackingRowWrite; + private final List<Operation> rangeDeletes; + private final List<Operation> writes; + private final List<Operation> columnDeletes; + private final ValueGenerators valueGenerators; + + PartitionStateBuilder(ValueGenerators valueGenerators, + PartitionState partitionState) + { + this.valueGenerators = valueGenerators; + this.partitionState = partitionState; + hadPartitionDeletion = false; + hadTrackingRowWrite = false; + rangeDeletes = new ArrayList<>(); + writes = new ArrayList<>(); + columnDeletes = new ArrayList<>(); + } + + public PartitionState build() + { + return partitionState; + } + + protected void operation(Operation operation) + { + if (hadPartitionDeletion) + return; + + long lts = operation.lts(); + + switch (operation.kind()) + { + case DELETE_RANGE: + rangeDeletes.add(operation); + break; + case DELETE_ROW: + long cd = ((DeleteRow) operation).cd(); + partitionState.delete(cd, lts); + break; + case DELETE_PARTITION: + partitionState.deletePartition(lts); + rangeDeletes.clear(); + writes.clear(); + columnDeletes.clear(); + hadPartitionDeletion = true; + break; + case INSERT: + case UPDATE: + writes.add(operation); + break; + case DELETE_COLUMNS: + columnDeletes.add(operation); + break; + default: + throw new IllegalStateException("Unknown operation " + operation.kind()); + } + } + + @Override + protected void beginLts(long lts) + { + rangeDeletes.clear(); + writes.clear(); + columnDeletes.clear(); + hadPartitionDeletion = false; + } + + @Override + protected void endLts(long lts) + { + if (hadPartitionDeletion) + return; + + for (Operation op : writes) + { + WriteOp writeOp = (WriteOp) op; + long cd = writeOp.cd(); + + if (hadTrackingRowWrite) + { + long[] statics = new long[valueGenerators.staticColumnCount()]; + Arrays.fill(statics, MagicConstants.UNSET_DESCR); + partitionState.writeStatic(statics, lts); + } + + switch (op.kind()) + { + case INSERT: + case UPDATE: + WriteOp writeStaticOp = (WriteOp) op; + // We could apply static columns during the first iteration, but it's more convenient + // to reconcile static-level deletions. + partitionState.writeStatic(writeStaticOp.sds(), lts); + partitionState.writeRegular(cd, writeOp.vds(), lts, op.kind() == INSERT); + break; + default: + throw new IllegalStateException(op.kind().toString()); + } + } + + for (Operation op : columnDeletes) + { + DeleteColumnsOp deleteColumnsOp = (DeleteColumnsOp) op; + partitionState.deleteStaticColumns(lts, deleteColumnsOp.staticColumns()); + partitionState.deleteRegularColumns(lts, deleteColumnsOp.cd(), deleteColumnsOp.staticColumns()); + } + + for (Operation rangeDelete : rangeDeletes) + { + partitionState.delete((DeleteRange) rangeDelete, lts); + } + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/model/QueryingNoOpValidator.java b/test/harry/main/org/apache/cassandra/harry/model/QueryingNoOpValidator.java deleted file mode 100644 index 723b8e7e44be..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/model/QueryingNoOpValidator.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.model; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonTypeName; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.operations.CompiledStatement; -import org.apache.cassandra.harry.operations.Query; - -/** - * A model that can be used to "simply" run random queries. Does not perform validation - * of the results that it sees. Useful for increasing concurrency and triggering - * exceptions rather than data loss issues. - */ -public class QueryingNoOpValidator implements Model -{ - private final Run run; - - public QueryingNoOpValidator(Run run) - { - this.run = run; - } - - @Override - public void validate(Query query) - { - CompiledStatement compiled = query.toSelectStatement(); - run.sut.execute(compiled.cql(), - SystemUnderTest.ConsistencyLevel.QUORUM, - compiled.bindings()); - } - - @JsonTypeName("no_op") - public static class QueryingNoOpCheckerConfig implements Configuration.ModelConfiguration - { - @JsonCreator - public QueryingNoOpCheckerConfig() - { - } - - public Model make(Run run) - { - return new QueryingNoOpValidator(run); - } - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/model/QuiescentChecker.java b/test/harry/main/org/apache/cassandra/harry/model/QuiescentChecker.java index 64af6520d5e6..a77904202c68 100644 --- a/test/harry/main/org/apache/cassandra/harry/model/QuiescentChecker.java +++ b/test/harry/main/org/apache/cassandra/harry/model/QuiescentChecker.java @@ -1,232 +1,215 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.cassandra.harry.model; -import java.util.*; -import java.util.function.Supplier; - -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.data.ResultSetRow; -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.model.reconciler.PartitionState; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.model.reconciler.Reconciler; -import org.apache.cassandra.harry.tracker.DataTracker; - -import static org.apache.cassandra.harry.gen.DataGenerators.NIL_DESCR; -import static org.apache.cassandra.harry.gen.DataGenerators.UNSET_DESCR; - +import java.util.Arrays; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.NavigableMap; + +import accord.utils.Invariants; +import org.apache.cassandra.harry.execution.DataTracker; +import org.apache.cassandra.harry.op.Operations; +import org.apache.cassandra.harry.execution.ResultSetRow; +import org.apache.cassandra.harry.gen.ValueGenerators; + +import static org.apache.cassandra.harry.MagicConstants.LTS_UNKNOWN; +import static org.apache.cassandra.harry.MagicConstants.NIL_DESCR; +import static org.apache.cassandra.harry.MagicConstants.NO_TIMESTAMP; +import static org.apache.cassandra.harry.MagicConstants.UNKNOWN_DESCR; +import static org.apache.cassandra.harry.MagicConstants.UNSET_DESCR; + +/** + * Unfortunately model is not reducible to a simple interface of apply/validate, at least not if + * we intend to support concurrent validation and in-flight/timed out queries. Validation needs to + * know which queries might have been applied. + * + * + */ public class QuiescentChecker implements Model { - protected final OpSelectors.Clock clock; - - protected final DataTracker tracker; - protected final SystemUnderTest sut; - protected final Reconciler reconciler; - protected final SchemaSpec schema; - - public QuiescentChecker(Run run) - { - this(run, new Reconciler(run)); - } + private final DataTracker tracker; + private final Replay replay; + private final ValueGenerators valueGenerators; - public QuiescentChecker(Run run, Reconciler reconciler) + public QuiescentChecker(ValueGenerators valueGenerators, DataTracker tracker, Replay replay) { - this(run.clock, run.sut, run.tracker, run.schemaSpec, reconciler); - } - - public QuiescentChecker(OpSelectors.Clock clock, - SystemUnderTest sut, - DataTracker tracker, - SchemaSpec schema, - Reconciler reconciler) - { - this.clock = clock; - this.sut = sut; - this.reconciler = reconciler; + this.valueGenerators = valueGenerators; this.tracker = tracker; - this.schema = schema; + this.replay = replay; } - public void validate(Query query) + @Override + public void validate(Operations.SelectStatement select, List<ResultSetRow> actualRows) { - tracker.beginValidation(query.pd); - validate(() -> SelectHelper.execute(sut, clock, query), query); - tracker.endValidation(query.pd); - } + PartitionState partitionState = new PartitionState(select.pd, valueGenerators); + PartitionStateBuilder stateBuilder = new PartitionStateBuilder(valueGenerators, partitionState); - protected void validate(Supplier<List<ResultSetRow>> rowsSupplier, Query query) - { - List<ResultSetRow> actualRows = rowsSupplier.get(); - PartitionState partitionState = reconciler.inflatePartitionState(query.pd, tracker, query); - validate(schema, tracker, partitionState, actualRows, query); - } - - public static void validate(SchemaSpec schema, DataTracker tracker, PartitionState partitionState, List<ResultSetRow> actualRows, Query query) - { - Set<ColumnSpec<?>> columns = new HashSet<>(); - columns.addAll(schema.allColumns); - validate(schema, tracker, columns, partitionState, actualRows, query); - } - - public static Reconciler.RowState adjustForSelection(Reconciler.RowState row, SchemaSpec schema, Set<ColumnSpec<?>> selection, boolean isStatic) - { - if (selection.size() == schema.allColumns.size()) - return row; - - List<ColumnSpec<?>> columns = isStatic ? schema.staticColumns : schema.regularColumns; - Reconciler.RowState newRowState = row.clone(); - assert newRowState.vds.length == columns.size(); - for (int i = 0; i < columns.size(); i++) + long prevLts = -1; + for (LtsOperationPair potentialVisit : tracker.potentialVisits(select.pd())) { - if (!selection.contains(columns.get(i))) + if (tracker.isFinished(potentialVisit.lts)) { - newRowState.vds[i] = UNSET_DESCR; - newRowState.lts[i] = NO_TIMESTAMP; + if (potentialVisit.lts != prevLts) + { + if (prevLts != -1) + stateBuilder.endLts(prevLts); + stateBuilder.beginLts(potentialVisit.lts); + prevLts = potentialVisit.lts; + } + Operations.Operation op = replay.replay(potentialVisit.lts, potentialVisit.opId); + stateBuilder.operation(op); } } - return newRowState; + + // Close last open LTS + if (prevLts != -1) + stateBuilder.endLts(prevLts); + + partitionState.filter(select); + if (select.orderBy() == Operations.ClusteringOrderBy.DESC) + { + partitionState.reverse(); + } + + validate(valueGenerators, partitionState, actualRows); } - public static void validate(SchemaSpec schema, DataTracker tracker, Set<ColumnSpec<?>> selection, PartitionState partitionState, List<ResultSetRow> actualRows, Query query) + // TODO: reverse + public static void validate(ValueGenerators valueGenerators, PartitionState partitionState, List<ResultSetRow> actualRows) { - boolean isWildcardQuery = selection == null; - String trackerBefore = tracker.toString(); - if (isWildcardQuery) - selection = new HashSet<>(schema.allColumns); - Iterator<ResultSetRow> actual = actualRows.iterator(); - Collection<Reconciler.RowState> expectedRows = partitionState.rows(query.reverse); + NavigableMap<Long, PartitionState.RowState> expectedRows = partitionState.rows(); - Iterator<Reconciler.RowState> expected = expectedRows.iterator(); - - String trackerState = String.format("Tracker before: %s, Tracker after: %s", trackerBefore, tracker); + Iterator<PartitionState.RowState> expected = expectedRows.values().iterator(); // It is possible that we only get a single row in response, and it is equal to static row if (partitionState.isEmpty() && partitionState.staticRow() != null && actual.hasNext()) { ResultSetRow actualRowState = actual.next(); + // TODO: it is possible to start distinguising between unknown and null values if (actualRowState.cd != UNSET_DESCR && actualRowState.cd != partitionState.staticRow().cd) { - throw new ValidationException(trackerState, - partitionState.toString(schema), - toString(actualRows), + throw new ValidationException(partitionState.toString(), + toString(valueGenerators, actualRows), "Found a row while model predicts statics only:" + "\nExpected: %s" + - "\nActual: %s" + - "\nQuery: %s", + "\nActual: %s", partitionState.staticRow(), - actualRowState, - query.toSelectStatement()); + actualRowState); } for (int i = 0; i < actualRowState.vds.length; i++) { - if (actualRowState.vds[i] != NIL_DESCR || actualRowState.lts[i] != NO_TIMESTAMP) - throw new ValidationException(trackerState, - partitionState.toString(schema), - toString(actualRows), + // If clustering is unset, all values should be equal to NIL: we have received a row with statics only. + if (actualRowState.vds[i] != NIL_DESCR || (actualRowState.lts != LTS_UNKNOWN && actualRowState.lts[i] != NO_TIMESTAMP)) + throw new ValidationException(partitionState.toString(), + toString(valueGenerators, actualRows), "Found a row while model predicts statics only:" + - "\nActual: %s" + - "\nQuery: %s", - actualRowState, query.toSelectStatement()); + "\nActual: %s", + actualRowState); } - assertStaticRow(partitionState, actualRows, - adjustForSelection(partitionState.staticRow(), schema, selection, true), - actualRowState, query, trackerState, schema); + assertStaticRow(partitionState, + actualRows, + partitionState.staticRow(), + actualRowState, + valueGenerators); } while (actual.hasNext() && expected.hasNext()) { ResultSetRow actualRowState = actual.next(); - Reconciler.RowState originalExpectedRowState = expected.next(); - Reconciler.RowState expectedRowState = adjustForSelection(originalExpectedRowState, schema, selection, false); - - if (schema.trackLts) - partitionState.compareVisitedLts(actualRowState.visited_lts); + PartitionState.RowState expectedRowState = expected.next(); // TODO: this is not necessarily true. It can also be that ordering is incorrect. - if (actualRowState.cd != UNSET_DESCR && actualRowState.cd != expectedRowState.cd) + if (actualRowState.cd != UNKNOWN_DESCR && actualRowState.cd != expectedRowState.cd) { - throw new ValidationException(trackerState, - partitionState.toString(schema), - toString(actualRows), + throw new ValidationException(partitionState.toString(), + toString(valueGenerators, actualRows), "Found a row in the model that is not present in the resultset:" + "\nExpected: %s" + - "\nActual: %s" + - "\nQuery: %s", - expectedRowState.toString(schema), - actualRowState, query.toSelectStatement()); + "\nActual: %s", + expectedRowState.toString(valueGenerators), + actualRowState); } - if (!Arrays.equals(expectedRowState.vds, actualRowState.vds)) - throw new ValidationException(trackerState, - partitionState.toString(schema), - toString(actualRows), + if (!vdsEqual(expectedRowState.vds, actualRowState.vds)) + throw new ValidationException(partitionState.toString(), + toString(valueGenerators, actualRows), "Returned row state doesn't match the one predicted by the model:" + - "\nExpected: %s (%s)" + - "\nActual: %s (%s)." + - "\nQuery: %s", - descriptorsToString(expectedRowState.vds), expectedRowState.toString(schema), - descriptorsToString(actualRowState.vds), actualRowState, - query.toSelectStatement()); + "\nExpected: %s" + + "\nActual: %s.", + expectedRowState.toString(valueGenerators), + actualRowState.toString(valueGenerators)); if (!ltsEqual(expectedRowState.lts, actualRowState.lts)) - throw new ValidationException(trackerState, - partitionState.toString(schema), - toString(actualRows), + throw new ValidationException(partitionState.toString(), + toString(valueGenerators, actualRows), "Timestamps in the row state don't match ones predicted by the model:" + - "\nExpected: %s (%s)" + - "\nActual: %s (%s)." + - "\nQuery: %s", - Arrays.toString(expectedRowState.lts), expectedRowState.toString(schema), - Arrays.toString(actualRowState.lts), actualRowState, - query.toSelectStatement()); + "\nExpected: %s" + + "\nActual: %s.", + expectedRowState.toString(valueGenerators), + actualRowState.toString(valueGenerators)); if (partitionState.staticRow() != null || actualRowState.hasStaticColumns()) { - Reconciler.RowState expectedStaticRowState = adjustForSelection(partitionState.staticRow(), schema, selection, true); - assertStaticRow(partitionState, actualRows, expectedStaticRowState, actualRowState, query, trackerState, schema); + PartitionState.RowState expectedStaticRowState = partitionState.staticRow(); + assertStaticRow(partitionState, actualRows, expectedStaticRowState, actualRowState, valueGenerators); } } if (actual.hasNext() || expected.hasNext()) { - throw new ValidationException(trackerState, - partitionState.toString(schema), - toString(actualRows), + throw new ValidationException(partitionState.toString(), + toString(valueGenerators, actualRows), "Expected results to have the same number of results, but %s result iterator has more results." + "\nExpected: %s" + - "\nActual: %s" + - "\nQuery: %s", + "\nActual: %s", actual.hasNext() ? "actual" : "expected", expectedRows, - actualRows, - query.toSelectStatement()); + actualRows); } } + public static boolean vdsEqual(long[] expected, long[] actual) + { + Invariants.checkState(expected.length == actual.length); + for (int i = 0; i < actual.length; i++) + { + long expectedD = expected[i]; + long actualD = actual[i]; + // An unset column will show as NIL + if (expectedD == UNSET_DESCR && actualD == NIL_DESCR) + continue; + if (actualD != expectedD) + return false; + } + return true; + } + public static boolean ltsEqual(long[] expected, long[] actual) { + if (actual == LTS_UNKNOWN) + return true; + if (actual == expected) return true; if (actual == null || expected == null) @@ -248,35 +231,27 @@ public static boolean ltsEqual(long[] expected, long[] actual) public static void assertStaticRow(PartitionState partitionState, List<ResultSetRow> actualRows, - Reconciler.RowState staticRow, + PartitionState.RowState staticRow, ResultSetRow actualRowState, - Query query, - String trackerState, - SchemaSpec schemaSpec) + ValueGenerators valueGenerators) { - if (!Arrays.equals(staticRow.vds, actualRowState.sds)) - throw new ValidationException(trackerState, - partitionState.toString(schemaSpec), - toString(actualRows), + if (!vdsEqual(staticRow.vds, actualRowState.sds)) + throw new ValidationException(partitionState.toString(), + toString(valueGenerators, actualRows), "Returned static row state doesn't match the one predicted by the model:" + "\nExpected: %s (%s)" + - "\nActual: %s (%s)." + - "\nQuery: %s", - descriptorsToString(staticRow.vds), staticRow.toString(schemaSpec), - descriptorsToString(actualRowState.sds), actualRowState, - query.toSelectStatement()); + "\nActual: %s (%s).", + descriptorsToString(staticRow.vds), staticRow.toString(valueGenerators), + descriptorsToString(actualRowState.sds), actualRowState); if (!ltsEqual(staticRow.lts, actualRowState.slts)) - throw new ValidationException(trackerState, - partitionState.toString(schemaSpec), - toString(actualRows), + throw new ValidationException(partitionState.toString(), + toString(valueGenerators, actualRows), "Timestamps in the static row state don't match ones predicted by the model:" + "\nExpected: %s (%s)" + - "\nActual: %s (%s)." + - "\nQuery: %s", - Arrays.toString(staticRow.lts), staticRow.toString(schemaSpec), - Arrays.toString(actualRowState.slts), actualRowState, - query.toSelectStatement()); + "\nActual: %s (%s).", + Arrays.toString(staticRow.lts), staticRow.toString(valueGenerators), + Arrays.toString(actualRowState.slts), actualRowState); } public static String descriptorsToString(long[] descriptors) @@ -284,33 +259,44 @@ public static String descriptorsToString(long[] descriptors) StringBuilder sb = new StringBuilder(); for (int i = 0; i < descriptors.length; i++) { + if (i > 0) + sb.append(", "); if (descriptors[i] == NIL_DESCR) sb.append("NIL"); - if (descriptors[i] == UNSET_DESCR) + else if (descriptors[i] == UNSET_DESCR) sb.append("UNSET"); else sb.append(descriptors[i]); - if (i > 0) - sb.append(", "); } return sb.toString(); } - public static String toString(Collection<Reconciler.RowState> collection, SchemaSpec schema) + public static String toString(Collection<PartitionState.RowState> collection, ValueGenerators valueGenerators) { StringBuilder builder = new StringBuilder(); - for (Reconciler.RowState rowState : collection) - builder.append(rowState.toString(schema)).append("\n"); + for (PartitionState.RowState rowState : collection) + builder.append(rowState.toString(valueGenerators)).append("\n"); return builder.toString(); } - public static String toString(List<ResultSetRow> collection) + public static String toString(ValueGenerators valueGenerators, List<ResultSetRow> collection) { StringBuilder builder = new StringBuilder(); for (ResultSetRow rowState : collection) - builder.append(rowState.toString()).append("\n"); + builder.append(rowState.toString(valueGenerators)).append("\n"); return builder.toString(); } -} + + + public static class ValidationException extends RuntimeException + { + public ValidationException(String partitionState, String observedState, String format, Object... objects) + { + super(String.format(format, objects) + + "\nPartition state:\n" + partitionState + + "\nObserved state:\n" + observedState); + } + } +} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/model/QuiescentLocalStateCheckerBase.java b/test/harry/main/org/apache/cassandra/harry/model/QuiescentLocalStateCheckerBase.java deleted file mode 100644 index 54ea8638b961..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/model/QuiescentLocalStateCheckerBase.java +++ /dev/null @@ -1,124 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.model; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.cassandra.harry.sut.TokenPlacementModel.Replica; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.data.ResultSetRow; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.reconciler.Reconciler; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.operations.CompiledStatement; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.tracker.DataTracker; - -import static org.apache.cassandra.harry.model.SelectHelper.resultSetToRow; -import static org.apache.cassandra.harry.sut.TokenPlacementModel.Node; - -public abstract class QuiescentLocalStateCheckerBase extends QuiescentChecker -{ - private static final Logger logger = LoggerFactory.getLogger(QuiescentLocalStateCheckerBase.class); - - public final SystemUnderTest sut; - public final TokenPlacementModel.ReplicationFactor rf; - private final OpSelectors.PdSelector pdSelector; - - public QuiescentLocalStateCheckerBase(Run run, TokenPlacementModel.ReplicationFactor rf) - { - this(run.clock, - run.pdSelector, - run.sut, - run.tracker, - run.schemaSpec, - new Reconciler(run), - rf); - } - - public QuiescentLocalStateCheckerBase(OpSelectors.Clock clock, - OpSelectors.PdSelector pdSelector, - SystemUnderTest sut, - DataTracker tracker, - SchemaSpec schema, - Reconciler reconciler, - TokenPlacementModel.ReplicationFactor rf) - { - super(clock, sut, tracker, schema, reconciler); - this.sut = sut; - this.rf = rf; - this.pdSelector = pdSelector; - } - - @SuppressWarnings("unused") - public void validateAll() - { - TokenPlacementModel.ReplicatedRanges ring = getRing(); - - for (int lts = 0; lts < clock.peek(); lts++) - validate(Query.selectAllColumns(schema, pdSelector.pd(lts, schema), false), ring); - } - - @Override - public void validate(Query query) - { - TokenPlacementModel.ReplicatedRanges ring = getRing(); - tracker.beginValidation(query.pd); - validate(query, ring); - tracker.endValidation(query.pd); - } - - protected void validate(Query query, TokenPlacementModel.ReplicatedRanges ring) - { - CompiledStatement compiled = query.toSelectStatement(); - List<Replica> replicas = ring.replicasFor(token(query.pd)); - - logger.trace("Predicted {} as replicas for {}. Ring: {}", replicas, query.pd, ring); - for (Replica replica : replicas) - { - try - { - validate(() -> { - Object[][] objects = executeNodeLocal(compiled.cql(), replica.node(), compiled.bindings()); - - List<ResultSetRow> result = new ArrayList<>(); - for (Object[] obj : objects) - result.add(resultSetToRow(query.schemaSpec, clock, obj)); - - return result; - }, query); - } - catch (ValidationException e) - { - throw new AssertionError(String.format("Caught error while validating replica %s of replica set %s", - replica, replicas), - e); - } - } - } - - protected abstract TokenPlacementModel.ReplicatedRanges getRing(); - protected abstract long token(long pd); - protected abstract Object[][] executeNodeLocal(String statement, Node node, Object... bindings); -} diff --git a/test/harry/main/org/apache/cassandra/harry/model/Reconciler.java b/test/harry/main/org/apache/cassandra/harry/model/Reconciler.java new file mode 100644 index 000000000000..c0956d51c20b --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/model/Reconciler.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.model; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +import org.apache.cassandra.harry.gen.ValueGenerators; +import org.apache.cassandra.harry.op.Visit; +import org.apache.cassandra.harry.op.Operations; + +import static org.apache.cassandra.harry.op.Operations.Operation; + +// TODO: unused at the moment +public class Reconciler +{ + private final ValueGenerators valueGenerators; + + public Reconciler(ValueGenerators valueGenerators) + { + this.valueGenerators = valueGenerators; + } + + public Map<Long, PartitionState> inflate(Iterator<Visit> iter) + { + Map<Long, PartitionStateBuilder> state = new HashMap<>(); + apply(state, iter); + + Map<Long, PartitionState> result = new HashMap<>(); + for (Map.Entry<Long, PartitionStateBuilder> e : state.entrySet()) + result.put(e.getKey(), e.getValue().build()); + return result; + } + + public void apply(Map<Long, PartitionStateBuilder> state, Iterator<Visit> iter) + { + assert iter.hasNext(); + + while (iter.hasNext()) + { + Visit visit = iter.next(); + for (Long pd : visit.visitedPartitions) + state.computeIfAbsent(pd, pd_ -> new PartitionStateBuilder(valueGenerators, new PartitionState(pd, valueGenerators))) + .beginLts(visit.lts); + + + for (Operation operation : visit.operations) + { + if (operation instanceof Operations.PartitionOperation) + state.get(((Operations.PartitionOperation) operation).pd()).operation(operation); + } + + for (Long pd : visit.visitedPartitions) + state.computeIfAbsent(pd, pd_ -> new PartitionStateBuilder(valueGenerators, new PartitionState(pd, valueGenerators))) + .endLts(visit.lts); + } + } +} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/model/RepairingLocalStateValidator.java b/test/harry/main/org/apache/cassandra/harry/model/RepairingLocalStateValidator.java deleted file mode 100644 index 2f310db93f25..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/model/RepairingLocalStateValidator.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.model; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.sut.injvm.InJvmSut; -import org.apache.cassandra.harry.visitors.AllPartitionsValidator; -import org.apache.cassandra.harry.visitors.QueryLogger; -import org.apache.cassandra.harry.visitors.Visitor; - -/** - * Validator similar to {@link AllPartitionsValidator}, but performs - * repair before checking node states. - * - * Can be useful for testing repair, bootstrap, and streaming code. - */ -public class RepairingLocalStateValidator extends AllPartitionsValidator -{ - private static final Logger logger = LoggerFactory.getLogger(RepairingLocalStateValidator.class); - - private final InJvmSut inJvmSut; - - public static Configuration.VisitorConfiguration factoryForTests(int concurrency, Model.ModelFactory modelFactory) - { - return (r) -> new RepairingLocalStateValidator(r, concurrency, modelFactory); - } - - public RepairingLocalStateValidator(Run run, int concurrency, Model.ModelFactory modelFactory) - { - this(run, concurrency, modelFactory, QueryLogger.NO_OP); - } - - public RepairingLocalStateValidator(Run run, int concurrency, Model.ModelFactory modelFactory, QueryLogger queryLogger) - { - super(run, concurrency, modelFactory, queryLogger); - this.inJvmSut = (InJvmSut) run.sut; - } - - public void visit() - { - logger.debug("Starting repair..."); - inJvmSut.cluster().stream().forEach((instance) -> instance.nodetool("repair", "--full")); - logger.debug("Validating partitions..."); - super.visit(); - } - - @JsonTypeName("repair_and_validate_local_states") - public static class RepairingLocalStateValidatorConfiguration implements Configuration.VisitorConfiguration - { - private final int concurrency; - private final Configuration.ModelConfiguration modelConfiguration; - private final Configuration.QueryLoggerConfiguration query_logger; - - @JsonCreator - public RepairingLocalStateValidatorConfiguration(@JsonProperty("concurrency") int concurrency, - @JsonProperty("model") Configuration.ModelConfiguration model, - @JsonProperty("query_logger") Configuration.QueryLoggerConfiguration query_logger) - { - this.concurrency = concurrency; - this.modelConfiguration = model; - this.query_logger = QueryLogger.thisOrDefault(query_logger); - } - - public Visitor make(Run run) - { - return new RepairingLocalStateValidator(run, concurrency, modelConfiguration, query_logger.make()); - } - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/model/SelectHelper.java b/test/harry/main/org/apache/cassandra/harry/model/SelectHelper.java deleted file mode 100644 index b2888afa1983..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/model/SelectHelper.java +++ /dev/null @@ -1,348 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.model; - -import java.util.*; - -import org.apache.cassandra.harry.data.ResultSetRow; -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.gen.DataGenerators; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.operations.CompiledStatement; -import org.apache.cassandra.harry.operations.Relation; -import org.apache.cassandra.harry.operations.Query; - -import static org.apache.cassandra.harry.gen.DataGenerators.UNSET_DESCR; - -public class SelectHelper -{ - private static final long[] EMPTY_ARR = new long[]{}; - public static CompiledStatement selectWildcard(SchemaSpec schema, long pd) - { - return select(schema, pd, null, Collections.emptyList(), false, true); - } - - public static CompiledStatement select(SchemaSpec schema, long pd) - { - return select(schema, pd, schema.allColumnsSet, Collections.emptyList(), false, true); - } - - /** - * Here, {@code reverse} should be understood not in ASC/DESC sense, but rather in terms - * of how we're going to iterate through this partition (in other words, if first clustering component order - * is DESC, we'll iterate in ASC order) - */ - public static CompiledStatement select(SchemaSpec schema, long pd, List<Relation> relations, boolean reverse, boolean includeWriteTime) - { - return select(schema, pd, schema.allColumnsSet, relations, reverse, includeWriteTime); - } - - public static CompiledStatement selectWildcard(SchemaSpec schema, long pd, List<Relation> relations, boolean reverse, boolean includeWriteTime) - { - return select(schema, pd, null, relations, reverse, includeWriteTime); - } - - public static CompiledStatement select(SchemaSpec schema, Long pd, Set<ColumnSpec<?>> columns, List<Relation> relations, boolean reverse, boolean includeWriteTime) - { - boolean isWildcardQuery = columns == null; - if (isWildcardQuery) - { - columns = schema.allColumnsSet; - includeWriteTime = false; - } - - StringBuilder b = new StringBuilder(); - b.append("SELECT "); - - boolean isFirst = true; - if (isWildcardQuery) - { - b.append("*"); - } - else - { - for (int i = 0; i < schema.allColumns.size(); i++) - { - ColumnSpec<?> spec = schema.allColumns.get(i); - if (columns != null && !columns.contains(spec)) - continue; - - if (isFirst) - isFirst = false; - else - b.append(", "); - b.append(spec.name); - } - } - - if (includeWriteTime) - { - for (ColumnSpec<?> spec : schema.staticColumns) - { - if (columns != null && !columns.contains(spec)) - continue; - b.append(", ") - .append("writetime(") - .append(spec.name) - .append(")"); - } - - for (ColumnSpec<?> spec : schema.regularColumns) - { - if (columns != null && !columns.contains(spec)) - continue; - b.append(", ") - .append("writetime(") - .append(spec.name) - .append(")"); - } - } - - if (schema.trackLts) - b.append(", visited_lts"); - - b.append(" FROM ") - .append(schema.keyspace) - .append(".") - .append(schema.table) - .append(" WHERE "); - - List<Object> bindings = new ArrayList<>(); - - SchemaSpec.AddRelationCallback consumer = new SchemaSpec.AddRelationCallback() - { - boolean isFirst = true; - public void accept(ColumnSpec<?> spec, Relation.RelationKind kind, Object value) - { - if (isFirst) - isFirst = false; - else - b.append(" AND "); - b.append(kind.getClause(spec)); - bindings.add(value); - } - }; - if (pd != null) - { - Object[] pk = schema.inflatePartitionKey(pd); - for (int i = 0; i < pk.length; i++) - consumer.accept(schema.partitionKeys.get(i), Relation.RelationKind.EQ, pk[i]); - - } - schema.inflateRelations(relations, consumer); - - addOrderBy(schema, b, reverse); - b.append(";"); - Object[] bindingsArr = bindings.toArray(new Object[bindings.size()]); - return new CompiledStatement(b.toString(), bindingsArr); - } - - public static CompiledStatement count(SchemaSpec schema, long pd) - { - StringBuilder b = new StringBuilder(); - b.append("SELECT count(*) "); - - b.append(" FROM ") - .append(schema.keyspace) - .append(".") - .append(schema.table) - .append(" WHERE "); - - List<Object> bindings = new ArrayList<>(schema.partitionKeys.size()); - - schema.inflateRelations(pd, - Collections.emptyList(), - new SchemaSpec.AddRelationCallback() - { - boolean isFirst = true; - public void accept(ColumnSpec<?> spec, Relation.RelationKind kind, Object value) - { - if (isFirst) - isFirst = false; - else - b.append(" AND "); - b.append(kind.getClause(spec)); - bindings.add(value); - } - }); - - Object[] bindingsArr = bindings.toArray(new Object[bindings.size()]); - return new CompiledStatement(b.toString(), bindingsArr); - } - - private static void addOrderBy(SchemaSpec schema, StringBuilder b, boolean reverse) - { - if (reverse && schema.clusteringKeys.size() > 0) - { - b.append(" ORDER BY "); - for (int i = 0; i < schema.clusteringKeys.size(); i++) - { - ColumnSpec<?> c = schema.clusteringKeys.get(i); - if (i > 0) - b.append(", "); - b.append(c.isReversed() ? asc(c.name) : desc(c.name)); - } - } - } - - public static String asc(String name) - { - return name + " ASC"; - } - - public static String desc(String name) - { - return name + " DESC"; - } - - - public static Object[] broadenResult(SchemaSpec schemaSpec, Set<ColumnSpec<?>> columns, Object[] result) - { - boolean isWildcardQuery = columns == null; - - if (isWildcardQuery) - columns = schemaSpec.allColumnsSet; - else if (schemaSpec.allColumns.size() == columns.size()) - return result; - - Object[] newRes = new Object[schemaSpec.allColumns.size() + schemaSpec.staticColumns.size() + schemaSpec.regularColumns.size()]; - - int origPointer = 0; - int newPointer = 0; - for (int i = 0; i < schemaSpec.allColumns.size(); i++) - { - ColumnSpec<?> column = schemaSpec.allColumns.get(i); - if (columns.contains(column)) - newRes[newPointer] = result[origPointer++]; - else - newRes[newPointer] = DataGenerators.UNSET_VALUE; - newPointer++; - } - - // Make sure to include writetime, but only in case query actually includes writetime (for example, it's not a wildcard query) - for (int i = 0; i < schemaSpec.staticColumns.size() && origPointer < result.length; i++) - { - ColumnSpec<?> column = schemaSpec.staticColumns.get(i); - if (columns.contains(column)) - newRes[newPointer] = result[origPointer++]; - else - newRes[newPointer] = null; - newPointer++; - } - - for (int i = 0; i < schemaSpec.regularColumns.size() && origPointer < result.length; i++) - { - ColumnSpec<?> column = schemaSpec.regularColumns.get(i); - if (columns.contains(column)) - newRes[newPointer] = result[origPointer++]; - else - newRes[newPointer] = null; - newPointer++; - } - - return newRes; - } - - static boolean isDeflatable(Object[] columns) - { - for (Object column : columns) - { - if (column == DataGenerators.UNSET_VALUE) - return false; - } - return true; - } - - public static ResultSetRow resultSetToRow(SchemaSpec schema, OpSelectors.Clock clock, Object[] result) - { - Object[] partitionKey = new Object[schema.partitionKeys.size()]; - Object[] clusteringKey = new Object[schema.clusteringKeys.size()]; - Object[] staticColumns = new Object[schema.staticColumns.size()]; - Object[] regularColumns = new Object[schema.regularColumns.size()]; - - System.arraycopy(result, 0, partitionKey, 0, partitionKey.length); - System.arraycopy(result, partitionKey.length, clusteringKey, 0, clusteringKey.length); - System.arraycopy(result, partitionKey.length + clusteringKey.length, staticColumns, 0, staticColumns.length); - System.arraycopy(result, partitionKey.length + clusteringKey.length + staticColumns.length, regularColumns, 0, regularColumns.length); - - - List<Long> visited_lts_list; - if (schema.trackLts) - { - visited_lts_list = (List<Long>) result[result.length - 1]; - visited_lts_list.sort(Long::compare); - } - else - { - visited_lts_list = Collections.emptyList(); - } - - long[] slts = new long[schema.staticColumns.size()]; - Arrays.fill(slts, Model.NO_TIMESTAMP); - for (int i = 0, sltsBase = schema.allColumns.size(); i < slts.length && sltsBase + i < result.length; i++) - { - Object v = result[schema.allColumns.size() + i]; - if (v != null) - slts[i] = clock.lts((long) v); - } - - long[] lts = new long[schema.regularColumns.size()]; - Arrays.fill(lts, Model.NO_TIMESTAMP); - for (int i = 0, ltsBase = schema.allColumns.size() + slts.length; i < lts.length && ltsBase + i < result.length; i++) - { - Object v = result[ltsBase + i]; - if (v != null) - lts[i] = clock.lts((long) v); - } - - return new ResultSetRow(isDeflatable(partitionKey) ? schema.deflatePartitionKey(partitionKey) : UNSET_DESCR, - isDeflatable(clusteringKey) ? schema.deflateClusteringKey(clusteringKey) : UNSET_DESCR, - schema.staticColumns.isEmpty() ? EMPTY_ARR : schema.deflateStaticColumns(staticColumns), - schema.staticColumns.isEmpty() ? EMPTY_ARR : slts, - schema.deflateRegularColumns(regularColumns), - lts, - visited_lts_list); - } - - public static List<ResultSetRow> execute(SystemUnderTest sut, OpSelectors.Clock clock, Query query) - { - return execute(sut, clock, query, query.schemaSpec.allColumnsSet); - } - - public static List<ResultSetRow> execute(SystemUnderTest sut, OpSelectors.Clock clock, Query query, Set<ColumnSpec<?>> columns) - { - CompiledStatement compiled = query.toSelectStatement(columns, true); - Object[][] objects = sut.executeIdempotent(compiled.cql(), SystemUnderTest.ConsistencyLevel.QUORUM, compiled.bindings()); - List<ResultSetRow> result = new ArrayList<>(); - for (Object[] obj : objects) - result.add(resultSetToRow(query.schemaSpec, clock, broadenResult(query.schemaSpec, columns, obj))); - return result; - } - - public static List<ResultSetRow> execute(SystemUnderTest sut, OpSelectors.Clock clock, CompiledStatement compiled, SchemaSpec schemaSpec) - { - Set<ColumnSpec<?>> columns = schemaSpec.allColumnsSet; - Object[][] objects = sut.executeIdempotent(compiled.cql(), SystemUnderTest.ConsistencyLevel.QUORUM, compiled.bindings()); - List<ResultSetRow> result = new ArrayList<>(); - for (Object[] obj : objects) - result.add(resultSetToRow(schemaSpec, clock, broadenResult(schemaSpec, columns, obj))); - return result; - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/sut/TokenPlacementModel.java b/test/harry/main/org/apache/cassandra/harry/model/TokenPlacementModel.java similarity index 90% rename from test/harry/main/org/apache/cassandra/harry/sut/TokenPlacementModel.java rename to test/harry/main/org/apache/cassandra/harry/model/TokenPlacementModel.java index a25d1ffc6e9d..2c24ec257c79 100644 --- a/test/harry/main/org/apache/cassandra/harry/sut/TokenPlacementModel.java +++ b/test/harry/main/org/apache/cassandra/harry/model/TokenPlacementModel.java @@ -1,22 +1,22 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.sut; + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.model; import java.net.InetAddress; import java.net.UnknownHostException; @@ -95,11 +95,17 @@ public static class ReplicatedRanges { public final Range[] ranges; public final NavigableMap<Range, List<Replica>> placementsForRange; + private final Map<Replica, List<Range>> replicaToRanges = new HashMap<>(); public ReplicatedRanges(Range[] ranges, NavigableMap<Range, List<Replica>> placementsForRange) { this.ranges = ranges; this.placementsForRange = placementsForRange; + for (Map.Entry<Range, List<Replica>> e : placementsForRange.entrySet()) + { + for (Replica replica : e.getValue()) + replicaToRanges.computeIfAbsent(replica, i -> new ArrayList<>()).add(e.getKey()); + } } public List<Replica> replicasFor(long token) @@ -118,7 +124,7 @@ public List<Replica> replicasFor(long token) return placementsForRange.get(ranges[idx]); } - public NavigableMap<Range, List<Replica>> asMap() + public NavigableMap<Range, List<Replica>> asMap() { return placementsForRange; } @@ -143,6 +149,11 @@ else if (cmp > 0) } return -(low + 1); // key not found } + + public List<Range> ranges(Replica replica) + { + return replicaToRanges.get(replica); + } } public interface CompareTo<V> @@ -150,12 +161,12 @@ public interface CompareTo<V> int compareTo(V v); } - public static void addIfUnique(List<Replica> replicas, Set<Integer> names, Replica replica) + private static void addIfUnique(List<Replica> replicas, Set<String> names, Replica replica) { - if (names.contains(replica.node().idx())) + if (names.contains(replica.node().id())) return; replicas.add(replica); - names.add(replica.node().idx()); + names.add(replica.node().id()); } /** @@ -210,22 +221,41 @@ public static List<Node> peerStateToNodes(Object[][] resultset) List<Node> nodes = new ArrayList<>(); for (Object[] row : resultset) { - InetAddress address = (InetAddress) row[0]; - Set<String> tokens = (Set<String>) row[1]; - String dc = (String) row[2]; - String rack = (String) row[3]; + InetAddress address = get(row, 0, "address"); + Set<String> tokens = get(row, 1, "tokens"); + String dc = get(row, 2, "dc"); + String rack = get(row, 3, "rack"); for (String token : tokens) { nodes.add(new Node(0, 0, 0, 0, - constantLookup(address.toString(), - Long.parseLong(token), - dc, - rack))); + constantLookup(address.toString(), + Long.parseLong(token), + dc, + rack))); } } return nodes; } + private static <T> T get(Object[] row, int idx, String name) + { + T t = (T) row[idx]; + if (t == null || ((t instanceof Collection<?>) && ((Collection<?>) t).isEmpty())) + throw new IncompletePeersStateException(name); + return t; + } + + /** + * When the node sees the new epoch, the update of the peers table is async, so checking the table may yield partial results, so need some way to detect this to enable retries. + */ + public static class IncompletePeersStateException extends RuntimeException + { + public IncompletePeersStateException(String column) + { + super("Found incomplete column " + column); + } + } + public static class NtsReplicationFactor extends ReplicationFactor { private KeyspaceParams keyspaceParams; @@ -399,8 +429,8 @@ private static Map<String, DCReplicas> toMap(int[] totalPerDc, int[] transientPe public String toString() { return "NtsReplicationFactor{" + - "map=" + asMap() + - '}'; + "map=" + asMap() + + '}'; } } @@ -483,11 +513,11 @@ public List<Replica> asReplicas() public String toString() { return "DatacenterNodes{" + - "nodes=" + nodes + - ", racks=" + racks + - ", rfLeft=" + rfLeft + - ", acceptableRackRepeats=" + acceptableRackRepeats + - '}'; + "nodes=" + nodes + + ", racks=" + racks + + ", rfLeft=" + rfLeft + + ", acceptableRackRepeats=" + acceptableRackRepeats + + '}'; } } @@ -589,7 +619,7 @@ public static ReplicatedRanges replicate(Range[] ranges, List<Node> nodes, DCRep Range skipped = null; for (Range range : ranges) { - Set<Integer> names = new HashSet<>(); + Set<String> names = new HashSet<>(); List<Replica> replicas = new ArrayList<>(); int idx = primaryReplica(nodes, range); if (idx >= 0) @@ -626,8 +656,8 @@ public static ReplicatedRanges replicate(Range[] ranges, List<Node> nodes, DCRep public String toString() { return "SimpleReplicationFactor{" + - "rf=" + dcReplicas().toString() + - '}'; + "rf=" + dcReplicas().toString() + + '}'; } } @@ -681,9 +711,9 @@ public int hashCode() public String toString() { return "(" + - "" + (start == Long.MIN_VALUE ? "MIN" : start) + - ", " + (end == Long.MIN_VALUE ? "MIN" : end) + - ']'; + "" + (start == Long.MIN_VALUE ? "MIN" : start) + + ", " + (end == Long.MIN_VALUE ? "MIN" : end) + + ']'; } } diff --git a/test/harry/main/org/apache/cassandra/harry/model/TokenPlacementModelHelper.java b/test/harry/main/org/apache/cassandra/harry/model/TokenPlacementModelHelper.java new file mode 100644 index 000000000000..c7976e8d2d6d --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/model/TokenPlacementModelHelper.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.model; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import com.google.common.util.concurrent.Uninterruptibles; + +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.ICoordinator; + +import static org.apache.cassandra.harry.model.TokenPlacementModel.peerStateToNodes; + +public class TokenPlacementModelHelper +{ + /** + * Updates to the peers table is async, which can see partial state... to avoid this issue retry until the writes have become stable. + */ + private static List<TokenPlacementModel.Node> retryWhenNotComplete(ICoordinator coordinator, String query) + { + TokenPlacementModel.IncompletePeersStateException last = null; + // 20 retries with 500ms ~10s... + for (int i = 0; i < 20; i++) + { + try + { + return peerStateToNodes(coordinator.execute(query, ConsistencyLevel.ONE)); + } + catch (TokenPlacementModel.IncompletePeersStateException e) + { + last = e; + Uninterruptibles.sleepUninterruptibly(500, TimeUnit.MILLISECONDS); + } + } + throw last; + } + + public static TokenPlacementModel.ReplicatedRanges getRing(ICoordinator coordinator, TokenPlacementModel.ReplicationFactor rf) + { + List<TokenPlacementModel.Node> other = retryWhenNotComplete(coordinator, "select peer, tokens, data_center, rack from system.peers"); + List<TokenPlacementModel.Node> self = retryWhenNotComplete(coordinator, "select broadcast_address, tokens, data_center, rack from system.local"); + List<TokenPlacementModel.Node> all = new ArrayList<>(); + all.addAll(self); + all.addAll(other); + all.sort(TokenPlacementModel.Node::compareTo); + return rf.replicate(all); + } +} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/model/reconciler/PartitionState.java b/test/harry/main/org/apache/cassandra/harry/model/reconciler/PartitionState.java deleted file mode 100644 index 3484b1cdc42a..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/model/reconciler/PartitionState.java +++ /dev/null @@ -1,310 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.model.reconciler; - -import java.util.*; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.operations.FilteringQuery; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.util.BitSet; -import org.apache.cassandra.harry.util.DescriptorRanges; -import org.apache.cassandra.harry.gen.DataGenerators; -import org.apache.cassandra.harry.model.Model; - -public class PartitionState implements Iterable<Reconciler.RowState> -{ - private static final Logger logger = LoggerFactory.getLogger(Reconciler.class); - - public final long pd; - final long debugCd; - final SchemaSpec schema; - final List<Long> visitedLts = new ArrayList<>(); - final List<Long> skippedLts = new ArrayList<>(); - - // Collected state - Reconciler.RowState staticRow; - - final NavigableMap<Long, Reconciler.RowState> rows; - - public PartitionState(long pd, long debugCd, SchemaSpec schema) - { - this.pd = pd; - this.rows = new TreeMap<>(); - this.staticRow = new Reconciler.RowState(this, - Reconciler.STATIC_CLUSTERING, - Reconciler.arr(schema.staticColumns.size(), DataGenerators.NIL_DESCR), - Reconciler.arr(schema.staticColumns.size(), Model.NO_TIMESTAMP)); - this.debugCd = debugCd; - this.schema = schema; - } - - private PartitionState(long pd, long debugCd, Reconciler.RowState staticRow, NavigableMap<Long, Reconciler.RowState> rows, SchemaSpec schema) - { - this.pd = pd; - this.rows = rows; - this.staticRow = new Reconciler.RowState(this, Reconciler.STATIC_CLUSTERING, - Arrays.copyOf(staticRow.vds, staticRow.vds.length), - Arrays.copyOf(staticRow.lts, staticRow.lts.length)); - this.debugCd = debugCd; - this.schema = schema; - } - - public void compareVisitedLts(List<Long> actualVisitedLts) - { - long min = actualVisitedLts.get(0); - int predictedIdx = Collections.binarySearch(visitedLts, min); - List<Long> predictedSublist = visitedLts.subList(predictedIdx, visitedLts.size()); - Set<Long> set = new HashSet<>(actualVisitedLts); - for (long lts : predictedSublist) - { - if (!set.contains(lts)) - throw new IllegalStateException(String.format("Predicted visit to %d, but did not see it in the debug row\n" + - "Actual: %s\n" + - "Predicted: %s", - lts, actualVisitedLts, predictedSublist)); - } - } - - public NavigableMap<Long, Reconciler.RowState> rows() - { - return rows; - } - - public PartitionState filter(FilteringQuery query) - { - NavigableMap<Long, Reconciler.RowState> rows = new TreeMap<>(); - for (Long cd : this.rows.keySet()) - { - Reconciler.RowState rowState = this.rows.get(cd); - if (query.match(rowState)) - rows.put(cd, rowState); - } - PartitionState ps = new PartitionState(pd, debugCd, staticRow, rows, schema); - return ps; - } - - public void writeStaticRow(long[] sds, long lts) - { - staticRow = updateRowState(staticRow, schema.staticColumns, Reconciler.STATIC_CLUSTERING, sds, lts, false); - } - - public void write(long cd, long[] vds, long lts, boolean writePrimaryKeyLiveness) - { - rows.compute(cd, (cd_, current) -> updateRowState(current, schema.regularColumns, cd, vds, lts, writePrimaryKeyLiveness)); - } - - public void delete(DescriptorRanges.DescriptorRange range, long lts) - { - if (range.minBound > range.maxBound) - return; - - Iterator<Map.Entry<Long, Reconciler.RowState>> iter = rows.subMap(range.minBound, range.minInclusive, - range.maxBound, range.maxInclusive) - .entrySet() - .iterator(); - while (iter.hasNext()) - { - Map.Entry<Long, Reconciler.RowState> e = iter.next(); - if (debugCd != -1 && e.getKey() == debugCd) - logger.info("Hiding {} at {} because of range tombstone {}", debugCd, lts, range); - - // assert row state doesn't have fresher lts - iter.remove(); - } - } - - public void delete(long cd, long lts) - { - Reconciler.RowState state = rows.remove(cd); - if (state != null) - { - for (long v : state.lts) - assert lts >= v : String.format("Attempted to remove a row with a tombstone that has older timestamp (%d): %s", lts, state); - } - } - - public boolean isEmpty() - { - return rows.isEmpty(); - } - - /** - * Method used to update row state of both static and regular rows. - */ - private Reconciler.RowState updateRowState(Reconciler.RowState currentState, List<ColumnSpec<?>> columns, long cd, long[] vds, long lts, boolean writePrimaryKeyLiveness) - { - if (currentState == null) - { - long[] ltss = new long[vds.length]; - long[] vdsCopy = new long[vds.length]; - for (int i = 0; i < vds.length; i++) - { - if (vds[i] != DataGenerators.UNSET_DESCR) - { - ltss[i] = lts; - vdsCopy[i] = vds[i]; - } - else - { - ltss[i] = Model.NO_TIMESTAMP; - vdsCopy[i] = DataGenerators.NIL_DESCR; - } - } - - currentState = new Reconciler.RowState(this, cd, vdsCopy, ltss); - } - else - { - assert currentState.vds.length == vds.length : String.format("Vds: %d, sds: %d", currentState.vds.length, vds.length); - for (int i = 0; i < vds.length; i++) - { - if (vds[i] == DataGenerators.UNSET_DESCR) - continue; - - assert lts >= currentState.lts[i] : String.format("Out-of-order LTS: %d. Max seen: %s", lts, currentState.lts[i]); // sanity check; we're iterating in lts order - - if (currentState.lts[i] == lts) - { - // Timestamp collision case - ColumnSpec<?> column = columns.get(i); - if (column.type.compareLexicographically(vds[i], currentState.vds[i]) > 0) - currentState.vds[i] = vds[i]; - } - else - { - currentState.vds[i] = vds[i]; - assert lts > currentState.lts[i]; - currentState.lts[i] = lts; - } - } - } - - if (writePrimaryKeyLiveness) - currentState.hasPrimaryKeyLivenessInfo = true; - - return currentState; - } - - public void deleteRegularColumns(long lts, long cd, int columnOffset, org.apache.cassandra.harry.util.BitSet columns, org.apache.cassandra.harry.util.BitSet mask) - { - deleteColumns(lts, rows.get(cd), columnOffset, columns, mask); - } - - public void deleteStaticColumns(long lts, int columnOffset, org.apache.cassandra.harry.util.BitSet columns, org.apache.cassandra.harry.util.BitSet mask) - { - deleteColumns(lts, staticRow, columnOffset, columns, mask); - } - - public void deleteColumns(long lts, Reconciler.RowState state, int columnOffset, org.apache.cassandra.harry.util.BitSet columns, BitSet mask) - { - if (state == null) - return; - - //TODO: optimise by iterating over the columns that were removed by this deletion - //TODO: optimise final decision to fully remove the column by counting a number of set/unset columns - boolean allNil = true; - for (int i = 0; i < state.vds.length; i++) - { - if (columns.isSet(columnOffset + i, mask)) - { - state.vds[i] = DataGenerators.NIL_DESCR; - state.lts[i] = Model.NO_TIMESTAMP; - } - else if (state.vds[i] != DataGenerators.NIL_DESCR) - { - allNil = false; - } - } - - if (state.cd != Reconciler.STATIC_CLUSTERING && allNil & !state.hasPrimaryKeyLivenessInfo) - delete(state.cd, lts); - } - - public void deletePartition(long lts) - { - if (debugCd != -1) - logger.info("Hiding {} at {} because partition deletion", debugCd, lts); - - rows.clear(); - if (!schema.staticColumns.isEmpty()) - { - Arrays.fill(staticRow.vds, DataGenerators.NIL_DESCR); - Arrays.fill(staticRow.lts, Model.NO_TIMESTAMP); - } - } - - public Iterator<Reconciler.RowState> iterator() - { - return iterator(false); - } - - public Iterator<Reconciler.RowState> iterator(boolean reverse) - { - if (reverse) - return rows.descendingMap().values().iterator(); - - return rows.values().iterator(); - } - - public Collection<Reconciler.RowState> rows(boolean reverse) - { - if (reverse) - return rows.descendingMap().values(); - - return rows.values(); - } - - public Reconciler.RowState staticRow() - { - return staticRow; - } - - public PartitionState apply(Query query) - { - PartitionState partitionState = new PartitionState(pd, debugCd, schema); - partitionState.staticRow = staticRow; - // TODO: we could improve this if we could get original descriptors - for (Reconciler.RowState rowState : rows.values()) - if (query.matchCd(rowState.cd)) - partitionState.rows.put(rowState.cd, rowState); - - return partitionState; - } - - public String toString(SchemaSpec schema) - { - StringBuilder sb = new StringBuilder(); - - sb.append("Visited LTS: " + visitedLts).append("\n"); - sb.append("Skipped LTS: " + skippedLts).append("\n"); - - if (staticRow != null) - sb.append("Static row:\n" + staticRow.toString(schema)).append("\n"); - - for (Reconciler.RowState row : rows.values()) - sb.append(row.toString(schema)).append("\n"); - - return sb.toString(); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/model/reconciler/Reconciler.java b/test/harry/main/org/apache/cassandra/harry/model/reconciler/Reconciler.java deleted file mode 100644 index 23ba8a41b545..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/model/reconciler/Reconciler.java +++ /dev/null @@ -1,342 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.model.reconciler; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.function.Function; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.tracker.DataTracker; -import org.apache.cassandra.harry.util.DescriptorRanges; -import org.apache.cassandra.harry.util.StringUtils; -import org.apache.cassandra.harry.visitors.GeneratingVisitor; -import org.apache.cassandra.harry.visitors.LtsVisitor; -import org.apache.cassandra.harry.visitors.VisitExecutor; -import org.apache.cassandra.harry.gen.DataGenerators; - -/** - * A simple Cassandra-style reconciler for operations against model state. - * <p> - * It is useful both as a testing/debugging tool (to avoid starting Cassandra - * cluster to get a result set), and as a quiescent model checker. - * - * TODO: it might be useful to actually record deletions instead of just removing values as we do right now. - */ -public class Reconciler -{ - private static final Logger logger = LoggerFactory.getLogger(Reconciler.class); - - public static long STATIC_CLUSTERING = DataGenerators.NIL_DESCR; - - private final OpSelectors.PdSelector pdSelector; - private final SchemaSpec schema; - - private final Function<VisitExecutor, LtsVisitor> visitorFactory; - - public Reconciler(Run run) - { - this(run, - (processor) -> new GeneratingVisitor(run, processor)); - } - - public Reconciler(Run run, Function<VisitExecutor, LtsVisitor> ltsVisitorFactory) - { - this(run.pdSelector, run.schemaSpec, ltsVisitorFactory); - } - - public Reconciler(OpSelectors.PdSelector pdSelector, SchemaSpec schema, Function<VisitExecutor, LtsVisitor> ltsVisitorFactory) - { - this.pdSelector = pdSelector; - this.schema = schema; - this.visitorFactory = ltsVisitorFactory; - } - - private final long debugCd = -1L; - - public PartitionState inflatePartitionState(final long pd, DataTracker tracker, Query query) - { - PartitionState partitionState = new PartitionState(pd, debugCd, schema); - - class Processor extends VisitExecutor - { - // Whether a partition deletion was encountered on this LTS. - private boolean hadPartitionDeletion = false; - private boolean hadTrackingRowWrite = false; - private final List<DescriptorRanges.DescriptorRange> rangeDeletes = new ArrayList<>(); - private final List<Operation> writes = new ArrayList<>(); - private final List<Operation> columnDeletes = new ArrayList<>(); - - @Override - protected void operation(Operation operation) - { - if (hadPartitionDeletion) - return; - - long lts = operation.lts(); - assert pdSelector.pd(operation.lts(), schema) == operation.pd() : String.format("Computed partition descriptor (%d) does for the lts %d. Does not match actual descriptor %d", - pdSelector.pd(operation.lts(), schema), - operation.lts(), - operation.pd()); - - if (operation.kind().hasVisibleVisit()) - partitionState.visitedLts.add(operation.lts()); - - if (schema.trackLts) - hadTrackingRowWrite = true; - - switch (operation.kind()) - { - case DELETE_RANGE: - case DELETE_SLICE: - DescriptorRanges.DescriptorRange range = ((DeleteOp) operation).relations().toRange(lts); - rangeDeletes.add(range); - partitionState.delete(range, lts); - break; - case DELETE_ROW: - long cd = ((DeleteRowOp) operation).cd(); - range = new DescriptorRanges.DescriptorRange(cd, cd, true, true, lts); - rangeDeletes.add(range); - partitionState.delete(cd, lts); - break; - case DELETE_PARTITION: - partitionState.deletePartition(lts); - rangeDeletes.clear(); - writes.clear(); - columnDeletes.clear(); - hadPartitionDeletion = true; - break; - case INSERT_WITH_STATICS: - case INSERT: - case UPDATE: - case UPDATE_WITH_STATICS: - writes.add(operation); - break; - case DELETE_COLUMN_WITH_STATICS: - case DELETE_COLUMN: - columnDeletes.add(operation); - break; - default: - throw new IllegalStateException(); - } - } - - @Override - protected void beforeLts(long lts, long pd) - { - rangeDeletes.clear(); - writes.clear(); - columnDeletes.clear(); - hadPartitionDeletion = false; - } - - @Override - protected void afterLts(long lts, long pd) - { - if (hadPartitionDeletion) - return; - - outer: for (Operation op : writes) - { - WriteOp writeOp = (WriteOp) op; - long opId = op.opId(); - long cd = writeOp.cd(); - - if (hadTrackingRowWrite) - { - long[] statics = new long[schema.staticColumns.size()]; - Arrays.fill(statics, DataGenerators.UNSET_DESCR); - partitionState.writeStaticRow(statics, lts); - } - - switch (op.kind()) - { - case INSERT_WITH_STATICS: - case UPDATE_WITH_STATICS: - WriteStaticOp writeStaticOp = (WriteStaticOp) op; - // We could apply static columns during the first iteration, but it's more convenient - // to reconcile static-level deletions. - partitionState.writeStaticRow(writeStaticOp.sds(), lts); - case INSERT: - case UPDATE: - if (!query.matchCd(cd)) - { - if (debugCd != -1 && cd == debugCd) - logger.info("Hiding {} at {}/{} because there was no query match", debugCd, lts, opId); - continue outer; - } - - for (DescriptorRanges.DescriptorRange range : rangeDeletes) - { - if (range.timestamp >= lts && range.contains(cd)) - { - if (debugCd != -1 && cd == debugCd) - logger.info("Hiding {} at {}/{} because of range tombstone {}", debugCd, lts, opId, range); - continue outer; - } - } - - partitionState.write(cd, - writeOp.vds(), - lts, - op.kind() == OpSelectors.OperationKind.INSERT || op.kind() == OpSelectors.OperationKind.INSERT_WITH_STATICS); - break; - default: - throw new IllegalStateException(op.kind().toString()); - } - } - - outer: for (Operation op : columnDeletes) - { - DeleteColumnsOp deleteColumnsOp = (DeleteColumnsOp) op; - long opId = op.opId(); - long cd = deleteColumnsOp.cd(); - - switch (op.kind()) - { - case DELETE_COLUMN_WITH_STATICS: - partitionState.deleteStaticColumns(lts, - schema.staticColumnsOffset, - deleteColumnsOp.columns(), // descriptorSelector.columnMask(pd, lts, opId, op.opKind()) - schema.staticColumnsMask()); - case DELETE_COLUMN: - if (!query.matchCd(cd)) - { - if (debugCd != -1 && cd == debugCd) - logger.info("Hiding {} at {}/{} because there was no query match", debugCd, lts, opId); - continue outer; - } - - for (DescriptorRanges.DescriptorRange range : rangeDeletes) - { - if (range.timestamp >= lts && range.contains(cd)) - { - if (debugCd != -1 && cd == debugCd) - logger.info("Hiding {} at {}/{} because of range tombstone {}", debugCd, lts, opId, range); - continue outer; - } - } - - partitionState.deleteRegularColumns(lts, - cd, - schema.regularColumnsOffset, - deleteColumnsOp.columns(), - schema.regularColumnsMask()); - break; - } - } - } - - @Override - public void shutdown() throws InterruptedException {} - } - - LtsVisitor visitor = visitorFactory.apply(new Processor()); - - long currentLts = pdSelector.minLtsFor(pd); - long maxStarted = tracker.maxStarted(); - while (currentLts <= maxStarted && currentLts >= 0) - { - if (tracker.isFinished(currentLts)) - { - visitor.visit(currentLts); - } - else - { - partitionState.skippedLts.add(currentLts); - } - - currentLts = pdSelector.nextLts(currentLts); - } - - return partitionState; - } - - public static long[] arr(int length, long fill) - { - long[] arr = new long[length]; - Arrays.fill(arr, fill); - return arr; - } - - public static class RowState - { - public boolean hasPrimaryKeyLivenessInfo = false; - - public final PartitionState partitionState; - public final long cd; - public final long[] vds; - public final long[] lts; - - public RowState(PartitionState partitionState, - long cd, - long[] vds, - long[] lts) - { - this.partitionState = partitionState; - this.cd = cd; - this.vds = vds; - this.lts = lts; - } - - public RowState clone() - { - RowState rowState = new RowState(partitionState, cd, Arrays.copyOf(vds, vds.length), Arrays.copyOf(lts, lts.length)); - rowState.hasPrimaryKeyLivenessInfo = hasPrimaryKeyLivenessInfo; - return rowState; - } - - public String toString() - { - return toString(null); - } - - public String toString(SchemaSpec schema) - { - if (cd == STATIC_CLUSTERING) - { - return " rowStateRow(" - + partitionState.pd + - "L, " + cd + "L" + - ", statics(" + StringUtils.toString(partitionState.staticRow.vds) + ")" + - ", lts(" + StringUtils.toString(partitionState.staticRow.lts) + ")"; - } - else - { - return " rowStateRow(" - + partitionState.pd + - "L, " + cd + - (partitionState.staticRow == null ? "" : ", statics(" + StringUtils.toString(partitionState.staticRow.vds) + ")") + - (partitionState.staticRow == null ? "" : ", lts(" + StringUtils.toString(partitionState.staticRow.lts) + ")") + - ", values(" + StringUtils.toString(vds) + ")" + - ", lts(" + StringUtils.toString(lts) + ")" + - (schema == null ? "" : ", clustering=" + Arrays.toString(schema.inflateClusteringKey(cd))) + - (schema == null ? "" : ", values=" + Arrays.toString(schema.inflateRegularColumns(vds))) + - ")"; - } - } - } -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/model/ring/PlacementSimulator.java b/test/harry/main/org/apache/cassandra/harry/model/ring/PlacementSimulator.java deleted file mode 100644 index f6b75132123f..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/model/ring/PlacementSimulator.java +++ /dev/null @@ -1,1052 +0,0 @@ -///* -// * Licensed to the Apache Software Foundation (ASF) under one -// * or more contributor license agreements. See the NOTICE file -// * distributed with this work for additional information -// * regarding copyright ownership. The ASF licenses this file -// * to you under the Apache License, Version 2.0 (the -// * "License"); you may not use this file except in compliance -// * with the License. You may obtain a copy of the License at -// * -// * http://www.apache.org/licenses/LICENSE-2.0 -// * -// * Unless required by applicable law or agreed to in writing, software -// * distributed under the License is distributed on an "AS IS" BASIS, -// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// * See the License for the specific language governing permissions and -// * limitations under the License. -// */ -// -//package org.apache.cassandra.harry.model.ring; -// -//import java.io.BufferedWriter; -//import java.io.File; -//import java.io.FileNotFoundException; -//import java.io.FileOutputStream; -//import java.io.IOException; -//import java.io.OutputStreamWriter; -//import java.net.UnknownHostException; -//import java.util.*; -//import java.util.function.Function; -//import java.util.function.Predicate; -//import java.util.stream.Collectors; -// -//import org.apache.cassandra.distributed.api.TokenSupplier; -//import org.apache.cassandra.harry.sut.TokenPlacementModel; -// -//import static org.apache.cassandra.harry.sut.TokenPlacementModel.*; -// -///** -// * A small class that helps to avoid doing mental arithmetics on ranges. -// */ -//public class PlacementSimulator -//{ -// @SuppressWarnings("unused") // for debugging convenience -// public static List<Long> readableTokens(int number) -// { -// final List<Long> longs; -// longs = new ArrayList<>(); -// for (int i = 0; i < number; i++) -// { -// longs.add((i + 1) * 100L); -// } -// Collections.shuffle(longs, new Random(1)); -// -// return longs; -// } -// -// public static DebugLog debug = new DebugLog(); -// -// public static class SimulatedPlacements -// { -// public final ReplicationFactor rf; -// public final List<Node> nodes; -// public final NavigableMap<Range, List<Node>> readPlacements; -// public final NavigableMap<Range, List<Node>> writePlacements; -// // Stashed states are steps required to finish the operation. For example, in case of -// // bootstrap, this could be adding replicas to write (and then read) sets after splitting ranges. -// public final List<Transformations> stashedStates; -// -// public SimulatedPlacements(ReplicationFactor rf, -// List<Node> nodes, -// NavigableMap<Range, List<Node>> readPlacements, -// NavigableMap<Range, List<Node>> writePlacements, -// List<Transformations> stashedStates) -// { -// this.rf = rf; -// this.nodes = nodes; -// this.readPlacements = readPlacements; -// this.writePlacements = writePlacements; -// this.stashedStates = stashedStates; -// } -// -// public SimulatedPlacements withNodes(List<Node> newNodes) -// { -// return new SimulatedPlacements(rf, newNodes, readPlacements, writePlacements, stashedStates); -// } -// -// public SimulatedPlacements withReadPlacements(NavigableMap<Range, List<Node>> newReadPlacements) -// { -// return new SimulatedPlacements(rf, nodes, newReadPlacements, writePlacements, stashedStates); -// } -// -// public SimulatedPlacements withWritePlacements(NavigableMap<Range, List<Node>> newWritePlacements) -// { -// return new SimulatedPlacements(rf, nodes, readPlacements, newWritePlacements, stashedStates); -// } -// -// public SimulatedPlacements withStashed(Transformations steps) -// { -// List<Transformations> newStashed = new ArrayList<>(); -// newStashed.addAll(stashedStates); -// newStashed.add(steps); -// return new SimulatedPlacements(rf, nodes, readPlacements, writePlacements, newStashed); -// } -// -// private SimulatedPlacements withoutStashed(Transformations finished) -// { -// List<Transformations> newStates = new ArrayList<>(); -// for (Transformations s : stashedStates) -// if (s != finished) -// newStates.add(s); -// return new SimulatedPlacements(rf, nodes, readPlacements, writePlacements, newStates); -// } -// -// public boolean isWriteTargetFor(long token, Predicate<Node> predicate) -// { -// return writePlacementsFor(token).stream().anyMatch(predicate); -// } -// -// public boolean isReadReplicaFor(long token, Predicate<Node> predicate) -// { -// return readReplicasFor(token).stream().anyMatch(predicate); -// } -// -// public boolean isReadReplicaFor(long minToken, long maxToken, Predicate<Node> predicate) -// { -// return readReplicasFor(minToken, maxToken).stream().anyMatch(predicate); -// } -// -// public List<Node> writePlacementsFor(long token) -// { -// for (Map.Entry<Range, List<Node>> e : writePlacements.entrySet()) -// { -// if (e.getKey().contains(token)) -// return e.getValue(); -// } -// -// throw new AssertionError(); -// } -// -// public List<Node> readReplicasFor(long minToken, long maxToken) -// { -// for (Map.Entry<Range, List<Node>> e : readPlacements.entrySet()) -// { -// if (e.getKey().contains(minToken, maxToken)) -// return e.getValue(); -// } -// -// throw new AssertionError(); -// } -// -// -// public List<Node> readReplicasFor(long token) -// { -// for (Map.Entry<Range, List<Node>> e : readPlacements.entrySet()) -// { -// if (e.getKey().contains(token)) -// return e.getValue(); -// } -// -// throw new AssertionError(); -// } -// -// public String toString() -// { -// return "ModelState{" + -// "\nrf=" + rf + -// "\nnodes=" + nodes + -// "\nreadPlacements=\n" + placementsToString(readPlacements) + -// "\nwritePlacements=\n" + placementsToString(writePlacements) + -// '}'; -// } -// } -// -// public interface SimulatedPlacementHolder -// { -// SimulatedPlacements get(); -// -// /** -// * Applies _one_ of the transformations given to the current state, returning the resulting state. -// * _Does_ set the state within the holder as well. -// */ -// SimulatedPlacements applyNext(Transformations fn); -// SimulatedPlacementHolder set(SimulatedPlacements placements); -// SimulatedPlacementHolder fork(); -// } -// -// public static class RefSimulatedPlacementHolder implements SimulatedPlacementHolder -// { -// private SimulatedPlacements state; -// -// public RefSimulatedPlacementHolder(SimulatedPlacements state) -// { -// this.state = state; -// } -// -// public SimulatedPlacements get() -// { -// return state; -// } -// -// public SimulatedPlacements applyNext(Transformations fn) -// { -// return state = fn.advance(state); -// } -// -// public SimulatedPlacementHolder set(SimulatedPlacements newState) -// { -// state = newState; -// return this; -// } -// -// public SimulatedPlacementHolder fork() -// { -// return new RefSimulatedPlacementHolder(state); -// } -// } -// -// public static class Transformation -// { -// private Function<SimulatedPlacements, SimulatedPlacements> apply; -// private Function<SimulatedPlacements, SimulatedPlacements> revert; -// -// Transformation(Function<SimulatedPlacements, SimulatedPlacements> apply, -// Function<SimulatedPlacements, SimulatedPlacements> revert) -// { -// this.apply = apply; -// this.revert = revert; -// } -// -// public Transformation prepare(Function<SimulatedPlacements, SimulatedPlacements> apply, -// Function<SimulatedPlacements, SimulatedPlacements> revert) -// { -// this.apply = apply; -// this.revert = revert; -// return this; -// } -// } -// -// public static class Transformations -// { -// private final List<Transformation> steps = new ArrayList<>(); -// private int idx = 0; -// -// public void add(Transformation step) -// { -// steps.add(step); -// } -// -// public boolean hasNext() -// { -// return idx < steps.size(); -// } -// -// public SimulatedPlacements advance(SimulatedPlacements prev) -// { -// if (idx >= steps.size()) -// throw new IllegalStateException("Cannot advance transformations, no more steps remaining"); -// -// SimulatedPlacements next = steps.get(idx++).apply.apply(prev); -// if (!hasNext()) -// next = next.withoutStashed(this); -// -// return next; -// } -// -// public boolean hasPrevious() -// { -// return idx > 0; -// } -// -// public SimulatedPlacements revertPublishedEffects(SimulatedPlacements state) -// { -// while (hasPrevious()) -// state = steps.get(--idx).revert.apply(state); -// -// return state.withoutStashed(this); -// } -// } -// -// public static SimulatedPlacements joinFully(SimulatedPlacements baseState, Node node) -// { -// Transformations transformations = join(baseState, node); -// baseState = baseState.withStashed(transformations); -// -// while (transformations.hasNext()) -// baseState = transformations.advance(baseState); -// -// return baseState; -// } -// -// /** -// * Diff-based bootstrap (very close implementation-wise to what production code does) -// */ -// public static Transformations join(SimulatedPlacements baseState, Node bootstrappingNode) -// { -// long token = bootstrappingNode.token(); -// List<Node> splitNodes = split(baseState.nodes, token); -// Map<Range, List<Node>> maximalStateWithPlacement = baseState.rf.replicate(move(splitNodes, token, bootstrappingNode)).placementsForRange; -// -// NavigableMap<Range, List<Node>> splitReadPlacements = baseState.rf.replicate(splitNodes).placementsForRange; -// NavigableMap<Range, List<Node>> splitWritePlacements = baseState.rf.replicate(splitNodes).placementsForRange; -// -// Map<Range, Diff<Node>> allWriteCommands = diff(splitWritePlacements, maximalStateWithPlacement); -// Map<Range, Diff<Node>> step1WriteCommands = map(allWriteCommands, Diff::onlyAdditions); -// Map<Range, Diff<Node>> step3WriteCommands = map(allWriteCommands, Diff::onlyRemovals); -// Map<Range, Diff<Node>> readCommands = diff(splitReadPlacements, maximalStateWithPlacement); -// -// Transformations steps = new Transformations(); -// -// steps.add(new Transformation( -// (model) -> { // apply -// // add the new node to the system and split ranges according to its token, while retaining current -// // placement. This step will always be executed immediately, whereas subsequent steps may be deferred -// debug.log("Splitting ranges to prepare for join of " + bootstrappingNode + "\n"); -// return model.withReadPlacements(splitReplicated(baseState.readPlacements, token)) -// .withWritePlacements(splitReplicated(baseState.writePlacements, token)); -// }, -// (model) -> { // revert -// // final stage of reverting a join is to undo the range splits performed by preparing the operation -// debug.log("Reverting range splits from prepare-join of " + bootstrappingNode + "\n"); -// return model.withWritePlacements(mergeReplicated(model.writePlacements, token)) -// .withReadPlacements(mergeReplicated(model.readPlacements, token)); -// }) -// ); -// -// // Step 1: add new node as a write replica to all ranges it is gaining -// steps.add(new Transformation( -// (model) -> { // apply -// debug.log("Executing start-join of " + bootstrappingNode + "\n"); -// debug.log(String.format("Commands for step 1 of bootstrap of %s.\n" + -// "\twriteModifications=\n%s", -// bootstrappingNode, diffsToString(step1WriteCommands))); -// return model.withWritePlacements(PlacementSimulator.apply(model.writePlacements, step1WriteCommands)); -// }, -// (model) -> { // revert -// debug.log("Reverting start-join of " + bootstrappingNode + "\n"); -// Map<Range, Diff<Node>> inverted = map(step1WriteCommands, Diff::invert); -// debug.log("Commands for reverting step 1 of bootstrap of %s.\n" + -// "\twriteModifications=\n%s", -// bootstrappingNode, diffsToString(inverted)); -// return model.withWritePlacements(PlacementSimulator.apply(model.writePlacements, inverted)); -// }) -// ); -// -// // Step 2: add new node as a read replica to the ranges it is gaining; remove old node from reads at the same time -// steps.add(new Transformation( -// (model) -> { // apply -// debug.log("Executing mid-join of " + bootstrappingNode + "\n"); -// debug.log(String.format("Commands for step 2 of bootstrap of %s.\n" + -// "\treadCommands=\n%s", -// bootstrappingNode, diffsToString(readCommands))); -// return model.withReadPlacements(PlacementSimulator.apply(model.readPlacements, readCommands)); -// }, -// (model) -> { // revert -// debug.log("Reverting mid-join of " + bootstrappingNode + "\n"); -// Map<Range, Diff<Node>> inverted = map(readCommands, Diff::invert); -// debug.log(String.format("Commands for reverting step 2 of bootstrap of %s.\n" + -// "\treadCommands=\n%s", -// bootstrappingNode, diffsToString(inverted))); -// return model.withReadPlacements(PlacementSimulator.apply(model.readPlacements, inverted)); -// }) -// ); -// -// -// // Step 3: finally remove the old node from writes -// steps.add(new Transformation( -// (model) -> { // apply -// debug.log("Executing finish-join of " + bootstrappingNode + "\n"); -// debug.log(String.format("Commands for step 3 of bootstrap of %s.\n" + -// "\twriteModifications=\n%s", -// bootstrappingNode, -// diffsToString(step3WriteCommands))); -// List<Node> newNodes = new ArrayList<>(model.nodes); -// newNodes.add(bootstrappingNode); -// newNodes.sort(Node::compareTo); -// return model.withNodes(newNodes) -// .withWritePlacements(PlacementSimulator.apply(model.writePlacements, step3WriteCommands)); -// }, -// (model) -> { //revert -// throw new IllegalStateException("Can't revert finish-join of " + bootstrappingNode + ", operation is already complete\n"); -// }) -// ); -// -// debug.log("Planned bootstrap of " + bootstrappingNode + "\n"); -// return steps; -// } -// -// public static Transformations move(SimulatedPlacements baseState, Node movingNode, long newToken) -// { -// List<Node> origNodes = new ArrayList<>(baseState.nodes); -// List<Node> finalNodes = new ArrayList<>(); -// for (int i = 0; i < origNodes.size(); i++) -// { -// if (origNodes.get(i).idx() == movingNode.idx()) -// continue; -// finalNodes.add(origNodes.get(i)); -// } -// finalNodes.add(movingNode.overrideToken(newToken)); -// finalNodes.sort(Node::compareTo); -// -// Map<Range, List<Node>> start = splitReplicated(baseState.rf.replicate(origNodes).placementsForRange, newToken); -// Map<Range, List<Node>> end = splitReplicated(baseState.rf.replicate(finalNodes).placementsForRange, movingNode.token()); -// -// Map<Range, Diff<Node>> fromStartToEnd = diff(start, end); -// -// Transformations steps = new Transformations(); -// -// // Step 1: Prepare Move, -// steps.add(new Transformation( -// (model) -> { // apply -// debug.log(String.format("Splitting ranges to prepare for move of %s to %d\n", movingNode, newToken)); -// return model.withReadPlacements(splitReplicated(model.readPlacements, newToken)) -// .withWritePlacements(splitReplicated(model.writePlacements, newToken)); -// }, -// (model) -> { // revert -// debug.log(String.format("Reverting range splits from prepare move of %s to %d\n", movingNode, newToken)); -// return model.withWritePlacements(mergeReplicated(model.writePlacements, newToken)) -// .withReadPlacements(mergeReplicated(model.readPlacements, newToken)); -// }) -// ); -// -// // Step 2: Start Move, add all potential owners to write quorums -// steps.add(new Transformation( -// (model) -> { // apply -// Map<Range, Diff<Node>> diff = map(fromStartToEnd, Diff::onlyAdditions); -// debug.log("Executing start-move of " + movingNode + "\n"); -// debug.log(String.format("Commands for step 1 of move of %s to %d.\n" + -// "\twriteModifications=\n%s", -// movingNode, newToken, diffsToString(diff))); -// -// NavigableMap<Range, List<Node>> placements = model.writePlacements; -// placements = PlacementSimulator.apply(placements, diff); -// return model.withWritePlacements(placements); -// }, -// (model) -> { // revert -// debug.log("Reverting start-move of " + movingNode + "\n"); -// Map<Range, Diff<Node>> diff = map(fromStartToEnd, Diff::onlyAdditions); -// Map<Range, Diff<Node>> inverted = map(diff, Diff::invert); -// debug.log(String.format("Commands for reverting step 1 of move of %s to %d.\n" + -// "\twriteModifications=\n%s", -// movingNode, newToken, diffsToString(inverted))); -// -// return model.withWritePlacements(PlacementSimulator.apply(model.writePlacements, inverted)); -// } -// )); -// // Step 3: Mid Move, remove all nodes that are losing ranges from read quorums, add all nodes gaining ranges to read quorums -// steps.add(new Transformation( -// (model) -> { -// debug.log("Executing mid-move of " + movingNode + "\n"); -// debug.log(String.format("Commands for step 2 of move of %s to %d.\n" + -// "\treadModifications=\n%s", -// movingNode, newToken, diffsToString(fromStartToEnd))); -// -// NavigableMap<Range, List<Node>> placements = model.readPlacements; -// placements = PlacementSimulator.apply(placements, fromStartToEnd); -// return model.withReadPlacements(placements); -// }, -// (model) -> { -// NavigableMap<Range, List<Node>> placements = PlacementSimulator.apply(model.readPlacements, map(fromStartToEnd, Diff::invert)); -// return model.withReadPlacements(placements); -// }) -// ); -// -// // Step 4: Finish Move, remove all nodes that are losing ranges from write quorums -// steps.add(new Transformation( -// (model) -> { -// Map<Range, Diff<Node>> diff = map(fromStartToEnd, Diff::onlyRemovals); -// -// debug.log("Executing finish-move of " + movingNode + "\n"); -// debug.log(String.format("Commands for step 2 of move of %s to %d.\n" + -// "\twriteModifications=\n%s", -// movingNode, newToken, diffsToString(diff))); -// -// List<Node> currentNodes = new ArrayList<>(model.nodes); -// List<Node> newNodes = new ArrayList<>(); -// for (int i = 0; i < currentNodes.size(); i++) -// { -// if (currentNodes.get(i).idx() == movingNode.idx()) -// continue; -// newNodes.add(currentNodes.get(i)); -// } -// newNodes.add(movingNode.overrideToken(newToken)); -// newNodes.sort(Node::compareTo); -// -// Map<Range, List<Node>> writePlacements = model.writePlacements; -// writePlacements = PlacementSimulator.apply(writePlacements, diff); -// -// return model.withWritePlacements(mergeReplicated(writePlacements, movingNode.token())) -// .withReadPlacements(mergeReplicated(model.readPlacements, movingNode.token())) -// .withNodes(newNodes); -// }, -// (model) -> { -// throw new IllegalStateException(String.format("Can't revert finish-move of %d, operation is already complete", newToken)); -// }) -// ); -// -// return steps; -// } -// -// public static Transformations leave(SimulatedPlacements baseState, Node toRemove) -// { -// // calculate current placements - this is start state -// Map<Range, List<Node>> start = baseState.rf.replicate(baseState.nodes).placementsForRange; -// -// List<Node> afterLeaveNodes = new ArrayList<>(baseState.nodes); -// afterLeaveNodes.remove(toRemove); -// // calculate placements based on existing ranges but final set of nodes - this is end state -// Map<Range, List<Node>> end = baseState.rf.replicate(toRanges(baseState.nodes), afterLeaveNodes).placementsForRange; -// // maximal state is union of start & end -// -// Map<Range, Diff<Node>> allWriteCommands = diff(start, end); -// Map<Range, Diff<Node>> step1WriteCommands = map(allWriteCommands, Diff::onlyAdditions); -// Map<Range, Diff<Node>> step3WriteCommands = map(allWriteCommands, Diff::onlyRemovals); -// Map<Range, Diff<Node>> readCommands = diff(start, end); -// Transformations steps = new Transformations(); -// steps.add(new Transformation( -// (model) -> { // apply -// debug.log("Executing start-leave of " + toRemove + "\n"); -// debug.log(String.format("Commands for step 1 of decommission of %s.\n" + -// "\twriteModifications=\n%s", -// toRemove, diffsToString(step1WriteCommands))); -// return model.withWritePlacements(PlacementSimulator.apply(model.writePlacements, step1WriteCommands)); -// }, -// (model) -> { // revert -// debug.log("Reverting start-leave of " + toRemove + "\n"); -// Map<Range, Diff<Node>> inverted = map(step1WriteCommands, Diff::invert); -// debug.log(String.format("Commands for reverting step 1 of decommission of %s.\n" + -// "\twriteModifications=\n%s", -// toRemove, diffsToString(inverted))); -// return model.withWritePlacements(PlacementSimulator.apply(model.writePlacements, inverted)); -// }) -// ); -// -// steps.add(new Transformation( -// (model) -> { // apply -// debug.log("Executing mid-leave of " + toRemove + "\n"); -// debug.log(String.format("Commands for step 2 of decommission of %s.\n" + -// "\treadModifications=\n%s", -// toRemove, -// diffsToString(readCommands))); -// return model.withReadPlacements(PlacementSimulator.apply(model.readPlacements, readCommands)); -// }, -// (model) -> { // revert -// debug.log("Reverting mid-leave of " + toRemove + "\n"); -// Map<Range, Diff<Node>> inverted = map(readCommands, Diff::invert); -// debug.log(String.format("Commands for reverting step 2 of decommission of %s.\n" + -// "\treadModifications=\n%s", -// toRemove, -// diffsToString(inverted))); -// return model.withReadPlacements(PlacementSimulator.apply(model.readPlacements, inverted)); -// }) -// ); -// -// steps.add(new Transformation( -// (model) -> { // apply -// debug.log("Executing finish-leave decommission of " + toRemove + "\n"); -// debug.log(String.format("Commands for step 3 of decommission of %s.\n" + -// "\twriteModifications=\n%s", -// toRemove, -// diffsToString(step3WriteCommands))); -// List<Node> newNodes = new ArrayList<>(model.nodes); -// newNodes.remove(toRemove); -// newNodes.sort(Node::compareTo); -// Map<Range, List<Node>> writes = PlacementSimulator.apply(model.writePlacements, step3WriteCommands); -// return model.withReadPlacements(mergeReplicated(model.readPlacements, toRemove.token())) -// .withWritePlacements(mergeReplicated(writes, toRemove.token())) -// .withNodes(newNodes); -// }, -// (model) -> { // revert -// throw new IllegalStateException("Can't revert finish-leave of " + toRemove + ", operation is already complete\n"); -// })); -// -// debug.log("Planned decommission of " + toRemove + "\n"); -// return steps; -// } -// -// public static Transformations replace(SimulatedPlacements baseState, Node toReplace, Node replacement) -// { -// Map<Range, List<Node>> start = baseState.rf.replicate(baseState.nodes).placementsForRange; -// Map<Range, Diff<Node>> allCommands = new TreeMap<>(); -// start.forEach((range, nodes) -> { -// if (nodes.contains(toReplace)) -// { -// allCommands.put(range, new Diff<>(Collections.singletonList(replacement), -// Collections.singletonList(toReplace))); -// } -// }); -// Map<Range, Diff<Node>> step1WriteCommands = map(allCommands, Diff::onlyAdditions); -// Map<Range, Diff<Node>> step3WriteCommands = map(allCommands, Diff::onlyRemovals); -// Map<Range, Diff<Node>> readCommands = allCommands; -// Transformations steps = new Transformations(); -// steps.add(new Transformation( -// (model) -> { // apply -// debug.log(String.format("Executing start-replace of %s for %s\n", replacement, toReplace)); -// debug.log(String.format("Commands for step 1 of bootstrap of %s for replacement of %s.\n" + -// "\twriteModifications=\n%s", -// replacement, toReplace, diffsToString(step1WriteCommands))); -// return model.withWritePlacements(PlacementSimulator.apply(model.writePlacements, step1WriteCommands)); -// }, -// (model) -> { // revert -// debug.log(String.format("Reverting start-replace of %s for %s\n", replacement, toReplace)); -// Map<Range, Diff<Node>> inverted = map(step1WriteCommands, Diff::invert); -// debug.log(String.format("Commands for reverting step 1 of bootstrap of %s for replacement of %s.\n" + -// "\twriteModifications=\n%s", -// replacement, toReplace, diffsToString(inverted))); -// return model.withWritePlacements(PlacementSimulator.apply(model.writePlacements, inverted)); -// }) -// ); -// -// steps.add(new Transformation( -// (model) -> { // apply -// debug.log(String.format("Executing mid-replace of %s for %s\n", replacement, toReplace)); -// debug.log(String.format("Commands for step 2 of bootstrap of %s for replacement of %s.\n" + -// "\treadModifications=\n%s", -// replacement, toReplace, -// diffsToString(readCommands))); -// return model.withReadPlacements(PlacementSimulator.apply(model.readPlacements, readCommands)); -// }, -// (model) -> { // revert -// debug.log(String.format("Reverting mid-replace of %s for %s\n", replacement, toReplace)); -// Map<Range, Diff<Node>> inverted = map(readCommands, Diff::invert); -// debug.log(String.format("Commands for reverting step 2 of bootstrap of %s for replacement of %s.\n" + -// "\treadModifications=\n%s", -// replacement, toReplace, -// diffsToString(inverted))); -// return model.withReadPlacements(PlacementSimulator.apply(model.readPlacements, inverted)); -// }) -// ); -// -// steps.add(new Transformation( -// (model) -> { // apply -// debug.log(String.format("Executing finish-replace of %s for %s\n", replacement, toReplace)); -// debug.log(String.format("Commands for step 3 of bootstrap of %sfor replacement of %s.\n" + -// "\twriteModifications=\n%s", -// replacement, toReplace, -// diffsToString(step3WriteCommands))); -// List<Node> newNodes = new ArrayList<>(model.nodes); -// newNodes.remove(toReplace); -// newNodes.add(replacement); -// newNodes.sort(Node::compareTo); -// return model.withNodes(newNodes) -// .withWritePlacements(PlacementSimulator.apply(model.writePlacements, step3WriteCommands)); -// }, -// (model) -> { // revert -// throw new IllegalStateException(String.format("Can't revert finish-replace of %s for %s, operation is already complete\n", replacement, toReplace)); -// }) -// ); -// -// debug.log(String.format("Planned bootstrap of %s for replacement of %s\n", replacement, toReplace)); -// return steps; -// } -// -// public static void assertPlacements(SimulatedPlacements placements, Map<Range, List<Node>> r, Map<Range, List<Node>> w) -// { -// assertRanges(r, placements.readPlacements); -// assertRanges(w, placements.writePlacements); -// } -// -// public static void assertRanges(Map<Range, List<Node>> expected, Map<Range, List<Node>> actual) -// { -// Assert.assertEquals(expected.keySet(), actual.keySet()); -// expected.forEach((k, v) -> { -// // When comparing replica sets, we only care about the endpoint (i.e. the node.id). For the purpose -// // of simulation, during split operations we duplicate the node holding the range being split as if giving -// // it two tokens, the original one and the split point. e.g. With N1@100, N2@200 then splitting at 150, -// // we will end up with (100, 150] -> N2@150 and (150, 200] -> N2@200. As this is purely an artefact of the -// // bootstrap_diffBased implementation and the real code doesn't do this, only the endpoint matters for -// // correctness, so we limit this comparison to endpoints only. -// Assert.assertEquals(String.format("For key: %s\n", k), -// expected.get(k).stream().map(n -> n.idx()).sorted().collect(Collectors.toList()), -// actual.get(k).stream().map(n -> n.idx()).sorted().collect(Collectors.toList())); -// }); -// } -// -// public static <T> boolean containsAll(Set<T> a, Set<T> b) -// { -// if (a.isEmpty() && !b.isEmpty()) -// return false; // empty set does not contain all entries of a non-empty one -// for (T v : b) -// if (!a.contains(v)) -// return false; -// -// return true; -// } -// -// /** -// * Applies a given diff to the placement map -// */ -// public static NavigableMap<Range, List<Node>> apply(Map<Range, List<Node>> orig, Map<Range, Diff<Node>> diff) -// { -// assert containsAll(orig.keySet(), diff.keySet()) : String.format("Can't apply diff to a map with different sets of keys:" + -// "\nOrig ks: %s" + -// "\nDiff ks: %s" + -// "\nDiff: %s", -// orig.keySet(), diff.keySet(), diff); -// NavigableMap<Range, List<Node>> res = new TreeMap<>(); -// for (Map.Entry<Range, List<Node>> entry : orig.entrySet()) -// { -// Range range = entry.getKey(); -// if (diff.containsKey(range)) -// res.put(range, apply(entry.getValue(), diff.get(range))); -// else -// res.put(range, entry.getValue()); -// } -// return Collections.unmodifiableNavigableMap(res); -// } -// -// /** -// * Apply diff to a list of nodes -// */ -// public static List<Node> apply(List<Node> nodes, Diff<Node> diff) -// { -// Set<Node> tmp = new HashSet<>(nodes); -// tmp.addAll(diff.additions); -// for (Node removal : diff.removals) -// tmp.remove(removal); -// List<Node> newNodes = new ArrayList<>(tmp); -// newNodes.sort(Node::compareTo); -// return Collections.unmodifiableList(newNodes); -// } -// -// /** -// * Diff two placement maps -// */ -// public static Map<Range, Diff<Node>> diff(Map<Range, List<Node>> l, Map<Range, List<Node>> r) -// { -// assert l.keySet().equals(r.keySet()) : String.format("Can't diff events from different bases %s %s", l.keySet(), r.keySet()); -// Map<Range, Diff<Node>> diff = new TreeMap<>(); -// for (Map.Entry<Range, List<Node>> entry : l.entrySet()) -// { -// Range range = entry.getKey(); -// Diff<Node> d = diff(entry.getValue(), r.get(range)); -// if (!d.removals.isEmpty() || !d.additions.isEmpty()) -// diff.put(range, d); -// } -// return Collections.unmodifiableMap(diff); -// } -// -// public static <T> Map<Range, T> map(Map<Range, T> diff, Function<T, T> fn) -// { -// Map<Range, T> newDiff = new TreeMap<>(); -// for (Map.Entry<Range, T> entry : diff.entrySet()) -// { -// T newV = fn.apply(entry.getValue()); -// if (newV != null) -// newDiff.put(entry.getKey(), newV); -// } -// return Collections.unmodifiableMap(newDiff); -// } -// -// public static <T> List<T> map(List<T> coll, Function<T, T> map) -// { -// List<T> newColl = new ArrayList<>(coll); -// for (T v : coll) -// newColl.add(map.apply(v)); -// return newColl; -// } -// -// /** -// * Produce a diff (i.e. set of additions/subtractions that should be applied to the list of nodes in order to produce -// * r from l) -// */ -// public static Diff<Node> diff(List<Node> l, List<Node> r) -// { -// // additions things present in r but not in l -// List<Node> additions = new ArrayList<>(); -// // removals are things present in l but not r -// List<Node> removals = new ArrayList<>(); -// -// for (Node i : r) -// { -// boolean isPresentInL = false; -// for (Node j : l) -// { -// if (i.equals(j)) -// { -// isPresentInL = true; -// break; -// } -// } -// -// if (!isPresentInL) -// additions.add(i); -// } -// -// for (Node i : l) -// { -// boolean isPresentInR = false; -// for (Node j : r) -// { -// if (i.equals(j)) -// { -// isPresentInR = true; -// break; -// } -// } -// -// if (!isPresentInR) -// removals.add(i); -// } -// return new Diff<>(additions, removals); -// } -// -// public static Map<Range, List<Node>> superset(Map<Range, List<Node>> l, Map<Range, List<Node>> r) -// { -// assert l.keySet().equals(r.keySet()) : String.format("%s != %s", l.keySet(), r.keySet()); -// -// Map<Range, List<Node>> newState = new TreeMap<>(); -// for (Map.Entry<Range, List<Node>> entry : l.entrySet()) -// { -// Range range = entry.getKey(); -// Set<Node> nodes = new HashSet<>(); -// nodes.addAll(entry.getValue()); -// nodes.addAll(r.get(range)); -// newState.put(range, new ArrayList<>(nodes)); -// } -// -// return newState; -// } -// -// public static NavigableMap<Range, List<Node>> mergeReplicated(Map<Range, List<Node>> orig, long removingToken) -// { -// NavigableMap<Range, List<Node>> newState = new TreeMap<>(); -// Iterator<Map.Entry<Range, List<Node>>> iter = orig.entrySet().iterator(); -// while (iter.hasNext()) -// { -// Map.Entry<Range, List<Node>> current = iter.next(); -// if (current.getKey().end == removingToken) -// { -// assert iter.hasNext() : "Cannot merge range, no more ranges in list"; -// Map.Entry<Range, List<Node>> next = iter.next(); -// assert current.getValue().containsAll(next.getValue()) && current.getValue().size() == next.getValue().size() -// : "Cannot merge ranges with different replica groups"; -// Range merged = new Range(current.getKey().start, next.getKey().end); -// newState.put(merged, current.getValue()); -// } -// else -// { -// newState.put(current.getKey(), current.getValue()); -// } -// } -// -// return newState; -// } -// -// public static NavigableMap<Range, List<Node>> splitReplicated(Map<Range, List<Node>> orig, long splitAt) -// { -// NavigableMap<Range, List<Node>> newState = new TreeMap<>(); -// for (Map.Entry<Range, List<Node>> entry : orig.entrySet()) -// { -// Range range = entry.getKey(); -// if (range.contains(splitAt)) -// { -// newState.put(new Range(range.start, splitAt), entry.getValue()); -// newState.put(new Range(splitAt, range.end), entry.getValue()); -// } -// else -// { -// newState.put(range, entry.getValue()); -// } -// } -// return newState; -// } -// -// /** -// * "Split" the list of nodes at splitAt, without changing ownership -// */ -// public static List<Node> split(List<Node> nodes, long splitAt) -// { -// List<Node> newNodes = new ArrayList<>(); -// boolean inserted = false; -// Node previous = null; -// for (int i = nodes.size() - 1; i >= 0; i--) -// { -// Node node = nodes.get(i); -// if (!inserted && splitAt > node.token()) -// { -// // We're trying to split rightmost range -// if (previous == null) -// { -// newNodes.add(nodes.get(0).overrideToken(splitAt)); -// } -// else -// { -// newNodes.add(previous.overrideToken(splitAt)); -// } -// inserted = true; -// } -// -// newNodes.add(node); -// previous = node; -// } -// -// // Leftmost is split -// if (!inserted) -// newNodes.add(previous.overrideToken(splitAt)); -// -// newNodes.sort(Node::compareTo); -// return Collections.unmodifiableList(newNodes); -// } -// -// /** -// * Change the ownership of the freshly split token -// */ -// public static List<Node> move(List<Node> nodes, long tokenToMove, Node newOwner) -// { -// List<Node> newNodes = new ArrayList<>(); -// for (Node node : nodes) -// { -// if (node.token() == tokenToMove) -// newNodes.add(newOwner.overrideToken(tokenToMove)); -// else -// newNodes.add(node); -// } -// newNodes.sort(Node::compareTo); -// return Collections.unmodifiableList(newNodes); -// } -// -// public static List<Node> filter(List<Node> nodes, Predicate<Node> pred) -// { -// List<Node> newNodes = new ArrayList<>(); -// for (Node node : nodes) -// { -// if (pred.test(node)) -// newNodes.add(node); -// } -// newNodes.sort(Node::compareTo); -// return Collections.unmodifiableList(newNodes); -// } -// -// private static <K extends Comparable<K>, T1, T2> Map<K, T2> mapValues(Map<K, T1> allDCs, Function<T1, T2> map) -// { -// NavigableMap<K, T2> res = new TreeMap<>(); -// for (Map.Entry<K, T1> e : allDCs.entrySet()) -// { -// res.put(e.getKey(), map.apply(e.getValue())); -// } -// return res; -// } -// -// public static Map<String, List<Node>> nodesByDC(List<Node> nodes) -// { -// Map<String, List<Node>> nodesByDC = new HashMap<>(); -// for (Node node : nodes) -// nodesByDC.computeIfAbsent(node.dc(), (k) -> new ArrayList<>()).add(node); -// -// return nodesByDC; -// } -// -// public static Map<String, Set<String>> racksByDC(List<Node> nodes) -// { -// Map<String, Set<String>> racksByDC = new HashMap<>(); -// for (Node node : nodes) -// racksByDC.computeIfAbsent(node.dc(), (k) -> new HashSet<>()).add(node.rack()); -// -// return racksByDC; -// } -// -// public static class Diff<T> { -// public final List<T> additions; -// public final List<T> removals; -// -// public Diff(List<T> additions, List<T> removals) -// { -// this.additions = additions; -// this.removals = removals; -// } -// -// public String toString() -// { -// return "Diff{" + -// "additions=" + additions + -// ", removals=" + removals + -// '}'; -// } -// -// public Diff<T> onlyAdditions() -// { -// if (additions.isEmpty()) return null; -// return new Diff<>(additions, Collections.emptyList()); -// } -// -// public Diff<T> onlyRemovals() -// { -// if (removals.isEmpty()) return null; -// return new Diff<>(Collections.emptyList(), removals); -// } -// -// public Diff<T> invert() -// { -// // invert removals & additions -// return new Diff<>(removals, additions); -// } -// } -// -// -// public static String diffsToString(Map<Range, Diff<Node>> placements) -// { -// StringBuilder builder = new StringBuilder(); -// for (Map.Entry<Range, Diff<Node>> e : placements.entrySet()) -// { -// builder.append("\t\t").append(e.getKey()).append(": ").append(e.getValue()).append("\n"); -// } -// return builder.toString(); -// } -// -// public static String placementsToString(Map<Range, List<Node>> placements) -// { -// StringBuilder builder = new StringBuilder(); -// for (Map.Entry<Range, List<Node>> e : placements.entrySet()) -// { -// builder.append("\t\t").append(e.getKey()).append(": ").append(e.getValue()).append("\n"); -// } -// return builder.toString(); -// } -// -// public static class DebugLog -// { -// private final BufferedWriter operationLog; -// public DebugLog() -// { -// File f = new File("simulated.log"); -// try -// { -// operationLog = new BufferedWriter(new OutputStreamWriter(new FileOutputStream(f))); -// } -// catch (FileNotFoundException e) -// { -// throw new RuntimeException(e); -// } -// } -// -// public void log(long seq, Object t) -// { -// log("%d: %s\n", seq, t); -// } -// -// private void log(String format, Object... objects) -// { -// try -// { -// operationLog.write(String.format(format, objects)); -// operationLog.flush(); -// } -// catch (IOException e) -// { -// // ignore -// } -// } -// } -// -//} diff --git a/test/harry/main/org/apache/cassandra/harry/op/Operations.java b/test/harry/main/org/apache/cassandra/harry/op/Operations.java new file mode 100644 index 000000000000..47ffa93c1c66 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/op/Operations.java @@ -0,0 +1,562 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.op; + +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import accord.utils.Invariants; +import org.apache.cassandra.harry.ColumnSpec; +import org.apache.cassandra.harry.MagicConstants; +import org.apache.cassandra.harry.Relations; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.util.BitSet; + +public class Operations +{ + public static class WriteOp extends PartitionOperation + { + private final long cd; + private final long vds[]; + private final long sds[]; + + public WriteOp(long lts, long pd, long cd, long[] vds, long[] sds, Kind kind) + { + super(lts, pd, kind); + this.cd = cd; + this.vds = vds; + this.sds = sds; + } + + public long cd() + { + return this.cd; + } + + public long[] vds() + { + return vds; + } + + public long[] sds() + { + return sds; + } + } + + public static class DeleteRow extends PartitionOperation + { + protected final long cd; + + public DeleteRow(long lts, long pd, long cd) + { + super(lts, pd, Kind.DELETE_ROW); + this.cd = cd; + } + + public long cd() + { + return cd; + } + } + + public static class DeleteColumns extends PartitionOperation + { + private final long cd; + private final BitSet regularColumns; + private final BitSet staticColumns; + + public DeleteColumns(long lts, long pd, long cd, BitSet regularColumns, BitSet staticColumns) + { + super(lts, pd, Kind.DELETE_COLUMNS); + this.cd = cd; + this.regularColumns = regularColumns; + this.staticColumns = staticColumns; + } + + public long cd() + { + return cd; + } + + public BitSet regularColumns() + { + return regularColumns; + } + + public BitSet staticColumns() + { + return staticColumns; + } + } + + + public static class DeleteRange extends PartitionOperation + { + final long lowerBound; + final long upperBound; + + final Relations.RelationKind[] lowBoundRelation; + final Relations.RelationKind[] highBoundRelation; + + public DeleteRange(long lts, long pd, + long lowerCdBound, + long upperCdBound, + Relations.RelationKind[] lowBoundRelation, + Relations.RelationKind[] highBoundRelation) + { + super(lts, pd, Kind.DELETE_RANGE); + this.lowerBound = lowerCdBound; + this.upperBound = upperCdBound; + this.lowBoundRelation = lowBoundRelation; + this.highBoundRelation = highBoundRelation; + } + + public Relations.RelationKind[] lowerBoundRelation() + { + return lowBoundRelation; + } + + public Relations.RelationKind[] upperBoundRelation() + { + return highBoundRelation; + } + + public long lowerBound() + { + return lowerBound; + } + + public long upperBound() + { + return upperBound; + } + } + + public static abstract class SelectStatement extends PartitionOperation + { + public SelectStatement(long lts, long pd, Kind kind) + { + super(lts, pd, kind); + } + + public abstract ClusteringOrderBy orderBy(); + public abstract BitSet selection(); + } + + public static class SelectRange extends SelectStatement + { + final long lowerBound; + final long upperBound; + + final Relations.RelationKind[] lowerBoundRelation; + final Relations.RelationKind[] upperBoundRelation; + + public SelectRange(long lts, long pd, + long lowerBound, + long highBound, + Relations.RelationKind[] lowerBoundRelation, + Relations.RelationKind[] upperBoundRelation) + { + super(lts, pd, Kind.SELECT_RANGE); + this.lowerBound = lowerBound; + this.upperBound = highBound; + this.lowerBoundRelation = lowerBoundRelation; + this.upperBoundRelation = upperBoundRelation; + } + + @Override + public ClusteringOrderBy orderBy() + { + return ClusteringOrderBy.ASC; + } + + @Override + public BitSet selection() + { + return MagicConstants.ALL_COLUMNS; + } + + public Relations.RelationKind[] lowerBoundRelation() + { + return lowerBoundRelation; + } + + public Relations.RelationKind[] upperBoundRelation() + { + return upperBoundRelation; + } + + public long lowerBound() + { + return lowerBound; + } + + public long upperBound() + { + return upperBound; + } + } + + public static class SelectCustom extends SelectStatement + { + private final Relations.Relation[] ckRelations; + private final Relations.Relation[] regularRelations; + private final Relations.Relation[] staticRelations; + + public SelectCustom(long lts, long pd, + Relations.Relation[] ckRelations, + Relations.Relation[] regularRelations, + Relations.Relation[] staticRelations) + { + super(lts, pd, Kind.SELECT_CUSTOM); + this.ckRelations = ckRelations; + this.regularRelations = regularRelations; + this.staticRelations = staticRelations; + } + + @Override + public ClusteringOrderBy orderBy() + { + return ClusteringOrderBy.ASC; + } + + @Override + public BitSet selection() + { + return MagicConstants.ALL_COLUMNS; + } + + public Relations.Relation[] ckRelations() + { + return ckRelations; + } + + public Relations.Relation[] staticRelations() + { + return staticRelations; + } + + public Relations.Relation[] regularRelations() + { + return regularRelations; + } + } + + public static class SelectRow extends SelectStatement + { + private final long cd; + public SelectRow(long lts, long pd, long cd) + { + super(lts, pd, Kind.SELECT_ROW); + this.cd = cd; + } + + public long cd() + { + return cd; + } + + public ClusteringOrderBy orderBy() + { + return ClusteringOrderBy.ASC; + } + + @Override + public BitSet selection() + { + return MagicConstants.ALL_COLUMNS; + } + } + + public static class SelectPartition extends SelectStatement + { + private final ClusteringOrderBy orderBy; + + public SelectPartition(long lts, long pd) + { + this(lts, pd, ClusteringOrderBy.ASC); + } + + public SelectPartition(long lts, long pd, ClusteringOrderBy orderBy) + { + super(lts, pd, Kind.SELECT_PARTITION); + this.orderBy = orderBy; + } + + public SelectPartition orderBy(ClusteringOrderBy orderBy) + { + return new SelectPartition(lts, pd, orderBy); + } + + public ClusteringOrderBy orderBy() + { + return orderBy; + } + + @Override + public BitSet selection() + { + return MagicConstants.ALL_COLUMNS; + } + } + + public static class DeletePartition extends PartitionOperation + { + public DeletePartition(long lts, long pd) + { + super(lts, pd, Kind.DELETE_PARTITION); + } + } + + public interface DeleteColumnsOp extends Operation + { + long cd(); + BitSet regularColumns(); + BitSet staticColumns(); + } + + public interface Operation + { + long lts(); + Kind kind(); + } + + public static abstract class PartitionOperation implements Operation + { + public final long lts; + public final long pd; + public final Kind kind; + + public PartitionOperation(long lts, long pd, Kind kind) + { + this.pd = pd; + this.lts = lts; + this.kind = kind; + } + + public final long pd() + { + return pd; + } + + @Override + public final long lts() + { + return lts; + } + + @Override + public final Kind kind() + { + return kind; + } + + public String toString() + { + return "Operation{" + + " lts=" + lts + + " pd=" + pd + + ", kind=" + kind + + '}'; + } + } + + public enum Kind + { + /** + * Custom operation such as flush + */ + CUSTOM(false), + + UPDATE(true), + INSERT(true), + + DELETE_PARTITION(true), + DELETE_ROW(false), + DELETE_COLUMNS(true), + DELETE_RANGE(false), + + SELECT_PARTITION(true), + SELECT_ROW(false), + SELECT_RANGE(true), + SELECT_CUSTOM(true); + public final boolean partititonLevel; + + Kind(boolean partitionLevel) + { + this.partititonLevel = partitionLevel; + } + + } + + public static class CustomRunnableOperation implements Operation + { + public final long lts; + public final long opId; + private final Runnable run; + + public CustomRunnableOperation(long lts, long opId, Runnable runnable) + { + assert opId >= 0; + this.lts = lts; + this.opId = opId; + this.run = runnable; + } + + public void execute() + { + run.run(); + } + + @Override + public long lts() + { + return lts; + } + + @Override + public Kind kind() + { + return Kind.CUSTOM; + } + } + + /** + * ClusteringOrder by should be understood in terms of how we're going to iterate through this partition + * (in other words, if first clustering component order is DESC, we'll iterate in ASC order) + */ + public enum ClusteringOrderBy + { + ASC, DESC + } + + public interface Selection + { + // TODO: allow expressions here + Collection<ColumnSpec<?>> columns(); + boolean includeTimestamps(); + boolean isWildcard(); + + boolean selects(ColumnSpec<?> column); + boolean selectsAllOf(List<ColumnSpec<?>> subSelection); + int indexOf(ColumnSpec<?> column); + + static Selection fromBitSet(BitSet bitSet, SchemaSpec schema) + { + if (bitSet == MagicConstants.ALL_COLUMNS) + { + Map<ColumnSpec<?>, Integer> columns = new HashMap<>(); + for (int i = 0; i < schema.allColumnInSelectOrder.size(); i++) + columns.put(schema.allColumnInSelectOrder.get(i), i); + return new Wildcard(columns); + } + else + { + Invariants.checkState(schema.allColumnInSelectOrder.size() == bitSet.size()); + Map<ColumnSpec<?>, Integer> columns = new HashMap<>(); + for (int i = 0; i < schema.allColumnInSelectOrder.size(); i++) + { + if (bitSet.isSet(i)) + columns.put(schema.allColumnInSelectOrder.get(i), i); + } + // TODO: timestamp + return new Columns(columns, false); + } + } + } + + public static class Wildcard extends Columns + { + private Wildcard(Map<ColumnSpec<?>, Integer> columns) + { + super(columns, false); + } + + @Override + public Collection<ColumnSpec<?>> columns() + { + return columns.keySet(); + } + + @Override + public boolean includeTimestamps() + { + return false; + } + + @Override + public boolean isWildcard() + { + return true; + } + } + + public static class Columns implements Selection + { + final Map<ColumnSpec<?>, Integer> columns; + final boolean includeTimestamp; + + public Columns(Map<ColumnSpec<?>, Integer> columns, boolean includeTimestamp) + { + this.columns = columns; + this.includeTimestamp = includeTimestamp; + } + + @Override + public Collection<ColumnSpec<?>> columns() + { + return columns.keySet(); + } + + @Override + public boolean includeTimestamps() + { + return includeTimestamp; + } + + @Override + public boolean isWildcard() + { + return false; + } + + public boolean selects(ColumnSpec<?> column) + { + return columns.containsKey(column); + } + + public boolean selectsAllOf(List<ColumnSpec<?>> subSelection) + { + for (ColumnSpec<?> column : subSelection) + { + if (!selects(column)) + return false; + } + return true; + } + + public int indexOf(ColumnSpec<?> column) + { + return columns.get(column); + } + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/op/Visit.java b/test/harry/main/org/apache/cassandra/harry/op/Visit.java new file mode 100644 index 000000000000..6c8c31390c89 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/op/Visit.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.op; + +import java.util.HashSet; +import java.util.Set; + +import org.junit.Assert; + +import org.apache.cassandra.harry.op.Operations.Operation; + +public class Visit +{ + public final long lts; + public final Operation[] operations; + public final Set<Long> visitedPartitions; + + public final boolean selectOnly; + public Visit(long lts, Operation[] operations) + { + Assert.assertTrue(operations.length > 0); + this.lts = lts; + this.operations = operations; + this.visitedPartitions = new HashSet<>(); + boolean selectOnly = true; + for (Operation operation : operations) + { + if (selectOnly && !(operation instanceof Operations.SelectStatement)) + selectOnly = false; + + if (operation instanceof Operations.PartitionOperation) + visitedPartitions.add(((Operations.PartitionOperation) operation).pd()); + + } + this.selectOnly = selectOnly; + } + + public String toString() + { + if (operations.length == 1) + return String.format("Visit %d: %s", lts, operations[0]); + + StringBuilder sb = new StringBuilder(); + sb.append("Visit ").append(lts).append(":\n"); + boolean first = true; + for (Operation operation : operations) + { + if (!first) + sb.append("\n"); + first = false; + sb.append(operation); + } + + return sb.toString(); + } +} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/operations/CompiledStatement.java b/test/harry/main/org/apache/cassandra/harry/operations/CompiledStatement.java deleted file mode 100644 index 3d1428f782a2..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/operations/CompiledStatement.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.operations; - -public class CompiledStatement -{ - private final String cql; - private final Object[] bindings; - - public CompiledStatement(String cql, Object... bindings) - { - this.cql = cql; - this.bindings = bindings; - } - - public String cql() - { - return cql; - } - - public CompiledStatement withSchema(String oldKs, String oldTable, String newKs, String newTable) - { - return new CompiledStatement(cql.replace(oldKs + "." + oldTable, - newKs + "." + newTable), - bindings); - } - - public CompiledStatement withFiltering() - { - return new CompiledStatement(cql.replace(";", - " ALLOW FILTERING;"), - bindings); - } - - public Object[] bindings() - { - return bindings; - } - - public static CompiledStatement create(String cql, Object... bindings) - { - return new CompiledStatement(cql, bindings); - } - - public String toString() - { - return "CompiledStatement{" + - "cql='" + cql + '\'' + - ", bindings=" + bindingsToString(bindings) + - '}'; - } - - public static String bindingsToString(Object... bindings) - { - StringBuilder sb = new StringBuilder(); - boolean isFirst = true; - for (Object binding : bindings) - { - if (isFirst) - isFirst = false; - else - sb.append(","); - - if (binding instanceof String) - sb.append("\"").append(binding).append("\""); - else if (binding instanceof Short) - sb.append("(short)").append(binding); - else if (binding instanceof Byte) - sb.append("(byte)").append(binding); - else if (binding instanceof Float) - sb.append("(float)").append(binding); - else if (binding instanceof Long) - sb.append(binding).append("L"); - else - sb.append(binding); - } - return sb.toString(); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/operations/DeleteHelper.java b/test/harry/main/org/apache/cassandra/harry/operations/DeleteHelper.java deleted file mode 100644 index 1ad5ede5606b..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/operations/DeleteHelper.java +++ /dev/null @@ -1,190 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.operations; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.function.IntConsumer; - -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.util.BitSet; - -public class DeleteHelper -{ - public static CompiledStatement deleteColumn(SchemaSpec schema, - long pd, - long cd, - BitSet columns, - BitSet mask, - long rts) - { - if (columns == null || columns.allUnset(mask)) - throw new IllegalArgumentException("Can't have a delete column query with no columns set. Column mask: " + columns); - - return delete(schema, pd, cd, columns, mask, rts); - } - - public static CompiledStatement deleteColumn(SchemaSpec schema, - long pd, - BitSet columns, - BitSet mask, - long rts) - { - if (columns == null || columns.allUnset(mask)) - throw new IllegalArgumentException("Can't have a delete column query with no columns set. Column mask: " + columns); - - return delete(schema, pd, columns, mask, rts); - } - - public static CompiledStatement deleteRow(SchemaSpec schema, - long pd, - long cd, - long rts) - { - return delete(schema, pd, cd, null, null, rts); - } - - public static CompiledStatement delete(SchemaSpec schema, - long pd, - List<Relation> relations, - BitSet columnsToDelete, - BitSet mask, - long rts) - { - assert (columnsToDelete == null && mask == null) || (columnsToDelete != null && mask != null); - return compile(schema, - pd, - relations, - columnsToDelete, - mask, - rts); - } - - private static CompiledStatement delete(SchemaSpec schema, - long pd, - long cd, - BitSet columnsToDelete, - BitSet mask, - long rts) - { - return compile(schema, - pd, - Relation.eqRelations(schema.ckGenerator.slice(cd), - schema.clusteringKeys), - columnsToDelete, - mask, - rts); - } - - private static CompiledStatement delete(SchemaSpec schema, - long pd, - BitSet columnsToDelete, - BitSet mask, - long rts) - { - return compile(schema, - pd, - new ArrayList<>(), - columnsToDelete, - mask, - rts); - } - - public static CompiledStatement delete(SchemaSpec schema, - long pd, - long rts) - { - return compile(schema, - pd, - Collections.emptyList(), - null, - null, - rts); - } - - private static CompiledStatement compile(SchemaSpec schema, - long pd, - List<Relation> relations, - BitSet columnsToDelete, - BitSet mask, - long ts) - { - StringBuilder b = new StringBuilder(); - b.append("DELETE "); - if (columnsToDelete != null) - { - assert mask != null; - assert relations == null || relations.stream().allMatch((r) -> r.kind == Relation.RelationKind.EQ); - String[] names = columnNames(schema.allColumns, columnsToDelete, mask); - for (int i = 0; i < names.length; i++) - { - if (i > 0) - b.append(", "); - b.append(names[i]); - } - b.append(" "); - } - b.append("FROM ") - .append(schema.keyspace).append(".").append(schema.table) - .append(" USING TIMESTAMP ") - .append(ts) - .append(" WHERE "); - - List<Object> bindings = new ArrayList<>(); - - schema.inflateRelations(pd, - relations, - new SchemaSpec.AddRelationCallback() - { - boolean isFirst = true; - public void accept(ColumnSpec<?> spec, Relation.RelationKind kind, Object value) - { - if (isFirst) - isFirst = false; - else - b.append(" AND "); - b.append(kind.getClause(spec)); - bindings.add(value); - } - }); - - b.append(";"); - - Object[] bindingsArr = bindings.toArray(new Object[bindings.size()]); - - return new CompiledStatement(b.toString(), bindingsArr); - } - - private static String[] columnNames(List<ColumnSpec<?>> columns, BitSet selectedColumns, BitSet mask) - { - String[] columnNames = new String[selectedColumns.setCount(mask)]; - selectedColumns.eachSetBit(new IntConsumer() - { - int i = 0; - - public void accept(int idx) - { - columnNames[i++] = columns.get(idx).name; - } - }, mask); - return columnNames; - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/operations/FilteringQuery.java b/test/harry/main/org/apache/cassandra/harry/operations/FilteringQuery.java deleted file mode 100644 index 1a39e50fa8da..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/operations/FilteringQuery.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.operations; - -import java.util.List; - -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.reconciler.Reconciler; -import org.apache.cassandra.harry.util.DescriptorRanges; - -public class FilteringQuery extends Query -{ - public FilteringQuery(long pd, - boolean reverse, - List<Relation> relations, - SchemaSpec schemaSpec) - { - super(QueryKind.SINGLE_PARTITION, pd, reverse, relations, schemaSpec); - } - - public boolean match(Reconciler.RowState rowState) - { - for (Relation relation : relations) - { - switch (relation.columnSpec.kind) - { - case CLUSTERING: - if (!matchCd(rowState.cd)) - return false; - break; - case REGULAR: - if (!relation.match(rowState.vds[relation.columnSpec.getColumnIndex()])) - return false; - break; - case STATIC: - if (!relation.match(rowState.partitionState.staticRow().vds[relation.columnSpec.getColumnIndex()])) - return false; - break; - case PARTITION_KEY: - if (!relation.match(rowState.partitionState.pd)) - return false; - break; - } - } - return true; - } - - public DescriptorRanges.DescriptorRange toRange(long ts) - { - throw new IllegalStateException("not implemented for filtering query"); - } - - @Override - public String toString() - { - return "FilteringQuery{pd=" + pd + ", reverse=" + reverse + ", relations=" + relations + '}'; - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/operations/Query.java b/test/harry/main/org/apache/cassandra/harry/operations/Query.java deleted file mode 100644 index 2ca8a429ee50..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/operations/Query.java +++ /dev/null @@ -1,557 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.operations; - -import java.util.*; -import java.util.function.LongSupplier; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.gen.DataGenerators; -import org.apache.cassandra.harry.gen.rng.RngUtils; -import org.apache.cassandra.harry.model.SelectHelper; -import org.apache.cassandra.harry.util.DescriptorRanges; - -import static org.apache.cassandra.harry.operations.QueryGenerator.relationKind; -import static org.apache.cassandra.harry.operations.Relation.FORWARD_COMPARATOR; - -/** - * A class representing relations in the query, essentially what WHERE clause means. - */ -public abstract class Query -{ - private static final Logger logger = LoggerFactory.getLogger(Query.class); - // TODO: There are queries without PD - public final long pd; - public final boolean reverse; - public final List<Relation> relations; - public final Map<String, List<Relation>> relationsMap; - public final SchemaSpec schemaSpec; - public final QueryKind queryKind; - public final Selection selection; - - public Query(QueryKind kind, long pd, boolean reverse, List<Relation> relations, SchemaSpec schemaSpec) - { - this(kind, pd, reverse, relations, schemaSpec, new Columns(schemaSpec.allColumnsSet, true)); - } - - public Query(QueryKind kind, long pd, boolean reverse, List<Relation> relations, SchemaSpec schemaSpec, Selection selection) - { - this.queryKind = kind; - this.pd = pd; - this.reverse = reverse; - this.relations = relations; - this.relationsMap = new HashMap<>(); - for (Relation relation : relations) - this.relationsMap.computeIfAbsent(relation.column(), column -> new ArrayList<>()).add(relation); - this.schemaSpec = schemaSpec; - this.selection = selection; - } - - // TODO: pd, values, filtering? - public boolean matchCd(long cd) - { - return simpleMatch(this, cd); - } - - public static boolean simpleMatch(Query query, - long cd) - { - long[] sliced = query.schemaSpec.ckGenerator.slice(cd); - for (int i = 0; i < query.schemaSpec.clusteringKeys.size(); i++) - { - List<Relation> relations = query.relationsMap.get(query.schemaSpec.clusteringKeys.get(i).name); - if (relations == null) - continue; - - for (Relation r : relations) - { - if (!r.match(sliced[i])) - return false; - } - } - - return true; - } - - public static class SinglePartitionQuery extends Query - { - public SinglePartitionQuery(QueryKind kind, long pd, boolean reverse, List<Relation> allRelations, SchemaSpec schemaSpec, Selection selection) - { - super(kind, pd, reverse, allRelations, schemaSpec, selection); - } - - public boolean matchCd(long cd) - { - return true; - } - - public DescriptorRanges.DescriptorRange toRange(long ts) - { - return new DescriptorRanges.DescriptorRange(Long.MIN_VALUE, Long.MAX_VALUE, true, true, ts); - } - - public String toString() - { - return "SinglePartitionQuery{" + - "pd=" + pd + - ", reverse=" + reverse + - ", relations=" + relations + - ", relationsMap=" + relationsMap + - ", schemaSpec=" + schemaSpec + - '}'; - } - } - - public static class SingleClusteringQuery extends Query - { - private final long cd; - - public SingleClusteringQuery(QueryKind kind, long pd, long cd, boolean reverse, List<Relation> allRelations, SchemaSpec schemaSpec) - { - super(kind, pd, reverse, allRelations, schemaSpec); - this.cd = cd; - } - - public DescriptorRanges.DescriptorRange toRange(long ts) - { - return new DescriptorRanges.DescriptorRange(cd, cd, true, true, ts); - } - - @Override - public boolean matchCd(long cd) - { - return cd == this.cd; - } - } - - public static class ClusteringSliceQuery extends ClusteringRangeQuery - { - public ClusteringSliceQuery(QueryKind kind, - long pd, - long cdMin, - long cdMax, - Relation.RelationKind minRelation, - Relation.RelationKind maxRelation, - boolean reverse, - List<Relation> allRelations, - SchemaSpec schemaSpec) - { - super(kind, pd, cdMin, cdMax, minRelation, maxRelation, reverse, allRelations, schemaSpec); - } - - public String toString() - { - return "ClusteringSliceQuery{" + - "\n" + toSelectStatement() + - "\npd=" + pd + - "\nreverse=" + reverse + - "\nrelations=" + relations + - "\nrelationsMap=" + relationsMap + - "\nschemaSpec=" + schemaSpec + - "\nqueryKind=" + queryKind + - "\ncdMin=" + cdMin + - "(" + Arrays.toString(schemaSpec.ckGenerator.slice(cdMin)) + ")" + - "\ncdMax=" + cdMax + - "(" + Arrays.toString(schemaSpec.ckGenerator.slice(cdMax)) + ")" + - "\nminRelation=" + minRelation + - "\nmaxRelation=" + maxRelation + - '}' + "\n" + toSelectStatement().cql(); - } - } - - public static class ClusteringRangeQuery extends Query - { - protected final long cdMin; - protected final long cdMax; - protected final Relation.RelationKind minRelation; - protected final Relation.RelationKind maxRelation; - - public ClusteringRangeQuery(QueryKind kind, - long pd, - long cdMin, - long cdMax, - Relation.RelationKind minRelation, - Relation.RelationKind maxRelation, - boolean reverse, - List<Relation> allRelations, - SchemaSpec schemaSpec) - { - super(kind, pd, reverse, allRelations, schemaSpec); - this.cdMin = cdMin; - this.cdMax = cdMax; - this.minRelation = minRelation; - this.maxRelation = maxRelation; - } - - public DescriptorRanges.DescriptorRange toRange(long ts) - { - return new DescriptorRanges.DescriptorRange(cdMin, cdMax, minRelation.isInclusive(), maxRelation.isInclusive(), ts); - } - - public boolean matchCd(long cd) - { - // TODO: looks like we don't really need comparator here. - Relation.LongComparator cmp = FORWARD_COMPARATOR; - boolean res = minRelation.match(cmp, cd, cdMin) && maxRelation.match(cmp, cd, cdMax); - if (!logger.isDebugEnabled()) - return res; - boolean superRes = super.matchCd(cd); - if (res != superRes) - { - logger.debug("Query did not pass a sanity check.\n{}\n Super call returned: {}, while current call: {}\n" + - "cd = {} {} ({})\n" + - "this.cdMin = {} {} ({})\n" + - "this.cdMax = {} {} ({})\n" + - "minRelation.match(cmp, cd, this.cdMin) = {}\n" + - "maxRelation.match(cmp, cd, this.cdMax) = {}\n", - this, - superRes, res, - cd, Long.toHexString(cd), Arrays.toString(schemaSpec.ckGenerator.slice(cd)), - cdMin, Long.toHexString(cdMin), Arrays.toString(schemaSpec.ckGenerator.slice(cdMin)), - cdMax, Long.toHexString(cdMax), Arrays.toString(schemaSpec.ckGenerator.slice(cdMax)), - minRelation.match(cmp, cd, cdMin), - maxRelation.match(cmp, cd, cdMax)); - } - return res; - } - - public String toString() - { - return "ClusteringRangeQuery{" + - "pd=" + pd + - ", reverse=" + reverse + - ", relations=" + relations + - ", relationsMap=" + relationsMap + - ", schemaSpec=" + schemaSpec + - ", cdMin=" + cdMin + - ", cdMax=" + cdMax + - ", minRelation=" + minRelation + - ", maxRelation=" + maxRelation + - '}'; - } - } - - public CompiledStatement toSelectStatement() - { - return SelectHelper.select(schemaSpec, pd, selection.columns(), relations, reverse, selection.includeTimestamp()); - } - - public CompiledStatement toSelectStatement(boolean includeWriteTime) - { - return SelectHelper.select(schemaSpec, pd, selection.columns(), relations, reverse, includeWriteTime); - } - - public CompiledStatement toSelectStatement(Set<ColumnSpec<?>> columns, boolean includeWriteTime) - { - return SelectHelper.select(schemaSpec, pd, columns, relations, reverse, includeWriteTime); - } - - public CompiledStatement toDeleteStatement(long rts) - { - return DeleteHelper.delete(schemaSpec, pd, relations, null, null, rts); - } - - public abstract DescriptorRanges.DescriptorRange toRange(long ts); - - public static Query selectAllColumns(SchemaSpec schemaSpec, long pd, boolean reverse) - { - return selectPartition(schemaSpec, pd, reverse, new Columns(schemaSpec.allColumnsSet, true)); - } - - public static Query selectAllColumnsWildcard(SchemaSpec schemaSpec, long pd, boolean reverse) - { - return selectPartition(schemaSpec, pd, reverse, Wildcard.instance); - } - - public static Query selectPartition(SchemaSpec schemaSpec, long pd, boolean reverse, Selection selection) - { - return new Query.SinglePartitionQuery(Query.QueryKind.SINGLE_PARTITION, - pd, - reverse, - Collections.emptyList(), - schemaSpec, - selection); - } - - public static Query singleClustering(SchemaSpec schema, long pd, long cd, boolean reverse) - { - return new Query.SingleClusteringQuery(Query.QueryKind.SINGLE_CLUSTERING, - pd, - cd, - reverse, - Relation.eqRelations(schema.ckGenerator.slice(cd), schema.clusteringKeys), - schema); - } - - public static Query clusteringSliceQuery(SchemaSpec schema, long pd, long cd, long queryDescriptor, boolean isGt, boolean isEquals, boolean reverse) - { - List<Relation> relations = new ArrayList<>(); - - long[] sliced = schema.ckGenerator.slice(cd); - long min; - long max; - int nonEqFrom = RngUtils.asInt(queryDescriptor, 0, sliced.length - 1); - - long[] minBound = new long[sliced.length]; - long[] maxBound = new long[sliced.length]; - - // Algorithm that determines boundaries for a clustering slice. - // - // Basic principles are not hard but there are a few edge cases. I haven't figured out how to simplify - // those, so there might be some room for improvement. In short, what we want to achieve is: - // - // 1. Every part that is restricted with an EQ relation goes into the bound verbatim. - // 2. Every part that is restricted with a non-EQ relation (LT, GT, LTE, GTE) is taken into the bound - // if it is required to satisfy the relationship. For example, in `ck1 = 0 AND ck2 < 5`, ck2 will go - // to the _max_ boundary, and minimum value will go to the _min_ boundary, since we can select every - // descriptor that is prefixed with ck1. - // 3. Every other part (e.g., ones that are not explicitly mentioned in the query) has to be restricted - // according to equality. For example, in `ck1 = 0 AND ck2 < 5`, ck3 that is present in schema but not - // mentioned in query, makes sure that any value between [0, min_value, min_value] and [0, 5, min_value] - // is matched. - // - // One edge case is a query on the first clustering key: `ck1 < 5`. In this case, we have to fixup the lower - // value to the minimum possible value. We could really just do Long.MIN_VALUE, but in case we forget to - // adjust entropy elsewhere, it'll be caught correctly here. - for (int i = 0; i < sliced.length; i++) - { - long v = sliced[i]; - DataGenerators.KeyGenerator gen = schema.ckGenerator; - ColumnSpec column = schema.clusteringKeys.get(i); - int idx = i; - LongSupplier maxSupplier = () -> gen.maxValue(idx); - LongSupplier minSupplier = () -> gen.minValue(idx); - - if (i < nonEqFrom) - { - relations.add(Relation.eqRelation(schema.clusteringKeys.get(i), v)); - minBound[i] = v; - maxBound[i] = v; - } - else if (i == nonEqFrom) - { - relations.add(Relation.relation(relationKind(isGt, isEquals), schema.clusteringKeys.get(i), v)); - - if (column.isReversed()) - { - minBound[i] = isGt ? minSupplier.getAsLong() : v; - maxBound[i] = isGt ? v : maxSupplier.getAsLong(); - } - else - { - minBound[i] = isGt ? v : minSupplier.getAsLong(); - maxBound[i] = isGt ? maxSupplier.getAsLong() : v; - } - } - else - { - if (isEquals) - { - minBound[i] = minSupplier.getAsLong(); - maxBound[i] = maxSupplier.getAsLong(); - } - // If we have a non-eq case, all subsequent bounds have to correspond to the maximum in normal case, - // or minimum in case the last bound locked with a relation was reversed. - // - // For example, if we have (ck1, ck2, ck3) as (ASC, DESC, ASC), and query ck1 > X, we'll have: - // [xxxxx | max_value | max_value] - // ck1 ck2 ck3 - // which will exclude xxxx, but take every possible (ck1 > xxxxx) prefixed value. - // - // Similarly, if we have (ck1, ck2, ck3) as (ASC, DESC, ASC), and query ck1 <= X, we'll have: - // [xxxxx | max_value | max_value] - // which will include every (ck1 < xxxxx), and any clustering prefixed with xxxxx. - else if (schema.clusteringKeys.get(nonEqFrom).isReversed()) - maxBound[i] = minBound[i] = isGt ? minSupplier.getAsLong() : maxSupplier.getAsLong(); - else - maxBound[i] = minBound[i] = isGt ? maxSupplier.getAsLong() : minSupplier.getAsLong(); - } - } - - if (schema.clusteringKeys.get(nonEqFrom).isReversed()) - isGt = !isGt; - - min = schema.ckGenerator.stitch(minBound); - max = schema.ckGenerator.stitch(maxBound); - - if (nonEqFrom == 0) - { - min = isGt ? min : schema.ckGenerator.minValue(); - max = !isGt ? max : schema.ckGenerator.maxValue(); - } - - // if we're about to create an "impossible" query, just bump the modifier and re-generate - if (min == max && !isEquals) - throw new IllegalArgumentException("Impossible Query"); - - return new Query.ClusteringSliceQuery(Query.QueryKind.CLUSTERING_SLICE, - pd, - min, - max, - relationKind(true, isGt ? isEquals : true), - relationKind(false, !isGt ? isEquals : true), - reverse, - relations, - schema); - } - - public static Query clusteringRangeQuery(SchemaSpec schema, long pd, long cd1, long cd2, long queryDescriptor, boolean isMinEq, boolean isMaxEq, boolean reverse) - { - List<Relation> relations = new ArrayList<>(); - - long[] minBound = schema.ckGenerator.slice(cd1); - long[] maxBound = schema.ckGenerator.slice(cd2); - - int nonEqFrom = RngUtils.asInt(queryDescriptor, 0, schema.clusteringKeys.size() - 1); - - // Logic here is similar to how clustering slices are implemented, except for both lower and upper bound - // get their values from sliced value in (1) and (2) cases: - // - // 1. Every part that is restricted with an EQ relation, takes its value from the min bound. - // TODO: this can actually be improved, since in case of hierarchical clustering generation we can - // pick out of the keys that are already locked. That said, we'll exercise more cases the way - // it is implemented right now. - // 2. Every part that is restricted with a non-EQ relation is taken into the bound, if it is used in - // the query. For example in, `ck1 = 0 AND ck2 > 2 AND ck2 < 5`, ck2 values 2 and 5 will be placed, - // correspondingly, to the min and max bound. - // 3. Every other part has to be restricted according to equality. Similar to clustering slice, we have - // to decide whether we use a min or the max value for the bound. Foe example `ck1 = 0 AND ck2 > 2 AND ck2 <= 5`, - // assuming we have ck3 that is present in schema but not mentioned in the query, we'll have bounds - // created as follows: [0, 2, max_value] and [0, 5, max_value]. Idea here is that since ck2 = 2 is excluded, - // we also disallow all ck3 values for [0, 2] prefix. Similarly, since ck2 = 5 is included, we allow every - // ck3 value with a prefix of [0, 5]. - for (int i = 0; i < schema.clusteringKeys.size(); i++) - { - ColumnSpec<?> col = schema.clusteringKeys.get(i); - if (i < nonEqFrom) - { - relations.add(Relation.eqRelation(col, minBound[i])); - maxBound[i] = minBound[i]; - } - else if (i == nonEqFrom) - { - long minLocked = Math.min(minBound[nonEqFrom], maxBound[nonEqFrom]); - long maxLocked = Math.max(minBound[nonEqFrom], maxBound[nonEqFrom]); - relations.add(Relation.relation(relationKind(true, col.isReversed() ? isMaxEq : isMinEq), col, - col.isReversed() ? maxLocked : minLocked)); - relations.add(Relation.relation(relationKind(false, col.isReversed() ? isMinEq : isMaxEq), col, - col.isReversed() ? minLocked : maxLocked)); - minBound[i] = minLocked; - maxBound[i] = maxLocked; - } - else - { - minBound[i] = isMinEq ? schema.ckGenerator.minValue(i) : schema.ckGenerator.maxValue(i); - maxBound[i] = isMaxEq ? schema.ckGenerator.maxValue(i) : schema.ckGenerator.minValue(i); - } - } - - long stitchedMin = schema.ckGenerator.stitch(minBound); - long stitchedMax = schema.ckGenerator.stitch(maxBound); - - // TODO: one of the ways to get rid of garbage here, and potentially even simplify the code is to - // simply return bounds here. After bounds are created, we slice them and generate query right - // from the bounds. In this case, we can even say that things like -inf/+inf are special values, - // and use them as placeholders. Also, it'll be easier to manipulate relations. - return new Query.ClusteringRangeQuery(Query.QueryKind.CLUSTERING_RANGE, - pd, - stitchedMin, - stitchedMax, - relationKind(true, isMinEq), - relationKind(false, isMaxEq), - reverse, - relations, - schema); - } - - public enum QueryKind - { - SINGLE_PARTITION, - SINGLE_CLUSTERING, - // Not sure if that's the best way to name these, but the difference is as follows: - // - // For a single clustering, clustering slice is essentially [x; ∞) or (-∞; x]. - // - // However, in case of multiple clusterings, such as (ck1, ck2, ck3), can result into a range query with locked prefix: - // ck1 = x and ck2 = y and ck3 > z - // - // This would translate into bounds such as: - // ( (x, y, z); (x, y, max_ck3) ) - // - // Logic here is that when we're "locking" x and y, and allow third slice to be in the range [z; ∞). - // Implementation is a bit more involved than that, since we have to make sure it works for all clustering sizes - // and for reversed type. - CLUSTERING_SLICE, - // For a single clustering, clustering slice is essentially [x; y]. - // - // For multiple-clusterings case, for example (ck1, ck2, ck3), we select how many clusterings we "lock" with EQ - // relation, and do a range slice for the rest. For example: - // ck1 = x and ck2 = y and ck3 > z1 and ck3 < z2 - // - // Such queries only make sense if written partition actually has clusterings that have intersecting parts. - CLUSTERING_RANGE - } - - public interface Selection - { - Set<ColumnSpec<?>> columns(); - boolean includeTimestamp(); - } - - public static class Wildcard implements Selection - { - public static final Wildcard instance = new Wildcard(); - - public Set<ColumnSpec<?>> columns() - { - return null; - } - - public boolean includeTimestamp() - { - return false; - } - } - - public static class Columns implements Selection - { - private Set<ColumnSpec<?>> columns; - private boolean includeTimestamp; - - public Columns(Set<ColumnSpec<?>> columns, boolean includeTimestamp) - { - this.columns = columns; - this.includeTimestamp = includeTimestamp; - } - - public Set<ColumnSpec<?>> columns() - { - return columns; - } - - public boolean includeTimestamp() - { - return includeTimestamp; - } - } -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/operations/QueryGenerator.java b/test/harry/main/org/apache/cassandra/harry/operations/QueryGenerator.java deleted file mode 100644 index aff09e3cfcaf..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/operations/QueryGenerator.java +++ /dev/null @@ -1,176 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.operations; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.gen.rng.RngUtils; -import org.apache.cassandra.harry.gen.Surjections; -import org.apache.cassandra.harry.model.OpSelectors; - -// TODO: there's a lot of potential to reduce an amount of garbage here. -// TODO: refactor. Currently, this class is a base for both SELECT and DELETE statements. In retrospect, -// a better way to do the same thing would've been to just inflate bounds, be able to inflate -// any type of query from the bounds, and leave things like "reverse" up to the last mile / implementation. -public class QueryGenerator -{ - private static final Logger logger = LoggerFactory.getLogger(QueryGenerator.class); - - private static final long GT_STREAM = 0b1; - private static final long E_STREAM = 0b10; - - private final OpSelectors.PureRng rng; - private final OpSelectors.PdSelector pdSelector; - private final OpSelectors.DescriptorSelector descriptorSelector; - private final SchemaSpec schema; - - public QueryGenerator(Run run) - { - this(run.schemaSpec, run.pdSelector, run.descriptorSelector, run.rng); - } - - public QueryGenerator(SchemaSpec schema, - OpSelectors.PdSelector pdSelector, - OpSelectors.DescriptorSelector descriptorSelector, - OpSelectors.PureRng rng) - { - this.pdSelector = pdSelector; - this.descriptorSelector = descriptorSelector; - this.schema = schema; - this.rng = rng; - } - - public static class TypedQueryGenerator - { - private final OpSelectors.PureRng rng; - private final QueryGenerator queryGenerator; - private final Surjections.Surjection<Query.QueryKind> queryKindGen; - - public TypedQueryGenerator(Run run) - { - this(run.rng, new QueryGenerator(run)); - } - - public TypedQueryGenerator(OpSelectors.PureRng rng, - QueryGenerator queryGenerator) - { - this(rng, Surjections.enumValues(Query.QueryKind.class), queryGenerator); - } - - public TypedQueryGenerator(OpSelectors.PureRng rng, - Surjections.Surjection<Query.QueryKind> queryKindGen, - QueryGenerator queryGenerator) - { - this.rng = rng; - this.queryGenerator = queryGenerator; - this.queryKindGen = queryKindGen; - } - - // Queries are inflated from LTS, which identifies the partition, and i, a modifier for the query to - // be able to generate different queries for the same lts. - public Query inflate(long lts, long modifier) - { - long descriptor = rng.next(modifier, lts); - Query.QueryKind queryKind = queryKindGen.inflate(descriptor); - return queryGenerator.inflate(lts, modifier, queryKind); - } - } - - public Query inflate(long lts, long modifier, Query.QueryKind queryKind) - { - long pd = pdSelector.pd(lts, schema); - long queryDescriptor = rng.next(modifier, lts); - boolean reverse = queryDescriptor % 2 == 0; - switch (queryKind) - { - case SINGLE_PARTITION: - return singlePartition(pd, reverse); - case SINGLE_CLUSTERING: - { - long cd = descriptorSelector.randomCd(pd, queryDescriptor, schema); - return singleClustering(pd, cd, reverse); - } - case CLUSTERING_SLICE: - { - long cd = descriptorSelector.randomCd(pd, queryDescriptor, schema); - try - { - return clusteringSliceQuery(pd, cd, queryDescriptor, reverse); - } - catch (IllegalArgumentException retry) - { - return inflate(lts, modifier + 1, queryKind); - } - } - case CLUSTERING_RANGE: - { - try - { - long cd1 = descriptorSelector.randomCd(pd, queryDescriptor, schema); - long cd2 = descriptorSelector.randomCd(pd, rng.next(queryDescriptor, lts), schema); - return clusteringRangeQuery(pd, cd1, cd2, queryDescriptor, reverse); - } - catch (IllegalArgumentException retry) - { - return inflate(lts, modifier + 1, queryKind); - } - } - default: - throw new IllegalArgumentException("Shouldn't happen"); - } - } - - public Query singlePartition(long pd, boolean reverse) - { - return Query.selectAllColumns(schema, pd, reverse); - } - - public Query singleClustering(long pd, long cd, boolean reverse) - { - return Query.singleClustering(schema, pd, cd, reverse); - } - - public Query clusteringSliceQuery(long pd, long cd, long queryDescriptor, boolean reverse) - { - boolean isGt = RngUtils.asBoolean(rng.next(queryDescriptor, GT_STREAM)); - // TODO: make generation of EQ configurable; turn it off and on - boolean isEquals = RngUtils.asBoolean(rng.next(queryDescriptor, E_STREAM)); - - return Query.clusteringSliceQuery(schema, pd, cd, queryDescriptor, isGt, isEquals, reverse); - } - - public Query clusteringRangeQuery(long pd, long cd1, long cd2, long queryDescriptor, boolean reverse) - { - boolean isMinEq = RngUtils.asBoolean(queryDescriptor); - boolean isMaxEq = RngUtils.asBoolean(rng.next(queryDescriptor, pd)); - - return Query.clusteringRangeQuery(schema, pd, cd1, cd2, queryDescriptor, isMinEq, isMaxEq, reverse); - } - - public static Relation.RelationKind relationKind(boolean isGt, boolean isEquals) - { - if (isGt) - return isEquals ? Relation.RelationKind.GTE : Relation.RelationKind.GT; - else - return isEquals ? Relation.RelationKind.LTE : Relation.RelationKind.LT; - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/operations/Relation.java b/test/harry/main/org/apache/cassandra/harry/operations/Relation.java deleted file mode 100644 index e9571251928e..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/operations/Relation.java +++ /dev/null @@ -1,299 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.operations; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.gen.DataGenerators; - -public class Relation -{ - public final RelationKind kind; - public final ColumnSpec<?> columnSpec; - // Theoretically, in model, we'll just be able to compare stuff according to relation, and pass it to DB - public long descriptor; - - Relation(RelationKind kind, - ColumnSpec<?> columnSpec, - long descriptor) - { - this.kind = kind; - this.columnSpec = columnSpec; - this.descriptor = descriptor; - } - - public boolean match(long l) - { - // TODO: there are == NULL queries - if (l == DataGenerators.NIL_DESCR || l == DataGenerators.UNSET_DESCR) - return false; - - return kind.match(columnSpec.type.generator()::compare, l, descriptor); - } - - public Object value() - { - return columnSpec.inflate(descriptor); - } - - public String column() - { - return columnSpec.name; - } - - public String toClause() - { - return kind.getClause(column()); - } - - public String toString() - { - return "Relation{" + - "kind=" + kind + - ", columnSpec=" + columnSpec + - ", descriptor=" + descriptor + " (" + Long.toHexString(descriptor) + ")" + - '}'; - } - - public static Relation relation(RelationKind kind, ColumnSpec<?> columnSpec, long descriptor) - { - return new Relation(kind, columnSpec, descriptor); - } - - public static Relation eqRelation(ColumnSpec<?> columnSpec, long descriptor) - { - return new Relation(RelationKind.EQ, columnSpec, descriptor); - } - - public static List<Relation> eqRelations(long[] key, List<ColumnSpec<?>> columnSpecs) - { - List<Relation> relations = new ArrayList<>(key.length); - addEqRelation(key, columnSpecs, relations); - return relations; - } - - public static void addEqRelation(long[] key, List<ColumnSpec<?>> columnSpecs, List<Relation> relations) - { - addRelation(key, columnSpecs, relations, RelationKind.EQ); - } - - public static void addRelation(long[] key, List<ColumnSpec<?>> columnSpecs, List<Relation> relations, RelationKind kind) - { - assert key.length == columnSpecs.size() || key.length > DataGenerators.KeyGenerator.MAX_UNIQUE_PREFIX_COLUMNS : - String.format("Key size (%d) should equal to column spec size (%d). Specs: %s", key.length, columnSpecs.size(), columnSpecs); - for (int i = 0; i < key.length; i++) - { - ColumnSpec<?> spec = columnSpecs.get(i); - relations.add(relation(kind, spec, key[i])); - } - } - - public enum RelationKind - { - LT - { - public boolean isNegatable() - { - return true; - } - - public boolean isInclusive() - { - return false; - } - - public RelationKind negate() - { - return GT; - } - - public long nextMatch(long n) - { - return Math.subtractExact(n, 1); - } - - public String toString() - { - return "<"; - } - - public boolean match(LongComparator comparator, long l, long r) - { - return comparator.compare(l, r) < 0; - } - }, - GT - { - public boolean isNegatable() - { - return true; - } - - public boolean isInclusive() - { - return false; - } - - public RelationKind negate() - { - return LT; - } - - public String toString() - { - return ">"; - } - - public boolean match(LongComparator comparator, long l, long r) - { - return comparator.compare(l, r) > 0; - } - - public long nextMatch(long n) - { - return Math.addExact(n, 1); - } - }, - LTE - { - public boolean isNegatable() - { - return true; - } - - public boolean isInclusive() - { - return true; - } - - public RelationKind negate() - { - return GTE; - } - - public String toString() - { - return "<="; - } - - public boolean match(LongComparator comparator, long l, long r) - { - return comparator.compare(l, r) <= 0; - } - - public long nextMatch(long n) - { - return Math.subtractExact(n, 1); - } - }, - GTE - { - public boolean isNegatable() - { - return true; - } - - public boolean isInclusive() - { - return true; - } - - public RelationKind negate() - { - return LTE; - } - - public String toString() - { - return ">="; - } - - public boolean match(LongComparator comparator, long l, long r) - { - return comparator.compare(l, r) >= 0; - } - - public long nextMatch(long n) - { - return Math.addExact(n, 1); - } - }, - EQ - { - public boolean isNegatable() - { - return false; - } - - public boolean isInclusive() - { - return true; - } - - public RelationKind negate() - { - throw new IllegalArgumentException("Cannot negate EQ"); - } - - public long nextMatch(long n) - { - return n; - } - - public String toString() - { - return "="; - } - - public boolean match(LongComparator comparator, long l, long r) - { - return comparator.compare(l, r) == 0; - } - }; - - public abstract boolean match(LongComparator comparator, long l, long r); - - public String getClause(String name) - { - return String.format("%s %s ?", name, toString()); - } - - public String getClause(ColumnSpec<?> spec) - { - return getClause(spec.name); - } - - public abstract boolean isNegatable(); - - public abstract boolean isInclusive(); - - public abstract RelationKind negate(); - - public abstract long nextMatch(long n); - } - - public static interface LongComparator - { - public int compare(long l, long r); - } - - public static LongComparator FORWARD_COMPARATOR = Long::compare; -} diff --git a/test/harry/main/org/apache/cassandra/harry/operations/WriteHelper.java b/test/harry/main/org/apache/cassandra/harry/operations/WriteHelper.java deleted file mode 100644 index 75921dfa47d0..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/operations/WriteHelper.java +++ /dev/null @@ -1,166 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.operations; - -import java.util.List; - -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.gen.DataGenerators; - -public class WriteHelper -{ - public static CompiledStatement inflateInsert(SchemaSpec schema, - long pd, - long cd, - long[] vds, - long[] sds, - long timestamp) - { - Object[] partitionKey = schema.inflatePartitionKey(pd); - Object[] clusteringKey = schema.inflateClusteringKey(cd); - Object[] staticColumns = sds == null ? null : schema.inflateStaticColumns(sds); - Object[] regularColumns = schema.inflateRegularColumns(vds); - - Object[] bindings = new Object[schema.allColumns.size()]; - - StringBuilder b = new StringBuilder(); - b.append("INSERT INTO ") - .append(schema.keyspace) - .append('.') - .append(schema.table) - .append(" ("); - - int bindingsCount = 0; - bindingsCount += appendStatements(b, bindings, schema.partitionKeys, partitionKey, bindingsCount, true, ",", "%s"); - bindingsCount += appendStatements(b, bindings, schema.clusteringKeys, clusteringKey, bindingsCount, false, ",", "%s"); - bindingsCount += appendStatements(b, bindings, schema.regularColumns, regularColumns, bindingsCount, false, ",", "%s"); - if (staticColumns != null) - bindingsCount += appendStatements(b, bindings, schema.staticColumns, staticColumns, bindingsCount, false, ",", "%s"); - - b.append(") VALUES ("); - - for (int i = 0; i < bindingsCount; i++) - { - if (i > 0) - b.append(", "); - b.append("?"); - } - - b.append(") USING TIMESTAMP ") - .append(timestamp) - .append(";"); - - return new CompiledStatement(b.toString(), adjustArraySize(bindings, bindingsCount)); - } - - public static Object[] adjustArraySize(Object[] bindings, int bindingsCount) - { - if (bindingsCount != bindings.length) - { - Object[] tmp = new Object[bindingsCount]; - System.arraycopy(bindings, 0, tmp, 0, bindingsCount); - bindings = tmp; - } - return bindings; - } - - public static CompiledStatement inflateUpdate(SchemaSpec schema, - long pd, - long cd, - long[] vds, - long[] sds, - long timestamp) - { - Object[] partitionKey = schema.inflatePartitionKey(pd); - Object[] clusteringKey = schema.inflateClusteringKey(cd); - Object[] staticColumns = sds == null ? null : schema.inflateStaticColumns(sds); - Object[] regularColumns = schema.inflateRegularColumns(vds); - - Object[] bindings = new Object[schema.allColumns.size()]; - - StringBuilder b = new StringBuilder(); - b.append("UPDATE ") - .append(schema.keyspace) - .append('.') - .append(schema.table) - .append(" USING TIMESTAMP ") - .append(timestamp) - .append(" SET "); - - int bindingsCount = 0; - bindingsCount += addSetStatements(b, bindings, schema.regularColumns, regularColumns, bindingsCount); - if (staticColumns != null) - bindingsCount += addSetStatements(b, bindings, schema.staticColumns, staticColumns, bindingsCount); - - assert bindingsCount > 0 : "Can not have an UPDATE statement without any updates"; - b.append(" WHERE "); - - bindingsCount += addWhereStatements(b, bindings, schema.partitionKeys, partitionKey, bindingsCount, true); - bindingsCount += addWhereStatements(b, bindings, schema.clusteringKeys, clusteringKey, bindingsCount, false); - b.append(";"); - return new CompiledStatement(b.toString(), adjustArraySize(bindings, bindingsCount)); - } - - private static int addSetStatements(StringBuilder b, - Object[] bindings, - List<ColumnSpec<?>> columns, - Object[] values, - int bound) - { - return appendStatements(b, bindings, columns, values, bound, bound == 0, ", ", "%s = ?"); - } - - private static int addWhereStatements(StringBuilder b, - Object[] bindings, - List<ColumnSpec<?>> columns, - Object[] values, - int bound, - boolean firstStatement) - { - return appendStatements(b, bindings, columns, values, bound, firstStatement, " AND ", "%s = ?"); - } - - private static int appendStatements(StringBuilder b, - Object[] allBindings, - List<ColumnSpec<?>> columns, - Object[] values, - int bound, - boolean firstStatement, - String separator, - String nameFormatter) - { - int bindingsCount = 0; - for (int i = 0; i < values.length; i++) - { - Object value = values[i]; - if (value == DataGenerators.UNSET_VALUE) - continue; - - ColumnSpec<?> column = columns.get(i); - if (bindingsCount > 0 || !firstStatement) - b.append(separator); - - b.append(String.format(nameFormatter, column.name)); - allBindings[bound + bindingsCount] = value; - bindingsCount++; - } - return bindingsCount; - } -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/runner/EarlyExitException.java b/test/harry/main/org/apache/cassandra/harry/runner/EarlyExitException.java deleted file mode 100644 index 76b4229fd8ea..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/runner/EarlyExitException.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.runner; - -public class EarlyExitException extends RuntimeException -{ - public EarlyExitException(String e) - { - super(e); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/runner/FlaggedRunner.java b/test/harry/main/org/apache/cassandra/harry/runner/FlaggedRunner.java deleted file mode 100644 index c2450cd8bd81..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/runner/FlaggedRunner.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.runner; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.cassandra.concurrent.ExecutorFactory; -import org.apache.cassandra.concurrent.Interruptible; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.visitors.Visitor; -import org.apache.cassandra.utils.concurrent.CountDownLatch; - -import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.Daemon.NON_DAEMON; -import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.Interrupts.UNSYNCHRONIZED; -import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.SimulatorSafe.SAFE; - -public class FlaggedRunner extends Runner -{ - private final List<Configuration.VisitorPoolConfiguration> poolConfigurations; - private final CountDownLatch stopLatch; - - public FlaggedRunner(Run run, Configuration config, List<Configuration.VisitorPoolConfiguration> poolConfigurations, CountDownLatch stopLatch) - { - super(run, config); - this.poolConfigurations = poolConfigurations; - this.stopLatch = stopLatch; - } - - @Override - protected void runInternal() throws Throwable - { - List<Interruptible> threads = new ArrayList<>(); - Map<String, Integer> counters = new HashMap<>(); - for (Configuration.VisitorPoolConfiguration poolConfiguration : poolConfigurations) - { - for (int i = 0; i < poolConfiguration.concurrency; i++) - { - Visitor visitor = poolConfiguration.visitor.make(run); - String name = String.format("%s-%d", poolConfiguration.prefix, i + 1); - counters.put(name, 0); - Interruptible thread = ExecutorFactory.Global.executorFactory().infiniteLoop(name, wrapInterrupt((state) -> { - if (state == Interruptible.State.NORMAL) - { - visitor.visit(); - counters.compute(name, (n, old) -> old + 1); - } - }, stopLatch::decrement, errors::add), SAFE, NON_DAEMON, UNSYNCHRONIZED); - threads.add(thread); - } - } - - stopLatch.await(); - shutdown(threads::stream); - System.out.println("counters = " + counters); - if (!errors.isEmpty()) - mergeAndThrow(errors); - } - - @Override - public String type() - { - return "concurrent_flagged"; - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/runner/HarryRunner.java b/test/harry/main/org/apache/cassandra/harry/runner/HarryRunner.java deleted file mode 100644 index 0bfbb31b663c..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/runner/HarryRunner.java +++ /dev/null @@ -1,115 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.runner; - -import java.io.File; -import java.io.FileNotFoundException; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.util.ThrowingRunnable; - -import static org.apache.cassandra.config.CassandraRelevantProperties.DISABLE_TCACTIVE_OPENSSL; -import static org.apache.cassandra.config.CassandraRelevantProperties.IO_NETTY_TRANSPORT_NONATIVE; -import static org.apache.cassandra.config.CassandraRelevantProperties.ORG_APACHE_CASSANDRA_DISABLE_MBEAN_REGISTRATION; - -public abstract class HarryRunner -{ - public static final Logger logger = LoggerFactory.getLogger(HarryRunner.class); - - public void run(Configuration config) throws Throwable - { - DISABLE_TCACTIVE_OPENSSL.setBoolean(true); - IO_NETTY_TRANSPORT_NONATIVE.setBoolean(true); - ORG_APACHE_CASSANDRA_DISABLE_MBEAN_REGISTRATION.setBoolean(true); - - Runner runner = config.createRunner(); - Run run = runner.getRun(); - - Throwable thrown = null; - - try - { - beforeRun(runner); - runner.run(); - } - catch (Throwable e) - { - logger.error("Failed due to exception: " + e.getMessage(), e); - thrown = e; - } - finally - { - logger.info("Shutting down runner.."); - boolean failed = thrown != null; - if (!failed) - { - logger.info("Shutting down cluster.."); - tryRun(run.sut::shutdown); - } - afterRun(runner, thrown); - logger.info("Exiting..."); - if (failed) - System.exit(1); - else - System.exit(0); - } - } - - public void tryRun(ThrowingRunnable runnable) - { - try - { - runnable.run(); - } - catch (Throwable t) - { - logger.error("Encountered an error while shutting down, ignoring.", t); - } - } - - /** - * Parses the command-line args and returns a File for the configuration YAML. - * @param args Command-line args. - * @return Configuration YAML file. - * @throws Exception If file is not found or cannot be read. - */ - public static File loadConfig(String... args) throws Exception { - if (args == null || args.length == 0) { - throw new Exception("Harry config YAML not provided."); - } - - File configFile = new File(args[0]); - if (!configFile.exists()) { - throw new FileNotFoundException(configFile.getAbsolutePath()); - } - - if (!configFile.canRead()) { - throw new Exception("Cannot read config file, check your permissions on " + configFile.getAbsolutePath()); - } - - return configFile; - } - - public abstract void beforeRun(Runner runner); - public abstract void afterRun(Runner runner, Object result); -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/runner/Runner.java b/test/harry/main/org/apache/cassandra/harry/runner/Runner.java deleted file mode 100644 index c62e1b671ae3..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/runner/Runner.java +++ /dev/null @@ -1,479 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.runner; - -import java.io.BufferedWriter; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStreamWriter; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.TimeUnit; -import java.util.function.Consumer; -import java.util.function.Supplier; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.concurrent.ExecutorFactory; -import org.apache.cassandra.concurrent.Interruptible; -import org.apache.cassandra.concurrent.Shutdownable; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.visitors.Visitor; -import org.apache.cassandra.utils.concurrent.WaitQueue; - -import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.Daemon.NON_DAEMON; -import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.Interrupts.UNSYNCHRONIZED; -import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.SimulatorSafe.SAFE; -import static org.apache.cassandra.utils.Clock.Global.nanoTime; - -public abstract class Runner -{ - private static final Logger logger = LoggerFactory.getLogger(Runner.class); - - protected final Run run; - protected final Configuration config; - - // If there's an error, there's a good chance we're going to hit it more than once - // since we have multiple concurrent checkers running - protected final List<Throwable> errors; - - public Runner(Run run, Configuration config) - { - this.run = run; - this.config = config; - this.errors = new CopyOnWriteArrayList<>(); - } - - public void run() throws Throwable - { - init(config, run); - try - { - runInternal(); - } - catch (EarlyExitException t) - { - logger.warn("Exiting early...", t); - } - teardown(); - } - - protected abstract void runInternal() throws Throwable; - - public Run getRun() - { - return run; - } - - public static void init(Configuration config, Run run) - { - if (config.create_schema) - { - if (config.keyspace_ddl == null) - run.sut.schemaChange("CREATE KEYSPACE IF NOT EXISTS " + run.schemaSpec.keyspace + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};"); - else - run.sut.schemaChange(config.keyspace_ddl); - - String schema = run.schemaSpec.compile().cql(); - logger.info("Creating table: " + schema); - run.sut.schemaChange(schema); - } - - if (config.truncate_table) - { - run.sut.schemaChange(String.format("truncate %s.%s;", - run.schemaSpec.keyspace, - run.schemaSpec.table)); - } - - run.sut.afterSchemaInit(); - - int res = run.sut.execute(String.format("SELECT * FROM %s.%s LIMIT 1", run.schemaSpec.keyspace, run.schemaSpec.table), SystemUnderTest.ConsistencyLevel.QUORUM).length; - if (res > 0) - { - System.out.println("========================================================================================================================"); - System.out.println("| |"); - System.out.println("| WARNING: Starting a run with non-empty tables! |"); - System.out.println("| |"); - System.out.println("========================================================================================================================"); - } - } - - public void teardown() - { - logger.info("Tearing down setup..."); - if (config.drop_schema) - { - if (!errors.isEmpty()) - { - logger.info("Preserving table {} due to errors during execution.", - run.schemaSpec.table); - return; - } - - logger.info("Dropping table: " + run.schemaSpec.table); - run.sut.schemaChange(String.format("DROP TABLE IF EXISTS %s.%s;", - run.schemaSpec.keyspace, - run.schemaSpec.table)); - } - } - - protected void maybeReportErrors() - { - if (!errors.isEmpty()) { - dumpStateToFile(run, config, errors); - } - } - - public abstract String type(); - - public interface RunnerFactory - { - Runner make(Run run, Configuration config); - } - - public abstract static class TimedRunner extends Runner - { - public final long runtime; - public final TimeUnit runtimeUnit; - - public TimedRunner(Run run, Configuration config, long runtime, TimeUnit runtimeUnit) - { - super(run, config); - - this.runtime = runtime; - this.runtimeUnit = runtimeUnit; - } - } - - public static Configuration.RunnerConfiguration single(Visitor.VisitorFactory visitor) - { - return (run, config) -> new SingleVisitRunner(run, config, Collections.singletonList(visitor)); - } - - public static class SingleVisitRunner extends Runner - { - private final List<Visitor> visitors; - - public SingleVisitRunner(Run run, - Configuration config, - List<? extends Visitor.VisitorFactory> visitorFactories) - { - super(run, config); - this.visitors = visitorFactories.stream().map(factory -> factory.make(run)).collect(Collectors.toList()); - } - - @Override - public String type() - { - return "single"; - } - - public void runInternal() - { - for (Visitor value : visitors) - value.visit(); - } - } - - public static Configuration.RunnerConfiguration sequential(Visitor.VisitorFactory visitor, - long runtime, TimeUnit runtimeUnit) - { - return (r, c) -> new SequentialRunner(r, c, Collections.singletonList(visitor), runtime, runtimeUnit); - } - - public static Runner sequential(Run run, - Configuration config, - List<Visitor.VisitorFactory> poolConfigurations, - long runtime, TimeUnit runtimeUnit) - { - return new SequentialRunner(run, config, poolConfigurations, runtime, runtimeUnit); - } - - /** - * Runs all visitors sequentially, in the loop, for a specified amount of time - */ - public static class SequentialRunner extends TimedRunner - { - protected final List<Visitor> visitors; - - public SequentialRunner(Run run, - Configuration config, - List<? extends Visitor.VisitorFactory> visitorFactories, - long runtime, TimeUnit runtimeUnit) - { - super(run, config, runtime, runtimeUnit); - - this.visitors = visitorFactories.stream().map(factory -> factory.make(run)).collect(Collectors.toList()); - } - - @Override - public String type() - { - return "sequential"; - } - - @Override - public void runInternal() throws Throwable - { - long deadline = nanoTime() + runtimeUnit.toNanos(runtime);; - while (nanoTime() < deadline) - { - for (Visitor visitor : visitors) - visitor.visit(); - } - } - } - - public static Runner concurrent(Configuration config, - List<Configuration.VisitorPoolConfiguration> poolConfigurations, - long runtime, TimeUnit runtimeUnit) - { - return new ConcurrentRunner(config.createRun(), config, poolConfigurations, runtime, runtimeUnit); - } - - /** - * Runs all visitors concurrently, each visitor in its own thread, looped, for a specified amount of time - */ - public static class ConcurrentRunner extends TimedRunner - { - private final List<Configuration.VisitorPoolConfiguration> poolConfigurations; - - public ConcurrentRunner(Run run, - Configuration config, - List<Configuration.VisitorPoolConfiguration> poolConfigurations, - long runtime, TimeUnit runtimeUnit) - { - super(run, config, runtime, runtimeUnit); - this.poolConfigurations = poolConfigurations; - } - - @Override - public String type() - { - return "concurrent"; - } - - public void runInternal() throws Throwable - { - List<Interruptible> threads = new ArrayList<>(); - WaitQueue queue = WaitQueue.newWaitQueue(); - WaitQueue.Signal interrupt = queue.register(); - - for (Configuration.VisitorPoolConfiguration poolConfiguration : poolConfigurations) - { - for (int i = 0; i < poolConfiguration.concurrency; i++) - { - Visitor visitor = poolConfiguration.visitor.make(run); - String name = String.format("%s-%d", poolConfiguration.prefix, i + 1); - Interruptible thread = ExecutorFactory.Global.executorFactory().infiniteLoop(name, wrapInterrupt((state) -> { - if (state == Interruptible.State.NORMAL) - visitor.visit(); - }, interrupt::signal, errors::add), SAFE, NON_DAEMON, UNSYNCHRONIZED); - threads.add(thread); - } - } - - interrupt.await(runtime, runtimeUnit); - shutdown(threads::stream); - if (!errors.isEmpty()) - mergeAndThrow(errors); - } - } - - public static Runner chain(Configuration config, - Configuration.RunnerConfiguration... runners) - { - return new ChainRunner(config.createRun(), config, Arrays.asList(runners)); - } - - /** - * Chain runner is somewhat similar to StagedRunner in that it would execute multiple runners one after - * another, but unlike StagedRunner, it is single-shot: it will just run them once. - */ - public static class ChainRunner extends Runner - { - private final List<Runner> stages; - - public ChainRunner(Run run, - Configuration config, - List<Configuration.RunnerConfiguration> runnerFactories) - { - super(run, config); - this.stages = new ArrayList<>(); - for (Configuration.RunnerConfiguration runner : runnerFactories) - stages.add(runner.make(run, config)); - } - - protected void runInternal() throws Throwable - { - for (Runner stage : stages) - stage.runInternal(); - } - - public String type() - { - return "chain"; - } - } - - public static void shutdown(Supplier<Stream<Interruptible>> threads) - { - threads.get().forEach(Shutdownable::shutdown); - long deadline = nanoTime(); - threads.get().forEach((interruptible) -> { - try - { - if (!interruptible.awaitTermination(nanoTime() - deadline, TimeUnit.NANOSECONDS)) - logger.info("Could not terminate before the timeout: " + threads.get().map(Shutdownable::isTerminated).collect(Collectors.toList())); - } - catch (InterruptedException e) - { - throw new RuntimeException(e); - } - }); - - } - - public static Interruptible.Task wrapInterrupt(Interruptible.Task runnable, Runnable signalStop, Consumer<Throwable> registerException) - { - return (state) -> { - try - { - runnable.run(state); - } - catch (InterruptedException t) - { - signalStop.run(); - } - catch (Throwable t) - { - // Since some of the exceptions are thrown from inside instances - if (!t.getClass().toString().contains("InterruptedException")) - registerException.accept(t); - signalStop.run(); - } - }; - } - - public static void mergeAndThrow(List<Throwable> existingFail) - { - List<Throwable> skipped = existingFail.stream().filter(e -> e instanceof EarlyExitException).collect(Collectors.toList()); - for (Throwable throwable : skipped) - { - logger.warn("Skipping exit early exceptions", throwable); - return; - } - - List<Throwable> errors = existingFail.stream().filter(e -> !(e instanceof EarlyExitException)).collect(Collectors.toList()); - if (errors.size() == 1) - throw new RuntimeException("Interrupting run because of an exception", errors.get(0)); - - Throwable e = errors.get(0); - Throwable ret = e; - for (int i = 1; i < errors.size(); i++) - { - Throwable current = errors.get(i); - e.addSuppressed(current); - e = current; - } - - throw new RuntimeException("Interrupting run because of an exception", ret); - } - - private static void dumpExceptionToFile(BufferedWriter bw, Throwable throwable) throws IOException - { - if (throwable.getMessage() != null) - bw.write(throwable.getMessage()); - else - bw.write("<no message>"); - - bw.newLine(); - for (StackTraceElement line : throwable.getStackTrace()) - { - bw.write(line.toString()); - bw.newLine(); - } - bw.newLine(); - if (throwable.getCause() != null) - { - bw.write("Inner Exception: "); - dumpExceptionToFile(bw, throwable.getCause()); - } - - bw.newLine(); - bw.newLine(); - } - - public static void dumpStateToFile(Run run, Configuration config, List<Throwable> t) - { - try - { - File f = new File("failure.dump"); - logger.error("Dumping results into the file:" + f); - try (BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(new FileOutputStream(f)))) - { - bw.write("Caught exception during the run: "); - for (Throwable throwable : t) - dumpExceptionToFile(bw, throwable); - - bw.flush(); - } - - File file = new File("run.yaml"); - Configuration.ConfigurationBuilder builder = config.unbuild(); - - // override stateful components - builder.setClock(run.clock.toConfig()); - builder.setDataTracker(run.tracker.toConfig()); - - Configuration.toFile(file, builder.build()); - } - - catch (Throwable e) - { - logger.error("Caught an error while trying to dump to file", e); - try - { - File f = new File("tmp.dump"); - - if (!f.createNewFile()) - logger.info("File {} already exists. Appending...", f); - - BufferedWriter tmp = new BufferedWriter(new OutputStreamWriter(new FileOutputStream(f))); - dumpExceptionToFile(tmp, e); - } - catch (IOException ex) - { - ex.printStackTrace(); - } - - throw new RuntimeException(e); - } - } -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/runner/StagedRunner.java b/test/harry/main/org/apache/cassandra/harry/runner/StagedRunner.java deleted file mode 100644 index bac36bfa2513..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/runner/StagedRunner.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.runner; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.TimeUnit; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; - -import static org.apache.cassandra.utils.Clock.Global.nanoTime; - -public class StagedRunner extends Runner.TimedRunner -{ - public static final String TYPE = "staged"; - - private final List<Runner> stages; - - public StagedRunner(Run run, - Configuration config, - List<Configuration.RunnerConfiguration> runnerFactories, - long runtime, TimeUnit runtimeUnit) - { - super(run, config, runtime, runtimeUnit); - this.stages = new ArrayList<>(); - for (Configuration.RunnerConfiguration runner : runnerFactories) - stages.add(runner.make(run, config)); - } - - @Override - public String type() { - return TYPE; - } - - @Override - protected void runInternal() throws Throwable - { - long deadline = nanoTime() + runtimeUnit.toNanos(nanoTime()); - - while (nanoTime() < deadline) - { - for (Runner runner : stages) - runner.runInternal(); - } - } - - @JsonTypeName(TYPE) - public static class StagedRunnerConfig implements Configuration.RunnerConfiguration - { - @JsonProperty(value = "stages") - public final List<Configuration.RunnerConfiguration> runnerFactories; - - public final long run_time; - public final TimeUnit run_time_unit; - - @JsonCreator - public StagedRunnerConfig(@JsonProperty(value = "stages") List<Configuration.RunnerConfiguration> stages, - @JsonProperty(value = "run_time", defaultValue = "2") long runtime, - @JsonProperty(value = "run_time_unit", defaultValue = "HOURS") TimeUnit runtimeUnit) - { - this.runnerFactories = stages; - this.run_time = runtime; - this.run_time_unit = runtimeUnit; - } - - @Override - public Runner make(Run run, Configuration config) - { - return new StagedRunner(run, config, runnerFactories, run_time, run_time_unit); - } - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/runner/TrivialShrinker.java b/test/harry/main/org/apache/cassandra/harry/runner/TrivialShrinker.java deleted file mode 100644 index a1bdc01af214..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/runner/TrivialShrinker.java +++ /dev/null @@ -1,189 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.runner; - -import java.io.File; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; -import java.util.function.Predicate; - -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.visitors.LtsVisitor; -import org.apache.cassandra.harry.visitors.SkippingVisitor; -import org.apache.cassandra.harry.visitors.Visitor; - -/** - * A most trivial imaginable shrinker: attempts to skip partitions and/or logical timestamps to see if the - * issue is still reproducible. - */ -public class TrivialShrinker -{ - public static void main(String[] args) throws Throwable - { - try - { - File configFile = HarryRunner.loadConfig(args); - Configuration configuration = Configuration.fromFile(configFile); - System.out.println(Configuration.toYamlString(configuration)); - - Set<Long> pdsToSkip = new HashSet<>(Arrays.asList( - // put pds you want to skip here, or Harry will find them for you - )); - - Set<Long> ltsToSkip = new HashSet<>(Arrays.asList( - // put lts you want to skip here, or Harry will find them for you - )); - - // Which LTS failure has occurred on - final long maxLts = 7000L; - - // Check if we've found exactly the exception that is causing the failure - Predicate<Exception> check = (e) -> true; - - Run run = configuration.createRun(); - Configuration.SequentialRunnerConfig config = (Configuration.SequentialRunnerConfig) configuration.runner; - List<Visitor> visitors = new ArrayList<>(); - for (Configuration.VisitorConfiguration factory : config.visitorFactories) - { - Visitor visitor = factory.make(run); - if (visitor instanceof LtsVisitor) - { - AtomicLong counter = new AtomicLong(); - visitors.add(new SkippingVisitor((LtsVisitor) visitor, - counter::getAndIncrement, - (lts) -> run.pdSelector.pd(lts, run.schemaSpec), - ltsToSkip, - pdsToSkip)) ; - } - else - { - visitors.add(visitor); - } - } - - Set<Long> partitions = new HashSet<>(); - for (long i = 0; i < maxLts; i++) - partitions.add(run.pdSelector.pd(i, run.schemaSpec)); - - // Step one: figure out which partitions we can skip while still keeping it reproducible - for (Long pdToCheck : partitions) - { - if (pdsToSkip.contains(pdToCheck)) - continue; - pdsToSkip.add(pdToCheck); - Runner.init(configuration, run); - - try - { - runOnce(visitors, maxLts); - System.out.println("Can not skip " + pdToCheck + "\nCan only skip these: " + toString(pdsToSkip)); - pdsToSkip.remove(pdToCheck); - } - catch (RuntimeException t) - { - if (check.test(t)) - { - System.out.printf("Safe to skip: %d because without it we're still hitting an exception %s.\n%s\n", - pdToCheck, - t.getMessage(), - toString(pdsToSkip)); - } - else - { - System.out.println("Can not skip " + pdToCheck + "\n, since we seem to repro a different issue. Can only skip these: " + toString(pdsToSkip)); - pdsToSkip.remove(pdToCheck); - } - } - run.sut.schemaChange("DROP KEYSPACE " + run.schemaSpec.keyspace); - } - - // Step two: figure out which lts can be skipped within the remaining partitions - for (long lts = 0; lts < maxLts; lts++) - { - long ltsToCheck = lts; - if (ltsToSkip.contains(ltsToCheck) || pdsToSkip.contains(run.pdSelector.pd(lts, run.schemaSpec))) - continue; - ltsToSkip.add(ltsToCheck); - Runner.init(configuration, run); - - try - { - runOnce(visitors, maxLts); - System.out.println("Can not skip " + ltsToCheck + "\nCan only skip these: " + toString(ltsToSkip)); - ltsToSkip.remove(ltsToCheck); - } - catch (RuntimeException t) - { - if (check.test(t)) - { - System.out.printf("Safe to skip: %d because without it we're still hitting an exception %s.\n%s\n", - ltsToCheck, - t.getMessage(), - toString(ltsToSkip)); - } - else - { - System.out.println("Can not skip " + lts + "\n, since we seem to repro a different issue. Can only skip these: " + toString(ltsToSkip)); - ltsToSkip.remove(ltsToCheck); - } - - } - run.sut.schemaChange("DROP KEYSPACE " + run.schemaSpec.keyspace); - } - } - catch (Throwable t) - { - System.out.println(t.getMessage()); - t.printStackTrace(); - } - finally - { - System.exit(1); - } - } - - public static void runOnce(List<Visitor> visitors, long maxLts) - { - for (long lts = 0; lts <= maxLts; lts++) - { - for (Visitor visitor : visitors) - { - visitor.visit(); - } - } - } - - public static String toString(Set<Long> longs) - { - if (longs.isEmpty()) - return ""; - - String s = ""; - for (Long aLong : longs) - { - s += aLong + "L,"; - } - return s.substring(0, s.length() - 1); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/runner/UpToLtsRunner.java b/test/harry/main/org/apache/cassandra/harry/runner/UpToLtsRunner.java deleted file mode 100644 index 9fc2cec5aa4b..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/runner/UpToLtsRunner.java +++ /dev/null @@ -1,111 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.runner; - -import java.util.Collections; -import java.util.List; -import java.util.concurrent.TimeUnit; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.visitors.Visitor; - -import static org.apache.cassandra.utils.Clock.Global.nanoTime; - -/** - * Runner that allows to run for a specific number of logical timestamps, rather - * than being configured by time. - */ -public class UpToLtsRunner extends Runner.SequentialRunner -{ - public static final String TYPE = "up_to_lts"; - - private final long maxLts; - - public static Configuration.RunnerConfiguration factory(List<? extends Visitor.VisitorFactory> visitorFactories, - long maxLts, - long runtime, TimeUnit runtimeUnit) - { - return (r, c) -> new UpToLtsRunner(r, c, visitorFactories, maxLts, runtime, runtimeUnit); - } - - public static Configuration.RunnerConfiguration factory(Visitor.VisitorFactory visitorFactory, - long maxLts, - long runtime, TimeUnit runtimeUnit) - { - return (r, c) -> new UpToLtsRunner(r, c, Collections.singletonList(visitorFactory), maxLts, runtime, runtimeUnit); - } - - public UpToLtsRunner(Run run, - Configuration config, - List<? extends Visitor.VisitorFactory> visitorFactories, - long maxLts, - long runtime, TimeUnit runtimeUnit) - { - super(run, config, visitorFactories, runtime, runtimeUnit); - this.maxLts = maxLts; - } - - @Override - public String type() - { - return TYPE; - } - - @Override - public void runInternal() - { - long deadline = nanoTime() + runtimeUnit.toNanos(runtime); - while (run.tracker.maxStarted() < maxLts && nanoTime() < deadline) - { - for (Visitor visitor : visitors) - visitor.visit(); - } - } - - @JsonTypeName(TYPE) - public static class UpToLtsRunnerConfig implements Configuration.RunnerConfiguration - { - public final List<Configuration.VisitorConfiguration> visitor_factories; - public final long max_lts; - public final long run_time; - public final TimeUnit run_time_unit; - - @JsonCreator - public UpToLtsRunnerConfig(@JsonProperty(value = "visitors") List<Configuration.VisitorConfiguration> visitors, - @JsonProperty(value = "max_lts") long maxLts, - @JsonProperty(value = "run_time", defaultValue = "2") long runtime, - @JsonProperty(value = "run_time_unit", defaultValue = "HOURS") TimeUnit runtimeUnit) - { - this.visitor_factories = visitors; - this.max_lts = maxLts; - this.run_time = runtime; - this.run_time_unit = runtimeUnit; - } - - @Override - public Runner make(Run run, Configuration config) - { - return new UpToLtsRunner(run, config, visitor_factories, max_lts, run_time, run_time_unit); - } - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/sut/DoubleWritingSut.java b/test/harry/main/org/apache/cassandra/harry/sut/DoubleWritingSut.java deleted file mode 100644 index 5c516f93ec82..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/sut/DoubleWritingSut.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.sut; - -import java.util.concurrent.CompletableFuture; -import java.util.regex.Pattern; - -public class DoubleWritingSut implements SystemUnderTest -{ - private final SystemUnderTest primary; - private final SystemUnderTest secondary; - - public DoubleWritingSut(SystemUnderTest primary, - SystemUnderTest secondary) - { - this.primary = primary; - this.secondary = secondary; - } - public boolean isShutdown() - { - return primary.isShutdown(); - } - - public void shutdown() - { - primary.shutdown(); - } - - private static final Pattern pattern = Pattern.compile("^select", Pattern.CASE_INSENSITIVE); - - public Object[][] execute(String statement, ConsistencyLevel cl, Object... bindings) - { - if (pattern.matcher(statement).find()) - return primary.execute(statement, cl, bindings); - - secondary.execute(statement, cl, bindings); - return primary.execute(statement, cl, bindings); - } - - public CompletableFuture<Object[][]> executeAsync(String statement, ConsistencyLevel cl, Object... bindings) - { - throw new UnsupportedOperationException("Not implemented (yet)"); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/sut/PrintlnSut.java b/test/harry/main/org/apache/cassandra/harry/sut/PrintlnSut.java deleted file mode 100644 index 8d9196370a6d..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/sut/PrintlnSut.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.sut; - -import java.util.Arrays; -import java.util.concurrent.CompletableFuture; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonTypeName; -import org.apache.cassandra.harry.core.Configuration; - -public class PrintlnSut implements SystemUnderTest -{ - public boolean isShutdown() - { - return false; - } - - public void shutdown() - { - } - - public Object[][] execute(String statement, ConsistencyLevel cl, Object... bindings) - { - System.out.println(String.format("%s | %s | %s", - statement, - cl, - Arrays.toString(bindings))); - return new Object[0][]; - } - - public CompletableFuture<Object[][]> executeAsync(String statement, ConsistencyLevel cl, Object... bindings) - { - return CompletableFuture.supplyAsync(() -> execute(statement, cl, bindings), - Runnable::run); - } - - @JsonTypeName("println") - public static class PrintlnSutConfiguration implements Configuration.SutConfiguration - { - @JsonCreator - public PrintlnSutConfiguration() - { - - } - public SystemUnderTest make() - { - return new PrintlnSut(); - } - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/sut/QueryModifyingSut.java b/test/harry/main/org/apache/cassandra/harry/sut/QueryModifyingSut.java deleted file mode 100644 index aae913d8ed41..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/sut/QueryModifyingSut.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.sut; - -import java.util.concurrent.CompletableFuture; - -import org.apache.cassandra.harry.model.AgainstSutChecker; - -/** - * Best thing for sanity checking of tricky range tombstone issues: - * write to memtable instead of writing to the usual system under test. - * - * Usually used in conjunction with {@link AgainstSutChecker}: - * - * SchemaSpec doubleWriteSchema = schema.cloneWithName(schema.keyspace, schema.keyspace + "_debug"); - * - * sut.schemaChange(doubleWriteSchema.compile().cql()); - * - * QueryModifyingSut sut = new QueryModifyingSut(this.sut, - * schema.table, - * doubleWriteSchema.table); - * - * - * Model model = new AgainstSutChecker(tracker, history.clock(), sut, schema, doubleWriteSchema); - */ -public class QueryModifyingSut implements SystemUnderTest -{ - private final SystemUnderTest delegate; - private final String toReplace; - private final String replacement; - - public QueryModifyingSut(SystemUnderTest delegate, - String toReplace, - String replacement) - { - this.delegate = delegate; - this.toReplace = toReplace; - this.replacement = replacement; - } - - public boolean isShutdown() - { - return delegate.isShutdown(); - } - - public void shutdown() - { - delegate.shutdown(); - } - - public Object[][] execute(String statement, ConsistencyLevel cl, Object... bindings) - { - return delegate.execute(statement.replaceAll(toReplace, replacement), - cl, - bindings); - } - - public CompletableFuture<Object[][]> executeAsync(String statement, ConsistencyLevel cl, Object... bindings) - { - return delegate.executeAsync(statement.replaceAll(toReplace, replacement), - cl, - bindings); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/sut/SystemUnderTest.java b/test/harry/main/org/apache/cassandra/harry/sut/SystemUnderTest.java deleted file mode 100644 index bc96e07b729e..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/sut/SystemUnderTest.java +++ /dev/null @@ -1,105 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.sut; - -import java.util.concurrent.CompletableFuture; - -import org.apache.cassandra.harry.operations.CompiledStatement; - -public interface SystemUnderTest -{ - public interface SUTFactory - { - public SystemUnderTest make(); - } - - public boolean isShutdown(); - - public void shutdown(); - - default void afterSchemaInit() - { - } - - default void schemaChange(String statement) - { - execute(statement, ConsistencyLevel.ALL, new Object[]{}); - } - - Object[][] execute(String statement, ConsistencyLevel cl, Object... bindings); - - default Object[][] execute(CompiledStatement statement, ConsistencyLevel cl) - { - return execute(statement.cql(), cl, statement.bindings()); - } - - default Object[][] executeIdempotent(String statement, ConsistencyLevel cl, Object... bindings) - { - return execute(statement, cl, bindings); - } - - CompletableFuture<Object[][]> executeAsync(String statement, ConsistencyLevel cl, Object... bindings); - - default CompletableFuture<Object[][]> executeIdempotentAsync(String statement, ConsistencyLevel cl, Object... bindings) - { - return executeAsync(statement, cl, bindings); - } - - interface SystemUnderTestFactory - { - SystemUnderTest create(); - } - - enum ConsistencyLevel { - ALL, QUORUM, NODE_LOCAL, ONE - } - - public static final SystemUnderTest NO_OP = new NoOpSut(); - - public class NoOpSut implements SystemUnderTest - { - private NoOpSut() {} - public boolean isShutdown() - { - return false; - } - - public void shutdown() - { - } - - public Object[][] execute(String statement, ConsistencyLevel cl, Object... bindings) - { - return new Object[0][]; - } - - public CompletableFuture<Object[][]> executeAsync(String statement, ConsistencyLevel cl, Object... bindings) - { - return CompletableFuture.supplyAsync(() -> execute(statement, cl, bindings), - Runnable::run); - } - } - - public static interface FaultInjectingSut extends SystemUnderTest - { - public Object[][] executeWithWriteFailure(String statement, ConsistencyLevel cl, Object... bindings); - public CompletableFuture<Object[][]> executeAsyncWithWriteFailure(String statement, ConsistencyLevel cl, Object... bindings); - } - -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/sut/injvm/ClusterState.java b/test/harry/main/org/apache/cassandra/harry/sut/injvm/ClusterState.java deleted file mode 100644 index 97886edda808..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/sut/injvm/ClusterState.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.sut.injvm; - -public interface ClusterState -{ - boolean isDown(int i); -} diff --git a/test/harry/main/org/apache/cassandra/harry/sut/injvm/ExistingClusterSUT.java b/test/harry/main/org/apache/cassandra/harry/sut/injvm/ExistingClusterSUT.java deleted file mode 100644 index 27d0c97ab65c..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/sut/injvm/ExistingClusterSUT.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.sut.injvm; - -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; - -import com.google.common.util.concurrent.Uninterruptibles; - -import org.apache.cassandra.distributed.api.ICluster; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.sut.SystemUnderTest; - -public class ExistingClusterSUT implements Configuration.SutConfiguration -{ - private final ICluster cluster; - private final ClusterState clusterState; - - public ExistingClusterSUT(ICluster cluster, ClusterState clusterState) - { - this.cluster = cluster; - this.clusterState = clusterState; - } - - @Override - public SystemUnderTest make() - { - return new SystemUnderTest() - { - int toQuery = 0; - @Override - public boolean isShutdown() - { - return false; - } - - @Override - public void shutdown() - { - } - - @Override - public void schemaChange(String schemaChange) - { - cluster.schemaChange(schemaChange); - } - - @Override - public Object[][] execute(String s, SystemUnderTest.ConsistencyLevel consistencyLevel, Object... objects) - { - Exception lastEx = null; - for (int i = 0; i < 20; i++) - { - toQuery++; - if (cluster.size() == 0) - continue; - int coordinator = (toQuery % cluster.size()) + 1; - if (clusterState.isDown(coordinator)) - continue; - try - { - return cluster.coordinator(coordinator).execute(s, org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM, objects); - } - catch (Exception e) - { - lastEx = e; - Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS); - } - } - throw new RuntimeException(lastEx); - } - - @Override - public CompletableFuture<Object[][]> executeAsync(String s, SystemUnderTest.ConsistencyLevel consistencyLevel, Object... objects) - { - return CompletableFuture.supplyAsync(() -> this.execute(s, consistencyLevel, objects)); - } - }; - } -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/sut/injvm/InJVMTokenAwareVisitExecutor.java b/test/harry/main/org/apache/cassandra/harry/sut/injvm/InJVMTokenAwareVisitExecutor.java deleted file mode 100644 index 5db0ee795d4c..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/sut/injvm/InJVMTokenAwareVisitExecutor.java +++ /dev/null @@ -1,165 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.sut.injvm; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.function.Function; - -import com.google.common.util.concurrent.Uninterruptibles; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import org.apache.cassandra.distributed.api.ICoordinator; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.operations.CompiledStatement; -import org.apache.cassandra.harry.util.ByteUtils; -import org.apache.cassandra.harry.util.TokenUtil; -import org.apache.cassandra.harry.visitors.GeneratingVisitor; -import org.apache.cassandra.harry.visitors.LoggingVisitor; -import org.apache.cassandra.harry.visitors.OperationExecutor; -import org.apache.cassandra.harry.visitors.VisitExecutor; -import org.apache.cassandra.harry.visitors.Visitor; -import org.apache.cassandra.distributed.api.ConsistencyLevel; -import org.apache.cassandra.distributed.api.IInstance; - -import static org.apache.cassandra.harry.sut.TokenPlacementModel.peerStateToNodes; - -public class InJVMTokenAwareVisitExecutor extends LoggingVisitor.LoggingVisitorExecutor -{ - private static final Logger logger = LoggerFactory.getLogger(InJVMTokenAwareVisitExecutor.class); - - private final InJvmSut sut; - private final TokenPlacementModel.ReplicationFactor rf; - private final SystemUnderTest.ConsistencyLevel cl; - private final SchemaSpec schema; - private final int MAX_RETRIES = 10; - - public static Function<Run, VisitExecutor> factory(OperationExecutor.RowVisitorFactory rowVisitorFactory, - SystemUnderTest.ConsistencyLevel cl, - TokenPlacementModel.ReplicationFactor rf) - { - return (run) -> new InJVMTokenAwareVisitExecutor(run, rowVisitorFactory, cl, rf); - } - - public InJVMTokenAwareVisitExecutor(Run run, - OperationExecutor.RowVisitorFactory rowVisitorFactory, - SystemUnderTest.ConsistencyLevel cl, - TokenPlacementModel.ReplicationFactor rf) - { - super(run, rowVisitorFactory.make(run)); - this.sut = (InJvmSut) run.sut; - this.schema = run.schemaSpec; - this.cl = cl; - this.rf = rf; - } - - @Override - protected Object[][] executeWithRetries(long lts, long pd, CompiledStatement statement) - { - if (sut.isShutdown()) - throw new IllegalStateException("System under test is shut down"); - - int retries = 0; - - Object[] pk = schema.inflatePartitionKey(pd); - List<TokenPlacementModel.Replica> replicas = getRing().replicasFor(TokenUtil.token(ByteUtils.compose(ByteUtils.objectsToBytes(pk)))); - while (retries++ < MAX_RETRIES) - { - try - { - TokenPlacementModel.Replica replica = replicas.get((int) (lts % replicas.size())); - if (cl == SystemUnderTest.ConsistencyLevel.NODE_LOCAL) - { - return executeNodeLocal(statement.cql(), replica.node(), statement.bindings()); - } - else - { - return sut.cluster - .stream() - .filter((n) -> n.config().broadcastAddress().toString().contains(replica.node().id())) - .findFirst() - .get() - .coordinator() - .execute(statement.cql(), InJvmSut.toApiCl(cl), statement.bindings()); - } - } - catch (Throwable t) - { - int delaySecs = 1; - logger.error(String.format("Caught message while trying to execute: %s. Scheduled to retry in %s seconds", statement, delaySecs), t); - Uninterruptibles.sleepUninterruptibly(delaySecs, TimeUnit.SECONDS); - } - } - throw new IllegalStateException(String.format("Can not execute statement %s after %d retries", statement, retries)); - } - - protected TokenPlacementModel.ReplicatedRanges getRing() - { - ICoordinator coordinator = sut.firstAlive().coordinator(); - List<TokenPlacementModel.Node> other = peerStateToNodes(coordinator.execute("select peer, tokens, data_center, rack from system.peers", ConsistencyLevel.ONE)); - List<TokenPlacementModel.Node> self = peerStateToNodes(coordinator.execute("select broadcast_address, tokens, data_center, rack from system.local", ConsistencyLevel.ONE)); - List<TokenPlacementModel.Node> all = new ArrayList<>(); - all.addAll(self); - all.addAll(other); - all.sort(TokenPlacementModel.Node::compareTo); - return rf.replicate(all); - } - - protected Object[][] executeNodeLocal(String statement, TokenPlacementModel.Node node, Object... bindings) - { - IInstance instance = sut.cluster - .stream() - .filter((n) -> n.config().broadcastAddress().toString().contains(node.id())) - .findFirst() - .get(); - return instance.executeInternal(statement, bindings); - } - - - @JsonTypeName("in_jvm_token_aware") - public static class Configuration implements org.apache.cassandra.harry.core.Configuration.VisitorConfiguration - { - public final org.apache.cassandra.harry.core.Configuration.RowVisitorConfiguration row_visitor; - public final SystemUnderTest.ConsistencyLevel consistency_level; - public final int rf; - @JsonCreator - public Configuration(@JsonProperty("row_visitor") org.apache.cassandra.harry.core.Configuration.RowVisitorConfiguration rowVisitor, - @JsonProperty("consistency_level") SystemUnderTest.ConsistencyLevel consistencyLevel, - @JsonProperty("rf") int rf) - { - this.row_visitor = rowVisitor; - this.consistency_level = consistencyLevel; - this.rf = rf; - } - - @Override - public Visitor make(Run run) - { - return new GeneratingVisitor(run, new InJVMTokenAwareVisitExecutor(run, row_visitor, consistency_level, new TokenPlacementModel.SimpleReplicationFactor(rf))); - } - } -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/sut/injvm/InJvmSut.java b/test/harry/main/org/apache/cassandra/harry/sut/injvm/InJvmSut.java deleted file mode 100644 index 16f5e13436cd..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/sut/injvm/InJvmSut.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.sut.injvm; - -import java.io.File; -import java.io.IOException; -import java.util.Arrays; -import java.util.function.Consumer; -import java.util.function.Function; -import java.util.function.Supplier; -import java.util.stream.Collectors; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.db.Keyspace; -import org.apache.cassandra.dht.Token; -import org.apache.cassandra.distributed.Cluster; -import org.apache.cassandra.distributed.api.IInstanceConfig; -import org.apache.cassandra.distributed.api.IInvokableInstance; -import org.apache.cassandra.locator.EndpointsForToken; -import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.tcm.ClusterMetadata; - -public class InJvmSut extends InJvmSutBase<IInvokableInstance, Cluster> -{ - private static final Logger logger = LoggerFactory.getLogger(InJvmSut.class); - - public InJvmSut(Cluster cluster) - { - super(cluster, roundRobin(cluster), retryOnTimeout(), 10); - } - - public InJvmSut(Cluster cluster, Supplier<Integer> loadBalancingStrategy, Function<Throwable, Boolean> retryStrategy) - { - super(cluster, loadBalancingStrategy, retryStrategy, 10); - } - - @JsonTypeName("in_jvm") - public static class InJvmSutConfiguration extends InJvmSutBaseConfiguration<IInvokableInstance, Cluster> - { - @JsonCreator - public InJvmSutConfiguration(@JsonProperty(value = "nodes", defaultValue = "3") int nodes, - @JsonProperty(value = "worker_threads", defaultValue = "10") int worker_threads, - @JsonProperty("root") String root) - { - super(nodes, worker_threads, root); - } - - protected Cluster cluster(Consumer<IInstanceConfig> cfg, int nodes, File root) - { - try - { - return Cluster.build().withConfig(cfg) - .withNodes(nodes) - .withRoot(root) - .createWithoutStarting(); - } - catch (IOException e) - { - throw new IllegalStateException(e); - } - } - - protected InJvmSutBase<IInvokableInstance, Cluster> sut(Cluster cluster) - { - return new InJvmSut(cluster); - } - } - - // TODO: this would only return _read_ (or natural) replicas for the token - public int[] getReadReplicasFor(Object[] partitionKey, String keyspace, String table) - { - return cluster.get(1).appliesOnInstance(InJvmSut::getReadReplicasForCallable).apply(partitionKey, keyspace, table); - } - - public static int[] getReadReplicasForCallable(Object[] pk, String ks, String table) - { - String pkString = Arrays.stream(pk).map(Object::toString).collect(Collectors.joining(":")); - EndpointsForToken endpoints = StorageService.instance.getNaturalReplicasForToken(ks, table, pkString); - int[] nodes = new int[endpoints.size()]; - for (int i = 0; i < endpoints.size(); i++) - nodes[i] = endpoints.get(i).endpoint().getAddress().getAddress()[3]; - - sanity_check: - { - Keyspace ksp = Keyspace.open(ks); - Token token = DatabaseDescriptor.getPartitioner().getToken(ksp.getMetadata().getTableOrViewNullable(table).partitionKeyType.fromString(pkString)); - - ClusterMetadata metadata = ClusterMetadata.current(); - EndpointsForToken replicas = metadata.placements.get(ksp.getMetadata().params.replication).reads.forToken(token).get(); - - assert replicas.endpoints().equals(endpoints.endpoints()) : String.format("Consistent metadata endpoints %s disagree with token metadata computation %s", endpoints.endpoints(), replicas.endpoints()); - } - return nodes; - } -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/sut/injvm/InJvmSutBase.java b/test/harry/main/org/apache/cassandra/harry/sut/injvm/InJvmSutBase.java deleted file mode 100644 index 4c19e672943a..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/sut/injvm/InJvmSutBase.java +++ /dev/null @@ -1,331 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.sut.injvm; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.util.Arrays; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; -import java.util.function.Consumer; -import java.util.function.Function; -import java.util.function.Supplier; - -import com.google.common.collect.Iterators; -import org.apache.commons.lang3.StringUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.cassandra.distributed.api.ICluster; -import org.apache.cassandra.distributed.api.IInstance; -import org.apache.cassandra.distributed.api.IInstanceConfig; -import org.apache.cassandra.distributed.api.IMessage; -import org.apache.cassandra.distributed.api.IMessageFilters; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.sut.SystemUnderTest; - -public class InJvmSutBase<NODE extends IInstance, CLUSTER extends ICluster<NODE>> implements SystemUnderTest.FaultInjectingSut -{ - private static final Logger logger = LoggerFactory.getLogger(InJvmSutBase.class); - - private final ExecutorService executor; - public final CLUSTER cluster; - private final AtomicBoolean isShutdown = new AtomicBoolean(false); - private final Supplier<Integer> loadBalancingStrategy; - private final Function<Throwable, Boolean> retryStrategy; - - public InJvmSutBase(CLUSTER cluster) - { - this(cluster, roundRobin(cluster), retryOnTimeout(), 10); - } - - public InJvmSutBase(CLUSTER cluster, Supplier<Integer> loadBalancingStrategy, Function<Throwable, Boolean> retryStrategy, int threads) - { - this.cluster = cluster; - this.executor = Executors.newFixedThreadPool(threads); - this.loadBalancingStrategy = loadBalancingStrategy; - this.retryStrategy = retryStrategy; - } - - public static Supplier<Integer> roundRobin(ICluster<?> cluster) - { - return new Supplier<Integer>() - { - private final AtomicLong cnt = new AtomicLong(); - - public Integer get() - { - for (int i = 0; i < 42; i++) - { - int selected = (int) (cnt.getAndIncrement() % cluster.size() + 1); - if (!cluster.get(selected).isShutdown()) - return selected; - } - throw new IllegalStateException("Unable to find an alive instance"); - } - }; - } - - public static Function<Throwable, Boolean> retryOnTimeout() - { - return new Function<Throwable, Boolean>() - { - public Boolean apply(Throwable t) - { - return t.getMessage().contains("timed out"); - } - }; - } - public CLUSTER cluster() - { - return cluster; - } - - @Override - public boolean isShutdown() - { - return isShutdown.get(); - } - - @Override - public void shutdown() - { - assert isShutdown.compareAndSet(false, true); - - try - { - cluster.close(); - executor.shutdown(); - if (!executor.awaitTermination(30, TimeUnit.SECONDS)) - throw new TimeoutException("Could not terminate cluster within expected timeout"); - } - catch (Throwable e) - { - logger.error("Could not terminate cluster.", e); - throw new RuntimeException(e); - } - } - - public void schemaChange(String statement) - { - cluster.schemaChange(statement); - } - - public IInstance firstAlive() - { - return cluster.stream().filter(i -> !i.isShutdown()).findFirst().get(); - } - - public Object[][] execute(String statement, ConsistencyLevel cl, int pageSize, Object... bindings) - { - return execute(statement, cl, loadBalancingStrategy.get(), pageSize, bindings); - } - - public Object[][] execute(String statement, ConsistencyLevel cl, Object... bindings) - { - return execute(statement, cl, loadBalancingStrategy.get(), 1, bindings); - } - - public Object[][] execute(String statement, ConsistencyLevel cl, int coordinator, int pageSize, Object... bindings) - { - if (isShutdown.get()) - throw new RuntimeException("Instance is shut down"); - - while (true) - { - try - { - if (cl == ConsistencyLevel.NODE_LOCAL) - { - return cluster.get(coordinator) - .executeInternal(statement, bindings); - } - else if (StringUtils.startsWithIgnoreCase(statement, "SELECT")) - { - return Iterators.toArray(cluster - // round-robin - .coordinator(coordinator) - .executeWithPaging(statement, toApiCl(cl), pageSize, bindings), - Object[].class); - } - else - { - return cluster - // round-robin - .coordinator(coordinator) - .execute(statement, toApiCl(cl), bindings); - } - } - catch (Throwable t) - { - if (retryStrategy.apply(t)) - continue; - - logger.error(String.format("Caught error while trying execute statement %s (%s): %s", - statement, Arrays.toString(bindings), t.getMessage()), - t); - throw t; - } - } - } - - // TODO: Ideally, we need to be able to induce a failure of a single specific message - public Object[][] executeWithWriteFailure(String statement, ConsistencyLevel cl, Object... bindings) - { - if (isShutdown.get()) - throw new RuntimeException("Instance is shut down"); - - try - { - int coordinator = loadBalancingStrategy.get(); - IMessageFilters filters = cluster.filters(); - - // Drop exactly one coordinated message - int MUTATION_REQ = 0; - // TODO: make dropping deterministic - filters.verbs(MUTATION_REQ).from(coordinator).messagesMatching(new IMessageFilters.Matcher() - { - private final AtomicBoolean issued = new AtomicBoolean(); - public boolean matches(int from, int to, IMessage message) - { - if (from != coordinator || message.verb() != MUTATION_REQ) - return false; - - return !issued.getAndSet(true); - } - }).drop().on(); - Object[][] res = cluster - .coordinator(coordinator) - .execute(statement, toApiCl(cl), bindings); - filters.reset(); - return res; - } - catch (Throwable t) - { - logger.error(String.format("Caught error while trying execute statement %s", statement), - t); - throw t; - } - } - - public CompletableFuture<Object[][]> executeAsync(String statement, ConsistencyLevel cl, Object... bindings) - { - return CompletableFuture.supplyAsync(() -> execute(statement, cl, bindings), executor); - } - - public CompletableFuture<Object[][]> executeAsyncWithWriteFailure(String statement, ConsistencyLevel cl, Object... bindings) - { - return CompletableFuture.supplyAsync(() -> executeWithWriteFailure(statement, cl, bindings), executor); - } - - public static abstract class InJvmSutBaseConfiguration<NODE extends IInstance, CLUSTER extends ICluster<NODE>> implements Configuration.SutConfiguration - { - public final int nodes; - public final int worker_threads; - public final String root; - - @JsonCreator - public InJvmSutBaseConfiguration(@JsonProperty(value = "nodes", defaultValue = "3") int nodes, - @JsonProperty(value = "worker_threads", defaultValue = "10") int worker_threads, - @JsonProperty("root") String root) - { - this.nodes = nodes; - this.worker_threads = worker_threads; - if (root == null) - { - try - { - this.root = Files.createTempDirectory("cluster_" + nodes + "_nodes").toString(); - } - catch (IOException e) - { - throw new IllegalArgumentException(e); - } - } - else - { - this.root = root; - } - } - - protected abstract CLUSTER cluster(Consumer<IInstanceConfig> cfg, int nodes, File root); - protected abstract InJvmSutBase<NODE, CLUSTER> sut(CLUSTER cluster); - - public SystemUnderTest make() - { - try - { - ICluster.setup(); - } - catch (Throwable throwable) - { - throw new RuntimeException(throwable); - } - - CLUSTER cluster; - - cluster = cluster(defaultConfig(), - nodes, - new File(root)); - - cluster.startup(); - return sut(cluster); - } - } - - public static Consumer<IInstanceConfig> defaultConfig() - { - return (cfg) -> { - cfg.set("row_cache_size", "50MiB") - .set("index_summary_capacity", "50MiB") - .set("counter_cache_size", "50MiB") - .set("key_cache_size", "50MiB") - .set("file_cache_size", "50MiB") - .set("index_summary_capacity", "50MiB") - .set("memtable_heap_space", "128MiB") - .set("memtable_offheap_space", "128MiB") - .set("memtable_flush_writers", 1) - .set("concurrent_compactors", 1) - .set("concurrent_reads", 5) - .set("concurrent_writes", 5) - .set("compaction_throughput_mb_per_sec", 10) - .set("hinted_handoff_enabled", false); - }; - } - - public static org.apache.cassandra.distributed.api.ConsistencyLevel toApiCl(ConsistencyLevel cl) - { - switch (cl) - { - case ALL: return org.apache.cassandra.distributed.api.ConsistencyLevel.ALL; - case QUORUM: return org.apache.cassandra.distributed.api.ConsistencyLevel.QUORUM; - case NODE_LOCAL: return org.apache.cassandra.distributed.api.ConsistencyLevel.NODE_LOCAL; - case ONE: return org.apache.cassandra.distributed.api.ConsistencyLevel.ONE; - } - throw new IllegalArgumentException("Don't know a CL: " + cl); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/sut/injvm/QuiescentLocalStateChecker.java b/test/harry/main/org/apache/cassandra/harry/sut/injvm/QuiescentLocalStateChecker.java deleted file mode 100644 index f9ee0d012585..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/sut/injvm/QuiescentLocalStateChecker.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.sut.injvm; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.cassandra.distributed.api.ICoordinator; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.model.QuiescentLocalStateCheckerBase; -import org.apache.cassandra.harry.model.reconciler.Reconciler; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.tracker.DataTracker; -import org.apache.cassandra.harry.util.ByteUtils; -import org.apache.cassandra.harry.util.TokenUtil; -import org.apache.cassandra.distributed.api.ConsistencyLevel; -import org.apache.cassandra.distributed.api.IInstance; - -public class QuiescentLocalStateChecker extends QuiescentLocalStateCheckerBase -{ - public static ModelFactory factory(TokenPlacementModel.ReplicationFactor rf) - { - return (run) -> new QuiescentLocalStateChecker(run, rf); - } - - public QuiescentLocalStateChecker(Run run, TokenPlacementModel.ReplicationFactor rf) - { - super(run, rf); - } - - public QuiescentLocalStateChecker(OpSelectors.Clock clock, - OpSelectors.PdSelector pdSelector, - SystemUnderTest sut, - DataTracker tracker, - SchemaSpec schema, - Reconciler reconciler, - TokenPlacementModel.ReplicationFactor rf) - { - super(clock, pdSelector, sut, tracker, schema, reconciler, rf); - } - - @Override - protected TokenPlacementModel.ReplicatedRanges getRing() - { - IInstance node = ((InJvmSutBase<?, ?>) sut).firstAlive(); - ICoordinator coordinator = node.coordinator(); - List<TokenPlacementModel.Node> other = TokenPlacementModel.peerStateToNodes(coordinator.execute("select peer, tokens, data_center, rack from system.peers", ConsistencyLevel.ONE)); - List<TokenPlacementModel.Node> self = TokenPlacementModel.peerStateToNodes(coordinator.execute("select broadcast_address, tokens, data_center, rack from system.local", ConsistencyLevel.ONE)); - List<TokenPlacementModel.Node> all = new ArrayList<>(); - all.addAll(self); - all.addAll(other); - all.sort(TokenPlacementModel.Node::compareTo); - return rf.replicate(all); - } - - @Override - protected Object[][] executeNodeLocal(String statement, TokenPlacementModel.Node node, Object... bindings) - { - IInstance instance = ((InJvmSutBase<?, ?>) sut).cluster - .stream() - .filter((n) -> n.config().broadcastAddress().toString().contains(node.id())) - .findFirst() - .get(); - return instance.executeInternal(statement, bindings); - } - - protected long token(long pd) - { - return TokenUtil.token(ByteUtils.compose(ByteUtils.objectsToBytes(schema.inflatePartitionKey(pd)))); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/test/HistoryBuilderTest.java b/test/harry/main/org/apache/cassandra/harry/test/HistoryBuilderTest.java new file mode 100644 index 000000000000..ec864b10e46d --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/test/HistoryBuilderTest.java @@ -0,0 +1,328 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.test; + +import java.util.Arrays; +import java.util.List; + +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.harry.ColumnSpec; +import org.apache.cassandra.harry.MagicConstants; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.checker.ModelChecker; +import org.apache.cassandra.harry.dsl.HistoryBuilder; +import org.apache.cassandra.harry.dsl.HistoryBuilderHelper; +import org.apache.cassandra.harry.execution.CQLTesterVisitExecutor; +import org.apache.cassandra.harry.execution.CQLVisitExecutor; +import org.apache.cassandra.harry.execution.DataTracker; +import org.apache.cassandra.harry.gen.Generator; +import org.apache.cassandra.harry.gen.Generators; +import org.apache.cassandra.harry.gen.SchemaGenerators; +import org.apache.cassandra.harry.model.QuiescentChecker; +import org.apache.cassandra.harry.op.Operations; +import org.apache.cassandra.harry.op.Visit; + +import static org.apache.cassandra.harry.Relations.RelationKind.GTE; +import static org.apache.cassandra.harry.Relations.RelationKind.LTE; +import static org.apache.cassandra.harry.checker.TestHelper.withRandom; +import static org.apache.cassandra.harry.dsl.SingleOperationBuilder.IdxRelation; + +public class HistoryBuilderTest extends CQLTester +{ + // TODO: go through all basic features of History builder here and test them!!! + // TODO: for example, inverse + private static final int STEPS_PER_ITERATION = 1_000; + + private static final Logger logger = LoggerFactory.getLogger(HistoryBuilderTest.class); + + private final Generator<SchemaSpec> simple_schema = rng -> { + return new SchemaSpec(rng.next(), + 1000, + KEYSPACE, + "harry" + rng.nextLong(0, Long.MAX_VALUE), + Arrays.asList(ColumnSpec.pk("pk1", ColumnSpec.asciiType), + ColumnSpec.pk("pk2", ColumnSpec.int64Type)), + Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.asciiType, false), + ColumnSpec.ck("ck2", ColumnSpec.int64Type, false)), + Arrays.asList(ColumnSpec.regularColumn("r1", ColumnSpec.asciiType), + ColumnSpec.regularColumn("r2", ColumnSpec.int64Type), + ColumnSpec.regularColumn("r3", ColumnSpec.asciiType)), + Arrays.asList(ColumnSpec.staticColumn("s1", ColumnSpec.asciiType), + ColumnSpec.staticColumn("s2", ColumnSpec.int64Type), + ColumnSpec.staticColumn("s3", ColumnSpec.asciiType))); + }; + + private final Generator<SchemaSpec> simple_schema_with_desc_ck = rng -> { + return new SchemaSpec(rng.next(), + 1000, + KEYSPACE, + "harry" + rng.nextLong(0, Long.MAX_VALUE), + Arrays.asList(ColumnSpec.pk("pk1", ColumnSpec.asciiType), + ColumnSpec.pk("pk2", ColumnSpec.int64Type)), + Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.asciiType, true), + ColumnSpec.ck("ck2", ColumnSpec.int64Type, false)), + Arrays.asList(ColumnSpec.regularColumn("r1", ColumnSpec.asciiType), + ColumnSpec.regularColumn("r2", ColumnSpec.int64Type), + ColumnSpec.regularColumn("r3", ColumnSpec.asciiType)), + Arrays.asList(ColumnSpec.staticColumn("s1", ColumnSpec.asciiType), + ColumnSpec.staticColumn("s2", ColumnSpec.int64Type), + ColumnSpec.staticColumn("s3", ColumnSpec.asciiType))); + }; + + @Test + public void orderByTest() + { + withRandom(rng -> { + for (Generator<SchemaSpec> gen : new Generator[]{ simple_schema, simple_schema_with_desc_ck }) + { + SchemaSpec schema = gen.generate(rng); + createTable(schema.compile()); + + HistoryBuilder history = new HistoryBuilder(schema.valueGenerators); + for (int i = 0; i < 100; i++) + history.insert(1); + + history.custom((lts, opId) -> new Operations.SelectPartition(lts, + history.valueGenerators().pkGen().descriptorAt(1), + Operations.ClusteringOrderBy.DESC)); + + replay(schema, history); + } + }); + } + + @Test + public void historyBuilderInsertTest() + { + withRandom(rng -> { + for (Generator<SchemaSpec> gen : new Generator[]{ simple_schema, simple_schema_with_desc_ck }) + { + SchemaSpec schema = gen.generate(rng); + createTable(schema.compile()); + + HistoryBuilder history = new HistoryBuilder(schema.valueGenerators); + for (int i = 0; i < 100; i++) + history.insert(1, i, values(i, i, i), values(i, i, i)); + + history.selectPartition(1); + + replay(schema, history); + } + }); + } + + @Test + public void historyBuilderInsertWithUnsetTest() + { + withRandom(rng -> { + for (Generator<SchemaSpec> gen : new Generator[]{ simple_schema, simple_schema_with_desc_ck }) + { + SchemaSpec schema = gen.generate(rng); + createTable(schema.compile()); + + HistoryBuilder history = new HistoryBuilder(schema.valueGenerators); + for (int i = 0; i < 100; i++) + { + int v = i % 2 == 0 ? MagicConstants.UNSET_IDX : i; + history.insert(1, i, values(v, v, v), values(v, v, v)); + } + + history.selectPartition(1); + + replay(schema, history); + } + }); + } + + @Test + public void historyBuilderFilteringTest() + { + withRandom(rng -> { + for (Generator<SchemaSpec> gen : new Generator[]{ simple_schema, simple_schema_with_desc_ck }) + { + for (boolean useUnset : new boolean[]{ false, true }) + { + SchemaSpec schema = gen.generate(rng); + createTable(schema.compile()); + + HistoryBuilder history = new HistoryBuilder(schema.valueGenerators); + for (int i = 0; i < 100; i++) + { + int v = (useUnset && i % 2 == 0) ? MagicConstants.UNSET_IDX : i; + history.insert(1, i, values(v, v, v), values(v, v, v)); + } + + history.select(1, + Arrays.asList().toArray(new IdxRelation[0]), + Arrays.asList(new IdxRelation(GTE, 20, 0), + new IdxRelation(LTE, 80, 0), + new IdxRelation(GTE, 30, 1), + new IdxRelation(LTE, 70, 1), + new IdxRelation(GTE, 40, 2), + new IdxRelation(LTE, 60, 2)) + .toArray(new IdxRelation[0]), + Arrays.asList().toArray(new IdxRelation[0])); + + replay(schema, history); + } + } + }); + } + + @Test + public void testSimpleFuzz() + { + Generator<SchemaSpec> schemaGen = SchemaGenerators.schemaSpecGen(KEYSPACE, "harry", 100); + withRandom(rng -> { + SchemaSpec schema = schemaGen.generate(rng); + // Generate at most X values, but not more than entropy allows + int maxPartitions = Math.min(1, schema.valueGenerators.pkPopulation()); + int maxPartitionSize = Math.min(100, schema.valueGenerators.ckPopulation()); + + Generator<Integer> partitionPicker = Generators.pick(0, maxPartitions); + Generator<Integer> rowPicker = Generators.int32(0, maxPartitionSize); + ModelChecker<HistoryBuilder, Void> modelChecker = new ModelChecker<>(); + HistoryBuilder historyBuilder = new HistoryBuilder(schema.valueGenerators); + modelChecker.init(historyBuilder) + .step((history, rng_) -> { + HistoryBuilderHelper.insertRandomData(schema, partitionPicker.generate(rng), rowPicker.generate(rng), rng, history); + history.selectPartition(partitionPicker.generate(rng)); + }) + .step((history, rng_) -> { + history.deleteRow(partitionPicker.generate(rng), rowPicker.generate(rng)); + history.selectPartition(partitionPicker.generate(rng)); + }) + .step((history, rng_) -> { + history.deletePartition(partitionPicker.generate(rng)); + history.selectPartition(partitionPicker.generate(rng)); + }) + .step((history, rng_) -> { + HistoryBuilderHelper.deleteRandomColumns(schema, partitionPicker.generate(rng), rowPicker.generate(rng), rng, history); + history.selectPartition(partitionPicker.generate(rng)); + }) + .step((history, rng_) -> { + history.deleteRowRange(partitionPicker.generate(rng), + rowPicker.generate(rng), + rowPicker.generate(rng), + rng.nextInt(schema.clusteringKeys.size()), + rng.nextBoolean(), + rng.nextBoolean() + ); + history.selectPartition(partitionPicker.generate(rng)); + }) + .step((history, rng_) -> { + history.selectRow(partitionPicker.generate(rng), rowPicker.generate(rng)); + }) + .step((history, rng_) -> { + history.selectRowRange(partitionPicker.generate(rng), + rowPicker.generate(rng), + rowPicker.generate(rng), + rng.nextInt(schema.clusteringKeys.size()), + rng.nextBoolean(), + rng.nextBoolean()); + }) + .step((history, rng_) -> { + history.custom(() -> flush(schema.keyspace, schema.table), "FLUSH"); + }) + .exitCondition((history) -> { + if (historyBuilder.size() < 1000) + return false; + + createTable(schema.compile()); + replay(schema, historyBuilder); + + return true; + }) + .run(STEPS_PER_ITERATION, Long.MAX_VALUE, rng); + }); + } + + @Test + public void fuzzFiltering() + { + Generator<SchemaSpec> schemaGen = SchemaGenerators.schemaSpecGen(KEYSPACE, "fuzz_filtering", 100); + withRandom(rng -> { + SchemaSpec schema = schemaGen.generate(rng); + // Generate at most X values, but not more than entropy allows + int maxPartitions = Math.min(1, schema.valueGenerators.ckPopulation()); + int maxPartitionSize = Math.min(100, schema.valueGenerators.ckPopulation()); + + Generator<Integer> pkGen = Generators.int32(0, Math.min(schema.valueGenerators.pkPopulation(), maxPartitionSize)); + Generator<Integer> ckGen = Generators.int32(0, Math.min(schema.valueGenerators.ckPopulation(), maxPartitionSize)); + + ModelChecker<HistoryBuilder, Void> modelChecker = new ModelChecker<>(); + HistoryBuilder historyBuilder = new HistoryBuilder(schema.valueGenerators); + + modelChecker.init(historyBuilder) + .step((history, rng_) -> HistoryBuilderHelper.insertRandomData(schema, pkGen, ckGen, rng, history)) + .step((history, rng_) -> { + for (int i = 0; i < 10; i++) + { + List<IdxRelation> ckRelations = HistoryBuilderHelper.generateClusteringRelations(rng, schema.clusteringKeys.size(), ckGen); + List<IdxRelation> regularRelations = HistoryBuilderHelper.generateValueRelations(rng, schema.regularColumns.size(), + column -> Math.min(schema.valueGenerators.regularPopulation(column), maxPartitionSize)); + List<IdxRelation> staticRelations = HistoryBuilderHelper.generateValueRelations(rng, schema.staticColumns.size(), + column -> Math.min(schema.valueGenerators.staticPopulation(column), maxPartitionSize)); + history.select(rng.nextInt(maxPartitions), + ckRelations.toArray(new IdxRelation[0]), + regularRelations.toArray(new IdxRelation[0]), + staticRelations.toArray(new IdxRelation[0])); + } + }) + .exitCondition((history) -> { + if (historyBuilder.size() < 10) + return false; + + createTable(schema.compile()); + replay(schema, historyBuilder); + + return true; + }) + .run(STEPS_PER_ITERATION, Long.MAX_VALUE, rng); + }); + } + + public void replay(SchemaSpec schema, HistoryBuilder historyBuilder) + { + CQLVisitExecutor executor = create(schema, historyBuilder); + for (Visit visit : historyBuilder) + executor.execute(visit); + } + + public CQLVisitExecutor create(SchemaSpec schema, HistoryBuilder historyBuilder) + { + DataTracker tracker = new DataTracker.SequentialDataTracker(); + return new CQLTesterVisitExecutor(schema, tracker, + new QuiescentChecker(schema.valueGenerators, tracker, historyBuilder), + statement -> { + if (logger.isTraceEnabled()) + logger.trace(statement.toString()); + return execute(statement.cql(), statement.bindings()); + }); + } + + public int[] values(int... values) + { + return values; + } +} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/test/QueryBuilderTest.java b/test/harry/main/org/apache/cassandra/harry/test/QueryBuilderTest.java new file mode 100644 index 000000000000..55759019a4e2 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/test/QueryBuilderTest.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.test; + +import org.junit.Assert; +import org.junit.Test; + +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.execution.CompiledStatement; +import org.apache.cassandra.harry.execution.QueryBuildingVisitExecutor; +import org.apache.cassandra.harry.gen.SchemaGenerators; +import org.apache.cassandra.harry.op.Operations; +import org.apache.cassandra.harry.op.Visit; + +import static org.apache.cassandra.harry.checker.TestHelper.withRandom; + +public class QueryBuilderTest +{ + @Test + public void testQueryBuilder() + { + withRandom(rng -> { + SchemaSpec schemaSpec = SchemaGenerators.trivialSchema("harry", "simplified", 10).generate(rng); + QueryBuildingVisitExecutor queryBuilder = new QueryBuildingVisitExecutor(schemaSpec, (v, q) -> String.format("__START__\n%s\n__END__;", q)); + CompiledStatement compiled = queryBuilder.compile(new Visit(1, + new Operations.Operation[]{ new Operations.SelectPartition(1, 1L) })); + Assert.assertTrue(compiled.cql().contains("SELECT")); + Assert.assertTrue(compiled.cql().contains("__START__")); + Assert.assertTrue(compiled.cql().contains("__END__")); + }); + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/test/SimpleBijectionTest.java b/test/harry/main/org/apache/cassandra/harry/test/SimpleBijectionTest.java new file mode 100644 index 000000000000..fde20aa363f3 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/test/SimpleBijectionTest.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.test; + +import accord.utils.Invariants; +import org.apache.cassandra.harry.ColumnSpec; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.dsl.HistoryBuilder; +import org.apache.cassandra.harry.gen.InvertibleGenerator; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.apache.cassandra.harry.checker.TestHelper.withRandom; +import static org.apache.cassandra.harry.gen.InvertibleGenerator.MAX_ENTROPY; + +public class SimpleBijectionTest +{ + @Test + public void testOrder() + { + withRandom(rng -> { + for (ColumnSpec.DataType<?> t : ColumnSpec.TYPES) + { + for (ColumnSpec.DataType type : new ColumnSpec.DataType[]{ t, ColumnSpec.ReversedType.cache.get(t) }) + { + ColumnSpec<Object> column = (ColumnSpec<Object>) ColumnSpec.regularColumn("regular", type); + InvertibleGenerator<Object> generator = InvertibleGenerator.fromType(rng,100, column); + + + Object previous = null; + for (int i = 0; i < generator.population(); i++) + { + Object next = generator.inflate(generator.descriptorAt(i)); + if (previous != null) + { + Invariants.checkState(column.type.comparator().compare(next, previous) > 0, + "%s should be > %s", next, previous); + } + previous = next; + } + } + } + }); + } + + @Test + public void testArrayOrder() + { + withRandom(rng -> { + for (boolean[] order : new boolean[][]{ { false, false }, { true, false }, { false, true }, { true, true } }) + { + List<ColumnSpec<?>> columns = new ArrayList<>(); + for (int i = 0; i < order.length; i++) + columns.add(ColumnSpec.ck("test", ColumnSpec.asciiType, order[i])); + InvertibleGenerator<Object[]> generator = new InvertibleGenerator<>(rng, + MAX_ENTROPY, + 100, + SchemaSpec.forKeys(columns), + (Object[] a, Object[] b) -> HistoryBuilder.compareKeys(columns, a, b)); + Object[] previous = null; + for (int i = 0; i < 100; i++) + { + long descr = generator.descriptorAt(i); + Object[] next = generator.inflate(descr); + if (previous != null) + Assert.assertTrue( HistoryBuilder.compareKeys(columns, next, previous) > 0); + Assert.assertEquals(descr, generator.deflate(next)); + previous = next; + } + } + }); + } + + // TODO (now): negative tests +} diff --git a/test/harry/main/org/apache/cassandra/harry/test/TestStateTest.java b/test/harry/main/org/apache/cassandra/harry/test/TestStateTest.java new file mode 100644 index 000000000000..f63717e7d28e --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/test/TestStateTest.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.test; + +import java.util.Arrays; + +import org.junit.Assert; +import org.junit.Test; + +import static org.apache.cassandra.harry.ColumnSpec.booleanType; +import static org.apache.cassandra.harry.ColumnSpec.int64Type; +import static org.apache.cassandra.harry.ColumnSpec.int8Type; +import static org.apache.cassandra.harry.ColumnSpec.regularColumn; +import static org.apache.cassandra.harry.gen.InvertibleGenerator.MAX_ENTROPY; +import static org.apache.cassandra.harry.SchemaSpec.cumulativeEntropy; + +public class TestStateTest +{ + @Test + public void testCumulativeEntropy() + { + Assert.assertEquals(512, + cumulativeEntropy(Arrays.asList(regularColumn("test", int8Type), + regularColumn("test", booleanType)))); + + Assert.assertEquals(256*256, + cumulativeEntropy(Arrays.asList(regularColumn("test", int8Type), + regularColumn("test", int8Type)))); + + Assert.assertEquals(MAX_ENTROPY, + cumulativeEntropy(Arrays.asList(regularColumn("test", int64Type), + regularColumn("test", int8Type)))); + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/tracker/DataTracker.java b/test/harry/main/org/apache/cassandra/harry/tracker/DataTracker.java deleted file mode 100644 index 86c7bca6138d..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/tracker/DataTracker.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.tracker; - -import java.util.function.LongConsumer; - -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.OpSelectors; - -public interface DataTracker -{ - void onLtsStarted(LongConsumer onLts); - void onLtsFinished(LongConsumer onLts); - - void beginModification(long lts); - void endModification(long lts); - - default void beginValidation(long pd) {} - default void endValidation(long pd) {} - - long maxStarted(); - boolean isFinished(long lts); - - Configuration.DataTrackerConfiguration toConfig(); - - interface DataTrackerFactory - { - DataTracker make(OpSelectors.PdSelector pdSelector, SchemaSpec schemaSpec); - } - - DataTracker NO_OP = new NoOpDataTracker(); - - class NoOpDataTracker implements DataTracker - { - private NoOpDataTracker() {} - - public void onLtsStarted(LongConsumer onLts){} - public void onLtsFinished(LongConsumer onLts){} - - public void beginModification(long lts){} - public void endModification(long lts){} - - public long maxStarted() { return 0; } - public boolean isFinished(long lts) { return false; } - - public Configuration.DataTrackerConfiguration toConfig(){ return null; } - } - -} diff --git a/test/harry/main/org/apache/cassandra/harry/tracker/DefaultDataTracker.java b/test/harry/main/org/apache/cassandra/harry/tracker/DefaultDataTracker.java deleted file mode 100644 index 2d6eae610a9f..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/tracker/DefaultDataTracker.java +++ /dev/null @@ -1,193 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.tracker; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.PriorityBlockingQueue; -import java.util.concurrent.atomic.AtomicLong; -import java.util.function.LongConsumer; - -import org.apache.cassandra.harry.core.Configuration; -import com.google.common.annotations.VisibleForTesting; -import org.apache.cassandra.utils.concurrent.WaitQueue; - -public class DefaultDataTracker implements DataTracker -{ - protected final AtomicLong maxSeenLts; - protected final AtomicLong maxCompleteLts; - protected final PriorityBlockingQueue<Long> reorderBuffer; - protected final DrainReorderQueueTask reorderTask; - - protected List<LongConsumer> onStarted = new ArrayList<>(); - protected List<LongConsumer> onFinished = new ArrayList<>(); - - public DefaultDataTracker() - { - this.maxSeenLts = new AtomicLong(-1); - this.maxCompleteLts = new AtomicLong(-1); - this.reorderBuffer = new PriorityBlockingQueue<>(100); - this.reorderTask = new DrainReorderQueueTask(); - this.reorderTask.start(); - } - - @Override - public void onLtsStarted(LongConsumer onLts) - { - this.onStarted.add(onLts); - } - - @Override - public void onLtsFinished(LongConsumer onLts) - { - this.onFinished.add(onLts); - } - - @Override - public void beginModification(long lts) - { - assert lts >= 0; - startedInternal(lts); - for (LongConsumer consumer : onStarted) - consumer.accept(lts); - } - - @Override - public void endModification(long lts) - { - finishedInternal(lts); - for (LongConsumer consumer : onFinished) - consumer.accept(lts); - } - - void startedInternal(long lts) - { - recordEvent(lts, false); - } - - void finishedInternal(long lts) - { - recordEvent(lts, true); - } - - private void recordEvent(long lts, boolean finished) - { - // all seen LTS are allowed to be "in-flight" - maxSeenLts.getAndUpdate((old) -> Math.max(lts, old)); - - if (!finished) - return; - - if (!maxCompleteLts.compareAndSet(lts - 1, lts)) - reorderBuffer.offer(lts); - - reorderTask.notify.signalAll(); - } - - private class DrainReorderQueueTask extends Thread - { - private final WaitQueue notify; - - private DrainReorderQueueTask() - { - super("DrainReorderQueueTask"); - this.notify = WaitQueue.newWaitQueue(); - } - - public void run() - { - while (!Thread.interrupted()) - { - try - { - WaitQueue.Signal signal = notify.register(); - runOnce(); - signal.awaitUninterruptibly(); - } - catch (Throwable t) - { - t.printStackTrace(); - } - } - } - - public void runOnce() - { - long maxAchievedConsecutive = maxCompleteLts.get(); - - Long smallest = reorderBuffer.peek(); - while (smallest != null && smallest == maxAchievedConsecutive + 1) - { - boolean res = maxCompleteLts.compareAndSet(maxAchievedConsecutive, smallest); - assert res : String.format("Should have exclusive access to maxCompleteLts, but someone wrote %d, while %d was expected", maxCompleteLts.get(), maxAchievedConsecutive); - maxAchievedConsecutive = smallest; - long removed = reorderBuffer.remove(); - assert smallest == removed : String.format("Tried to remove %d but removed %d", smallest, removed); - smallest = reorderBuffer.peek(); - } - } - } - - - public long maxStarted() - { - return maxSeenLts.get(); - } - - public long maxConsecutiveFinished() - { - return maxCompleteLts.get(); - } - - public boolean isFinished(long lts) - { - // Since we _first_ add the item to maxConsecutive, and only then yank it from reorderBuffer, - // it may happen that we have checked for lts against maxConsecutive while it was still in reorderBuffer - // but then, by the time we check for it in the reorderBuffer, it is already removed; - return reorderBuffer.contains(lts) || lts <= maxConsecutiveFinished(); - } - - public Configuration.DataTrackerConfiguration toConfig() - { - return new Configuration.DefaultDataTrackerConfiguration(maxSeenLts.get(), maxCompleteLts.get(), new ArrayList<>(reorderBuffer)); - } - - @VisibleForTesting - public void forceLts(long maxSeen, long maxComplete, List<Long> reorderBuffer) - { - System.out.printf("Forcing maxSeen: %d, maxComplete: %d, reorderBuffer: %s%n", maxSeen, maxComplete, reorderBuffer); - this.maxSeenLts.set(maxSeen); - this.maxCompleteLts.set(maxComplete); - if (reorderBuffer != null) - { - reorderBuffer.sort(Long::compareTo); - this.reorderBuffer.addAll(reorderBuffer); - } - } - - public String toString() - { - List<Long> buf = new ArrayList<>(reorderBuffer); - return "DataTracker{" + - "maxSeenLts=" + maxSeenLts + - ", maxCompleteLts=" + maxCompleteLts + - ", reorderBuffer=" + buf + - '}'; - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/tracker/LockingDataTracker.java b/test/harry/main/org/apache/cassandra/harry/tracker/LockingDataTracker.java deleted file mode 100644 index 3edcf525f5dc..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/tracker/LockingDataTracker.java +++ /dev/null @@ -1,286 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.tracker; - -import java.util.ArrayList; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentSkipListSet; -import java.util.concurrent.atomic.AtomicLongFieldUpdater; - -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.utils.concurrent.WaitQueue; - -/** - * Locking data tracker, that can be used with a quiescent model checker while providing - * a high degree of concurrency. It works by isolating readers from writers. In other words, - * readers can intersect with other readers, and writers can coincide with other writers. - * - * We achieve quiescence on a partition level, not on LTS level, and we know for sure - * which operations have finished for a partition, even if their LTS are non-contiguous. - * - * We use a simple wait queue for queuing up waiters, and a compact long counter for - * tracking the number of concurrent readers and writers. Lower 32 bits hold a number of - * readers, and higher 32 bits - a number of writers. - */ -public class LockingDataTracker extends DefaultDataTracker -{ - private final Map<Long, ReadersWritersLock> locked = new ConcurrentHashMap<>(); - - private final WaitQueue readersQueue = WaitQueue.newWaitQueue(); - private final WaitQueue writersQueue = WaitQueue.newWaitQueue(); - private final OpSelectors.PdSelector pdSelector; - private final SchemaSpec schemaSpec; - - private final Set<Long> readingFrom = new ConcurrentSkipListSet<>(); - private final Set<Long> writingTo = new ConcurrentSkipListSet<>(); - - public LockingDataTracker(OpSelectors.PdSelector pdSelector, SchemaSpec schemaSpec) - { - this.pdSelector = pdSelector; - this.schemaSpec = schemaSpec; - } - - @Override - public void beginModification(long lts) - { - ReadersWritersLock partitionLock = getLockForLts(lts); - partitionLock.lockForWrite(); - assert !readingFrom.contains(partitionLock.descriptor) : String.format("Reading from should not have contained %d", partitionLock.descriptor); - writingTo.add(partitionLock.descriptor); - super.beginModification(lts); - } - - @Override - public void endModification(long lts) - { - super.endModification(lts); - ReadersWritersLock partitionLock = getLockForLts(lts); - assert !readingFrom.contains(partitionLock.descriptor) : String.format("Reading from should not have contained %d", partitionLock.descriptor); - writingTo.remove(partitionLock.descriptor); - partitionLock.unlockAfterWrite(); - } - - @Override - public void beginValidation(long pd) - { - ReadersWritersLock partitionLock = getLock(pd); - partitionLock.lockForRead(); - assert !writingTo.contains(pd) : String.format("Writing to should not have contained %d", pd); - readingFrom.add(pd); - super.beginValidation(pd); - } - - @Override - public void endValidation(long pd) - { - super.endValidation(pd); - ReadersWritersLock partitionLock = getLock(pd); - assert !writingTo.contains(pd) : String.format("Writing to should not have contained %d", pd); - readingFrom.remove(partitionLock.descriptor); - partitionLock.unlockAfterRead(); - } - - public void validate(long pd, Runnable runnable) - { - ReadersWritersLock partitionLock = getLockForLts(pd); - partitionLock.lockForRead(); - runnable.run(); - partitionLock.unlockAfterRead(); - } - - private ReadersWritersLock getLockForLts(long lts) - { - long pd = pdSelector.pd(lts, schemaSpec); - return getLock(pd); - } - - private ReadersWritersLock getLock(long pd) - { - return locked.computeIfAbsent(pd, (pd_) -> new ReadersWritersLock(readersQueue, writersQueue, pd)); - } - - /** - * Readers/writers lock. It was decided not to use signals here, and instead go for a - * busyspin instead, since we expect locks to be released briefly and contention to be minimal. - */ - public static class ReadersWritersLock - { - private static final AtomicLongFieldUpdater<ReadersWritersLock> fieldUpdater = AtomicLongFieldUpdater.newUpdater(ReadersWritersLock.class, "lock"); - private volatile long lock; - - final long descriptor; - final WaitQueue readersQueue; - final WaitQueue writersQueue; - - public ReadersWritersLock(WaitQueue readersQueue, WaitQueue writersQueue, long descriptor) - { - this.readersQueue = readersQueue; - this.writersQueue = writersQueue; - this.lock = 0L; - this.descriptor = descriptor; - } - - @Override - public String toString() - { - long lock = this.lock; - return "PartitionLock{" + - "pd = " + descriptor + - ", readers = " + getReaders(lock) + - ", writers = " + getWriters(lock) + - '}'; - } - - public void lockForWrite() - { - while (true) - { - WaitQueue.Signal signal = writersQueue.register(); - long v = lock; - if (getReaders(v) == 0) - { - if (fieldUpdater.compareAndSet(this, v, incWriters(v))) - { - signal.cancel(); - return; - } - } - signal.awaitUninterruptibly(); - } - } - - public void unlockAfterWrite() - { - while (true) - { - long v = lock; - if (fieldUpdater.compareAndSet(this, v, decWriters(v))) - { - readersQueue.signalAll(); - writersQueue.signalAll(); - return; - } - } - } - - public void lockForRead() - { - while (true) - { - WaitQueue.Signal signal = readersQueue.register(); - long v = lock; - if (getWriters(v) == 0) - { - if (fieldUpdater.compareAndSet(this, v, incReaders(v))) - { - signal.cancel(); - return; - } - } - signal.awaitUninterruptibly(); - } - } - - public boolean tryLockForRead() - { - long v = lock; - if (getWriters(v) == 0 && fieldUpdater.compareAndSet(this, v, incReaders(v))) - return true; - - return false; - } - - public void unlockAfterRead() - { - while (true) - { - long v = lock; - if (fieldUpdater.compareAndSet(this, v, decReaders(v))) - { - writersQueue.signalAll(); - readersQueue.signalAll(); - return; - } - } - } - - private long incReaders(long v) - { - long readers = getReaders(v); - assert getWriters(v) == 0; - v &= ~0x00000000ffffffffL; // erase all readers - return v | (readers + 1L); - } - - private long decReaders(long v) - { - long readers = getReaders(v); - assert getWriters(v) == 0; - assert readers >= 1; - v &= ~0x00000000ffffffffL; // erase all readers - return v | (readers - 1L); - } - - private long incWriters(long v) - { - long writers = getWriters(v); - assert getReaders(v) == 0; - v &= ~0xffffffff00000000L; // erase all writers - return v | ((writers + 1L) << 32); - } - - private long decWriters(long v) - { - long writers = getWriters(v); - assert getReaders(v) == 0; - assert writers >= 1 : "Writers left " + writers; - v &= ~0xffffffff00000000L; // erase all writers - return v | ((writers - 1L) << 32); - } - - public int getReaders(long v) - { - v &= 0xffffffffL; - return (int) v; - } - - public int getWriters(long v) - { - v >>= 32; - v &= 0xffffffffL; - return (int) v; - } - } - - @Override - public Configuration.DataTrackerConfiguration toConfig() - { - return new Configuration.LockingDataTrackerConfiguration(maxSeenLts.get(), maxCompleteLts.get(), new ArrayList<>(reorderBuffer)); - } - - @Override - public String toString() - { - return "Locking" + super.toString(); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/dsl/ArrayWrapper.java b/test/harry/main/org/apache/cassandra/harry/util/ArrayWrapper.java similarity index 95% rename from test/harry/main/org/apache/cassandra/harry/dsl/ArrayWrapper.java rename to test/harry/main/org/apache/cassandra/harry/util/ArrayWrapper.java index bddc9ade1fee..69c29b76763e 100644 --- a/test/harry/main/org/apache/cassandra/harry/dsl/ArrayWrapper.java +++ b/test/harry/main/org/apache/cassandra/harry/util/ArrayWrapper.java @@ -16,14 +16,14 @@ * limitations under the License. */ -package org.apache.cassandra.harry.dsl; +package org.apache.cassandra.harry.util; import java.util.Arrays; /** * A small wrapper to allow object arrays to be used as keys in maps. */ -class ArrayWrapper +public class ArrayWrapper { private final Object[] objects; diff --git a/test/harry/main/org/apache/cassandra/harry/util/ByteUtils.java b/test/harry/main/org/apache/cassandra/harry/util/ByteUtils.java index 86195421d5cf..4ebf7e9d5762 100644 --- a/test/harry/main/org/apache/cassandra/harry/util/ByteUtils.java +++ b/test/harry/main/org/apache/cassandra/harry/util/ByteUtils.java @@ -18,14 +18,19 @@ package org.apache.cassandra.harry.util; +import java.math.BigDecimal; +import java.math.BigInteger; import java.net.InetAddress; import java.nio.ByteBuffer; import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; +import java.util.Date; import java.util.List; import java.util.Set; import java.util.UUID; +import org.apache.cassandra.utils.ByteBufferUtil; + public class ByteUtils { public static ByteBuffer bytes(String s) @@ -78,6 +83,35 @@ public static ByteBuffer bytes(UUID uuid) return ByteBuffer.wrap(decompose(uuid)); } + public static ByteBuffer bytes(Date date) + { + return ByteBufferUtil.bytes(date.getTime()); + } + + public static ByteBuffer bytes(BigInteger value) + { + if (value == null) + return ByteBufferUtil.EMPTY_BYTE_BUFFER; + + return ByteBuffer.wrap(value.toByteArray()); + } + + public static ByteBuffer bytes(BigDecimal value) + { + if (value == null) + return ByteBufferUtil.EMPTY_BYTE_BUFFER; + + BigInteger bi = value.unscaledValue(); + int scale = value.scale(); + byte[] bibytes = bi.toByteArray(); + + ByteBuffer bytes = ByteBuffer.allocate(4 + bibytes.length); + bytes.putInt(scale); + bytes.put(bibytes); + bytes.rewind(); + return bytes; + } + public static byte[] decompose(UUID uuid) { long most = uuid.getMostSignificantBits(); @@ -121,6 +155,12 @@ else if (obj instanceof InetAddress) return bytes((InetAddress) obj); else if (obj instanceof String) return bytes((String) obj); + else if (obj instanceof Date) + return bytes((Date) obj); + else if (obj instanceof BigInteger) + return bytes((BigInteger) obj); + else if (obj instanceof BigDecimal) + return bytes((BigDecimal) obj); else if (obj instanceof List) { throw new UnsupportedOperationException("Please use ByteUtils from integration package"); diff --git a/test/harry/main/org/apache/cassandra/harry/util/DescriptorRanges.java b/test/harry/main/org/apache/cassandra/harry/util/DescriptorRanges.java deleted file mode 100644 index 597d7e8bad0c..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/util/DescriptorRanges.java +++ /dev/null @@ -1,150 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.util; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; -import java.util.Objects; -import java.util.stream.Collectors; - -// TODO: this is not really an interval tree, just two sorted arrays. However, given that ExhaustiveChecker has -// to inflate a partition every time we execute the query, we always know all boundaries at any given point in time. -public class DescriptorRanges -{ - private List<DescriptorRange> sortedByMin; - - public DescriptorRanges(List<DescriptorRange> ranges) - { - this.sortedByMin = new ArrayList<>(ranges); - Collections.sort(sortedByMin, Comparator.comparingLong(a -> a.minBound)); - } - - public boolean isShadowed(long cd, long lts) - { - return !shadowedBy(cd, lts).isEmpty(); - } - - public List<DescriptorRange> shadowedBy(long cd, long lts) - { - List<DescriptorRange> shadowedBy = new ArrayList<>(); - for (DescriptorRange range : sortedByMin) - { - if (range.minBound > cd) - break; - - if (range.contains(cd, lts)) - shadowedBy.add(range); - } - - return shadowedBy; - } - - public List<DescriptorRange> newerThan(long ts) - { - return sortedByMin.stream().filter((rt) -> { - return rt.timestamp >= ts; - }).collect(Collectors.toList()); - } - - - private static int toIdx(int idxOrIP) - { - if (idxOrIP >= 0) - return idxOrIP; - - return -1 * (idxOrIP + 1); - } - - public static class DescriptorRange - { - public final long minBound; - public final long maxBound; - public final boolean minInclusive; - public final boolean maxInclusive; - - public final long timestamp; - - public DescriptorRange(long minBound, long maxBound, boolean minInclusive, boolean maxInclusive, long timestamp) - { - this.minBound = minBound; - this.maxBound = maxBound; - this.minInclusive = minInclusive; - this.maxInclusive = maxInclusive; - - this.timestamp = timestamp; - } - - public boolean contains(long descriptor) - { - if (minInclusive && maxInclusive) - return descriptor >= minBound && descriptor <= maxBound; - - if (!minInclusive && !maxInclusive) - return descriptor > minBound && descriptor < maxBound; - - if (!minInclusive && maxInclusive) - return descriptor > minBound && descriptor <= maxBound; - - assert (minInclusive && !maxInclusive); - return descriptor >= minBound && descriptor < maxBound; - } - - public boolean contains(long descriptor, long ts) - { - return contains(descriptor) && timestamp >= ts; - } - - public boolean equals(Object o) - { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - DescriptorRange range = (DescriptorRange) o; - return minBound == range.minBound && - maxBound == range.maxBound && - minInclusive == range.minInclusive && - maxInclusive == range.maxInclusive && - timestamp == range.timestamp; - } - - public int hashCode() - { - return Objects.hash(minBound, maxBound, minInclusive, maxInclusive, timestamp); - } - - public String toString() - { - return "Range{" + - "minBound=" + minBound + - ", maxBound=" + maxBound + - ", minInclusive=" + minInclusive + - ", maxInclusive=" + maxInclusive + - ", timestamp=" + timestamp + - '}'; - } - } - - public String toString() - { - return "Ranges{" + - "sortedByMin=" + sortedByMin + - '}'; - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/util/IteratorsUtil.java b/test/harry/main/org/apache/cassandra/harry/util/IteratorsUtil.java new file mode 100644 index 000000000000..72984b30fa57 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/util/IteratorsUtil.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.util; + +import java.util.Iterator; + +public class IteratorsUtil +{ + public static <T> Iterable<T> concat(Iterable<T>... iterables) + { + assert iterables != null && iterables.length > 0; + if (iterables.length == 1) + return iterables[0]; + + return () -> { + return new Iterator<T>() + { + int idx; + Iterator<T> current; + boolean hasNext; + + { + idx = 0; + prepareNext(); + } + + private void prepareNext() + { + if (current != null && current.hasNext()) + { + hasNext = true; + return; + } + + while (idx < iterables.length) + { + current = iterables[idx].iterator(); + idx++; + if (current.hasNext()) + { + hasNext = true; + return; + } + } + + hasNext = false; + } + + public boolean hasNext() + { + return hasNext; + } + + public T next() + { + T next = current.next(); + prepareNext(); + return next; + } + }; + }; + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/util/StringUtils.java b/test/harry/main/org/apache/cassandra/harry/util/StringUtils.java index 956b3a34bfac..f6571c44784a 100644 --- a/test/harry/main/org/apache/cassandra/harry/util/StringUtils.java +++ b/test/harry/main/org/apache/cassandra/harry/util/StringUtils.java @@ -18,18 +18,57 @@ package org.apache.cassandra.harry.util; +import org.apache.cassandra.harry.MagicConstants; + public class StringUtils { public static String toString(long[] arr) { - String s = ""; + if (arr.length == 0) + return "EMPTY"; + StringBuilder s = new StringBuilder(); for (int i = 0; i < arr.length; i++) { - s += arr[i]; - s += "L"; + if (arr[i] == MagicConstants.UNSET_DESCR) + s.append("UNSET"); + else if (arr[i] == MagicConstants.UNKNOWN_DESCR) + s.append("UNKNOWN"); + else if (arr[i] == MagicConstants.NIL_DESCR) + s.append("NIL"); + else + { + s.append(arr[i]); + s.append("L"); + } if (i < (arr.length - 1)) - s += ','; + s.append(','); } - return s; + return s.toString(); + } + + public static String toString(int[] arr) + { + if (arr.length == 0) + return "EMPTY"; + StringBuilder s = new StringBuilder(); + for (int i = 0; i < arr.length; i++) + { + s.append(toString(arr[i])); + if (i < (arr.length - 1)) + s.append(','); + } + return s.toString(); + } + + public static String toString(int idx) + { + if (idx == MagicConstants.UNSET_IDX) + return "UNSET"; + else if (idx == MagicConstants.UNKNOWN_IDX) + return "UNKNOWN"; + else if (idx == MagicConstants.NIL_IDX) + return "NIL"; + else + return Integer.toString(idx); } } diff --git a/test/harry/main/org/apache/cassandra/harry/util/ThrowingRunnable.java b/test/harry/main/org/apache/cassandra/harry/util/ThrowingRunnable.java index 092721707883..c75bc0dd7f93 100644 --- a/test/harry/main/org/apache/cassandra/harry/util/ThrowingRunnable.java +++ b/test/harry/main/org/apache/cassandra/harry/util/ThrowingRunnable.java @@ -21,10 +21,10 @@ public interface ThrowingRunnable { void run() throws Throwable; - static Runnable toRunnable(ThrowingRunnable runnable) { + default Runnable toRunnable() { return () -> { try { - runnable.run(); + this.run(); } catch (Throwable var2) { throw new RuntimeException(var2); } diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/AllPartitionsValidator.java b/test/harry/main/org/apache/cassandra/harry/visitors/AllPartitionsValidator.java deleted file mode 100644 index 8a44b6a2a829..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/AllPartitionsValidator.java +++ /dev/null @@ -1,156 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.visitors; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; - -import org.junit.Assert; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.concurrent.ExecutorFactory; -import org.apache.cassandra.concurrent.InfiniteLoopExecutor; -import org.apache.cassandra.concurrent.Interruptible; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.MetricReporter; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.Model; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.tracker.DataTracker; -import org.apache.cassandra.harry.runner.Runner; -import org.apache.cassandra.utils.concurrent.WaitQueue; - -import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.Daemon.NON_DAEMON; -import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.Interrupts.UNSYNCHRONIZED; -import static org.apache.cassandra.concurrent.InfiniteLoopExecutor.SimulatorSafe.SAFE; - -/** - * Concurrently validates all partitions that were visited during this run. - */ -public class AllPartitionsValidator implements Visitor -{ - private static final Logger logger = LoggerFactory.getLogger(AllPartitionsValidator.class); - - protected final Model model; - protected final SchemaSpec schema; - protected final QueryLogger queryLogger; - protected final OpSelectors.Clock clock; - protected final OpSelectors.PdSelector pdSelector; - protected final MetricReporter metricReporter; - protected final SystemUnderTest sut; - protected final DataTracker tracker; - - protected final int concurrency; - - public static Configuration.VisitorConfiguration factoryForTests(int concurrency, - Model.ModelFactory modelFactory) - { - return (r) -> new AllPartitionsValidator(r, concurrency, modelFactory, QueryLogger.NO_OP); - } - - public AllPartitionsValidator(Run run, - int concurrency, - Model.ModelFactory modelFactory) - { - this(run, concurrency, modelFactory, QueryLogger.NO_OP); - } - - public AllPartitionsValidator(Run run, - int concurrency, - Model.ModelFactory modelFactory, - QueryLogger logger) - { - this.metricReporter = run.metricReporter; - this.model = modelFactory.make(run); - this.schema = run.schemaSpec; - this.clock = run.clock; - this.sut = run.sut; - this.pdSelector = run.pdSelector; - this.concurrency = concurrency; - this.tracker = run.tracker; - this.queryLogger = logger; - } - - protected void validateAllPartitions() throws Throwable - { - List<Interruptible> threads = new ArrayList<>(); - WaitQueue queue = WaitQueue.newWaitQueue(); - WaitQueue.Signal interrupt = queue.register(); - List<Throwable> errors = new CopyOnWriteArrayList<>(); - - final long maxPosition = pdSelector.maxPosition(tracker.maxStarted()); - - AtomicLong currentPosition = new AtomicLong(); - for (int i = 0; i < concurrency; i++) - { - Interruptible thread = ExecutorFactory.Global.executorFactory().infiniteLoop(String.format("AllPartitionsValidator-%d", i + 1), - Runner.wrapInterrupt((state) -> { - if (state == Interruptible.State.NORMAL) - { - metricReporter.validatePartition(); - long pos = currentPosition.getAndIncrement(); - if (pos < maxPosition) - { - for (boolean reverse : new boolean[]{ true, false }) - { - Query query = Query.selectAllColumns(schema, pdSelector.pd(pdSelector.minLtsAt(pos), schema), reverse); - model.validate(query); - queryLogger.logSelectQuery((int)pos, query); - } - } - else - { - interrupt.signalAll(); - } - } - }, interrupt::signal, errors::add), SAFE, NON_DAEMON, UNSYNCHRONIZED); - threads.add(thread); - } - - interrupt.awaitUninterruptibly(); - - for (Interruptible thread : threads) - { - ((InfiniteLoopExecutor)thread).shutdown(false); - Assert.assertTrue(thread.awaitTermination(1, TimeUnit.MINUTES)); - } - - if (!errors.isEmpty()) - Runner.mergeAndThrow(errors); - } - - public void visit() - { - try - { - validateAllPartitions(); - } - catch (Throwable e) - { - throw new RuntimeException(e); - } - } -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/CorruptingVisitor.java b/test/harry/main/org/apache/cassandra/harry/visitors/CorruptingVisitor.java deleted file mode 100644 index 043a0804d9bc..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/CorruptingVisitor.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.harry.visitors; - -import java.util.Random; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.corruptor.AddExtraRowCorruptor; -import org.apache.cassandra.harry.corruptor.ChangeValueCorruptor; -import org.apache.cassandra.harry.corruptor.HideRowCorruptor; -import org.apache.cassandra.harry.corruptor.HideValueCorruptor; -import org.apache.cassandra.harry.corruptor.QueryResponseCorruptor; -import org.apache.cassandra.harry.runner.HarryRunner; -import org.apache.cassandra.harry.operations.Query; - -public class CorruptingVisitor implements Visitor -{ - public static final Logger logger = LoggerFactory.getLogger(HarryRunner.class); - - private final Run run; - private final QueryResponseCorruptor[] corruptors; - private final int triggerAfter; - - public CorruptingVisitor(int triggerAfter, - Run run) - { - this.run = run; - this.triggerAfter = triggerAfter; - - this.corruptors = new QueryResponseCorruptor[]{ - new QueryResponseCorruptor.SimpleQueryResponseCorruptor(run.schemaSpec, - run.clock, - HideRowCorruptor::new), - new AddExtraRowCorruptor(run.schemaSpec, - run.clock, - run.tracker, - run.descriptorSelector), - new QueryResponseCorruptor.SimpleQueryResponseCorruptor(run.schemaSpec, - run.clock, - HideValueCorruptor::new), - new QueryResponseCorruptor.SimpleQueryResponseCorruptor(run.schemaSpec, - run.clock, - ChangeValueCorruptor::new) - }; - } - - public void visit() - { - long lts = run.clock.peek(); - - if (lts > triggerAfter) - return; - - // TODO: switch to a better entropy source - Random random = new Random(1); - - QueryResponseCorruptor corruptor = corruptors[random.nextInt(corruptors.length)]; - long maxPos = run.pdSelector.maxPosition(run.tracker.maxStarted()); - long pd = run.pdSelector.pd(random.nextInt((int) maxPos), run.schemaSpec); - try - { - boolean success = corruptor.maybeCorrupt(Query.selectAllColumns(run.schemaSpec, pd, false), - run.sut); - logger.info("{} tried to corrupt a partition with a pd {}@{} my means of {}", success ? "Successfully" : "Unsuccessfully", pd, lts, corruptor.getClass()); - } - catch (Throwable t) - { - logger.error("Caught an exception while trying to corrupt a partition.", t); - } - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/DoubleWriteVisitExecutor.java b/test/harry/main/org/apache/cassandra/harry/visitors/DoubleWriteVisitExecutor.java deleted file mode 100644 index 06bc1d74583c..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/DoubleWriteVisitExecutor.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.visitors; - -public class DoubleWriteVisitExecutor extends VisitExecutor -{ - private final VisitExecutor delegate1; - private final VisitExecutor delegate2; - - public DoubleWriteVisitExecutor(VisitExecutor delegate1, - VisitExecutor delegate2) - { - this.delegate1 = delegate1; - this.delegate2 = delegate2; - } - - protected void beforeLts(long lts, long pd) - { - delegate1.beforeLts(lts, pd); - delegate2.beforeLts(lts, pd); - } - - protected void afterLts(long lts, long pd) - { - delegate1.afterLts(lts, pd); - delegate2.afterLts(lts, pd); - } - - protected void operation(Operation operation) - { - delegate1.operation(operation); - delegate2.operation(operation); - } - - public void shutdown() throws InterruptedException - { - delegate1.shutdown(); - delegate2.shutdown(); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/FaultInjectingVisitor.java b/test/harry/main/org/apache/cassandra/harry/visitors/FaultInjectingVisitor.java deleted file mode 100644 index 428c699266fa..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/FaultInjectingVisitor.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.visitors; - -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.operations.CompiledStatement; - -/** - * Fault injecting visitor: randomly fails some of the queries. - * - * Requires {@code FaultInjectingSut} to function. - */ -public class FaultInjectingVisitor extends LoggingVisitor -{ - private final AtomicInteger cnt = new AtomicInteger(); - - private final SystemUnderTest.FaultInjectingSut sut; - protected final ScheduledExecutorService executor = Executors.newScheduledThreadPool(2); - - public FaultInjectingVisitor(Run run, OperationExecutor.RowVisitorFactory rowVisitorFactory) - { - super(run, rowVisitorFactory); - this.sut = (SystemUnderTest.FaultInjectingSut) run.sut; - } - - void executeAsyncWithRetries(CompletableFuture<Object[][]> originator, CompiledStatement statement) - { - executeAsyncWithRetries(originator, statement, true); - } - - void executeAsyncWithRetries(CompletableFuture<Object[][]> originator, CompiledStatement statement, boolean allowFailures) - { - if (sut.isShutdown()) - throw new IllegalStateException("System under test is shut down"); - - CompletableFuture<Object[][]> future; - if (allowFailures && cnt.getAndIncrement() % 2 == 0) - { - future = sut.executeAsyncWithWriteFailure(statement.cql(), SystemUnderTest.ConsistencyLevel.QUORUM, statement.bindings()); - } - else - { - future = sut.executeAsync(statement.cql(), SystemUnderTest.ConsistencyLevel.QUORUM, statement.bindings()); - } - - future.whenComplete((res, t) -> { - if (t != null) - executor.schedule(() -> executeAsyncWithRetries(originator, statement, false), 1, TimeUnit.SECONDS); - else - originator.complete(res); - }); - } - - @JsonTypeName("fault_injecting") - public static class FaultInjectingVisitorConfiguration extends Configuration.MutatingVisitorConfiguation - { - @JsonCreator - public FaultInjectingVisitorConfiguration(@JsonProperty("row_visitor") Configuration.RowVisitorConfiguration row_visitor) - { - super(row_visitor); - } - - @Override - public Visitor make(Run run) - { - return new FaultInjectingVisitor(run, row_visitor); - } - } - -} diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/GeneratingVisitor.java b/test/harry/main/org/apache/cassandra/harry/visitors/GeneratingVisitor.java deleted file mode 100644 index 3cda4179d51d..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/GeneratingVisitor.java +++ /dev/null @@ -1,232 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.visitors; - -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.operations.QueryGenerator; -import org.apache.cassandra.harry.util.BitSet; - -public class GeneratingVisitor extends LtsVisitor -{ - private final OpSelectors.PdSelector pdSelector; - private final OpSelectors.DescriptorSelector descriptorSelector; - private final QueryGenerator rangeSelector; - private final SchemaSpec schema; - - public GeneratingVisitor(Run run, - VisitExecutor delegate) - { - super(delegate, run.clock::nextLts); - - this.pdSelector = run.pdSelector; - this.descriptorSelector = run.descriptorSelector; - this.schema = run.schemaSpec; - this.rangeSelector = run.rangeSelector; - } - - @Override - public void visit(long lts) - { - generate(lts, pdSelector.pd(lts, schema)); - } - - private void generate(long lts, long pd) - { - beforeLts(lts, pd); - int opsPerLts = descriptorSelector.operationsPerLts(lts); - for (long opId = 0; opId < opsPerLts; opId++) - { - OpSelectors.OperationKind kind = descriptorSelector.operationType(pd, lts, opId); - BaseOperation operation; - switch (kind) - { - case INSERT: - case UPDATE: - operation = writeOp(lts, pd, opId, kind); - break; - case INSERT_WITH_STATICS: - case UPDATE_WITH_STATICS: - operation = writeRegularAndStatic(lts, pd, opId, kind); - break; - case DELETE_ROW: - { - long cd = descriptorSelector.cd(pd, lts, opId, schema); - operation = new GeneratedDeleteRowOp(lts, pd, cd, opId, kind); - break; - } - case DELETE_COLUMN: - case DELETE_COLUMN_WITH_STATICS: - { - long cd = descriptorSelector.cd(pd, lts, opId, schema); - BitSet columns = descriptorSelector.columnMask(pd, lts, opId, kind); - operation = new GeneratedDeleteColumnsOp(lts, pd, cd, opId, kind, columns); - break; - } - case DELETE_PARTITION: - case DELETE_RANGE: - case DELETE_SLICE: - operation = new GeneratedDeleteOp(lts, pd, opId, kind, rangeSelector); - break; - default: - throw new IllegalStateException("All cases are covered but not " + kind); - } - operation(operation); - } - afterLts(lts, pd); - } - - public BaseOperation writeOp(long lts, long pd, long opId, OpSelectors.OperationKind kind) - { - long cd = descriptorSelector.cd(pd, lts, opId, schema); - - return new GeneratedWriteOp(lts, pd, cd, opId, kind) - { - @Override - public long[] vds() - { - return descriptorSelector.vds(pd, cd(), lts, opId, kind(), schema); - } - }; - } - - public BaseOperation writeRegularAndStatic(long lts, long pd, long opId, OpSelectors.OperationKind kind) - { - long cd = descriptorSelector.cd(pd, lts, opId, schema); - - return new GeneratedWriteWithStaticOp(lts, pd, cd, opId, kind) - { - public long[] sds() - { - return descriptorSelector.sds(pd, cd(), lts, opId, kind(), schema); - } - - @Override - public long[] vds() - { - return descriptorSelector.vds(pd, cd(), lts, opId, kind(), schema); - } - }; - } - - public abstract static class GeneratedWriteOp extends BaseOperation implements ReplayingVisitor.WriteOp - { - protected final long cd; - public GeneratedWriteOp(long lts, long pd, long cd, long opId, OpSelectors.OperationKind kind) - { - super(lts, pd, opId, kind); - this.cd = cd; - } - - @Override - public long cd() - { - return cd; - } - } - - public abstract static class GeneratedWriteWithStaticOp extends GeneratedWriteOp implements ReplayingVisitor.WriteStaticOp - { - public GeneratedWriteWithStaticOp(long lts, long pd, long cd, long opId, OpSelectors.OperationKind kind) - { - super(lts, pd, cd, opId, kind); - } - } - - public static class GeneratedDeleteRowOp extends BaseOperation implements ReplayingVisitor.DeleteRowOp - { - private final long cd; - public GeneratedDeleteRowOp(long lts, long pd, long cd, long opId, OpSelectors.OperationKind kind) - { - super(lts, pd, opId, kind); - this.cd = cd; - } - - @Override - public long cd() - { - return cd; - } - } - - public static class GeneratedDeleteOp extends BaseOperation implements ReplayingVisitor.DeleteOp - { - private final Query relations; - - public GeneratedDeleteOp(long lts, long pd, long opId, OpSelectors.OperationKind kind, QueryGenerator queryGenerator) - { - this(lts, pd, opId, kind, queryGenerator.inflate( lts, opId, queryKind(kind))); - } - - public GeneratedDeleteOp(long lts, long pd, long opId, OpSelectors.OperationKind kind, Query relations) - { - super(lts, pd, opId, kind); - this.relations = relations; - } - - @Override - public Query relations() - { - return relations; - } - - protected static Query.QueryKind queryKind(OpSelectors.OperationKind kind) - { - switch (kind) - { - case DELETE_PARTITION: - return Query.QueryKind.SINGLE_PARTITION; - case DELETE_ROW: - return Query.QueryKind.SINGLE_CLUSTERING; - case DELETE_RANGE: - return Query.QueryKind.CLUSTERING_RANGE; - case DELETE_SLICE: - return Query.QueryKind.CLUSTERING_SLICE; - default: - throw new IllegalStateException(String.format("Can not transform %s into delete", kind)); - } - } - } - - public static class GeneratedDeleteColumnsOp extends BaseOperation implements ReplayingVisitor.DeleteColumnsOp - { - private final long cd; - private final BitSet columnMask; - - public GeneratedDeleteColumnsOp(long lts, long pd, long cd, long opId, OpSelectors.OperationKind kind, BitSet columnMask) - { - super(lts, pd, opId, kind); - this.cd = cd; - this.columnMask = columnMask; - } - - public long cd() - { - return cd; - } - - @Override - public BitSet columns() - { - return columnMask; - } - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/LoggingVisitor.java b/test/harry/main/org/apache/cassandra/harry/visitors/LoggingVisitor.java deleted file mode 100644 index eee0996db956..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/LoggingVisitor.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.visitors; - -import java.io.BufferedWriter; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStreamWriter; - -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.operations.CompiledStatement; - -public class LoggingVisitor extends GeneratingVisitor -{ - public LoggingVisitor(Run run, - OperationExecutor.RowVisitorFactory rowVisitorFactory) - { - super(run, new LoggingVisitorExecutor(run, rowVisitorFactory.make(run))); - } - - public static class LoggingVisitorExecutor extends MutatingVisitor.MutatingVisitExecutor - { - private final BufferedWriter operationLog; - - public LoggingVisitorExecutor(Run run, OperationExecutor rowVisitor) - { - super(run, rowVisitor); - - File f = new File("operation.log"); - try - { - operationLog = new BufferedWriter(new OutputStreamWriter(new FileOutputStream(f))); - } - catch (FileNotFoundException e) - { - throw new RuntimeException(e); - } - } - - public void afterLts(long lts, long pd) - { - super.afterLts(lts, pd); - log("LTS: %d. Pd %d. Finished\n", lts, pd); - } - - @Override - protected void operationInternal(Operation operation, CompiledStatement statement) - { - super.operationInternal(operation, statement); - log(String.format("LTS: %d. Pd %d. Operation: %s Statement %s\n", - operation.lts(), operation.pd(), operation, statement)); - } - - private void log(String format, Object... objects) - { - try - { - operationLog.write(String.format(format, objects)); - operationLog.flush(); - } - catch (IOException e) - { - // ignore - } - } - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/LtsVisitor.java b/test/harry/main/org/apache/cassandra/harry/visitors/LtsVisitor.java deleted file mode 100644 index f34b5df5a551..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/LtsVisitor.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.visitors; - -import java.util.function.LongSupplier; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Common class for all visitors that support visits at a specific logical timestamp. - * - * Classes inheriting from LTS Visitor have to visit drawn LTS: every LTS that has been received - * from the consumer _has to_ actually be visited. If this is not done, during model checking - * drawn LTS will be considered visited, which will lead to data inconsistencies. - * - * This class and its implementations such as Mutating visitor are NOT thread safe. If you'd like - * to have several threads generating data, use multiple copies of delegating visitor, since - */ -public abstract class LtsVisitor extends VisitExecutor implements Visitor -{ - private static final Logger logger = LoggerFactory.getLogger(LtsVisitor.class); - protected final VisitExecutor delegate; - private final LongSupplier ltsSource; - - public LtsVisitor(VisitExecutor delegate, - LongSupplier ltsSource) - { - this.delegate = delegate; - this.ltsSource = ltsSource; - } - - public final void visit() - { - long lts = ltsSource.getAsLong(); - if (lts > 0 && lts % 10_000 == 0) - logger.info("Visiting lts {}...", lts); - visit(lts); - } - - public abstract void visit(long lts); - - @Override - protected void beforeLts(long lts, long pd) - { - delegate.beforeLts(lts, pd); - } - - @Override - protected void afterLts(long lts, long pd) - { - delegate.afterLts(lts, pd); - } - - @Override - protected void operation(Operation operation) - { - delegate.operation(operation); - } - - @Override - public void shutdown() throws InterruptedException - { - delegate.shutdown(); - } - -} diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/MutatingRowVisitor.java b/test/harry/main/org/apache/cassandra/harry/visitors/MutatingRowVisitor.java deleted file mode 100644 index d394693c21d3..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/MutatingRowVisitor.java +++ /dev/null @@ -1,115 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.visitors; - -import org.apache.cassandra.harry.core.MetricReporter; -import org.apache.cassandra.harry.core.Run; -import com.google.common.annotations.VisibleForTesting; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.operations.CompiledStatement; -import org.apache.cassandra.harry.operations.DeleteHelper; -import org.apache.cassandra.harry.operations.WriteHelper; -import org.apache.cassandra.harry.util.BitSet; - -public class MutatingRowVisitor implements OperationExecutor -{ - protected final SchemaSpec schema; - protected final OpSelectors.Clock clock; - protected final MetricReporter metricReporter; - - public MutatingRowVisitor(Run run) - { - this(run.schemaSpec, - run.clock, - run.metricReporter); - } - - @VisibleForTesting - public MutatingRowVisitor(SchemaSpec schema, - OpSelectors.Clock clock, - MetricReporter metricReporter) - { - this.metricReporter = metricReporter; - this.schema = schema; - this.clock = clock; - } - - public CompiledStatement insert(VisitExecutor.WriteOp op) - { - metricReporter.insert(); - return WriteHelper.inflateInsert(schema, op.pd(), op.cd(), op.vds(), null, clock.rts(op.lts())); - } - - public CompiledStatement insertWithStatics(VisitExecutor.WriteStaticOp op) - { - metricReporter.insert(); - return WriteHelper.inflateInsert(schema, op.pd(), op.cd(), op.vds(), op.sds(), clock.rts(op.lts())); - } - - public CompiledStatement update(VisitExecutor.WriteOp op) - { - metricReporter.insert(); - return WriteHelper.inflateUpdate(schema, op.pd(), op.cd(), op.vds(), null, clock.rts(op.lts())); - } - - public CompiledStatement updateWithStatics(VisitExecutor.WriteStaticOp op) - { - metricReporter.insert(); - return WriteHelper.inflateUpdate(schema, op.pd(), op.cd(), op.vds(), op.sds(), clock.rts(op.lts())); - } - - public CompiledStatement deleteColumn(VisitExecutor.DeleteColumnsOp op) - { - metricReporter.columnDelete(); - BitSet mask = schema.regularColumnsMask(); - return DeleteHelper.deleteColumn(schema, op.pd(), op.cd(), op.columns(), mask, clock.rts(op.lts())); - } - - public CompiledStatement deleteColumnWithStatics(VisitExecutor.DeleteColumnsOp op) - { - metricReporter.columnDelete(); - BitSet mask = schema.regularAndStaticColumnsMask(); - return DeleteHelper.deleteColumn(schema, op.pd(), op.cd(), op.columns(), mask, clock.rts(op.lts())); - } - - public CompiledStatement deleteRow(VisitExecutor.DeleteRowOp op) - { - metricReporter.rowDelete(); - return DeleteHelper.deleteRow(schema, op.pd(), op.cd(), clock.rts(op.lts())); - } - - public CompiledStatement deletePartition(VisitExecutor.DeleteOp op) - { - metricReporter.partitionDelete(); - return DeleteHelper.delete(schema, op.pd(), clock.rts(op.lts())); - } - - public CompiledStatement deleteRange(VisitExecutor.DeleteOp op) - { - metricReporter.rangeDelete(); - return op.relations().toDeleteStatement(clock.rts(op.lts())); - } - - public CompiledStatement deleteSlice(VisitExecutor.DeleteOp op) - { - metricReporter.rangeDelete(); - return op.relations().toDeleteStatement(clock.rts(op.lts())); - } -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/MutatingVisitor.java b/test/harry/main/org/apache/cassandra/harry/visitors/MutatingVisitor.java deleted file mode 100644 index 4380fd1de939..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/MutatingVisitor.java +++ /dev/null @@ -1,220 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.visitors; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.function.Function; - -import com.google.common.util.concurrent.Uninterruptibles; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.operations.CompiledStatement; -import org.apache.cassandra.harry.tracker.DataTracker; - -public class MutatingVisitor extends GeneratingVisitor -{ - private static final Logger logger = LoggerFactory.getLogger(MutatingVisitor.class); - - public MutatingVisitor(Run run) - { - this(run, MutatingRowVisitor::new); - } - - public MutatingVisitor(Run run, - OperationExecutor.RowVisitorFactory rowVisitorFactory) - { - this(run, new MutatingVisitExecutor(run, rowVisitorFactory.make(run))); - } - - public static Configuration.VisitorConfiguration factory() - { - return MutatingVisitor::new; - } - - public static Configuration.VisitorConfiguration factory(Function<Run, VisitExecutor> rowVisitorFactory) - { - return (r) -> new MutatingVisitor(r, rowVisitorFactory.apply(r)); - } - - public MutatingVisitor(Run run, - VisitExecutor visitExecutor) - { - super(run, visitExecutor); - } - - public static class LtsTrackingVisitExecutor extends MutatingVisitExecutor - { - public LtsTrackingVisitExecutor(OpSelectors.DescriptorSelector descriptorSelector, DataTracker tracker, SystemUnderTest sut, SchemaSpec schema, OperationExecutor rowVisitor, SystemUnderTest.ConsistencyLevel consistencyLevel) - { - super(descriptorSelector, tracker, sut, schema, rowVisitor, consistencyLevel); - assert schema.trackLts : "LTS Tracking visitor can only be used when tracking LTS in schema"; - } - - @Override - public void afterLts(long lts, long pd) - { - if (hadVisibleVisit) - { - StringBuilder sb = new StringBuilder(); - sb.append("UPDATE ").append(schema.keyspace).append(".").append(schema.table) - .append(" SET visited_lts = visited_lts + [").append(lts).append("] ") - .append("WHERE "); - Object[] pk = schema.inflatePartitionKey(pd); - for (int i = 0; i < pk.length; i++) - { - if (i > 0) - sb.append(" AND "); - sb.append(schema.partitionKeys.get(i).name + " = ?"); - bindings.add(pk[i]); - } - sb.append(";"); - statements.add(sb.toString()); - } - - super.afterLts(lts, pd); - } - } - - public static class MutatingVisitExecutor extends VisitExecutor - { - protected final List<String> statements = new ArrayList<>(); - protected final List<Object> bindings = new ArrayList<>(); - - protected final OpSelectors.DescriptorSelector descriptorSelector; - protected final DataTracker tracker; - protected final SystemUnderTest sut; - protected final OperationExecutor rowVisitor; - protected final SystemUnderTest.ConsistencyLevel consistencyLevel; - protected final SchemaSpec schema; - private final int maxRetries = 10; - // Apart from partition deletion, we register all operations on partition level - protected Boolean hadVisibleVisit = null; - - public MutatingVisitExecutor(Run run, OperationExecutor rowVisitor) - { - this(run, rowVisitor, SystemUnderTest.ConsistencyLevel.QUORUM); - } - - public MutatingVisitExecutor(Run run, OperationExecutor rowVisitor, SystemUnderTest.ConsistencyLevel consistencyLevel) - { - this(run.descriptorSelector, run.tracker, run.sut, run.schemaSpec, rowVisitor, consistencyLevel); - } - - public MutatingVisitExecutor(OpSelectors.DescriptorSelector descriptorSelector, - DataTracker tracker, - SystemUnderTest sut, - SchemaSpec schema, - OperationExecutor rowVisitor, - SystemUnderTest.ConsistencyLevel consistencyLevel) - { - this.descriptorSelector = descriptorSelector; - this.tracker = tracker; - this.sut = sut; - this.schema = schema; - this.rowVisitor = rowVisitor; - this.consistencyLevel = consistencyLevel; - } - - @Override - public void beforeLts(long lts, long pd) - { - tracker.beginModification(lts); - } - - @Override - public void afterLts(long lts, long pd) - { - if (statements.isEmpty()) - { - logger.warn("Encountered an empty batch on {}", lts); - return; - } - - String query = String.join(" ", statements); - if (statements.size() > 1) - query = String.format("BEGIN UNLOGGED BATCH\n%s\nAPPLY BATCH;", query); - - Object[] bindingsArray = new Object[bindings.size()]; - bindings.toArray(bindingsArray); - statements.clear(); - bindings.clear(); - - CompiledStatement compiledStatement = new CompiledStatement(query, bindingsArray); - executeWithRetries(lts, pd, compiledStatement); - tracker.endModification(lts); - hadVisibleVisit = null; - } - - @Override - public void operation(Operation operation) - { - // Partition deletions have highest precedence even in batches, so we have to make - // a distinction between "we have not seen any operations yet" and "there was a partition deletion" - if (hadVisibleVisit == null) - hadVisibleVisit = operation.kind().hasVisibleVisit(); - else - hadVisibleVisit &= operation.kind().hasVisibleVisit(); - - operationInternal(operation, rowVisitor.perform(operation)); - } - - protected void operationInternal(Operation operation, CompiledStatement statement) - { - statements.add(statement.cql()); - Collections.addAll(bindings, statement.bindings()); - } - - protected Object[][] executeWithRetries(long lts, long pd, CompiledStatement statement) - { - if (sut.isShutdown()) - throw new IllegalStateException("System under test is shut down"); - - int retries = 0; - - while (retries++ < maxRetries) - { - try - { - return sut.execute(statement.cql(), consistencyLevel, statement.bindings()); - } - catch (Throwable t) - { - int delaySecs = 1; - logger.error(String.format("Caught message while trying to execute: %s. Scheduled to retry in %s seconds", statement, delaySecs), t); - Uninterruptibles.sleepUninterruptibly(delaySecs, TimeUnit.SECONDS); - } - } - - throw new IllegalStateException(String.format("Can not execute statement %s after %d retries", statement, retries)); - } - - public void shutdown() throws InterruptedException - { - } - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/OperationExecutor.java b/test/harry/main/org/apache/cassandra/harry/visitors/OperationExecutor.java deleted file mode 100644 index a5997a2a1d01..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/OperationExecutor.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.visitors; - -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.operations.CompiledStatement; - -public interface OperationExecutor -{ - interface RowVisitorFactory - { - OperationExecutor make(Run run); - } - - default CompiledStatement perform(VisitExecutor.Operation operation) - { - switch (operation.kind()) - { - // TODO: switch to EnumMap - // TODO: pluggable capabilities; OperationKind can/should bear its own logic - case INSERT: - return insert((VisitExecutor.WriteOp) operation); - case UPDATE: - return update((VisitExecutor.WriteOp) operation); - case DELETE_ROW: - return deleteRow((VisitExecutor.DeleteRowOp) operation); - case INSERT_WITH_STATICS: - return insertWithStatics((VisitExecutor.WriteStaticOp) operation); - case UPDATE_WITH_STATICS: - return updateWithStatics((VisitExecutor.WriteStaticOp) operation); - case DELETE_PARTITION: - return deletePartition((VisitExecutor.DeleteOp) operation); - case DELETE_COLUMN: - return deleteColumn((VisitExecutor.DeleteColumnsOp) operation); - case DELETE_COLUMN_WITH_STATICS: - return deleteColumnWithStatics((VisitExecutor.DeleteColumnsOp) operation); - case DELETE_RANGE: - return deleteRange((VisitExecutor.DeleteOp) operation); - case DELETE_SLICE: - return deleteSlice((VisitExecutor.DeleteOp) operation); - default: - throw new IllegalStateException(); - } - } - - CompiledStatement insert(VisitExecutor.WriteOp operation); - CompiledStatement update(VisitExecutor.WriteOp operation); - - CompiledStatement insertWithStatics(VisitExecutor.WriteStaticOp operation); - CompiledStatement updateWithStatics(VisitExecutor.WriteStaticOp operation); - - CompiledStatement deleteColumn(VisitExecutor.DeleteColumnsOp operation); - CompiledStatement deleteColumnWithStatics(VisitExecutor.DeleteColumnsOp operation); - - CompiledStatement deleteRow(VisitExecutor.DeleteRowOp operation); - - CompiledStatement deletePartition(VisitExecutor.DeleteOp deleteOp); - CompiledStatement deleteRange(VisitExecutor.DeleteOp deleteOp); - CompiledStatement deleteSlice(VisitExecutor.DeleteOp deleteOp); - - -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/ParallelRecentValidator.java b/test/harry/main/org/apache/cassandra/harry/visitors/ParallelRecentValidator.java deleted file mode 100644 index 61cd1b212260..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/ParallelRecentValidator.java +++ /dev/null @@ -1,149 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.visitors; - -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.atomic.AtomicLong; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.gen.Surjections; -import org.apache.cassandra.harry.model.Model; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.operations.QueryGenerator; -import org.apache.cassandra.harry.tracker.DataTracker; - -public class ParallelRecentValidator extends ParallelValidator<ParallelRecentValidator.State> -{ - private static final Logger logger = LoggerFactory.getLogger(ParallelRecentValidator.class); - - private final int partitionCount; - private final int queries; - private final QueryGenerator.TypedQueryGenerator querySelector; - private final Model model; - private final QueryLogger queryLogger; - private final OpSelectors.PdSelector pdSelector; - private final DataTracker tracker; - - public ParallelRecentValidator(int partitionCount, int concurrency, int queries, - Run run, - Model.ModelFactory modelFactory, - QueryLogger queryLogger) - { - super(concurrency, run); - this.pdSelector = run.pdSelector; - this.tracker = run.tracker; - this.partitionCount = partitionCount; - this.queries = Math.max(queries, 1); - this.querySelector = new QueryGenerator.TypedQueryGenerator(run.rng, - // TODO: make query kind configurable - Surjections.enumValues(Query.QueryKind.class), - run.rangeSelector); - this.model = modelFactory.make(run); - this.queryLogger = queryLogger; - } - - protected void doOne(State state) - { - long claim = state.claim(); - if (claim < 0) - return; - - long visitLts = run.pdSelector.minLtsAt(state.position - claim); - for (int i = 0; i < queries; i++) - { - run.metricReporter.validateRandomQuery(); - Query query = querySelector.inflate(visitLts, i); - model.validate(query); - queryLogger.logSelectQuery(i, query); - } - } - - protected CompletableFuture<Void> startThreads(ExecutorService executor, int parallelism) - { - logger.info("Validating {} recent partitions", partitionCount); - return super.startThreads(executor, parallelism); - } - - protected State initialState() - { - return new State(pdSelector.maxPosition(tracker.maxStarted())); - } - - public class State extends ParallelValidator.State - { - private final long position; - private final AtomicLong counter; - - public State(long maxPos) - { - this.position = maxPos; - this.counter = new AtomicLong(partitionCount); - } - - public long claim() - { - long v = counter.getAndDecrement(); - if (v <= 0) - signal(); - - return v; - } - } - - @JsonTypeName("parallel_validate_recent_partitions") - public static class ParallelRecentValidatorConfig implements Configuration.VisitorConfiguration - { - public final int partition_count; - public final int queries; - public final int concurrency; - public final Configuration.ModelConfiguration modelConfiguration; - public final Configuration.QueryLoggerConfiguration query_logger; - - // TODO: make query selector configurable - @JsonCreator - public ParallelRecentValidatorConfig(@JsonProperty("partition_count") int partition_count, - @JsonProperty("concurrency") int concurrency, - @JsonProperty("queries_per_partition") int queries, - @JsonProperty("model") Configuration.ModelConfiguration model, - @JsonProperty("query_logger") Configuration.QueryLoggerConfiguration query_logger) - { - this.partition_count = partition_count; - this.concurrency = concurrency; - this.queries = Math.max(queries, 1); - this.modelConfiguration = model; - this.query_logger = QueryLogger.thisOrDefault(query_logger); - } - - @Override - public Visitor make(Run run) - { - return new ParallelRecentValidator(partition_count, concurrency, queries, run, modelConfiguration, query_logger.make()); - } - } - -} diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/ParallelValidator.java b/test/harry/main/org/apache/cassandra/harry/visitors/ParallelValidator.java deleted file mode 100644 index 4c2dea20fc39..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/ParallelValidator.java +++ /dev/null @@ -1,103 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.visitors; - -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.harry.core.Run; - -public abstract class ParallelValidator<T extends ParallelValidator.State> implements Visitor -{ - private static final Logger logger = LoggerFactory.getLogger(AllPartitionsValidator.class); - - protected final Run run; - protected final int parallelism; - protected final ExecutorService executor; - - public ParallelValidator(int parallelism, - Run run) - { - this.run = run; - this.parallelism = parallelism; - this.executor = Executors.newFixedThreadPool(parallelism); - } - - protected abstract void doOne(T state); - protected abstract T initialState(); - - protected CompletableFuture<Void> startThreads(ExecutorService executor, int parallelism) - { - CompletableFuture<?>[] futures = new CompletableFuture[parallelism]; - T shared = initialState(); - - for (int i = 0; i < parallelism; i++) - { - futures[i] = CompletableFuture.supplyAsync(() -> { - while (!shared.signalled()) - doOne(shared); - - return null; - }, executor); - } - - return CompletableFuture.allOf(futures); - } - - public abstract static class State - { - private final AtomicBoolean isDone = new AtomicBoolean(false); - - public void signal() - { - isDone.set(true); - } - - public boolean signalled() - { - return isDone.get(); - } - } - - public void visit() - { - try - { - startThreads(executor, parallelism).get(); - } - catch (Throwable e) - { - throw new RuntimeException(e); - } - logger.info("Finished validations"); - } - - @Override - public void shutdown() throws InterruptedException - { - executor.shutdown(); - executor.awaitTermination(60, TimeUnit.SECONDS); - } -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/QueryLogger.java b/test/harry/main/org/apache/cassandra/harry/visitors/QueryLogger.java deleted file mode 100644 index 3eb60df74247..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/QueryLogger.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.visitors; - -import java.io.BufferedWriter; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStreamWriter; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.operations.Query; - -public interface QueryLogger -{ - static Configuration.QueryLoggerConfiguration thisOrDefault(Configuration.QueryLoggerConfiguration config) - { - if (config == null) - return () -> NO_OP; - return config; - } - - QueryLogger NO_OP = new NoOpQueryLogger(); - - void println(String a, Object... interpolate); - default void logSelectQuery(int modifier, Query query) - { - println(String.format("PD: %d. Modifier: %d.\t%s", query.pd, modifier, query.toSelectStatement())); - } - interface QueryLoggerFactory - { - QueryLogger make(); - } - - class FileQueryLogger implements QueryLogger - { - private static final Logger logger = LoggerFactory.getLogger(FileQueryLogger.class); - private final BufferedWriter log; - - public FileQueryLogger(String filename) - { - File f = new File(filename); - try - { - log = new BufferedWriter(new OutputStreamWriter(new FileOutputStream(f))); - } - catch (FileNotFoundException e) - { - throw new RuntimeException(e); - } - } - - public void println(String s, Object... interpolate) - { - try - { - log.write(String.format(s, interpolate)); - } - catch (IOException e) - { - logger.error("Could not log line", e); - } - } - } - - class NoOpQueryLogger implements QueryLogger - { - public void println(String a, Object... interpolate) - { - } - - public void logSelectQuery(int modifier, Query query) - { - - } - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/RandomPartitionValidator.java b/test/harry/main/org/apache/cassandra/harry/visitors/RandomPartitionValidator.java deleted file mode 100644 index d1bd1d931e52..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/RandomPartitionValidator.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.visitors; - -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.model.Model; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.operations.QueryGenerator; - -public class RandomPartitionValidator implements Visitor -{ - protected final Model model; - protected final QueryGenerator.TypedQueryGenerator queryGenerator; - protected final Run run; - protected final AtomicLong modifier; - protected final QueryLogger logger; - - public RandomPartitionValidator(Run run, - Model.ModelFactory modelFactory, - QueryLogger logger) - { - this.model = modelFactory.make(run); - this.queryGenerator = new QueryGenerator.TypedQueryGenerator(run); - this.run = run; - this.modifier = new AtomicLong(); - this.logger = logger; - } - - public void visit() - { - if (run.tracker.maxStarted() == 0) - return; - - long modifier = this.modifier.incrementAndGet(); - long pd = ((OpSelectors.DefaultPdSelector)run.pdSelector).randomVisitedPd(run.tracker.maxStarted(), - modifier, - run.schemaSpec); - model.validate(queryGenerator.inflate(run.pdSelector.maxLtsFor(pd), modifier)); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/RandomValidator.java b/test/harry/main/org/apache/cassandra/harry/visitors/RandomValidator.java deleted file mode 100644 index f159e1475da2..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/RandomValidator.java +++ /dev/null @@ -1,93 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.visitors; - -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.cassandra.harry.core.MetricReporter; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.gen.Surjections; -import org.apache.cassandra.harry.model.Model; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.operations.QueryGenerator; -import org.apache.cassandra.harry.tracker.DataTracker; - -public class RandomValidator implements Visitor -{ - private final QueryLogger logger; - private final Model model; - - private final OpSelectors.DefaultPdSelector pdSelector; - private final QueryGenerator.TypedQueryGenerator querySelector; - private final MetricReporter metricReporter; - private final DataTracker tracker; - private final AtomicLong modifier; - private final SchemaSpec schemaSpec; - - private final int partitionCount; - private final int queries; - - public RandomValidator(int partitionCount, - int queries, - Run run, - Model.ModelFactory modelFactory, - QueryLogger logger) - { - this.partitionCount = partitionCount; - this.queries = Math.max(queries, 1); - this.metricReporter = run.metricReporter; - this.pdSelector = (OpSelectors.DefaultPdSelector) run.pdSelector; - this.querySelector = new QueryGenerator.TypedQueryGenerator(run.rng, - Surjections.pick(Query.QueryKind.SINGLE_PARTITION), - run.rangeSelector); - this.model = modelFactory.make(run); - this.logger = logger; - this.tracker = run.tracker; - this.schemaSpec = run.schemaSpec; - - this.modifier = new AtomicLong(); - } - - // TODO: expose metric, how many times validated recent partitions - private int validateRandomPartitions() - { - for (int i = 0; i < partitionCount && !Thread.currentThread().isInterrupted(); i++) - { - metricReporter.validateRandomQuery(); - long modifier = this.modifier.incrementAndGet(); - long pd = pdSelector.randomVisitedPd(tracker.maxStarted(), modifier, schemaSpec); - for (int j = 0; j < queries && !Thread.currentThread().isInterrupted(); j++) - { - Query query = querySelector.inflate(pdSelector.maxLtsFor(pd), j); - logger.logSelectQuery(j, query); - model.validate(query); - } - } - - return partitionCount; - } - - @Override - public void visit() - { - validateRandomPartitions(); - } -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/RecentValidator.java b/test/harry/main/org/apache/cassandra/harry/visitors/RecentValidator.java deleted file mode 100644 index 30f34c56c84d..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/RecentValidator.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.visitors; - -import org.apache.cassandra.harry.core.MetricReporter; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.gen.Surjections; -import org.apache.cassandra.harry.model.Model; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.operations.QueryGenerator; - -public class RecentValidator implements Visitor -{ - private final QueryLogger queryLogger; - private final Model model; - - private final OpSelectors.PdSelector pdSelector; - private final QueryGenerator.TypedQueryGenerator querySelector; - private final MetricReporter metricReporter; - private final OpSelectors.Clock clock; - - private final int partitionCount; - private final int queries; - - public RecentValidator(int partitionCount, - int queries, - Run run, - Model.ModelFactory modelFactory, - QueryLogger queryLogger) - { - this.partitionCount = partitionCount; - this.queries = Math.max(queries, 1); - this.metricReporter = run.metricReporter; - this.pdSelector = run.pdSelector; - this.clock = run.clock; - this.querySelector = new QueryGenerator.TypedQueryGenerator(run.rng, - // TODO: make query kind configurable - Surjections.enumValues(Query.QueryKind.class), - run.rangeSelector); - this.model = modelFactory.make(run); - this.queryLogger = queryLogger; - } - - // TODO: expose metric, how many times validated recent partitions - private int validateRecentPartitions() - { - long pos = pdSelector.maxPosition(clock.peek()); - - int maxPartitions = partitionCount; - while (pos >= 0 && maxPartitions > 0 && !Thread.currentThread().isInterrupted()) - { - long visitLts = pdSelector.minLtsAt(pos); - for (int i = 0; i < queries; i++) - { - metricReporter.validateRandomQuery(); - Query query = querySelector.inflate(visitLts, i); - // TODO: add pd skipping from shrinker here, too - log(i, query); - model.validate(query); - } - - pos--; - maxPartitions--; - } - - return partitionCount - maxPartitions; - } - - @Override - public void visit() - { - validateRecentPartitions(); - } - - private void log(int modifier, Query query) - { - queryLogger.println(String.format("PD: %d. Modifier: %d.\t%s", query.pd, modifier, query.toSelectStatement())); - } -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/ReplayingVisitor.java b/test/harry/main/org/apache/cassandra/harry/visitors/ReplayingVisitor.java deleted file mode 100644 index 65fcdb87d9aa..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/ReplayingVisitor.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.visitors; - -import java.util.Arrays; -import java.util.function.LongSupplier; - -public abstract class ReplayingVisitor extends LtsVisitor -{ - public ReplayingVisitor(VisitExecutor delegate, LongSupplier ltsSource) - { - super(delegate, ltsSource); - } - - @Override - public void visit(long lts) - { - replay(getVisit(lts)); - } - - public abstract Visit getVisit(long lts); - - public abstract void replayAll(); - - private void replay(Visit visit) - { - beforeLts(visit.lts, visit.pd); - for (Operation operation : visit.operations) - operation(operation); - afterLts(visit.lts, visit.pd); - } - - public static class Visit - { - public final long lts; - public final long pd; - public final Operation[] operations; - - public Visit(long lts, long pd, Operation[] operations) - { - this.lts = lts; - this.pd = pd; - this.operations = operations; - } - - public String toString() - { - return "Visit{" + - "lts=" + lts + - ", pd=" + pd + - ", operations=[" + Arrays.toString(operations) + - "]}"; - } - - - } -} \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/Sampler.java b/test/harry/main/org/apache/cassandra/harry/visitors/Sampler.java deleted file mode 100644 index 0b2dced72832..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/Sampler.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.visitors; - -import org.apache.commons.math3.stat.descriptive.DescriptiveStatistics; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.gen.rng.RngUtils; -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.model.SelectHelper; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.tracker.DataTracker; - -public class Sampler implements Visitor -{ - private static final Logger logger = LoggerFactory.getLogger(AllPartitionsValidator.class); - - private final SystemUnderTest sut; - private final OpSelectors.PdSelector pdSelector; - private final OpSelectors.Clock clock; - private final DataTracker tracker; - private final SchemaSpec schema; - private final int samplePartitions; - - public Sampler(Run run, int samplePartitions) - { - this.sut = run.sut; - this.pdSelector = run.pdSelector; - this.clock = run.clock; - this.tracker = run.tracker; - this.schema = run.schemaSpec; - this.samplePartitions = samplePartitions; - } - - public void visit() - { - long lts = clock.peek(); - long max = pdSelector.maxPosition(tracker.maxStarted()); - DescriptiveStatistics ds = new DescriptiveStatistics(); - int empty = 0; - - long n = RngUtils.next(lts); - for (long i = 0; i < this.samplePartitions; i++) - { - long posLts = pdSelector.minLtsAt(RngUtils.asInt(n, (int) max)); - n = RngUtils.next(n); - // TODO: why not just pd at pos? - long pd = pdSelector.pd(posLts, schema); - long count = (long) sut.execute(SelectHelper.count(schema, pd), SystemUnderTest.ConsistencyLevel.ONE)[0][0]; - if (count == 0) - empty++; - ds.addValue(count); - } - logger.info("Visited {} partitions (sampled {} empty out of {}), with mean size of {}. Median: {}. Min: {}. Max: {}", - max, empty, samplePartitions, ds.getMean(), ds.getPercentile(0.5), ds.getMin(), ds.getMax()); - } - - @JsonTypeName("sampler") - public static class SamplerConfiguration implements Configuration.VisitorConfiguration - { - public final int sample_partitions; - - @JsonCreator - public SamplerConfiguration(@JsonProperty(value = "sample_partitions", defaultValue = "10") int sample_partitions) - { - this.sample_partitions = sample_partitions; - } - - public Visitor make(Run run) - { - return new Sampler(run, sample_partitions); - } - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/SingleValidator.java b/test/harry/main/org/apache/cassandra/harry/visitors/SingleValidator.java deleted file mode 100644 index 3783958e4d5b..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/SingleValidator.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.visitors; - -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.model.Model; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.operations.QueryGenerator; - -public class SingleValidator implements Visitor -{ - protected final int iterations; - protected final Model model; - protected final QueryGenerator queryGenerator; - protected final Run run; - - public SingleValidator(int iterations, - Run run, - Model.ModelFactory modelFactory) - { - this.iterations = iterations; - this.model = modelFactory.make(run); - this.queryGenerator = new QueryGenerator(run); - this.run = run; - } - - @Override - public void visit() - { - visit(run.clock.peek()); - } - - public void visit(long lts) - { - model.validate(queryGenerator.inflate(lts, 0, Query.QueryKind.SINGLE_PARTITION)); - - for (boolean reverse : new boolean[]{ true, false }) - { - model.validate(Query.selectAllColumns(run.schemaSpec, run.pdSelector.pd(lts, run.schemaSpec), reverse)); - } - - for (Query.QueryKind queryKind : new Query.QueryKind[]{ Query.QueryKind.CLUSTERING_RANGE, Query.QueryKind.CLUSTERING_SLICE, Query.QueryKind.SINGLE_CLUSTERING }) - { - for (int i = 0; i < iterations; i++) - { - model.validate(queryGenerator.inflate(lts, i, queryKind)); - } - } - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/SkippingVisitor.java b/test/harry/main/org/apache/cassandra/harry/visitors/SkippingVisitor.java deleted file mode 100644 index a8c5fb6ccb1b..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/SkippingVisitor.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.visitors; - -import java.util.Set; -import java.util.function.LongSupplier; - -public class SkippingVisitor extends LtsVisitor -{ - private final Set<Long> ltsToSkip; - private final Set<Long> pdsToSkip; - private final LtsToPd ltsToPd; - // Use DelegatingVisitor class instead of VisitExecutor available via protected field - private LtsVisitor delegateShadow; - - public SkippingVisitor(LtsVisitor delegate, - LongSupplier ltsSupplier, - LtsToPd ltsToPd, - Set<Long> ltsToSkip, - Set<Long> pdsToSkip) - { - super(delegate, ltsSupplier); - this.ltsToSkip = ltsToSkip; - this.pdsToSkip = pdsToSkip; - this.ltsToPd = ltsToPd; - } - - public void visit(long lts) - { - if (ltsToSkip.contains(lts) || pdsToSkip.contains(ltsToPd.convert(lts))) - return; - - delegateShadow.visit(lts); - } - - public static interface LtsToPd - { - public long convert(long lts); - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/VisitExecutor.java b/test/harry/main/org/apache/cassandra/harry/visitors/VisitExecutor.java deleted file mode 100644 index a0e9c8e73c06..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/VisitExecutor.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.visitors; - -import org.apache.cassandra.harry.model.OpSelectors; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.util.BitSet; - -public abstract class VisitExecutor -{ - protected abstract void beforeLts(long lts, long pd); - - protected abstract void afterLts(long lts, long pd); - - protected abstract void operation(Operation operation); - - public abstract void shutdown() throws InterruptedException; - - public interface WriteOp extends Operation - { - long cd(); - long[] vds(); - } - - public interface WriteStaticOp extends WriteOp - { - long[] sds(); - } - - public interface DeleteRowOp extends Operation - { - long cd(); - } - - public interface DeleteOp extends Operation - { - Query relations(); - } - - public interface DeleteColumnsOp extends Operation - { - long cd(); - BitSet columns(); - } - - public interface Operation - { - long pd(); - long lts(); - long opId(); - OpSelectors.OperationKind kind(); - } - - public static abstract class BaseOperation implements Operation - { - public final long lts; - public final long opId; - public final long pd; - public final OpSelectors.OperationKind kind; - - public BaseOperation(long lts, long pd, long opId, OpSelectors.OperationKind kind) - { - assert opId >= 0; - this.pd = pd; - this.lts = lts; - this.opId = opId; - this.kind = kind; - } - - @Override - public final long pd() - { - return pd; - - } - - @Override - public final long lts() - { - return lts; - } - - @Override - public final long opId() - { - return opId; - } - - @Override - public final OpSelectors.OperationKind kind() - { - return kind; - - } - - public String toString() - { - return "Operation{" + - " lts=" + lts + - " pd=" + pd + - " opId=" + opId + - ", kind=" + kind + - '}'; - } - } -} diff --git a/test/harry/main/org/apache/cassandra/harry/visitors/Visitor.java b/test/harry/main/org/apache/cassandra/harry/visitors/Visitor.java deleted file mode 100644 index ccac2ded574d..000000000000 --- a/test/harry/main/org/apache/cassandra/harry/visitors/Visitor.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.harry.visitors; - -import org.apache.cassandra.harry.core.Run; - -public interface Visitor -{ - void visit(); - - default void shutdown() throws InterruptedException {} - - interface VisitorFactory - { - Visitor make(Run run); - } -} \ No newline at end of file diff --git a/test/simulator/main/org/apache/cassandra/simulator/cluster/KeyspaceActions.java b/test/simulator/main/org/apache/cassandra/simulator/cluster/KeyspaceActions.java index 4b6caa0164c8..5d32ab6eaa91 100644 --- a/test/simulator/main/org/apache/cassandra/simulator/cluster/KeyspaceActions.java +++ b/test/simulator/main/org/apache/cassandra/simulator/cluster/KeyspaceActions.java @@ -33,7 +33,7 @@ import org.apache.cassandra.dht.Murmur3Partitioner.LongToken; import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.api.ConsistencyLevel; -import org.apache.cassandra.harry.sut.TokenPlacementModel; +import org.apache.cassandra.harry.model.TokenPlacementModel; import org.apache.cassandra.simulator.Action; import org.apache.cassandra.simulator.ActionList; import org.apache.cassandra.simulator.ActionListener; diff --git a/test/simulator/main/org/apache/cassandra/simulator/harry/HarryValidatingQuery.java b/test/simulator/main/org/apache/cassandra/simulator/harry/HarryValidatingQuery.java deleted file mode 100644 index 05620b5517ff..000000000000 --- a/test/simulator/main/org/apache/cassandra/simulator/harry/HarryValidatingQuery.java +++ /dev/null @@ -1,146 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.simulator.harry; - -import java.util.ArrayList; -import java.util.List; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.data.ResultSetRow; -import org.apache.cassandra.harry.model.Model; -import org.apache.cassandra.harry.model.SelectHelper; -import org.apache.cassandra.harry.operations.CompiledStatement; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.distributed.Cluster; -import org.apache.cassandra.distributed.api.IInstance; -import org.apache.cassandra.harry.sut.TokenPlacementModel; -import org.apache.cassandra.harry.sut.injvm.QuiescentLocalStateChecker; -import org.apache.cassandra.simulator.systems.InterceptedExecution; -import org.apache.cassandra.simulator.systems.InterceptingExecutor; -import org.apache.cassandra.simulator.systems.SimulatedAction; -import org.apache.cassandra.simulator.systems.SimulatedSystems; - -public class HarryValidatingQuery extends SimulatedAction -{ - private static final Logger logger = LoggerFactory.getLogger(HarryValidatingQuery.class); - private final Run run; - private final Query query; - private final InterceptingExecutor on; - private final Cluster cluster; - private final List<TokenPlacementModel.Node> owernship; - private final TokenPlacementModel.ReplicationFactor rf; - - public HarryValidatingQuery(SimulatedSystems simulated, - Cluster cluster, - TokenPlacementModel.ReplicationFactor rf, - Run run, - List<TokenPlacementModel.Node> owernship, - Query query) - { - super(query, Modifiers.RELIABLE_NO_TIMEOUTS, Modifiers.RELIABLE_NO_TIMEOUTS, null, simulated); - this.rf = rf; - this.cluster = cluster; - this.on = (InterceptingExecutor) cluster.get(1).executor(); - this.run = run; - this.query = query; - this.owernship = owernship; - } - - protected InterceptedExecution task() - { - return new InterceptedExecution.InterceptedTaskExecution(on) - { - public void run() - { - try - { - QuiescentLocalStateChecker model = new QuiescentLocalStateChecker(run, rf) - { - @Override - protected TokenPlacementModel.ReplicatedRanges getRing() - { - return rf.replicate(owernship); - } - - @Override - protected void validate(Query query, TokenPlacementModel.ReplicatedRanges ring) - { - CompiledStatement compiled = query.toSelectStatement(); - List<TokenPlacementModel.Replica> replicas = ring.replicasFor(this.token(query.pd)); - logger.trace("Predicted {} as replicas for {}. Ring: {}", new Object[]{ replicas, query.pd, ring }); - List<Throwable> throwables = new ArrayList<>(); - for (TokenPlacementModel.Replica replica : replicas) - { - try - { - validate(() -> { - Object[][] objects = this.executeNodeLocal(compiled.cql(), replica.node(), compiled.bindings()); - List<ResultSetRow> result = new ArrayList(); - int length = objects.length; - for (int i = 0; i < length; ++i) - { - Object[] res = objects[i]; - result.add(SelectHelper.resultSetToRow(query.schemaSpec, this.clock, res)); - } - - return result; - }, query); - } - catch (Model.ValidationException t) - { - throwables.add(new AssertionError(String.format("Caught an exception while validating %s on %s", query, replica), t)); - } - } - - if (!throwables.isEmpty()) - { - AssertionError error = new AssertionError(String.format("Could not validate %d out of %d replicas %s", throwables.size(), replicas.size(), replicas)); - throwables.forEach(error::addSuppressed); - throw error; - } - } - - - @Override - protected Object[][] executeNodeLocal(String statement, TokenPlacementModel.Node node, Object... bindings) - { - IInstance instance = cluster - .stream() - .filter((n) -> n.config().broadcastAddress().toString().equals(node.id())) - .findFirst() - .get(); - return instance.executeInternal(statement, bindings); - } - }; - - model.validate(query); - - } - catch (Throwable t) - { - logger.error("Caught an exception while validating", t); - throw t; - } - } - }; - } -} diff --git a/test/simulator/test/org/apache/cassandra/simulator/test/HarrySimulatorTest.java b/test/simulator/test/org/apache/cassandra/simulator/test/HarrySimulatorTest.java index e98b8ddd0d0d..9fc7e6983243 100644 --- a/test/simulator/test/org/apache/cassandra/simulator/test/HarrySimulatorTest.java +++ b/test/simulator/test/org/apache/cassandra/simulator/test/HarrySimulatorTest.java @@ -25,13 +25,13 @@ import java.util.EnumMap; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Random; import java.util.Set; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; @@ -41,18 +41,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.harry.clock.OffsetClock; -import org.apache.cassandra.harry.core.Configuration; -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.ddl.ColumnSpec; -import org.apache.cassandra.harry.ddl.SchemaGenerators; -import org.apache.cassandra.harry.ddl.SchemaSpec; -import org.apache.cassandra.harry.gen.Surjections; -import org.apache.cassandra.harry.operations.Query; -import org.apache.cassandra.harry.sut.SystemUnderTest; -import org.apache.cassandra.harry.sut.injvm.InJvmSut; -import org.apache.cassandra.harry.tracker.DefaultDataTracker; -import org.apache.cassandra.harry.visitors.GeneratingVisitor; +import accord.utils.Invariants; import io.airlift.airline.Command; import io.airlift.airline.HelpOption; import io.airlift.airline.Option; @@ -63,9 +52,22 @@ import org.apache.cassandra.distributed.api.IInstanceConfig; import org.apache.cassandra.distributed.api.IInvokableInstance; import org.apache.cassandra.distributed.api.IIsolatedExecutor; -import org.apache.cassandra.harry.HarryHelper; +import org.apache.cassandra.distributed.impl.Query; import org.apache.cassandra.distributed.shared.WithProperties; -import org.apache.cassandra.harry.sut.TokenPlacementModel; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.op.Visit; +import org.apache.cassandra.harry.op.Operations; +import org.apache.cassandra.harry.gen.OperationsGenerators; +import org.apache.cassandra.harry.execution.CompiledStatement; +import org.apache.cassandra.harry.execution.DataTracker; +import org.apache.cassandra.harry.execution.QueryBuildingVisitExecutor; +import org.apache.cassandra.harry.gen.EntropySource; +import org.apache.cassandra.harry.gen.Generator; +import org.apache.cassandra.harry.gen.SchemaGenerators; +import org.apache.cassandra.harry.gen.rng.JdkRandomEntropySource; +import org.apache.cassandra.harry.model.Model; +import org.apache.cassandra.harry.model.QuiescentChecker; +import org.apache.cassandra.harry.model.TokenPlacementModel; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.Verb; import org.apache.cassandra.schema.ReplicationParams; @@ -88,10 +90,10 @@ import org.apache.cassandra.simulator.cluster.ClusterActionListener.NoOpListener; import org.apache.cassandra.simulator.cluster.ClusterActions; import org.apache.cassandra.simulator.cluster.ClusterActions.Options; -import org.apache.cassandra.simulator.harry.HarryValidatingQuery; import org.apache.cassandra.simulator.systems.Failures; import org.apache.cassandra.simulator.systems.InterceptedExecution; import org.apache.cassandra.simulator.systems.InterceptingExecutor; +import org.apache.cassandra.simulator.systems.SimulatedActionCallable; import org.apache.cassandra.simulator.systems.SimulatedActionTask; import org.apache.cassandra.simulator.systems.SimulatedSystems; import org.apache.cassandra.simulator.systems.SimulatedTime; @@ -103,10 +105,11 @@ import org.apache.cassandra.tcm.sequences.SingleNodeSequences; import org.apache.cassandra.tcm.transformations.PrepareJoin; import org.apache.cassandra.utils.CloseableIterator; +import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException; import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL; -import static org.apache.cassandra.harry.sut.TokenPlacementModel.constantLookup; +import static org.apache.cassandra.harry.model.TokenPlacementModel.constantLookup; import static org.apache.cassandra.simulator.ActionSchedule.Mode.UNLIMITED; import static org.apache.cassandra.simulator.cluster.ClusterActions.Options.noActions; @@ -205,9 +208,9 @@ public static void main(String... args) throws Throwable @Test public void test() throws Exception { - rowsPerPhase = 1; // To rerun a failing test for a given seed, uncomment the below and set the seed // this.seed = "<your seed here>"; + this.seed = "0xdd3bb3793a6b925a"; harryTest(); } @@ -248,17 +251,6 @@ private void harryTest() throws Exception // Backoff should be larger than read timeout, since otherwise we will simply saturate the stage with retries .set("progress_barrier_backoff", "1000ms") .set("cms_await_timeout", "600000ms"), - HarryHelper.defaultConfiguration() - .setSchemaProvider(new Configuration.SchemaProviderConfiguration() - { - private final Surjections.Surjection<SchemaSpec> schema = schemaSpecGen("harry", "tbl"); - public SchemaSpec make(long l, SystemUnderTest systemUnderTest) - { - return schema.inflate(l); - } - }) - .setPartitionDescriptorSelector(new Configuration.DefaultPDSelectorConfiguration(2, 1)) - .setClusteringDescriptorSelector(HarryHelper.singleRowPerModification().setMaxPartitionSize(100).build()), arr(), (simulation) -> { simulation.cluster.stream().forEach((IInvokableInstance i) -> { @@ -285,16 +277,16 @@ public SchemaSpec make(long l, SystemUnderTest systemUnderTest) work.add(work("Create Keyspace", simulation.clusterActions.schemaChange(1, String.format("CREATE KEYSPACE %s WITH replication = {'class': 'NetworkTopologyStrategy', " + rfString + "};", - simulation.harryRun.schemaSpec.keyspace)))); + simulation.schema.keyspace)))); work.add(work("Create table", simulation.clusterActions.schemaChange(1, - simulation.harryRun.schemaSpec.compile().cql()))); + simulation.schema.compile()))); simulation.cluster.stream().forEach(i -> { work.add(work("Output epoch", lazy(simulation.simulated, i, () -> logger.warn(ClusterMetadata.current().epoch.toString())))); }); - work.add(interleave("Start generating", HarrySimulatorTest.generate(rowsPerPhase, simulation, cl))); + work.add(interleave("Start generating", HarrySimulatorTest.generateWrites(rowsPerPhase, simulation, cl))); work.add(work("Validate all data locally", lazy(() -> validateAllLocal(simulation, simulation.nodeState.ring, rf)))); @@ -319,7 +311,7 @@ public SchemaSpec make(long l, SystemUnderTest systemUnderTest) long token = simulation.simulated.random.uniform(Long.MIN_VALUE, Long.MAX_VALUE); work.add(interleave("Bootstrap and generate data", ActionList.of(bootstrap(simulation.simulated, simulation.cluster, token, node)), - generate(rowsPerPhase, simulation, cl) + generateWrites(rowsPerPhase, simulation, cl) )); simulation.cluster.stream().forEach(i -> { work.add(work("Output epoch", @@ -337,7 +329,7 @@ public SchemaSpec make(long l, SystemUnderTest systemUnderTest) node = bootstrappedNodes.remove(0); work.add(interleave("Decommission and generate data", ActionList.of(decommission(simulation.simulated, simulation.cluster, node)), - generate(rowsPerPhase, simulation, cl) + generateWrites(rowsPerPhase, simulation, cl) )); simulation.cluster.stream().forEach(i -> { work.add(work("Output epoch", @@ -372,41 +364,94 @@ static class HarrySimulation implements Simulation { protected final ClusterActions clusterActions; protected final SimulatedNodeState nodeState; - protected final Run harryRun; protected final SimulatedSystems simulated; protected final RunnableActionScheduler scheduler; protected final Cluster cluster; protected final Function<HarrySimulation, ActionSchedule.Work[]> schedule; - public HarrySimulation(SimulatedSystems simulated, RunnableActionScheduler scheduler, Cluster cluster, Run run, Function<HarrySimulation, ActionSchedule.Work[]> schedule) + protected final EntropySource rng; + protected final SchemaSpec schema; + protected final Generator<OperationsGenerators.ToOp> insertGen; + protected final QueryBuildingVisitExecutor queryBuilder; + protected final QuiescentChecker model; + + protected final Map<Long, Visit> log; + protected final Generator<Long> ltsGen; + protected final DataTracker tracker; + + private HarrySimulation(SchemaSpec schema, + EntropySource rng, + SimulatedSystems simulated, + RunnableActionScheduler scheduler, + Cluster cluster, + Function<HarrySimulation, SimulatedNodeState> nodeState, + Function<HarrySimulation, ActionSchedule.Work[]> schedule, + Map<Long, Visit> log, + Generator<Long> ltsGen, + DataTracker tracker) { - this(simulated, scheduler, cluster, run, SimulatedNodeState::new, schedule); - } + this.rng = rng; + this.schema = schema; + this.insertGen = OperationsGenerators.writeOp(schema); + this.queryBuilder = new QueryBuildingVisitExecutor(schema, QueryBuildingVisitExecutor.WrapQueries.UNLOGGED_BATCH); + this.model = new QuiescentChecker(schema.valueGenerators, tracker, new Model.Replay() + { + @Override + public Visit replay(long lts) + { + return log.get(lts); + } + + @Override + public Operations.Operation replay(long lts, int opId) + { + return log.get(lts).operations[opId]; + } + + @Override + public Iterator<Visit> iterator() + { + List<Long> visited = new ArrayList<>(log.keySet()); + visited.sort(Long::compare); + return new Iterator<>() + { + int idx = 0; + + @Override + public boolean hasNext() + { + return idx < visited.size(); + } + + @Override + public Visit next() + { + return replay(visited.get(idx++)); + } + }; + } + }); - protected HarrySimulation(SimulatedSystems simulated, - RunnableActionScheduler scheduler, - Cluster cluster, - Run run, - Function<HarrySimulation, SimulatedNodeState> nodeState, - Function<HarrySimulation, ActionSchedule.Work[]> schedule) - { this.simulated = simulated; this.scheduler = scheduler; this.cluster = cluster; - this.harryRun = run; Options options = noActions(cluster.size()); this.clusterActions = new ClusterActions(simulated, cluster, options, new NoOpListener(), new Debug(new EnumMap<>(Debug.Info.class), new int[0])); this.nodeState = nodeState.apply(this); this.schedule = schedule; + + this.log = log; + this.ltsGen = ltsGen; + this.tracker = tracker; } public HarrySimulation withScheduler(RunnableActionScheduler scheduler) { - return new HarrySimulation(simulated, scheduler, cluster, harryRun, (ignore) -> nodeState, schedule); + return new HarrySimulation(schema, rng, simulated, scheduler, cluster, (ignore) -> nodeState, schedule, log, ltsGen, tracker); } public HarrySimulation withSchedulers(Function<HarrySimulation, Map<Verb, FutureActionScheduler>> schedulers) @@ -423,12 +468,12 @@ public HarrySimulation withSchedulers(Function<HarrySimulation, Map<Verb, Future perVerbFutureActionScheduler, this.simulated.debug, this.simulated.failures); - return new HarrySimulation(simulated, scheduler, cluster, harryRun, (ignore) -> nodeState, schedule); + return new HarrySimulation(schema, rng, simulated, scheduler, cluster, (ignore) -> nodeState, schedule, log, ltsGen, tracker); } public HarrySimulation withSchedule(Function<HarrySimulation, ActionSchedule.Work[]> schedule) { - return new HarrySimulation(simulated, scheduler, cluster, harryRun, (ignore) -> nodeState, schedule); + return new HarrySimulation(schema, rng, simulated, scheduler, cluster, (ignore) -> nodeState, schedule, log, ltsGen, tracker); } @Override @@ -470,13 +515,10 @@ public void close() throws Exception static class HarrySimulationBuilder extends ClusterSimulation.Builder<HarrySimulation> { - protected final Configuration.ConfigurationBuilder harryConfig; protected final Consumer<IInstanceConfig> configUpdater; - HarrySimulationBuilder(Configuration.ConfigurationBuilder harryConfig, - Consumer<IInstanceConfig> configUpdater) + HarrySimulationBuilder(Consumer<IInstanceConfig> configUpdater) { - this.harryConfig = harryConfig; this.configUpdater = configUpdater; } @@ -497,29 +539,22 @@ public ClusterSimulation<HarrySimulation> create(long seed) throws IOException { RandomSource random = new RandomSource.Default(); random.reset(seed); - this.harryConfig.setSeed(seed); - return new ClusterSimulation<>(random, seed, 1, this, configUpdater, (simulated, scheduler, cluster, options) -> { - - InJvmSut sut = new InJvmSut(cluster) - { - public void shutdown() - { - // Let simulation shut down the cluster, as it uses `nanoTime` - } - }; - - Configuration configuration = harryConfig.setClock(() -> new OffsetClock(1000)) - .setSUT(() -> sut) - .build(); - return new HarrySimulation(simulated, + EntropySource rng = new JdkRandomEntropySource(seed); + SchemaSpec schema = schemaSpecGen("harry", "tbl").generate(rng); + return new HarrySimulation(schema, + rng, + simulated, scheduler, cluster, - configuration.createRun(), + SimulatedNodeState::new, // No work initially - (sim) -> new ActionSchedule.Work[0]); + (sim) -> new ActionSchedule.Work[0], + new HashMap<>(), + OperationsGenerators.lts(), + new DataTracker.SimpleDataTracker()); }); } } @@ -529,13 +564,12 @@ public void shutdown() */ void simulate(Consumer<ClusterSimulation.Builder<HarrySimulation>> configure, Consumer<IInstanceConfig> instanceConfigUpdater, - Configuration.ConfigurationBuilder harryConfig, String[] properties, Function<HarrySimulation, ActionSchedule.Work[]>... phases) throws IOException { try (WithProperties p = new WithProperties().with(properties)) { - HarrySimulationBuilder factory = new HarrySimulationBuilder(harryConfig, instanceConfigUpdater); + HarrySimulationBuilder factory = new HarrySimulationBuilder(instanceConfigUpdater); SimulationRunner.beforeAll(); long seed = SimulationRunner.parseHex(Optional.ofNullable(this.seed)).orElseGet(() -> new Random().nextLong()); @@ -569,7 +603,6 @@ public static Map<Verb, FutureActionScheduler> networkSchedulers(int nodes, Simu { Set<Verb> extremelyLossy = new HashSet<>(Arrays.asList(Verb.TCM_ABORT_MIG, Verb.TCM_REPLICATION, Verb.TCM_COMMIT_REQ, Verb.TCM_NOTIFY_REQ, - Verb.TCM_FETCH_CMS_LOG_REQ, Verb.TCM_FETCH_PEER_LOG_REQ, Verb.TCM_INIT_MIG_REQ, Verb.TCM_INIT_MIG_RSP, Verb.TCM_DISCOVER_REQ, Verb.TCM_DISCOVER_RSP)); @@ -578,6 +611,7 @@ public static Map<Verb, FutureActionScheduler> networkSchedulers(int nodes, Simu Set<Verb> somewhatLossy = new HashSet<>(Arrays.asList(Verb.TCM_CURRENT_EPOCH_REQ, Verb.TCM_NOTIFY_RSP, Verb.TCM_FETCH_CMS_LOG_RSP, Verb.TCM_FETCH_PEER_LOG_RSP, Verb.TCM_COMMIT_RSP, + Verb.TCM_FETCH_CMS_LOG_REQ, Verb.TCM_FETCH_PEER_LOG_REQ, Verb.PAXOS2_COMMIT_REMOTE_REQ, Verb.PAXOS2_COMMIT_REMOTE_RSP, Verb.PAXOS2_PREPARE_REQ, Verb.PAXOS2_PREPARE_RSP, Verb.PAXOS2_PROPOSE_REQ, Verb.PAXOS2_PROPOSE_RSP, @@ -713,74 +747,101 @@ private static Action assertNodeState(SimulatedSystems simulated, Cluster cluste /** * Creates an action list with a fixed number of data-generating operations that conform to the given Harry configuration. */ - public static ActionList generate(int ops, HarrySimulation simulation, ConsistencyLevel cl) + public static ActionList generateWrites(int ops, HarrySimulation simulation, ConsistencyLevel cl) { Action[] actions = new Action[ops]; OrderOn orderOn = new OrderOn.Strict(actions, 2); - generate(ops, simulation, new Consumer<Action>() - { - int i = 0; + for (int i = 0; i < ops; i++) + { + long lts = simulation.ltsGen.generate(simulation.rng); + + Visit visit = new Visit(lts, new Operations.Operation[]{ simulation.insertGen.generate(simulation.rng).toOp(lts) }); + Visit prev_ = simulation.log.put(lts, visit); + Invariants.checkState(prev_ == null); + + actions[i] = new Actions.LambdaAction("", Action.Modifiers.RELIABLE_NO_TIMEOUTS, () -> { + CompiledStatement compiledStatement = simulation.queryBuilder.compile(visit); + DataTracker tracker = simulation.tracker; + + RetryingQuery query = new RetryingQuery(compiledStatement.cql(), cl, compiledStatement.bindings()); + Action wrapper = new SimulatedActionCallable<>("Query", + Action.Modifiers.RELIABLE_NO_TIMEOUTS, + Action.Modifiers.RELIABLE_NO_TIMEOUTS, + simulation.simulated, + simulation.cluster.get((int) ((lts % simulation.cluster.size()) + 1)), + query) + { + @Override + protected InterceptedExecution.InterceptedTaskExecution task() + { + return new InterceptedExecution.InterceptedTaskExecution((InterceptingExecutor) on.executor()) + { + public void run() + { + tracker.begin(visit); + System.out.println("Started visit = " + visit); + // we'll be invoked on the node's executor, but we need to ensure the task is loaded on its classloader + try + { + accept(on.unsafeCallOnThisThread(execute), null); + } + catch (Throwable t) + { + accept(null, t); + } + finally + { + execute = null; + } + } + }; + } + + @Override + public void accept(Object[][] result, Throwable failure) + { + if (failure != null) + simulated.failures.accept(failure); + else + { + System.out.println("Finished visit = " + visit); + tracker.end(visit); + } + } + }; - public void accept(Action action) - { - actions[i++] = action; - } - }, - cl); + return ActionList.of(wrapper); + }); + } return ActionList.of(actions).orderOn(orderOn); } - public static void generate(int ops, HarrySimulation simulation, Consumer<Action> add, org.apache.cassandra.distributed.api.ConsistencyLevel cl) + public static class RetryingQuery extends Query { - SimulatedVisitExectuor visitExectuor = new SimulatedVisitExectuor(simulation, - simulation.harryRun, - cl); - GeneratingVisitor generatingVisitor = new GeneratingVisitor(simulation.harryRun, visitExectuor); - - for (int i = 0; i < ops; i++) + public RetryingQuery(String query, ConsistencyLevel cl, Object[] boundValues) { - generatingVisitor.visit(simulation.harryRun.clock.nextLts()); - // A tiny chance of executing a multi-partition batch - if (ops % 10 == 0) - generatingVisitor.visit(simulation.harryRun.clock.nextLts()); - add.accept(visitExectuor.build()); + super(query, -1, cl, null, boundValues); } - } - - /** - * Create an infinite stream to generate data. - */ - public static Supplier<Action> generate(HarrySimulation simulation, org.apache.cassandra.distributed.api.ConsistencyLevel cl) - { - SimulatedVisitExectuor visitExectuor = new SimulatedVisitExectuor(simulation, - simulation.harryRun, - cl); - GeneratingVisitor generatingVisitor = new GeneratingVisitor(simulation.harryRun, visitExectuor); - - DefaultDataTracker tracker = (DefaultDataTracker) simulation.harryRun.tracker; - return new Supplier<Action>() + @Override + public Object[][] call() { - public Action get() + while (true) { - // Limit how many queries can be in-flight simultaneously to reduce noise - if (tracker.maxStarted() - tracker.maxConsecutiveFinished() == 0) + try { - generatingVisitor.visit(); - return visitExectuor.build(); + return super.call(); } - else + catch (UncheckedInterruptedException e) { - // No-op - return run(() -> {}); + throw new RuntimeException(e); + } + catch (Throwable t) + { + logger.error("Caught error while executing query. Will ignore and retry: " + t.getMessage()); } } - - public String toString() - { - return "Query Generator"; - } - }; + } } /** @@ -791,24 +852,37 @@ public static Action validateAllLocal(HarrySimulation simulation, List<TokenPlac { return new Actions.LambdaAction("Validate", Action.Modifiers.RELIABLE_NO_TIMEOUTS, () -> { - if (!simulation.harryRun.tracker.isFinished(simulation.harryRun.tracker.maxStarted())) - throw new IllegalStateException("Can not begin validation, as writing has not quiesced yet: " + simulation.harryRun.tracker); + if (!simulation.tracker.allFinished()) + throw new IllegalStateException("Can not begin validation, as writing has not quiesced yet: " + simulation.tracker); + + logger.warn("Starting validation. Ring view: {}", simulation.nodeState); + Set<Long> pds = visitedPds(simulation); List<Action> actions = new ArrayList<>(); - long maxLts = simulation.harryRun.tracker.maxStarted(); - long maxPosition = simulation.harryRun.pdSelector.maxPosition(maxLts); - logger.warn("Starting validation of {} written partitions. Highest LTS is {}. Ring view: {}", maxPosition, maxLts, simulation.nodeState); - for (int position = 0; position < maxPosition; position++) + for (Long pd : pds) { - long minLts = simulation.harryRun.pdSelector.minLtsAt(position); - long pd = simulation.harryRun.pdSelector.pd(minLts, simulation.harryRun.schemaSpec); - Query query = Query.selectAllColumns(simulation.harryRun.schemaSpec, pd, false); - actions.add(new HarryValidatingQuery(simulation.simulated, simulation.cluster, rf, - simulation.harryRun, owernship, query)); + Operations.SelectPartition select = new Operations.SelectPartition(Long.MAX_VALUE, pd); + actions.add(new HarryValidatingQuery(simulation, simulation.cluster, rf, + owernship, new Visit(Long.MAX_VALUE, new Operations.Operation[]{ select }), + simulation.queryBuilder)); } return ActionList.of(actions).setStrictlySequential(); }); } + private static Set<Long> visitedPds(HarrySimulation simulation) + { + Set<Long> pds = new HashSet<>(); + for (Visit visit : simulation.log.values()) + { + for (Operations.Operation operation : visit.operations) + { + if (operation instanceof Operations.PartitionOperation) + pds.add(((Operations.PartitionOperation) operation).pd); + } + } + return pds; + } + private static ActionSchedule.Work work(String toString, Action... actions) { return new ActionSchedule.Work(UNLIMITED, Collections.singletonList(ActionList.of(actions).setStrictlySequential())) { @@ -963,29 +1037,9 @@ public static <T> T[] arr(T... arr) return arr; } - // Use only types that can guarantee we will et 64 bits of entropy here, at least for now. - public static Surjections.Surjection<SchemaSpec> schemaSpecGen(String keyspace, String prefix) + public static Generator<SchemaSpec> schemaSpecGen(String keyspace, String prefix) { - AtomicInteger counter = new AtomicInteger(); - return new SchemaGenerators.Builder(keyspace, () -> prefix + counter.getAndIncrement()) - .partitionKeySpec(1, 2, - ColumnSpec.int64Type, - ColumnSpec.asciiType, - ColumnSpec.textType) - .clusteringKeySpec(1, 1, - ColumnSpec.int64Type, - ColumnSpec.asciiType, - ColumnSpec.textType, - ColumnSpec.ReversedType.getInstance(ColumnSpec.int64Type), - ColumnSpec.ReversedType.getInstance(ColumnSpec.asciiType), - ColumnSpec.ReversedType.getInstance(ColumnSpec.textType)) - .regularColumnSpec(5, 5, - ColumnSpec.int64Type, - ColumnSpec.asciiType(4, 128)) - .staticColumnSpec(5, 5, - ColumnSpec.int64Type, - ColumnSpec.asciiType(4, 128)) - .surjection(); + return SchemaGenerators.schemaSpecGen(keyspace, prefix, 1000); } public static class HaltOnError extends Failures diff --git a/test/simulator/test/org/apache/cassandra/simulator/test/HarryValidatingQuery.java b/test/simulator/test/org/apache/cassandra/simulator/test/HarryValidatingQuery.java new file mode 100644 index 000000000000..2a011c944c74 --- /dev/null +++ b/test/simulator/test/org/apache/cassandra/simulator/test/HarryValidatingQuery.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.simulator.test; + +import java.util.List; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import accord.utils.Invariants; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.IInstance; +import org.apache.cassandra.harry.op.Visit; +import org.apache.cassandra.harry.op.Operations; +import org.apache.cassandra.harry.execution.CompiledStatement; +import org.apache.cassandra.harry.execution.InJvmDTestVisitExecutor; +import org.apache.cassandra.harry.execution.QueryBuildingVisitExecutor; +import org.apache.cassandra.harry.execution.ResultSetRow; +import org.apache.cassandra.harry.model.TokenPlacementModel; +import org.apache.cassandra.harry.util.ByteUtils; +import org.apache.cassandra.harry.util.TokenUtil; +import org.apache.cassandra.simulator.systems.InterceptedExecution; +import org.apache.cassandra.simulator.systems.InterceptingExecutor; +import org.apache.cassandra.simulator.systems.SimulatedAction; + +public class HarryValidatingQuery extends SimulatedAction +{ + private static final Logger logger = LoggerFactory.getLogger(HarryValidatingQuery.class); + + private final InterceptingExecutor on; + private final Cluster cluster; + private final List<TokenPlacementModel.Node> owernship; + private final TokenPlacementModel.ReplicationFactor rf; + + private final HarrySimulatorTest.HarrySimulation simulation; + private final Visit visit; + private final QueryBuildingVisitExecutor queryBuilder; + + public HarryValidatingQuery(HarrySimulatorTest.HarrySimulation simulation, + Cluster cluster, + TokenPlacementModel.ReplicationFactor rf, + List<TokenPlacementModel.Node> owernship, + Visit visit, + QueryBuildingVisitExecutor queryBuilder) + { + super(visit, Modifiers.RELIABLE_NO_TIMEOUTS, Modifiers.RELIABLE_NO_TIMEOUTS, null, simulation.simulated); + this.rf = rf; + this.cluster = cluster; + this.on = (InterceptingExecutor) cluster.get(1).executor(); + this.owernship = owernship; + this.visit = visit; + this.queryBuilder = queryBuilder; + this.simulation = simulation; + + } + + protected InterceptedExecution task() + { + return new InterceptedExecution.InterceptedTaskExecution(on) + { + public void run() + { + try + { + TokenPlacementModel.ReplicatedRanges ring = rf.replicate(owernship); + Invariants.checkState(visit.operations.length == 1); + Invariants.checkState(visit.operations[0] instanceof Operations.SelectStatement); + Operations.SelectStatement select = (Operations.SelectStatement) visit.operations[0]; + for (TokenPlacementModel.Replica replica : ring.replicasFor(token(select.pd))) + { + CompiledStatement compiled = queryBuilder.compile(visit); + Object[][] objects = executeNodeLocal(compiled.cql(), replica.node(), compiled.bindings()); + List<ResultSetRow> actualRows = InJvmDTestVisitExecutor.rowsToResultSet(simulation.schema, select, objects); + simulation.model.validate(select, actualRows); + } + } + catch (Throwable t) + { + logger.error("Caught an exception while validating", t); + throw t; + } + } + }; + } + + protected long token(long pd) + { + return TokenUtil.token(ByteUtils.compose(ByteUtils.objectsToBytes(simulation.schema.valueGenerators.pkGen().inflate(pd)))); + } + + protected Object[][] executeNodeLocal(String statement, TokenPlacementModel.Node node, Object... bindings) + { + IInstance instance = cluster + .stream() + .filter((n) -> n.config().broadcastAddress().toString().equals(node.id())) + .findFirst() + .get(); + return instance.executeInternal(statement, bindings); + } +} diff --git a/test/simulator/test/org/apache/cassandra/simulator/test/SimulatedVisitExectuor.java b/test/simulator/test/org/apache/cassandra/simulator/test/SimulatedVisitExectuor.java deleted file mode 100644 index d676ec8409f7..000000000000 --- a/test/simulator/test/org/apache/cassandra/simulator/test/SimulatedVisitExectuor.java +++ /dev/null @@ -1,175 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.simulator.test; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.harry.core.Run; -import org.apache.cassandra.harry.operations.CompiledStatement; -import org.apache.cassandra.harry.tracker.DataTracker; -import org.apache.cassandra.harry.visitors.MutatingRowVisitor; -import org.apache.cassandra.harry.visitors.VisitExecutor; -import org.apache.cassandra.distributed.api.ConsistencyLevel; -import org.apache.cassandra.distributed.impl.Query; -import org.apache.cassandra.simulator.Action; -import org.apache.cassandra.simulator.systems.InterceptedExecution; -import org.apache.cassandra.simulator.systems.InterceptingExecutor; -import org.apache.cassandra.simulator.systems.SimulatedActionCallable; -import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException; - -/** - * Point of integration between Harry and Simulator. Creates series of stictly-ordered tasks that constitute a visit - * of a single LTS. - */ -public class SimulatedVisitExectuor extends VisitExecutor -{ - private static final Logger logger = LoggerFactory.getLogger(SimulatedVisitExectuor.class); - - private Action action = null; - private final List<Long> lts = new ArrayList<>(); - private final List<String> statements = new ArrayList<>(); - private final List<Object> bindings = new ArrayList<>(); - private final MutatingRowVisitor rowVisitor; - private final DataTracker tracker; - private final HarrySimulatorTest.HarrySimulation simulation; - private final ConsistencyLevel cl; - - public SimulatedVisitExectuor(HarrySimulatorTest.HarrySimulation simulation, - Run run, - ConsistencyLevel cl) - { - this.rowVisitor = new MutatingRowVisitor(run); - this.simulation = simulation; - this.tracker = run.tracker; - this.cl = cl; - } - - public Action build() - { - String query = String.join(" ", statements); - - if (statements.size() > 1) - query = String.format("BEGIN BATCH\n%s\nAPPLY BATCH;", query); - - Object[] bindingsArray = new Object[bindings.size()]; - bindings.toArray(bindingsArray); - - action = new SimulatedActionCallable<Object[][]>("Batch", - Action.Modifiers.RELIABLE_NO_TIMEOUTS, - Action.Modifiers.RELIABLE_NO_TIMEOUTS, - simulation.simulated, - simulation.cluster.get((int) ((lts.get(0) % simulation.cluster.size()) + 1)), - new RetryingQuery(query, cl, bindingsArray)) - { - private final List<Long> localLts = new ArrayList<>(SimulatedVisitExectuor.this.lts); - - @Override - protected InterceptedExecution.InterceptedTaskExecution task() - { - return new InterceptedExecution.InterceptedTaskExecution((InterceptingExecutor) on.executor()) - { - public void run() - { - for (Long l : localLts) - tracker.beginModification(l); - - // we'll be invoked on the node's executor, but we need to ensure the task is loaded on its classloader - try { accept(on.unsafeCallOnThisThread(execute), null); } - catch (Throwable t) { accept(null, t); } - finally { execute = null; } - } - }; - } - - @Override - public void accept(Object[][] result, Throwable failure) - { - if (failure != null) - simulated.failures.accept(failure); - else - for (Long l : localLts) - tracker.endModification(l); - } - }; - - statements.clear(); - bindings.clear(); - lts.clear(); - - Action current = action; - action = null; - return current; - } - - @Override - protected void beforeLts(long lts, long pd) - { - this.lts.add(lts); - } - - @Override - protected void afterLts(long lts, long pd) - { - } - - @Override - protected void operation(Operation operation) - { - CompiledStatement statement = rowVisitor.perform(operation); - statements.add(statement.cql()); - Collections.addAll(bindings, statement.bindings()); - } - - public void shutdown() throws InterruptedException - { - } - - private static class RetryingQuery extends Query - { - public RetryingQuery(String query, ConsistencyLevel cl, Object[] boundValues) - { - super(query, -1, cl, null, boundValues); - } - - @Override - public Object[][] call() - { - while (true) - { - try - { - return super.call(); - } - catch (UncheckedInterruptedException e) - { - throw new RuntimeException(e); - } - catch (Throwable t) - { - logger.error("Caught error while executing query. Will ignore and retry: " + t.getMessage()); - } - } - } - } -} \ No newline at end of file diff --git a/test/unit/accord/utils/Invariants.java b/test/unit/accord/utils/Invariants.java index b960d8b268a3..2977272d4aa7 100644 --- a/test/unit/accord/utils/Invariants.java +++ b/test/unit/accord/utils/Invariants.java @@ -22,6 +22,7 @@ import javax.annotation.Nullable; import java.util.function.Predicate; +import java.util.function.Supplier; import static java.lang.String.format; @@ -39,9 +40,14 @@ public static boolean debug() return DEBUG; } - private static void illegalState(String msg) + public static IllegalStateException createIllegalState(String msg) { - throw new IllegalStateException(msg); + return new IllegalStateException(msg); + } + + public static IllegalStateException illegalState(String msg) + { + throw createIllegalState(msg); } private static void illegalState() @@ -91,6 +97,12 @@ public static void checkState(boolean condition) illegalState(); } + public static void checkState(boolean condition, Supplier<String> msg) + { + if (!condition) + throw illegalState(msg.get()); + } + public static void checkState(boolean condition, String msg) { if (!condition) diff --git a/test/unit/accord/utils/Property.java b/test/unit/accord/utils/Property.java index e45642c1f133..e6d0d1fcc9f6 100644 --- a/test/unit/accord/utils/Property.java +++ b/test/unit/accord/utils/Property.java @@ -19,6 +19,7 @@ package accord.utils; import accord.utils.async.TimeoutUtils; +import org.agrona.collections.LongArrayList; import java.time.Duration; import java.util.ArrayList; @@ -32,6 +33,7 @@ import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; +import java.util.function.BiFunction; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Predicate; @@ -212,9 +214,21 @@ private static String statefulPropertyError(StatefulBuilder input, Throwable cau String stateStr = state == null ? null : state.toString().replace("\n", "\n\t\t"); sb.append("\tState: ").append(stateStr).append(": ").append(state == null ? "unknown type" : state.getClass().getCanonicalName()).append('\n'); sb.append("\tHistory:").append('\n'); + addList(sb, "\t\t", history); + return sb.toString(); + } + + private static void addList(StringBuilder sb, String prefix, List<String> list) + { int idx = 0; - for (var event : history) - sb.append("\t\t").append(++idx).append(": ").append(event).append('\n'); + for (var event : list) + sb.append(prefix).append(++idx).append(": ").append(event).append('\n'); + } + + public static String formatList(String prefix, List<String> list) + { + StringBuilder sb = new StringBuilder(); + addList(sb, prefix, list); return sb.toString(); } @@ -432,6 +446,7 @@ public <State, SystemUnderTest> void check(Commands<State, SystemUnderTest> comm { State state = null; List<String> history = new ArrayList<>(steps); + LongArrayList historyTiming = stepTimeout == null ? null : new LongArrayList(); try { checkInterrupted(); @@ -456,17 +471,18 @@ public <State, SystemUnderTest> void check(Commands<State, SystemUnderTest> comm for (Command<State, SystemUnderTest, ?> sub : ((MultistepCommand<State, SystemUnderTest>) cmd)) { history.add(sub.detailed(state)); - process(sub, state, sut, history.size()); + process(sub, state, sut, history.size(), historyTiming); } } else { history.add(cmd.detailed(state)); - process(cmd, state, sut, history.size()); + process(cmd, state, sut, history.size(), historyTiming); } } commands.destroySut(sut, null); commands.destroyState(state, null); + commands.onSuccess(state, sut, maybeRewriteHistory(history, historyTiming)); } catch (Throwable t) { @@ -484,7 +500,8 @@ public <State, SystemUnderTest> void check(Commands<State, SystemUnderTest> comm } catch (Throwable t) { - throw new PropertyError(statefulPropertyError(this, t, state, history), t); + + throw new PropertyError(statefulPropertyError(this, t, state, maybeRewriteHistory(history, historyTiming)), t); } if (pure) { @@ -494,14 +511,35 @@ public <State, SystemUnderTest> void check(Commands<State, SystemUnderTest> comm } } - private <State, SystemUnderTest> void process(Command cmd, State state, SystemUnderTest sut, int id) throws Throwable + private static List<String> maybeRewriteHistory(List<String> history, @Nullable LongArrayList historyTiming) + { + if (historyTiming == null) return history; + List<String> newHistory = new ArrayList<>(history.size()); + for (int i = 0; i < history.size(); i++) + { + String step = history.get(i); + long timeNanos = historyTiming.getLong(i); + newHistory.add(step + ";\tDuration " + Duration.ofNanos(timeNanos)); + } + return newHistory; + } + + private <State, SystemUnderTest> void process(Command cmd, State state, SystemUnderTest sut, int id, @Nullable LongArrayList stepTiming) throws Throwable { if (stepTimeout == null) { cmd.process(state, sut); return; } - TimeoutUtils.runBlocking(stepTimeout, "Stateful Step " + id, () -> cmd.process(state, sut)); + long startNanos = System.nanoTime(); + try + { + TimeoutUtils.runBlocking(stepTimeout, "Stateful Step " + id + ": " + cmd.detailed(state), () -> cmd.process(state, sut)); + } + finally + { + stepTiming.add(System.nanoTime() - startNanos); + } } } @@ -517,7 +555,59 @@ default void checkPostconditions(State state, Result expected, default void process(State state, SystemUnderTest sut) throws Throwable { checkPostconditions(state, apply(state), - sut, run(sut)); + sut, run(sut)); + } + } + + public static class ForwardingCommand<State, SystemUnderTest, Result> implements Command<State, SystemUnderTest, Result> + { + private final Command<State, SystemUnderTest, Result> delegate; + + public ForwardingCommand(Command<State, SystemUnderTest, Result> delegate) + { + this.delegate = delegate; + } + + protected Command<State, SystemUnderTest, Result> delegate() + { + return delegate; + } + + @Override + public PreCheckResult checkPreconditions(State state) + { + return delegate().checkPreconditions(state); + } + + @Override + public Result apply(State state) throws Throwable + { + return delegate().apply(state); + } + + @Override + public Result run(SystemUnderTest sut) throws Throwable + { + return delegate().run(sut); + } + + @Override + public void checkPostconditions(State state, Result expected, SystemUnderTest sut, Result actual) throws Throwable + { + delegate().checkPostconditions(state, expected, sut, actual); + } + + @Override + public String detailed(State state) + { + return delegate().detailed(state); + } + + @Override + public void process(State state, SystemUnderTest sut) throws Throwable + { + // don't call delegate here else the process function calls the delegate and not this class + Command.super.process(state, sut); } } @@ -682,6 +772,7 @@ public interface Commands<State, SystemUnderTest> { Gen<State> genInitialState() throws Throwable; SystemUnderTest createSut(State state) throws Throwable; + default void onSuccess(State state, SystemUnderTest sut, List<String> history) throws Throwable {} default void destroyState(State state, @Nullable Throwable cause) throws Throwable {} default void destroySut(SystemUnderTest sut, @Nullable Throwable cause) throws Throwable {} Gen<Command<State, SystemUnderTest, ?>> commands(State state) throws Throwable; @@ -697,6 +788,11 @@ public static <State> CommandsBuilder<State, Void> commands(Supplier<Gen<State>> return new CommandsBuilder<>(stateGen, ignore -> null); } + public interface StatefulSuccess<State, SystemUnderTest> + { + void apply(State state, SystemUnderTest sut, List<String> history) throws Throwable; + } + public static class CommandsBuilder<State, SystemUnderTest> { public interface Setup<State, SystemUnderTest> @@ -708,6 +804,8 @@ public interface Setup<State, SystemUnderTest> private final Map<Setup<State, SystemUnderTest>, Integer> knownWeights = new LinkedHashMap<>(); @Nullable private Set<Setup<State, SystemUnderTest>> unknownWeights = null; + @Nullable + private Map<Predicate<State>, List<Setup<State, SystemUnderTest>>> conditionalCommands = null; private Gen.IntGen unknownWeightGen = Gens.ints().between(1, 10); @Nullable private FailingConsumer<State> preCommands = null; @@ -715,6 +813,9 @@ public interface Setup<State, SystemUnderTest> private FailingBiConsumer<State, Throwable> destroyState = null; @Nullable private FailingBiConsumer<SystemUnderTest, Throwable> destroySut = null; + @Nullable + private BiFunction<State, Gen<Command<State, SystemUnderTest, ?>>, Gen<Command<State, SystemUnderTest, ?>>> commandsTransformer = null; + private final List<StatefulSuccess<State, SystemUnderTest>> onSuccess = new ArrayList<>(); public CommandsBuilder(Supplier<Gen<State>> stateGen, Function<State, SystemUnderTest> sutFactory) { @@ -792,18 +893,15 @@ public CommandsBuilder<State, SystemUnderTest> add(Setup<State, SystemUnderTest> public CommandsBuilder<State, SystemUnderTest> addIf(Predicate<State> predicate, Gen<Command<State, SystemUnderTest, ?>> cmd) { - return add((rs, state) -> { - if (!predicate.test(state)) return ignoreCommand(); - return cmd.next(rs); - }); + return addIf(predicate, (rs, state) -> cmd.next(rs)); } public CommandsBuilder<State, SystemUnderTest> addIf(Predicate<State> predicate, Setup<State, SystemUnderTest> cmd) { - return add((rs, state) -> { - if (!predicate.test(state)) return ignoreCommand(); - return cmd.setup(rs, state); - }); + if (conditionalCommands == null) + conditionalCommands = new LinkedHashMap<>(); + conditionalCommands.computeIfAbsent(predicate, i -> new ArrayList<>()).add(cmd); + return this; } public CommandsBuilder<State, SystemUnderTest> addAllIf(Predicate<State> predicate, Consumer<IfBuilder<State, SystemUnderTest>> sub) @@ -816,6 +914,12 @@ public IfBuilder<State, SystemUnderTest> add(Setup<State, SystemUnderTest> cmd) CommandsBuilder.this.addIf(predicate, cmd); return this; } + + @Override + public IfBuilder<State, SystemUnderTest> addIf(Predicate<State> nextPredicate, Setup<State, SystemUnderTest> cmd) { + CommandsBuilder.this.addIf(predicate.and(nextPredicate), cmd); + return this; + } }); return this; } @@ -823,6 +927,7 @@ public IfBuilder<State, SystemUnderTest> add(Setup<State, SystemUnderTest> cmd) public interface IfBuilder<State, SystemUnderTest> { IfBuilder<State, SystemUnderTest> add(Setup<State, SystemUnderTest> cmd); + IfBuilder<State, SystemUnderTest> addIf(Predicate<State> predicate, Setup<State, SystemUnderTest> cmd); } public CommandsBuilder<State, SystemUnderTest> unknownWeight(Gen.IntGen unknownWeightGen) @@ -831,10 +936,22 @@ public CommandsBuilder<State, SystemUnderTest> unknownWeight(Gen.IntGen unknownW return this; } + public CommandsBuilder<State, SystemUnderTest> commandsTransformer(BiFunction<State, Gen<Command<State, SystemUnderTest, ?>>, Gen<Command<State, SystemUnderTest, ?>>> commandsTransformer) + { + this.commandsTransformer = commandsTransformer; + return this; + } + + public CommandsBuilder<State, SystemUnderTest> onSuccess(StatefulSuccess<State, SystemUnderTest> fn) + { + onSuccess.add(fn); + return this; + } + public Commands<State, SystemUnderTest> build() { Gen<Setup<State, SystemUnderTest>> commandsGen; - if (unknownWeights == null) + if (unknownWeights == null && conditionalCommands == null) { commandsGen = Gens.pick(new LinkedHashMap<>(knownWeights)); } @@ -842,25 +959,52 @@ public Commands<State, SystemUnderTest> build() { class DynamicWeightsGen implements Gen<Setup<State, SystemUnderTest>>, Gens.Reset { - Gen<Setup<State, SystemUnderTest>> gen; + LinkedHashMap<Setup<State, SystemUnderTest>, Integer> weights; + LinkedHashMap<Setup<State, SystemUnderTest>, Integer> conditionalWeights; + Gen<Setup<State, SystemUnderTest>> nonConditional; @Override public Setup<State, SystemUnderTest> next(RandomSource rs) { - if (gen == null) + if (weights == null) { // create random weights - LinkedHashMap<Setup<State, SystemUnderTest>, Integer> clone = new LinkedHashMap<>(knownWeights); - for (Setup<State, SystemUnderTest> s : unknownWeights) - clone.put(s, unknownWeightGen.nextInt(rs)); - gen = Gens.pick(clone); + weights = new LinkedHashMap<>(knownWeights); + if (unknownWeights != null) + { + for (Setup<State, SystemUnderTest> s : unknownWeights) + weights.put(s, unknownWeightGen.nextInt(rs)); + } + nonConditional = Gens.pick(weights); + if (conditionalCommands != null) + { + conditionalWeights = new LinkedHashMap<>(); + for (List<Setup<State, SystemUnderTest>> commands : conditionalCommands.values()) + { + for (Setup<State, SystemUnderTest> c : commands) + conditionalWeights.put(c, unknownWeightGen.nextInt(rs)); + } + } } - return gen.next(rs); + if (conditionalWeights == null) return nonConditional.next(rs); + return (r, s) -> { + // need to figure out what conditions apply... + LinkedHashMap<Setup<State, SystemUnderTest>, Integer> clone = new LinkedHashMap<>(weights); + for (Map.Entry<Predicate<State>, List<Setup<State, SystemUnderTest>>> e : conditionalCommands.entrySet()) + { + if (e.getKey().test(s)) + e.getValue().forEach(c -> clone.put(c, conditionalWeights.get(c))); + } + Setup<State, SystemUnderTest> select = Gens.pick(clone).next(r); + return select.setup(r, s); + }; } @Override public void reset() { - gen = null; + weights = null; + nonConditional = null; + conditionalWeights = null; } } commandsGen = new DynamicWeightsGen(); @@ -884,7 +1028,8 @@ public SystemUnderTest createSut(State state) throws Throwable { if (preCommands != null) preCommands.accept(state); - return commandsGen.map((rs, setup) -> setup.setup(rs, state)); + Gen<Command<State, SystemUnderTest, ?>> map = commandsGen.map((rs, setup) -> setup.setup(rs, state)); + return commandsTransformer == null ? map : commandsTransformer.apply(state, map); } @Override @@ -901,6 +1046,13 @@ public void destroySut(SystemUnderTest sut, @Nullable Throwable cause) throws Th if (destroySut != null) destroySut.accept(sut, cause); } + + @Override + public void onSuccess(State state, SystemUnderTest sut, List<String> history) throws Throwable + { + for (var fn : onSuccess) + fn.apply(state, sut, history); + } }; } diff --git a/test/unit/org/apache/cassandra/tcm/ClusterMetadataTest.java b/test/unit/org/apache/cassandra/tcm/ClusterMetadataTest.java index c17067521bfd..e6d7243a469d 100644 --- a/test/unit/org/apache/cassandra/tcm/ClusterMetadataTest.java +++ b/test/unit/org/apache/cassandra/tcm/ClusterMetadataTest.java @@ -27,7 +27,7 @@ import org.apache.cassandra.ServerTestUtils; import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; import org.apache.cassandra.distributed.test.log.CMSTestBase; -import org.apache.cassandra.harry.sut.TokenPlacementModel; +import org.apache.cassandra.harry.model.TokenPlacementModel; import org.apache.cassandra.tcm.sequences.BootstrapAndJoin; import org.apache.cassandra.tcm.sequences.UnbootstrapAndLeave; import org.apache.cassandra.schema.KeyspaceMetadata; @@ -37,7 +37,6 @@ import static org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper.getLeavePlan; import static org.junit.Assert.assertTrue; - public class ClusterMetadataTest { @BeforeClass diff --git a/test/unit/org/apache/cassandra/tcm/ownership/UniformRangePlacementIntegrationTest.java b/test/unit/org/apache/cassandra/tcm/ownership/UniformRangePlacementIntegrationTest.java index 322acdc4ed20..00c9d27441a2 100644 --- a/test/unit/org/apache/cassandra/tcm/ownership/UniformRangePlacementIntegrationTest.java +++ b/test/unit/org/apache/cassandra/tcm/ownership/UniformRangePlacementIntegrationTest.java @@ -32,7 +32,7 @@ import org.apache.cassandra.distributed.test.log.CMSTestBase; import org.apache.cassandra.distributed.test.log.ClusterMetadataTestHelper; import org.apache.cassandra.distributed.test.log.MetadataChangeSimulationTest; -import org.apache.cassandra.harry.sut.TokenPlacementModel; +import org.apache.cassandra.harry.model.TokenPlacementModel; import org.apache.cassandra.schema.Keyspaces; import org.apache.cassandra.schema.ReplicationParams; import org.apache.cassandra.tcm.AtomicLongBackedProcessor; diff --git a/test/unit/org/apache/cassandra/tcm/sequences/ProgressBarrierTest.java b/test/unit/org/apache/cassandra/tcm/sequences/ProgressBarrierTest.java index 91bd26d479c7..fe73677e2575 100644 --- a/test/unit/org/apache/cassandra/tcm/sequences/ProgressBarrierTest.java +++ b/test/unit/org/apache/cassandra/tcm/sequences/ProgressBarrierTest.java @@ -43,7 +43,7 @@ import org.apache.cassandra.harry.gen.rng.PCGFastPure; import org.apache.cassandra.harry.gen.rng.PcgRSUFast; import org.apache.cassandra.harry.gen.rng.RngUtils; -import org.apache.cassandra.harry.sut.TokenPlacementModel; +import org.apache.cassandra.harry.model.TokenPlacementModel; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.metrics.TCMMetrics; import org.apache.cassandra.net.ConnectionType; From 6e8d0cde9e585569a64097beb26eedbf7373b314 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Wed, 15 Jan 2025 15:45:35 +0100 Subject: [PATCH 086/225] Make JsonUtils serialize Instant always with the same format patch by Stefan Miklosovic; reviewed by Maxim Muzafarov for CASSANDRA-20209 --- CHANGES.txt | 1 + .../org/apache/cassandra/utils/JsonUtils.java | 19 ++++++++-- .../cassandra/db/ColumnFamilyStoreTest.java | 5 ++- .../apache/cassandra/utils/JsonUtilsTest.java | 36 +++++++++++++++++++ 4 files changed, 55 insertions(+), 6 deletions(-) create mode 100644 test/unit/org/apache/cassandra/utils/JsonUtilsTest.java diff --git a/CHANGES.txt b/CHANGES.txt index b4a5206aa326..23a694b0bd0d 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Make JsonUtils serialize Instant always with the same format (CASSANDRA-20209) * Port Harry v2 to trunk (CASSANDRA-20200) * Enable filtering of snapshots on keyspace, table and snapshot name in nodetool listsnapshots (CASSANDRA-20151) * Create manifest upon loading where it does not exist or enrich it (CASSANDRA-20150) diff --git a/src/java/org/apache/cassandra/utils/JsonUtils.java b/src/java/org/apache/cassandra/utils/JsonUtils.java index aa0963bf5859..4868fedd4598 100644 --- a/src/java/org/apache/cassandra/utils/JsonUtils.java +++ b/src/java/org/apache/cassandra/utils/JsonUtils.java @@ -21,6 +21,9 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.time.Instant; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -29,8 +32,8 @@ import com.fasterxml.jackson.core.util.BufferRecyclers; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectWriter; -import com.fasterxml.jackson.databind.SerializationFeature; import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +import com.fasterxml.jackson.datatype.jsr310.ser.InstantSerializer; import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.FileInputStreamPlus; import org.apache.cassandra.io.util.FileOutputStreamPlus; @@ -44,10 +47,20 @@ public final class JsonUtils public static final ObjectMapper JSON_OBJECT_MAPPER = new ObjectMapper(new JsonFactory()); // checkstyle: permit this instantiation public static final ObjectWriter JSON_OBJECT_PRETTY_WRITER; + private static class GlobalInstantSerializer extends InstantSerializer + { + private GlobalInstantSerializer() + { + super(InstantSerializer.INSTANCE, + false, + false, + DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'").withZone(ZoneOffset.UTC)); + } + } + static { - JSON_OBJECT_MAPPER.registerModule(new JavaTimeModule()); - JSON_OBJECT_MAPPER.disable(SerializationFeature.WRITE_DATES_AS_TIMESTAMPS); + JSON_OBJECT_MAPPER.registerModule(new JavaTimeModule().addSerializer(Instant.class, new GlobalInstantSerializer())); JSON_OBJECT_PRETTY_WRITER = JSON_OBJECT_MAPPER.writerWithDefaultPrettyPrinter(); } diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java index 7f5b28c7596d..0febbe660cfa 100644 --- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java +++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java @@ -449,9 +449,8 @@ private void testTrueSnapshotSizeInternal(String keyspace, String table, boolean long anotherWithIndexSize = listedSnapshots.get("snapshot2_with_index").computeSizeOnDiskBytes(); long anotherWithIndexTrueSize = listedSnapshots.get("snapshot2_with_index").computeTrueSizeBytes(); - // TODO CASSANDRA-20209 - assertTrue(withIndexSize == anotherWithIndexSize || (withIndexSize + 4 == anotherWithIndexSize) || (withIndexSize - 4 == anotherWithIndexSize)); - assertTrue(withIndexTrueSize == anotherWithIndexTrueSize || (withIndexTrueSize + 4 == anotherWithIndexTrueSize) || (withIndexTrueSize - 4 == anotherWithIndexTrueSize)); + assertEquals(withIndexSize, anotherWithIndexSize); + assertEquals(withIndexTrueSize, anotherWithIndexTrueSize); } private void rebuildIndices(ColumnFamilyStore cfs) diff --git a/test/unit/org/apache/cassandra/utils/JsonUtilsTest.java b/test/unit/org/apache/cassandra/utils/JsonUtilsTest.java new file mode 100644 index 000000000000..1bd32079f8f4 --- /dev/null +++ b/test/unit/org/apache/cassandra/utils/JsonUtilsTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils; + +import org.junit.Test; + +import static java.time.Instant.parse; +import static org.apache.cassandra.utils.JsonUtils.writeAsJsonString; +import static org.junit.Assert.assertEquals; + +public class JsonUtilsTest +{ + @Test + public void testTimestampSerialisation() + { + assertEquals("\"2025-01-15T13:26:45.040Z\"", writeAsJsonString(parse("2025-01-15T13:26:45.04Z"))); + assertEquals("\"2025-01-15T13:26:45.100Z\"", writeAsJsonString(parse("2025-01-15T13:26:45.1Z"))); + assertEquals("\"2025-01-15T13:26:45.123Z\"", writeAsJsonString(parse("2025-01-15T13:26:45.123Z"))); + } +} From f33267c8fae6d71166886efc3e7ddda4114ebeef Mon Sep 17 00:00:00 2001 From: Sam Tunnicliffe <samt@apache.org> Date: Wed, 20 Nov 2024 18:54:23 +0000 Subject: [PATCH 087/225] Tighten up permissions on system keyspaces * Restrict which permissions can be granted on system keyspaces * Ensure that GRANT... ON ALL KEYSPACES excludes system keyspaces * Add system_traces to the always readable set Patch by Sam Tunnicliffe and Francisco Guerrero; reviewed by Sam Tunnicliffe and Francisco Guerrero for CASSANDRA-20090 Co-authored-by: Francisco Guerrero <frankgh@apache.org> --- CHANGES.txt | 3 +- .../org/apache/cassandra/auth/Permission.java | 7 + .../org/apache/cassandra/auth/Resources.java | 28 +- .../cassandra/config/DatabaseDescriptor.java | 15 + .../org/apache/cassandra/config/Schema.java | 6 + .../statements/GrantPermissionsStatement.java | 21 + .../apache/cassandra/service/ClientState.java | 80 +++- .../cassandra/auth/GrantAndRevokeTest.java | 375 ++++++++++++++++++ .../org/apache/cassandra/cql3/CQLTester.java | 27 +- 9 files changed, 531 insertions(+), 31 deletions(-) create mode 100644 test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 9804496fcb6c..d9a3ecbe4d28 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,5 +1,6 @@ 3.0.31 - * Fix incorrect column identifier bytes problem when renaming a column (CASSANDRA-18965) + * Tighten up permissions on system keyspaces (CASSANDRA-20090) + * Fix incorrect column identifier bytes problem when renaming a column (CASSANDRA-18956) * Upgrade OWASP to 10.0.0 (CASSANDRA-19738) Merged from 2.2: * Add termin-8-jdk as a valid jdk8 candidate in the debian package (CASSANDRA-19752) diff --git a/src/java/org/apache/cassandra/auth/Permission.java b/src/java/org/apache/cassandra/auth/Permission.java index d552280e64bd..11c7aeb05b70 100644 --- a/src/java/org/apache/cassandra/auth/Permission.java +++ b/src/java/org/apache/cassandra/auth/Permission.java @@ -66,4 +66,11 @@ public enum Permission public static final Set<Permission> ALL = Sets.immutableEnumSet(EnumSet.range(Permission.CREATE, Permission.EXECUTE)); public static final Set<Permission> NONE = ImmutableSet.of(); + + /** + * Set of Permissions which may never be granted on any system keyspace, or table in a system keyspace, to any role. + * (Only SELECT, DESCRIBE and ALTER may ever be granted). + */ + public static final Set<Permission> INVALID_FOR_SYSTEM_KEYSPACES = + Sets.immutableEnumSet(EnumSet.complementOf(EnumSet.of(Permission.SELECT, Permission.DESCRIBE, Permission.ALTER))); } diff --git a/src/java/org/apache/cassandra/auth/Resources.java b/src/java/org/apache/cassandra/auth/Resources.java index ebcfc1644347..c6f784a185b3 100644 --- a/src/java/org/apache/cassandra/auth/Resources.java +++ b/src/java/org/apache/cassandra/auth/Resources.java @@ -19,6 +19,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.function.Predicate; import org.apache.cassandra.utils.Hex; @@ -27,18 +28,33 @@ public final class Resources /** * Construct a chain of resource parents starting with the resource and ending with the root. * - * @param resource The staring point. + * @param resource The starting point. * @return list of resource in the chain form start to the root. */ public static List<? extends IResource> chain(IResource resource) { - List<IResource> chain = new ArrayList<IResource>(); + return chain(resource, (r) -> true); + } + + /** + * Construct a chain of resource parents starting with the resource and ending with the root. Only resources which + * satisfy the supplied predicate will be included. + * + * @param resource The starting point. + * @param filter can be used to omit specific resources from the chain + * @return list of resource in the chain form start to the root. + */ + public static List<? extends IResource> chain(IResource resource, Predicate<IResource> filter) + { + + List<IResource> chain = new ArrayList<>(4); while (true) { - chain.add(resource); - if (!resource.hasParent()) - break; - resource = resource.getParent(); + if (filter.test(resource)) + chain.add(resource); + if (!resource.hasParent()) + break; + resource = resource.getParent(); } return chain; } diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index ca946ce355d4..2ec1944a458b 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -862,16 +862,31 @@ public static IAuthenticator getAuthenticator() return authenticator; } + public static void setAuthenticator(IAuthenticator authenticator) + { + DatabaseDescriptor.authenticator = authenticator; + } + public static IAuthorizer getAuthorizer() { return authorizer; } + public static void setAuthorizer(IAuthorizer authorizer) + { + DatabaseDescriptor.authorizer = authorizer; + } + public static IRoleManager getRoleManager() { return roleManager; } + public static void setRoleManager(IRoleManager roleManager) + { + DatabaseDescriptor.roleManager = roleManager; + } + public static int getPermissionsValidity() { return conf.permissions_validity_in_ms; diff --git a/src/java/org/apache/cassandra/config/Schema.java b/src/java/org/apache/cassandra/config/Schema.java index c733c8d680ac..de308e2d3cff 100644 --- a/src/java/org/apache/cassandra/config/Schema.java +++ b/src/java/org/apache/cassandra/config/Schema.java @@ -125,6 +125,12 @@ public static boolean isReplicatedSystemKeyspace(String keyspaceName) return REPLICATED_SYSTEM_KEYSPACE_NAMES.contains(keyspaceName.toLowerCase()); } + public static boolean isSystemKeyspace(String keyspaceName) + { + final String lowercaseKeyspaceName = keyspaceName.toLowerCase(); + return LOCAL_SYSTEM_KEYSPACE_NAMES.contains(lowercaseKeyspaceName) + || REPLICATED_SYSTEM_KEYSPACE_NAMES.contains(lowercaseKeyspaceName); + } /** * load keyspace (keyspace) definitions, but do not initialize the keyspace instances. * Schema version may be updated as the result. diff --git a/src/java/org/apache/cassandra/cql3/statements/GrantPermissionsStatement.java b/src/java/org/apache/cassandra/cql3/statements/GrantPermissionsStatement.java index 06a53e2c2e17..d08e541c584c 100644 --- a/src/java/org/apache/cassandra/cql3/statements/GrantPermissionsStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/GrantPermissionsStatement.java @@ -17,14 +17,18 @@ */ package org.apache.cassandra.cql3.statements; +import java.util.Collections; import java.util.Set; +import org.apache.cassandra.auth.DataResource; import org.apache.cassandra.auth.IResource; import org.apache.cassandra.auth.Permission; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.Schema; import org.apache.cassandra.cql3.RoleName; import org.apache.cassandra.exceptions.RequestExecutionException; import org.apache.cassandra.exceptions.RequestValidationException; +import org.apache.cassandra.exceptions.UnauthorizedException; import org.apache.cassandra.service.ClientState; import org.apache.cassandra.transport.messages.ResultMessage; @@ -35,6 +39,23 @@ public GrantPermissionsStatement(Set<Permission> permissions, IResource resource super(permissions, resource, grantee); } + public void validate(ClientState state) throws RequestValidationException + { + super.validate(state); + if (resource instanceof DataResource) + { + DataResource data = (DataResource) resource; + // Only a subset of permissions can be granted on system keyspaces + if (!data.isRootLevel() + && Schema.isSystemKeyspace(data.getKeyspace()) + && !Collections.disjoint(permissions, Permission.INVALID_FOR_SYSTEM_KEYSPACES)) + { + throw new UnauthorizedException("Granting permissions on system keyspaces is strictly limited, " + + "this operation is not permitted"); + } + } + } + public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException { DatabaseDescriptor.getAuthorizer().grant(state.getUser(), permissions, resource, grantee); diff --git a/src/java/org/apache/cassandra/service/ClientState.java b/src/java/org/apache/cassandra/service/ClientState.java index 1cbbfd484b30..8ab6887c5be2 100644 --- a/src/java/org/apache/cassandra/service/ClientState.java +++ b/src/java/org/apache/cassandra/service/ClientState.java @@ -20,10 +20,11 @@ import java.net.InetSocketAddress; import java.net.SocketAddress; import java.util.Arrays; -import java.util.HashSet; +import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; +import com.google.common.collect.ImmutableSet; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,6 +42,7 @@ import org.apache.cassandra.exceptions.UnauthorizedException; import org.apache.cassandra.schema.SchemaKeyspace; import org.apache.cassandra.thrift.ThriftValidation; +import org.apache.cassandra.tracing.TraceKeyspace; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.CassandraVersion; @@ -54,29 +56,40 @@ public class ClientState private static final Logger logger = LoggerFactory.getLogger(ClientState.class); public static final CassandraVersion DEFAULT_CQL_VERSION = org.apache.cassandra.cql3.QueryProcessor.CQL_VERSION; - private static final Set<IResource> READABLE_SYSTEM_RESOURCES = new HashSet<>(); - private static final Set<IResource> PROTECTED_AUTH_RESOURCES = new HashSet<>(); - private static final Set<IResource> DROPPABLE_SYSTEM_AUTH_TABLES = new HashSet<>(); + public static final ImmutableSet<IResource> READABLE_SYSTEM_RESOURCES; + public static final ImmutableSet<IResource> PROTECTED_AUTH_RESOURCES; + public static final ImmutableSet<IResource> DROPPABLE_SYSTEM_AUTH_TABLES; static { // We want these system cfs to be always readable to authenticated users since many tools rely on them // (nodetool, cqlsh, bulkloader, etc.) - for (String cf : Arrays.asList(SystemKeyspace.LOCAL, SystemKeyspace.PEERS)) - READABLE_SYSTEM_RESOURCES.add(DataResource.table(SystemKeyspace.NAME, cf)); + ImmutableSet.Builder<IResource> readableBuilder = ImmutableSet.builder(); + for (String cf : Arrays.asList(SystemKeyspace.LOCAL, SystemKeyspace.PEERS, SystemKeyspace.SIZE_ESTIMATES)) + readableBuilder.add(DataResource.table(SystemKeyspace.NAME, cf)); - SchemaKeyspace.ALL.forEach(table -> READABLE_SYSTEM_RESOURCES.add(DataResource.table(SchemaKeyspace.NAME, table))); + // make all schema tables readable by default (required by the drivers) + SchemaKeyspace.ALL.forEach(table -> readableBuilder.add(DataResource.table(SchemaKeyspace.NAME, table))); + // make system_traces readable by all or else tracing will require explicit grants + readableBuilder.add(DataResource.table(TraceKeyspace.NAME, TraceKeyspace.EVENTS)); + readableBuilder.add(DataResource.table(TraceKeyspace.NAME, TraceKeyspace.SESSIONS)); + READABLE_SYSTEM_RESOURCES = readableBuilder.build(); + + ImmutableSet.Builder<IResource> protectedBuilder = ImmutableSet.builder(); + // neither clients nor tools need authentication/authorization if (!Config.isClientMode()) { - PROTECTED_AUTH_RESOURCES.addAll(DatabaseDescriptor.getAuthenticator().protectedResources()); - PROTECTED_AUTH_RESOURCES.addAll(DatabaseDescriptor.getAuthorizer().protectedResources()); - PROTECTED_AUTH_RESOURCES.addAll(DatabaseDescriptor.getRoleManager().protectedResources()); + protectedBuilder.addAll(DatabaseDescriptor.getAuthenticator().protectedResources()); + protectedBuilder.addAll(DatabaseDescriptor.getAuthorizer().protectedResources()); + protectedBuilder.addAll(DatabaseDescriptor.getRoleManager().protectedResources()); } - + PROTECTED_AUTH_RESOURCES = protectedBuilder.build(); + ImmutableSet.Builder<IResource> droppableBuilder = ImmutableSet.builder(); // allow users with sufficient privileges to drop legacy tables (users, credentials, permissions) from AUTH_KS - DROPPABLE_SYSTEM_AUTH_TABLES.add(DataResource.table(AuthKeyspace.NAME, PasswordAuthenticator.LEGACY_CREDENTIALS_TABLE)); - DROPPABLE_SYSTEM_AUTH_TABLES.add(DataResource.table(AuthKeyspace.NAME, CassandraRoleManager.LEGACY_USERS_TABLE)); - DROPPABLE_SYSTEM_AUTH_TABLES.add(DataResource.table(AuthKeyspace.NAME, CassandraAuthorizer.USER_PERMISSIONS)); + droppableBuilder.add(DataResource.table(AuthKeyspace.NAME, PasswordAuthenticator.LEGACY_CREDENTIALS_TABLE)); + droppableBuilder.add(DataResource.table(AuthKeyspace.NAME, CassandraRoleManager.LEGACY_USERS_TABLE)); + droppableBuilder.add(DataResource.table(AuthKeyspace.NAME, CassandraAuthorizer.USER_PERMISSIONS)); + DROPPABLE_SYSTEM_AUTH_TABLES = droppableBuilder.build(); } // Current user for the session @@ -328,9 +341,12 @@ private void hasAccess(String keyspace, Permission perm, DataResource resource) preventSystemKSSchemaModification(keyspace, resource, perm); + // Some system data is always readable if ((perm == Permission.SELECT) && READABLE_SYSTEM_RESOURCES.contains(resource)) return; + // Modifications to any resource upon which the authenticator, authorizer or role manager depend should not be + // be performed by users if (PROTECTED_AUTH_RESOURCES.contains(resource)) if ((perm == Permission.CREATE) || (perm == Permission.ALTER) || (perm == Permission.DROP)) throw new UnauthorizedException(String.format("%s schema is protected", resource)); @@ -348,7 +364,25 @@ public void ensureHasPermission(Permission perm, IResource resource) throws Unau if (((FunctionResource)resource).getKeyspace().equals(SystemKeyspace.NAME)) return; - checkPermissionOnResourceChain(perm, resource); + if (resource instanceof DataResource && !(user.isSuper() || user.isSystem())) + { + DataResource dataResource = (DataResource)resource; + if (!dataResource.isRootLevel()) + { + String keyspace = dataResource.getKeyspace(); + // A user may have permissions granted on ALL KEYSPACES, but this should exclude system keyspaces. Any + // permission on those keyspaces or their tables must be granted to the user either explicitly or + // transitively. The set of grantable permissions for system keyspaces is further limited, + // see the Permission enum for details. + if (Schema.isSystemKeyspace(keyspace)) + { + ensurePermissionOnResourceChain(perm, Resources.chain(dataResource, IResource::hasParent)); + return; + } + } + } + + ensurePermissionOnResourceChain(perm, resource); } // Convenience method called from checkAccess method of CQLStatement @@ -363,14 +397,20 @@ public void ensureHasPermission(Permission permission, Function function) if (function.isNative()) return; - checkPermissionOnResourceChain(permission, FunctionResource.function(function.name().keyspace, - function.name().name, - function.argTypes())); + ensurePermissionOnResourceChain(permission, FunctionResource.function(function.name().keyspace, + function.name().name, + function.argTypes())); + } + + private void ensurePermissionOnResourceChain(Permission perm, IResource resource) + { + ensurePermissionOnResourceChain(perm, Resources.chain(resource)); } - private void checkPermissionOnResourceChain(Permission perm, IResource resource) + private void ensurePermissionOnResourceChain(Permission perm, List<? extends IResource> resources) { - for (IResource r : Resources.chain(resource)) + IResource resource = resources.get(0); + for (IResource r : resources) if (authorize(r).contains(perm)) return; diff --git a/test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java b/test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java new file mode 100644 index 000000000000..80e2db463ff8 --- /dev/null +++ b/test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java @@ -0,0 +1,375 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.auth; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +import com.google.common.collect.Iterables; +import org.junit.After; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.PlainTextAuthProvider; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.exceptions.UnauthorizedException; +import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.Util; +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.Schema; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.db.SystemKeyspace; +import org.apache.cassandra.metrics.ClientMetrics; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.schema.SchemaKeyspace; +import org.apache.cassandra.schema.Types; +import org.apache.cassandra.tracing.TraceKeyspace; +import org.apache.cassandra.transport.ConfiguredLimit; +import org.apache.cassandra.transport.Server; +import org.apache.cassandra.utils.Pair; + +import static java.lang.String.format; +import static org.apache.cassandra.config.Schema.LOCAL_SYSTEM_KEYSPACE_NAMES; +import static org.apache.cassandra.config.Schema.REPLICATED_SYSTEM_KEYSPACE_NAMES; + +public class GrantAndRevokeTest extends CQLTester +{ + private static final String user = "user"; + private static final String pass = "12345"; + private static final Pair<String, String> USER = Pair.create(user, pass); + private static final Pair<String, String> SUPERUSER = Pair.create("cassandra", "cassandra"); + private static int counter = 0; + + @BeforeClass + public static void setUpClass() + { + System.setProperty("cassandra.superuser_setup_delay_ms", "0"); + DatabaseDescriptor.setPermissionsValidity(0); + CQLTester.setUpClass(); + SchemaLoader.createKeyspace(AuthKeyspace.NAME, KeyspaceParams.simple(1), AuthKeyspace.metadata().tables, Types.none()); + + DatabaseDescriptor.setAuthenticator(new PasswordAuthenticator()); + DatabaseDescriptor.setRoleManager(new CassandraRoleManager()); + DatabaseDescriptor.setAuthorizer(new CassandraAuthorizer()); + // needed as the driver reads system.local to get the release version, which determines the + // schema tables it attempts to read (current, or legacy in case it's connecting to a 2.2 node) + SystemKeyspace.persistLocalMetadata(); + prepareNetwork(); + protocolVersionLimit = ConfiguredLimit.newLimit(); + server = new Server.Builder().withHost(nativeAddr) + .withPort(nativePort) + .withProtocolVersionLimit(protocolVersionLimit) + .build(); + ClientMetrics.instance.init(Collections.singleton(server)); + server.start(); + } + + @After + public void tearDown() throws Throwable + { + session(SUPERUSER).execute("DROP ROLE " + user); + } + + private Session session(Pair<String, String> credentials) + { + Cluster cluster = Cluster.builder() + .addContactPoints(nativeAddr) + .withClusterName("Test Cluster " + counter++) + .withPort(nativePort) + .withAuthProvider(new PlainTextAuthProvider(credentials.left, credentials.right)) + .build(); + + return cluster.connect(); + } + + @Test + public void testGrantedKeyspace() throws Throwable + { + Session superuser = session(SUPERUSER); + superuser.execute(String.format("CREATE ROLE %s WITH LOGIN = TRUE AND password='%s'", user, pass)); + superuser.execute("GRANT CREATE ON KEYSPACE " + KEYSPACE_PER_TEST + " TO " + user); + String table = KEYSPACE_PER_TEST + '.' + createTable(KEYSPACE_PER_TEST, "CREATE TABLE %s (pk int, ck int, val int, val_2 text, PRIMARY KEY (pk, ck))"); + String index = KEYSPACE_PER_TEST + ".idx_01"; + createIndex(KEYSPACE_PER_TEST, "CREATE INDEX idx_01 ON %s (val_2)"); + String type = KEYSPACE_PER_TEST + '.' + createType(KEYSPACE_PER_TEST, "CREATE TYPE %s (a int, b text)"); + String mv = KEYSPACE_PER_TEST + ".ks_mv_01"; + superuser.execute("CREATE MATERIALIZED VIEW " + mv + " AS SELECT * FROM " + table + " WHERE val IS NOT NULL AND pk IS NOT NULL AND ck IS NOT NULL PRIMARY KEY (val, pk, ck)"); + + Session nonsuperuser = session(USER); + + // ALTER and DROP tables created by somebody else + // Spin assert for effective auth changes. + final String spinAssertTable = table; + Util.spinAssertEquals(false, () -> { + try + { + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table " + spinAssertTable + "> or any of its parents", + formatQuery(KEYSPACE_PER_TEST, "INSERT INTO %s (pk, ck, val, val_2) VALUES (1, 1, 1, '1')")); + } + catch(Throwable e) + { + return true; + } + return false; + }, 10); + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table " + table + "> or any of its parents", + formatQuery(KEYSPACE_PER_TEST, "UPDATE %s SET val = 1 WHERE pk = 1 AND ck = 1")); + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table " + table + "> or any of its parents", + formatQuery(KEYSPACE_PER_TEST, "DELETE FROM %s WHERE pk = 1 AND ck = 2")); + assertUnauthorizedQuery(nonsuperuser, "User user has no SELECT permission on <table " + table + "> or any of its parents", + formatQuery(KEYSPACE_PER_TEST, "SELECT * FROM %s WHERE pk = 1 AND ck = 1")); + assertUnauthorizedQuery(nonsuperuser, "User user has no SELECT permission on <table " + table + "> or any of its parents", + "SELECT * FROM " + mv + " WHERE val = 1 AND pk = 1 AND ck = 1"); + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table " + table + "> or any of its parents", + formatQuery(KEYSPACE_PER_TEST, "TRUNCATE TABLE %s")); + assertUnauthorizedQuery(nonsuperuser, "User user has no ALTER permission on <table " + table + "> or any of its parents", + formatQuery(KEYSPACE_PER_TEST, "ALTER TABLE %s ADD val_3 int")); + assertUnauthorizedQuery(nonsuperuser, "User user has no DROP permission on <table " + table + "> or any of its parents", + formatQuery(KEYSPACE_PER_TEST, "DROP TABLE %s")); + assertUnauthorizedQuery(nonsuperuser, "User user has no ALTER permission on <keyspace " + KEYSPACE_PER_TEST + "> or any of its parents", + "ALTER TYPE " + type + " ADD c bigint"); + assertUnauthorizedQuery(nonsuperuser, "User user has no DROP permission on <keyspace " + KEYSPACE_PER_TEST + "> or any of its parents", + "DROP TYPE " + type); + assertUnauthorizedQuery(nonsuperuser, "User user has no ALTER permission on <table " + table + "> or any of its parents", + "DROP MATERIALIZED VIEW " + mv); + assertUnauthorizedQuery(nonsuperuser, "User user has no ALTER permission on <table " + table + "> or any of its parents", + "DROP INDEX " + index); + + superuser.execute("GRANT ALTER ON KEYSPACE " + KEYSPACE_PER_TEST + " TO " + user); + superuser.execute("GRANT DROP ON KEYSPACE " + KEYSPACE_PER_TEST + " TO " + user); + superuser.execute("GRANT SELECT ON KEYSPACE " + KEYSPACE_PER_TEST + " TO " + user); + superuser.execute("GRANT MODIFY ON KEYSPACE " + KEYSPACE_PER_TEST + " TO " + user); + // Spin assert for effective auth changes. + Util.spinAssertEquals(false, () -> { + try + { + nonsuperuser.execute("ALTER KEYSPACE " + KEYSPACE_PER_TEST + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}"); + } + catch(Throwable e) + { + return true; + } + return false; + }, 10); + + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "INSERT INTO %s (pk, ck, val, val_2) VALUES (1, 1, 1, '1')")); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "UPDATE %s SET val = 1 WHERE pk = 1 AND ck = 1")); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "DELETE FROM %s WHERE pk = 1 AND ck = 2")); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "SELECT * FROM %s WHERE pk = 1 AND ck = 1")); + nonsuperuser.execute("SELECT * FROM " + mv + " WHERE val = 1 AND pk = 1"); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "TRUNCATE TABLE %s")); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "ALTER TABLE %s ADD val_3 int")); + nonsuperuser.execute("DROP MATERIALIZED VIEW " + mv); + nonsuperuser.execute("DROP INDEX " + index); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "DROP TABLE %s")); + nonsuperuser.execute("ALTER TYPE " + type + " ADD c bigint"); + nonsuperuser.execute("DROP TYPE " + type); + + // calling creatTableName to create a new table name that will be used by the formatQuery + table = createTableName(); + type = KEYSPACE_PER_TEST + "." + createTypeName(); + mv = KEYSPACE_PER_TEST + ".ks_mv_02"; + nonsuperuser.execute("CREATE TYPE " + type + " (a int, b text)"); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "CREATE TABLE %s (pk int, ck int, val int, val_2 text, PRIMARY KEY (pk, ck))")); + nonsuperuser.execute("CREATE MATERIALIZED VIEW " + mv + " AS SELECT * FROM " + table + " WHERE val IS NOT NULL AND pk IS NOT NULL AND ck IS NOT NULL PRIMARY KEY (val, pk, ck)"); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "INSERT INTO %s (pk, ck, val, val_2) VALUES (1, 1, 1, '1')")); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "UPDATE %s SET val = 1 WHERE pk = 1 AND ck = 1")); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "DELETE FROM %s WHERE pk = 1 AND ck = 2")); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "SELECT * FROM %s WHERE pk = 1 AND ck = 1")); + nonsuperuser.execute("SELECT * FROM " + mv + " WHERE val = 1 AND pk = 1"); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "TRUNCATE TABLE %s")); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "ALTER TABLE %s ADD val_3 int")); + nonsuperuser.execute("DROP MATERIALIZED VIEW " + mv); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "DROP TABLE %s")); + nonsuperuser.execute("ALTER TYPE " + type + " ADD c bigint"); + nonsuperuser.execute("DROP TYPE " + type); + + superuser.execute("REVOKE ALTER ON KEYSPACE " + KEYSPACE_PER_TEST + " FROM " + user); + superuser.execute("REVOKE DROP ON KEYSPACE " + KEYSPACE_PER_TEST + " FROM " + user); + superuser.execute("REVOKE SELECT ON KEYSPACE " + KEYSPACE_PER_TEST + " FROM " + user); + superuser.execute("REVOKE MODIFY ON KEYSPACE " + KEYSPACE_PER_TEST + " FROM " + user); + + table = KEYSPACE_PER_TEST + "." + createTable(KEYSPACE_PER_TEST, "CREATE TABLE %s (pk int, ck int, val int, val_2 text, PRIMARY KEY (pk, ck))"); + type = KEYSPACE_PER_TEST + "." + createType(KEYSPACE_PER_TEST, "CREATE TYPE %s (a int, b text)"); + index = KEYSPACE_PER_TEST + ".idx_02"; + createIndex(KEYSPACE_PER_TEST, "CREATE INDEX idx_02 ON %s (val_2)"); + mv = KEYSPACE_PER_TEST + ".ks_mv_03"; + superuser.execute("CREATE MATERIALIZED VIEW " + mv + " AS SELECT * FROM " + table + " WHERE val IS NOT NULL AND pk IS NOT NULL AND ck IS NOT NULL PRIMARY KEY (val, pk, ck)"); + + // Spin assert for effective auth changes. + final String spinAssertTable2 = table; + Util.spinAssertEquals(false, () -> { + try + { + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table " + spinAssertTable2 + "> or any of its parents", + "INSERT INTO " + spinAssertTable2 + " (pk, ck, val, val_2) VALUES (1, 1, 1, '1')"); + } + catch(Throwable e) + { + return true; + } + return false; + }, 10); + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table " + table + "> or any of its parents", + "UPDATE " + table + " SET val = 1 WHERE pk = 1 AND ck = 1"); + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table " + table + "> or any of its parents", + "DELETE FROM " + table + " WHERE pk = 1 AND ck = 2"); + assertUnauthorizedQuery(nonsuperuser, "User user has no SELECT permission on <table " + table + "> or any of its parents", + "SELECT * FROM " + table + " WHERE pk = 1 AND ck = 1"); + assertUnauthorizedQuery(nonsuperuser, "User user has no SELECT permission on <table " + table + "> or any of its parents", + "SELECT * FROM " + mv + " WHERE val = 1 AND pk = 1 AND ck = 1"); + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table " + table + "> or any of its parents", + "TRUNCATE TABLE " + table); + assertUnauthorizedQuery(nonsuperuser, "User user has no ALTER permission on <table " + table + "> or any of its parents", + formatQuery(KEYSPACE_PER_TEST, "ALTER TABLE " + table + " ADD val_3 int")); + assertUnauthorizedQuery(nonsuperuser, "User user has no DROP permission on <table " + table + "> or any of its parents", + formatQuery(KEYSPACE_PER_TEST, "DROP TABLE " + table)); + assertUnauthorizedQuery(nonsuperuser, "User user has no ALTER permission on <keyspace " + KEYSPACE_PER_TEST + "> or any of its parents", + "ALTER TYPE " + type + " ADD c bigint"); + assertUnauthorizedQuery(nonsuperuser, "User user has no DROP permission on <keyspace " + KEYSPACE_PER_TEST + "> or any of its parents", + "DROP TYPE " + type); + assertUnauthorizedQuery(nonsuperuser, "User user has no ALTER permission on <table " + table + "> or any of its parents", + "DROP MATERIALIZED VIEW " + mv); + assertUnauthorizedQuery(nonsuperuser, "User user has no ALTER permission on <table " + table + "> or any of its parents", + "DROP INDEX " + index); + } + + + @Test + public void testSpecificGrantsOnSystemKeyspaces() throws Throwable + { + // Granting specific permissions on system keyspaces should not be allowed if those permissions include any from + // the denylist Permission.INVALID_FOR_SYSTEM_KEYSPACES. By this definition, GRANT ALL on any system keyspace, + // or a table within one, should be rejected. + Session superuser = session(SUPERUSER); + superuser.execute("CREATE ROLE '" + user + "'"); + String responseMsg = "Granting permissions on system keyspaces is strictly limited, this operation is not permitted"; + for (String keyspace : Iterables.concat(LOCAL_SYSTEM_KEYSPACE_NAMES, REPLICATED_SYSTEM_KEYSPACE_NAMES)) + { + assertUnauthorizedQuery(superuser, responseMsg, format("GRANT ALL PERMISSIONS ON KEYSPACE %s TO %s", keyspace, user)); + DataResource keyspaceResource = DataResource.keyspace(keyspace); + for (Permission p : keyspaceResource.applicablePermissions()) + maybeRejectGrant(superuser, p, responseMsg, format("GRANT %s ON KEYSPACE %s TO %s", p.name(), keyspace, user)); + + for (CFMetaData table : Schema.instance.getKSMetaData(keyspace).tables) + { + DataResource tableResource = DataResource.table(keyspace, table.cfName); + assertUnauthorizedQuery(superuser, responseMsg, format("GRANT ALL PERMISSIONS ON %s.\"%s\" TO %s", table.ksName, table.cfName, user)); + for (Permission p : tableResource.applicablePermissions()) + maybeRejectGrant(superuser, p, responseMsg, format("GRANT %s ON %s.\"%s\" TO %s", p.name(), table.ksName, table.cfName, user)); + } + } + } + + @Test + public void testGrantOnAllKeyspaces() throws Throwable + { + // Granting either specific or ALL permissions on ALL KEYSPACES is allowed, however these permissions are + // effective for non-system keyspaces only. If for any reason it is necessary to modify permissions on + // on a system keyspace, it must be done using keyspace specific grant statements. + Session superuser = session(SUPERUSER); + superuser.execute(String.format("CREATE ROLE %s WITH LOGIN = TRUE AND password='%s'", user, pass)); + superuser.execute(String.format("ALTER KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", TraceKeyspace.NAME)); + superuser.execute("CREATE KEYSPACE user_keyspace WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}"); + superuser.execute("CREATE TABLE user_keyspace.t1 (k int PRIMARY KEY)"); + + Session nonsuperuser = session(USER); + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table user_keyspace.t1> or any of its parents", + "INSERT INTO user_keyspace.t1 (k) VALUES (0)"); + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table system.local> or any of its parents", + "INSERT INTO system.local(key) VALUES ('invalid')"); + + superuser.execute(format("GRANT MODIFY ON ALL KEYSPACES TO %s", user)); + // User now has write permission on non-system keyspaces only + nonsuperuser.execute("INSERT INTO user_keyspace.t1 (k) VALUES (0)"); + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table system.local> or any of its parents", + "INSERT INTO system.local(key) VALUES ('invalid')"); + + // A non-superuser only has read access to a pre-defined set of system tables and all system_schema/traces + // tables and granting ALL permissions on ALL keyspaces also does not affect this. + maybeReadSystemTables(nonsuperuser, false); + superuser.execute(format("GRANT ALL PERMISSIONS ON ALL KEYSPACES TO %s", user)); + maybeReadSystemTables(nonsuperuser, false); + + // A superuser can still read system tables + maybeReadSystemTables(superuser, true); + // and also write to them, though this is still strongly discouraged + superuser.execute("INSERT INTO system.peers(peer, data_center) VALUES ('127.0.100.100', 'invalid_dc')"); + + } + + private void maybeReadSystemTables(Session session, boolean isSuper) throws Throwable + { + Set<String> readableKeyspaces = new HashSet<>(Arrays.asList(SchemaKeyspace.NAME, TraceKeyspace.NAME)); + Set<String> readableSystemTables = new HashSet<>(Arrays.asList(SystemKeyspace.LOCAL, + SystemKeyspace.PEERS, + SystemKeyspace.SIZE_ESTIMATES)); + + for (String keyspace : Iterables.concat(LOCAL_SYSTEM_KEYSPACE_NAMES, REPLICATED_SYSTEM_KEYSPACE_NAMES)) + { + for (CFMetaData table : Schema.instance.getKSMetaData(keyspace).tables) + { + if (isSuper || (readableKeyspaces.contains(keyspace) || (keyspace.equals(SystemKeyspace.NAME) && readableSystemTables.contains(table.cfName)))) + { + session.execute(format("SELECT * FROM %s.\"%s\" LIMIT 1", table.ksName, table.cfName)); + } + else + { + assertUnauthorizedQuery(session, format("User %s has no SELECT permission on %s or any of its parents", user, DataResource.table(table.ksName, table.cfName)), + format("SELECT * FROM %s.\"%s\" LIMIT 1", table.ksName, table.cfName)); + } + } + } + } + + private void maybeRejectGrant(Session session, Permission p, String errorResponse, String grant) throws Throwable + { + if (Permission.INVALID_FOR_SYSTEM_KEYSPACES.contains(p)) + assertUnauthorizedQuery(session, errorResponse, grant); + else + session.execute(grant); + } + + private void assertUnauthorizedQuery(Session session, String errorMessage, String query) throws Throwable + { + try + { + session.execute(query); + Assert.fail("Query should be invalid but no error was thrown. Query is: " + query); + } + catch (Exception e) + { + if (!UnauthorizedException.class.isAssignableFrom(e.getClass())) + { + Assert.fail("Query should be invalid but wrong error was thrown. " + + "Expected: " + UnauthorizedException.class.getName() + ", got: " + e.getClass().getName() + ". " + + "Query is: " + query); + } + if (errorMessage != null) + { + assertMessageContains(errorMessage, e); + } + } + } + +} diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index 58240e8b10ef..10c02f512cd2 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -45,6 +45,7 @@ import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.concurrent.ScheduledExecutors; import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.db.ConsistencyLevel; import org.apache.cassandra.metrics.ClientMetrics; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.Schema; @@ -87,7 +88,7 @@ public abstract class CQLTester private static final AtomicInteger seqNumber = new AtomicInteger(); protected static final ByteBuffer TOO_BIG = ByteBuffer.allocate(FBUtilities.MAX_UNSIGNED_SHORT + 1024); - private static org.apache.cassandra.transport.Server server; + protected static org.apache.cassandra.transport.Server server; protected static final int nativePort; protected static final InetAddress nativeAddr; protected static ConfiguredLimit protocolVersionLimit; @@ -567,9 +568,14 @@ protected void disablePreparedReuseForTest() } protected String createType(String query) + { + return createType(KEYSPACE, query); + } + + protected String createType(String keyspace, String query) { String typeName = createTypeName(); - String fullQuery = String.format(query, KEYSPACE + "." + typeName); + String fullQuery = String.format(query, keyspace + "." + typeName); logger.info(fullQuery); schemaChange(fullQuery); return typeName; @@ -706,7 +712,12 @@ protected void dropFormattedTable(String formattedQuery) protected void createIndex(String query) { - createFormattedIndex(formatQuery(query)); + createIndex(KEYSPACE, query); + } + + protected void createIndex(String keyspace, String query) + { + createFormattedIndex(formatQuery(keyspace, query)); } protected void createFormattedIndex(String formattedQuery) @@ -803,10 +814,18 @@ protected CFMetaData currentTableMetadata() return Schema.instance.getCFMetaData(KEYSPACE, currentTable()); } + protected com.datastax.driver.core.ResultSet executeNet(int protocolVersion, ConsistencyLevel consistency, String query) throws Throwable + { + Statement statement = new SimpleStatement(formatQuery(query)); + statement = statement.setConsistencyLevel(com.datastax.driver.core.ConsistencyLevel.valueOf(consistency.name())); + return sessionNet(protocolVersion).execute(statement); + } + protected com.datastax.driver.core.ResultSet executeNet(String query, Object... values) throws Throwable { return sessionNet().execute(formatQuery(query), values); } + protected com.datastax.driver.core.ResultSet executeNet(int protocolVersion, String query, Object... values) throws Throwable { return sessionNet(protocolVersion).execute(formatQuery(query), values); @@ -1312,7 +1331,7 @@ protected void assertInvalidSyntaxMessage(String errorMessage, String query, Obj * @param text the text that the exception message must contains * @param e the exception to check */ - private static void assertMessageContains(String text, Exception e) + protected static void assertMessageContains(String text, Exception e) { Assert.assertTrue("Expected error message to contain '" + text + "', but got '" + e.getMessage() + "'", e.getMessage().contains(text)); From 79630fb42ae9e42691f516590100c279f372ac89 Mon Sep 17 00:00:00 2001 From: Sunil Ramchandra Pawar <pawar_sr@apple.com> Date: Thu, 16 Jan 2025 20:43:25 +0530 Subject: [PATCH 088/225] Add selected SAI index state and query performance metrics to nodetool tablestats patch by Sunil Ramchandra Pawar; reviewed by Caleb Rackliffe, Matt Byrd, and Maxim Muzafarov for CASSANDRA-20026 --- CHANGES.txt | 1 + .../index/sai/metrics/IndexGroupMetrics.java | 3 +- .../index/sai/metrics/TableQueryMetrics.java | 4 +- .../org/apache/cassandra/tools/NodeProbe.java | 71 +++++++++++++++ .../cassandra/tools/nodetool/TableStats.java | 6 +- .../tools/nodetool/stats/StatsTable.java | 10 +++ .../nodetool/stats/StatsTableComparator.java | 46 +++++++++- .../nodetool/stats/TableStatsHolder.java | 59 +++++++++++++ .../nodetool/stats/TableStatsPrinter.java | 14 +++ .../tools/nodetool/TableStatsTest.java | 6 +- .../nodetool/stats/TableStatsPrinterTest.java | 88 ++++++++++++------- .../nodetool/stats/TableStatsTestBase.java | 24 +++++ 12 files changed, 294 insertions(+), 38 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index feae546032da..85c50621d145 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.3 + * Add selected SAI index state and query performance metrics to nodetool tablestats (CASSANDRA-20026) * Remove v30 and v3X from 5.x in-JVM upgrade tests (CASSANDRA-20103) * Avoid memory allocation in offheap_object's NativeCell.valueSize() and NativeClustering.dataSize() (CASSANDRA-20162) * Add flag to avoid invalidating key cache on sstable deletions (CASSANDRA-20068) diff --git a/src/java/org/apache/cassandra/index/sai/metrics/IndexGroupMetrics.java b/src/java/org/apache/cassandra/index/sai/metrics/IndexGroupMetrics.java index f6545760b7aa..fff6291ab062 100644 --- a/src/java/org/apache/cassandra/index/sai/metrics/IndexGroupMetrics.java +++ b/src/java/org/apache/cassandra/index/sai/metrics/IndexGroupMetrics.java @@ -25,9 +25,10 @@ public class IndexGroupMetrics extends AbstractMetrics { + public static final String INDEX_GROUP_METRICS_TYPE = "IndexGroupMetrics"; public IndexGroupMetrics(TableMetadata table, StorageAttachedIndexGroup group) { - super(table.keyspace, table.name, "IndexGroupMetrics"); + super(table.keyspace, table.name, INDEX_GROUP_METRICS_TYPE); Metrics.register(createMetricName("OpenIndexFiles"), (Gauge<Integer>) group::openIndexFiles); Metrics.register(createMetricName("DiskUsedBytes"), (Gauge<Long>) group::diskUsage); diff --git a/src/java/org/apache/cassandra/index/sai/metrics/TableQueryMetrics.java b/src/java/org/apache/cassandra/index/sai/metrics/TableQueryMetrics.java index 987c70ef75fe..bbfbe1d28701 100644 --- a/src/java/org/apache/cassandra/index/sai/metrics/TableQueryMetrics.java +++ b/src/java/org/apache/cassandra/index/sai/metrics/TableQueryMetrics.java @@ -71,6 +71,8 @@ public void release() public class PerQueryMetrics extends AbstractMetrics { + public static final String PER_QUERY_METRICS_TYPE = "PerQuery"; + private final Timer queryLatency; /** @@ -99,7 +101,7 @@ public class PerQueryMetrics extends AbstractMetrics public PerQueryMetrics(TableMetadata table) { - super(table.keyspace, table.name, "PerQuery"); + super(table.keyspace, table.name, PER_QUERY_METRICS_TYPE); queryLatency = Metrics.timer(createMetricName("QueryLatency")); diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index 93cb38128ab0..a917793ecca7 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -90,6 +90,9 @@ import org.apache.cassandra.gms.GossiperMBean; import org.apache.cassandra.hints.HintsService; import org.apache.cassandra.hints.HintsServiceMBean; +import org.apache.cassandra.index.sai.metrics.IndexGroupMetrics; +import org.apache.cassandra.index.sai.metrics.TableQueryMetrics; +import org.apache.cassandra.index.sai.metrics.TableStateMetrics; import org.apache.cassandra.locator.DynamicEndpointSnitchMBean; import org.apache.cassandra.locator.EndpointSnitchInfoMBean; import org.apache.cassandra.metrics.CIDRAuthorizerMetrics; @@ -1867,6 +1870,74 @@ public Object getThreadPoolMetric(String pathName, String poolName, String metri } } + public Object getSaiMetric(String ks, String cf, String metricName) + { + try + { + String scope = getSaiMetricScope(metricName); + String objectNameStr = String.format("org.apache.cassandra.metrics:type=StorageAttachedIndex,keyspace=%s,table=%s,scope=%s,name=%s",ks, cf, scope, metricName); + ObjectName oName = new ObjectName(objectNameStr); + + Set<ObjectName> matchingMBeans = mbeanServerConn.queryNames(oName, null); + if (matchingMBeans.isEmpty()) + return null; + + return getSaiMetricValue(metricName, oName); + } + catch (MalformedObjectNameException e) + { + throw new RuntimeException("Invalid ObjectName format: " + e.getMessage(), e); + } + catch (IOException e) + { + throw new RuntimeException("Error accessing MBean server: " + e.getMessage(), e); + } + } + + private Object getSaiMetricValue(String metricName, ObjectName oName) throws IOException + { + switch (metricName) + { + case "QueryLatency": + return JMX.newMBeanProxy(mbeanServerConn, oName, CassandraMetricsRegistry.JmxTimerMBean.class); + case "PostFilteringReadLatency": + case "SSTableIndexesHit": + case "IndexSegmentsHit": + case "RowsFiltered": + return JMX.newMBeanProxy(mbeanServerConn, oName, CassandraMetricsRegistry.JmxHistogramMBean.class); + case "DiskUsedBytes": + case "TotalIndexCount": + case "TotalQueryableIndexCount": + return JMX.newMBeanProxy(mbeanServerConn, oName, CassandraMetricsRegistry.JmxGaugeMBean.class).getValue(); + case "TotalQueryTimeouts": + return JMX.newMBeanProxy(mbeanServerConn, oName, CassandraMetricsRegistry.JmxCounterMBean.class).getCount(); + default: + throw new IllegalArgumentException("Unknown metric name: " + metricName); + } + } + + private String getSaiMetricScope(String metricName) + { + switch (metricName) + { + case "QueryLatency": + case "SSTableIndexesHit": + case "IndexSegmentsHit": + case "RowsFiltered": + return TableQueryMetrics.PerQueryMetrics.PER_QUERY_METRICS_TYPE; + case "PostFilteringReadLatency": + case "TotalQueryTimeouts": + return TableQueryMetrics.TABLE_QUERY_METRIC_TYPE; + case "DiskUsedBytes": + return IndexGroupMetrics.INDEX_GROUP_METRICS_TYPE; + case "TotalIndexCount": + case "TotalQueryableIndexCount": + return TableStateMetrics.TABLE_STATE_METRIC_TYPE; + default: + throw new IllegalArgumentException("Unknown metric name: " + metricName); + } + } + /** * Retrieve threadpool paths and names for threadpools with metrics. * @return Multimap from path (internal, request, etc.) to name diff --git a/src/java/org/apache/cassandra/tools/nodetool/TableStats.java b/src/java/org/apache/cassandra/tools/nodetool/TableStats.java index fc641e789155..a347fd5f7d5c 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/TableStats.java +++ b/src/java/org/apache/cassandra/tools/nodetool/TableStats.java @@ -60,8 +60,10 @@ public class TableStats extends NodeToolCmd + "memtable_off_heap_memory_used, memtable_switch_count, number_of_partitions_estimate, " + "off_heap_memory_used_total, pending_flushes, percent_repaired, read_latency, reads, " + "space_used_by_snapshots_total, space_used_live, space_used_total, " - + "sstable_compression_ratio, sstable_count, table_name, write_latency, writes, " + - "max_sstable_size, local_read_write_ratio, twcs_max_duration)") + + "sstable_compression_ratio, sstable_count, table_name, write_latency, writes, " + + "max_sstable_size, local_read_write_ratio, twcs_max_duration, sai_local_query_latency_ms," + + "sai_post_filtering_read_latency, sai_disk_used_bytes, sai_sstable_indexes_hit, sai_index_segments_hit " + + "sai_rows_filtered, sai_total_query_timeouts, sai_total_queryable_index_ratio)") private String sortKey = ""; @Option(title = "top", diff --git a/src/java/org/apache/cassandra/tools/nodetool/stats/StatsTable.java b/src/java/org/apache/cassandra/tools/nodetool/stats/StatsTable.java index 680ebf649a9b..9272babf4b65 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/stats/StatsTable.java +++ b/src/java/org/apache/cassandra/tools/nodetool/stats/StatsTable.java @@ -82,4 +82,14 @@ public class StatsTable public double localReadWriteRatio; public Long twcsDurationInMillis; public String twcs; + + public double saiQueryLatencyMs; + public double saiPostFilteringReadLatencyMs; + public String saiDiskUsedBytes; + public double saiSSTableIndexesHit; + public double saiIndexSegmentsHit; + public double saiRowsFiltered; + public long saiTotalQueryTimeouts; + public int saiTotalIndexCount; + public String saiTotalQueryableIndexRatio; } diff --git a/src/java/org/apache/cassandra/tools/nodetool/stats/StatsTableComparator.java b/src/java/org/apache/cassandra/tools/nodetool/stats/StatsTableComparator.java index c668829cdca9..98d38120ccbb 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/stats/StatsTableComparator.java +++ b/src/java/org/apache/cassandra/tools/nodetool/stats/StatsTableComparator.java @@ -66,7 +66,10 @@ public class StatsTableComparator implements Comparator<StatsTable> "space_used_by_snapshots_total", "space_used_live", "space_used_total", "sstable_compression_ratio", "sstable_count", "table_name", "write_latency", "writes", "max_sstable_size", - "local_read_write_ratio", "twcs_max_duration"}; + "local_read_write_ratio", "twcs_max_duration", "sai_local_query_latency_ms", + "sai_post_filtering_read_latency","sai_disk_used_bytes","sai_sstable_indexes_hit", + "sai_index_segments_hit","sai_rows_filtered","sai_total_query_timeouts", + "sai_total_queryable_index_ratio"}; public StatsTableComparator(String sortKey, boolean humanReadable) { @@ -338,6 +341,47 @@ else if (sortKey.equals("table_name")) { return sign * stx.tableName.compareTo(sty.tableName); } + else if(sortKey.equals("sai_local_query_latency_ms")) + { + result = compareDoubles(stx.saiQueryLatencyMs, sty.saiQueryLatencyMs); + } + else if(sortKey.equals("sai_post_filtering_read_latency")) + { + result = compareDoubles(stx.saiPostFilteringReadLatencyMs, sty.saiPostFilteringReadLatencyMs); + } + else if(sortKey.equals("sai_disk_used_bytes")) + { + result = compareFileSizes(stx.saiDiskUsedBytes, + sty.saiDiskUsedBytes); + } + else if(sortKey.equals("sai_sstable_indexes_hit")) + { + result = compareDoubles(stx.saiSSTableIndexesHit, sty.saiSSTableIndexesHit); + } + else if(sortKey.equals("sai_index_segments_hit")) + { + result = compareDoubles(stx.saiIndexSegmentsHit, sty.saiIndexSegmentsHit); + } + else if(sortKey.equals("sai_rows_filtered")) + { + result = compareDoubles(stx.saiRowsFiltered, sty.saiRowsFiltered); + } + else if(sortKey.equals("sai_total_query_timeouts")) + { + result = sign * Long.valueOf(stx.saiTotalQueryTimeouts) + .compareTo(Long.valueOf(sty.saiTotalQueryTimeouts)); + } + else if(sortKey.equals("sai_total_queryable_index_ratio")) + { + if (stx.saiTotalQueryableIndexRatio == null && sty.saiTotalQueryableIndexRatio != null) + return sign * -1; + else if (stx.saiTotalQueryableIndexRatio != null && sty.saiTotalQueryableIndexRatio == null) + return sign; + else if (stx.saiTotalQueryableIndexRatio == null && sty.saiTotalQueryableIndexRatio == null) + return 0; + + result = sign * stx.saiTotalQueryableIndexRatio.compareTo(sty.saiTotalQueryableIndexRatio); + } else { throw new IllegalStateException(String.format("Unsupported sort key: %s", sortKey)); diff --git a/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsHolder.java b/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsHolder.java index af5190e647ce..a225ccb88cd4 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsHolder.java +++ b/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsHolder.java @@ -33,6 +33,7 @@ import org.apache.cassandra.db.compaction.TimeWindowCompactionStrategyOptions; import org.apache.cassandra.io.util.*; import org.apache.cassandra.metrics.*; +import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.tools.*; public class TableStatsHolder implements StatsHolder @@ -173,6 +174,16 @@ private Map<String, Object> convertStatsTableToMap(StatsTable table) mpTable.put("top_tombstone_partitions", table.topTombstonePartitions); if (locationCheck) mpTable.put("sstables_in_correct_location", table.isInCorrectLocation); + + mpTable.put("sai_local_query_latency_ms",String.format("%01.3f", table.saiQueryLatencyMs)); + mpTable.put("sai_post_filtering_read_latency",String.format("%01.3f", table.saiPostFilteringReadLatencyMs)); + mpTable.put("sai_disk_used_bytes",table.saiDiskUsedBytes); + mpTable.put("sai_sstable_indexes_hit",table.saiSSTableIndexesHit); + mpTable.put("sai_index_segments_hit",table.saiIndexSegmentsHit); + mpTable.put("sai_rows_filtered",table.saiRowsFiltered); + mpTable.put("sai_total_query_timeouts",table.saiTotalQueryTimeouts); + mpTable.put("sai_total_queryable_index_ratio", table.saiTotalQueryableIndexRatio); + return mpTable; } @@ -385,12 +396,60 @@ private void initializeKeyspaces(NodeProbe probe, boolean ignore, List<String> t if (table.getTopTombstonePartitionsLastUpdate() != null) statsTable.topTombstonePartitionsLastUpdate = millisToDateString(table.getTopTombstonePartitionsLastUpdate()); + if (!SchemaConstants.isSystemKeyspace(keyspaceName)) + { + Object totalIndexCount = probe.getSaiMetric(keyspaceName, tableName, "TotalIndexCount"); + statsTable.saiTotalIndexCount = (totalIndexCount != null) ? (int) totalIndexCount : 0; + + if (statsTable.saiTotalIndexCount > 0) + { + Object queryLatencyMetric = probe.getSaiMetric(keyspaceName, tableName, "QueryLatency"); + double queryLatency = getMetricMean(queryLatencyMetric); + statsTable.saiQueryLatencyMs = queryLatency > 0 ? queryLatency : Double.NaN; + + Object PostFilteringReadLatency = probe.getSaiMetric(keyspaceName, tableName, "PostFilteringReadLatency"); + double postfilteringreadlatency = getMetricMean(PostFilteringReadLatency); + statsTable.saiPostFilteringReadLatencyMs = postfilteringreadlatency > 0 ? postfilteringreadlatency : Double.NaN; + + Object diskUsedBytes = probe.getSaiMetric(keyspaceName, tableName, "DiskUsedBytes"); + long saidiskusedbytes = (diskUsedBytes != null) ? (long) diskUsedBytes : 0L; + statsTable.saiDiskUsedBytes = format(saidiskusedbytes, humanReadable); + + Object SSTableIndexesHit = probe.getSaiMetric(keyspaceName, tableName, "SSTableIndexesHit"); + statsTable.saiSSTableIndexesHit = getMetricMean(SSTableIndexesHit); + + Object IndexSegmentsHit = probe.getSaiMetric(keyspaceName, tableName, "IndexSegmentsHit"); + statsTable.saiIndexSegmentsHit = getMetricMean(IndexSegmentsHit); + + Object RowsFiltered = probe.getSaiMetric(keyspaceName, tableName, "RowsFiltered"); + statsTable.saiRowsFiltered = getMetricMean(RowsFiltered); + + Object totalQueryTimeouts = probe.getSaiMetric(keyspaceName, tableName, "TotalQueryTimeouts"); + statsTable.saiTotalQueryTimeouts = (totalQueryTimeouts != null) ? (Long) totalQueryTimeouts : 0L; + + Object totalQueryableIndexCount = probe.getSaiMetric(keyspaceName, tableName, "TotalQueryableIndexCount"); + int saiTotalQueryableIndexCount = (totalQueryableIndexCount != null) ? (int) totalQueryableIndexCount : 0; + + statsTable.saiTotalQueryableIndexRatio = String.format("%d/%d", saiTotalQueryableIndexCount, statsTable.saiTotalIndexCount); + } + } + statsKeyspace.tables.add(statsTable); } keyspaces.add(statsKeyspace); } } + private double getMetricMean(Object metricObject) { + if (metricObject instanceof CassandraMetricsRegistry.JmxTimerMBean) { + return ((CassandraMetricsRegistry.JmxTimerMBean) metricObject).getMean() / 1000; + } + if (metricObject instanceof CassandraMetricsRegistry.JmxHistogramMBean) { + return Math.round(((CassandraMetricsRegistry.JmxHistogramMBean) metricObject).getMean() * 100.0) / 100.0; + } + throw new IllegalArgumentException("Unsupported metric object type: " + metricObject.getClass().getName()); + } + private void maybeAddTWCSWindowWithMaxDuration(StatsTable statsTable, NodeProbe probe, String keyspaceName, String tableName) { Map<String, String> compactionParameters = probe.getCfsProxy(statsTable.keyspaceName, statsTable.tableName) diff --git a/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsPrinter.java b/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsPrinter.java index 02c0787a15d3..eb272d38dc0e 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsPrinter.java +++ b/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsPrinter.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Map; +import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.utils.FBUtilities; public class TableStatsPrinter<T extends StatsHolder> @@ -163,6 +164,19 @@ protected void printStatsTable(StatsTable table, String tableDisplayName, String for (Map.Entry<String, Long> tombstonecnt : table.topTombstonePartitions.entrySet()) out.printf(indent + " %-" + maxWidth + "s %s%n", tombstonecnt.getKey(), tombstonecnt.getValue()); } + + if (!SchemaConstants.isSystemKeyspace(table.keyspaceName) && table.saiTotalIndexCount > 0) + { + out.println(indent + "SAI local query latency (mean): " + String.format("%.3f ms", table.saiQueryLatencyMs)); + out.println(indent + "SAI post-filtering latency (mean): " + String.format("%.3f ms",table.saiPostFilteringReadLatencyMs)); + out.println(indent + "SAI space used (bytes): " + table.saiDiskUsedBytes); + out.println(indent + "SAI sstable indexes hit per query (mean): " + table.saiSSTableIndexesHit); + out.println(indent + "SAI index segments hit per query (mean): " + table.saiIndexSegmentsHit); + out.println(indent + "SAI rows filtered per query (mean): " + table.saiRowsFiltered); + out.println(indent + "SAI local query timeouts: " + table.saiTotalQueryTimeouts); + out.println(indent + "SAI queryable/total indexes: " + table.saiTotalQueryableIndexRatio); + } + out.println(""); } } diff --git a/test/unit/org/apache/cassandra/tools/nodetool/TableStatsTest.java b/test/unit/org/apache/cassandra/tools/nodetool/TableStatsTest.java index b4e9b90f4a45..bfd7b8bee4bd 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/TableStatsTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/TableStatsTest.java @@ -112,7 +112,11 @@ public void testMaybeChangeDocs() " read_latency, reads, space_used_by_snapshots_total, space_used_live,\n" + " space_used_total, sstable_compression_ratio, sstable_count,\n" + " table_name, write_latency, writes, max_sstable_size,\n" + - " local_read_write_ratio, twcs_max_duration)\n" + + " local_read_write_ratio, twcs_max_duration,\n" + + " sai_local_query_latency_ms,sai_post_filtering_read_latency,\n" + + " sai_disk_used_bytes, sai_sstable_indexes_hit, sai_index_segments_hit\n" + + " sai_rows_filtered, sai_total_query_timeouts,\n" + + " sai_total_queryable_index_ratio)\n" + "\n" + " -t <top>, --top <top>\n" + " Show only the top K tables for the sort key (specify the number K of\n" + diff --git a/test/unit/org/apache/cassandra/tools/nodetool/stats/TableStatsPrinterTest.java b/test/unit/org/apache/cassandra/tools/nodetool/stats/TableStatsPrinterTest.java index 80dcb9764272..427c5e2a4066 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/stats/TableStatsPrinterTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/stats/TableStatsPrinterTest.java @@ -228,6 +228,14 @@ public class TableStatsPrinterTest extends TableStatsTestBase "\tAverage tombstones per slice (last five minutes): 4.01\n" + "\tMaximum tombstones per slice (last five minutes): 5\n" + "\tDroppable tombstone ratio: 0.55556\n" + + "\tSAI local query latency (mean): 10.000 ms\n" + + "\tSAI post-filtering latency (mean): 1.000 ms\n" + + "\tSAI space used (bytes): 40 bytes\n" + + "\tSAI sstable indexes hit per query (mean): 3.5\n" + + "\tSAI index segments hit per query (mean): 4.0\n" + + "\tSAI rows filtered per query (mean): 55.0\n" + + "\tSAI local query timeouts: 4\n" + + "\tSAI queryable/total indexes: 5/5\n" + "\n"; public static final String expectedDefaultTable6Output = @@ -412,48 +420,56 @@ public void testJsonPrinter() throws Exception " \"write_latency_ms\" : null,\n" + " \"tables\" : {\n" + " \"table6\" : {\n" + + " \"twcs\" : null,\n" + + " \"sai_total_queryable_index_ratio\" : \"0/0\",\n" + + " \"memtable_switch_count\" : 6,\n" + + " \"sai_post_filtering_read_latency\" : \"NaN\",\n" + + " \"speculative_retries\" : 0,\n" + + " \"average_live_cells_per_slice_last_five_minutes\" : 5.0,\n" + + " \"local_read_latency_ms\" : \"1.000\",\n" + + " \"sstable_count\" : 1000,\n" + + " \"compacted_partition_minimum_bytes\" : 6,\n" + + " \"sstable_compression_ratio\" : 0.68,\n" + + " \"max_sstable_size\" : 0,\n" + + " \"sai_index_segments_hit\" : \"NaN\",\n" + + " \"memtable_off_heap_memory_used\" : \"161803398\",\n" + + " \"bloom_filter_space_used\" : \"101112\",\n" + + " \"compacted_partition_maximum_bytes\" : 20,\n" + + " \"sai_rows_filtered\" : \"NaN\",\n" + + " \"droppable_tombstone_ratio\" : \"0.66667\",\n" + + " \"compression_metadata_off_heap_memory_used\" : \"1\",\n" + + " \"bloom_filter_false_ratio\" : \"0.03000\",\n" + + " \"percent_repaired\" : 0.0,\n" + " \"average_tombstones_per_slice_last_five_minutes\" : 6.0,\n" + " \"top_tombstone_partitions\" : null,\n" + + " \"sai_total_query_timeouts\" : 0,\n" + " \"bloom_filter_off_heap_memory_used\" : \"667408\",\n" + - " \"twcs\" : null,\n" + " \"bytes_pending_repair\" : 0,\n" + - " \"memtable_switch_count\" : 6,\n" + - " \"speculative_retries\" : 0,\n" + " \"maximum_tombstones_per_slice_last_five_minutes\" : 6,\n" + " \"memtable_cell_count\" : 6666,\n" + " \"memtable_data_size\" : \"1000000\",\n" + - " \"average_live_cells_per_slice_last_five_minutes\" : 5.0,\n" + - " \"local_read_latency_ms\" : \"1.000\",\n" + - " \"sstable_count\" : 1000,\n" + " \"local_write_latency_ms\" : \"0.500\",\n" + " \"pending_flushes\" : 66,\n" + - " \"compacted_partition_minimum_bytes\" : 6,\n" + " \"local_read_count\" : 5,\n" + - " \"sstable_compression_ratio\" : 0.68,\n" + - " \"max_sstable_size\" : 0,\n" + " \"top_size_partitions\" : null,\n" + " \"bloom_filter_false_positives\" : 400,\n" + " \"off_heap_memory_used_total\" : \"162470810\",\n" + - " \"memtable_off_heap_memory_used\" : \"161803398\",\n" + " \"index_summary_off_heap_memory_used\" : \"3\",\n" + - " \"bloom_filter_space_used\" : \"101112\",\n" + + " \"sai_local_query_latency_ms\" : \"NaN\",\n" + " \"sstables_in_each_level\" : [ ],\n" + - " \"compacted_partition_maximum_bytes\" : 20,\n" + " \"sstable_bytes_in_each_level\" : [ ],\n" + " \"space_used_total\" : \"0\",\n" + " \"local_write_count\" : 0,\n" + - " \"droppable_tombstone_ratio\" : \"0.66667\",\n" + - " \"compression_metadata_off_heap_memory_used\" : \"1\",\n" + + " \"sai_sstable_indexes_hit\" : \"NaN\",\n" + " \"local_read_write_ratio\" : \"0.00000\",\n" + " \"number_of_partitions_estimate\" : 6,\n" + + " \"sai_disk_used_bytes\" : \"NaN\",\n" + " \"bytes_repaired\" : 0,\n" + " \"maximum_live_cells_per_slice_last_five_minutes\" : 2,\n" + " \"space_used_live\" : \"666666\",\n" + " \"compacted_partition_mean_bytes\" : 3,\n" + - " \"bloom_filter_false_ratio\" : \"0.03000\",\n" + " \"old_sstable_count\" : 0,\n" + " \"bytes_unrepaired\" : 0,\n" + - " \"percent_repaired\" : 0.0,\n" + " \"space_used_by_snapshots_total\" : \"0\"\n" + " }\n" + " },\n" + @@ -481,48 +497,56 @@ public void testYamlPrinter() throws Exception " write_latency_ms: .NaN\n" + " tables:\n" + " table6:\n" + + " twcs: null\n" + + " sai_total_queryable_index_ratio: 0/0\n" + + " memtable_switch_count: 6\n" + + " sai_post_filtering_read_latency: NaN\n" + + " speculative_retries: 0\n" + + " average_live_cells_per_slice_last_five_minutes: 5.0\n" + + " local_read_latency_ms: '1.000'\n" + + " sstable_count: 1000\n" + + " compacted_partition_minimum_bytes: 6\n" + + " sstable_compression_ratio: 0.68\n" + + " max_sstable_size: 0\n" + + " sai_index_segments_hit: .NaN\n" + + " memtable_off_heap_memory_used: '161803398'\n" + + " bloom_filter_space_used: '101112'\n" + + " compacted_partition_maximum_bytes: 20\n" + + " sai_rows_filtered: .NaN\n" + + " droppable_tombstone_ratio: '0.66667'\n" + + " compression_metadata_off_heap_memory_used: '1'\n" + + " bloom_filter_false_ratio: '0.03000'\n" + + " percent_repaired: 0.0\n" + " average_tombstones_per_slice_last_five_minutes: 6.0\n" + " top_tombstone_partitions: null\n" + + " sai_total_query_timeouts: 0\n" + " bloom_filter_off_heap_memory_used: '667408'\n" + - " twcs: null\n" + " bytes_pending_repair: 0\n" + - " memtable_switch_count: 6\n" + - " speculative_retries: 0\n" + " maximum_tombstones_per_slice_last_five_minutes: 6\n" + " memtable_cell_count: 6666\n" + " memtable_data_size: '1000000'\n" + - " average_live_cells_per_slice_last_five_minutes: 5.0\n" + - " local_read_latency_ms: '1.000'\n" + - " sstable_count: 1000\n" + " local_write_latency_ms: '0.500'\n" + " pending_flushes: 66\n" + - " compacted_partition_minimum_bytes: 6\n" + " local_read_count: 5\n" + - " sstable_compression_ratio: 0.68\n" + - " max_sstable_size: 0\n" + " top_size_partitions: null\n" + " bloom_filter_false_positives: 400\n" + " off_heap_memory_used_total: '162470810'\n" + - " memtable_off_heap_memory_used: '161803398'\n" + " index_summary_off_heap_memory_used: '3'\n" + - " bloom_filter_space_used: '101112'\n" + + " sai_local_query_latency_ms: NaN\n" + " sstables_in_each_level: []\n" + - " compacted_partition_maximum_bytes: 20\n" + " sstable_bytes_in_each_level: []\n" + " space_used_total: '0'\n" + " local_write_count: 0\n" + - " droppable_tombstone_ratio: '0.66667'\n" + - " compression_metadata_off_heap_memory_used: '1'\n" + + " sai_sstable_indexes_hit: .NaN\n" + " local_read_write_ratio: '0.00000'\n" + " number_of_partitions_estimate: 6\n" + + " sai_disk_used_bytes: NaN\n" + " bytes_repaired: 0\n" + " maximum_live_cells_per_slice_last_five_minutes: 2\n" + " space_used_live: '666666'\n" + " compacted_partition_mean_bytes: 3\n" + - " bloom_filter_false_ratio: '0.03000'\n" + " old_sstable_count: 0\n" + " bytes_unrepaired: 0\n" + - " percent_repaired: 0.0\n" + " space_used_by_snapshots_total: '0'\n" + " read_latency_ms: 0.0\n" + " pending_flushes: 66\n" + diff --git a/test/unit/org/apache/cassandra/tools/nodetool/stats/TableStatsTestBase.java b/test/unit/org/apache/cassandra/tools/nodetool/stats/TableStatsTestBase.java index 317eff01f468..0e55e5a3d7be 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/stats/TableStatsTestBase.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/stats/TableStatsTestBase.java @@ -102,6 +102,14 @@ private static StatsTable createStatsTableTemplate(String keyspaceName, String t template.maximumTombstonesPerSliceLastFiveMinutes = 0L; template.twcs = null; template.twcsDurationInMillis = 0L; + template.saiQueryLatencyMs = Double.NaN; + template.saiPostFilteringReadLatencyMs = Double.NaN; + template.saiDiskUsedBytes = "NaN"; + template.saiSSTableIndexesHit = Double.NaN; + template.saiIndexSegmentsHit = Double.NaN; + template.saiRowsFiltered = Double.NaN; + template.saiTotalQueryTimeouts = 0L; + template.saiTotalQueryableIndexRatio = "0/0"; return template; } @@ -337,6 +345,18 @@ public static void createTestVector() table2.twcsDurationInMillis = 2000L; table4.twcsDurationInMillis = 1000L; table5.twcsDurationInMillis = null; + + // Table5 with 5 SAI. + table5.saiQueryLatencyMs = 10.000D; + table5.saiPostFilteringReadLatencyMs = 1.000D; + table5.saiDiskUsedBytes = "40 bytes"; + table5.saiSSTableIndexesHit = 3.5D; + table5.saiIndexSegmentsHit = 4.0D; + table5.saiRowsFiltered = 55.0D; + table5.saiTotalQueryTimeouts = 4L; + table5.saiTotalIndexCount = 5; + table5.saiTotalQueryableIndexRatio = "5/5"; + // create test keyspaces from templates testKeyspaces = new ArrayList<>(); StatsKeyspace keyspace1 = createStatsKeyspaceTemplate("keyspace1"); @@ -395,6 +415,10 @@ public static void createTestVector() humanReadableTable4.memtableDataSize = "999 bytes"; humanReadableTable5.memtableDataSize = "3.14 MiB"; humanReadableTable6.memtableDataSize = "0 bytes"; + + // cretae human-readable SAI disk space used size: + humanReadableTable5.saiDiskUsedBytes = "40 bytes"; + // create human readable keyspaces from template humanReadableKeyspaces = new ArrayList<>(); StatsKeyspace humanReadableKeyspace1 = createStatsKeyspaceTemplate("keyspace1"); From 4bc61e5209d474c50639a4858e273653fbb4e399 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Mon, 9 Dec 2024 11:52:49 +0100 Subject: [PATCH 089/225] Do not attach rows and partitions to CheckForAbort when already attached patch by Stefan Miklosovic; reviewed by Josh McKenzie for CASSANDRA-20135 --- CHANGES.txt | 1 + .../org/apache/cassandra/db/ReadCommand.java | 18 ++++++++++++++++++ .../db/transform/StoppingTransformation.java | 4 ++-- 3 files changed, 21 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 1ef6a902e4fc..834c20e5c46d 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0.16 + * Do not attach rows and partitions to CheckForAbort when already attached (CASSANDRA-20135) * Allow hint delivery during schema mismatch (CASSANDRA-20188) * Fix gossip issue with gossip-only and bootstrapping nodes missing DC/Rack/Host ID endpoint state (CASSANDRA-19983) * IndexOutOfBoundsException when accessing partition where the column was deleted (CASSANDRA-20108) diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java index 1df23cc7fd0d..bdbbb4cd7452 100644 --- a/src/java/org/apache/cassandra/db/ReadCommand.java +++ b/src/java/org/apache/cassandra/db/ReadCommand.java @@ -35,6 +35,8 @@ import org.apache.cassandra.config.*; import org.apache.cassandra.db.filter.*; +import org.apache.cassandra.db.transform.BasePartitions; +import org.apache.cassandra.db.transform.BaseRows; import org.apache.cassandra.net.MessageFlag; import org.apache.cassandra.net.Verb; import org.apache.cassandra.db.partitions.*; @@ -559,6 +561,22 @@ protected class CheckForAbort extends StoppingTransformation<UnfilteredRowIterat { long lastChecked = 0; + @Override + protected void attachTo(BasePartitions partitions) + { + Preconditions.checkArgument(this.partitions == null || this.partitions == partitions, + "Attempted to attach 2nd different BasePartitions in StoppingTransformation; this is a bug."); + this.partitions = partitions; + } + + @Override + protected void attachTo(BaseRows rows) + { + Preconditions.checkArgument(this.rows == null || this.rows == rows, + "Attempted to attach 2nd different BaseRows in StoppingTransformation; this is a bug."); + this.rows = rows; + } + protected UnfilteredRowIterator applyToPartition(UnfilteredRowIterator partition) { if (maybeAbort()) diff --git a/src/java/org/apache/cassandra/db/transform/StoppingTransformation.java b/src/java/org/apache/cassandra/db/transform/StoppingTransformation.java index 79563e976331..691ee39bdbcb 100644 --- a/src/java/org/apache/cassandra/db/transform/StoppingTransformation.java +++ b/src/java/org/apache/cassandra/db/transform/StoppingTransformation.java @@ -26,8 +26,8 @@ // A Transformation that can stop an iterator earlier than its natural exhaustion public abstract class StoppingTransformation<I extends BaseRowIterator<?>> extends Transformation<I> { - private BaseIterator rows; - private BaseIterator partitions; + protected BaseIterator rows; + protected BaseIterator partitions; /** * If invoked by a subclass, any partitions iterator this transformation has been applied to will terminate From f66183a711dd440b3619a960ba4e5444c1c900ff Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Fri, 17 Jan 2025 13:42:08 +0100 Subject: [PATCH 090/225] Run audit_logging_options through santiation and validation on startup patch by Stefan Miklosovic; reviewed by Brandon Williams for CASSANDRA-20208 --- CHANGES.txt | 1 + src/java/org/apache/cassandra/config/DatabaseDescriptor.java | 3 +++ 2 files changed, 4 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index eea12c7fd77d..2b7592eab053 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.1.8 + * Run audit_logging_options through santiation and validation on startup (CASSANDRA-20208) * Enforce CQL message size limit on multiframe messages (CASSANDRA-20052) * Add nodetool checktokenmetadata command that checks TokenMetadata is insync with Gossip endpointState (CASSANDRA-18758) * Backport Java 11 support for Simulator (CASSANDRA-17178/CASSANDRA-19935) diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 3ff03ce80150..920e3ae90cf6 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -935,6 +935,9 @@ else if (conf.max_value_size.toMebibytes() >= 2048) conf.native_transport_min_backoff_on_queue_overload, conf.native_transport_max_backoff_on_queue_overload)); + // run audit logging options through sanitation and validation + if (conf.audit_logging_options != null) + setAuditLoggingOptions(conf.audit_logging_options); } @VisibleForTesting From c46b6cd21bd1da0c17de59d313ed84630be7dd04 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Mon, 20 Jan 2025 09:18:59 +0100 Subject: [PATCH 091/225] ninja: CASSANDRA-18956 was committed only to 3.0 branch This was most probably added by mistake when merging CASSANDRA-20090 up. --- CHANGES.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 8c4f6e449284..dcea57c25e9c 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,7 +1,6 @@ 3.11.18 Merged from 3.0: * Tighten up permissions on system keyspaces (CASSANDRA-20090) - * Fix incorrect column identifier bytes problem when renaming a column (CASSANDRA-18956) * Upgrade OWASP to 10.0.0 (CASSANDRA-19738) Merged from 2.2: * Add termin-8-jdk as a valid jdk8 candidate in the debian package (CASSANDRA-19752) From 4f0ad22f122f8233aa99545bc234083690f53ea7 Mon Sep 17 00:00:00 2001 From: Matt Byrd <matthew_byrd@apple.com> Date: Thu, 7 Nov 2024 09:37:24 -0800 Subject: [PATCH 092/225] Use mutation creation time for hint expiry rather than hint submission time patch by Matt Byrd; reviewed by Chris Lohfink, reviewed by Blake Eggleston for CASSANDRA-20014 --- CHANGES.txt | 1 + .../org/apache/cassandra/config/Config.java | 1 + .../cassandra/config/DatabaseDescriptor.java | 10 + .../org/apache/cassandra/db/Mutation.java | 5 + src/java/org/apache/cassandra/hints/Hint.java | 4 +- .../service/AbstractWriteResponseHandler.java | 5 + .../cassandra/service/StorageProxy.java | 14 +- .../cassandra/service/StorageService.java | 28 +- .../service/StorageServiceMBean.java | 10 + .../test/AbstractHintWindowTest.java | 100 +++++++ .../test/HintDataReappearingTest.java | 257 ++++++++++++++++++ 11 files changed, 430 insertions(+), 5 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/test/AbstractHintWindowTest.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/HintDataReappearingTest.java diff --git a/CHANGES.txt b/CHANGES.txt index ef327e6be95c..67ad77673e57 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0.16 + * Make hint expiry use request start time rather than timeout time for TTL (CASSANDRA-20014) * Do not attach rows and partitions to CheckForAbort when already attached (CASSANDRA-20135) * Allow hint delivery during schema mismatch (CASSANDRA-20188) * Fix gossip issue with gossip-only and bootstrapping nodes missing DC/Rack/Host ID endpoint state (CASSANDRA-19983) diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 57388eed685c..24df356a4bb4 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -300,6 +300,7 @@ public class Config public int max_hints_file_size_in_mb = 128; public ParameterizedClass hints_compression; public volatile boolean transfer_hints_on_decommission = true; + public volatile boolean use_creation_time_for_hint_ttl = true; public volatile boolean incremental_backups = false; public boolean trickle_fsync = false; diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index bf96a3f8856e..593235d5771e 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -2638,6 +2638,16 @@ public static void setTransferHintsOnDecommission(boolean enabled) conf.transfer_hints_on_decommission = enabled; } + public static boolean isUseCreationTimeForHintTtl() + { + return conf.use_creation_time_for_hint_ttl; + } + + public static void setUseCreationTimeForHintTtl(boolean enabled) + { + conf.use_creation_time_for_hint_ttl = enabled; + } + public static boolean isIncrementalBackupsEnabled() { return conf.incremental_backups; diff --git a/src/java/org/apache/cassandra/db/Mutation.java b/src/java/org/apache/cassandra/db/Mutation.java index 8a1ffc123c54..78291e07bef6 100644 --- a/src/java/org/apache/cassandra/db/Mutation.java +++ b/src/java/org/apache/cassandra/db/Mutation.java @@ -147,6 +147,11 @@ public PartitionUpdate getPartitionUpdate(TableMetadata table) return table == null ? null : modifications.get(table.id); } + public long getApproxCreatedAtNanos() + { + return approxCreatedAtNanos; + } + public boolean isEmpty() { return modifications.isEmpty(); diff --git a/src/java/org/apache/cassandra/hints/Hint.java b/src/java/org/apache/cassandra/hints/Hint.java index 6c7c5d46baee..06541dfdf49c 100644 --- a/src/java/org/apache/cassandra/hints/Hint.java +++ b/src/java/org/apache/cassandra/hints/Hint.java @@ -51,13 +51,13 @@ * compacted away while the hint was in storage. * * We also look at the smallest current value of the gcgs param for each affected table when applying the hint, and use - * creation time + min(recorded gc gs, current gcgs + current gc grace) as the overall hint expiration time. + * mutation creation time + min(recorded gc gs, current gcgs + current gc grace) as the overall hint expiration time. * This allows now to safely reduce gc gs on tables without worrying that an applied old hint might resurrect any data. */ public final class Hint { public static final Serializer serializer = new Serializer(); - static final int maxHintTTL = Integer.getInteger("cassandra.maxHintTTL", Integer.MAX_VALUE); + public static volatile int maxHintTTL = Integer.getInteger("cassandra.maxHintTTL", Integer.MAX_VALUE); final Mutation mutation; final long creationTime; // time of hint creation (in milliseconds) diff --git a/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java b/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java index 0ef944c24701..f0d7766fa7ee 100644 --- a/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java +++ b/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java @@ -237,6 +237,11 @@ protected boolean waitingFor(InetAddressAndPort from) */ protected abstract int ackCount(); + public long getQueryStartNanoTime() + { + return queryStartNanoTime; + } + /** * null message means "response from local write" */ diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java index 3e4292ae7bdf..23ee648454a9 100644 --- a/src/java/org/apache/cassandra/service/StorageProxy.java +++ b/src/java/org/apache/cassandra/service/StorageProxy.java @@ -130,6 +130,9 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.NANOSECONDS; + +import static org.apache.cassandra.utils.MonotonicClock.approxTime; + import static org.apache.cassandra.metrics.ClientRequestsMetricsHolder.casReadMetrics; import static org.apache.cassandra.metrics.ClientRequestsMetricsHolder.casWriteMetrics; import static org.apache.cassandra.metrics.ClientRequestsMetricsHolder.readMetrics; @@ -2427,8 +2430,15 @@ public void runMayThrow() else logger.debug("Discarding hint for endpoint not part of ring: {}", target); } - logger.trace("Adding hints for {}", validTargets); - HintsService.instance.write(hostIds, Hint.create(mutation, System.currentTimeMillis())); + + long creationTime = System.currentTimeMillis(); + if (DatabaseDescriptor.isUseCreationTimeForHintTtl()) + { + long mutationCreationTimeNanos = responseHandler != null ? responseHandler.getQueryStartNanoTime() : mutation.getApproxCreatedAtNanos(); + creationTime -= TimeUnit.MILLISECONDS.convert(Math.max(0, approxTime.now() - mutationCreationTimeNanos), NANOSECONDS); + } + logger.trace("Adding hints for {} with creation time {} ms", validTargets, creationTime); + HintsService.instance.write(hostIds, Hint.create(mutation, creationTime)); validTargets.forEach(HintsService.instance.metrics::incrCreatedHints); // Notify the handler only for CL == ANY if (responseHandler != null && responseHandler.replicaPlan.consistencyLevel() == ConsistencyLevel.ANY) diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 5b29e4bee9e2..d3dab125e4d9 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -139,6 +139,7 @@ import org.apache.cassandra.gms.IFailureDetector; import org.apache.cassandra.gms.TokenSerializer; import org.apache.cassandra.gms.VersionedValue; +import org.apache.cassandra.hints.Hint; import org.apache.cassandra.hints.HintsService; import org.apache.cassandra.io.sstable.SSTableLoader; import org.apache.cassandra.io.sstable.format.SSTableFormat; @@ -6036,12 +6037,37 @@ public void setTransferHintsOnDecommission(boolean enabled) logger.info("updated transfer_hints_on_decommission to {}", enabled); } - public void setHintedHandoffThrottleInKB(int throttleInKB) + @Override + public boolean isHintTtlUseMutationCreationTime() + { + return DatabaseDescriptor.isUseCreationTimeForHintTtl(); + } + + @Override + public void setUseCreationTimeForHintTtl(boolean enabled) { + DatabaseDescriptor.setUseCreationTimeForHintTtl(enabled); + logger.info("updated use_creation_time_for_hint_ttl to {}", enabled); + } + + public void setHintedHandoffThrottleInKB(int throttleInKB) { DatabaseDescriptor.setHintedHandoffThrottleInKB(throttleInKB); logger.info("updated hinted_handoff_throttle_in_kb to {}", throttleInKB); } + @Override + public int getMaxHintTTL() + { + return Hint.maxHintTTL; + } + + @Override + public void setMaxHintTTL(int maxHintTTL) + { + Hint.maxHintTTL = maxHintTTL; + logger.info("updated Hint.maxHintTTL to {}", maxHintTTL); + } + @Override public void clearConnectionHistory() { diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java index 0cc6dbd21af0..ecdfb0e67c0a 100644 --- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java +++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java @@ -777,6 +777,16 @@ public interface StorageServiceMBean extends NotificationEmitter public boolean getTransferHintsOnDecommission(); public void setTransferHintsOnDecommission(boolean enabled); + /** Returns whether we are using the creation time of the mutation for determining hint ttl **/ + public boolean isHintTtlUseMutationCreationTime(); + /** Sets whether we are using the creation time of the mutation for determining hint ttl **/ + public void setUseCreationTimeForHintTtl(boolean enabled); + + /** Returns upper bound on the hint ttl **/ + public int getMaxHintTTL(); + /** Sets the upper bound on the hint ttl **/ + public void setMaxHintTTL(int maxHintTTL); + /** * Resume bootstrap streaming when there is failed data streaming. * diff --git a/test/distributed/org/apache/cassandra/distributed/test/AbstractHintWindowTest.java b/test/distributed/org/apache/cassandra/distributed/test/AbstractHintWindowTest.java new file mode 100644 index 000000000000..fbb2e6311653 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/AbstractHintWindowTest.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import java.util.UUID; + +import org.junit.Ignore; + +import org.apache.cassandra.auth.CassandraRoleManager; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.api.IIsolatedExecutor; +import org.apache.cassandra.gms.FailureDetector; +import org.apache.cassandra.hints.HintsService; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.metrics.StorageMetrics; +import org.apache.cassandra.service.StorageService; + +import static java.util.concurrent.TimeUnit.MINUTES; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.cassandra.distributed.api.ConsistencyLevel.ONE; +import static org.awaitility.Awaitility.await; + +@Ignore +public abstract class AbstractHintWindowTest extends TestBaseImpl +{ + + void pauseHintsDelivery(IInvokableInstance node) + { + node.runOnInstance((IIsolatedExecutor.SerializableRunnable) () -> { + HintsService.instance.pauseDispatch(); + }); + } + + void transferHints(IInvokableInstance node, UUID transferToNode) + { + node.runOnInstance((IIsolatedExecutor.SerializableRunnable) () -> { + HintsService.instance.transferHints(() -> transferToNode); + }); + } + + void waitUntilNodeState(IInvokableInstance node, UUID node2UUID, boolean shouldBeOnline) + { + await().pollInterval(10, SECONDS) + .timeout(1, MINUTES) + .until(() -> node.appliesOnInstance((IIsolatedExecutor.SerializableBiFunction<UUID, Boolean, Boolean>) (secondNode, online) -> { + InetAddressAndPort address = StorageService.instance.getEndpointForHostId(secondNode); + return online == FailureDetector.instance.isAlive(address); + }).apply(node2UUID, shouldBeOnline)); + } + + Long getTotalHintsCount(IInvokableInstance node) + { + return node.callOnInstance(() -> StorageMetrics.totalHints.getCount()); + } + + Long insertData(final Cluster cluster) { + + // insert data and sleep every 10k to have a chance to flush hints + for (int i = 0; i < 70000; i++) { + cluster.coordinator(1) + .execute(withKeyspace("INSERT INTO %s.cf (k, c1) VALUES (?, ?);"), + ONE, UUID.randomUUID().toString(), UUID.randomUUID().toString()); + + if (i % 10000 == 0) + await().atLeast(2, SECONDS).pollDelay(2, SECONDS).until(() -> true); + } + + await().atLeast(3, SECONDS).pollDelay(3, SECONDS).until(() -> true); + + // we see that metrics are updated + + await().until(() -> cluster.get(1).callOnInstance(() -> StorageMetrics.totalHints.getCount()) > 0); + return cluster.get(1).callOnInstance(() -> StorageMetrics.totalHints.getCount()); + } + + static void waitForExistingRoles(Cluster cluster) + { + cluster.forEach(instance -> await().pollDelay(1, SECONDS) + .pollInterval(1, SECONDS) + .atMost(60, SECONDS) + .until(() -> instance.callOnInstance(CassandraRoleManager::hasExistingRoles))); + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/HintDataReappearingTest.java b/test/distributed/org/apache/cassandra/distributed/test/HintDataReappearingTest.java new file mode 100644 index 000000000000..e4df267eb834 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/HintDataReappearingTest.java @@ -0,0 +1,257 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import java.util.*; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; + +import org.apache.cassandra.db.SystemKeyspace; +import org.apache.cassandra.distributed.api.IIsolatedExecutor; +import org.apache.cassandra.exceptions.WriteTimeoutException; + +import org.apache.commons.lang3.ArrayUtils; +import org.junit.Assert; +import org.junit.Test; + +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.api.SimpleQueryResult; +import org.apache.cassandra.net.Verb; +import org.assertj.core.api.Assertions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static java.lang.String.format; +import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL; +import static org.apache.cassandra.distributed.api.ConsistencyLevel.LOCAL_QUORUM; +import static org.apache.cassandra.distributed.api.Feature.GOSSIP; +import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL; +import static org.apache.cassandra.distributed.api.Feature.NETWORK; + +public class HintDataReappearingTest extends AbstractHintWindowTest +{ + private static final Logger logger = LoggerFactory.getLogger(HintDataReappearingTest.class); + + private static final ScheduledExecutorService scheduler = Executors + .newScheduledThreadPool(1, + new ThreadFactoryBuilder() + .setNameFormat("hint reappearance test") + .setDaemon(true) + .build()); + + @Test + public void testHintCausesDataReappearance() throws Exception + { + doHintReappearData(true, false); + } + + @Test + public void demonstrateHintCausesDataReappearance() throws Exception + { + doHintReappearData(false, false); + } + + @Test + public void testHintCausesDataReappearanceWriteTimeout() throws Exception + { + doHintReappearData(true, true); + } + + @Test + public void demonstrateHintCausesDataReappearanceWriteTimeout() throws Exception + { + doHintReappearData(false, true); + } + + public void doHintReappearData(final boolean preventReappearance, final boolean dropTwoWrites) throws Exception + { + try (Cluster cluster = init(Cluster.build(3) + .withDataDirCount(1) + + .withConfig(config -> config.with(NETWORK, GOSSIP, NATIVE_PROTOCOL) + .set("hinted_handoff_enabled", true) + .set("max_hints_delivery_threads", "1") + .set("use_creation_time_for_hint_ttl", preventReappearance ? "true" : "false") + .set("write_request_timeout_in_ms", "30000")) + .start(), 3)) + { + final IInvokableInstance node1 = cluster.get(1); + final IInvokableInstance node2 = cluster.get(2); + final IInvokableInstance node3 = cluster.get(3); + + waitForExistingRoles(cluster); + + // We create a table with low gc_grace_seconds to ensure we can check the interactions in a timely manner + // Also make compaction fairly proactive in an attempt avoiding to force compact repeatedly + final int gc_grace_seconds = 40; + final String createTableStatement = format("CREATE TABLE %s.cf (k text PRIMARY KEY, c1 text) " + + "WITH gc_grace_seconds = " + gc_grace_seconds + + " AND compaction = {'class': 'SizeTieredCompactionStrategy', 'min_threshold': 2, 'max_threshold': 32 } ", KEYSPACE); + cluster.schemaChange(createTableStatement); + + // setup a message filter to drop mutations requests from node1 to node2 so it creates hints for those mutations + AtomicBoolean dropWritesForNode2 = new AtomicBoolean(true); + // toggling this on allows the test to timeout the original write, a slight variation, both cause the problem. + AtomicBoolean dropWritesForNode3 = new AtomicBoolean(dropTwoWrites); + cluster.filters() + .verbs(Verb.MUTATION_REQ.id) + .from(1) + .messagesMatching((from, to, message) -> + (to == 2 && dropWritesForNode2.get() + || (to == 3 && dropWritesForNode3.get()))) + .drop(); + + + logger.info("Pausing hint delivery"); + // pause hint delivery to imitate hints being behind/backed up + pauseHintsDelivery(node1); + + logger.info("Inserting data"); + + List<UUID> keys = IntStream.range(0, 1).mapToObj(x -> UUID.randomUUID()).collect(Collectors.toList()); + + scheduler.submit(() -> { + if (dropTwoWrites) + { + Assertions.assertThatThrownBy(() -> insertData(cluster, keys)) + .isInstanceOf(WriteTimeoutException.class); + } + else + { + insertData(cluster, keys); + } + }); + + Thread.sleep(1000); + dropWritesForNode2.set(false); + dropWritesForNode3.set(false); + + node1.flush(KEYSPACE); + node2.flush(KEYSPACE); + node3.flush(KEYSPACE); + + logger.info("Deleting data"); + deleteData(cluster, keys); + long afterDelete = System.currentTimeMillis(); + + node1.flush(KEYSPACE); + node2.flush(KEYSPACE); + node3.flush(KEYSPACE); + + + logger.info("Repairing"); + for (IInvokableInstance node : Arrays.asList(node1, node2, node3)) + { + node.nodetoolResult(ArrayUtils.addAll(new String[]{ "repair", KEYSPACE }, "--full")).asserts().success(); + } + logger.info("Done repairing"); + + + for (SimpleQueryResult result : selectData(cluster, keys)) + { + Object[][] objectArrays = result.toObjectArrays(); + logger.info("Result after delete: {} {}", result, Arrays.deepToString(objectArrays)); + // We expect the data to appear to be deleted initially + Assert.assertNull(objectArrays[0][1]); + } + + // wait to pass gc_grace_seconds with slight buffer of 2 seconds to ensure delete persisted long enough to be gced + long msSinceDelete = Math.abs(System.currentTimeMillis() - afterDelete); + long sleepFor = Math.max(0, 1000 * gc_grace_seconds + 2000 - msSinceDelete); + logger.info("Sleeping {} ms to ensure gc_grace_seconds has ellapsed after tombstone creation", sleepFor); + Thread.sleep(sleepFor); + + // ensure tombstone purged on all 3 nodes + node1.forceCompact(KEYSPACE, "cf"); + node3.forceCompact(KEYSPACE, "cf"); + node2.forceCompact(KEYSPACE, "cf"); + + + IIsolatedExecutor.CallableNoExcept<UUID> node2hostId = node2.callsOnInstance(SystemKeyspace::getLocalHostId); + + transferHints(node1, node2hostId.call()); + + // Sleep a bit more to ensure hint is delivered after tombstone is GCed + + Thread.sleep(200); + logger.info("Total Hints after sleeping: {}", getTotalHintsCount(node1)); + + // Check the results of reading, we expect the data to remain deleted + // and hint not to cause data to be visibile again as the hint should have expired + for (SimpleQueryResult result : selectData(cluster, keys)) + { + logger.info("Result: {}", result); + Object[][] objectArrays = result.toObjectArrays(); + if (preventReappearance) + { + logger.info("Preventing reappearance with mutation ttl time, hence expecting null as column value"); + Assert.assertNull(objectArrays[0][1]); + } + else + { + logger.info("Demonstrating reappearance possible, hence observing non null, non empty column"); + Assert.assertFalse(UUID.fromString((String) objectArrays[0][1]).toString().isEmpty()); + } + } + } + } + + private List<SimpleQueryResult> selectData(Cluster cluster, List<UUID> keys) + { + List<SimpleQueryResult> results = new ArrayList<>(); + for (UUID partitionKey : keys) + { + SimpleQueryResult result = cluster.coordinator(1) + .executeWithResult(withKeyspace("SELECT * FROM %s.cf where k=?;"), + ALL, partitionKey.toString()); + results.add(result); + } + return results; + } + + private void insertData(Cluster cluster, List<UUID> inserts) + { + for (int i = 0; i < inserts.size(); i++) + { + final UUID partitionKey = inserts.get(i); + cluster.coordinator(1) + .execute(withKeyspace("INSERT INTO %s.cf (k, c1) VALUES (?, ?);"), + LOCAL_QUORUM, partitionKey.toString(), UUID.randomUUID().toString()); + } + } + + void deleteData(Cluster cluster, List<UUID> insertedKeys) + { + + for (UUID partitionKey : insertedKeys) + { + cluster.coordinator(1) + .execute(withKeyspace("DELETE c1 FROM %s.cf where k=?;"), + LOCAL_QUORUM, partitionKey.toString()); + } + } +} + From 751d3f24cf01ffb841ba90c5ddae06900cdadbfa Mon Sep 17 00:00:00 2001 From: Marcus Eriksson <marcuse@apache.org> Date: Thu, 16 Jan 2025 10:45:45 +0100 Subject: [PATCH 093/225] Make sure we can set parameters when configuring CassandraCIDRAuthorizer patch by Marcus Eriksson; reviewed by Stefan Miklosovic for CASSANDRA-20220 --- CHANGES.txt | 1 + .../auth/CassandraCIDRAuthorizer.java | 15 ++++- .../cassandra/config/DatabaseDescriptor.java | 14 ----- .../test/auth/CIDRAuthorizerConfigTest.java | 58 +++++++++++++++++++ .../apache/cassandra/auth/AuthTestUtils.java | 13 +---- 5 files changed, 74 insertions(+), 27 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/test/auth/CIDRAuthorizerConfigTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 7666f06070da..4e36bfd35f96 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.3 + * Make sure we can set parameters when configuring CassandraCIDRAuthorizer (CASSANDRA-20220) * Add selected SAI index state and query performance metrics to nodetool tablestats (CASSANDRA-20026) * Remove v30 and v3X from 5.x in-JVM upgrade tests (CASSANDRA-20103) * Avoid memory allocation in offheap_object's NativeCell.valueSize() and NativeClustering.dataSize() (CASSANDRA-20162) diff --git a/src/java/org/apache/cassandra/auth/CassandraCIDRAuthorizer.java b/src/java/org/apache/cassandra/auth/CassandraCIDRAuthorizer.java index ffbfdcadbaff..518cf9b22473 100644 --- a/src/java/org/apache/cassandra/auth/CassandraCIDRAuthorizer.java +++ b/src/java/org/apache/cassandra/auth/CassandraCIDRAuthorizer.java @@ -19,6 +19,7 @@ package org.apache.cassandra.auth; import java.net.InetAddress; +import java.util.Locale; import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -28,7 +29,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.utils.MonotonicClock; import org.apache.cassandra.utils.NoSpamLogger; @@ -39,11 +39,20 @@ */ public class CassandraCIDRAuthorizer extends AbstractCIDRAuthorizer { + static final String CIDR_AUTHORIZER_MODE_PARAM = "cidr_authorizer_mode"; + private static final Logger logger = LoggerFactory.getLogger(AuthenticatedUser.class); private static final NoSpamLogger noSpamLogger = NoSpamLogger.getLogger(logger, 1, TimeUnit.MINUTES); - protected static CIDRPermissionsCache cidrPermissionsCache; protected static CIDRGroupsMappingCache cidrGroupsMappingCache; + private final CIDRAuthorizerMode cidrAuthorizerMode; + + public CassandraCIDRAuthorizer(Map<String, String> params) + { + cidrAuthorizerMode = (params != null && params.containsKey(CIDR_AUTHORIZER_MODE_PARAM)) + ? CIDRAuthorizerMode.valueOf(params.get(CIDR_AUTHORIZER_MODE_PARAM).toUpperCase(Locale.US)) + : CIDRAuthorizerMode.MONITOR; + } @Override public void setup() @@ -105,7 +114,7 @@ public Set<String> lookupCidrGroupsForIp(InetAddress ip) @VisibleForTesting protected boolean isMonitorMode() { - return DatabaseDescriptor.getCidrAuthorizerMode() == CIDRAuthorizerMode.MONITOR; + return cidrAuthorizerMode == CIDRAuthorizerMode.MONITOR; } private boolean hasCidrAccess(RoleResource role, InetAddress ipAddress) diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index d282cf3e3df3..4a11285e4132 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -1731,20 +1731,6 @@ public static boolean getCidrChecksForSuperusers() return Boolean.parseBoolean(value); } - public static ICIDRAuthorizer.CIDRAuthorizerMode getCidrAuthorizerMode() - { - ICIDRAuthorizer.CIDRAuthorizerMode defaultCidrAuthorizerMode = ICIDRAuthorizer.CIDRAuthorizerMode.MONITOR; - - if (conf.cidr_authorizer == null || conf.cidr_authorizer.parameters == null) - return defaultCidrAuthorizerMode; - - String cidrAuthorizerMode = conf.cidr_authorizer.parameters.get("cidr_authorizer_mode"); - if (cidrAuthorizerMode == null || cidrAuthorizerMode.isEmpty()) - return defaultCidrAuthorizerMode; - - return ICIDRAuthorizer.CIDRAuthorizerMode.valueOf(cidrAuthorizerMode.toUpperCase()); - } - public static int getCidrGroupsCacheRefreshInterval() { int defaultCidrGroupsCacheRefreshInterval = 5; // mins diff --git a/test/distributed/org/apache/cassandra/distributed/test/auth/CIDRAuthorizerConfigTest.java b/test/distributed/org/apache/cassandra/distributed/test/auth/CIDRAuthorizerConfigTest.java new file mode 100644 index 000000000000..084a85542d6c --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/auth/CIDRAuthorizerConfigTest.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.auth; + +import java.io.IOException; + +import com.google.common.collect.ImmutableMap; +import org.junit.Test; + +import org.apache.cassandra.config.ParameterizedClass; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.test.TestBaseImpl; + +public class CIDRAuthorizerConfigTest extends TestBaseImpl +{ + @Test + public void testParameterizedClass() throws IOException + { + try (Cluster cluster = init(builder().withNodes(1) + .withConfig(c -> c.set("cidr_authorizer", new ParameterizedClass("CassandraCIDRAuthorizer", + ImmutableMap.of("cidr_authorizer_mode", "ENFORCE"))) + .set("authorizer.class_name", "CassandraAuthorizer") + .set("authenticator.class_name", "PasswordAuthenticator")) + .start())) + { + // just makes sure we can start with a param in the ParameterizedClass + } + } + + @Test + public void testParameterizedClass_no_params() throws IOException + { + try (Cluster cluster = init(builder().withNodes(1) + .withConfig(c -> c.set("cidr_authorizer.class_name","CassandraCIDRAuthorizer") + .set("authorizer.class_name", "CassandraAuthorizer") + .set("authenticator.class_name", "PasswordAuthenticator")) + .start())) + { + // just makes sure we can start without a param in the ParameterizedClass + } + } +} diff --git a/test/unit/org/apache/cassandra/auth/AuthTestUtils.java b/test/unit/org/apache/cassandra/auth/AuthTestUtils.java index 610832ffb466..16c5df8118a3 100644 --- a/test/unit/org/apache/cassandra/auth/AuthTestUtils.java +++ b/test/unit/org/apache/cassandra/auth/AuthTestUtils.java @@ -33,6 +33,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.TimeoutException; +import com.google.common.collect.ImmutableMap; import org.apache.cassandra.auth.jmx.AuthorizationProxy; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.CIDR; @@ -183,16 +184,14 @@ public UntypedResultSet process(String query, ConsistencyLevel cl) public static class LocalCassandraCIDRAuthorizer extends CassandraCIDRAuthorizer { - CIDRAuthorizerMode cidrAuthorizerMode; - public LocalCassandraCIDRAuthorizer() { - cidrAuthorizerMode = CIDRAuthorizerMode.ENFORCE; + this(CIDRAuthorizerMode.ENFORCE); } public LocalCassandraCIDRAuthorizer(CIDRAuthorizerMode mode) { - cidrAuthorizerMode = mode; + super(ImmutableMap.of(CIDR_AUTHORIZER_MODE_PARAM, mode.name())); } @Override @@ -202,12 +201,6 @@ protected void createManagers() cidrGroupsMappingManager = new LocalCIDRGroupsMappingManager(); } - @Override - protected boolean isMonitorMode() - { - return cidrAuthorizerMode == CIDRAuthorizerMode.MONITOR; - } - CIDRPermissionsCache getCidrPermissionsCache() { return cidrPermissionsCache; From 0d39ea4917716e9996c908b3661a8db2b53ac40e Mon Sep 17 00:00:00 2001 From: Joseph Lynch <joe.e.lynch@gmail.com> Date: Thu, 21 Nov 2024 08:12:04 -0800 Subject: [PATCH 094/225] Support octet_length and length functions Previously users would either have to read the data and check the length themselves or enable UDF and register UDFs to check the length of columns. This patch adds a subset of the SQL99 (binary) string functions: "octet_length" defined on all types and "length" defined on UTF8 strings. patch by Joey Lynch; reviewed by Chris Lohfink and Jordan West for CASSANDRA-20102 --- CHANGES.txt | 1 + NEWS.txt | 4 + .../pages/developing/cql/functions.adoc | 48 +++++++++ .../cassandra/cql3/functions/LengthFcts.java | 93 ++++++++++++++++++ .../cql3/functions/NativeFunctions.java | 1 + .../cql3/functions/LengthFctsTest.java | 98 +++++++++++++++++++ 6 files changed, 245 insertions(+) create mode 100644 src/java/org/apache/cassandra/cql3/functions/LengthFcts.java create mode 100644 test/unit/org/apache/cassandra/cql3/functions/LengthFctsTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 914523113cf1..a3c113321d13 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Support octet_length and length functions (CASSANDRA-20102) * Make JsonUtils serialize Instant always with the same format (CASSANDRA-20209) * Port Harry v2 to trunk (CASSANDRA-20200) * Enable filtering of snapshots on keyspace, table and snapshot name in nodetool listsnapshots (CASSANDRA-20151) diff --git a/NEWS.txt b/NEWS.txt index a4cec79660a9..530acc0bd6de 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -109,6 +109,10 @@ New features Enabling both ways of configuring JMX will result in a node failing to start. - CEP-43 - it is possible to create a table by "copying" as `CREATE TABLE ks.tb_copy LIKE ks.tb;`. A newly created table will have no data. + - New functions `octet_length` and `length` were introduced. Previously, users would either have to read + the data and check the length themselves or enable UDF and register UDFs to check the length of columns. + CASSANDRA-20102 adds a subset of the SQL99 (binary) string functions: "octet_length" defined on all types + and "length" defined on UTF8 strings. See CASSANDRA-20102 for more information. Upgrading --------- diff --git a/doc/modules/cassandra/pages/developing/cql/functions.adoc b/doc/modules/cassandra/pages/developing/cql/functions.adoc index 75786de271a3..2344cdb7288c 100644 --- a/doc/modules/cassandra/pages/developing/cql/functions.adoc +++ b/doc/modules/cassandra/pages/developing/cql/functions.adoc @@ -231,6 +231,54 @@ For every xref:cassandra:developing/cql/types.adoc#native-types[type] supported Conversely, the function `blob_as_type` takes a 64-bit `blob` argument and converts it to a `bigint` value. For example, `bigint_as_blob(3)` returns `0x0000000000000003` and `blob_as_bigint(0x0000000000000003)` returns `3`. +==== Length Functions +CQL supports two functions to retrieve the length of data without returning the data itself. Note that while +network bandwidth is reduced and memory is freed earlier, these functions still require data to be read from disk. + +[cols=",,",options="header",] +|=== +|Function name |Input type |Description + +| `octet_length` | All xref:cassandra:developing/cql/types.adoc#native-types[types] | Returns the length of the data in bytes. + +| `length` | `text` | Returns the string's length - the count of UTF-8 code units. +|=== + +The `octet_length` function is defined for every xref:cassandra:developing/cql/types.adoc#native-types[type] +supported by CQL, and represents the number of bytes of the underlying `ByteBuffer` representation, not accounting for +metadata overhead. Equivalent to Java's +https://docs.oracle.com/en/java/javase/22/docs/api/java.base/java/nio/Buffer.html#remaining()[`ByteBuffer#remaining()`] +when data has just been read, which is also the `length` of the type when encoded to `byte[]`. Some examples +for common types: + +[cols=",,",options="header",] +|=== +|CQL Type | `octet_length` |Description + +| `tinyint` | `1` | 8-bit signed integer + +| `smallint` | `2` | 16-bit signed integer + +| `int` | `4` | 32-bit signed integer + +| `bigint` | `8` | 64-bit signed integer + +| `float` | `4` | 32-bit IEEE-754 floating point + +| `double` | `8` | 64-bit IEEE-754 floating point + +| `blob` | Number of bytes in blob | Variable length byte string + +| `text` | Number of bytes in `text_as_blob` representation | Variable length character string +|=== + +The `length` function is only defined for UTF-8 strings (`text`) and returns the length of that string, meaning the +number of UTF-8 code units. Equivalent to Java's +https://docs.oracle.com/en/java/javase/22/docs/api/java.base/java/lang/String.html#length()[`String#length()`] or +Python 3's `len` function when applied to a string. + +These length functions return `null` when the input is `null`. + ==== Math Functions Cql provides the following math functions: `abs`, `exp`, `log`, `log10`, and `round`. diff --git a/src/java/org/apache/cassandra/cql3/functions/LengthFcts.java b/src/java/org/apache/cassandra/cql3/functions/LengthFcts.java new file mode 100644 index 000000000000..64cb94060605 --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/functions/LengthFcts.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.functions; + +import java.nio.ByteBuffer; +import java.util.HashSet; +import java.util.Set; + +import org.apache.cassandra.cql3.CQL3Type; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.transport.ProtocolVersion; +import org.apache.cassandra.utils.ByteBufferUtil; + +/** + * Lengths of stored data without returning the data. + */ +public class LengthFcts +{ + public static void addFunctionsTo(NativeFunctions functions) + { + // As all types ultimately end up as bytebuffers they should all be compatible with + // octet_length + Set<AbstractType<?>> types = new HashSet<>(); + for (CQL3Type type : CQL3Type.Native.values()) + { + AbstractType<?> udfType = type.getType().udfType(); + if (!types.add(udfType)) + continue; + functions.add(makeOctetLengthFunction(type.getType().udfType())); + } + + // Special handling for string length which is number of UTF-8 code units + functions.add(length); + } + + + public static NativeFunction makeOctetLengthFunction(AbstractType<?> fromType) + { + // Matches SQL99 OCTET_LENGTH functions defined on bytestring and char strings + return new NativeScalarFunction("octet_length", Int32Type.instance, fromType) + { + // Do not deserialize + @Override + public Arguments newArguments(ProtocolVersion version) + { + return FunctionArguments.newNoopInstance(version, 1); + } + + @Override + public ByteBuffer execute(Arguments arguments) throws InvalidRequestException + { + if (arguments.get(0) == null) + return null; + + final ByteBuffer buffer = arguments.get(0); + return ByteBufferUtil.bytes(buffer.remaining()); + } + }; + } + + // Matches PostgreSQL length function defined as returning the number of UTF-8 code units in the text string + public static final NativeFunction length = new NativeScalarFunction("length", Int32Type.instance, UTF8Type.instance) + { + @Override + public ByteBuffer execute(Arguments arguments) throws InvalidRequestException + { + if (arguments.get(0) == null) + return null; + + final String value = arguments.get(0); + return ByteBufferUtil.bytes(value.length()); + } + }; +} diff --git a/src/java/org/apache/cassandra/cql3/functions/NativeFunctions.java b/src/java/org/apache/cassandra/cql3/functions/NativeFunctions.java index a9deba5cb180..e521c5e807ff 100644 --- a/src/java/org/apache/cassandra/cql3/functions/NativeFunctions.java +++ b/src/java/org/apache/cassandra/cql3/functions/NativeFunctions.java @@ -44,6 +44,7 @@ public class NativeFunctions AggregateFcts.addFunctionsTo(this); CollectionFcts.addFunctionsTo(this); BytesConversionFcts.addFunctionsTo(this); + LengthFcts.addFunctionsTo(this); MathFcts.addFunctionsTo(this); MaskingFcts.addFunctionsTo(this); VectorFcts.addFunctionsTo(this); diff --git a/test/unit/org/apache/cassandra/cql3/functions/LengthFctsTest.java b/test/unit/org/apache/cassandra/cql3/functions/LengthFctsTest.java new file mode 100644 index 000000000000..b1f1354bc299 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/functions/LengthFctsTest.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.functions; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; + +import org.junit.Test; + +import org.apache.cassandra.cql3.CQLTester; +import org.quicktheories.WithQuickTheories; + +public class LengthFctsTest extends CQLTester implements WithQuickTheories +{ + @Test + public void testOctetLengthNonStrings() + { + createTable("CREATE TABLE %s (a tinyint primary key," + + " b smallint," + + " c int," + + " d bigint," + + " e float," + + " f double," + + " g decimal," + + " h varint," + + " i int)"); + + execute("INSERT INTO %s (a, b, c, d, e, f, g, h) VALUES (?, ?, ?, ?, ?, ?, ?, ?)", + (byte) 1, (short) 2, 3, 4L, 5.2F, 6.3, BigDecimal.valueOf(6.3), BigInteger.valueOf(4)); + + assertRows(execute("SELECT OCTET_LENGTH(a), " + + "OCTET_LENGTH(b), " + + "OCTET_LENGTH(c), " + + "OCTET_LENGTH(d), " + + "OCTET_LENGTH(e), " + + "OCTET_LENGTH(f), " + + "OCTET_LENGTH(g), " + + "OCTET_LENGTH(h), " + + "OCTET_LENGTH(i) FROM %s"), + row(1, 2, 4, 8, 4, 8, 5, 1, null)); + } + + @Test + public void testStringLengthUTF8() throws Throwable + { + createTable("CREATE TABLE %s (key text primary key, value blob)"); + // UTF-8 7 codepoint, 21 byte encoded string + String key = "こんにちは世界"; + execute("INSERT INTO %s (key) VALUES (?)", key); + + assertRows(execute("SELECT LENGTH(key), OCTET_LENGTH(key), OCTET_LENGTH(value) FROM %s where key = ?", key), + row(7, 21, null)); + + // Quickly check that multiple arguments leads to an exception as expected + assertInvalidMessage("Invalid number of arguments in call to function system.length", + "SELECT LENGTH(key, value) FROM %s where key = 'こんにちは世界'"); + assertInvalidMessage("Invalid call to function octet_length, none of its type signatures match", + "SELECT OCTET_LENGTH(key, value) FROM %s where key = 'こんにちは世界'"); + } + + @Test + public void testOctetLengthStringFuzz() + { + createTable("CREATE TABLE %s (key text primary key, value blob)"); + + qt().withExamples(1024).forAll(strings().allPossible().ofLengthBetween(32, 100)).checkAssert( + (randString) -> { + int sLen = randString.length(); + byte[] randBytes = randString.getBytes(StandardCharsets.UTF_8); + + // UTF-8 length (code unit count) and byte length are often + // different. Spot checked a few of these, and they are different + // most of the time in this test - but testing that reproducibly + // requires seeding that would decrease the test power... + execute("INSERT INTO %s (key, value) VALUES (?, ?)", randString, ByteBuffer.wrap(randBytes)); + assertRows(execute("SELECT LENGTH(key), OCTET_LENGTH(key), OCTET_LENGTH(value) FROM %s where key = ?", randString), + row(sLen, randBytes.length, randBytes.length)); + }); + } +} From b4bcdfa7855109de49e5bac2f538eb773c44377f Mon Sep 17 00:00:00 2001 From: Maxwell Guo <cclive1601@gmail.com> Date: Thu, 16 Jan 2025 20:46:37 +0800 Subject: [PATCH 095/225] Fail CREATE TABLE LIKE statement if UDTs in target keyspace do not exist or they have different structure from ones in source keyspace patch by Maxwell Guo; reviewed by Stefan Miklosovic, Benjamin Lerer for CASSANDRA-19966 --- CHANGES.txt | 1 + .../statements/schema/CopyTableStatement.java | 44 ++++- .../apache/cassandra/db/marshal/UserType.java | 8 + .../schema/createlike/CreateLikeTest.java | 163 +++++++++++------- 4 files changed, 153 insertions(+), 63 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index a3c113321d13..b9ae2539227a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Fail CREATE TABLE LIKE statement if UDTs in target keyspace do not exist or they have different structure from ones in source keyspace (CASSANDRA-19966) * Support octet_length and length functions (CASSANDRA-20102) * Make JsonUtils serialize Instant always with the same format (CASSANDRA-20209) * Port Harry v2 to trunk (CASSANDRA-20200) diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CopyTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CopyTableStatement.java index 0df8d00f45e8..e8ee7e2cd308 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/CopyTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/CopyTableStatement.java @@ -18,12 +18,21 @@ package org.apache.cassandra.cql3.statements.schema; +import java.nio.ByteBuffer; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import com.google.common.collect.Sets; + import org.apache.cassandra.audit.AuditLogContext; import org.apache.cassandra.audit.AuditLogEntryType; import org.apache.cassandra.auth.Permission; import org.apache.cassandra.cql3.CQLStatement; import org.apache.cassandra.cql3.QualifiedName; import org.apache.cassandra.db.guardrails.Guardrails; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.marshal.UserType; import org.apache.cassandra.db.marshal.VectorType; import org.apache.cassandra.exceptions.AlreadyExistsException; import org.apache.cassandra.schema.Indexes; @@ -114,14 +123,39 @@ public Keyspaces apply(ClusterMetadata metadata) if (targetKeyspaceMeta.hasTable(targetTableName)) { - if(ifNotExists) + if (ifNotExists) return schema; throw new AlreadyExistsException(targetKeyspace, targetTableName); } - // todo support udt for differenet ks latter - if (!sourceKeyspace.equalsIgnoreCase(targetKeyspace) && !sourceTableMeta.getReferencedUserTypes().isEmpty()) - throw ire("Cannot use CREATE TABLE LIKE across different keyspace when source table have UDTs."); + + if (!sourceKeyspace.equalsIgnoreCase(targetKeyspace)) + { + Set<String> missingUserTypes = Sets.newHashSet(); + // for different keyspace, if source table used some udts and the target table also need them + for (ByteBuffer sourceUserTypeName : sourceTableMeta.getReferencedUserTypes()) + { + Optional<UserType> targetUserType = targetKeyspaceMeta.types.get(sourceUserTypeName); + Optional<UserType> sourceUserType = sourceKeyspaceMeta.types.get(sourceUserTypeName); + if (targetUserType.isPresent() && sourceUserType.isPresent()) + { + if (!sourceUserType.get().equalsWithOutKs(targetUserType.get())) + throw ire("Target keyspace '%s' has same UDT name '%s' as source keyspace '%s' but with different structure.", + targetKeyspace, + UTF8Type.instance.getString(targetUserType.get().name), + sourceKeyspace); + } + else + { + missingUserTypes.add(UTF8Type.instance.compose(sourceUserTypeName)); + } + } + + if (!missingUserTypes.isEmpty()) + throw ire("UDTs %s do not exist in target keyspace '%s'.", + missingUserTypes.stream().sorted().collect(Collectors.joining(", ")), + targetKeyspace); + } // Guardrail on columns per table Guardrails.columnsPerTable.guard(sourceTableMeta.columns().size(), targetTableName, false, state); @@ -130,7 +164,7 @@ public Keyspaces apply(ClusterMetadata metadata) if (columnMetadata.type.isVector()) { Guardrails.vectorTypeEnabled.ensureEnabled(columnMetadata.name.toString(), state); - int dimensions = ((VectorType)columnMetadata.type).dimension; + int dimensions = ((VectorType) columnMetadata.type).dimension; Guardrails.vectorDimensions.guard(dimensions, columnMetadata.name.toString(), false, state); } }); diff --git a/src/java/org/apache/cassandra/db/marshal/UserType.java b/src/java/org/apache/cassandra/db/marshal/UserType.java index bbd16e36d199..82e9aafcc67a 100644 --- a/src/java/org/apache/cassandra/db/marshal/UserType.java +++ b/src/java/org/apache/cassandra/db/marshal/UserType.java @@ -370,6 +370,14 @@ private boolean equalsWithoutTypes(UserType other) && isMultiCell == other.isMultiCell; } + public boolean equalsWithOutKs(UserType other) + { + return name.equals(other.name) + && fieldNames.equals(other.fieldNames) + && types.equals(other.types) + && isMultiCell == other.isMultiCell; + } + public Optional<Difference> compare(UserType other) { if (!equalsWithoutTypes(other)) diff --git a/test/unit/org/apache/cassandra/schema/createlike/CreateLikeTest.java b/test/unit/org/apache/cassandra/schema/createlike/CreateLikeTest.java index 02406acdfe07..1fe72121a6bd 100644 --- a/test/unit/org/apache/cassandra/schema/createlike/CreateLikeTest.java +++ b/test/unit/org/apache/cassandra/schema/createlike/CreateLikeTest.java @@ -26,7 +26,9 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.stream.Collectors; +import com.google.common.collect.Sets; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -105,7 +107,7 @@ public void testTableSchemaCopy() { String sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b duration, c text);"); String targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a, b, c) VALUES (?, ?, ?)", 1, duration1, "1"); execute("INSERT INTO " + targetKs + "." + targetTb + " (a, b, c) VALUES (?, ?, ?)", 2, duration2, "2"); assertRows(execute("SELECT * FROM " + sourceKs + "." + sourceTb), @@ -115,7 +117,7 @@ public void testTableSchemaCopy() sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY);"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a) VALUES (1)"); execute("INSERT INTO " + targetKs + "." + targetTb + " (a) VALUES (2)"); assertRows(execute("SELECT * FROM " + sourceKs + "." + sourceTb), @@ -125,7 +127,7 @@ public void testTableSchemaCopy() sourceTb = createTable(sourceKs, "CREATE TABLE %s (a frozen<map<text, text>> PRIMARY KEY);"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a) VALUES (?)", map("k", "v")); execute("INSERT INTO " + targetKs + "." + targetTb + " (a) VALUES (?)", map("nk", "nv")); assertRows(execute("SELECT * FROM " + sourceKs + "." + sourceTb), @@ -135,7 +137,7 @@ public void testTableSchemaCopy() sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b set<frozen<list<text>>>, c map<text, int>, d smallint, e duration, f tinyint);"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 1, set(list("1", "2"), list("3", "4")), map("k", 1), (short)2, duration1, (byte)4); execute("INSERT INTO " + targetKs + "." + targetTb + " (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", @@ -147,7 +149,7 @@ public void testTableSchemaCopy() sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int , b double, c tinyint, d float, e list<text>, f map<text, int>, g duration, PRIMARY KEY((a, b, c), d));"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a, b, c, d, e, f, g) VALUES (?, ?, ?, ?, ?, ?, ?) ", 1, d1, (byte)4, f1, list("a", "b"), map("k", 1), duration1); execute("INSERT INTO " + targetKs + "." + targetTb + " (a, b, c, d, e, f, g) VALUES (?, ?, ?, ?, ?, ?, ?) ", @@ -170,7 +172,7 @@ public void testTableSchemaCopy() "PRIMARY KEY((a, b), c, d)) " + "WITH CLUSTERING ORDER BY (c DESC, d ASC);"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a, b, c, d, e, f, g, h, i, j) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?)", 1, "b", 100L, decimal1, set("1", "2"), uuid1, vector1, list(1.1F, 2.2F), timeUuid1, map("k", set(1, 2))); execute("INSERT INTO " + targetKs + "." + targetTb + " (a, b, c, d, e, f, g, h, i, j) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?)", @@ -191,7 +193,7 @@ public void testIfNotExists() throws Throwable { String sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int, b text, c duration, d float, PRIMARY KEY(a, b));"); String targetTb = createTableLike("CREATE TABLE IF NOT EXISTS %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); createTableLike("CREATE TABLE IF NOT EXISTS %s LIKE %s", sourceTb, sourceKs, targetTb, targetKs); assertInvalidThrowMessage("Cannot add already existing table \"" + targetTb + "\" to keyspace \"" + targetKs + "\"", AlreadyExistsException.class, @@ -203,19 +205,19 @@ public void testCopyAfterAlterTable() { String sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int, b text, c duration, d float, PRIMARY KEY(a, b));"); String targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " DROP d"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " ADD e uuid"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " ADD f float"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a, b, c, e, f) VALUES (?, ?, ?, ?, ?)", 1, "1", duration1, uuid1, f1); execute("INSERT INTO " + targetKs + "." + targetTb + " (a, b, c, e, f) VALUES (?, ?, ?, ?, ?)", 2, "2", duration2, uuid2, f2); @@ -226,15 +228,15 @@ public void testCopyAfterAlterTable() alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " DROP f USING TIMESTAMP 20000"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " RENAME b TO bb "); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " WITH compaction = {'class':'LeveledCompactionStrategy', 'sstable_size_in_mb':10, 'fanout_size':16} "); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a, bb, c, e) VALUES (?, ?, ?, ?)", 1, "1", duration1, uuid1); execute("INSERT INTO " + targetKs + "." + targetTb + " (a, bb, c, e) VALUES (?, ?, ?, ?)", 2, "2", duration2, uuid2); @@ -313,21 +315,21 @@ public void testTableOptionsCopy() throws Throwable String tbLikeCompactionUcs = createTableLike("CREATE TABLE %s LIKE %s", tableCompactionUcs, sourceKs, targetKs); String tbLikeCompactionOthers= createTableLike("CREATE TABLE %s LIKE %s", tableOtherOptions, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, tbCompressionDefault1, tbLikeCompressionDefault1); - assertTableMetaEquals(sourceKs, targetKs, tbCompressionDefault2, tbLikeCompressionDefault2); - assertTableMetaEquals(sourceKs, targetKs, tbCompressionSnappy1, tbLikeCompressionSp1); - assertTableMetaEquals(sourceKs, targetKs, tbCompressionSnappy2, tbLikeCompressionSp2); - assertTableMetaEquals(sourceKs, targetKs, tbCompressionSnappy3, tbLikeCompressionSp3); - assertTableMetaEquals(sourceKs, targetKs, tbCompressionSnappy4, tbLikeCompressionSp4); - assertTableMetaEquals(sourceKs, targetKs, tbCompressionSnappy5, tbLikeCompressionSp5); - assertTableMetaEquals(sourceKs, targetKs, tableMemtableSkipList, tbLikeMemtableSkipList); - assertTableMetaEquals(sourceKs, targetKs, tableMemtableTrie, tbLikeMemtableTrie); - assertTableMetaEquals(sourceKs, targetKs, tableMemtableDefault, tbLikeMemtableDefault); - assertTableMetaEquals(sourceKs, targetKs, tableCompactionStcs, tbLikeCompactionStcs); - assertTableMetaEquals(sourceKs, targetKs, tableCompactionLcs, tbLikeCompactionLcs); - assertTableMetaEquals(sourceKs, targetKs, tableCompactionTwcs, tbLikeCompactionTwcs); - assertTableMetaEquals(sourceKs, targetKs, tableCompactionUcs, tbLikeCompactionUcs); - assertTableMetaEquals(sourceKs, targetKs, tableOtherOptions, tbLikeCompactionOthers); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tbCompressionDefault1, tbLikeCompressionDefault1); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tbCompressionDefault2, tbLikeCompressionDefault2); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tbCompressionSnappy1, tbLikeCompressionSp1); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tbCompressionSnappy2, tbLikeCompressionSp2); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tbCompressionSnappy3, tbLikeCompressionSp3); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tbCompressionSnappy4, tbLikeCompressionSp4); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tbCompressionSnappy5, tbLikeCompressionSp5); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tableMemtableSkipList, tbLikeMemtableSkipList); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tableMemtableTrie, tbLikeMemtableTrie); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tableMemtableDefault, tbLikeMemtableDefault); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tableCompactionStcs, tbLikeCompactionStcs); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tableCompactionLcs, tbLikeCompactionLcs); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tableCompactionTwcs, tbLikeCompactionTwcs); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tableCompactionUcs, tbLikeCompactionUcs); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tableOtherOptions, tbLikeCompactionOthers); // a copy of the table with the table parameters set String tableCopyAndSetCompression = createTableLike("CREATE TABLE %s LIKE %s WITH compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 64 };", @@ -351,9 +353,9 @@ public void testTableOptionsCopy() throws Throwable " AND read_repair = 'NONE'" + " AND memtable_flush_period_in_ms = 3600;", tableOtherOptions, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, tbCompressionDefault1, tableCopyAndSetCompression, false, false, false); - assertTableMetaEquals(sourceKs, targetKs, tableCompactionLcs, tableCopyAndSetLCSCompaction, false, false, false); - assertTableMetaEquals(sourceKs, targetKs, tableOtherOptions, tableCopyAndSetAllParams, false, false, false); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tbCompressionDefault1, tableCopyAndSetCompression, false, false, false); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tableCompactionLcs, tableCopyAndSetLCSCompaction, false, false, false); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tableOtherOptions, tableCopyAndSetAllParams, false, false, false); TableParams paramsSetCompression = getTableMetadata(targetKs, tableCopyAndSetCompression).params; TableParams paramsSetLCSCompaction = getTableMetadata(targetKs, tableCopyAndSetLCSCompaction).params; TableParams paramsSetAllParams = getTableMetadata(targetKs, tableCopyAndSetAllParams).params; @@ -393,7 +395,7 @@ public void testStaticColumnCopy() // create with static column String sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int , b int , c int static, d int, e list<text>, PRIMARY KEY(a, b));", "tb1"); String targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); execute("INSERT INTO " + targetKs + "." + targetTb + " (a, b, c, d, e) VALUES (0, 1, 2, 3, ?)", list("1", "2", "3", "4")); assertRows(execute("SELECT * FROM " + targetKs + "." + targetTb), row(0, 1, 2, 3, list("1", "2", "3", "4"))); @@ -401,7 +403,7 @@ public void testStaticColumnCopy() sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b))"); alterTable("ALTER TABLE " + sourceKs + "." + sourceTb + " ADD d int static"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); } @Test @@ -411,38 +413,38 @@ public void testColumnMaskTableCopy() // masked partition key String sourceTb = createTable(sourceKs, "CREATE TABLE %s (k int MASKED WITH mask_default() PRIMARY KEY, r int)"); String targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); // masked partition key component sourceTb = createTable(sourceKs, "CREATE TABLE %s (k1 int, k2 text MASKED WITH DEFAULT, r int, PRIMARY KEY(k1, k2))"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); // masked clustering key sourceTb = createTable(sourceKs, "CREATE TABLE %s (k int, c int MASKED WITH mask_default(), r int, PRIMARY KEY (k, c))"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); // masked clustering key with reverse order sourceTb = createTable(sourceKs, "CREATE TABLE %s (k int, c text MASKED WITH mask_default(), r int, PRIMARY KEY (k, c)) " + "WITH CLUSTERING ORDER BY (c DESC)"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); // masked clustering key component sourceTb = createTable(sourceKs, "CREATE TABLE %s (k int, c1 int, c2 text MASKED WITH DEFAULT, r int, PRIMARY KEY (k, c1, c2))"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); // masked regular column sourceTb = createTable(sourceKs, "CREATE TABLE %s (k int PRIMARY KEY, r1 text MASKED WITH DEFAULT, r2 int)"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); // masked static column sourceTb = createTable(sourceKs, "CREATE TABLE %s (k int, c int, r int, s int STATIC MASKED WITH DEFAULT, PRIMARY KEY (k, c))"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); // multiple masked columns sourceTb = createTable(sourceKs, "CREATE TABLE %s (" + @@ -452,7 +454,7 @@ public void testColumnMaskTableCopy() "s1 int static, s2 int static MASKED WITH DEFAULT, " + "PRIMARY KEY((k1, k2), c1, c2))"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); sourceTb = createTable(sourceKs, "CREATE TABLE %s (k int PRIMARY KEY, " + "s set<int> MASKED WITH DEFAULT, " + @@ -462,40 +464,85 @@ public void testColumnMaskTableCopy() "fl frozen<list<int>> MASKED WITH DEFAULT, " + "fm frozen<map<int, int>> MASKED WITH DEFAULT)"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); } @Test public void testUDTTableCopy() throws Throwable { - // normal udt + //normal udt String udt = createType(sourceKs, "CREATE TYPE %s (a int, b uuid, c text)"); - // collection udt + String udtNew = createType(sourceKs, "CREATE TYPE %s (a int, b text)"); + //collection udt String udtSet = createType(sourceKs, "CREATE TYPE %s (a int, c frozen <set<text>>)"); - // frozen udt + //frozen udt String udtFrozen = createType(sourceKs, "CREATE TYPE %s (a int, c frozen<" + udt + ">)"); + String udtFrozenNotExist = createType(sourceKs, "CREATE TYPE %s (a int, c frozen<" + udtNew + ">)"); + // source table's column's data type is udt, and its subtypes are all native type String sourceTbUdt = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b duration, c " + udt + ");"); + // source table's column's data type is udt, and its subtypes are native type and collection type String sourceTbUdtSet = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b duration, c " + udtSet + ");"); + // source table's column's data type is udt, and its subtypes are native type and udt String sourceTbUdtFrozen = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b duration, c " + udtFrozen + ");"); + // source table's column's data type is udt, and its subtypes are native type and more than one udt + String sourceTbUdtComb = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b duration, c " + udtFrozen + ", d " + udt+ ");"); + // source table's column's data type is udt, and its subtypes are native type and more than one udt + String sourceTbUdtCombNotExist = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b duration, c " + udtFrozen + ", d " + udtFrozenNotExist+ ");"); if (differentKs) { - assertInvalidThrowMessage("Cannot use CREATE TABLE LIKE across different keyspace when source table have UDT", - InvalidRequestException.class, "CREATE TABLE " + targetKs + ".tbudt LIKE " + sourceKs + "." + sourceTbUdt); - assertInvalidThrowMessage("Cannot use CREATE TABLE LIKE across different keyspace when source table have UDT", - InvalidRequestException.class, "CREATE TABLE " + targetKs + ".tbdtset LIKE " + sourceKs + "." + sourceTbUdt); - assertInvalidThrowMessage("Cannot use CREATE TABLE LIKE across different keyspace when source table have UDT", InvalidRequestException.class, - "CREATE TABLE " + targetKs + ".tbudtfrozen LIKE " + sourceKs + "." + sourceTbUdt); + assertInvalidThrowMessage("UDTs " + udt + " do not exist in target keyspace '" + targetKs +"'.", + InvalidRequestException.class, + "CREATE TABLE " + targetKs + ".tbudt LIKE " + sourceKs + "." + sourceTbUdt); + assertInvalidThrowMessage("UDTs " + udtSet + " do not exist in target keyspace '" + targetKs +"'.", + InvalidRequestException.class, + "CREATE TABLE " + targetKs + ".tbdtset LIKE " + sourceKs + "." + sourceTbUdtSet); + assertInvalidThrowMessage(String.format("UDTs %s do not exist in target keyspace '%s'.", Sets.newHashSet(udt, udtFrozen).stream().sorted().collect(Collectors.joining(", ")), targetKs), + InvalidRequestException.class, + "CREATE TABLE " + targetKs + ".tbudtfrozen LIKE " + sourceKs + "." + sourceTbUdtFrozen); + assertInvalidThrowMessage(String.format("UDTs %s do not exist in target keyspace '%s'.", Sets.newHashSet(udt, udtFrozen).stream().sorted().collect(Collectors.joining(", ")), targetKs), + InvalidRequestException.class, + "CREATE TABLE " + targetKs + ".tbudtfrozen LIKE " + sourceKs + "." + sourceTbUdtFrozen); + assertInvalidThrowMessage(String.format("UDTs %s do not exist in target keyspace '%s'.", Sets.newHashSet(udt, udtFrozen).stream().sorted().collect(Collectors.joining(", ")), targetKs), + InvalidRequestException.class, + "CREATE TABLE " + targetKs + ".tbudtcomb LIKE " + sourceKs + "." + sourceTbUdtComb); + assertInvalidThrowMessage(String.format("UDTs %s do not exist in target keyspace '%s'.", Sets.newHashSet(udtNew, udt,udtFrozenNotExist, udtFrozen).stream().sorted().collect(Collectors.joining(", ")), targetKs), + InvalidRequestException.class, + "CREATE TABLE " + targetKs + ".tbudtcomb LIKE " + sourceKs + "." + sourceTbUdtCombNotExist); + // different keyspaces with udts that have same udt name, different fields + String udtWithDifferentField = createType(sourceKs, "CREATE TYPE %s (aa int, bb text)"); + createType("CREATE TYPE IF NOT EXISTS " + targetKs + "." + udtWithDifferentField + " (aa int, cc text)"); + String sourceTbDiffUdt = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b duration, c " + udtWithDifferentField + ");"); + assertInvalidThrowMessage("Target keyspace '" + targetKs + "' has same UDT name '"+ udtWithDifferentField +"' as source keyspace '" + sourceKs + "' but with different structure.", + InvalidRequestException.class, + "CREATE TABLE " + targetKs + ".tbdiffudt LIKE " + sourceKs + "." + sourceTbDiffUdt); } else { + // copy table that have udt, and udt's subtype are all native type, target table will create this udt String targetTbUdt = createTableLike("CREATE TABLE %s LIKE %s", sourceTbUdt, sourceKs, "tbudt", targetKs); + // copy table that have udt, and udt's subtype are all native type and collection typ, target table will create this udt String targetTbUdtSet = createTableLike("CREATE TABLE %s LIKE %s", sourceTbUdtSet, sourceKs, "tbdtset", targetKs); + // copy table that have udt, and udt's subtype are all native type and udt, target table will create udt in order String targetTbUdtFrozen = createTableLike("CREATE TABLE %s LIKE %s", sourceTbUdtFrozen, sourceKs, "tbudtfrozen", targetKs); - assertTableMetaEquals(sourceKs, targetKs, sourceTbUdt, targetTbUdt); - assertTableMetaEquals(sourceKs, targetKs, sourceTbUdtSet, targetTbUdtSet); - assertTableMetaEquals(sourceKs, targetKs, sourceTbUdtFrozen, targetTbUdtFrozen); + // copy table that have udt, and udt's subtype are all native type and more than one udt, target table will create udt in order + String targetTbUdtComb = createTableLike("CREATE TABLE %s LIKE %s", sourceTbUdtComb, sourceKs, "tbudtcomb", targetKs); + // copy table that have udt, and udt's subtype are all native type and udt, target table will create udt in order + String targetTbUdtCombNotExist = createTableLike("CREATE TABLE %s LIKE %s", sourceTbUdtCombNotExist, sourceKs, "tbudtcombnotexist", targetKs); + + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTbUdt, targetTbUdt); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTbUdtSet, targetTbUdtSet); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTbUdtFrozen, targetTbUdtFrozen); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTbUdtComb, targetTbUdtComb); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTbUdtCombNotExist, targetTbUdtCombNotExist); + + // same udt already exist in target ks, the existed udt will be used + String udtWithSameField = createType(sourceKs, "CREATE TYPE %s (a int, b text)"); + createType("CREATE TYPE IF NOT EXISTS " + targetKs + "." + udtWithSameField + " (a int, b text)"); + String sourceTbSameUdt = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b duration, c " + udtWithSameField + ");"); + String targetTbSameUdt = createTableLike("CREATE TABLE %s LIKE %s", sourceTbSameUdt, sourceKs, "tbsameudt", targetKs); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTbSameUdt, targetTbSameUdt); } } @@ -539,12 +586,12 @@ public void testUnSupportedSchema() throws Throwable "CREATE TABLE system_views.newtb LIKE system_views.snapshots ;"); } - private void assertTableMetaEquals(String sourceKs, String targetKs, String sourceTb, String targetTb) + private void assertTableMetaEqualsWithoutKs(String sourceKs, String targetKs, String sourceTb, String targetTb) { - assertTableMetaEquals(sourceKs, targetKs, sourceTb, targetTb, true, true, true); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb, true, true, true); } - private void assertTableMetaEquals(String sourceKs, String targetKs, String sourceTb, String targetTb, boolean compareParams, boolean compareIndexes, boolean compareTrigger) + private void assertTableMetaEqualsWithoutKs(String sourceKs, String targetKs, String sourceTb, String targetTb, boolean compareParams, boolean compareIndexes, boolean compareTrigger) { TableMetadata left = getTableMetadata(sourceKs, sourceTb); TableMetadata right = getTableMetadata(targetKs, targetTb); From 06f0965a0840dd8bfa93c532b7cf30a3efb0d2b0 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Wed, 22 Jan 2025 13:33:51 +0100 Subject: [PATCH 096/225] Fix error when trying to assign a tuple to target type not being a tuple patch by Stefan Miklosovic; reviewed by David Capwell for CASSANDRA-20237 --- CHANGES.txt | 1 + src/java/org/apache/cassandra/cql3/terms/Tuples.java | 4 ++-- .../cassandra/cql3/validation/entities/TupleTypeTest.java | 5 ++++- .../validation/operations/SelectMultiColumnRelationTest.java | 4 ++-- 4 files changed, 9 insertions(+), 5 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index b9ae2539227a..a301dc9bafe6 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Fix error when trying to assign a tuple to target type not being a tuple (CASSANDRA-20237) * Fail CREATE TABLE LIKE statement if UDTs in target keyspace do not exist or they have different structure from ones in source keyspace (CASSANDRA-19966) * Support octet_length and length functions (CASSANDRA-20102) * Make JsonUtils serialize Instant always with the same format (CASSANDRA-20209) diff --git a/src/java/org/apache/cassandra/cql3/terms/Tuples.java b/src/java/org/apache/cassandra/cql3/terms/Tuples.java index 6825a16b985d..f9ba523567bd 100644 --- a/src/java/org/apache/cassandra/cql3/terms/Tuples.java +++ b/src/java/org/apache/cassandra/cql3/terms/Tuples.java @@ -68,14 +68,14 @@ public Term prepare(String keyspace, ColumnSpecification receiver) throws Invali if (elements.size() == 1 && !checkIfTupleType(receiver.type)) return elements.get(0).prepare(keyspace, receiver); + validateTupleAssignableTo(receiver, elements); + TupleType tupleType = getTupleType(receiver.type); if (elements.size() != tupleType.size()) throw invalidRequest("Expected %d elements in value for tuple %s, but got %d: %s", tupleType.size(), receiver.name, elements.size(), this); - validateTupleAssignableTo(receiver, elements); - List<Term> values = new ArrayList<>(elements.size()); boolean allTerminal = true; for (int i = 0; i < elements.size(); i++) diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java index 452bb0bc3672..90c9778df41d 100644 --- a/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java +++ b/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java @@ -149,7 +149,7 @@ public void testInvalidQueries() throws Throwable assertInvalidSyntax("INSERT INTO %s (k, t) VALUES (0, ())"); - assertInvalidMessage("Expected 3 elements in value for tuple t, but got 4: (2, 'foo', 3.1, 'bar')", + assertInvalidMessage("Invalid tuple literal for t: too many elements. Type frozen<tuple<int, text, double>> expects 3 but got 4", "INSERT INTO %s (k, t) VALUES (0, (2, 'foo', 3.1, 'bar'))"); createTable("CREATE TABLE %s (k int PRIMARY KEY, t frozen<tuple<int, tuple<int, text, double>>>)"); @@ -159,6 +159,9 @@ public void testInvalidQueries() throws Throwable assertInvalidMessage("Invalid tuple literal for t: component 1 is not of type frozen<tuple<int, text, double>>", "INSERT INTO %s (k, t) VALUES (0, (1, (1, '1', 1.0, 1)))"); + + assertInvalidMessage("Invalid tuple type literal for k of type int", + "SELECT * FROM %s WHERE k = ('a', 'b')"); } @Test diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java index 1029196754a5..2fe3cffa7f8d 100644 --- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java +++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java @@ -58,7 +58,7 @@ public void testMultiClusteringInvalidQueries() throws Throwable createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))"); assertInvalidSyntax("SELECT * FROM %s WHERE a = 0 AND (b, c) > ()"); - assertInvalidMessage("Expected 2 elements in value for tuple (b, c), but got 3: (?, ?, ?)", + assertInvalidMessage("Invalid tuple literal for (b, c): too many elements. Type frozen<tuple<int, int>> expects 2 but got 3", "SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?, ?)", 1, 2, 3); assertInvalidMessage("Invalid null value for c in tuple (b, c)", "SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?)", 1, null); @@ -72,7 +72,7 @@ public void testMultiClusteringInvalidQueries() throws Throwable // Wrong number of values assertInvalidMessage("Expected 3 elements in value for tuple (b, c, d), but got 2: (?, ?)", "SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?))", 0, 1); - assertInvalidMessage("Expected 3 elements in value for tuple (b, c, d), but got 5: (?, ?, ?, ?, ?)", + assertInvalidMessage("Invalid tuple literal for (b, c, d): too many elements. Type frozen<tuple<int, int, int>> expects 3 but got 5", "SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?, ?, ?, ?))", 0, 1, 2, 3, 4); // Missing first clustering column From ba4a0d4fcb200983bfe505059df1f936657a5525 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Fri, 26 Apr 2024 13:40:42 +0200 Subject: [PATCH 097/225] Add format_bytes and format_time functions patch by Stefan Miklosovic; reviewed by Jordan West for CASSANDRA-19546 Co-authored-by: Cheng Wang <chengw@netflix.com> --- CHANGES.txt | 1 + NEWS.txt | 1 + .../pages/developing/cql/functions.adoc | 183 +++++++ .../cassandra/config/DataStorageSpec.java | 87 +++- .../apache/cassandra/config/DurationSpec.java | 2 +- .../cassandra/cql3/functions/FormatFcts.java | 447 ++++++++++++++++++ .../cql3/functions/NativeFunctions.java | 1 + .../cql3/functions/AbstractFormatFctTest.java | 84 ++++ .../cql3/functions/FormatBytesFctTest.java | 307 ++++++++++++ .../cql3/functions/FormatTimeFctTest.java | 268 +++++++++++ 10 files changed, 1356 insertions(+), 25 deletions(-) create mode 100644 src/java/org/apache/cassandra/cql3/functions/FormatFcts.java create mode 100644 test/unit/org/apache/cassandra/cql3/functions/AbstractFormatFctTest.java create mode 100644 test/unit/org/apache/cassandra/cql3/functions/FormatBytesFctTest.java create mode 100644 test/unit/org/apache/cassandra/cql3/functions/FormatTimeFctTest.java diff --git a/CHANGES.txt b/CHANGES.txt index a301dc9bafe6..cc1262d32b45 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Add format_bytes and format_time functions (CASSANDRA-19546) * Fix error when trying to assign a tuple to target type not being a tuple (CASSANDRA-20237) * Fail CREATE TABLE LIKE statement if UDTs in target keyspace do not exist or they have different structure from ones in source keyspace (CASSANDRA-19966) * Support octet_length and length functions (CASSANDRA-20102) diff --git a/NEWS.txt b/NEWS.txt index 530acc0bd6de..dc4c766e9f21 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -113,6 +113,7 @@ New features the data and check the length themselves or enable UDF and register UDFs to check the length of columns. CASSANDRA-20102 adds a subset of the SQL99 (binary) string functions: "octet_length" defined on all types and "length" defined on UTF8 strings. See CASSANDRA-20102 for more information. + - New functions `format_bytes` and `format_time` were added. See CASSANDRA-19546. Upgrading --------- diff --git a/doc/modules/cassandra/pages/developing/cql/functions.adoc b/doc/modules/cassandra/pages/developing/cql/functions.adoc index 2344cdb7288c..ec8cd085276b 100644 --- a/doc/modules/cassandra/pages/developing/cql/functions.adoc +++ b/doc/modules/cassandra/pages/developing/cql/functions.adoc @@ -336,6 +336,189 @@ A number of functions allow to obtain the similarity score between vectors of fl include::cassandra:partial$vector-search/vector_functions.adoc[] +[[human-helper-functions]] +==== Human helper functions + +For user's convenience, there are currently two functions which are converting values to more human-friendly represetations. + +[cols=",,",options="header",] +|=== +| Function name | Input type | Description +| `format_bytes` |`int`, `tinyint`, `smallint`, `bigint`, `varint`, `ascii`, `text` | Converts values in bytes to a more human-friendly representation. + +| `format_time` |`int`, `tinyint`, `smallint`, `bigint`, `varint`, `ascii`, `text` | Converts values in milliseconds to a more human-friendly representation. + +|=== + + +===== format_bytes + +This function looks at values in a column as if it was in bytes, and it will convert it to whatever a user pleases. Supported units are: `B`, `KiB`, `MiB` and `GiB`. The result will be rounded to two decimal places. + +Supported column types on which this function is possible to be applied: +`INT`, `TINYINT`, `SMALLINT`, `BIGINT`, `VARINT`, `ASCII`, `TEXT`. +For `ASCII` and `TEXT` types, text of such column has to be a non-negative number. + +Return values can be max of `Long.MAX_VALUE`, If the conversion produces overflown value, `Long.MAX_VALUE` will be returned. + +[NOTE] +==== +The actual return value of the `Long.MAX_VALUE` will be 9223372036854776000 due to the limitations of floating-point precision. +==== + +There are three ways how to call this function. +Let's have this table: + +[source,cql] +---- +cqlsh> select * from ks.tb; + + id | val +----+---------------- + 5 | 60000 + 1 | 1234234 + 2 | 12342341234234 + 4 | 60001 + 7 | null + 6 | 43 + 3 | 123423 + +---- + +with schema + +[source,cql] +---- +CREATE TABLE ks.tb ( + id int PRIMARY KEY, + val bigint +) +---- + +Imagine that we wanted to look at `val` values as if they were in mebibytes. We would like to have more human-friendly output in order to not visually divide the values by 1024 in order to get them in respective bigger units. The following function call may take just a column itself as an argument, and it will +automatically convert it. + +[NOTE] +==== +The default source unit for `format_bytes` function is _bytes_, (`B`). +==== + +[source,cql] +---- +cqlsh> select format_bytes(val) from ks.tb; + + system.format_bytes(val) +-------------------------- + 58.59 KiB + 1.18 MiB + 11494.7 GiB + 58.59 KiB + null + 43 B + 120.53 KiB +---- + +The second way to call `format_bytes` functions is to specify into what size unit we would like to see all +values to be converted to. For example, we want all size to be represented in mebibytes, hence we do: + +[source,cql] +---- +cqlsh> select format_bytes(val, 'MiB') from ks.tb; + + system.format_bytes(val, 'MiB') +---------------------------------- + 0.06 MiB + 1.18 MiB + 11770573.84 MiB + 0.06 MiB + null + 0 MiB + 0.12 MiB +---- + +Lastly, we can specify a source unit and a target unit. A source unit tells what unit that column is logically of, the target unit tells what unit we want these values to be converted to. For example, +if we know that our column is logically in kibibytes and we want them to be converted into mebibytes, we would do: + +[source,cql] +---- +cqlsh> select format_bytes(val, 'Kib', 'MiB') from ks.tb; + + system.format_bytes(val, 'KiB', 'MiB') +---------------------------------------- + 58.59 MiB + 1205.31 MiB + 12053067611.56 MiB + 58.59 MiB + null + 0.04 MiB + 120.53 MiB +---- + +===== format_time + +Similarly to `format_bytes`, we can do transformations on duration-like columns. + +Supported units are: `d`, `h`, `m`, `s`, `ms`, `us`, `µs`, `ns`. + +Supported column types on which this function is possible to be applied: +`INT`, `TINYINT`, `SMALLINT`, `BIGINT`, `VARINT`, `ASCII`, `TEXT`. For `ASCII` and `TEXT` types, text of such column has to be a non-negative number. + +Return values can be max of `Double.MAX_VALUE`, If the conversion produces overflown value, `Double.MAX_VALUE` will be returned. + +[NOTE] +==== +The default source unit for `format_time` function is _milliseconds_, (`ms`). +==== + +[source,cql] +---- +cqlsh> select format_time(val) from ks.tb; + + system.format_time(val) +------------------------- + 1 m + 20.57 m + 142851.17 d + 1 m + null + 43 ms + 2.06 m +---- + +We may specify what unit we want that value to be converted to, give the column's values are in millseconds: + +[source,cql] +---- +cqlsh> select format_time(val, 'm') from ks.tb; + + system.format_time(val, 'm') +------------------------------ + 1 m + 20.57 m + 205705687.24 m + 1 m + null + 0 m + 2.06 m +---- + +Lastly, we can specify both source and target values: + +[source,cql] +---- +cqlsh> select format_time(val, 's', 'h') from ks.tb; + + system.format_time(val, 's', 'h') +----------------------------------- + 16.67 h + 342.84 h + 3428428120.62 h + 16.67 h + null + 0.01 h + 34.28 h +---- + [[user-defined-scalar-functions]] === User-defined functions diff --git a/src/java/org/apache/cassandra/config/DataStorageSpec.java b/src/java/org/apache/cassandra/config/DataStorageSpec.java index 53d71717eff3..3c7086f20f15 100644 --- a/src/java/org/apache/cassandra/config/DataStorageSpec.java +++ b/src/java/org/apache/cassandra/config/DataStorageSpec.java @@ -480,25 +480,21 @@ public enum DataStorageUnit { BYTES("B") { - public long toBytes(long d) - { - return d; - } + public long toBytes(long d) { return d; } - public long toKibibytes(long d) - { - return (d / 1024L); - } + public long toKibibytes(long d) { return (d / 1024L); } - public long toMebibytes(long d) - { - return (d / (1024L * 1024)); - } + public long toMebibytes(long d) { return (d / (1024L * 1024)); } - public long toGibibytes(long d) - { - return (d / (1024L * 1024 * 1024)); - } + public long toGibibytes(long d) { return (d / (1024L * 1024 * 1024)); } + + public double toBytesDouble(long d) { return (double) d; } + + public double toKibibytesDouble(long d) { return d / 1024.0; } + + public double toMebibytesDouble(long d) { return d / (1024.0 * 1024); } + + public double toGibibytesDouble(long d) { return d / (1024.0 * 1024 * 1024); } public long convert(long source, DataStorageUnit sourceUnit) { @@ -527,6 +523,14 @@ public long toGibibytes(long d) return (d / (1024L * 1024)); } + public double toBytesDouble(long d) { return (double) toBytes(d); } + + public double toKibibytesDouble(long d) { return (double) d; } + + public double toMebibytesDouble(long d) { return d / 1024.0; } + + public double toGibibytesDouble(long d) { return d / (1024.0 * 1024); } + public long convert(long source, DataStorageUnit sourceUnit) { return sourceUnit.toKibibytes(source); @@ -544,16 +548,21 @@ public long toKibibytes(long d) return x(d, 1024L, (MAX / 1024L)); } - public long toMebibytes(long d) - { - return d; - } + public long toMebibytes(long d) { return d; } public long toGibibytes(long d) { return (d / 1024L); } + public double toBytesDouble(long d) { return (double) toBytes(d); } + + public double toKibibytesDouble(long d) { return (double) toKibibytes(d); } + + public double toMebibytesDouble(long d) { return (double) d; } + + public double toGibibytesDouble(long d) { return d / 1024.0; } + public long convert(long source, DataStorageUnit sourceUnit) { return sourceUnit.toMebibytes(source); @@ -576,10 +585,15 @@ public long toMebibytes(long d) return x(d, 1024L, (MAX / 1024L)); } - public long toGibibytes(long d) - { - return d; - } + public long toGibibytes(long d) { return d; } + + public double toBytesDouble(long d) { return (double) toBytes(d); } + + public double toKibibytesDouble(long d) { return (double) toKibibytes(d); } + + public double toMebibytesDouble(long d) { return (double) toMebibytes(d); } + + public double toGibibytesDouble(long d) { return (double) d; } public long convert(long source, DataStorageUnit sourceUnit) { @@ -628,26 +642,51 @@ public static DataStorageUnit fromSymbol(String symbol) this.symbol = symbol; } + public String getSymbol() + { + return symbol; + } + public long toBytes(long d) { throw new AbstractMethodError(); } + public double toBytesDouble(long d) + { + throw new AbstractMethodError(); + } + public long toKibibytes(long d) { throw new AbstractMethodError(); } + public double toKibibytesDouble(long d) + { + throw new AbstractMethodError(); + } + public long toMebibytes(long d) { throw new AbstractMethodError(); } + public double toMebibytesDouble(long d) + { + throw new AbstractMethodError(); + } + public long toGibibytes(long d) { throw new AbstractMethodError(); } + public double toGibibytesDouble(long d) + { + throw new AbstractMethodError(); + } + public long convert(long source, DataStorageUnit sourceUnit) { throw new AbstractMethodError(); diff --git a/src/java/org/apache/cassandra/config/DurationSpec.java b/src/java/org/apache/cassandra/config/DurationSpec.java index 3a9a03ab1fb2..bf0fc21334ad 100644 --- a/src/java/org/apache/cassandra/config/DurationSpec.java +++ b/src/java/org/apache/cassandra/config/DurationSpec.java @@ -141,7 +141,7 @@ public TimeUnit unit() * @param symbol the time unit symbol * @return the time unit associated to the specified symbol */ - static TimeUnit fromSymbol(String symbol) + public static TimeUnit fromSymbol(String symbol) { switch (toLowerCaseLocalized(symbol)) { diff --git a/src/java/org/apache/cassandra/cql3/functions/FormatFcts.java b/src/java/org/apache/cassandra/cql3/functions/FormatFcts.java new file mode 100644 index 000000000000..caee5190c427 --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/functions/FormatFcts.java @@ -0,0 +1,447 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.functions; + +import java.math.RoundingMode; +import java.nio.ByteBuffer; +import java.text.DecimalFormat; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.TimeUnit; + +import org.apache.cassandra.config.DataStorageSpec.DataStorageUnit; +import org.apache.cassandra.config.DurationSpec; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.utils.Pair; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.cassandra.config.DataStorageSpec.DataStorageUnit.BYTES; +import static org.apache.cassandra.config.DataStorageSpec.DataStorageUnit.GIBIBYTES; +import static org.apache.cassandra.config.DataStorageSpec.DataStorageUnit.KIBIBYTES; +import static org.apache.cassandra.config.DataStorageSpec.DataStorageUnit.MEBIBYTES; +import static org.apache.cassandra.cql3.CQL3Type.Native.ASCII; +import static org.apache.cassandra.cql3.CQL3Type.Native.BIGINT; +import static org.apache.cassandra.cql3.CQL3Type.Native.INT; +import static org.apache.cassandra.cql3.CQL3Type.Native.SMALLINT; +import static org.apache.cassandra.cql3.CQL3Type.Native.TEXT; +import static org.apache.cassandra.cql3.CQL3Type.Native.TINYINT; +import static org.apache.cassandra.cql3.CQL3Type.Native.VARINT; +import static org.apache.cassandra.cql3.functions.FunctionParameter.fixed; +import static org.apache.cassandra.cql3.functions.FunctionParameter.optional; + +public class FormatFcts +{ + private static final DecimalFormat decimalFormat; + + static + { + decimalFormat = new DecimalFormat("#.##"); + decimalFormat.setRoundingMode(RoundingMode.HALF_UP); + } + + /** + * Formats a double value to a string with two decimal places. + * <p> + * Supported column types on which this function is possible to be applied: + * <pre>DOUBLE</pre> + */ + public static String format(double value) + { + return decimalFormat.format(value); + } + + public static void addFunctionsTo(NativeFunctions functions) + { + functions.add(FormatBytesFct.factory()); + functions.add(FormatTimeFct.factory()); + } + + private static long validateAndGetValue(Arguments arguments) + { + if (arguments.containsNulls()) + throw new InvalidRequestException("none of the arguments may be null"); + + long value = getValue(arguments); + + if (value < 0) + throw new InvalidRequestException("value must be non-negative"); + + return value; + } + + private static long getValue(Arguments arguments) + { + Optional<String> maybeString = getAsString(arguments, 0); + + if (maybeString.isPresent()) + { + try + { + return Long.parseLong(maybeString.get()); + } + catch (Exception ex) + { + throw new InvalidRequestException("unable to convert string '" + maybeString.get() + "' to a value of type long"); + } + } + else + { + return arguments.getAsLong(0); + } + } + + private static Optional<String> getAsString(Arguments arguments, int i) + { + try + { + return Optional.ofNullable(arguments.get(i)); + } + catch (Exception ex) + { + return Optional.empty(); + } + } + + private FormatFcts() + { + } + + /** + * Converts numeric value in a column to a value of specified unit. + * <p> + * If the function call contains just one argument - value to convert - then it will be + * looked at as the value is of unit 'ms' and it will be converted to a value of a unit which is closest to it. + * The result will be rounded to two decimal places. + * E.g. If a value is (20 * 1000 + 250) then the unit will be in seconds and converted value will be 20.25. + * <p> + * If the function call contains two arguments - value to convert and a unit - then it will be looked at + * as the unit of such value is 'ms' and it will be converted into the value of the second (unit) argument. + * <p> + * If the function call contains three arguments - value to covert and source and target unit - then the value + * will be considered of a unit of the second argument, and it will be converted + * into a value of the third (unit) argument. + * <p> + * Examples: + * <pre> + * format_time(val) + * format_time(val, 'm') = format_time(val, 'ms', 'm') + * format_time(val, 's', 'm') + * format_time(val, 's', 'h') + * format_time(val, 's', 'd') + * format_time(val, 's') = format_time(val, 'ms', 's') + * format_time(val, 'h') = format_time(val, 'ms', 'h') + * </pre> + * <p> + * It is possible to convert values of a bigger unit to values of a smaller unit, e.g. this is possible: + * + * <pre> + * format_time(val, 'm', 's') + * </pre> + * <p> + * Values can be max of Double.MAX_VALUE, If the conversion produces overflown value, Double.MAX_VALUE will be returned. + * <p> + * Supported units are: d, h, m, s, ms, us, µs, ns + * <p> + * Supported column types on which this function is possible to be applied: + * <pre>INT, TINYINT, SMALLINT, BIGINT, VARINT, ASCII, TEXT</pre> + * For ASCII and TEXT types, text of such column has to be a non-negative number. + * <p> + * The conversion of negative values is not supported. + */ + public static class FormatTimeFct extends NativeScalarFunction + { + private static final String FUNCTION_NAME = "format_time"; + + private static final String[] UNITS = { "d", "h", "m", "s" }; + private static final long[] CONVERSION_FACTORS = { 86400000, 3600000, 60000, 1000 }; // Milliseconds in a day, hour, minute, second + + private FormatTimeFct(AbstractType<?>... argsTypes) + { + super(FUNCTION_NAME, UTF8Type.instance, argsTypes); + } + + @Override + public ByteBuffer execute(Arguments arguments) throws InvalidRequestException + { + if (arguments.get(0) == null) + return null; + + long value = validateAndGetValue(arguments); + + if (arguments.size() == 1) + { + Pair<Double, String> convertedValue = convertValue(value); + return UTF8Type.instance.fromString(format(convertedValue.left) + ' ' + convertedValue.right); + } + + TimeUnit sourceUnit; + TimeUnit targetUnit; + String targetUnitAsString; + + if (arguments.size() == 2) + { + sourceUnit = MILLISECONDS; + targetUnitAsString = arguments.get(1); + } + else + { + sourceUnit = validateUnit(arguments.get(1)); + targetUnitAsString = arguments.get(2); + } + + targetUnit = validateUnit(targetUnitAsString); + + double convertedValue = convertValue(value, sourceUnit, targetUnit); + return UTF8Type.instance.fromString(format(convertedValue) + ' ' + targetUnitAsString); + } + + private TimeUnit validateUnit(String unitAsString) + { + try + { + return DurationSpec.fromSymbol(unitAsString); + } + catch (Exception ex) + { + throw new InvalidRequestException(ex.getMessage()); + } + } + + private Pair<Double, String> convertValue(long valueToConvert) + { + for (int i = 0; i < CONVERSION_FACTORS.length; i++) + { + if (valueToConvert >= CONVERSION_FACTORS[i]) + { + double convertedValue = (double) valueToConvert / CONVERSION_FACTORS[i]; + return Pair.create(convertedValue, UNITS[i]); + } + } + return Pair.create((double) valueToConvert, "ms"); + } + + private Double convertValue(long valueToConvert, TimeUnit sourceUnit, TimeUnit targetUnit) + { + try + { + double conversionFactor = getConversionFactor(sourceUnit, targetUnit); + return valueToConvert * conversionFactor; + } + catch (ArithmeticException ex) + { + return Double.MAX_VALUE; + } + } + + private static double getConversionFactor(TimeUnit sourceUnit, TimeUnit targetUnit) + { + // Define conversion factors between units + double nanosPerSourceUnit = getNanosPerUnit(sourceUnit); + double nanosPerTargetUnit = getNanosPerUnit(targetUnit); + + // Calculate the conversion factor + return nanosPerSourceUnit / nanosPerTargetUnit; + } + + private static double getNanosPerUnit(TimeUnit unit) + { + switch (unit) + { + case NANOSECONDS: + return 1.0; + case MICROSECONDS: + return 1_000.0; + case MILLISECONDS: + return 1_000_000.0; + case SECONDS: + return 1_000_000_000.0; + case MINUTES: + return 60.0 * 1_000_000_000.0; + case HOURS: + return 3600.0 * 1_000_000_000.0; + case DAYS: + return 86400.0 * 1_000_000_000.0; + default: + throw new IllegalArgumentException("Unsupported time unit: " + unit); + } + } + + public static FunctionFactory factory() + { + return new FunctionFactory(FUNCTION_NAME, + fixed(INT, TINYINT, SMALLINT, BIGINT, VARINT, ASCII, TEXT), + optional(fixed(ASCII)), + optional(fixed(ASCII))) + { + @Override + protected NativeFunction doGetOrCreateFunction(List<AbstractType<?>> argTypes, AbstractType<?> receiverType) + { + if (argTypes.isEmpty() || argTypes.size() > 3) + throw invalidNumberOfArgumentsException(); + + return new FormatTimeFct(argTypes.toArray(new AbstractType<?>[0])); + } + }; + } + } + + /** + * Converts numeric value in a column to a size value of specified unit. + * <p> + * If the function call contains just one argument - value to convert - then it will be + * looked at as the value is of unit 'B' and it will be converted to a value of a unit which is closest to it. + * The result will be rounded to two decimal places. + * E.g. If a value is (100 * 1024 + 150) then the unit will be in KiB and converted value will be 100.15. + * <p> + * If the function call contains two arguments - value to convert and a unit - then it will be looked at + * as the unit of such value is 'B' and it will be converted into the value of the second (unit) argument. + * <p> + * If the function call contains three arguments - value to covert and source and target unit - then the value + * will be considered of a unit of the second argument, and it will be converted + * into a value of the third (unit) argument. + * <p> + * Examples: + * <pre> + * format_bytes(val) = format_bytes(val, 'B', 'MiB') + * format_bytes(val, 'B', 'MiB') + * format_bytes(val, 'B', 'GiB') + * format_bytes(val, 'KiB', 'GiB') + * format_bytes(val, 'MiB') = format_bytes(val, 'B', 'MiB') + * format_bytes(val, 'GiB') = format_bytes(val, 'B', 'GiB') + * </pre> + * <p> + * It is possible to convert values of a bigger unit to values of a smaller unit, e.g. this is possible: + * + * <pre> + * format_bytes(val, 'GiB', 'B') + * </pre> + * <p> + * Values can be max of Long.MAX_VALUE, If the conversion produces overflown value, Long.MAX_VALUE will be returned. + * Note that the actual return value will be 9223372036854776000 due to the limitations of double precision. + * <p> + * Supported units are: B, KiB, MiB, GiB + * <p> + * Supported column types on which this function is possible to be applied: + * <pre>INT, TINYINT, SMALLINT, BIGINT, VARINT, ASCII, TEXT</pre> + * For ASCII and TEXT types, text of such column has to be a non-negative number. + * <p> + * <p> + * The conversion of negative values is not supported. + */ + public static class FormatBytesFct extends NativeScalarFunction + { + private static final String FUNCTION_NAME = "format_bytes"; + + private FormatBytesFct(AbstractType<?>... argsTypes) + { + super(FUNCTION_NAME, UTF8Type.instance, argsTypes); + } + + @Override + public ByteBuffer execute(Arguments arguments) throws InvalidRequestException + { + if (arguments.get(0) == null) + return null; + + long value = validateAndGetValue(arguments); + + DataStorageUnit sourceUnit; + DataStorageUnit targetUnit; + + if (arguments.size() == 1) + { + sourceUnit = BYTES; + + if (value > FileUtils.ONE_GIB) + targetUnit = GIBIBYTES; + else if (value > FileUtils.ONE_MIB) + targetUnit = MEBIBYTES; + else if (value > FileUtils.ONE_KIB) + targetUnit = KIBIBYTES; + else + targetUnit = BYTES; + } + else if (arguments.size() == 2) + { + sourceUnit = BYTES; + targetUnit = validateUnit(arguments.get(1)); + } + else + { + sourceUnit = validateUnit(arguments.get(1)); + targetUnit = validateUnit(arguments.get(2)); + } + + double convertedValue = convertValue(value, sourceUnit, targetUnit); + String convertedValueAsString = format(convertedValue); + + return UTF8Type.instance.fromString(convertedValueAsString + ' ' + targetUnit.getSymbol()); + } + + private double convertValue(long valueToConvert, DataStorageUnit sourceUnit, DataStorageUnit targetUnit) + { + switch (targetUnit) + { + case BYTES: + return sourceUnit.toBytesDouble(valueToConvert); + case KIBIBYTES: + return sourceUnit.toKibibytesDouble(valueToConvert); + case MEBIBYTES: + return sourceUnit.toMebibytesDouble(valueToConvert); + case GIBIBYTES: + return sourceUnit.toGibibytesDouble(valueToConvert); + default: + throw new InvalidRequestException("unsupported target unit " + targetUnit); + } + } + + private DataStorageUnit validateUnit(String unitAsString) + { + try + { + return DataStorageUnit.fromSymbol(unitAsString); + } + catch (Exception ex) + { + throw new InvalidRequestException(ex.getMessage()); + } + } + + public static FunctionFactory factory() + { + return new FunctionFactory(FUNCTION_NAME, + fixed(INT, TINYINT, SMALLINT, BIGINT, VARINT, ASCII, TEXT), + optional(fixed(ASCII)), + optional(fixed(ASCII))) + { + @Override + protected NativeFunction doGetOrCreateFunction(List<AbstractType<?>> argTypes, AbstractType<?> receiverType) + { + if (argTypes.isEmpty() || argTypes.size() > 3) + throw invalidNumberOfArgumentsException(); + + return new FormatBytesFct(argTypes.toArray(new AbstractType<?>[0])); + } + }; + } + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/cql3/functions/NativeFunctions.java b/src/java/org/apache/cassandra/cql3/functions/NativeFunctions.java index e521c5e807ff..d359d741ae46 100644 --- a/src/java/org/apache/cassandra/cql3/functions/NativeFunctions.java +++ b/src/java/org/apache/cassandra/cql3/functions/NativeFunctions.java @@ -49,6 +49,7 @@ public class NativeFunctions MaskingFcts.addFunctionsTo(this); VectorFcts.addFunctionsTo(this); ClusterMetadataFcts.addFunctionsTo(this); + FormatFcts.addFunctionsTo(this); } }; diff --git a/test/unit/org/apache/cassandra/cql3/functions/AbstractFormatFctTest.java b/test/unit/org/apache/cassandra/cql3/functions/AbstractFormatFctTest.java new file mode 100644 index 000000000000..8030bbddf0b1 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/functions/AbstractFormatFctTest.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.functions; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import org.junit.Ignore; + +import org.apache.cassandra.cql3.CQL3Type; +import org.apache.cassandra.cql3.CQLTester; + +@Ignore +public abstract class AbstractFormatFctTest extends CQLTester +{ + protected void createTable(List<CQL3Type.Native> columnTypes, Object[][] rows) + { + String[][] columns = new String[columnTypes.size() + 1][2]; + + columns[0][0] = "pk"; + columns[0][1] = "int"; + + for (int i = 1; i <= columnTypes.size(); i++) + { + columns[i][0] = "col" + i; + columns[i][1] = columnTypes.get(i - 1).name().toLowerCase(); + } + + createTable(columns, rows); + } + + protected void createDefaultTable(Object[][] rows) + { + createTable(new String[][]{ { "pk", "int" }, { "col1", "int" }, { "col2", "int" } }, rows); + } + + protected void createTable(String[][] columns, Object[][] rows) + { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < columns.length; i++) + { + sb.append(columns[i][0]); + sb.append(' '); + sb.append(columns[i][1]); + + if (i == 0) + sb.append(" primary key"); + + if (i + 1 != columns.length) + sb.append(", "); + } + String columnsDefinition = sb.toString(); + createTable(KEYSPACE, "CREATE TABLE %s (" + columnsDefinition + ')'); + + String cols = Arrays.stream(columns).map(s -> s[0]).collect(Collectors.joining(", ")); + + for (Object[] row : rows) + { + String vals = Arrays.stream(row).map(v -> { + if (v == null) + return "null"; + return v.toString(); + }).collect(Collectors.joining(", ")); + execute("INSERT INTO %s (" + cols + ") values (" + vals + ')'); + } + } +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/cql3/functions/FormatBytesFctTest.java b/test/unit/org/apache/cassandra/cql3/functions/FormatBytesFctTest.java new file mode 100644 index 000000000000..7dbba2f68b20 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/functions/FormatBytesFctTest.java @@ -0,0 +1,307 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.functions; + +import org.junit.Test; +import org.apache.cassandra.exceptions.InvalidRequestException; + +import static java.util.List.of; +import static org.apache.cassandra.cql3.CQL3Type.Native.ASCII; +import static org.apache.cassandra.cql3.CQL3Type.Native.BIGINT; +import static org.apache.cassandra.cql3.CQL3Type.Native.INT; +import static org.apache.cassandra.cql3.CQL3Type.Native.SMALLINT; +import static org.apache.cassandra.cql3.CQL3Type.Native.TEXT; +import static org.apache.cassandra.cql3.CQL3Type.Native.TINYINT; +import static org.apache.cassandra.cql3.CQL3Type.Native.VARINT; +import static org.apache.cassandra.cql3.functions.FormatFcts.format; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.quicktheories.QuickTheory.qt; +import static org.quicktheories.generators.SourceDSL.integers; + +public class FormatBytesFctTest extends AbstractFormatFctTest +{ + @Test + public void testOneValueArgumentExact() + { + createTable(of(INT), new Object[][]{ { 1, 1073741825 }, + { 2, 1073741823 }, + { 3, 0 } }); // 0 B + assertRows(execute("select format_bytes(col1) from %s where pk = 1"), row("1 GiB")); + assertRows(execute("select format_bytes(col1) from %s where pk = 2"), row("1024 MiB")); + assertRows(execute("select format_bytes(col1) from %s where pk = 3"), row("0 B")); + } + + @Test + public void testOneValueArgumentDecimalRoundup() + { + createTable(of(INT), new Object[][]{ { 1, 1563401650 }, + { 2, 1072441589 }, + { 3, 102775 }, + { 4, 102 } }); + assertRows(execute("select format_bytes(col1) from %s where pk = 1"), row("1.46 GiB")); // 1.4560 + assertRows(execute("select format_bytes(col1) from %s where pk = 2"), row("1022.76 MiB")); // 1022.7599 + assertRows(execute("select format_bytes(col1) from %s where pk = 3"), row("100.37 KiB")); // 100.3662 + assertRows(execute("select format_bytes(col1) from %s where pk = 4"), row("102 B")); + } + + @Test + public void testOneValueArgumentDecimalRoundDown() + { + createTable(of(INT), new Object[][]{ { 1, 1557999386 }, + { 2, 1072433201 }, + { 3, 102769 }, + { 4, 102 } }); + assertRows(execute("select format_bytes(col1) from %s where pk = 1"), row("1.45 GiB")); // 1.451 + assertRows(execute("select format_bytes(col1) from %s where pk = 2"), row("1022.75 MiB")); // 1022.752 + assertRows(execute("select format_bytes(col1) from %s where pk = 3"), row("100.36 KiB")); // 100.3613 + assertRows(execute("select format_bytes(col1) from %s where pk = 4"), row("102 B")); + } + + @Test + public void testValueAndUnitArgumentsExact() + { + createTable(of(INT), new Object[][]{ { 1, 1073741825 }, + { 2, 0 } }); + assertRows(execute("select format_bytes(col1, 'B') from %s where pk = 1"), row("1073741825 B")); + assertRows(execute("select format_bytes(col1, 'KiB') from %s where pk = 1"), row("1048576 KiB")); + assertRows(execute("select format_bytes(col1, 'MiB') from %s where pk = 1"), row("1024 MiB")); + assertRows(execute("select format_bytes(col1, 'GiB') from %s where pk = 1"), row("1 GiB")); + + assertRows(execute("select format_bytes(col1, 'B') from %s where pk = 2"), row("0 B")); + assertRows(execute("select format_bytes(col1, 'KiB') from %s where pk = 2"), row("0 KiB")); + assertRows(execute("select format_bytes(col1, 'MiB') from %s where pk = 2"), row("0 MiB")); + assertRows(execute("select format_bytes(col1, 'GiB') from %s where pk = 2"), row("0 GiB")); + } + + @Test + public void testValueAndUnitArgumentsDecimal() + { + createTable(of(INT), new Object[][]{ { 1, 1563401650 }, + { 2, 1557999336 } }); + assertRows(execute("select format_bytes(col1, 'B') from %s where pk = 1"), row("1563401650 B")); + assertRows(execute("select format_bytes(col1, 'KiB') from %s where pk = 1"), row("1526759.42 KiB")); + assertRows(execute("select format_bytes(col1, 'MiB') from %s where pk = 1"), row("1490.98 MiB")); + assertRows(execute("select format_bytes(col1, 'GiB') from %s where pk = 1"), row("1.46 GiB")); + + assertRows(execute("select format_bytes(col1, 'B') from %s where pk = 2"), row("1557999336 B")); + assertRows(execute("select format_bytes(col1, 'KiB') from %s where pk = 2"), row("1521483.73 KiB")); + assertRows(execute("select format_bytes(col1, 'MiB') from %s where pk = 2"), row("1485.82 MiB")); + assertRows(execute("select format_bytes(col1, 'GiB') from %s where pk = 2"), row("1.45 GiB")); + } + + @Test + public void testValueWithSourceAndTargetArgumentExact() + { + createTable(of(INT), new Object[][]{ { 1, 1073741825 }, + { 2, 1 }, + { 3, 0 } }); + assertRows(execute("select format_bytes(col1, 'B', 'B') from %s where pk = 1"), row("1073741825 B")); + assertRows(execute("select format_bytes(col1, 'B', 'KiB') from %s where pk = 1"), row("1048576 KiB")); + assertRows(execute("select format_bytes(col1, 'B', 'MiB') from %s where pk = 1"), row("1024 MiB")); + assertRows(execute("select format_bytes(col1, 'B', 'GiB') from %s where pk = 1"), row("1 GiB")); + + assertRows(execute("select format_bytes(col1, 'GiB', 'GiB') from %s where pk = 2"), row("1 GiB")); + assertRows(execute("select format_bytes(col1, 'GiB', 'MiB') from %s where pk = 2"), row("1024 MiB")); + assertRows(execute("select format_bytes(col1, 'GiB', 'KiB') from %s where pk = 2"), row("1048576 KiB")); + assertRows(execute("select format_bytes(col1, 'GiB', 'B') from %s where pk = 2"), row("1073741824 B")); + + assertRows(execute("select format_bytes(col1, 'GiB', 'GiB') from %s where pk = 3"), row("0 GiB")); + assertRows(execute("select format_bytes(col1, 'GiB', 'MiB') from %s where pk = 3"), row("0 MiB")); + assertRows(execute("select format_bytes(col1, 'GiB', 'KiB') from %s where pk = 3"), row("0 KiB")); + assertRows(execute("select format_bytes(col1, 'GiB', 'B') from %s where pk = 3"), row("0 B")); + } + + @Test + public void testValueWithSourceAndTargetArgumentDecimal() + { + createTable(of(INT), new Object[][]{ { 1, 1563401650 }, + { 2, 1557999336 },}); + assertRows(execute("select format_bytes(col1, 'B', 'B') from %s where pk = 1"), row("1563401650 B")); + assertRows(execute("select format_bytes(col1, 'B', 'KiB') from %s where pk = 1"), row("1526759.42 KiB")); + assertRows(execute("select format_bytes(col1, 'B', 'MiB') from %s where pk = 1"), row("1490.98 MiB")); + assertRows(execute("select format_bytes(col1, 'B', 'GiB') from %s where pk = 1"), row("1.46 GiB")); + + assertRows(execute("select format_bytes(col1, 'B', 'B') from %s where pk = 2"), row("1557999336 B")); + assertRows(execute("select format_bytes(col1, 'B', 'KiB') from %s where pk = 2"), row("1521483.73 KiB")); + assertRows(execute("select format_bytes(col1, 'B', 'MiB') from %s where pk = 2"), row("1485.82 MiB")); + assertRows(execute("select format_bytes(col1, 'B', 'GiB') from %s where pk = 2"), row("1.45 GiB")); + } + + @Test + public void testFuzzNumberGenerators() + { + createTable("CREATE TABLE %s (pk int primary key, col1 int)"); + + qt().withExamples(1024).forAll(integers().allPositive()).checkAssert( + (randInt) -> { + execute("INSERT INTO %s (pk, col1) VALUES (?, ?)", 1, randInt); + + assertRows(execute("select format_bytes(col1, 'MiB') from %s where pk = 1"), row(format(randInt / 1024.0 / 1024.0) + " MiB")); + assertRows(execute("select format_bytes(col1, 'KiB', 'GiB') from %s where pk = 1"), row(format(randInt / 1024.0 / 1024.0) + " GiB")); + assertRows(execute("select format_bytes(col1, 'B', 'GiB') from %s where pk = 1"), row(format(randInt / 1024.0 / 1024.0 / 1024.0 ) + " GiB")); + }); + } + + @Test + public void testOverflow() + { + createTable(of(BIGINT, INT, SMALLINT, TINYINT), + new Object[][]{ { 1, + 1073741825L * 1024 + 1, + Integer.MAX_VALUE - 1, + Short.MAX_VALUE - 1, + Byte.MAX_VALUE - 1 }, + { 2, + 1073741825L * 1024 + 1, + Integer.MAX_VALUE, + Short.MAX_VALUE, + Byte.MAX_VALUE } }); + + // this will stop at Long.MAX_VALUE + assertRows(execute("select format_bytes(col1, 'GiB', 'B') from %s where pk = 1"), row("9223372036854776000 B")); + assertRows(execute("select format_bytes(col2, 'GiB', 'B') from %s where pk = 1"), row("2305843007066210300 B")); + assertRows(execute("select format_bytes(col3, 'GiB', 'B') from %s where pk = 1"), row("35182224605184 B")); + assertRows(execute("select format_bytes(col4, 'GiB', 'B') from %s where pk = 1"), row("135291469824 B")); + + assertRows(execute("select format_bytes(col2, 'GiB', 'B') from %s where pk = 2"), row("2305843008139952130 B")); + assertRows(execute("select format_bytes(col3, 'GiB', 'B') from %s where pk = 2"), row("35183298347008 B")); + assertRows(execute("select format_bytes(col4, 'GiB', 'B') from %s where pk = 2"), row("136365211648 B")); + } + + @Test + public void testAllSupportedColumnTypes() + { + createTable(of(INT, TINYINT, SMALLINT, BIGINT, VARINT, ASCII, TEXT), + new Object[][]{ { 1, + Integer.MAX_VALUE, + Byte.MAX_VALUE, + Short.MAX_VALUE, + Long.MAX_VALUE, + Integer.MAX_VALUE, + '\'' + Integer.valueOf(Integer.MAX_VALUE).toString() + '\'', + '\'' + Integer.valueOf(Integer.MAX_VALUE).toString() + '\'', + } }); + + assertRows(execute("select format_bytes(col1) from %s where pk = 1"), row("2 GiB")); + assertRows(execute("select format_bytes(col2) from %s where pk = 1"), row("127 B")); + assertRows(execute("select format_bytes(col3) from %s where pk = 1"), row("32 KiB")); + assertRows(execute("select format_bytes(col4) from %s where pk = 1"), row("8589934592 GiB")); + assertRows(execute("select format_bytes(col5) from %s where pk = 1"), row("2 GiB")); + assertRows(execute("select format_bytes(col6) from %s where pk = 1"), row("2 GiB")); + assertRows(execute("select format_bytes(col7) from %s where pk = 1"), row("2 GiB")); + } + + @Test + public void testNegativeValueIsInvalid() + { + createDefaultTable(new Object[][]{ { "1", "-1", "-2" } }); + assertThatThrownBy(() -> execute("select format_bytes(col1) from %s where pk = 1")) + .isInstanceOf(InvalidRequestException.class) + .hasMessageContaining("value must be non-negative"); + } + + @Test + public void testUnparsableTextIsInvalid() + { + createTable(of(TEXT), new Object[][]{ { 1, "'abc'" }, { 2, "'-1'" } }); + + assertThatThrownBy(() -> execute("select format_bytes(col1) from %s where pk = 1")) + .isInstanceOf(InvalidRequestException.class) + .hasMessageContaining("unable to convert string 'abc' to a value of type long"); + + assertThatThrownBy(() -> execute("select format_bytes(col1) from %s where pk = 2")) + .isInstanceOf(InvalidRequestException.class) + .hasMessageContaining("value must be non-negative"); + } + + @Test + public void testInvalidUnits() + { + createDefaultTable(new Object[][]{ { "1", "1", "2" } }); + for (String functionCall : new String[] { + "format_bytes(col1, 'abc')", + "format_bytes(col1, 'B', 'abc')", + "format_bytes(col1, 'abc', 'B')", + "format_bytes(col1, 'abc', 'abc')" + }) + { + assertThatThrownBy(() -> execute("select " + functionCall + " from %s where pk = 1")) + .isInstanceOf(InvalidRequestException.class) + .hasMessageContaining("Unsupported data storage unit: abc. Supported units are: B, KiB, MiB, GiB"); + } + } + + @Test + public void testInvalidArgumentsSize() + { + createDefaultTable(new Object[][]{ { "1", "1", "2" } }); + + // Test arguemnt size = 0 + assertThatThrownBy(() -> execute("select format_bytes() from %s where pk = 1")) + .isInstanceOf(InvalidRequestException.class) + .hasMessageContaining("Invalid number of arguments for function system.format_bytes([int|tinyint|smallint|bigint|varint|ascii|text], [ascii], [ascii])"); + + // Test argument size > 3 + assertThatThrownBy(() -> execute("select format_bytes(col1, 'B', 'KiB', 'GiB') from %s where pk = 1")) + .isInstanceOf(InvalidRequestException.class) + .hasMessageContaining("Invalid number of arguments for function system.format_bytes([int|tinyint|smallint|bigint|varint|ascii|text], [ascii], [ascii])"); + } + + @Test + public void testHandlingNullValues() + { + createTable(of(TEXT, ASCII, INT), + new Object[][]{ { 1, null, null, null } }); + + assertRows(execute("select format_bytes(col1), format_bytes(col2), format_bytes(col3) from %s where pk = 1"), + row(null, null, null)); + + assertRows(execute("select format_bytes(col1, 'B') from %s where pk = 1"), row((Object) null)); + assertRows(execute("select format_bytes(col1, 'B', 'KiB') from %s where pk = 1"), row((Object) null)); + } + + @Test + public void testHandlingNullArguments() + { + createTable(of(TEXT, ASCII, INT), + new Object[][]{ { 1, null, null, null }, + { 2, "'1'", "'2'", 3 } }); + + assertRows(execute("select format_bytes(col1, null) from %s where pk = 1"), row((Object) null)); + + for (String functionCall : new String[] { + "format_bytes(col3, null)", + "format_bytes(col3, null, null)", + "format_bytes(col3, null, 'KiB')", + "format_bytes(col3, 'KiB', null)" + }) + { + assertThatThrownBy(() -> execute("select " + functionCall + " from %s where pk = 2")) + .isInstanceOf(InvalidRequestException.class) + .hasMessageContaining("none of the arguments may be null"); + } + } + + @Test + public void testSizeSmallerThan1KibiByte() + { + createDefaultTable(new Object[][]{ { "1", "900", "2000" } }); + assertRows(execute("select format_bytes(col1) from %s where pk = 1"), row("900 B")); + } +} diff --git a/test/unit/org/apache/cassandra/cql3/functions/FormatTimeFctTest.java b/test/unit/org/apache/cassandra/cql3/functions/FormatTimeFctTest.java new file mode 100644 index 000000000000..9888fd36fbba --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/functions/FormatTimeFctTest.java @@ -0,0 +1,268 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.functions; + +import org.junit.Test; +import org.apache.cassandra.exceptions.InvalidRequestException; + +import static java.util.List.of; +import static org.apache.cassandra.cql3.CQL3Type.Native.ASCII; +import static org.apache.cassandra.cql3.CQL3Type.Native.BIGINT; +import static org.apache.cassandra.cql3.CQL3Type.Native.INT; +import static org.apache.cassandra.cql3.CQL3Type.Native.SMALLINT; +import static org.apache.cassandra.cql3.CQL3Type.Native.TEXT; +import static org.apache.cassandra.cql3.CQL3Type.Native.TINYINT; +import static org.apache.cassandra.cql3.CQL3Type.Native.VARINT; +import static org.apache.cassandra.cql3.functions.FormatFcts.format; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import org.quicktheories.WithQuickTheories; + +public class FormatTimeFctTest extends AbstractFormatFctTest implements WithQuickTheories +{ + @Test + public void testOneValueArgument() + { + createTable(of(INT), new Object[][]{ { 1, 7200001 }, // 2h + 1ms + { 2, 7199999 }, // 2h - 1ms + { 3, 0 } }); // 0 B + assertRows(execute("select format_time(col1) from %s where pk = 1"), row("2 h")); + assertRows(execute("select format_time(col1) from %s where pk = 2"), row("2 h")); + assertRows(execute("select format_time(col1) from %s where pk = 3"), row("0 ms")); + } + + @Test + public void testOneValueArgumentDecimal() + { + createTable(of(INT), new Object[][]{ { 1, 9000000 }, // 2.5h + { 2, 7704000 }, // 2.14h + { 3, 7848000 } }); // 2.18h + assertRows(execute("select format_time(col1) from %s where pk = 1"), row("2.5 h")); + assertRows(execute("select format_time(col1) from %s where pk = 2"), row("2.14 h")); + assertRows(execute("select format_time(col1) from %s where pk = 3"), row("2.18 h")); + } + + @Test + public void testValueAndUnitArguments() + { + createTable(of(INT), new Object[][]{ { 1, 1073741826 }, + { 2, 0 }}); + assertRows(execute("select format_time(col1, 's') from %s where pk = 1"), row("1073741.83 s")); + assertRows(execute("select format_time(col1, 'm') from %s where pk = 1"), row("17895.7 m")); + assertRows(execute("select format_time(col1, 'h') from %s where pk = 1"), row("298.26 h")); + assertRows(execute("select format_time(col1, 'd') from %s where pk = 1"), row("12.43 d")); + + assertRows(execute("select format_time(col1, 's') from %s where pk = 2"), row("0 s")); + assertRows(execute("select format_time(col1, 'm') from %s where pk = 2"), row("0 m")); + assertRows(execute("select format_time(col1, 'h') from %s where pk = 2"), row("0 h")); + assertRows(execute("select format_time(col1, 'd') from %s where pk = 2"), row("0 d")); + } + + @Test + public void testValueWithSourceAndTargetArgument() + { + createTable(of(INT), new Object[][]{ { 1, 1073741826 }, + { 2, 1 }, + { 3, 0 } }); + assertRows(execute("select format_time(col1, 'ns', 'us') from %s where pk = 1"), row("1073741.83 us")); + assertRows(execute("select format_time(col1, 'ns', 'ms') from %s where pk = 1"), row("1073.74 ms")); + assertRows(execute("select format_time(col1, 'ns', 's') from %s where pk = 1"), row("1.07 s")); + assertRows(execute("select format_time(col1, 'ns', 'm') from %s where pk = 1"), row("0.02 m")); + + assertRows(execute("select format_time(col1, 'us', 'ns') from %s where pk = 1"), row("1073741826000 ns")); + assertRows(execute("select format_time(col1, 'us', 'ms') from %s where pk = 1"), row("1073741.83 ms")); + assertRows(execute("select format_time(col1, 'us', 's') from %s where pk = 1"), row("1073.74 s")); + assertRows(execute("select format_time(col1, 'us', 'm') from %s where pk = 1"), row("17.9 m")); + assertRows(execute("select format_time(col1, 'us', 'h') from %s where pk = 1"), row("0.3 h")); + assertRows(execute("select format_time(col1, 'us', 'd') from %s where pk = 1"), row("0.01 d")); + + assertRows(execute("select format_time(col1, 'ms', 'ms') from %s where pk = 1"), row("1073741826 ms")); + assertRows(execute("select format_time(col1, 'ms', 's') from %s where pk = 1"), row("1073741.83 s")); + assertRows(execute("select format_time(col1, 'ms', 'm') from %s where pk = 1"), row("17895.7 m")); + assertRows(execute("select format_time(col1, 'ms', 'h') from %s where pk = 1"), row("298.26 h")); + assertRows(execute("select format_time(col1, 'ms', 'd') from %s where pk = 1"), row("12.43 d")); + + assertRows(execute("select format_time(col1, 'd', 'd') from %s where pk = 2"), row("1 d")); + assertRows(execute("select format_time(col1, 'd', 'h') from %s where pk = 2"), row("24 h")); + assertRows(execute("select format_time(col1, 'd', 'm') from %s where pk = 2"), row("1440 m")); + assertRows(execute("select format_time(col1, 'd', 's') from %s where pk = 2"), row("86400 s")); + + assertRows(execute("select format_time(col1, 'd', 'd') from %s where pk = 3"), row("0 d")); + assertRows(execute("select format_time(col1, 'd', 'h') from %s where pk = 3"), row("0 h")); + assertRows(execute("select format_time(col1, 'd', 'm') from %s where pk = 3"), row("0 m")); + assertRows(execute("select format_time(col1, 'd', 's') from %s where pk = 3"), row("0 s")); + assertRows(execute("select format_time(col1, 'd', 'ms') from %s where pk = 3"), row("0 ms")); + assertRows(execute("select format_time(col1, 'd', 'us') from %s where pk = 3"), row("0 us")); + } + + @Test + public void testNoOverflow() + { + createTable(of(BIGINT, INT, SMALLINT, TINYINT), + new Object[][]{ { 1, + Long.MAX_VALUE - 1, + Integer.MAX_VALUE - 1, + Short.MAX_VALUE - 1, + Byte.MAX_VALUE - 1 }, + { 2, + Long.MAX_VALUE, + Integer.MAX_VALUE, + Short.MAX_VALUE, + Byte.MAX_VALUE } }); + + // Won't overlfow because the value is one less than the Double.MAX_VALUE + assertRows(execute("select format_time(col1, 'd', 'ns') from %s where pk = 1"), row("796899343984252600000000000000000 ns")); + assertRows(execute("select format_time(col2, 'd', 'ns') from %s where pk = 1"), row("185542587014400000000000 ns")); + assertRows(execute("select format_time(col3, 'd', 'ns') from %s where pk = 1"), row("2830982400000000000 ns")); + assertRows(execute("select format_time(col4, 'd', 'ns') from %s where pk = 1"), row("10886400000000000 ns")); + + assertRows(execute("select format_time(col1, 'd', 'ns') from %s where pk = 2"), row("796899343984252600000000000000000 ns")); + assertRows(execute("select format_time(col2, 'd', 'ns') from %s where pk = 2"), row("185542587100800000000000 ns")); + assertRows(execute("select format_time(col3, 'd', 'ns') from %s where pk = 2"), row("2831068800000000000 ns")); + assertRows(execute("select format_time(col4, 'd', 'ns') from %s where pk = 2"), row("10972800000000000 ns")); + } + + @Test + public void testAllSupportedColumnTypes() + { + createTable(of(INT, TINYINT, SMALLINT, BIGINT, VARINT, ASCII, TEXT), + new Object[][]{ { 1, + Integer.MAX_VALUE, + Byte.MAX_VALUE, + Short.MAX_VALUE, + Long.MAX_VALUE, + Integer.MAX_VALUE, + '\'' + Integer.valueOf(Integer.MAX_VALUE).toString() + '\'', + '\'' + Integer.valueOf(Integer.MAX_VALUE).toString() + '\'', + } }); + + assertRows(execute("select format_time(col1) from %s where pk = 1"), row("24.86 d")); + assertRows(execute("select format_time(col2) from %s where pk = 1"), row("127 ms")); + assertRows(execute("select format_time(col3) from %s where pk = 1"), row("32.77 s")); + assertRows(execute("select format_time(col4) from %s where pk = 1"), row("106751991167.3 d")); + assertRows(execute("select format_time(col5) from %s where pk = 1"), row("24.86 d")); + assertRows(execute("select format_time(col6) from %s where pk = 1"), row("24.86 d")); + assertRows(execute("select format_time(col7) from %s where pk = 1"), row("24.86 d")); + } + + @Test + public void testNegativeValueIsInvalid() + { + createDefaultTable(new Object[][]{ { "1", "-1", "-2" } }); + assertThatThrownBy(() -> execute("select format_time(col1) from %s where pk = 1")) + .isInstanceOf(InvalidRequestException.class) + .hasMessageContaining("value must be non-negative"); + } + + @Test + public void testUnparsableTextIsInvalid() + { + createTable(of(TEXT), new Object[][]{ { 1, "'abc'" }, { 2, "'-1'" } }); + + assertThatThrownBy(() -> execute("select format_time(col1) from %s where pk = 1")) + .isInstanceOf(InvalidRequestException.class) + .hasMessageContaining("unable to convert string 'abc' to a value of type long"); + + assertThatThrownBy(() -> execute("select format_time(col1) from %s where pk = 2")) + .isInstanceOf(InvalidRequestException.class) + .hasMessageContaining("value must be non-negative"); + } + + @Test + public void testInvalidUnits() + { + createDefaultTable(new Object[][]{ { "1", "1", "2" } }); + for (String functionCall : new String[] { + "format_time(col1, 'abc')", + "format_time(col1, 'd', 'abc')", + "format_time(col1, 'abc', 'd')", + "format_time(col1, 'abc', 'abc')" + }) + { + assertThatThrownBy(() -> execute("select " + functionCall + " from %s where pk = 1")) + .isInstanceOf(InvalidRequestException.class) + .hasMessageContaining("Unsupported time unit: abc. Supported units are: ns, us, ms, s, m, h, d"); + } + } + + @Test + public void testInvalidArgumentsSize() + { + createDefaultTable(new Object[][]{ { "1", "1", "2" } }); + // test arguemnt size = 0 + assertThatThrownBy(() -> execute("select format_time() from %s where pk = 1")) + .isInstanceOf(InvalidRequestException.class) + .hasMessageContaining("Invalid number of arguments for function system.format_time([int|tinyint|smallint|bigint|varint|ascii|text], [ascii], [ascii])"); + + // Test argument size > 3 + assertThatThrownBy(() -> execute("select format_time(col1, 'ms', 's', 'h') from %s where pk = 1")) + .isInstanceOf(InvalidRequestException.class) + .hasMessageContaining("Invalid number of arguments for function system.format_time([int|tinyint|smallint|bigint|varint|ascii|text], [ascii], [ascii])"); + } + + @Test + public void testHandlingNullValues() + { + createTable(of(TEXT, ASCII, INT), + new Object[][]{ { 1, null, null, null } }); + + assertRows(execute("select format_time(col1), format_time(col2), format_time(col3) from %s where pk = 1"), + row(null, null, null)); + + assertRows(execute("select format_time(col1, 's') from %s where pk = 1"), row((Object) null)); + assertRows(execute("select format_time(col1, 's', 'd') from %s where pk = 1"), row((Object) null)); + } + + @Test + public void testHandlingNullArguments() + { + createTable(of(TEXT, ASCII, INT), + new Object[][]{ { 1, null, null, null }, + { 2, "'1'", "'2'", 3 } }); + + assertRows(execute("select format_time(col1, null) from %s where pk = 1"), row((Object) null)); + + for (String functionCall : new String[] { + "format_time(col3, null)", + "format_time(col3, null, null)", + "format_time(col3, null, 'd')", + "format_time(col3, 'd', null)" + }) + { + assertThatThrownBy(() -> execute("select " + functionCall + " from %s where pk = 2")) + .isInstanceOf(InvalidRequestException.class) + .hasMessageContaining("none of the arguments may be null"); + } + } + + @Test + public void testFuzzRandomGenerators() + { + createTable("CREATE TABLE %s (pk int primary key, col1 int)"); + qt().withExamples(1024).forAll(integers().allPositive()).checkAssert( + (randInt) -> { + execute("INSERT INTO %s (pk, col1) VALUES (?, ?)", 1, randInt); + assertRows(execute("select format_time(col1, 's', 'm') from %s where pk = 1"), row(format((double) randInt * (1 / 60.0)) + " m")); + assertRows(execute("select format_time(col1, 's', 'h') from %s where pk = 1"), row(format((double) randInt * (1 / 3600.0)) + " h")); + assertRows(execute("select format_time(col1, 's', 'd') from %s where pk = 1"), row(format((double) randInt * (1 / 86400.0)) + " d")); + assertRows(execute("select format_time(col1, 'ms', 'm') from %s where pk = 1"), row(format((double) randInt * (1 / (60 * 1000.0))) + " m")); + assertRows(execute("select format_time(col1, 'ms', 'h') from %s where pk = 1"), row(format((double) randInt * (1 / (3600 * 1000.0))) + " h")); + assertRows(execute("select format_time(col1, 'ms', 'd') from %s where pk = 1"), row(format((double) randInt * (1 / (86400 * 1000.0))) + " d")); + }); + } +} From 1f17ef3b5cacb50664ef273eddfcdee5bd1eb425 Mon Sep 17 00:00:00 2001 From: Alex Petrov <oleksandr.petrov@gmail.com> Date: Mon, 20 Jan 2025 14:23:28 +0100 Subject: [PATCH 098/225] Fix ConsistentBootstrapTest Patch by Alex Petrov; reviewed by David Capwell for CASSANDRA-20239 --- .../fuzz/ring/ConsistentBootstrapTest.java | 95 ++++++++++--------- .../execution/InJvmDTestVisitExecutor.java | 1 + 2 files changed, 49 insertions(+), 47 deletions(-) diff --git a/test/distributed/org/apache/cassandra/fuzz/ring/ConsistentBootstrapTest.java b/test/distributed/org/apache/cassandra/fuzz/ring/ConsistentBootstrapTest.java index c6e8db78b18c..57c12b768471 100644 --- a/test/distributed/org/apache/cassandra/fuzz/ring/ConsistentBootstrapTest.java +++ b/test/distributed/org/apache/cassandra/fuzz/ring/ConsistentBootstrapTest.java @@ -35,10 +35,11 @@ import org.apache.cassandra.distributed.shared.NetworkTopology; import org.apache.cassandra.distributed.test.log.FuzzTestBase; import org.apache.cassandra.harry.SchemaSpec; -import org.apache.cassandra.harry.op.Visit; import org.apache.cassandra.harry.dsl.HistoryBuilder; import org.apache.cassandra.harry.dsl.HistoryBuilderHelper; +import org.apache.cassandra.harry.dsl.ReplayingHistoryBuilder; import org.apache.cassandra.harry.execution.DataTracker; +import org.apache.cassandra.harry.execution.InJvmDTestVisitExecutor; import org.apache.cassandra.harry.execution.RingAwareInJvmDTestVisitExecutor; import org.apache.cassandra.harry.gen.Generator; import org.apache.cassandra.harry.gen.Generators; @@ -163,7 +164,11 @@ public void coordinatorIsBehindTest() throws Throwable SchemaSpec schema = schemaGen.generate(rng); Generator<Integer> ckGen = Generators.int32(0, Math.min(schema.valueGenerators.ckPopulation(), 1000)); - HistoryBuilder history = new HistoryBuilder(schema.valueGenerators); + HistoryBuilder history = new ReplayingHistoryBuilder(schema.valueGenerators, + hb -> RingAwareInJvmDTestVisitExecutor.builder() + .replicationFactor(new TokenPlacementModel.SimpleReplicationFactor(3)) + .consistencyLevel(ConsistencyLevel.ALL) + .build(schema, hb, cluster)); cluster.schemaChange(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 3};", KEYSPACE)); cluster.schemaChange(schema.compile()); @@ -177,39 +182,36 @@ public void coordinatorIsBehindTest() throws Throwable .drop() .on(); - IInstanceConfig config = cluster.newInstanceConfig() - .set("auto_bootstrap", true) - .set(Constants.KEY_DTEST_FULL_STARTUP, true) - .set("progress_barrier_default_consistency_level", "NODE_LOCAL"); - IInvokableInstance newInstance = cluster.bootstrap(config); - // Prime the CMS node to pause before the finish join event is committed - long[] metricCounts = new long[4]; - for (int i = 1; i <= 4; i++) - metricCounts[i - 1] = cluster.get(i).callOnInstance(() -> TCMMetrics.instance.coordinatorBehindPlacements.getCount()); - DataTracker tracker = new DataTracker.SequentialDataTracker(); RingAwareInJvmDTestVisitExecutor executor = RingAwareInJvmDTestVisitExecutor.builder() .replicationFactor(new TokenPlacementModel.SimpleReplicationFactor(2)) .nodeSelector(i -> 2) .consistencyLevel(ConsistencyLevel.ALL) + .retryPolicy(InJvmDTestVisitExecutor.RetryPolicy.NO_RETRY) .build(schema, tracker, new QuiescentChecker(schema.valueGenerators, tracker, history), cluster); + // Prime the CMS node to pause before the finish join event is committed + Callable<?> pending = pauseBeforeCommit(cmsInstance, (e) -> e instanceof PrepareJoin.MidJoin); + IInstanceConfig config = cluster.newInstanceConfig() + .set("auto_bootstrap", true) + .set(Constants.KEY_DTEST_FULL_STARTUP, true) + .set("progress_barrier_default_consistency_level", "NODE_LOCAL"); + IInvokableInstance newInstance = cluster.bootstrap(config); + long[] metricCounts = new long[4]; + for (int i = 1; i <= 4; i++) + metricCounts[i - 1] = cluster.get(i).callOnInstance(() -> TCMMetrics.instance.coordinatorBehindPlacements.getCount()); Thread startup = new Thread(() -> newInstance.startup()); - - history.customThrowing(() -> { - Callable<?> pending = pauseBeforeCommit(cmsInstance, (e) -> e instanceof PrepareJoin.MidJoin); - startup.start(); - pending.call(); - }, "Startup"); + startup.start(); + pending.call(); long[] markers = new long[4]; - history.custom(() -> { - for (int n = 0; n < 4; n++) - markers[n] = cluster.get(n + 1).logs().mark(); - }, "Start grep"); + for (int n = 0; n < 4; n++) + markers[n] = cluster.get(n + 1).logs().mark(); + + boolean triggered = false; outer: for (int i = 0; i < history.valueGenerators().pkPopulation(); i++) @@ -219,15 +221,17 @@ public void coordinatorIsBehindTest() throws Throwable { if (cluster.get(1).config().broadcastAddress().toString().contains(replica.node().id())) { - HistoryBuilderHelper.insertRandomData(schema, i, ckGen.generate(rng), rng, history); - break outer; + try + { + HistoryBuilderHelper.insertRandomData(schema, i, ckGen.generate(rng), rng, history); + } + catch (Throwable t) + { + // ignore; expected + } } } - } - - history.customThrowing(() -> { - boolean triggered = false; for (int n = 0; n < markers.length; n++) { if ((n + 1) == 2) // skip 2nd node @@ -240,31 +244,28 @@ public void coordinatorIsBehindTest() throws Throwable .isEmpty()) { triggered = true; - break; + break outer; } } + } - Assert.assertTrue("Should have triggered routing exception on the replica", triggered); - boolean metricTriggered = false; - for (int i = 1; i <= 4; i++) + Assert.assertTrue("Should have triggered routing exception on the replica", triggered); + boolean metricTriggered = false; + for (int i = 1; i <= 4; i++) + { + long prevMetric = metricCounts[i - 1]; + long newMetric = cluster.get(i).callOnInstance(() -> TCMMetrics.instance.coordinatorBehindPlacements.getCount()); + if (newMetric - prevMetric > 0) { - long prevMetric = metricCounts[i - 1]; - long newMetric = cluster.get(i).callOnInstance(() -> TCMMetrics.instance.coordinatorBehindPlacements.getCount()); - if (newMetric - prevMetric > 0) - { - metricTriggered = true; - break; - } + metricTriggered = true; + break; } - Assert.assertTrue("Metric CoordinatorBehindRing should have been bumped by at least one replica", metricTriggered); - - cluster.filters().reset(); - unpauseCommits(cmsInstance); - startup.join(); - }, "Validate triggered"); + } + Assert.assertTrue("Metric CoordinatorBehindRing should have been bumped by at least one replica", metricTriggered); - for (Visit visit : history) - executor.execute(visit); + cluster.filters().reset(); + unpauseCommits(cmsInstance); + startup.join(); }); } catch (Throwable t) diff --git a/test/harry/main/org/apache/cassandra/harry/execution/InJvmDTestVisitExecutor.java b/test/harry/main/org/apache/cassandra/harry/execution/InJvmDTestVisitExecutor.java index 8362a3111bc6..dca2b7001f6d 100644 --- a/test/harry/main/org/apache/cassandra/harry/execution/InJvmDTestVisitExecutor.java +++ b/test/harry/main/org/apache/cassandra/harry/execution/InJvmDTestVisitExecutor.java @@ -296,6 +296,7 @@ public interface RetryPolicy AssertionUtils.isInstanceof(RequestTimeoutException.class) .matches(Throwables.getRootCause(t)); }; + RetryPolicy NO_RETRY = (t) -> false; boolean retry(Throwable t); } From 3ddccf45213f69ac282690186484ae7b9c0151ba Mon Sep 17 00:00:00 2001 From: David Capwell <dcapwell@apache.org> Date: Thu, 23 Jan 2025 16:02:42 -0800 Subject: [PATCH 099/225] CBUtil serialization of UTF8 does not handle all UTF8 properly patch by David Capwell; reviewed by Yifan Cai for CASSANDRA-20234 --- CHANGES.txt | 1 + .../org/apache/cassandra/db/TypeSizes.java | 10 ++++++++ .../apache/cassandra/transport/CBUtil.java | 23 +++++++++++++++---- .../cassandra/transport/CBUtilTest.java | 19 +++++++++++++++ 4 files changed, 48 insertions(+), 5 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 67ad77673e57..8388efa3315f 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0.16 + * CBUtil serialization of UTF8 does not handle all UTF8 properly (CASSANDRA-20234) * Make hint expiry use request start time rather than timeout time for TTL (CASSANDRA-20014) * Do not attach rows and partitions to CheckForAbort when already attached (CASSANDRA-20135) * Allow hint delivery during schema mismatch (CASSANDRA-20188) diff --git a/src/java/org/apache/cassandra/db/TypeSizes.java b/src/java/org/apache/cassandra/db/TypeSizes.java index 8df83b6ba2cd..54ed3eeac882 100644 --- a/src/java/org/apache/cassandra/db/TypeSizes.java +++ b/src/java/org/apache/cassandra/db/TypeSizes.java @@ -45,6 +45,16 @@ public static int sizeof(String value) return sizeof((short) length) + length; } + /** + * Java uses a Modified UTF-8 (see {@link java.io.DataOutput#writeUTF(String)}), and this method attempts to + * calculate the modified utf-8 length; this method only works when the utf-8 writing logic is java's modified utf-8 + * and will not work when normal utf-8 is written. + * + * If normal utf-8 is written, then {@link org.apache.cassandra.transport.CBUtil#encodedUTF8Length(String)} should be + * used instread of this one. + * + * @see <a href="https://docs.oracle.com/en/java/javase/23/docs/api/java.base/java/io/DataInput.html#modified-utf-8">Modified UTF 8</a> + */ public static int encodedUTF8Length(String st) { int strlen = st.length(); diff --git a/src/java/org/apache/cassandra/transport/CBUtil.java b/src/java/org/apache/cassandra/transport/CBUtil.java index fd6a0ff7c322..bbb880245392 100644 --- a/src/java/org/apache/cassandra/transport/CBUtil.java +++ b/src/java/org/apache/cassandra/transport/CBUtil.java @@ -41,7 +41,6 @@ import io.netty.util.concurrent.FastThreadLocal; import org.apache.cassandra.config.Config; import org.apache.cassandra.db.ConsistencyLevel; -import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.Pair; import org.apache.cassandra.utils.UUIDGen; @@ -149,14 +148,28 @@ public static void writeAsciiString(String str, ByteBuf cb) public static void writeString(String str, ByteBuf cb) { - int length = TypeSizes.encodedUTF8Length(str); + int length = encodedUTF8Length(str); cb.writeShort(length); ByteBufUtil.reserveAndWriteUtf8(cb, str, length); } public static int sizeOfString(String str) { - return 2 + TypeSizes.encodedUTF8Length(str); + return 2 + encodedUTF8Length(str); + } + + /** + * Java uses a Modified UTF-8, whereas Netty uses UTF-8 proper... this means that the encoded UTF8 lengths + * do not match, and you must be careful to use the correct length method... + * + * When using {@link ByteBufUtil#reserveAndWriteUtf8(ByteBuf, CharSequence, int)} or similiar logic, you must use + * this method. When using {@link java.io.DataOutput#writeUTF(String)} you must use {@link org.apache.cassandra.db.TypeSizes#encodedUTF8Length(String)}. + * + * @see <a href="https://docs.oracle.com/en/java/javase/23/docs/api/java.base/java/io/DataInput.html#modified-utf-8">Modified UTF 8</a> + */ + public static int encodedUTF8Length(String str) + { + return ByteBufUtil.utf8Bytes(str); } /** @@ -184,14 +197,14 @@ public static String readLongString(ByteBuf cb) public static void writeLongString(String str, ByteBuf cb) { - int length = TypeSizes.encodedUTF8Length(str); + int length = encodedUTF8Length(str); cb.writeInt(length); ByteBufUtil.reserveAndWriteUtf8(cb, str, length); } public static int sizeOfLongString(String str) { - return 4 + TypeSizes.encodedUTF8Length(str); + return 4 + encodedUTF8Length(str); } public static byte[] readBytes(ByteBuf cb) diff --git a/test/unit/org/apache/cassandra/transport/CBUtilTest.java b/test/unit/org/apache/cassandra/transport/CBUtilTest.java index c20efec8f83d..9e0cd74b8385 100644 --- a/test/unit/org/apache/cassandra/transport/CBUtilTest.java +++ b/test/unit/org/apache/cassandra/transport/CBUtilTest.java @@ -25,6 +25,10 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.PooledByteBufAllocator; +import org.assertj.core.api.Assertions; +import org.quicktheories.generators.SourceDSL; + +import static org.quicktheories.QuickTheory.qt; public class CBUtilTest { @@ -38,6 +42,21 @@ public void tearDown() buf.release(buf.refCnt()); } + @Test + public void stringList() + { + qt().forAll(SourceDSL.lists().of(SourceDSL.strings().allPossible().ofLengthBetween(0, 20)).ofSizeBetween(0, 10)).checkAssert(list -> { + int expectedSize = CBUtil.sizeOfStringList(list); + ByteBuf body = CBUtil.allocator.buffer(expectedSize * 2); + CBUtil.writeStringList(list, body); + Assertions.assertThat(body.readableBytes()).isEqualTo(expectedSize); + // In CASSANDRA-20234 the sizeOf method now reflects the write method, but the work to do this in read + // was not done; read took a stance that it wants to limit things in CASSANDRA-8101 but that never made + // it to the other methods... write and read are not compatable with the full domain of UTF-8, and fixing + // that was higher bar than CASSANDRA-20234 wanted to tackle out of fear of unknown regressions it would cause. + }); + } + @Test public void writeAndReadString() { From 2a623b6cf3a3923ac64b0542c06d8b518a09b8cb Mon Sep 17 00:00:00 2001 From: Brad Schoening <bschoeni@gmail.com> Date: Fri, 24 Jan 2025 16:00:00 -0500 Subject: [PATCH 100/225] Fix asciidoc markup in cql_singlefile.adoc patch by Brad Schoening; reviewed by Michael Semb Wever for CASSANDRA-20195 --- .../pages/developing/cql/cql_singlefile.adoc | 1621 +++++++++-------- 1 file changed, 884 insertions(+), 737 deletions(-) diff --git a/doc/modules/cassandra/pages/developing/cql/cql_singlefile.adoc b/doc/modules/cassandra/pages/developing/cql/cql_singlefile.adoc index 18dd52e13dd7..e7af76d077c4 100644 --- a/doc/modules/cassandra/pages/developing/cql/cql_singlefile.adoc +++ b/doc/modules/cassandra/pages/developing/cql/cql_singlefile.adoc @@ -1,6 +1,7 @@ = Cassandra Query Language (CQL) v3.4.3 -\{toc:maxLevel=3} +:toc: +:toclevels: 3 == CQL Syntax @@ -28,7 +29,8 @@ conventions in this document: http://en.wikipedia.org/wiki/Backus%E2%80%93Naur_Form[BNF] -like notation: -bc(syntax). ::= TERMINAL +[source,bnf] +::= TERMINAL * Nonterminal symbols will have `<angle brackets>`. * As additional shortcut notations to BNF, we’ll use traditional regular @@ -43,7 +45,8 @@ though the provided grammar in this document suggest it is not supported. * Sample code will be provided in a code block: -bc(sample). SELECT sample_usage FROM cql; +[source,sql] +SELECT sample_usage FROM cql; * References to keywords or pieces of CQL code in running text will be shown in a `fixed-width font`. @@ -127,7 +130,7 @@ include::cassandra:example$BNF/term.bnf[] A term is thus one of: -* A xref:cassandra:developing/cql/defintions.adoc#constants[constant] +* A xref:cassandra:developing/cql/definitions.adoc#constants[constant] * A literal for either a xref:cassandra:developing/cql/types.adoc#collections[collection], a xref:cassandra:developing/cql/types.adoc#vectors[vector], a xref:cassandra:developing/cql/types.adoc#udts[user-defined type] or a xref:cassandra:developing/cql/types.adoc#tuples[tuple] * A xref:cassandra:developing/cql/functions.adoc#cql-functions[function] call, either a xref:cassandra:developing/cql/functions.adoc#scalar-native-functions[native function] @@ -145,14 +148,16 @@ and should generally be preferred. A comment in CQL is a line beginning by either double dashes (`--`) or double slash (`//`). -Multi-line comments are also supported through enclosure within `/*` and -`*/` (but nesting is not supported). +Multi-line comments are also supported through enclosure within `++/*++` and +`++*/++` (but nesting is not supported). -bc(sample). + -— This is a comment + -// This is a comment too + -/* This is + +[source,sql] +---- +-- This is a comment +// This is a comment too +/* This is a multi-line comment */ +---- === Statements @@ -169,42 +174,44 @@ omitted when dealing with a single statement. The supported statements are described in the following sections. When describing the grammar of said statements, we will reuse the non-terminal symbols defined below: -bc(syntax).. + -::= any quoted or unquoted identifier, excluding reserved keywords + +[source,bnf] +---- +::= any quoted or unquoted identifier, excluding reserved keywords ::= ( `.')? -::= a string constant + -::= an integer constant + -::= a float constant + -::= | + -::= a uuid constant + -::= a boolean constant + +::= a string constant +::= an integer constant +::= a float constant +::= | +::= a uuid constant +::= a boolean constant ::= a blob constant -::= + -| + -| + -| + -| + -::= `?' + -| `:' + -::= + -| + -| + +::= +| +| +| +| +::= `?' +| `:' +::= +| +| | `(' ( (`,' )*)? `)' -::= + -| + -| + -::= `\{' ( `:' ( `,' `:' )* )? `}' + -::= `\{' ( ( `,' )* )? `}' + +::= +| +| +::= `\{' ( `:' ( `,' `:' )* )? `}' +::= `\{' ( ( `,' )* )? `}' ::= `[' ( ( `,' )* )? `]' ::= -::= (AND )* + -::= `=' ( | | ) + -p. + +::= (AND )* +::= `=' ( | | ) +---- + Please note that not every possible productions of the grammar above will be valid in practice. Most notably, `<variable>` and nested `<collection-literal>` are currently not allowed inside @@ -238,11 +245,13 @@ code fragments for user-defined functions. _Sample:_ -bc(sample).. + -`some string value' +[source,text] +---- +'some string value' +$$A King's ransom$$ +---- -$$double-dollar string can contain single ’ quotes$$ + -p. +$$double-dollar string can contain single ’ quotes$$ [[preparedStatement]] === Prepared Statement @@ -272,21 +281,23 @@ bind markers are used, the names for the query parameters will be _Syntax:_ -bc(syntax).. + -::= CREATE KEYSPACE (IF NOT EXISTS)? WITH + -p. + +[source,bnf] +::= CREATE KEYSPACE (IF NOT EXISTS)? WITH + _Sample:_ -bc(sample).. + -CREATE KEYSPACE Excelsior + -WITH replication = \{’class’: `SimpleStrategy', `replication_factor' : +[source,sql] +---- +CREATE KEYSPACE Excelsior +WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 3}; -CREATE KEYSPACE Excalibur + -WITH replication = \{’class’: `NetworkTopologyStrategy', `DC1' : 1, -`DC2' : 3} + -AND durable_writes = false; + -p. + +CREATE KEYSPACE Excalibur +WITH replication = {'class': 'NetworkTopologyStrategy', 'DC1' : 1, +'DC2' : 3} +AND durable_writes = false; +---- + The `CREATE KEYSPACE` statement creates a new top-level _keyspace_. A keyspace is a namespace that defines a replication strategy and some options for a set of tables. Valid keyspaces names are identifiers @@ -330,11 +341,13 @@ will be a no-op if the keyspace already exists. _Syntax:_ -bc(syntax). ::= USE +[source,bnf] +::= USE _Sample:_ -bc(sample). USE myApp; +[source,sql] +USE myApp; The `USE` statement takes an existing keyspace name as argument and set it as the per-connection current working keyspace. All subsequent @@ -347,14 +360,14 @@ another USE statement is issued or the connection terminates. _Syntax:_ -bc(syntax).. + -::= ALTER KEYSPACE (IF EXISTS)? WITH + -p. + +[source,bnf] +::= ALTER KEYSPACE (IF EXISTS)? WITH + _Sample:_ -bc(sample).. + -ALTER KEYSPACE Excelsior + -WITH replication = \{’class’: `SimpleStrategy', `replication_factor' : +[source,sql] +ALTER KEYSPACE Excelsior +WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 4}; The `ALTER KEYSPACE` statement alters the properties of an existing @@ -366,11 +379,13 @@ link:#createKeyspaceStmt[`CREATE KEYSPACE`] statement. _Syntax:_ -bc(syntax). ::= DROP KEYSPACE ( IF EXISTS )? +[source,bnf] +::= DROP KEYSPACE ( IF EXISTS )? _Sample:_ -bc(sample). DROP KEYSPACE myApp; +[source,sql] + DROP KEYSPACE myApp; A `DROP KEYSPACE` statement results in the immediate, irreversible removal of an existing keyspace, including all column families in it, @@ -384,40 +399,43 @@ unless `IF EXISTS` is used in which case the operation is a no-op. _Syntax:_ -bc(syntax).. + -::= CREATE ( TABLE | COLUMNFAMILY ) ( IF NOT EXISTS )? + -`(' ( `,' )* `)' + +[source,bnf] +---- +::= CREATE ( TABLE | COLUMNFAMILY ) ( IF NOT EXISTS )? +`(' ( `,' )* `)' ( WITH ( AND )* )? -::= ( STATIC )? ( PRIMARY KEY )? + +::= ( STATIC )? ( PRIMARY KEY )? | PRIMARY KEY `(' ( `,' )* `)' -::= + +::= | `(' (`,' )* `)' -::= + -| COMPACT STORAGE + -| CLUSTERING ORDER + -p. + +::= +| COMPACT STORAGE +| CLUSTERING ORDER +---- + _Sample:_ -bc(sample).. + -CREATE TABLE monkeySpecies ( + -species text PRIMARY KEY, + -common_name text, + -population varint, + -average_size int + -) WITH comment=`Important biological records'; - -CREATE TABLE timeline ( + -userid uuid, + -posted_month int, + -posted_time uuid, + -body text, + -posted_by text, + -PRIMARY KEY (userid, posted_month, posted_time) + -) WITH compaction = \{ `class' : `LeveledCompactionStrategy' }; + -p. + +[source,sql] +---- +CREATE TABLE monkeySpecies ( +species text PRIMARY KEY, +common_name text, +population varint, +average_size int +) WITH comment='Important biological records'; + +CREATE TABLE timeline ( +userid uuid, +posted_month int, +posted_time uuid, +body text, +posted_by text, +PRIMARY KEY (userid, posted_month, posted_time) +) WITH compaction = { 'class' : 'LeveledCompactionStrategy' }; +---- The `CREATE TABLE` statement creates a new table. Each such table is a set of _rows_ (usually representing related entities) for which it defines a number of properties. A table is defined by a @@ -458,20 +476,24 @@ Otherwise, it must be specified by following `PRIMARY KEY` by the comma-separated list of column names composing the key within parenthesis. Note that: -bc(sample). + -CREATE TABLE t ( + -k int PRIMARY KEY, + -other text + +[source,sql] +---- +CREATE TABLE t ( +k int PRIMARY KEY, +other text ) +---- is equivalent to -bc(sample). + -CREATE TABLE t ( + -k int, + -other text, + -PRIMARY KEY (k) + +[source,sql] +---- +CREATE TABLE t ( +k int, +other text, +PRIMARY KEY (k) ) +---- [[createTablepartitionClustering]] ==== Partition key and clustering columns @@ -499,17 +521,19 @@ Some columns can be declared as `STATIC` in a table definition. A column that is static will be ``shared'' by all the rows belonging to the same partition (having the same partition key). For instance, in: -bc(sample). + -CREATE TABLE test ( + -pk int, + -t int, + -v text, + -s text static, + -PRIMARY KEY (pk, t) + -); + -INSERT INTO test(pk, t, v, s) VALUES (0, 0, `val0', `static0'); + -INSERT INTO test(pk, t, v, s) VALUES (0, 1, `val1', `static1'); + +[source,sql] +---- +CREATE TABLE test ( +pk int, +t int, +v text, +s text static, +PRIMARY KEY (pk, t) +); +INSERT INTO test(pk, t, v, s) VALUES (0, 0, 'val0', 'static0'); +INSERT INTO test(pk, t, v, s) VALUES (0, 1, 'val1', 'static1'); SELECT * FROM test WHERE pk=0 AND t=0; +---- the last query will return `'static1'` as value for `s`, since `s` is static and thus the 2nd insertion modified this ``shared'' value. Note @@ -725,27 +749,31 @@ haven’t) when creating a table. _Syntax:_ -bc(syntax).. + +[source,bnf] +---- ::= ALTER (TABLE | COLUMNFAMILY) (IF EXISTS)? -::= ADD (IF NOT EXISTS)? + -| ADD (IF NOT EXISTS)? ( ( , )* ) + -| DROP (IF EXISTS)? + -| DROP (IF EXISTS)? ( ( , )* ) + -| RENAME (IF EXISTS)? TO (AND TO)* + -| WITH ( AND )* + -p. + +::= ADD (IF NOT EXISTS)? +| ADD (IF NOT EXISTS)? ( ( , )* ) +| DROP (IF EXISTS)? +| DROP (IF EXISTS)? ( ( , )* ) +| RENAME (IF EXISTS)? TO (AND TO)* +| WITH ( AND )* +---- + _Sample:_ -bc(sample).. + +[source,sql] +---- ALTER TABLE addamsFamily -ALTER TABLE addamsFamily + +ALTER TABLE addamsFamily ADD gravesite varchar; -ALTER TABLE addamsFamily + -WITH comment = `A most excellent and useful column family'; + -p. + +ALTER TABLE addamsFamily +WITH comment = 'A most excellent and useful column family'; +---- + The `ALTER` statement is used to manipulate table definitions. It allows for adding new columns, dropping existing ones, or updating the table options. As with table creation, `ALTER COLUMNFAMILY` is allowed as an @@ -824,11 +852,13 @@ conversions are allowed. _Syntax:_ -bc(syntax). ::= DROP TABLE ( IF EXISTS )? +[source,bnf] +::= DROP TABLE ( IF EXISTS )? _Sample:_ -bc(sample). DROP TABLE worldSeriesAttendees; +[source,sql] + DROP TABLE worldSeriesAttendees; The `DROP TABLE` statement results in the immediate, irreversible removal of a table, including all data contained in it. As for table @@ -842,11 +872,13 @@ If the table does not exist, the statement will return an error, unless _Syntax:_ -bc(syntax). ::= TRUNCATE ( TABLE | COLUMNFAMILY )? +[source,bnf] +::= TRUNCATE ( TABLE | COLUMNFAMILY )? _Sample:_ -bc(sample). TRUNCATE superImportantData; +[source,sql] + TRUNCATE superImportantData; The `TRUNCATE` statement permanently removes all data from a table. @@ -860,24 +892,28 @@ desired. _Syntax:_ -bc(syntax).. + -::= CREATE ( CUSTOM )? INDEX ( IF NOT EXISTS )? ( )? + -ON `(' `)' + +[source,bnf] +---- +::= CREATE ( CUSTOM )? INDEX ( IF NOT EXISTS )? ( )? +ON `(' `)' ( USING ( WITH OPTIONS = )? )? -::= + -| keys( ) + -p. + +::= +| keys( ) +---- + _Sample:_ -bc(sample). + -CREATE INDEX userIndex ON NerdMovies (user); + -CREATE INDEX ON Mutants (abilityId); + -CREATE INDEX ON users (keys(favs)); + -CREATE INDEX ON users (age) USING 'sai'; + -CREATE CUSTOM INDEX ON users (email) USING `path.to.the.IndexClass'; + -CREATE CUSTOM INDEX ON users (email) USING `path.to.the.IndexClass' WITH -OPTIONS = \{’storage’: `/mnt/ssd/indexes/'}; +[source,sql] +---- +CREATE INDEX userIndex ON NerdMovies (user); +CREATE INDEX ON Mutants (abilityId); +CREATE INDEX ON users (keys(favs)); +CREATE INDEX ON users (age) USING 'sai'; +CREATE CUSTOM INDEX ON users (email) USING 'path.to.the.IndexClass'; +CREATE CUSTOM INDEX ON users (email) USING 'path.to.the.IndexClass' WITH +OPTIONS = {'storage': '/mnt/ssd/indexes/'}; +---- If data already exists for the column, it will be indexed asynchronously. After the index is created, new data for the column is @@ -909,15 +945,17 @@ the map values. _Syntax:_ -bc(syntax). ::= DROP INDEX ( IF EXISTS )? ( `.' )? +[source,bnf] +DROP INDEX ( IF EXISTS )? ('.')? _Sample:_ -bc(sample).. + +[source,sql] +---- DROP INDEX userIndex; +DROP INDEX userkeyspace.address_index; +---- -DROP INDEX userkeyspace.address_index; + -p. + The `DROP INDEX` statement is used to drop an existing secondary index. The argument of the statement is the index name, which may optionally specify the keyspace of the index. @@ -930,24 +968,28 @@ If the index does not exists, the statement will return an error, unless _Syntax:_ -bc(syntax).. + -::= CREATE MATERIALIZED VIEW ( IF NOT EXISTS )? AS + -SELECT ( `(' ( `,' ) * `)' | `*' ) + -FROM + -( WHERE )? + -PRIMARY KEY `(' ( `,' )* `)' + -( WITH ( AND )* )? + -p. + +[source,bnf] +---- +::= CREATE MATERIALIZED VIEW ( IF NOT EXISTS )? AS +SELECT ( `(' ( `,' ) * `)' | `*' ) +FROM +( WHERE )? +PRIMARY KEY `(' ( `,' )* `)' +( WITH ( AND )* )? +---- + _Sample:_ -bc(sample).. + -CREATE MATERIALIZED VIEW monkeySpecies_by_population AS + -SELECT * + -FROM monkeySpecies + -WHERE population IS NOT NULL AND species IS NOT NULL + -PRIMARY KEY (population, species) + -WITH comment=`Allow query by population instead of species'; + -p. + +[source,sql] +---- +CREATE MATERIALIZED VIEW monkeySpecies_by_population AS +SELECT * +FROM monkeySpecies +WHERE population IS NOT NULL AND species IS NOT NULL +PRIMARY KEY (population, species) +WITH comment='Allow query by population instead of species'; +---- + The `CREATE MATERIALIZED VIEW` statement creates a new materialized view. Each such view is a set of _rows_ which corresponds to rows which are present in the underlying, or base, table specified in the `SELECT` @@ -975,8 +1017,8 @@ may be lifted in the future.) _Syntax:_ -bc(syntax). ::= ALTER MATERIALIZED VIEW + -WITH ( AND )* +[source,bnf] +::= ALTER MATERIALIZED VIEW WITH ( AND )* The `ALTER MATERIALIZED VIEW` statement allows options to be update; these options are the same as `CREATE TABLE`’s options. @@ -986,11 +1028,13 @@ these options are the same as `CREATE TABLE`’s options. _Syntax:_ -bc(syntax). ::= DROP MATERIALIZED VIEW ( IF EXISTS )? +[source,bnf] +::= DROP MATERIALIZED VIEW ( IF EXISTS )? _Sample:_ -bc(sample). DROP MATERIALIZED VIEW monkeySpecies_by_population; +[source,sql] +DROP MATERIALIZED VIEW monkeySpecies_by_population; The `DROP MATERIALIZED VIEW` statement is used to drop an existing materialized view. @@ -1004,30 +1048,34 @@ no-op. _Syntax:_ -bc(syntax).. + -::= CREATE TYPE ( IF NOT EXISTS )? + +[source,bnf] +---- +::= CREATE TYPE ( IF NOT EXISTS )? `(' ( `,' )* `)' ::= ( `.' )? ::= +---- _Sample:_ -bc(sample).. + -CREATE TYPE address ( + -street_name text, + -street_number int, + -city text, + -state text, + -zip int + +[source,sql] +---- +CREATE TYPE address ( +street_name text, +street_number int, +city text, +state text, +zip int +) + +CREATE TYPE work_and_home_addresses ( +home_address address, +work_address address ) +---- -CREATE TYPE work_and_home_addresses ( + -home_address address, + -work_address address + -) + -p. + The `CREATE TYPE` statement creates a new user-defined type. Each type is a set of named, typed fields. Field types may be any valid type, including collections and other existing user-defined types. @@ -1052,19 +1100,22 @@ keyspace. _Syntax:_ -bc(syntax).. + +[source,bnf] +---- ::= ALTER TYPE (IF EXISTS)? -::= ADD (IF NOT EXISTS)? + -| RENAME (IF EXISTS)? TO ( AND TO )* + -p. + +::= ADD (IF NOT EXISTS)? +| RENAME (IF EXISTS)? TO ( AND TO )* +---- + _Sample:_ -bc(sample).. + +[source,sql] +---- ALTER TYPE address ADD country text +ALTER TYPE address RENAME zip TO zipcode AND street_name TO street +---- -ALTER TYPE address RENAME zip TO zipcode AND street_name TO street + -p. + The `ALTER TYPE` statement is used to manipulate type definitions. It allows for adding new fields, renaming existing fields, or changing the type of existing fields. If the type does not exist, the statement will return an error, unless `IF EXISTS` is used in which case the operation is a no-op. @@ -1074,9 +1125,9 @@ type of existing fields. If the type does not exist, the statement will return a _Syntax:_ -bc(syntax).. + -::= DROP TYPE ( IF EXISTS )? + -p. + +[source,bnf] +::= DROP TYPE ( IF EXISTS )? + The `DROP TYPE` statement results in the immediate, irreversible removal of a type. Attempting to drop a type that is still in use by another type or a table will result in an error. @@ -1089,16 +1140,14 @@ is used, in which case the operation is a no-op. _Syntax:_ -bc(syntax).. + -::= CREATE TRIGGER ( IF NOT EXISTS )? ( )? + -ON + -USING +[source,bnf] +::= CREATE TRIGGER ( IF NOT EXISTS )? ( )? ON USING _Sample:_ -bc(sample). + +[source,sql] CREATE TRIGGER myTrigger ON myTable USING -`org.apache.cassandra.triggers.InvertedIndex'; +'org.apache.cassandra.triggers.InvertedIndex'; The actual logic that makes up the trigger can be written in any Java (JVM) language and exists outside the database. You place the trigger @@ -1113,13 +1162,12 @@ the transaction. _Syntax:_ -bc(syntax).. + -::= DROP TRIGGER ( IF EXISTS )? ( )? + -ON + -p. + +[source,bnf] +::= DROP TRIGGER ( IF EXISTS )? ( )? ON + _Sample:_ -bc(sample). + +[source,sql] DROP TRIGGER myTrigger ON myTable; `DROP TRIGGER` statement removes the registration of a trigger created @@ -1130,34 +1178,36 @@ using `CREATE TRIGGER`. _Syntax:_ -bc(syntax).. + -::= CREATE ( OR REPLACE )? + -FUNCTION ( IF NOT EXISTS )? + -( `.' )? + -`(' ( `,' )* `)' + -( CALLED | RETURNS NULL ) ON NULL INPUT + -RETURNS + -LANGUAGE + +[source,bnf] +---- +::= CREATE ( OR REPLACE )? +FUNCTION ( IF NOT EXISTS )? +( `.' )? +`(' ( `,' )* `)' +( CALLED | RETURNS NULL ) ON NULL INPUT +RETURNS +LANGUAGE AS +---- _Sample:_ -bc(sample). + -CREATE OR REPLACE FUNCTION somefunction + -( somearg int, anotherarg text, complexarg frozen, listarg list ) + -RETURNS NULL ON NULL INPUT + -RETURNS text + -LANGUAGE java + -AS $$ + -// some Java code + -$$; + -CREATE FUNCTION akeyspace.fname IF NOT EXISTS + -( someArg int ) + -CALLED ON NULL INPUT + -RETURNS text + -LANGUAGE java + -AS $$ + -// some Java code + +[source,sql] +CREATE OR REPLACE FUNCTION somefunction +( somearg int, anotherarg text, complexarg frozen, listarg list ) +RETURNS NULL ON NULL INPUT +RETURNS text +LANGUAGE java +AS $$ +// some Java code +$$; +CREATE FUNCTION akeyspace.fname IF NOT EXISTS +( someArg int ) +CALLED ON NULL INPUT +RETURNS text +LANGUAGE java +AS $$ +// some Java code $$; `CREATE FUNCTION` creates or replaces a user-defined function. @@ -1207,18 +1257,20 @@ information. _Syntax:_ -bc(syntax).. + -::= DROP FUNCTION ( IF EXISTS )? + -( `.' )? + +[source,bnf] +::= DROP FUNCTION ( IF EXISTS )? +( `.' )? ( `(' ( `,' )* `)' )? _Sample:_ -bc(sample). + -DROP FUNCTION myfunction; + -DROP FUNCTION mykeyspace.afunction; + -DROP FUNCTION afunction ( int ); + +[source,sql] +---- +DROP FUNCTION myfunction; +DROP FUNCTION mykeyspace.afunction; +DROP FUNCTION afunction ( int ); DROP FUNCTION afunction ( text ); +---- `DROP FUNCTION` statement removes a function created using `CREATE FUNCTION`. + @@ -1234,24 +1286,28 @@ if it exists. _Syntax:_ -bc(syntax).. + -::= CREATE ( OR REPLACE )? + -AGGREGATE ( IF NOT EXISTS )? + -( `.' )? + -`(' ( `,' )* `)' + -SFUNC + -STYPE + -( FINALFUNC )? + -( INITCOND )? + -p. + +[source,bnf] +---- +::= CREATE ( OR REPLACE )? +AGGREGATE ( IF NOT EXISTS )? +( `.' )? +`(' ( `,' )* `)' +SFUNC +STYPE +( FINALFUNC )? +( INITCOND )? +---- + _Sample:_ -bc(sample). + -CREATE AGGREGATE myaggregate ( val text ) + -SFUNC myaggregate_state + -STYPE text + -FINALFUNC myaggregate_final + -INITCOND `foo'; +[source,sql] +---- +CREATE AGGREGATE myaggregate ( val text ) +SFUNC myaggregate_state +STYPE text +FINALFUNC myaggregate_final +INITCOND 'foo'; +---- See the section on link:#udas[user-defined aggregates] for a complete example. @@ -1308,19 +1364,22 @@ information. _Syntax:_ -bc(syntax).. + -::= DROP AGGREGATE ( IF EXISTS )? + -( `.' )? + -( `(' ( `,' )* `)' )? + -p. +[source,bnf] +---- +::= DROP AGGREGATE ( IF EXISTS )? +( `.' )? +( `(' ( `,' )* `)' )? +---- _Sample:_ -bc(sample). + -DROP AGGREGATE myAggregate; + -DROP AGGREGATE myKeyspace.anAggregate; + -DROP AGGREGATE someAggregate ( int ); + +[source,sql] +---- +DROP AGGREGATE myAggregate; +DROP AGGREGATE myKeyspace.anAggregate; +DROP AGGREGATE someAggregate ( int ); DROP AGGREGATE someAggregate ( text ); +---- The `DROP AGGREGATE` statement removes an aggregate created using `CREATE AGGREGATE`. You must specify the argument types of the aggregate @@ -1342,30 +1401,34 @@ link:#functionSignature[same rules] as for user-defined functions. _Syntax:_ -bc(syntax).. + -::= INSERT INTO + -( ( VALUES ) + -| ( JSON )) + -( IF NOT EXISTS )? + +[source,bnf] +---- +::= INSERT INTO +( ( VALUES ) +| ( JSON )) +( IF NOT EXISTS )? ( USING ( AND )* )? ::= `(' ( `,' )* `)' ::= `(' ( `,' )* `)' -::= TIMESTAMP + -| TTL + -p. + +::= TIMESTAMP +| TTL +---- + _Sample:_ -bc(sample).. + -INSERT INTO NerdMovies (movie, director, main_actor, year) + -VALUES (`Serenity', `Joss Whedon', `Nathan Fillion', 2005) + +[source,sql] +---- +INSERT INTO NerdMovies (movie, director, main_actor, year) +VALUES ('Serenity', 'Joss Whedon', 'Nathan Fillion', 2005) USING TTL 86400; -INSERT INTO NerdMovies JSON `\{``movie'': ``Serenity'', ``director'': -``Joss Whedon'', ``year'': 2005}' + -p. + +INSERT INTO NerdMovies JSON '{"movie": "Serenity", "director": +"Joss Whedon", "year": 2005}' +---- + The `INSERT` statement writes one or more columns for a given row in a table. Note that since a row is identified by its `PRIMARY KEY`, at least the columns composing it must be specified. The list of columns to @@ -1396,54 +1459,58 @@ counters, while `UPDATE` does. _Syntax:_ -bc(syntax).. + -::= UPDATE + -( USING ( AND )* )? + -SET ( `,' )* + -WHERE + +[source,bnf] +---- +::= UPDATE +( USING ( AND )* )? +SET ( `,' )* +WHERE ( IF ( AND condition )* )? -::= `=' + -| `=' (`+' | `-') ( | | ) + -| `=' `+' + -| `[' `]' `=' + +::= `=' +| `=' (`+' | `-') ( | | ) +| `=' `+' +| `[' `]' `=' | `.' `=' -::= + -| CONTAINS (KEY)? + -| IN + -| `[' `]' + -| `[' `]' IN + -| `.' + +::= +| CONTAINS (KEY)? +| IN +| `[' `]' +| `[' `]' IN +| `.' | `.' IN -::= `<' | `<=' | `=' | `!=' | `>=' | `>' + +::= `<' | `<=' | `=' | `!=' | `>=' | `>' ::= ( | `(' ( ( `,' )* )? `)') ::= ( AND )* -::= `=' + -| `(' (`,' )* `)' `=' + -| IN `(' ( ( `,' )* )? `)' + -| IN + -| `(' (`,' )* `)' IN `(' ( ( `,' )* )? `)' + +::= `=' +| `(' (`,' )* `)' `=' +| IN `(' ( ( `,' )* )? `)' +| IN +| `(' (`,' )* `)' IN `(' ( ( `,' )* )? `)' | `(' (`,' )* `)' IN -::= TIMESTAMP + -| TTL + -p. + +::= TIMESTAMP +| TTL +---- + _Sample:_ -bc(sample).. + -UPDATE NerdMovies USING TTL 400 + -SET director = `Joss Whedon', + -main_actor = `Nathan Fillion', + -year = 2005 + -WHERE movie = `Serenity'; +[source,sql] +---- +UPDATE NerdMovies USING TTL 400 +SET director = 'Joss Whedon', +main_actor = 'Nathan Fillion', +year = 2005 +WHERE movie = 'Serenity'; UPDATE UserActions SET total = total + 2 WHERE user = -B70DE1D0-9908-4AE3-BE34-5573E5B09F14 AND action = `click'; + -p. + +B70DE1D0-9908-4AE3-BE34-5573E5B09F14 AND action = 'click'; +---- + The `UPDATE` statement writes one or more columns for a given row in a table. The `<where-clause>` is used to select the row to update and must include all columns composing the `PRIMARY KEY`. Other columns values @@ -1499,47 +1566,51 @@ TTL of 0 will remove the TTL for the inserted or updated values. _Syntax:_ -bc(syntax).. + -::= DELETE ( ( `,' )* )? + -FROM + -( USING TIMESTAMP )? + -WHERE + +[source,bnf] +---- +::= DELETE ( ( `,' )* )? +FROM +( USING TIMESTAMP )? +WHERE ( IF ( EXISTS | ( ( AND )*) ) )? -::= + -| `[' `]' + +::= +| `[' `]' | `.' ::= ( AND )* -::= + -| `(' (`,' )* `)' + -| IN `(' ( ( `,' )* )? `)' + -| IN + -| `(' (`,' )* `)' IN `(' ( ( `,' )* )? `)' + +::= +| `(' (`,' )* `)' +| IN `(' ( ( `,' )* )? `)' +| IN +| `(' (`,' )* `)' IN `(' ( ( `,' )* )? `)' | `(' (`,' )* `)' IN -::= `=' | `<' | `>' | `<=' | `>=' + +::= `=' | `<' | `>' | `<=' | `>=' ::= ( | `(' ( ( `,' )* )? `)') -::= ( | `!=') + -| CONTAINS (KEY)? + -| IN + -| `[' `]' ( | `!=') + -| `[' `]' IN + -| `.' ( | `!=') + +::= ( | `!=') +| CONTAINS (KEY)? +| IN +| `[' `]' ( | `!=') +| `[' `]' IN +| `.' ( | `!=') | `.' IN +---- _Sample:_ -bc(sample).. + +[source,sql] +---- DELETE FROM NerdMovies USING TIMESTAMP 1240003134 WHERE movie = -`Serenity'; +'Serenity'; DELETE phone FROM Users WHERE userid IN (C73DE1D3-AF08-40F3-B124-3FF3E5109F22, -B70DE1D0-9908-4AE3-BE34-5573E5B09F14); + -p. + +B70DE1D0-9908-4AE3-BE34-5573E5B09F14); +---- + The `DELETE` statement deletes columns and rows. If column names are provided directly after the `DELETE` keyword, only those columns are deleted from the row indicated by the `<where-clause>`. The `id[value]` @@ -1568,28 +1639,32 @@ sparingly. _Syntax:_ -bc(syntax).. + -::= BEGIN ( UNLOGGED | COUNTER ) BATCH + -( USING ( AND )* )? + -( `;' )* + +[source,bnf] +---- +::= BEGIN ( UNLOGGED | COUNTER ) BATCH +( USING ( AND )* )? +( `;' )* APPLY BATCH -::= + -| + +::= +| | -::= TIMESTAMP + -p. + +::= TIMESTAMP +---- + _Sample:_ -bc(sample). + -BEGIN BATCH + -INSERT INTO users (userid, password, name) VALUES (`user2', `ch@ngem3b', -`second user'); + -UPDATE users SET password = `ps22dhds' WHERE userid = `user3'; + -INSERT INTO users (userid, password) VALUES (`user4', `ch@ngem3c'); + -DELETE name FROM users WHERE userid = `user1'; + +[source,sql] +---- +BEGIN BATCH +INSERT INTO users (userid, password, name) VALUES ('user2', 'ch@ngem3b', +'second user'); +UPDATE users SET password = 'ps22dhds' WHERE userid = 'user3'; +INSERT INTO users (userid, password) VALUES ('user4', 'ch@ngem3c'); +DELETE name FROM users WHERE userid = 'user1'; APPLY BATCH; +---- The `BATCH` statement group multiple modification statements (insertions/updates and deletions) into a single statement. It serves @@ -1651,65 +1726,69 @@ used, `TIMESTAMP` *must not* be used in the statements within the batch. _Syntax:_ -bc(syntax).. + -::= SELECT ( JSON )? + -FROM + -( WHERE )? + -( GROUP BY )? + -( ORDER BY )? + -( PER PARTITION LIMIT )? + -( LIMIT )? + +[source,bnf] +---- +::= SELECT ( JSON )? +FROM +( WHERE )? +( GROUP BY )? +( ORDER BY )? +( PER PARTITION LIMIT )? +( LIMIT )? ( ALLOW FILTERING )? ::= DISTINCT? -::= (AS )? ( `,' (AS )? )* + +::= (AS )? ( `,' (AS )? )* | `*' -::= + -| + -| WRITETIME `(' `)' + -| MAXWRITETIME `(' `)' + -| COUNT `(' `*' `)' + -| TTL `(' `)' + -| CAST `(' AS `)' + -| `(' ( (`,' )*)? `)' + -| `.' + -| `[' `]' + +::= +| +| WRITETIME `(' `)' +| MAXWRITETIME `(' `)' +| COUNT `(' `*' `)' +| TTL `(' `)' +| CAST `(' AS `)' +| `(' ( (`,' )*)? `)' +| `.' +| `[' `]' | `[' ? .. ? `]' ::= ( AND )* -::= + -| `(' (`,' )* `)' + -| IN `(' ( ( `,' )* )? `)' + -| `(' (`,' )* `)' IN `(' ( ( `,' )* )? `)' + +::= +| `(' (`,' )* `)' +| IN `(' ( ( `,' )* )? `)' +| `(' (`,' )* `)' IN `(' ( ( `,' )* )? `)' | TOKEN `(' ( `,' )* `)' -::= `=' | `<' | `>' | `<=' | `>=' | CONTAINS | CONTAINS KEY + -::= (`,' )* + -::= ( `,' )* + -::= ( ASC | DESC )? + -::= `(' (`,' )* `)' + -p. + +::= `=' | `<' | `>' | `<=' | `>=' | CONTAINS | CONTAINS KEY +::= (`,' )* +::= ( `,' )* +::= ( ASC | DESC )? +::= `(' (`,' )* `)' +---- + _Sample:_ -bc(sample).. + +[source,sql] +---- SELECT name, occupation FROM users WHERE userid IN (199, 200, 207); SELECT JSON name, occupation FROM users WHERE userid = 199; SELECT name AS user_name, occupation AS user_occupation FROM users; -SELECT time, value + -FROM events + -WHERE event_type = `myEvent' + -AND time > `2011-02-03' + -AND time <= `2012-01-01' +SELECT time, value +FROM events +WHERE event_type = 'myEvent' +AND time > '2011-02-03' +AND time <= '2012-01-01' SELECT COUNT (*) FROM users; SELECT COUNT (*) AS user_count FROM users; +---- The `SELECT` statements reads one or more columns for one or more rows in a table. It returns a result-set of rows, where each row contains the @@ -1769,31 +1848,37 @@ key, the clustering columns induce an ordering of rows and relations on them is restricted to the relations that allow to select a *contiguous* (for the ordering) set of rows. For instance, given -bc(sample). + -CREATE TABLE posts ( + -userid text, + -blog_title text, + -posted_at timestamp, + -entry_title text, + -content text, + -category int, + -PRIMARY KEY (userid, blog_title, posted_at) + +[source,sql] +---- +CREATE TABLE posts ( +userid text, +blog_title text, +posted_at timestamp, +entry_title text, +content text, +category int, +PRIMARY KEY (userid, blog_title, posted_at) ) +---- The following query is allowed: -bc(sample). + -SELECT entry_title, content FROM posts WHERE userid=`john doe' AND -blog_title=`John'`s Blog' AND posted_at >= `2012-01-01' AND posted_at < -`2012-01-31' +[source,sql] +---- +SELECT entry_title, content FROM posts WHERE userid='john doe' AND +blog_title='John''s Blog' AND posted_at >= '2012-01-01' AND posted_at < +'2012-01-31' +---- But the following one is not, as it does not select a contiguous set of rows (and we suppose no secondary indexes are set): -bc(sample). + -// Needs a blog_title to be set to select ranges of posted_at + -SELECT entry_title, content FROM posts WHERE userid=`john doe' AND -posted_at >= `2012-01-01' AND posted_at < `2012-01-31' +[source,sql] +---- +// Needs a blog_title to be set to select ranges of posted_at +SELECT entry_title, content FROM posts WHERE userid='john doe' AND +posted_at >= '2012-01-01' AND posted_at < '2012-01-31' +---- When specifying relations, the `TOKEN` function can be used on the `PARTITION KEY` column to query. In that case, rows will be selected @@ -1804,9 +1889,9 @@ note that ordering partitioners always order token values by bytes (so even if the partition key is of type int, `token(-1) > token(0)` in particular). Example: -bc(sample). + -SELECT * FROM posts WHERE token(userid) > token(`tom') AND token(userid) -< token(`bob') +[source,sql] +SELECT * FROM posts WHERE token(userid) > token('tom') AND token(userid) +< token('bob') Moreover, the `IN` relation is only allowed on the last column of the partition key and on the last column of the full primary key. @@ -1814,9 +1899,9 @@ partition key and on the last column of the full primary key. It is also possible to ``group'' `CLUSTERING COLUMNS` together in a relation using the tuple notation. For instance: -bc(sample). + -SELECT * FROM posts WHERE userid=`john doe' AND (blog_title, posted_at) -> (`John'`s Blog', `2012-01-01') +[source,sql] +SELECT * FROM posts WHERE userid='john doe' AND (blog_title, posted_at) +> ('John''s Blog', '2012-01-01') will request all rows that sorts after the one having ``John’s Blog'' as `blog_tile` and `2012-01-01' for `posted_at` in the clustering order. In @@ -1824,16 +1909,16 @@ particular, rows having a `post_at <= '2012-01-01'` will be returned as long as their `blog_title > 'John''s Blog'`, which wouldn’t be the case for: -bc(sample). + -SELECT * FROM posts WHERE userid=`john doe' AND blog_title > `John'`s -Blog' AND posted_at > `2012-01-01' +[source,sql] +SELECT * FROM posts WHERE userid='john doe' AND blog_title > 'John''s +Blog' AND posted_at > '2012-01-01' The tuple notation may also be used for `IN` clauses on `CLUSTERING COLUMNS`: -bc(sample). + -SELECT * FROM posts WHERE userid=`john doe' AND (blog_title, posted_at) -IN ((`John'`s Blog', `2012-01-01), (’Extreme Chess', `2014-06-01')) +[source,sql] +SELECT * FROM posts WHERE userid='john doe' AND (blog_title, posted_at) +IN (('John''s Blog', '2012-01-01'), ('Extreme Chess', '2014-06-01')) The `CONTAINS` operator may only be used on collection columns (lists, sets, and maps). In the case of maps, `CONTAINS` applies to the map @@ -1906,23 +1991,26 @@ For instance, considering the following table holding user profiles with their year of birth (with a secondary index on it) and country of residence: -bc(sample).. + -CREATE TABLE users ( + -username text PRIMARY KEY, + -firstname text, + -lastname text, + -birth_year int, + -country text + +[source,sql] +---- +CREATE TABLE users ( +username text PRIMARY KEY, +firstname text, +lastname text, +birth_year int, +country text ) -CREATE INDEX ON users(birth_year); + -p. +CREATE INDEX ON users(birth_year); +---- Then the following queries are valid: -bc(sample). + -SELECT * FROM users; + +[source,sql] +---- +SELECT * FROM users; SELECT firstname, lastname FROM users WHERE birth_year = 1981; +---- because in both case, Cassandra guarantees that these queries performance will be proportional to the amount of data returned. In @@ -1938,9 +2026,9 @@ data returned can always be controlled by adding a `LIMIT`. However, the following query will be rejected: -bc(sample). + +[source,sql] SELECT firstname, lastname FROM users WHERE birth_year = 1981 AND -country = `FR'; +country = 'FR'; because Cassandra cannot guarantee that it won’t have to scan large amount of data even if the result to those query is small. Typically, it @@ -1949,9 +2037,9 @@ handful are actually from France. However, if you ``know what you are doing'', you can force the execution of this query by using `ALLOW FILTERING` and so the following query is valid: -bc(sample). + +[source,sql] SELECT firstname, lastname FROM users WHERE birth_year = 1981 AND -country = `FR' ALLOW FILTERING; +country = 'FR' ALLOW FILTERING; [[databaseRoles]] === Database Roles @@ -1961,24 +2049,27 @@ country = `FR' ALLOW FILTERING; _Syntax:_ -bc(syntax).. + +[source,bnf] +---- ::= CREATE ROLE ( IF NOT EXISTS )? ( WITH ( AND )* )? -::= PASSWORD = + -| LOGIN = + -| SUPERUSER = + -| OPTIONS = + -p. +::= PASSWORD = +| LOGIN = +| SUPERUSER = +| OPTIONS = +---- _Sample:_ -bc(sample). + -CREATE ROLE new_role; + -CREATE ROLE alice WITH PASSWORD = `password_a' AND LOGIN = true; + -CREATE ROLE bob WITH PASSWORD = `password_b' AND LOGIN = true AND -SUPERUSER = true; + -CREATE ROLE carlos WITH OPTIONS = \{ `custom_option1' : `option1_value', -`custom_option2' : 99 }; +[source,sql] +---- +CREATE ROLE new_role; +CREATE ROLE alice WITH PASSWORD = 'password_a' AND LOGIN = true; +CREATE ROLE bob WITH PASSWORD = 'password_b' AND LOGIN = true AND +SUPERUSER = true; +CREATE ROLE carlos WITH OPTIONS = { 'custom_option1' : 'option1_value', +'custom_option2' : 99 }; +---- By default roles do not possess `LOGIN` privileges or `SUPERUSER` status. @@ -2016,28 +2107,31 @@ Attempting to create an existing role results in an invalid query condition unless the `IF NOT EXISTS` option is used. If the option is used and the role exists, the statement is a no-op. -bc(sample). + -CREATE ROLE other_role; + +[source,sql] +---- +CREATE ROLE other_role; CREATE ROLE IF NOT EXISTS other_role; +---- [[alterRoleStmt]] ==== ALTER ROLE _Syntax:_ -bc(syntax).. + +[source,bnf] +---- ::= ALTER ROLE (IF EXISTS)? ( WITH ( AND )* )? -::= PASSWORD = + -| LOGIN = + -| SUPERUSER = + -| OPTIONS = + -p. +::= PASSWORD = +| LOGIN = +| SUPERUSER = +| OPTIONS = +---- _Sample:_ -bc(sample). + -ALTER ROLE bob WITH PASSWORD = `PASSWORD_B' AND SUPERUSER = false; +[source,sql] +ALTER ROLE bob WITH PASSWORD = 'PASSWORD_B' AND SUPERUSER = false; If the role does not exist, the statement will return an error, unless `IF EXISTS` is used in which case the operation is a no-op. @@ -2057,15 +2151,16 @@ link:#permissions[permission] on that role _Syntax:_ -bc(syntax).. + -::= DROP ROLE ( IF EXISTS )? + -p. +[source,bnf] +::= DROP ROLE ( IF EXISTS )? _Sample:_ -bc(sample). + -DROP ROLE alice; + +[source,sql] +---- +DROP ROLE alice; DROP ROLE IF EXISTS bob; +---- `DROP ROLE` requires the client to have `DROP` link:#permissions[permission] on the role in question. In addition, @@ -2081,12 +2176,12 @@ used and the role does not exist the statement is a no-op. _Syntax:_ -bc(syntax). + +[source,bnf] ::= GRANT TO _Sample:_ -bc(sample). + +[source,sql] GRANT report_writer TO alice; This statement grants the `report_writer` role to `alice`. Any @@ -2094,26 +2189,30 @@ permissions granted to `report_writer` are also acquired by `alice`. + Roles are modelled as a directed acyclic graph, so circular grants are not permitted. The following examples result in error conditions: -bc(sample). + -GRANT role_a TO role_b; + +[source,sql] +---- +GRANT role_a TO role_b; GRANT role_b TO role_a; +---- -bc(sample). + -GRANT role_a TO role_b; + -GRANT role_b TO role_c; + +[source,sql] +---- +GRANT role_a TO role_b; +GRANT role_b TO role_c; GRANT role_c TO role_a; +---- [[revokeRoleStmt]] ==== REVOKE ROLE _Syntax:_ -bc(syntax). + +[source,bnf] ::= REVOKE FROM _Sample:_ -bc(sample). + +[source,sql] REVOKE report_writer FROM alice; This statement revokes the `report_writer` role from `alice`. Any @@ -2125,25 +2224,25 @@ also revoked. _Syntax:_ -bc(syntax). + +[source,bnf] ::= LIST ROLES ( OF )? ( NORECURSIVE )? _Sample:_ -bc(sample). + +[source,sql] LIST ROLES; Return all known roles in the system, this requires `DESCRIBE` permission on the database roles resource. -bc(sample). + -LIST ROLES OF `alice`; +[source,sql] +LIST ROLES OF 'alice'; Enumerate all roles granted to `alice`, including those transitively aquired. -bc(sample). + -LIST ROLES OF `bob` NORECURSIVE +[source,sql] +LIST ROLES OF 'bob' NORECURSIVE List all roles directly granted to `bob`. @@ -2157,57 +2256,64 @@ statments becoming synonyms for the `ROLE` based equivalents. _Syntax:_ -bc(syntax).. + +[source,bnf] +---- ::= CREATE USER ( IF NOT EXISTS )? ( WITH PASSWORD )? ()? -::= SUPERUSER + -| NOSUPERUSER + -p. +::= SUPERUSER | NOSUPERUSER +---- _Sample:_ -bc(sample). + -CREATE USER alice WITH PASSWORD `password_a' SUPERUSER; + -CREATE USER bob WITH PASSWORD `password_b' NOSUPERUSER; +[source,sql] +---- +CREATE USER alice WITH PASSWORD 'password_a' SUPERUSER; +CREATE USER bob WITH PASSWORD 'password_b' NOSUPERUSER; +---- `CREATE USER` is equivalent to `CREATE ROLE` where the `LOGIN` option is `true`. So, the following pairs of statements are equivalent: -bc(sample).. + -CREATE USER alice WITH PASSWORD `password_a' SUPERUSER; + -CREATE ROLE alice WITH PASSWORD = `password_a' AND LOGIN = true AND +[source,sql] +---- +CREATE USER alice WITH PASSWORD 'password_a' SUPERUSER; +CREATE ROLE alice WITH PASSWORD = 'password_a' AND LOGIN = true AND SUPERUSER = true; -CREATE USER IF NOT EXISTS alice WITH PASSWORD `password_a' SUPERUSER; + -CREATE ROLE IF NOT EXISTS alice WITH PASSWORD = `password_a' AND LOGIN = +CREATE USER IF NOT EXISTS alice WITH PASSWORD 'password_a' SUPERUSER; +CREATE ROLE IF NOT EXISTS alice WITH PASSWORD = 'password_a' AND LOGIN = true AND SUPERUSER = true; -CREATE USER alice WITH PASSWORD `password_a' NOSUPERUSER; + -CREATE ROLE alice WITH PASSWORD = `password_a' AND LOGIN = true AND +CREATE USER alice WITH PASSWORD 'password_a' NOSUPERUSER; +CREATE ROLE alice WITH PASSWORD = 'password_a' AND LOGIN = true AND SUPERUSER = false; -CREATE USER alice WITH PASSWORD `password_a' NOSUPERUSER; + -CREATE ROLE alice WITH PASSWORD = `password_a' AND LOGIN = true; +CREATE USER alice WITH PASSWORD 'password_a' NOSUPERUSER; +CREATE ROLE alice WITH PASSWORD = 'password_a' AND LOGIN = true; -CREATE USER alice WITH PASSWORD `password_a'; + -CREATE ROLE alice WITH PASSWORD = `password_a' AND LOGIN = true; + -p. +CREATE USER alice WITH PASSWORD 'password_a'; +CREATE ROLE alice WITH PASSWORD = 'password_a' AND LOGIN = true; +---- [[alterUserStmt]] ==== ALTER USER _Syntax:_ -bc(syntax).. + +[source,bnf] +---- ::= ALTER USER (IF EXISTS)? ( WITH PASSWORD )? ( )? -::= SUPERUSER + -| NOSUPERUSER + -p. +::= SUPERUSER | NOSUPERUSER +---- + +_Sample:_ -bc(sample). + -ALTER USER alice WITH PASSWORD `PASSWORD_A'; + +[source,sql] +---- +ALTER USER alice WITH PASSWORD 'PASSWORD_A'; ALTER USER bob SUPERUSER; +---- If the user does not exist, the statement will return an error, unless `IF EXISTS` is used in which case the operation is a no-op. @@ -2216,32 +2322,33 @@ If the user does not exist, the statement will return an error, unless `IF EXIST _Syntax:_ -bc(syntax).. + -::= DROP USER ( IF EXISTS )? + -p. +[source,bnf] +::= DROP USER ( IF EXISTS )? _Sample:_ -bc(sample). + -DROP USER alice; + +[source,sql] +---- +DROP USER alice; DROP USER IF EXISTS bob; +---- [[listUsersStmt]] ==== LIST USERS _Syntax:_ -bc(syntax). + +[source,bnf] ::= LIST USERS; _Sample:_ -bc(sample). + +[source,sql] LIST USERS; This statement is equivalent to -bc(sample). + +[source,sql] LIST ROLES; but only roles with the `LOGIN` privilege are included in the output. @@ -2254,12 +2361,12 @@ but only roles with the `LOGIN` privilege are included in the output. _Syntax:_ -bc(syntax).. + +[source,bnf] ::= ADD IDENTITY ( IF NOT EXISTS )? TO ROLE ? _Sample:_ -bc(sample). + +[source,sql] ADD IDENTITY 'id1' TO ROLE 'role1'; Only a user with privileges to add roles can add identities @@ -2273,7 +2380,7 @@ Attempting to add an existing identity results in an invalid query condition unless the `IF NOT EXISTS` option is used. If the option is used and the identity exists, the statement is a no-op. -bc(sample). + +[source,sql] ADD IDENTITY IF NOT EXISTS 'id1' TO ROLE 'role1'; [[dropIdentityStmt]] @@ -2281,15 +2388,16 @@ ADD IDENTITY IF NOT EXISTS 'id1' TO ROLE 'role1'; _Syntax:_ -bc(syntax).. + -::= DROP IDENTITY ( IF EXISTS )? + -p. +[source,bnf] +::= DROP IDENTITY ( IF EXISTS )? _Sample:_ -bc(sample). + -DROP IDENTITY 'testIdentity'; + +[source,sql] +---- +DROP IDENTITY 'testIdentity'; DROP IDENTITY IF EXISTS 'testIdentity'; +---- Only a user with privileges to drop roles can remove identities @@ -2516,45 +2624,46 @@ wildcard pattern | | | _Syntax:_ -bc(syntax).. + +[source,bnf] +---- ::= GRANT ( ALL ( PERMISSIONS )? | ( PERMISSION )? ) ON TO ::= CREATE | ALTER | DROP | SELECT | MODIFY | AUTHORIZE | DESCRIBE | UNMASK | SELECT_MASKED EXECUTE -::= ALL KEYSPACES + -| KEYSPACE + -| ( TABLE )? + -| ALL ROLES + -| ROLE + -| ALL FUNCTIONS ( IN KEYSPACE )? + -| FUNCTION + -| ALL MBEANS + -| ( MBEAN | MBEANS ) + -p. +::= ALL KEYSPACES +| KEYSPACE +| ( TABLE )? +| ALL ROLES +| ROLE +| ALL FUNCTIONS ( IN KEYSPACE )? +| FUNCTION +| ALL MBEANS +| ( MBEAN | MBEANS ) +---- _Sample:_ -bc(sample). + +[source,sql] GRANT SELECT ON ALL KEYSPACES TO data_reader; This gives any user with the role `data_reader` permission to execute `SELECT` statements on any table across all keyspaces -bc(sample). + +[source,sql] GRANT MODIFY ON KEYSPACE keyspace1 TO data_writer; This give any user with the role `data_writer` permission to perform `UPDATE`, `INSERT`, `UPDATE`, `DELETE` and `TRUNCATE` queries on all tables in the `keyspace1` keyspace -bc(sample). + +[source,sql] GRANT DROP ON keyspace1.table1 TO schema_owner; This gives any user with the `schema_owner` role permissions to `DROP` `keyspace1.table1`. -bc(sample). + +[source,sql] GRANT EXECUTE ON FUNCTION keyspace1.user_function( int ) TO report_writer; @@ -2562,7 +2671,7 @@ This grants any user with the `report_writer` role permission to execute `SELECT`, `INSERT` and `UPDATE` queries which use the function `keyspace1.user_function( int )` -bc(sample). + +[source,sql] GRANT DESCRIBE ON ALL ROLES TO role_admin; This grants any user with the `role_admin` role permission to view any @@ -2588,64 +2697,67 @@ the new resource. _Syntax:_ -bc(syntax).. + +[source,bnf] +---- ::= REVOKE ( ALL ( PERMISSIONS )? | ( PERMISSION )? ) ON FROM ::= CREATE | ALTER | DROP | SELECT | MODIFY | AUTHORIZE | DESCRIBE | UNMASK | SELECT_MASKED EXECUTE -::= ALL KEYSPACES + -| KEYSPACE + -| ( TABLE )? + -| ALL ROLES + -| ROLE + -| ALL FUNCTIONS ( IN KEYSPACE )? + -| FUNCTION + -| ALL MBEANS + -| ( MBEAN | MBEANS ) + -p. +::= ALL KEYSPACES +| KEYSPACE +| ( TABLE )? +| ALL ROLES +| ROLE +| ALL FUNCTIONS ( IN KEYSPACE )? +| FUNCTION +| ALL MBEANS +| ( MBEAN | MBEANS ) +---- _Sample:_ -bc(sample).. + -REVOKE SELECT ON ALL KEYSPACES FROM data_reader; + -REVOKE MODIFY ON KEYSPACE keyspace1 FROM data_writer; + -REVOKE DROP ON keyspace1.table1 FROM schema_owner; + +[source,sql] +---- +REVOKE SELECT ON ALL KEYSPACES FROM data_reader; +REVOKE MODIFY ON KEYSPACE keyspace1 FROM data_writer; +REVOKE DROP ON keyspace1.table1 FROM schema_owner; REVOKE EXECUTE ON FUNCTION keyspace1.user_function( int ) FROM -report_writer; + -REVOKE DESCRIBE ON ALL ROLES FROM role_admin; + -p. +report_writer; +REVOKE DESCRIBE ON ALL ROLES FROM role_admin; +---- [[listPermissionsStmt]] ===== LIST PERMISSIONS _Syntax:_ -bc(syntax).. + -::= LIST ( ALL ( PERMISSIONS )? | ) + -( ON )? + +[source,bnf] +---- +::= LIST ( ALL ( PERMISSIONS )? | ) +( ON )? ( OF ( NORECURSIVE )? )? -::= ALL KEYSPACES + -| KEYSPACE + -| ( TABLE )? + -| ALL ROLES + -| ROLE + -| ALL FUNCTIONS ( IN KEYSPACE )? + -| FUNCTION + -| ALL MBEANS + -| ( MBEAN | MBEANS ) + -p. +::= ALL KEYSPACES +| KEYSPACE +| ( TABLE )? +| ALL ROLES +| ROLE +| ALL FUNCTIONS ( IN KEYSPACE )? +| FUNCTION +| ALL MBEANS +| ( MBEAN | MBEANS ) +---- _Sample:_ -bc(sample). + +[source,sql] LIST ALL PERMISSIONS OF alice; Show all permissions granted to `alice`, including those acquired transitively from any other roles. -bc(sample). + +[source,sql] LIST ALL PERMISSIONS ON keyspace1.table1 OF bob; Show all permissions on `keyspace1.table1` granted to `bob`, including @@ -2656,7 +2768,7 @@ permissions higher up the resource hierarchy which can be applied to the `NORECURSIVE` switch restricts the results to only those permissions which were directly granted to `bob` or one of `bob`’s roles. -bc(sample). + +[source,sql] LIST SELECT PERMISSIONS OF carlos; Show any permissions granted to `carlos` or any of `carlos`’s roles, @@ -2671,38 +2783,41 @@ and collection types, users can also provide custom types (through a JAVA class extending `AbstractType` loadable by + Cassandra). The syntax of types is thus: -bc(syntax).. + -::= + -| + -| + +[source,bnf] +---- +::= +| +| | // Used for custom types. The fully-qualified name of a JAVA class -::= ascii + -| bigint + -| blob + -| boolean + -| counter + -| date + -| decimal + -| double + -| float + -| inet + -| int + -| smallint + -| text + -| time + -| timestamp + -| timeuuid + -| tinyint + -| uuid + -| varchar + +::= ascii +| bigint +| blob +| boolean +| counter +| date +| decimal +| double +| float +| inet +| int +| smallint +| text +| time +| timestamp +| timeuuid +| tinyint +| uuid +| varchar | varint -::= list `<' `>' + -| set `<' `>' + -| map `<' `,' `>' + -::= tuple `<' (`,' )* `>' + -p. Note that the native types are keywords and as such are +::= list `<' `>' +| set `<' `>' +| map `<' `,' `>' +::= tuple `<' (`,' )* `>' +---- + +Note that the native types are keywords and as such are case-insensitive. They are however not reserved ones. The following table gives additional informations on the native data @@ -2900,50 +3015,56 @@ keys and will thus always be returned in that order. To create a column of type `map`, use the `map` keyword suffixed with comma-separated key and value types, enclosed in angle brackets. For example: -bc(sample). + -CREATE TABLE users ( + -id text PRIMARY KEY, + -given text, + -surname text, + -favs map<text, text> // A map of text keys, and text values + +[source,sql] +---- +CREATE TABLE users ( +id text PRIMARY KEY, +given text, +surname text, +favs map<text, text> -- A map of text keys, and text values ) +---- Writing `map` data is accomplished with a JSON-inspired syntax. To write a record using `INSERT`, specify the entire map as a JSON-style associative array. _Note: This form will always replace the entire map._ -bc(sample). + -// Inserting (or Updating) + -INSERT INTO users (id, given, surname, favs) + -VALUES (`jsmith', `John', `Smith', \{ `fruit' : `apple', `band' : -`Beatles' }) +[source,sql] +---- +// Inserting (or Updating) +INSERT INTO users (id, given, surname, favs) +VALUES ('jsmith', 'John', 'Smith', { 'fruit' : 'apple', 'band' : +'Beatles' }) +---- Adding or updating key-values of a (potentially) existing map can be accomplished either by subscripting the map column in an `UPDATE` statement or by adding a new map literal: -bc(sample). + -// Updating (or inserting) + -UPDATE users SET favs[`author'] = `Ed Poe' WHERE id = `jsmith' + -UPDATE users SET favs = favs + \{ `movie' : `Cassablanca' } WHERE id = -`jsmith' +[source,sql] +---- +// Updating (or inserting) +UPDATE users SET favs['author'] = 'Ed Poe' WHERE id = 'jsmith' +UPDATE users SET favs = favs + {'`movie' : 'Cassablanca' } WHERE id = +'jsmith' +---- Note that TTLs are allowed for both `INSERT` and `UPDATE`, but in both case the TTL set only apply to the newly inserted/updated _values_. In other words, -bc(sample). + -// Updating (or inserting) + -UPDATE users USING TTL 10 SET favs[`color'] = `green' WHERE id = -`jsmith' +[source,sql] +// Updating (or inserting) +UPDATE users USING TTL 10 SET favs['color'] = 'green' WHERE id = +'jsmith' will only apply the TTL to the `{ 'color' : 'green' }` record, the rest of the map remaining unaffected. Deleting a map record is done with: -bc(sample). + -DELETE favs[`author'] FROM users WHERE id = `jsmith' +[source,sql] +DELETE favs['author'] FROM users WHERE id = 'jsmith' [[set]] ===== Sets @@ -2953,29 +3074,31 @@ ordered by their values. To create a column of type `set`, use the `set` keyword suffixed with the value type enclosed in angle brackets. For example: -bc(sample). + -CREATE TABLE images ( + -name text PRIMARY KEY, + -owner text, + -date timestamp, + -tags set + +[source,sql] +---- +CREATE TABLE images ( +name text PRIMARY KEY, +owner text, +date timestamp, +tags set ); +---- Writing a `set` is accomplished by comma separating the set values, and enclosing them in curly braces. _Note: An `INSERT` will always replace the entire set._ -bc(sample). + -INSERT INTO images (name, owner, date, tags) + -VALUES (`cat.jpg', `jsmith', `now', \{ `kitten', `cat', `pet' }); +[source,sql] +INSERT INTO images (name, owner, date, tags) +VALUES ('cat.jpg', 'jsmith', 'now', { 'kitten', 'cat', 'pet' }); Adding and removing values of a set can be accomplished with an `UPDATE` by adding/removing new set values to an existing `set` column. -bc(sample). + -UPDATE images SET tags = tags + \{ `cute', `cuddly' } WHERE name = -`cat.jpg'; + -UPDATE images SET tags = tags - \{ `lame' } WHERE name = `cat.jpg'; +[source,sql] +UPDATE images SET tags = tags + { 'cute', 'cuddly' } WHERE name = +'cat.jpg'; +UPDATE images SET tags = tags - { 'lame' } WHERE name = 'cat.jpg'; As with link:#map[maps], TTLs if used only apply to the newly inserted/updated _values_. @@ -2988,13 +3111,15 @@ elements are ordered by there position in the list. To create a column of type `list`, use the `list` keyword suffixed with the value type enclosed in angle brackets. For example: -bc(sample). + -CREATE TABLE plays ( + -id text PRIMARY KEY, + -game text, + -players int, + -scores list + +[source,sql] +---- +CREATE TABLE plays ( +id text PRIMARY KEY, +game text, +players int, +scores list ) +---- Do note that as explained below, lists have some limitations and performance considerations to take into account, and it is advised to @@ -3004,18 +3129,20 @@ Writing `list` data is accomplished with a JSON-style syntax. To write a record using `INSERT`, specify the entire list as a JSON array. _Note: An `INSERT` will always replace the entire list._ -bc(sample). + -INSERT INTO plays (id, game, players, scores) + -VALUES (`123-afde', `quake', 3, [17, 4, 2]); +[source,sql] +INSERT INTO plays (id, game, players, scores) +VALUES ('123-afde', 'quake', 3, [17, 4, 2]); Adding (appending or prepending) values to a list can be accomplished by adding a new JSON-style array to an existing `list` column. -bc(sample). + +[source,sql] +---- UPDATE plays SET players = 5, scores = scores + [ 14, 21 ] WHERE id = -`123-afde'; + +'123-afde'; UPDATE plays SET players = 5, scores = [ 12 ] + scores WHERE id = -`123-afde'; +'123-afde'; +---- It should be noted that append and prepend are not idempotent operations. This means that if during an append or a prepend the @@ -3030,14 +3157,17 @@ operations induce an internal read before the update, and will thus typically have slower performance characteristics_. Those operations have the following syntax: -bc(sample). + -UPDATE plays SET scores[1] = 7 WHERE id = `123-afde'; // sets the 2nd +[source,sql] +---- +UPDATE plays SET scores[1] = 7 WHERE id = '123-afde'; // sets the 2nd element of scores to 7 (raises an error is scores has less than 2 -elements) + -DELETE scores[1] FROM plays WHERE id = `123-afde'; // deletes the 2nd -element of scores (raises an error is scores has less than 2 elements) + -UPDATE plays SET scores = scores - [ 12, 21 ] WHERE id = `123-afde'; // -removes all occurrences of 12 and 21 from scores +elements) + +DELETE scores[1] FROM plays WHERE id = '123-afde'; // deletes the 2nd +element of scores (raises an error is scores has less than 2 elements) + +UPDATE plays SET scores = scores - [ 12, 21 ] WHERE id = '123-afde'; // removes all occurrences of 12 and 21 from scores +---- As with link:#map[maps], TTLs if used only apply to the newly inserted/updated _values_. @@ -3049,7 +3179,7 @@ Vectors are fixed-size sequences of non-null values of a certain data type. They You can define, insert and update a vector with: -[source,cql] +[source,sql] ---- include::cassandra:example$CQL/vector.cql[] ---- @@ -3123,7 +3253,7 @@ into its own datatype. The conversions rely strictly on Java’s semantics. For example, the double value 1 will be converted to the text value `1.0'. -bc(sample). + +[source,sql] SELECT avg(cast(count as double)) FROM myTable [[tokenFun]] @@ -3143,12 +3273,14 @@ partition key columns. The return type depend on the partitioner in use: For instance, in a cluster using the default Murmur3Partitioner, if a table is defined by -bc(sample). + -CREATE TABLE users ( + -userid text PRIMARY KEY, + -username text, + -… + +[source,sql] +---- +CREATE TABLE users ( +userid text PRIMARY KEY, +username text, +... ) +---- then the `token` function will take a single argument of type `text` (in that case, the partition key is `userid` (there is no clustering columns @@ -3171,7 +3303,7 @@ node, a new unique timeuuid (at the time where the statement using it is executed). Note that this method is useful for insertion but is largely non-sensical in `WHERE` clauses. For instance, a query of the form -bc(sample). + +[source,sql] SELECT * FROM myTable WHERE t = now() will never return any result by design, since the value returned by @@ -3185,9 +3317,9 @@ date string] ) and return a _fake_ `timeuuid` corresponding to the _smallest_ (resp. _biggest_) possible `timeuuid` having for timestamp `t`. So for instance: -bc(sample). + -SELECT * FROM myTable WHERE t > max_timeuuid(`2013-01-01 00:05+0000') AND -t < min_timeuuid(`2013-02-02 10:00+0000') +[source,sql] +SELECT * FROM myTable WHERE t > max_timeuuid('2013-01-01 00:05+0000') AND +t < min_timeuuid('2013-02-02 10:00+0000') will select all rows where the `timeuuid` column `t` is strictly older than `2013-01-01 00:05+0000' but strictly younger than `2013-02-02 @@ -3264,14 +3396,16 @@ several native aggregates, described below: The `count` function can be used to count the rows returned by a query. Example: -bc(sample). + -SELECT COUNT (*) FROM plays; + +[source,sql] +---- +SELECT COUNT (*) FROM plays; SELECT COUNT (1) FROM plays; +---- It also can be used to count the non null value of a given column. Example: -bc(sample). + +[source,sql] SELECT COUNT (scores) FROM plays; [[maxMinFcts]] @@ -3280,8 +3414,8 @@ SELECT COUNT (scores) FROM plays; The `max` and `min` functions can be used to compute the maximum and the minimum value returned by a query for a given column. -bc(sample). + -SELECT MIN (players), MAX (players) FROM plays WHERE game = `quake'; +[source,sql] +SELECT MIN (players), MAX (players) FROM plays WHERE game = 'quake'; [[sumFct]] ==== Sum @@ -3289,7 +3423,7 @@ SELECT MIN (players), MAX (players) FROM plays WHERE game = `quake'; The `sum` function can be used to sum up all the values returned by a query for a given column. -bc(sample). + +[source,sql] SELECT SUM (players) FROM plays; [[avgFct]] @@ -3298,7 +3432,7 @@ SELECT SUM (players) FROM plays; The `avg` function can be used to compute the average of all the values returned by a query for a given column. -bc(sample). + +[source,sql] SELECT AVG (players) FROM plays; [[udfs]] @@ -3315,9 +3449,11 @@ propagated to all nodes in the cluster. UDFs can be _overloaded_ - i.e. multiple UDFs with different argument types but the same function name. Example: -bc(sample). + -CREATE FUNCTION sample ( arg int ) …; + -CREATE FUNCTION sample ( arg text ) …; +[source,sql] +---- +CREATE FUNCTION sample ( arg int ) ...; +CREATE FUNCTION sample ( arg text ) ...; +---- User-defined functions are susceptible to all of the normal problems with the chosen programming language. Accordingly, implementations @@ -3337,24 +3473,27 @@ placeholders can be used, too. Note that you can use the double-quoted string syntax to enclose the UDF source code. For example: -bc(sample).. + -CREATE FUNCTION some_function ( arg int ) + -RETURNS NULL ON NULL INPUT + -RETURNS int + -LANGUAGE java + +[source,sql] +---- +CREATE FUNCTION some_function ( arg int ) +RETURNS NULL ON NULL INPUT +RETURNS int +LANGUAGE java AS $$ return arg; $$; -SELECT some_function(column) FROM atable …; + -UPDATE atable SET col = some_function(?) …; + -p. +SELECT some_function(column) FROM atable ...; +UPDATE atable SET col = some_function(?) ...; +---- -bc(sample). + -CREATE TYPE custom_type (txt text, i int); + -CREATE FUNCTION fct_using_udt ( udtarg frozen ) + -RETURNS NULL ON NULL INPUT + -RETURNS text + -LANGUAGE java + -AS $$ return udtarg.getString(``txt''); $$; +[source,sql] +---- +CREATE TYPE custom_type (txt text, i int); +CREATE FUNCTION fct_using_udt ( udtarg frozen ) +RETURNS NULL ON NULL INPUT +RETURNS text +LANGUAGE java +AS $$ return udtarg.getString("txt"); $$; +---- User-defined functions can be used in link:#selectStmt[`SELECT`], link:#insertStmt[`INSERT`] and link:#updateStmt[`UPDATE`] statements. @@ -3363,50 +3502,56 @@ The implicitly available `udfContext` field (or binding for script UDFs) provides the neccessary functionality to create new UDT and tuple values. -bc(sample). + -CREATE TYPE custom_type (txt text, i int); + -CREATE FUNCTION fct_using_udt ( somearg int ) + -RETURNS NULL ON NULL INPUT + -RETURNS custom_type + -LANGUAGE java + -AS $$ + -UDTValue udt = udfContext.newReturnUDTValue(); + -udt.setString(``txt'', ``some string''); + -udt.setInt(``i'', 42); + -return udt; + +[source,sql] +---- +CREATE TYPE custom_type (txt text, i int); +CREATE FUNCTION fct_using_udt ( somearg int ) +RETURNS NULL ON NULL INPUT +RETURNS custom_type +LANGUAGE java +AS $$ +UDTValue udt = udfContext.newReturnUDTValue(); +udt.setString("txt", "some string"); +udt.setInt("i", 42); +return udt; $$; +---- The definition of the `UDFContext` interface can be found in the Apache Cassandra source code for `org.apache.cassandra.cql3.functions.UDFContext`. -bc(sample). + -public interface UDFContext + -\{ + -UDTValue newArgUDTValue(String argName); + -UDTValue newArgUDTValue(int argNum); + -UDTValue newReturnUDTValue(); + -UDTValue newUDTValue(String udtName); + -TupleValue newArgTupleValue(String argName); + -TupleValue newArgTupleValue(int argNum); + -TupleValue newReturnTupleValue(); + -TupleValue newTupleValue(String cqlDefinition); + +[source,java] +---- +public interface UDFContext +{ +UDTValue newArgUDTValue(String argName); +UDTValue newArgUDTValue(int argNum); +UDTValue newReturnUDTValue(); +UDTValue newUDTValue(String udtName); +TupleValue newArgTupleValue(String argName); +TupleValue newArgTupleValue(int argNum); +TupleValue newReturnTupleValue(); +TupleValue newTupleValue(String cqlDefinition); } +---- Java UDFs already have some imports for common interfaces and classes defined. These imports are: + Please note, that these convenience imports are not available for script UDFs. -bc(sample). + -import java.nio.ByteBuffer; + -import java.util.List; + -import java.util.Map; + -import java.util.Set; + -import org.apache.cassandra.cql3.functions.UDFContext; + -import com.datastax.driver.core.TypeCodec; + -import com.datastax.driver.core.TupleValue; + +[source,java] +---- +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.cassandra.cql3.functions.UDFContext; +import com.datastax.driver.core.TypeCodec; +import com.datastax.driver.core.TupleValue; import com.datastax.driver.core.UDTValue; +---- See link:#createFunctionStmt[`CREATE FUNCTION`] and link:#dropFunctionStmt[`DROP FUNCTION`]. @@ -3436,47 +3581,48 @@ statement. A complete working example for user-defined aggregates (assuming that a keyspace has been selected using the link:#useStmt[`USE`] statement): -bc(sample).. + +[source,sql] +---- CREATE OR REPLACE FUNCTION averageState ( state tuple<int,bigint>, val -int ) + -CALLED ON NULL INPUT + -RETURNS tuple<int,bigint> + -LANGUAGE java + -AS ’ + -if (val != null) \{ + -state.setInt(0, state.getInt(0)+1); + -state.setLong(1, state.getLong(1)+val.intValue()); + -} + -return state; + +int ) +CALLED ON NULL INPUT +RETURNS tuple<int,bigint> +LANGUAGE java +AS ’ +if (val != null) { +state.setInt(0, state.getInt(0)+1); +state.setLong(1, state.getLong(1)+val.intValue()); +} +return state; ’; -CREATE OR REPLACE FUNCTION averageFinal ( state tuple<int,bigint> ) + -CALLED ON NULL INPUT + -RETURNS double + -LANGUAGE java + -AS ’ + -double r = 0; + -if (state.getInt(0) == 0) return null; + -r = state.getLong(1); + -r /= state.getInt(0); + -return Double.valueOf®; + +CREATE OR REPLACE FUNCTION averageFinal ( state tuple<int,bigint> ) +CALLED ON NULL INPUT +RETURNS double +LANGUAGE java +AS ’ +double r = 0; +if (state.getInt(0) == 0) return null; +r = state.getLong(1); +r /= state.getInt(0); +return Double.valueOf(r); ’; -CREATE OR REPLACE AGGREGATE average ( int ) + -SFUNC averageState + -STYPE tuple<int,bigint> + -FINALFUNC averageFinal + +CREATE OR REPLACE AGGREGATE average ( int ) +SFUNC averageState +STYPE tuple<int,bigint> +FINALFUNC averageFinal INITCOND (0, 0); -CREATE TABLE atable ( + -pk int PRIMARY KEY, + -val int); + -INSERT INTO atable (pk, val) VALUES (1,1); + -INSERT INTO atable (pk, val) VALUES (2,2); + -INSERT INTO atable (pk, val) VALUES (3,3); + -INSERT INTO atable (pk, val) VALUES (4,4); + -SELECT average(val) FROM atable; + -p. +CREATE TABLE atable ( +pk int PRIMARY KEY, +val int); +INSERT INTO atable (pk, val) VALUES (1,1); +INSERT INTO atable (pk, val) VALUES (2,2); +INSERT INTO atable (pk, val) VALUES (3,3); +INSERT INTO atable (pk, val) VALUES (4,4); +SELECT average(val) FROM atable; +---- See link:#createAggregateStmt[`CREATE AGGREGATE`] and link:#dropAggregateStmt[`DROP AGGREGATE`]. @@ -3498,12 +3644,13 @@ each row as a single `JSON` encoded map. The remainder of the `SELECT` statment behavior is the same. The result map keys are the same as the column names in a normal result -set. For example, a statement like ```SELECT JSON a, ttl(b) FROM ...`'' +set. For example, a statement like +`SELECT JSON a, ttl(b) FROM ...` would result in a map with keys `"a"` and `"ttl(b)"`. However, this is one notable exception: for symmetry with `INSERT JSON` behavior, case-sensitive column names with upper-case letters will be surrounded -with double quotes. For example, ```SELECT JSON myColumn FROM ...`'' -would result in a map key `"\"myColumn\""` (note the escaped quotes). +with double quotes. For example, `SELECT JSON myColumn FROM ...` + would result in a map key `"\"myColumn\""` (note the escaped quotes). The map values will `JSON`-encoded representations (as described below) of the result set values. @@ -3518,10 +3665,10 @@ the same table. In particular, case-sensitive column names should be surrounded with double quotes. For example, to insert into a table with two columns named ``myKey'' and ``value'', you would do the following: -bc(sample). + -INSERT INTO mytable JSON `\{``\''myKey\``'': 0, ``value'': 0}' +[source,sql] +INSERT INTO mytable JSON '{"\"myKey\"": 0, "value": 0}' -Any columns which are ommitted from the `JSON` map will be defaulted to +Any columns which are omitted from the `JSON` map will be defaulted to a `NULL` value (which will result in a tombstone being created). [[jsonEncoding]] From 6207a305ba2b0cebc3241e00a843ab5dbf86d2ed Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Mon, 20 Jan 2025 16:50:38 +0100 Subject: [PATCH 101/225] Tighten up permissions in dc authorizers patch by Stefan Miklosovic; reviewed by Francisco Guerrero for CASSANDRA-20225 --- CHANGES.txt | 1 + .../cql3/statements/AlterRoleStatement.java | 3 ++ .../auth/CassandraNetworkAuthorizerTest.java | 38 ++++++++++++++++++- 3 files changed, 41 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 8388efa3315f..906090e2c650 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0.16 + * Tighten up permissions in dc authorizers (CASSANDRA-20225) * CBUtil serialization of UTF8 does not handle all UTF8 properly (CASSANDRA-20234) * Make hint expiry use request start time rather than timeout time for TTL (CASSANDRA-20014) * Do not attach rows and partitions to CheckForAbort when already attached (CASSANDRA-20135) diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterRoleStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterRoleStatement.java index 2ffd050d66c2..26b6b29243bd 100644 --- a/src/java/org/apache/cassandra/cql3/statements/AlterRoleStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/AlterRoleStatement.java @@ -78,6 +78,9 @@ public void authorize(ClientState state) throws UnauthorizedException if (opts.getSuperuser().isPresent() && !isSuper) throw new UnauthorizedException("Only superusers are allowed to alter superuser status"); + if (dcPermissions != null && !isSuper) + throw new UnauthorizedException("Only superusers are allowed to alter access to datacenters."); + // superusers can do whatever else they like if (isSuper) return; diff --git a/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java b/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java index 350ee2090e2f..4de709d953bd 100644 --- a/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java +++ b/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java @@ -40,7 +40,9 @@ import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.marshal.UTF8Type; import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.exceptions.UnauthorizedException; import org.apache.cassandra.service.ClientState; +import org.assertj.core.api.Assertions; import static org.apache.cassandra.auth.AuthKeyspace.NETWORK_PERMISSIONS; import static org.apache.cassandra.auth.RoleTestUtils.LocalCassandraRoleManager; @@ -113,7 +115,19 @@ private static ClientState getClientState() return state; } + private static ClientState getClientState(String role) + { + ClientState state = ClientState.forInternalCalls(); + state.login(new AuthenticatedUser(role)); + return state; + } + private static void auth(String query, Object... args) + { + auth(query, getClientState(), args); + } + + private static void auth(String query, ClientState clientState, Object... args) { CQLStatement statement = QueryProcessor.parseStatement(String.format(query, args)).prepare(ClientState.forInternalCalls()); assert statement instanceof CreateRoleStatement @@ -123,7 +137,8 @@ private static void auth(String query, Object... args) // invalidate roles cache so that any changes to the underlying roles are picked up Roles.clearCache(); - authStmt.execute(getClientState()); + authStmt.authorize(clientState); + authStmt.execute(clientState); } private static DCPermissions dcPerms(String username) @@ -170,6 +185,27 @@ public void alter() assertDcPermRow(username); } + @Test + public void alterAsUser() + { + String username = createName(); + + assertNoDcPermRow(username); + auth("CREATE ROLE %s WITH PASSWORD = 'password' AND LOGIN = true AND ACCESS TO DATACENTERS {'dc1'}", username); + Assert.assertEquals(DCPermissions.subset("dc1"), dcPerms(username)); + assertDcPermRow(username, "dc1"); + + // try to alter as a user + ClientState userState = getClientState(username); + Assertions.assertThatThrownBy(() -> auth("ALTER ROLE %s WITH ACCESS TO DATACENTERS {'dc1', 'dc2'}", userState, username)) + .hasMessage("Only superusers are allowed to alter access to datacenters.") + .isInstanceOf(UnauthorizedException.class); + + // nothing changed + Assert.assertEquals(DCPermissions.subset("dc1"), dcPerms(username)); + assertDcPermRow(username, "dc1"); + } + @Test public void drop() { From 634a5985d3721031ad87536aadd4ff4d1bb751c8 Mon Sep 17 00:00:00 2001 From: Marcus Eriksson <marcuse@apache.org> Date: Thu, 16 Jan 2025 10:06:11 +0100 Subject: [PATCH 102/225] Make sure we can parse the expanded CQL before writing it to the log or sending it to replicas Patch by marcuse; reviewed by Sam Tunnicliffe and Stefan Miklosovic for CASSANDRA-20218 --- CHANGES.txt | 1 + .../schema/AlterSchemaStatement.java | 20 ++++ .../schema/CreateIndexStatement.java | 3 +- .../schema/CreateKeyspaceStatement.java | 9 +- .../schema/CreateTableStatement.java | 3 +- .../distributed/test/ExpandCQLTest.java | 102 ++++++++++++++++++ 6 files changed, 129 insertions(+), 9 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/test/ExpandCQLTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 04236dca9b2d..4a749676fb1c 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Make sure we can parse the expanded CQL before writing it to the log or sending it to replicas (CASSANDRA-20218) * Add format_bytes and format_time functions (CASSANDRA-19546) * Fix error when trying to assign a tuple to target type not being a tuple (CASSANDRA-20237) * Fail CREATE TABLE LIKE statement if UDTs in target keyspace do not exist or they have different structure from ones in source keyspace (CASSANDRA-19966) diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java index 590a55a3f926..0282cbd40943 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java @@ -22,15 +22,20 @@ import com.google.common.collect.ImmutableSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.cassandra.auth.AuthenticatedUser; import org.apache.cassandra.auth.IResource; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.CQLStatement; import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.db.compaction.TimeWindowCompactionStrategy; import org.apache.cassandra.db.guardrails.Guardrails; import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.schema.*; +import org.apache.cassandra.exceptions.SyntaxException; import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff; import org.apache.cassandra.service.ClientState; import org.apache.cassandra.service.ClientWarn; @@ -42,6 +47,8 @@ abstract public class AlterSchemaStatement implements CQLStatement.SingleKeyspaceCqlStatement, SchemaTransformation { + private static final Logger logger = LoggerFactory.getLogger(AlterSchemaStatement.class); + protected final String keyspaceName; // name of the keyspace affected by the statement protected ClientState state; // TODO: not sure if this is going to stay the same, or will be replaced by more efficient serialization/sanitation means @@ -229,6 +236,19 @@ private void grantPermissionsOnResource(IResource resource, AuthenticatedUser us } } + protected void verifyExpandedCql(String expandedCql) + { + try + { + QueryProcessor.parseStatement(expandedCql); + } + catch (SyntaxException e) + { + logger.error("Expanded CQL [{}] is not parseable (original CQL: [{}]) - this is most likely due to a bug in the toCqlString method", expandedCql, cql()); + throw e; + } + } + static InvalidRequestException ire(String format, Object... args) { return new InvalidRequestException(String.format(format, args)); diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateIndexStatement.java index d5293f5a610c..e3bdd0ef2c51 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateIndexStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateIndexStatement.java @@ -202,7 +202,8 @@ public Keyspaces apply(ClusterMetadata metadata) throw ire(INDEX_DUPLICATE_OF_EXISTING, index.name, equalIndex.name); } - this.expandedCql = index.toCqlString(table, ifNotExists); + expandedCql = index.toCqlString(table, ifNotExists); + verifyExpandedCql(expandedCql); TableMetadata newTable = table.withSwapped(table.indexes.with(index)); newTable.validate(); diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateKeyspaceStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateKeyspaceStatement.java index 533d48bd0910..882117f21844 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateKeyspaceStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateKeyspaceStatement.java @@ -21,9 +21,6 @@ import com.google.common.collect.ImmutableSet; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import org.apache.cassandra.audit.AuditLogContext; import org.apache.cassandra.audit.AuditLogEntryType; import org.apache.cassandra.auth.DataResource; @@ -45,8 +42,6 @@ public final class CreateKeyspaceStatement extends AlterSchemaStatement { - private static final Logger logger = LoggerFactory.getLogger(CreateKeyspaceStatement.class); - private final KeyspaceAttributes attrs; private final boolean ifNotExists; private String expandedCql; @@ -97,8 +92,8 @@ public Keyspaces apply(ClusterMetadata metadata) keyspaceMetadata.params.validate(keyspaceName, state, metadata); keyspaceMetadata.replicationStrategy.validateExpectedOptions(metadata); - this.expandedCql = keyspaceMetadata.toCqlString(false, true, ifNotExists); - + expandedCql = keyspaceMetadata.toCqlString(false, true, ifNotExists); + verifyExpandedCql(expandedCql); return schema.withAddedOrUpdated(keyspaceMetadata); } diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java index 5970360f24d5..529054a1b7bf 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java @@ -124,7 +124,8 @@ public Keyspaces apply(ClusterMetadata metadata) TableMetadata.Builder builder = builder(keyspace.types, ufBuilder.build()).epoch(metadata.nextEpoch()); // We do not want to set table ID here just yet, since we are using CQL for serialising a fully expanded CREATE TABLE statement. - this.expandedCql = builder.build().toCqlString(false, attrs.hasProperty(TableAttributes.ID), ifNotExists); + expandedCql = builder.build().toCqlString(false, attrs.hasProperty(TableAttributes.ID), ifNotExists); + verifyExpandedCql(expandedCql); if (!attrs.hasProperty(TableAttributes.ID)) builder.id(TableId.get(metadata)); diff --git a/test/distributed/org/apache/cassandra/distributed/test/ExpandCQLTest.java b/test/distributed/org/apache/cassandra/distributed/test/ExpandCQLTest.java new file mode 100644 index 000000000000..5c926834effb --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/ExpandCQLTest.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import java.io.IOException; +import java.util.concurrent.Callable; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.junit.Test; + +import net.bytebuddy.ByteBuddy; +import net.bytebuddy.dynamic.loading.ClassLoadingStrategy; +import net.bytebuddy.implementation.MethodDelegation; +import net.bytebuddy.implementation.bind.annotation.SuperCall; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.schema.IndexMetadata; +import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.schema.TableMetadata; + +import static net.bytebuddy.matcher.ElementMatchers.named; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +public class ExpandCQLTest extends TestBaseImpl +{ + @Test + public void testCreateTable() throws IOException + { + createHelper(TableMetadata.class, withKeyspace("create table %s.x (id int primary key)")); + } + + @Test + public void testCreateKeyspace() throws IOException + { + createHelper(KeyspaceMetadata.class, "create keyspace abc with replication = {'class': 'SimpleStrategy', 'replication_factor': 1}"); + } + + @Test + public void testCreateIndex() throws IOException + { + createHelper(IndexMetadata.class, withKeyspace("create index abc on %s.t (x)")); + } + + private void createHelper(Class<?> clazz, String query) throws IOException + { + try (Cluster cluster = init(Cluster.build(1) + .withInstanceInitializer((cl, i) -> BBTable.install(cl, clazz)) + .start())) + { + cluster.schemaChange(withKeyspace("create table %s.t (id int primary key, x int)")); + try + { + cluster.get(1).runOnInstance(() -> BBTable.enabled.set(true)); + cluster.schemaChange(withKeyspace(query)); + fail("expected exception"); + } + catch (Exception e) + { + assertEquals("SyntaxException", e.getClass().getSimpleName()); + } + cluster.get(1).runOnInstance(() -> BBTable.enabled.set(false)); + cluster.schemaChange(withKeyspace(query)); + } + } + + public static class BBTable + { + static AtomicBoolean enabled = new AtomicBoolean(); + static void install(ClassLoader cl, Class<?> c) + { + new ByteBuddy().rebase(c) + .method(named("toCqlString")) + .intercept(MethodDelegation.to(BBTable.class)) + .make() + .load(cl, ClassLoadingStrategy.Default.INJECTION); + } + + public static String toCqlString(@SuperCall Callable<String> zuper) throws Exception + { + if (!enabled.get()) + return zuper.call(); + return zuper.call().toLowerCase().replace("c", "x"); + } + } + +} From 09e7a4d6ae23b7ac5c9867235c9d900d0c99649a Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Mon, 27 Jan 2025 11:47:16 +0100 Subject: [PATCH 103/225] Prepare debian changelog for 3.0.31 --- debian/changelog | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/debian/changelog b/debian/changelog index ca25260488ee..53060d642101 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,3 +1,9 @@ +cassandra (3.0.31) unstable; urgency=medium + + * New release + + -- Stefan Miklosovic <smiklosovic@apache.org> Mon, 27 Jan 2025 11:45:14 +0100 + cassandra (3.0.30) unstable; urgency=medium * New release From 429f3ad83e1c9eff5c288a7c8fb4781939d00090 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Mon, 27 Jan 2025 12:18:28 +0100 Subject: [PATCH 104/225] Prepare debian changelog for 3.11.18 --- debian/changelog | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/debian/changelog b/debian/changelog index 7759d84c7589..c25a9c352ef5 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,3 +1,9 @@ +cassandra (3.11.18) unstable; urgency=medium + + * New release + + -- Stefan Miklosovic <smiklosovic@apache.org> Mon, 27 Jan 2025 12:16:12 +0100 + cassandra (3.11.17) unstable; urgency=medium * New release From c989c02be66411991c8536ca00ee6481f43e642e Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Mon, 27 Jan 2025 12:38:34 +0100 Subject: [PATCH 105/225] Prepare debian changelog for 4.0.16 --- debian/changelog | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/debian/changelog b/debian/changelog index 92c3bb3271d3..aebe56de817e 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,3 +1,9 @@ +cassandra (4.0.16) unstable; urgency=medium + + * New release + + -- Stefan Miklosovic <smiklosovic@apache.org> Mon, 27 Jan 2025 12:37:18 +0100 + cassandra (4.0.15) unstable; urgency=medium * New release From 044727aabafeab2f6fef74c52d349d55c8732ef5 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Mon, 27 Jan 2025 14:29:35 +0100 Subject: [PATCH 106/225] Prepare debian changelog for 4.1.8 --- debian/changelog | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/debian/changelog b/debian/changelog index 4900a48f7b95..6c3a1779c65c 100644 --- a/debian/changelog +++ b/debian/changelog @@ -2,7 +2,7 @@ cassandra (4.1.8) unstable; urgency=medium * New release - -- Brandon Williams <brandonwilliams@apache.org> Wed, 13 Nov 2024 14:51:52 -0600 + -- Stefan Miklosovic <smiklosovic@apache.org> Mon, 27 Jan 2025 14:28:38 +0100 cassandra (4.1.7) unstable; urgency=medium From b0226c8ea122c3e5ea8680efb0744d33924fd732 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Mon, 27 Jan 2025 14:49:46 +0100 Subject: [PATCH 107/225] Prepare debian changelog for 5.0.3 --- debian/changelog | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/debian/changelog b/debian/changelog index d6f3c47d085d..a72f4f21003a 100644 --- a/debian/changelog +++ b/debian/changelog @@ -2,7 +2,7 @@ cassandra (5.0.3) unstable; urgency=medium * New release - -- Brandon Williams <brandonwilliams@apache.org> Wed, 13 Nov 2024 15:34:07 -0600 + -- Stefan Miklosovic <smiklosovic@apache.org> Mon, 27 Jan 2025 14:49:28 +0100 cassandra (5.0.2) unstable; urgency=medium From 225eb2feeb6ba856c5bf3f46fd0534d074539433 Mon Sep 17 00:00:00 2001 From: Caleb Rackliffe <calebrackliffe@gmail.com> Date: Thu, 23 Jan 2025 11:52:38 -0600 Subject: [PATCH 108/225] Correct the default behavior of compareTo() when comparing WIDE and STATIC PrimaryKeys patch by Caleb Rackliffe; reviewed by David Capwell for CASSANDRA-20238 --- CHANGES.txt | 1 + .../v1/postings/PostingListRangeIterator.java | 4 +- .../sai/iterators/KeyRangeConcatIterator.java | 6 +-- .../KeyRangeIntersectionIterator.java | 13 +++--- .../index/sai/iterators/KeyRangeIterator.java | 15 ++++-- .../sai/iterators/KeyRangeListIterator.java | 2 +- .../sai/iterators/KeyRangeUnionIterator.java | 4 +- .../sai/memory/InMemoryKeyRangeIterator.java | 4 +- .../plan/StorageAttachedIndexSearcher.java | 2 +- .../cassandra/index/sai/utils/PrimaryKey.java | 44 +++++++++--------- .../cql/CompositePartitionKeyIndexTest.java | 46 +++++++++++++++++++ .../index/sai/iterators/LongIterator.java | 4 +- .../index/sai/utils/PrimaryKeyTest.java | 7 +-- 13 files changed, 101 insertions(+), 51 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 1019b4d5c304..84a20e6c712e 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.3 + * Correct the default behavior of compareTo() when comparing WIDE and STATIC PrimaryKeys (CASSANDRA-20238) * Make sure we can set parameters when configuring CassandraCIDRAuthorizer (CASSANDRA-20220) * Add selected SAI index state and query performance metrics to nodetool tablestats (CASSANDRA-20026) * Remove v30 and v3X from 5.x in-JVM upgrade tests (CASSANDRA-20103) diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PostingListRangeIterator.java b/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PostingListRangeIterator.java index 4f05e2a17159..813017db8d3a 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PostingListRangeIterator.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PostingListRangeIterator.java @@ -89,7 +89,7 @@ public PostingListRangeIterator(IndexIdentifier indexIdentifier, @Override protected void performSkipTo(PrimaryKey nextKey) { - if (skipToKey != null && skipToKey.compareTo(nextKey) > 0) + if (skipToKey != null && skipToKey.compareTo(nextKey, false) > 0) return; skipToKey = nextKey; @@ -137,7 +137,7 @@ public void close() private boolean exhausted() { - return needsSkipping && skipToKey.compareTo(getMaximum()) > 0; + return needsSkipping && skipToKey.compareTo(getMaximum(), false) > 0; } /** diff --git a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeConcatIterator.java b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeConcatIterator.java index bb83f1ea81c4..cd47ff219be4 100644 --- a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeConcatIterator.java +++ b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeConcatIterator.java @@ -59,10 +59,10 @@ protected void performSkipTo(PrimaryKey nextKey) { KeyRangeIterator currentIterator = ranges.get(current); - if (currentIterator.hasNext() && currentIterator.peek().compareTo(nextKey) >= 0) + if (currentIterator.hasNext() && currentIterator.peek().compareTo(nextKey, false) >= 0) break; - if (currentIterator.getMaximum().compareTo(nextKey) >= 0) + if (currentIterator.getMaximum().compareTo(nextKey, false) >= 0) { currentIterator.skipTo(nextKey); break; @@ -178,7 +178,7 @@ public void update(KeyRangeIterator range) { min = range.getMinimum(); } - else if (count > 0 && max.compareTo(range.getMinimum()) > 0) + else if (count > 0 && max.compareTo(range.getMinimum(), false) > 0) { throw new IllegalArgumentException(String.format(MUST_BE_SORTED_ERROR, max, range.getMinimum())); } diff --git a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeIntersectionIterator.java b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeIntersectionIterator.java index 5391dd75edb7..e820b2fc4b73 100644 --- a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeIntersectionIterator.java +++ b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeIntersectionIterator.java @@ -74,7 +74,7 @@ protected PrimaryKey computeNext() // compare to other keys only by partition.) This loop continues until all iterators point to the same key, // or if we run out of keys on any of them, or if we exceed the maximum key. // There is no point in iterating after maximum, because no keys will match beyond that point. - while (highestKey != null && highestKey.compareTo(getMaximum()) <= 0) + while (highestKey != null && highestKey.compareTo(getMaximum(), false) <= 0) { // Try to advance all iterators to the highest key seen so far. // Once this inner loop finishes normally, all iterators are guaranteed to be at the same value. @@ -83,7 +83,7 @@ protected PrimaryKey computeNext() if (!range.hasNext()) return endOfData(); - if (range.peek().compareTo(highestKey) < 0) + if (range.peek().compareTo(highestKey, false) < 0) { // If we advance a STATIC key, then we must advance it to the same partition as the highestKey. // Advancing a STATIC key to a WIDE key directly (without throwing away the clustering) would @@ -95,7 +95,7 @@ protected PrimaryKey computeNext() // We use strict comparison here, since it orders WIDE primary keys after STATIC primary keys // in the same partition. When WIDE keys are present, we want to return them rather than STATIC // keys to avoid retrieving and post-filtering entire partitions. - if (nextKey == null || nextKey.compareToStrict(highestKey) > 0) + if (nextKey == null || nextKey.compareTo(highestKey, true) > 0) { // We jumped over the highest key seen so far, so make it the new highest key. highestKey = nextKey; @@ -105,7 +105,7 @@ protected PrimaryKey computeNext() // Therefore, restart the inner loop in order to advance the lagging iterators. continue outer; } - assert nextKey.compareTo(highestKey) == 0 : + assert nextKey.compareTo(highestKey, false) == 0 : String.format("Skipped to a key smaller than the target! " + "iterator: %s, target key: %s, returned key: %s", range, highestKey, nextKey); } @@ -161,7 +161,7 @@ protected PrimaryKey computeNext() { if (!range.hasNext()) return null; - if (range.peek().compareToStrict(max) > 0) + if (range.peek().compareTo(max, true) > 0) max = range.peek(); } return max; @@ -394,6 +394,7 @@ protected static boolean isDisjoint(KeyRangeIterator a, KeyRangeIterator b) */ private static boolean isDisjointInternal(PrimaryKey min, PrimaryKey max, KeyRangeIterator b) { - return min == null || max == null || b.getMaxKeys() == 0 || min.compareTo(b.getMaximum()) > 0 || (b.hasNext() && b.peek().compareTo(max) > 0); + return min == null || max == null || b.getMaxKeys() == 0 + || min.compareTo(b.getMaximum(), false) > 0 || (b.hasNext() && b.peek().compareTo(max, false) > 0); } } diff --git a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeIterator.java b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeIterator.java index 52c390770b2e..a4ada8ec5765 100644 --- a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeIterator.java +++ b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeIterator.java @@ -108,10 +108,10 @@ public final void skipTo(PrimaryKey nextKey) if (state == State.DONE) return; - if (state == State.READY && next.compareTo(nextKey) >= 0) + if (state == State.READY && next.compareTo(nextKey, false) >= 0) return; - if (max.compareTo(nextKey) < 0) + if (max.compareTo(nextKey, false) < 0) { endOfData(); return; @@ -229,7 +229,7 @@ protected static PrimaryKey nullSafeMin(PrimaryKey a, PrimaryKey b) if (a == null) return b; if (b == null) return a; - return a.compareToStrict(b) > 0 ? b : a; + return a.compareTo(b) > 0 ? b : a; } protected static PrimaryKey nullSafeMax(PrimaryKey a, PrimaryKey b) @@ -237,6 +237,13 @@ protected static PrimaryKey nullSafeMax(PrimaryKey a, PrimaryKey b) if (a == null) return b; if (b == null) return a; - return a.compareToStrict(b) > 0 ? a : b; + // The STATIC key sorts before WIDE keys in its partition, but to avoid missing rows while + // intersecting, the STATIC key must override any WIDE key. + if (a.kind() == PrimaryKey.Kind.STATIC && b.kind() == PrimaryKey.Kind.WIDE) + return a.compareTo(b, false) >= 0 ? a : b; + else if (b.kind() == PrimaryKey.Kind.STATIC && a.kind() == PrimaryKey.Kind.WIDE) + return b.compareTo(a, false) >= 0 ? b : a; + + return a.compareTo(b) > 0 ? a : b; } } diff --git a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeListIterator.java b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeListIterator.java index 334e740db87f..4bf5d19bc3c3 100644 --- a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeListIterator.java +++ b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeListIterator.java @@ -50,7 +50,7 @@ protected void performSkipTo(PrimaryKey nextKey) { while (keyQueue.hasNext()) { - if (keyQueue.peek().compareTo(nextKey) >= 0) + if (keyQueue.peek().compareTo(nextKey, false) >= 0) break; keyQueue.next(); } diff --git a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeUnionIterator.java b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeUnionIterator.java index 7bf73ab53453..340333f2af2b 100644 --- a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeUnionIterator.java +++ b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeUnionIterator.java @@ -61,7 +61,7 @@ public PrimaryKey computeNext() { PrimaryKey peeked = range.peek(); - int cmp = candidateKey.compareTo(peeked); + int cmp = candidateKey.compareTo(peeked, false); if (cmp == 0) { @@ -91,7 +91,7 @@ else if (cmp > 0) // Consume the remaining values equal to the candidate key: candidate.next(); } - while (candidate.hasNext() && candidate.peek().compareTo(candidateKey) == 0); + while (candidate.hasNext() && candidate.peek().compareTo(candidateKey, false) == 0); } return candidateKey; diff --git a/src/java/org/apache/cassandra/index/sai/memory/InMemoryKeyRangeIterator.java b/src/java/org/apache/cassandra/index/sai/memory/InMemoryKeyRangeIterator.java index d502afebddda..5131965278f0 100644 --- a/src/java/org/apache/cassandra/index/sai/memory/InMemoryKeyRangeIterator.java +++ b/src/java/org/apache/cassandra/index/sai/memory/InMemoryKeyRangeIterator.java @@ -70,7 +70,7 @@ protected PrimaryKey computeNextKey() if (uniqueKeys) return key; - if (lastKey == null || lastKey.compareTo(key) != 0) + if (lastKey == null || lastKey.compareTo(key, false) != 0) { next = key; lastKey = key; @@ -87,7 +87,7 @@ protected void performSkipTo(PrimaryKey nextKey) while (!keys.isEmpty()) { PrimaryKey key = keys.peek(); - if (key.compareTo(nextKey) >= 0) + if (key.compareTo(nextKey, false) >= 0) break; // consume smaller key diff --git a/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexSearcher.java b/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexSearcher.java index b235c251ef8c..858242006aa2 100644 --- a/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexSearcher.java +++ b/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexSearcher.java @@ -316,7 +316,7 @@ private void fillNextSelectedKeysInPartition(DecoratedKey partitionKey, List<Pri */ private boolean isWithinUpperBound(PrimaryKey key) { - return lastPrimaryKey.token().isMinimum() || lastPrimaryKey.compareTo(key) >= 0; + return lastPrimaryKey.token().isMinimum() || lastPrimaryKey.compareTo(key, false) >= 0; } /** diff --git a/src/java/org/apache/cassandra/index/sai/utils/PrimaryKey.java b/src/java/org/apache/cassandra/index/sai/utils/PrimaryKey.java index 4e5081ad0c05..3cc503304450 100644 --- a/src/java/org/apache/cassandra/index/sai/utils/PrimaryKey.java +++ b/src/java/org/apache/cassandra/index/sai/utils/PrimaryKey.java @@ -269,10 +269,9 @@ public int compareTo(PrimaryKey o) // Otherwise, if the other key is token only we can only compare tokens // This is used by the ResultRetriever to skip to the current key range start position // during result retrieval. - if ((cmp != 0) || o.kind() == Kind.TOKEN) + if (cmp != 0 || o.kind() == Kind.TOKEN) return cmp; - // Finally compare the partition keys return partitionKey().compareTo(o.partitionKey()); } @@ -320,24 +319,25 @@ public ByteSource asComparableBytes(ByteComparable.Version version) } @Override - public int compareTo(PrimaryKey o) + public int compareTo(PrimaryKey o, boolean strict) { int cmp = super.compareTo(o); if (cmp != 0 || o.kind() == Kind.TOKEN || o.kind() == Kind.SKINNY) return cmp; - // At this point the other key is in the same partition as this static key so is equal to it. This - // has to be the case because otherwise, intersections between static column indexes and ordinary - // indexes will fail. + + // If we're comparing strictly, order this STATIC key before a WIDE key, as this corresponds to the + // order of the corresponding row IDs in an on-disk postings list. If we're not being strict, treat + // the keys as being equal, given they are in the same partition. + if (strict && o.kind() == Kind.WIDE) + return -1; + return 0; } @Override - public int compareToStrict(PrimaryKey o) + public int compareTo(PrimaryKey o) { - int cmp = compareTo(o); - // Always order this STATIC key before a WIDE key in the same partition, as this corresponds to the - // order of the corresponding row IDs in an on-disk postings list. - return o.kind() == Kind.WIDE && cmp == 0 ? -1 : cmp; + return compareTo(o, true); } @Override @@ -395,25 +395,25 @@ public ByteSource asComparableBytes(ByteComparable.Version version) } @Override - public int compareTo(PrimaryKey o) + public int compareTo(PrimaryKey o, boolean strict) { int cmp = super.compareTo(o); if (cmp != 0 || o.kind() == Kind.TOKEN || o.kind() == Kind.SKINNY) return cmp; - // At this point this key is in the same partition as the other key so if the other key is a static - // key then it must be equal to it. See comment in the compareTo for static keys above. + if (o.kind() == Kind.STATIC) - return 0; + // If we're comparing strictly, order this WIDE key after the STATIC key, as this corresponds to the + // order of the corresponding row IDs in an on-disk postings list. If we're not being strict, treat + // the keys as being equal, given they are in the same partition. + return strict ? 1 : 0; + return clusteringComparator.compare(clustering(), o.clustering()); } @Override - public int compareToStrict(PrimaryKey o) + public int compareTo(PrimaryKey o) { - int cmp = compareTo(o); - // Always order this WIDE key before a STATIC key in the same partition, as this corresponds to the - // order of the corresponding row IDs in an on-disk postings list. - return o.kind() == Kind.STATIC && cmp == 0 ? 1 : cmp; + return compareTo(o, true); } @Override @@ -490,8 +490,8 @@ default PrimaryKey toStatic() throw new UnsupportedOperationException("Only STATIC and WIDE keys can be converted to STATIC"); } - default int compareToStrict(PrimaryKey o) + default int compareTo(PrimaryKey key, boolean strict) { - return compareTo(o); + return compareTo(key); } } diff --git a/test/unit/org/apache/cassandra/index/sai/cql/CompositePartitionKeyIndexTest.java b/test/unit/org/apache/cassandra/index/sai/cql/CompositePartitionKeyIndexTest.java index 99fcdf8319d1..f384ca74e449 100644 --- a/test/unit/org/apache/cassandra/index/sai/cql/CompositePartitionKeyIndexTest.java +++ b/test/unit/org/apache/cassandra/index/sai/cql/CompositePartitionKeyIndexTest.java @@ -17,13 +17,59 @@ */ package org.apache.cassandra.index.sai.cql; +import java.math.BigInteger; + import org.junit.Test; import org.apache.cassandra.cql3.restrictions.StatementRestrictions; +import org.apache.cassandra.db.marshal.SimpleDateType; +import org.apache.cassandra.db.marshal.TimeType; import org.apache.cassandra.index.sai.SAITester; public class CompositePartitionKeyIndexTest extends SAITester { + @Test + public void testIntersectionWithStaticOverlap() throws Throwable + { + createTable("CREATE TABLE %s (pk0 int, pk1 int, ck0 int, s1 int static, v0 int, PRIMARY KEY((pk0, pk1), ck0))"); + createIndex("CREATE INDEX ON %s(pk0) USING 'sai'"); + + execute("UPDATE %s USING TIMESTAMP 1 SET s1 = 0, v0 = 0 WHERE pk0 = 0 AND pk1 = 1 AND ck0 = 0"); + execute("DELETE FROM %s USING TIMESTAMP 2 WHERE pk0 = 0 AND pk1 = 1"); + + // If the STATIC and WIDE PrimaryKey objects in this partition are not compared strictly, the new WIDE key + // will be interpreted as a duplicate and not added to the Memtable-adjacent index. Then, on flush, the row + // corresponding to that WIDE key will be missing from the index. + execute("UPDATE %s USING TIMESTAMP 3 SET v0 = 1 WHERE pk0 = 0 AND pk1 = 1 AND ck0 = 1"); + + beforeAndAfterFlush(() -> assertRows(execute("SELECT * FROM %s WHERE v0 = 1 AND pk0 = 0 ALLOW FILTERING"), row(0, 1, 1, null, 1))); + } + + @Test + public void testIntersectionWithStaticUpdate() throws Throwable + { + createTable("CREATE TABLE %s (pk0 time, pk1 varint, ck0 date, s0 boolean static, s1 text static, v0 boolean, PRIMARY KEY ((pk0, pk1), ck0))"); + createIndex("CREATE INDEX tbl_pk0 ON %s(pk0) USING 'sai'"); + createIndex("CREATE INDEX tbl_s0 ON %s(s0) USING 'sai'"); + + // pk0: 23:15:13.897962392 -> (static clustering, -1296648-01-08) + // s0: false -> (static clustering, -1296648-01-08) + execute("INSERT INTO %s (pk0, pk1, ck0, s0, s1, v0) VALUES ('23:15:13.897962392', -2272, '-1296648-01-08', false, 'ᕊଖꥬ㨢걲映㚃', false)"); + + // pk0: 23:15:13.897962392 -> (static clustering (existing), -1296648-01-08, -1306427-11-21) + // s0: true -> (static clustering, -1306427-11-21) + execute("UPDATE %s SET s0=true, s1='뾕⌒籖' + '鋿紞', v0=true WHERE pk0 = '23:15:13.897962392' AND pk1 = -2272 AND ck0 = '-1306427-11-21'"); + + // Since the value of "true" is never mapped to the clustering -1296648-01-08, the intersection must begin + // at the STATIC key. Otherwise, we will miss the WIDE key for clustering -1296648-01-08. + beforeAndAfterFlush(() -> + assertRows(execute("SELECT * FROM %s WHERE s0 = true AND pk0 = '23:15:13.897962392'"), + row(TimeType.instance.fromString("23:15:13.897962392"), new BigInteger("-2272"), + SimpleDateType.instance.fromString("-1306427-11-21"), true, "뾕⌒籖鋿紞", true), + row(TimeType.instance.fromString("23:15:13.897962392"), new BigInteger("-2272"), + SimpleDateType.instance.fromString("-1296648-01-08"), true, "뾕⌒籖鋿紞", false))); + } + @Test public void testCompositePartitionIndex() throws Throwable { diff --git a/test/unit/org/apache/cassandra/index/sai/iterators/LongIterator.java b/test/unit/org/apache/cassandra/index/sai/iterators/LongIterator.java index 45f655456a24..5bbc14cdaa86 100644 --- a/test/unit/org/apache/cassandra/index/sai/iterators/LongIterator.java +++ b/test/unit/org/apache/cassandra/index/sai/iterators/LongIterator.java @@ -63,8 +63,8 @@ protected void performSkipTo(PrimaryKey nextKey) { for ( ; currentIdx < keys.size(); currentIdx++) { - PrimaryKey token = keys.get(currentIdx); - if (token.compareTo(nextKey) >= 0) + PrimaryKey key = keys.get(currentIdx); + if (key.compareTo(nextKey, false) >= 0) break; } } diff --git a/test/unit/org/apache/cassandra/index/sai/utils/PrimaryKeyTest.java b/test/unit/org/apache/cassandra/index/sai/utils/PrimaryKeyTest.java index 51edc67389c4..c64372feab0b 100644 --- a/test/unit/org/apache/cassandra/index/sai/utils/PrimaryKeyTest.java +++ b/test/unit/org/apache/cassandra/index/sai/utils/PrimaryKeyTest.java @@ -372,14 +372,9 @@ private void compareToAndEqualsTests(PrimaryKey.Factory factory, PrimaryKey... k assertCompareToAndEquals(key, key, 0); assertCompareToAndEquals(tokenOnlyKey, tokenOnlyKey, 0); - // StaticPrimaryKey is a special case. All other keys in the partition are equal to it - boolean staticComparison = key.kind() == PrimaryKey.Kind.STATIC; - boolean inPartition = staticComparison; for (int comparisonIndex = index + 1; comparisonIndex < keys.length; comparisonIndex++) { - if (staticComparison && keys[comparisonIndex].kind() == PrimaryKey.Kind.STATIC) - inPartition = false; - assertCompareToAndEquals(key, keys[comparisonIndex], inPartition ? 0 : -1); + assertCompareToAndEquals(key, keys[comparisonIndex], -1); assertCompareToAndEquals(tokenOnlyKey, keys[comparisonIndex], tokenOnlyKey.token().equals(keys[comparisonIndex].token()) ? 0 : -1); } } From d7258ac8f31eedc9c28f8c5e381893d192209d48 Mon Sep 17 00:00:00 2001 From: Dmitry Konstantinov <netudima@gmail.com> Date: Sun, 8 Dec 2024 16:10:01 +0300 Subject: [PATCH 109/225] Add table metric PurgeableTombstoneScannedHistogram and a tracing event for scanned purgeable tombstones patch by Dmitry Konstantinov; reviewed by Chris Lohfink, Stefan Miklosovic for CASSANDRA-20132 --- CHANGES.txt | 1 + conf/cassandra.yaml | 14 + conf/cassandra_latest.yaml | 14 + .../pages/managing/operating/metrics.adoc | 3 + .../org/apache/cassandra/config/Config.java | 30 ++ .../cassandra/config/DatabaseDescriptor.java | 11 + .../org/apache/cassandra/db/ReadCommand.java | 139 ++++++ .../db/virtual/TableMetricTables.java | 1 + .../cassandra/metrics/KeyspaceMetrics.java | 3 + .../cassandra/metrics/TableMetrics.java | 3 + .../config/DatabaseDescriptorRefTest.java | 1 + .../apache/cassandra/db/ReadCommandTest.java | 465 ++++++++++++++++-- 12 files changed, 641 insertions(+), 44 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 4a749676fb1c..28ab975f79ed 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Add table metric PurgeableTombstoneScannedHistogram and a tracing event for scanned purgeable tombstones (CASSANDRA-20132) * Make sure we can parse the expanded CQL before writing it to the log or sending it to replicas (CASSANDRA-20218) * Add format_bytes and format_time functions (CASSANDRA-19546) * Fix error when trying to assign a tuple to target type not being a tuple (CASSANDRA-20237) diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index 55fc50f6c6e6..58f8945d653e 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -1964,6 +1964,20 @@ transparent_data_encryption_options: tombstone_warn_threshold: 1000 tombstone_failure_threshold: 100000 +# Controls the granularity of purgeable tombstones reported to PurgeableTombstoneScannedHistogram table metric +# Possible values: +# 'disabled' - do not collect the metric at all +# 'row' - track only partition/range/row level tombstone, +# a good compromise between overheads and usability. +# For CPU-bound workload you may get less than 1% of overhead for throughput. +# For IO-bound workload the overhead is negligible. +# 'cell' - track partition/range/row/cell level tombstones. +# This is the most granular option, +# but it has some performance overheads due to iteration over cells. +# For CPU-bound workload you may get about 5% of overhead for throughput. +# For IO-bound workload the overhead is almost negligible. +# tombstone_read_purgeable_metric_granularity: disabled + # Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a # mechanism called replica filtering protection to ensure that results from stale replicas do # not violate consistency. (See CASSANDRA-8272 and CASSANDRA-15907 for more details.) This diff --git a/conf/cassandra_latest.yaml b/conf/cassandra_latest.yaml index 387565fe1851..ce1d97c476d2 100644 --- a/conf/cassandra_latest.yaml +++ b/conf/cassandra_latest.yaml @@ -1838,6 +1838,20 @@ transparent_data_encryption_options: tombstone_warn_threshold: 1000 tombstone_failure_threshold: 100000 +# Controls the granularity of purgeable tombstones reported to PurgeableTombstoneScannedHistogram table metric +# Possible values: +# 'disabled' - do not collect the metric at all +# 'row' - track only partition/range/row level tombstone, +# a good compromise between overheads and usability. +# For CPU-bound workload you may get less than 1% of overhead for throughput. +# For IO-bound workload the overhead is negligible. +# 'cell' - track partition/range/row/cell level tombstones. +# This is the most granular option, +# but it has some performance overheads due to iteration over cells. +# For CPU-bound workload you may get about 5% of overhead for throughput. +# For IO-bound workload the overhead is almost negligible. +# tombstone_read_purgeable_metric_granularity: disabled + # Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a # mechanism called replica filtering protection to ensure that results from stale replicas do # not violate consistency. (See CASSANDRA-8272 and CASSANDRA-15907 for more details.) This diff --git a/doc/modules/cassandra/pages/managing/operating/metrics.adoc b/doc/modules/cassandra/pages/managing/operating/metrics.adoc index 78dbd165867a..4f3d66652c24 100644 --- a/doc/modules/cassandra/pages/managing/operating/metrics.adoc +++ b/doc/modules/cassandra/pages/managing/operating/metrics.adoc @@ -181,6 +181,9 @@ by compression meta data. |TombstoneScannedHistogram |Histogram |Histogram of tombstones scanned in queries on this table. +|PurgeableTombstoneScannedHistogram |Histogram |Histogram of purgeable tombstones scanned +in queries on this table. Use tombstone_read_purgeable_metric_granularity property in cassandra.yaml to enable it. + |LiveScannedHistogram |Histogram |Histogram of live cells scanned in queries on this table. diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 932423d24881..77f7d9a34176 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -544,6 +544,8 @@ public static class SSTableConfig public volatile int tombstone_warn_threshold = 1000; public volatile int tombstone_failure_threshold = 100000; + public TombstonesMetricGranularity tombstone_read_purgeable_metric_granularity = TombstonesMetricGranularity.disabled; + public final ReplicaFilteringProtectionOptions replica_filtering_protection = new ReplicaFilteringProtectionOptions(); @Replaces(oldName = "index_summary_capacity_in_mb", converter = Converters.MEBIBYTES_DATA_STORAGE_LONG, deprecated = true) @@ -1327,6 +1329,34 @@ public enum BatchlogEndpointStrategy } } + /** + * Allow to control the granularity of metrics related to tombstones. + * It is a trade-off between granularity of a metric vs performance overheads. + * See CASSANDRA-20132 for more details. + */ + public enum TombstonesMetricGranularity + { + /** + * Do not collect the metric at all. + */ + disabled, + /** + * Track only partition/range/row level tombstone, + * a good compromise between overheads and usability. + * For CPU-bound workload you may get less than 1% of overhead for throughput. + * For IO-bound workload the overhead is negligible. + */ + row, + /** + * Track partition/range/row/cell level tombstones. + * This is the most granular option, + * but it has some performance overheads due to iteration over cells. + * For CPU-bound workload you may get about 5% of overhead for throughput. + * For IO-bound workload the overhead is almost negligible. + */ + cell + } + private static final Set<String> SENSITIVE_KEYS = new HashSet<String>() {{ add("client_encryption_options"); add("server_encryption_options"); diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 8a09f4fd76ea..b784571d01fd 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -5523,4 +5523,15 @@ public static boolean isPasswordValidatorReconfigurationEnabled() { return conf.password_validator_reconfiguration_enabled; } + + public static Config.TombstonesMetricGranularity getPurgeableTobmstonesMetricGranularity() + { + return conf.tombstone_read_purgeable_metric_granularity; + } + + @VisibleForTesting + public static void setPurgeableTobmstonesMetricGranularity(Config.TombstonesMetricGranularity granularity) + { + conf.tombstone_read_purgeable_metric_granularity = granularity; + } } diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java index 3bb50cbcfaeb..e4ea5f12d74b 100644 --- a/src/java/org/apache/cassandra/db/ReadCommand.java +++ b/src/java/org/apache/cassandra/db/ReadCommand.java @@ -466,6 +466,7 @@ public UnfilteredPartitionIterator executeLocally(ReadExecutionController execut iterator = withQuerySizeTracking(iterator); iterator = maybeSlowDownForTesting(iterator); iterator = withQueryCancellation(iterator); + iterator = maybeRecordPurgeableTombstones(iterator, cfs); iterator = RTBoundValidator.validate(withoutPurgeableTombstones(iterator, cfs, executionController), Stage.PURGED, false); iterator = withMetricsRecording(iterator, cfs.metric, startTimeNanos); @@ -894,6 +895,144 @@ protected LongPredicate getPurgeEvaluator() return Transformation.apply(iterator, new WithoutPurgeableTombstones()); } + + /** + * Wraps the provided iterator so that metrics on count of purgeable tombstones are tracked and traced. + * It tracks only tombstones with localDeletionTime < now - gc_grace_period. + * Other (non-purgeable) tombstones will be tracked by regular Cassandra logic later. + */ + private UnfilteredPartitionIterator maybeRecordPurgeableTombstones(UnfilteredPartitionIterator iter, + ColumnFamilyStore cfs) + { + class PurgeableTombstonesMetricRecording extends Transformation<UnfilteredRowIterator> + { + private int purgeableTombstones = 0; + + @Override + public UnfilteredRowIterator applyToPartition(UnfilteredRowIterator iter) + { + if (!iter.partitionLevelDeletion().isLive()) + purgeableTombstones++; + return Transformation.apply(iter, this); + } + + @Override + public Row applyToStatic(Row row) + { + return applyToRow(row); + } + + @Override + public Row applyToRow(Row row) + { + final long nowInSec = nowInSec(); + boolean hasTombstones = false; + + if (isPurgeableCellTombstonesTrackingEnabled()) + { + for (Cell<?> cell : row.cells()) + { + if (!cell.isLive(nowInSec) && isPurgeable(cell.localDeletionTime(), nowInSec)) + { + purgeableTombstones++; + hasTombstones = true; // allows to avoid counting an extra tombstone if the whole row expired + } + } + } + + // we replicate the logic is used for non-purged tombstones metric here + if (!row.primaryKeyLivenessInfo().isLive(nowInSec) + && row.hasDeletion(nowInSec) + && isPurgeable(row.deletion().time(), nowInSec) + && !hasTombstones) + { + // We're counting primary key deletions only here. + purgeableTombstones++; + } + + return row; + } + + @Override + public RangeTombstoneMarker applyToMarker(RangeTombstoneMarker marker) + { + final long nowInSec = nowInSec(); + + // for boundary markers - increment metric only if both - close and open - markers are purgeable + if (marker.isBoundary()) + { + countIfBothPurgeable(marker.closeDeletionTime(false), + marker.openDeletionTime(false), + nowInSec); + } + // for bound markers - just increment if it is purgeable + else if (marker instanceof RangeTombstoneBoundMarker) + { + countIfPurgeable(((RangeTombstoneBoundMarker) marker).deletionTime(), nowInSec); + } + + return marker; + } + + @Override + public void onClose() + { + cfs.metric.purgeableTombstoneScannedHistogram.update(purgeableTombstones); + if (purgeableTombstones > 0) + Tracing.trace("Read {} purgeable tombstone cells", purgeableTombstones); + } + + /** + * Increments if both - close and open - deletion times less than (now - gc_grace_period) + */ + private void countIfBothPurgeable(DeletionTime closeDeletionTime, + DeletionTime openDeletionTime, + long nowInSec) + { + if (isPurgeable(closeDeletionTime, nowInSec) && isPurgeable(openDeletionTime, nowInSec)) + purgeableTombstones++; + } + + /** + * Increments if deletion time less than (now - gc_grace_period) + */ + private void countIfPurgeable(DeletionTime deletionTime, + long nowInSec) + { + if (isPurgeable(deletionTime, nowInSec)) + purgeableTombstones++; + } + + /** + * Checks that deletion time < now - gc_grace_period + */ + private boolean isPurgeable(DeletionTime deletionTime, + long nowInSec) + { + return isPurgeable(deletionTime.localDeletionTime(), nowInSec); + } + + /** + * Checks that deletion time < now - gc_grace_period + */ + private boolean isPurgeable(long localDeletionTime, + long nowInSec) + { + return localDeletionTime < cfs.gcBefore(nowInSec); + } + + private boolean isPurgeableCellTombstonesTrackingEnabled() + { + return DatabaseDescriptor.getPurgeableTobmstonesMetricGranularity() == Config.TombstonesMetricGranularity.cell; + } + } + + if (DatabaseDescriptor.getPurgeableTobmstonesMetricGranularity() != Config.TombstonesMetricGranularity.disabled) + return Transformation.apply(iter, new PurgeableTombstonesMetricRecording()); + else + return iter; + } + /** * Return the queried token(s) for logging */ diff --git a/src/java/org/apache/cassandra/db/virtual/TableMetricTables.java b/src/java/org/apache/cassandra/db/virtual/TableMetricTables.java index 5528c92011cc..e1defe51e69e 100644 --- a/src/java/org/apache/cassandra/db/virtual/TableMetricTables.java +++ b/src/java/org/apache/cassandra/db/virtual/TableMetricTables.java @@ -75,6 +75,7 @@ public static Collection<VirtualTable> getAll(String name) new LatencyTableMetric(name, "local_write_latency", t -> t.writeLatency.latency), new LatencyTableMetric(name, "coordinator_write_latency", t -> t.coordinatorWriteLatency), new HistogramTableMetric(name, "tombstones_per_read", t -> t.tombstoneScannedHistogram.cf), + new HistogramTableMetric(name, "purgeable_tombstones_per_read", t -> t.purgeableTombstoneScannedHistogram.cf), new HistogramTableMetric(name, "rows_per_read", t -> t.liveScannedHistogram.cf), new StorageTableMetric(name, "disk_usage", (TableMetrics t) -> t.totalDiskSpaceUsed), new StorageTableMetric(name, "max_partition_size", (TableMetrics t) -> t.maxPartitionSize), diff --git a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java index 237fd03e2d7d..a1916bebd071 100644 --- a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java +++ b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java @@ -85,6 +85,8 @@ public class KeyspaceMetrics public final Histogram sstablesPerRangeReadHistogram; /** Tombstones scanned in queries on this Keyspace */ public final Histogram tombstoneScannedHistogram; + /** Purgeable tombstones scanned in queries on this Keyspace */ + public final Histogram purgeableTombstoneScannedHistogram; /** Live cells scanned in queries on this Keyspace */ public final Histogram liveScannedHistogram; /** Column update time delta on this Keyspace */ @@ -234,6 +236,7 @@ public KeyspaceMetrics(final Keyspace ks) sstablesPerReadHistogram = createKeyspaceHistogram("SSTablesPerReadHistogram", true); sstablesPerRangeReadHistogram = createKeyspaceHistogram("SSTablesPerRangeReadHistogram", true); tombstoneScannedHistogram = createKeyspaceHistogram("TombstoneScannedHistogram", false); + purgeableTombstoneScannedHistogram = createKeyspaceHistogram("PurgeableTombstoneScannedHistogram", false); liveScannedHistogram = createKeyspaceHistogram("LiveScannedHistogram", false); colUpdateTimeDeltaHistogram = createKeyspaceHistogram("ColUpdateTimeDeltaHistogram", false); viewLockAcquireTime = createKeyspaceTimer("ViewLockAcquireTime"); diff --git a/src/java/org/apache/cassandra/metrics/TableMetrics.java b/src/java/org/apache/cassandra/metrics/TableMetrics.java index ab1ab6eb73a3..fabb0814e49a 100644 --- a/src/java/org/apache/cassandra/metrics/TableMetrics.java +++ b/src/java/org/apache/cassandra/metrics/TableMetrics.java @@ -156,6 +156,8 @@ public class TableMetrics public final Gauge<Long> compressionMetadataOffHeapMemoryUsed; /** Tombstones scanned in queries on this CF */ public final TableHistogram tombstoneScannedHistogram; + /** Purgeable tombstones scanned in queries on this CF */ + public final TableHistogram purgeableTombstoneScannedHistogram; /** Live rows scanned in queries on this CF */ public final TableHistogram liveScannedHistogram; /** Column update time delta on this CF */ @@ -768,6 +770,7 @@ public Long getValue() additionalWriteLatencyNanos = createTableGauge("AdditionalWriteLatencyNanos", () -> MICROSECONDS.toNanos(cfs.additionalWriteLatencyMicros)); tombstoneScannedHistogram = createTableHistogram("TombstoneScannedHistogram", cfs.keyspace.metric.tombstoneScannedHistogram, false); + purgeableTombstoneScannedHistogram = createTableHistogram("PurgeableTombstoneScannedHistogram", cfs.keyspace.metric.purgeableTombstoneScannedHistogram, true); liveScannedHistogram = createTableHistogram("LiveScannedHistogram", cfs.keyspace.metric.liveScannedHistogram, false); colUpdateTimeDeltaHistogram = createTableHistogram("ColUpdateTimeDeltaHistogram", cfs.keyspace.metric.colUpdateTimeDeltaHistogram, false); coordinatorReadLatency = createTableTimer("CoordinatorReadLatency"); diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java index 89af53b175a8..34656eac55ae 100644 --- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java +++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java @@ -103,6 +103,7 @@ public class DatabaseDescriptorRefTest "org.apache.cassandra.config.ConfigurationLoader", "org.apache.cassandra.config.Config$CorruptedTombstoneStrategy", "org.apache.cassandra.config.Config$BatchlogEndpointStrategy", + "org.apache.cassandra.config.Config$TombstonesMetricGranularity", "org.apache.cassandra.config.DatabaseDescriptor$ByteUnit", "org.apache.cassandra.config.DataRateSpec", "org.apache.cassandra.config.DataRateSpec$DataRateUnit", diff --git a/test/unit/org/apache/cassandra/db/ReadCommandTest.java b/test/unit/org/apache/cassandra/db/ReadCommandTest.java index cc4ceee6952c..51a49c436587 100644 --- a/test/unit/org/apache/cassandra/db/ReadCommandTest.java +++ b/test/unit/org/apache/cassandra/db/ReadCommandTest.java @@ -35,6 +35,7 @@ import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.Util; +import org.apache.cassandra.config.Config; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.filter.ClusteringIndexSliceFilter; import org.apache.cassandra.db.filter.ColumnFilter; @@ -93,6 +94,9 @@ public class ReadCommandTest { + private static final String CREATE = "1"; + private static final String DELETE = "-1"; + private static final String KEYSPACE = "ReadCommandTest"; private static final String CF1 = "Standard1"; private static final String CF2 = "Standard2"; @@ -103,6 +107,12 @@ public class ReadCommandTest private static final String CF7 = "Counter7"; private static final String CF8 = "Standard8"; private static final String CF9 = "Standard9"; + private static final String CF10 = "Standard10"; + private static final String CF11 = "Standard11"; + private static final String CF12 = "Standard12"; + private static final String CF13 = "Standard13"; + private static final String CF14 = "Standard14"; + private static final InetAddressAndPort REPAIR_COORDINATOR; static { @@ -194,6 +204,61 @@ public static void defineSchema() throws ConfigurationException .addClusteringColumn("col", ReversedType.getInstance(Int32Type.instance)) .addRegularColumn("a", AsciiType.instance); + TableMetadata.Builder metadata10 = + TableMetadata.builder(KEYSPACE, CF10) + .addPartitionKeyColumn("key", BytesType.instance) + .addClusteringColumn("col", AsciiType.instance) + .addRegularColumn("a", AsciiType.instance) + .addRegularColumn("b", AsciiType.instance) + .addRegularColumn("c", AsciiType.instance) + .addRegularColumn("d", AsciiType.instance) + .addRegularColumn("e", AsciiType.instance) + .addRegularColumn("f", AsciiType.instance); + + TableMetadata.Builder metadata11 = + TableMetadata.builder(KEYSPACE, CF11) + .addPartitionKeyColumn("key", BytesType.instance) + .addClusteringColumn("col", AsciiType.instance) + .addRegularColumn("a", AsciiType.instance) + .addRegularColumn("b", AsciiType.instance) + .addRegularColumn("c", AsciiType.instance) + .addRegularColumn("d", AsciiType.instance) + .addRegularColumn("e", AsciiType.instance) + .addRegularColumn("f", AsciiType.instance); + + TableMetadata.Builder metadata12 = + TableMetadata.builder(KEYSPACE, CF12) + .addPartitionKeyColumn("key", BytesType.instance) + .addClusteringColumn("col", AsciiType.instance) + .addRegularColumn("a", AsciiType.instance) + .addRegularColumn("b", AsciiType.instance) + .addRegularColumn("c", AsciiType.instance) + .addRegularColumn("d", AsciiType.instance) + .addRegularColumn("e", AsciiType.instance) + .addRegularColumn("f", AsciiType.instance); + + TableMetadata.Builder metadata13 = + TableMetadata.builder(KEYSPACE, CF13) + .addPartitionKeyColumn("key", BytesType.instance) + .addClusteringColumn("col", AsciiType.instance) + .addRegularColumn("a", AsciiType.instance) + .addRegularColumn("b", AsciiType.instance) + .addRegularColumn("c", AsciiType.instance) + .addRegularColumn("d", AsciiType.instance) + .addRegularColumn("e", AsciiType.instance) + .addRegularColumn("f", AsciiType.instance); + + TableMetadata.Builder metadata14 = + TableMetadata.builder(KEYSPACE, CF14) + .addPartitionKeyColumn("key", BytesType.instance) + .addClusteringColumn("col", AsciiType.instance) + .addRegularColumn("a", AsciiType.instance) + .addRegularColumn("b", AsciiType.instance) + .addRegularColumn("c", AsciiType.instance) + .addRegularColumn("d", AsciiType.instance) + .addRegularColumn("e", AsciiType.instance) + .addRegularColumn("f", AsciiType.instance); + SchemaLoader.prepareServer(); SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(1), @@ -205,7 +270,12 @@ public static void defineSchema() throws ConfigurationException metadata6, metadata7, metadata8, - metadata9); + metadata9, + metadata10, + metadata11, + metadata12, + metadata13, + metadata14); LocalSessionAccessor.startup(); } @@ -332,23 +402,23 @@ public void testSinglePartitionGroupMerge() throws Exception String[][][] groups = new String[][][] { new String[][] { - new String[] { "1", "key1", "aa", "a" }, // "1" indicates to create the data, "-1" to delete the row - new String[] { "1", "key2", "bb", "b" }, - new String[] { "1", "key3", "cc", "c" } + new String[] { CREATE, "key1", "aa", "a" }, + new String[] { CREATE, "key2", "bb", "b" }, + new String[] { CREATE, "key3", "cc", "c" } }, new String[][] { - new String[] { "1", "key3", "dd", "d" }, - new String[] { "1", "key2", "ee", "e" }, - new String[] { "1", "key1", "ff", "f" } + new String[] { CREATE, "key3", "dd", "d" }, + new String[] { CREATE, "key2", "ee", "e" }, + new String[] { CREATE, "key1", "ff", "f" } }, new String[][] { - new String[] { "1", "key6", "aa", "a" }, - new String[] { "1", "key5", "bb", "b" }, - new String[] { "1", "key4", "cc", "c" } + new String[] { CREATE, "key6", "aa", "a" }, + new String[] { CREATE, "key5", "bb", "b" }, + new String[] { CREATE, "key4", "cc", "c" } }, new String[][] { - new String[] { "-1", "key6", "aa", "a" }, - new String[] { "-1", "key2", "bb", "b" } + new String[] { DELETE, "key6", "aa", "a" }, + new String[] { DELETE, "key2", "bb", "b" } } }; @@ -371,7 +441,7 @@ public void testSinglePartitionGroupMerge() throws Exception for (String[] data : group) { - if (data[0].equals("1")) + if (data[0].equals(CREATE)) { new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes(data[1])) .clustering(data[2]) @@ -493,33 +563,32 @@ public void testCountDeletedRows() throws Exception String[][][] groups = new String[][][] { new String[][] { - new String[] { "1", "key1", "aa", "a" }, // "1" indicates to create the data, "-1" to delete the - // row - new String[] { "1", "key2", "bb", "b" }, - new String[] { "1", "key3", "cc", "c" } + new String[] { CREATE, "key1", "aa", "a" }, + new String[] { CREATE, "key2", "bb", "b" }, + new String[] { CREATE, "key3", "cc", "c" } }, new String[][] { - new String[] { "1", "key3", "dd", "d" }, - new String[] { "1", "key2", "ee", "e" }, - new String[] { "1", "key1", "ff", "f" } + new String[] { CREATE, "key3", "dd", "d" }, + new String[] { CREATE, "key2", "ee", "e" }, + new String[] { CREATE, "key1", "ff", "f" } }, new String[][] { - new String[] { "1", "key6", "aa", "a" }, - new String[] { "1", "key5", "bb", "b" }, - new String[] { "1", "key4", "cc", "c" } + new String[] { CREATE, "key6", "aa", "a" }, + new String[] { CREATE, "key5", "bb", "b" }, + new String[] { CREATE, "key4", "cc", "c" } }, new String[][] { - new String[] { "1", "key2", "aa", "a" }, - new String[] { "1", "key2", "cc", "c" }, - new String[] { "1", "key2", "dd", "d" } + new String[] { CREATE, "key2", "aa", "a" }, + new String[] { CREATE, "key2", "cc", "c" }, + new String[] { CREATE, "key2", "dd", "d" } }, new String[][] { - new String[] { "-1", "key6", "aa", "a" }, - new String[] { "-1", "key2", "bb", "b" }, - new String[] { "-1", "key2", "ee", "e" }, - new String[] { "-1", "key2", "aa", "a" }, - new String[] { "-1", "key2", "cc", "c" }, - new String[] { "-1", "key2", "dd", "d" } + new String[] { DELETE, "key6", "aa", "a" }, + new String[] { DELETE, "key2", "bb", "b" }, + new String[] { DELETE, "key2", "ee", "e" }, + new String[] { DELETE, "key2", "aa", "a" }, + new String[] { DELETE, "key2", "cc", "c" }, + new String[] { DELETE, "key2", "dd", "d" } } }; @@ -539,7 +608,7 @@ public void testCountDeletedRows() throws Exception for (String[] data : group) { - if (data[0].equals("1")) + if (data[0].equals(CREATE)) { new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes(data[1])) .clustering(data[2]) @@ -582,20 +651,19 @@ public void testCountWithNoDeletedRow() throws Exception String[][][] groups = new String[][][] { new String[][] { - new String[] { "1", "key1", "aa", "a" }, // "1" indicates to create the data, "-1" to delete the - // row - new String[] { "1", "key2", "bb", "b" }, - new String[] { "1", "key3", "cc", "c" } + new String[] { CREATE, "key1", "aa", "a" }, + new String[] { CREATE, "key2", "bb", "b" }, + new String[] { CREATE, "key3", "cc", "c" } }, new String[][] { - new String[] { "1", "key3", "dd", "d" }, - new String[] { "1", "key2", "ee", "e" }, - new String[] { "1", "key1", "ff", "f" } + new String[] { CREATE, "key3", "dd", "d" }, + new String[] { CREATE, "key2", "ee", "e" }, + new String[] { CREATE, "key1", "ff", "f" } }, new String[][] { - new String[] { "1", "key6", "aa", "a" }, - new String[] { "1", "key5", "bb", "b" }, - new String[] { "1", "key4", "cc", "c" } + new String[] { CREATE, "key6", "aa", "a" }, + new String[] { CREATE, "key5", "bb", "b" }, + new String[] { CREATE, "key4", "cc", "c" } } }; @@ -615,7 +683,7 @@ public void testCountWithNoDeletedRow() throws Exception for (String[] data : group) { - if (data[0].equals("1")) + if (data[0].equals(CREATE)) { new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes(data[1])) .clustering(data[2]) @@ -651,6 +719,315 @@ public void testCountWithNoDeletedRow() throws Exception assertEquals(1, cfs.metric.tombstoneScannedHistogram.cf.getSnapshot().getMax()); } + @Test + public void testCountPurgeableRowTombstones() throws Exception + { + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(CF10); + TestWriteOperation[] operations = new TestWriteOperation[] + { + TestWriteOperation.insert("key1", "aa", "a"), + TestWriteOperation.insert("key1", "ff", "f"), + + TestWriteOperation.insert("key2", "aa", "e"), + TestWriteOperation.deleteRow("key2", "aa", PURGEABLE_DELETION), + TestWriteOperation.deleteRow("key2", "bb", NEW_DELETION), + TestWriteOperation.deleteRow("key2", "cc", PURGEABLE_DELETION), + TestWriteOperation.deleteRow("key2", "dd", PURGEABLE_DELETION), + TestWriteOperation.deleteRow("key2", "ee", NEW_DELETION), + }; + + runTestWriteOperationsAndReadResults(cfs, operations, Config.TombstonesMetricGranularity.row); + + assertEquals(2, cfs.metric.purgeableTombstoneScannedHistogram.cf.getCount()); + assertEquals(0, cfs.metric.purgeableTombstoneScannedHistogram.cf.getSnapshot().getMin()); + assertEquals(3, cfs.metric.purgeableTombstoneScannedHistogram.cf.getSnapshot().getMax()); + } + @Test + public void testCountPurgeablePartitionTombstones() throws Exception + { + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(CF11); + TestWriteOperation[] operations = new TestWriteOperation[] + { + TestWriteOperation.insert("key1", "aa", "a"), + TestWriteOperation.insert("key1", "ff", "f"), + + TestWriteOperation.insert("key2", "aa", "a"), + TestWriteOperation.insert("key2", "cc", "c"), + TestWriteOperation.insert("key2", "dd", "d"), + + TestWriteOperation.deletePartition("key2", PURGEABLE_DELETION), + TestWriteOperation.deletePartition("key3", NEW_DELETION) + }; + runTestWriteOperationsAndReadResults(cfs, operations, Config.TombstonesMetricGranularity.row); + + assertEquals(3, cfs.metric.purgeableTombstoneScannedHistogram.cf.getCount()); + assertEquals(0, cfs.metric.purgeableTombstoneScannedHistogram.cf.getSnapshot().getMin()); + assertEquals(1, cfs.metric.purgeableTombstoneScannedHistogram.cf.getSnapshot().getMax()); + } + + @Test + public void testCountPurgeableCellTombstones() throws Exception + { + DatabaseDescriptor.setPurgeableTobmstonesMetricGranularity(Config.TombstonesMetricGranularity.cell); + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(CF12); + TestWriteOperation[] operations = new TestWriteOperation[] + { + TestWriteOperation.insert("key1", "aa", "a"), + TestWriteOperation.insert("key1", "ff", "f"), + + TestWriteOperation.insert("key2", "aa", "a"), + TestWriteOperation.deleteCell("key2", "aa", "b", PURGEABLE_DELETION), + TestWriteOperation.deleteCell("key2", "aa", "f", NEW_DELETION), + TestWriteOperation.insert("key2", "cc", "c"), + TestWriteOperation.insert("key2", "dd", "d") + }; + runTestWriteOperationsAndReadResults(cfs, operations, Config.TombstonesMetricGranularity.cell); + + assertEquals(2, cfs.metric.purgeableTombstoneScannedHistogram.cf.getCount()); + assertEquals(0, cfs.metric.purgeableTombstoneScannedHistogram.cf.getSnapshot().getMin()); + assertEquals(1, cfs.metric.purgeableTombstoneScannedHistogram.cf.getSnapshot().getMax()); + + } + + /** + * Test purgeable tombstones count for range tombstones with non-overlapping ranges, + * i.e. only Bound (not Boundary) Markers will be created and counted + */ + @Test + public void testCountPurgeableRangeTombstones_nonOverlappingRanges() throws Exception + { + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(CF13); + TestWriteOperation[] operations = new TestWriteOperation[] + { + TestWriteOperation.insert("key1", "aa", "a"), + TestWriteOperation.insert("key1", "ff", "f"), + + TestWriteOperation.insert("key2", "aa", "a"), + TestWriteOperation.insert("key2", "cc", "c"), + TestWriteOperation.insert("key2", "dd", "d"), + + TestWriteOperation.deleteRange("key2", "aa", "bb", NEW_DELETION), + TestWriteOperation.deleteRange("key2", "dd", "ee", PURGEABLE_DELETION), + TestWriteOperation.deleteRange("key2", "ff", "ff", PURGEABLE_DELETION) + }; + runTestWriteOperationsAndReadResults(cfs, operations, Config.TombstonesMetricGranularity.row); + + assertEquals(2, cfs.metric.purgeableTombstoneScannedHistogram.cf.getCount()); + assertEquals(0, cfs.metric.purgeableTombstoneScannedHistogram.cf.getSnapshot().getMin()); + assertEquals(4, cfs.metric.purgeableTombstoneScannedHistogram.cf.getSnapshot().getMax()); + } + + /** + * Test purgeable tombstones count for range tombstones with overlapping ranges + */ + @Test + public void testCountPurgeableRangeTombstones_overlappingRanges() throws Exception + { + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(CF14); + + TestWriteOperation[] operations = new TestWriteOperation[] + { + TestWriteOperation.insert("key1", "aa", "a"), + TestWriteOperation.insert("key1", "ff", "f"), + + TestWriteOperation.insert("key2", "aa", "a"), + TestWriteOperation.insert("key2", "bb", "b"), + TestWriteOperation.insert("key2", "cc", "c"), + TestWriteOperation.insert("key2", "dd", "d"), + TestWriteOperation.insert("key2", "ee", "e"), + + // this range tombstone is non-purgeable and overlaps with the next one, + // so it will create one non-purgeable bound marker + // and one non-purgeable boundary marker so TWO NON-PURGEABLE tombstones + TestWriteOperation.deleteRange("key2", "aa", "bb", NEW_DELETION), + + // this range tombstone is purgeable and overlaps with previous and next ones, + // so it will create one non-purgeable bound marker + // and one non-purgeable boundary marker so TWO non-purgeable tombstones will be counted + TestWriteOperation.deleteRange("key2", "bb", "ee", PURGEABLE_DELETION), + + // this range tombstone is purgeable and overlaps with previous one, + // it has a different deletion time to not combine into a single range, + // so it will create one non-purgeable boundary marker (same as previous one) + // and one purgeable bound marker, so it will increment purgeable tombstones counter on one, + // we expect TWO purgeable tombstones in total + TestWriteOperation.deleteRange("key2", "ee", "ff", PURGEABLE_DELETION - 1) + }; + + runTestWriteOperationsAndReadResults(cfs, operations, Config.TombstonesMetricGranularity.row); + + assertEquals(0, cfs.metric.purgeableTombstoneScannedHistogram.cf.getSnapshot().getMin()); + assertEquals(2, cfs.metric.purgeableTombstoneScannedHistogram.cf.getSnapshot().getMax()); + } + + + private static void runTestWriteOperationsAndReadResults(ColumnFamilyStore cfs, TestWriteOperation[] operations, Config.TombstonesMetricGranularity granularity) throws IOException + { + Config.TombstonesMetricGranularity original = DatabaseDescriptor.getPurgeableTobmstonesMetricGranularity(); + DatabaseDescriptor.setPurgeableTobmstonesMetricGranularity(granularity); + try + { + Set<String> usedPartitionKeys = runWriteOperations(cfs, operations); + runPartitionReadCommands(cfs, usedPartitionKeys); + } + finally + { + DatabaseDescriptor.setPurgeableTobmstonesMetricGranularity(original); + cfs.truncateBlocking(); + } + } + + private static void runPartitionReadCommands(ColumnFamilyStore cfs, Set<String> partitionKeys) throws IOException + { + List<SinglePartitionReadCommand> commands = new ArrayList<>(partitionKeys.size()); + long nowInSeconds = FBUtilities.nowInSeconds(); // all reads within a group must have the same nowInSec + for (String partitionKey : partitionKeys) + { + commands.add(getWholePartitionReadCommand(cfs, partitionKey, nowInSeconds)); + } + executeReadCommands(commands); + } + + private static Set<String> runWriteOperations(ColumnFamilyStore cfs, TestWriteOperation[] operations) + { + Set<String> usedPartitionKeys = new HashSet<>(); + for (TestWriteOperation operation : operations) + { + if (operation.type == OperationType.CREATE) + { + new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes(operation.partitionKey)) + .clustering(operation.clusteringKey) + .add(operation.columnName, ByteBufferUtil.bytes(operation.columnValue)) + .build() + .apply(); + } + else if (operation.type == OperationType.DELETE_PARTITION) + { + new Mutation(PartitionUpdate.simpleBuilder(cfs.metadata(), ByteBufferUtil.bytes(operation.partitionKey)) + .nowInSec(operation.deletionTime) + .delete() + .build()).apply(); + } + else if (operation.type == OperationType.DELETE_RANGE) + { + new RowUpdateBuilder(cfs.metadata(), operation.deletionTime, 0L, ByteBufferUtil.bytes(operation.partitionKey)) + .addRangeTombstone(operation.clusteringRangeStart, operation.clusteringRangeEnd).build().apply(); + } + else if (operation.type == OperationType.DELETE_ROW) + { + RowUpdateBuilder.deleteRowAt(cfs.metadata(), 0, operation.deletionTime, + ByteBufferUtil.bytes(operation.partitionKey), operation.clusteringKey + ).apply(); + } + else if (operation.type == OperationType.DELETE_CELL) + { + new RowUpdateBuilder(cfs.metadata(), operation.deletionTime, 0L, ByteBufferUtil.bytes(operation.partitionKey)) + .clustering(operation.clusteringKey) + .delete(operation.columnName) + .build().apply(); + } + + usedPartitionKeys.add(operation.partitionKey); + } + return usedPartitionKeys; + } + + private static final long NEW_DELETION = FBUtilities.nowInSeconds(); + private static final long PURGEABLE_DELETION = 42; + + private enum OperationType + { + CREATE, + DELETE_PARTITION, + DELETE_RANGE, + DELETE_ROW, + DELETE_CELL + } + + private static class TestWriteOperation + { + OperationType type; + String partitionKey; + String clusteringKey; + + String clusteringRangeStart, clusteringRangeEnd; + String columnName; + String columnValue = "bla"; + + long deletionTime; + + public TestWriteOperation(OperationType type, String partitionKey, + String clusteringKey, String clusteringRangeStart, String clusteringRangeEnd, + String columnName, long deletionTime) + { + this.type = type; + this.partitionKey = partitionKey; + this.clusteringKey = clusteringKey; + this.clusteringRangeStart = clusteringRangeStart; + this.clusteringRangeEnd = clusteringRangeEnd; + this.columnName = columnName; + this.deletionTime = deletionTime; + } + + public static TestWriteOperation insert(String partitionKey, String clusteringKey, + String columnName) + { + return new TestWriteOperation(OperationType.CREATE, partitionKey, clusteringKey, null, null, columnName, 0); + } + + public static TestWriteOperation deletePartition(String partitionKey, long deletionTime) + { + return new TestWriteOperation(OperationType.DELETE_PARTITION, partitionKey, + null, null, null, null, deletionTime); + } + + public static TestWriteOperation deleteRange(String partitionKey, String clusteringRangeStart, String clusteringRangeEnd, long deletionTime) + { + return new TestWriteOperation(OperationType.DELETE_RANGE, partitionKey, + null, clusteringRangeStart, clusteringRangeEnd, null, deletionTime); + } + + public static TestWriteOperation deleteRow(String partitionKey, String clusteringKey, long deletionTime) + { + return new TestWriteOperation(OperationType.DELETE_ROW, partitionKey, clusteringKey, + null, null, null, deletionTime); + } + + public static TestWriteOperation deleteCell(String partitionKey, String clusteringKey, String columnName, long deletionTime) + { + return new TestWriteOperation(OperationType.DELETE_CELL, partitionKey, clusteringKey, + null, null, columnName, deletionTime); + } + + + } + + private static void executeReadCommands(List<SinglePartitionReadCommand> commands) throws IOException + { + ReadQuery query = SinglePartitionReadCommand.Group.create(commands, DataLimits.NONE); + + try (ReadExecutionController executionController = query.executionController(); + UnfilteredPartitionIterator iter = query.executeLocally(executionController); + DataOutputBuffer buffer = new DataOutputBuffer()) + { + UnfilteredPartitionIterators.serializerForIntraNode().serialize(iter, + query.columnFilter(), + buffer, + MessagingService.current_version); + } + } + + private static SinglePartitionReadCommand getWholePartitionReadCommand(ColumnFamilyStore cfs, String partitionKey, long nowInSeconds) + { + ColumnFilter columnFilter = ColumnFilter.allRegularColumnsBuilder(cfs.metadata(), false).build(); + RowFilter rowFilter = RowFilter.create(true); + Slice slice = Slice.make(BufferClusteringBound.BOTTOM, BufferClusteringBound.TOP); + ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.with(cfs.metadata().comparator, slice), false); + return SinglePartitionReadCommand.create(cfs.metadata(), nowInSeconds, + columnFilter, rowFilter, + DataLimits.NONE, Util.dk(partitionKey), sliceFilter); + } + @Test public void testSinglePartitionSliceRepairedDataTracking() throws Exception { From 10c0481f462565fff1d7cf3df8dd7f1c14612e98 Mon Sep 17 00:00:00 2001 From: David Capwell <dcapwell@apache.org> Date: Tue, 28 Jan 2025 10:46:51 -0800 Subject: [PATCH 110/225] Forward port randomized testing improvements from cep-15-accord to trunk patch by David Capwell; reviewed by Caleb Rackliffe for CASSANDRA-20253 --- .../config/CassandraRelevantProperties.java | 2 +- .../dht/ReversedLongLocalPartitioner.java | 4 +- .../cassandra/schema/KeyspaceMetadata.java | 7 + .../cassandra/schema/ReplicationParams.java | 6 + .../fuzz/snapshots/SnapshotsTest.java | 18 +- test/unit/accord/utils/Gens.java | 244 ++++- test/unit/accord/utils/Property.java | 7 +- test/unit/accord/utils/RandomSource.java | 211 ++-- test/unit/accord/utils/random/Picker.java | 118 +++ .../cassandra/cql3/RandomSchemaTest.java | 6 +- .../cassandra/repair/FailedAckTest.java | 2 +- .../apache/cassandra/repair/FuzzTestBase.java | 2 +- .../apache/cassandra/utils/ASTGenerators.java | 929 ++++++++++++++++++ .../utils/AbstractTypeGenerators.java | 136 ++- .../cassandra/utils/CassandraGenerators.java | 735 ++++++++++++-- .../apache/cassandra/utils/Generators.java | 102 +- 16 files changed, 2319 insertions(+), 210 deletions(-) create mode 100644 test/unit/accord/utils/random/Picker.java create mode 100644 test/unit/org/apache/cassandra/utils/ASTGenerators.java diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java index 70a6ea702e08..b90389db0fe7 100644 --- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java +++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java @@ -554,7 +554,7 @@ public enum CassandraRelevantProperties TCM_USE_NO_OP_REPLICATOR("cassandra.test.use_no_op_replicator", "false"), TEST_BBFAILHELPER_ENABLED("test.bbfailhelper.enabled"), - TEST_BLOB_SHARED_SEED("cassandra.test.blob.shared.seed"), + TEST_BLOB_SHARED_SEED("cassandra.test.blob.shared.seed", "42"), TEST_BYTEMAN_TRANSFORMATIONS_DEBUG("cassandra.test.byteman.transformations.debug"), TEST_CASSANDRA_KEEPBRIEFBRIEF("cassandra.keepBriefBrief"), TEST_CASSANDRA_RELEVANT_PROPERTIES("org.apache.cassandra.conf.CassandraRelevantPropertiesTest"), diff --git a/src/java/org/apache/cassandra/dht/ReversedLongLocalPartitioner.java b/src/java/org/apache/cassandra/dht/ReversedLongLocalPartitioner.java index c56f3e920e0d..43f9ab832d18 100644 --- a/src/java/org/apache/cassandra/dht/ReversedLongLocalPartitioner.java +++ b/src/java/org/apache/cassandra/dht/ReversedLongLocalPartitioner.java @@ -24,6 +24,7 @@ import java.util.Map; import java.util.Random; +import com.google.common.annotations.VisibleForTesting; import com.google.common.primitives.Longs; import org.apache.cassandra.db.CachedHashDecoratedKey; @@ -155,7 +156,8 @@ public AbstractType<?> partitionOrdering() return LongType.instance; } - private static class ReversedLongLocalToken extends Token + @VisibleForTesting + public static class ReversedLongLocalToken extends Token { private final long token; diff --git a/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java b/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java index fff222334d4d..aec93c9f47b2 100644 --- a/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java +++ b/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java @@ -25,6 +25,7 @@ import javax.annotation.Nullable; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; import com.google.common.base.Objects; import com.google.common.collect.Iterables; @@ -84,6 +85,12 @@ private KeyspaceMetadata(String keyspaceName, Kind kind, KeyspaceParams params, this.replicationStrategy = AbstractReplicationStrategy.createReplicationStrategy(keyspaceName, params.replication); } + @VisibleForTesting + public static KeyspaceMetadata createUnsafe(String keyspaceName, Kind kind, KeyspaceParams params, Tables tables, Views views, Types types, UserFunctions functions) + { + return new KeyspaceMetadata(keyspaceName, kind, params, tables, views, types, functions); + } + public static KeyspaceMetadata create(String name, KeyspaceParams params) { return new KeyspaceMetadata(name, Kind.REGULAR, params, Tables.none(), Views.none(), Types.none(), UserFunctions.none()); diff --git a/src/java/org/apache/cassandra/schema/ReplicationParams.java b/src/java/org/apache/cassandra/schema/ReplicationParams.java index 277c95914617..da44292e0d63 100644 --- a/src/java/org/apache/cassandra/schema/ReplicationParams.java +++ b/src/java/org/apache/cassandra/schema/ReplicationParams.java @@ -64,6 +64,12 @@ private ReplicationParams(Class<? extends AbstractReplicationStrategy> klass, Ma this.options = ImmutableMap.copyOf(options); } + @VisibleForTesting + public static ReplicationParams fromStrategy(AbstractReplicationStrategy strategy) + { + return new ReplicationParams(strategy.getClass(), strategy.configOptions); + } + public static ReplicationParams local() { return new ReplicationParams(LocalStrategy.class, ImmutableMap.of()); diff --git a/test/distributed/org/apache/cassandra/fuzz/snapshots/SnapshotsTest.java b/test/distributed/org/apache/cassandra/fuzz/snapshots/SnapshotsTest.java index 8c82b1cb11b6..2145a0ac6af3 100644 --- a/test/distributed/org/apache/cassandra/fuzz/snapshots/SnapshotsTest.java +++ b/test/distributed/org/apache/cassandra/fuzz/snapshots/SnapshotsTest.java @@ -132,21 +132,21 @@ private Pair<SnapshotsHolder, Map<String, String>> generateParams() case 1: if (!state.truncatedSnapshots.isEmpty()) { - keyspace = state.rs.pick(state.truncatedSnapshots.keySet()).split("\\.")[0]; + keyspace = state.rs.pickUnorderedSet(state.truncatedSnapshots.keySet()).split("\\.")[0]; picked = true; } break; case 2: if (!state.droppedSnapshots.isEmpty()) { - keyspace = state.rs.pick(state.droppedSnapshots).split("\\.")[0]; + keyspace = state.rs.pickUnorderedSet(state.droppedSnapshots).split("\\.")[0]; picked = true; } break; case 3: if (!state.snapshots.isEmpty()) { - keyspace = state.rs.pick(state.snapshots).getKeyspaceName(); + keyspace = state.rs.pickUnorderedSet(state.snapshots).getKeyspaceName(); picked = true; } break; @@ -630,7 +630,7 @@ private void prepare() case 1: if (!state.truncatedSnapshots.isEmpty()) { - String randomKsTb = state.rs.pick(state.truncatedSnapshots.keySet()); + String randomKsTb = state.rs.pickUnorderedSet(state.truncatedSnapshots.keySet()); Integer numberOfTruncatedSnapshots = state.truncatedSnapshots.get(randomKsTb); if (numberOfTruncatedSnapshots == 1) state.truncatedSnapshots.remove(randomKsTb); @@ -645,14 +645,14 @@ private void prepare() case 2: if (!state.droppedSnapshots.isEmpty()) { - state.droppedSnapshots.remove(state.rs.pick(state.droppedSnapshots)); + state.droppedSnapshots.remove(state.rs.pickUnorderedSet(state.droppedSnapshots)); picked = true; } break; case 3: if (!state.snapshots.isEmpty()) { - TestSnapshot pickedSnapshot = state.rs.pick(state.snapshots); + TestSnapshot pickedSnapshot = state.rs.pickUnorderedSet(state.snapshots); state.snapshots.remove(pickedSnapshot); picked = true; } @@ -817,7 +817,7 @@ public Optional<String> pickRandomKeyspace(boolean inBuiltIncluded) if (withoutInBuilt.isEmpty()) return Optional.empty(); else - return Optional.of(rs.pick(withoutInBuilt)); + return Optional.of(rs.pickUnorderedSet(withoutInBuilt)); } public Optional<String> pickRandomKeyspace() @@ -825,7 +825,7 @@ public Optional<String> pickRandomKeyspace() if (schema.keySet().isEmpty()) return Optional.empty(); else - return Optional.of(rs.pick(schema.keySet())); + return Optional.of(rs.pickUnorderedSet(schema.keySet())); } public Optional<Pair<String, String>> pickRandomTable(boolean inBuiltIncluded) @@ -864,7 +864,7 @@ public Optional<Pair<String, String>> pickRandomTable(boolean inBuiltIncluded) if (keyspacesWithTables.isEmpty()) return Optional.empty(); - String randomKeyspaceWithTables = rs.pick(keyspacesWithTables.keySet()); + String randomKeyspaceWithTables = rs.pickUnorderedSet(keyspacesWithTables.keySet()); List<String> tables = keyspacesWithTables.get(randomKeyspaceWithTables); String randomTable = rs.pick(tables); diff --git a/test/unit/accord/utils/Gens.java b/test/unit/accord/utils/Gens.java index 975aee9f4f04..218189206f39 100644 --- a/test/unit/accord/utils/Gens.java +++ b/test/unit/accord/utils/Gens.java @@ -29,6 +29,7 @@ import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashMap; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.NavigableSet; @@ -43,6 +44,8 @@ import com.google.common.collect.Iterables; +import accord.utils.random.Picker; + public class Gens { private Gens() { } @@ -88,6 +91,63 @@ public static <T> Gen<T> oneOf(Map<Gen<T>, Integer> values) return rs -> gen.next(rs).next(rs); } + public static <T> OneOfBuilder<T> oneOf() + { + return new OneOfBuilder<>(); + } + + public static class OneOfBuilder<T> + { + private final Map<Gen<T>, Integer> weighted = new LinkedHashMap<>(); + private final Set<Gen<T>> unweighted = new LinkedHashSet<>(); + private Gen.IntGen unknownWeightGen = Gens.ints().between(1, 10); + + public OneOfBuilder<T> add(Gen<T> gen) + { + unweighted.add(gen); + return this; + } + + public OneOfBuilder<T> add(int weight, Gen<T> gen) + { + weighted.put(gen, weight); + return this; + } + + public OneOfBuilder<T> unknownWeights(Gen.IntGen gen) + { + this.unknownWeightGen = gen; + return this; + } + + public Gen<Gen<T>> buildWithDynamicWeights() + { + if (unweighted.isEmpty()) + { + Gen<T> gen = build(); + return i -> gen; + } + return rs -> { + Map<Gen<T>, Integer> commands = new LinkedHashMap<>(); + commands.putAll(weighted); + for (var gen : unweighted) + commands.put(gen, unknownWeightGen.nextInt(rs)); + var top = pick(commands); + return rs2 -> top.next(rs2).next(rs2); + }; + } + + public Gen<T> build() + { + Map<Gen<T>, Integer> commands = new LinkedHashMap<>(); + commands.putAll(weighted); + for (var gen : unweighted) + commands.put(gen, 1); + var top = pick(commands); + return rs -> top.next(rs).next(rs); + } + } + public static Gen.IntGen pickInt(int... ts) { return rs -> ts[rs.nextInt(0, ts.length)]; @@ -225,6 +285,95 @@ public static <T> Gen<T> pickZipf(List<T> array) }; } + public static Gen<Gen.IntGen> randomWeights(int[] array) + { + return rs -> { + float[] weights = Picker.randomWeights(rs, array.length); + return r -> array[index(r, weights)]; + }; + } + + public static Gen<Gen.LongGen> randomWeights(long[] array) + { + return rs -> { + float[] weights = Picker.randomWeights(rs, array.length); + return r -> array[index(r, weights)]; + }; + } + + public static <T> Gen<Gen<T>> randomWeights(T[] array) + { + return rs -> { + float[] weights = Picker.randomWeights(rs, array.length); + return r -> array[index(r, weights)]; + }; + } + + public static <T> Gen<Gen<T>> randomWeights(List<T> array) + { + return rs -> { + float[] weights = Picker.randomWeights(rs, array.size()); + return r -> array.get(index(r, weights)); + }; + } + + private static int index(RandomSource rs, float[] weights) + { + int i = Arrays.binarySearch(weights, rs.nextFloat()); + if (i < 0) i = -1 - i; + return i; + } + + public static Gen<Gen.IntGen> mixedDistribution(int minInclusive, int maxExclusive, int numBuckets) + { + int domainSize = (maxExclusive - minInclusive); + if (domainSize < 0) + throw new IllegalArgumentException("Range is too large; min=" + minInclusive + ", max=" + maxExclusive); + if (numBuckets <= 0 || numBuckets > domainSize) + throw new IllegalArgumentException("Num buckets must be between 1 and " + domainSize + "; given " + numBuckets); + int[] bucket, indexes; + bucket = new int[numBuckets]; + int delta = domainSize / numBuckets; + for (int i = 0; i < numBuckets; i++) + bucket[i] = minInclusive + i * delta; + indexes = IntStream.range(0, bucket.length).toArray(); + Gen<Gen.IntGen> indexDistro = mixedDistribution(indexes); + return rs -> { + Gen.IntGen indexGen = indexDistro.next(rs); + switch (rs.nextInt(0, 2)) + { + case 0: // uniform + { + return r -> { + int idx = indexGen.next(rs); + int start = bucket[idx]; + int end = idx == bucket.length - 1 ? maxExclusive : bucket[idx + 1]; + return r.nextInt(start, end); + }; + } + case 1: // median biased + { + int medians[] = new int[bucket.length]; + for (int i = 0; i < medians.length; i++) + { + int start = bucket[i]; + int end = i == bucket.length - 1 ? maxExclusive : bucket[i + 1]; + medians[i] = rs.nextInt(start, end); + } + return r -> { + int idx = indexGen.next(rs); + int start = bucket[idx]; + int end = idx == bucket.length - 1 ? maxExclusive : bucket[idx + 1]; + int median = medians[idx]; + return r.nextBiasedInt(start, median, end); + }; + } + default: + throw new AssertionError(); + } + }; + } + public static Gen<Gen.IntGen> mixedDistribution(int minInclusive, int maxExclusive) { int domainSize = (maxExclusive - minInclusive + 1); @@ -246,11 +395,14 @@ public static Gen<Gen.IntGen> mixedDistribution(int minInclusive, int maxExclusi indexes = null; } return rs -> { - switch (rs.nextInt(0, 2)) + switch (rs.nextInt(0, 4)) { case 0: // uniform return r -> r.nextInt(minInclusive, maxExclusive); - case 1: // zipf + case 1: // median biased + int median = rs.nextInt(minInclusive, maxExclusive); + return r -> r.nextBiasedInt(minInclusive, median, maxExclusive); + case 2: // zipf if (indexes == null) return Gens.pickZipf(rs.nextBoolean() ? reverseAndCopy(array) : array); return Gens.pickZipf(rs.nextBoolean() ? reverseAndCopy(indexes) : indexes).mapAsInt((r, index) -> { @@ -258,6 +410,14 @@ public static Gen<Gen.IntGen> mixedDistribution(int minInclusive, int maxExclusi int end = index == array.length - 1 ? maxExclusive : array[index + 1]; return r.nextInt(start, end); }); + case 3: // random weight + if (indexes == null) + return randomWeights(array).next(rs); + return randomWeights(indexes).next(rs).mapAsInt((r, index) -> { + int start = array[index]; + int end = index == array.length - 1 ? maxExclusive : array[index + 1]; + return r.nextInt(start, end); + }); default: throw new AssertionError(); } @@ -298,11 +458,14 @@ public static Gen<Gen.LongGen> mixedDistribution(long minInclusive, long maxExcl indexes = null; } return rs -> { - switch (rs.nextInt(0, 2)) + switch (rs.nextInt(0, 4)) { case 0: // uniform return r -> r.nextLong(minInclusive, maxExclusive); - case 1: // zipf + case 1: // median biased + long median = rs.nextLong(minInclusive, maxExclusive); + return r -> r.nextBiasedLong(minInclusive, median, maxExclusive); + case 2: // zipf if (indexes == null) return Gens.pickZipf(rs.nextBoolean() ? reverseAndCopy(array) : array); return Gens.pickZipf(rs.nextBoolean() ? reverseAndCopy(indexes) : indexes).mapAsLong((r, index) -> { @@ -310,6 +473,14 @@ public static Gen<Gen.LongGen> mixedDistribution(long minInclusive, long maxExcl long end = index == array.length - 1 ? maxExclusive : array[index + 1]; return r.nextLong(start, end); }); + case 3: // random weight + if (indexes == null) + return randomWeights(array).next(rs); + return randomWeights(indexes).next(rs).mapAsLong((r, index) -> { + long start = array[index]; + long end = index == array.length - 1 ? maxExclusive : array[index + 1]; + return r.nextLong(start, end); + }); default: throw new AssertionError(); } @@ -336,11 +507,14 @@ public static <T> Gen<Gen<T>> mixedDistribution(T... list) public static <T> Gen<Gen<T>> mixedDistribution(List<T> list) { return rs -> { - switch (rs.nextInt(0, 2)) + switch (rs.nextInt(0, 4)) { case 0: // uniform return r -> list.get(rs.nextInt(0, list.size())); - case 1: // zipf + case 1: // median biased + int median = rs.nextInt(0, list.size()); + return r -> list.get(r.nextBiasedInt(0, median, list.size())); + case 2: // zipf List<T> array = list; if (rs.nextBoolean()) { @@ -348,12 +522,55 @@ public static <T> Gen<Gen<T>> mixedDistribution(List<T> list) Collections.reverse(array); } return pickZipf(array); + case 3: // random weight + return randomWeights(list).next(rs); default: throw new AssertionError(); } }; } + public static <T> Gen<Gen.IntGen> mixedDistribution(int[] list) + { + return rs -> { + switch (rs.nextInt(0, 4)) + { + case 0: // uniform + return r -> list[rs.nextInt(0, list.length)]; + case 1: // median biased + int median = rs.nextInt(0, list.length); + return r -> list[r.nextBiasedInt(0, median, list.length)]; + case 2: // zipf + int[] array = list; + if (rs.nextBoolean()) + { + array = Arrays.copyOf(array, array.length); + reverse(array); + } + return pickZipf(array); + case 3: // random weight + return randomWeights(list).next(rs); + default: + throw new AssertionError(); + } + }; + } + + /** + * This is a change from accord as that uses {@link accord.utils.Utils#reverse}, which doesn't exist in this forward port. + * + * To avoid adding another class and merge conflicts to cep-15-accord, this method was inlined + */ + private static void reverse(int[] array) + { + for (int i = 0; i < array.length / 2; i++) + { + int tmp = array[i]; + array[i] = array[array.length- 1 - i]; + array[array.length - 1 - i] = tmp; + } + } + public static Gen<char[]> charArray(Gen.IntGen sizes, char[] domain) { return charArray(sizes, domain, (a, b) -> true); @@ -535,6 +752,11 @@ public Gen<Gen.IntGen> mixedDistribution(int minInclusive, int maxExclusive) { return Gens.mixedDistribution(minInclusive, maxExclusive); } + + public Gen<Gen.IntGen> mixedDistribution(int minInclusive, int maxExclusive, int numBuckets) + { + return Gens.mixedDistribution(minInclusive, maxExclusive, numBuckets); + } } public static class LongDSL { @@ -843,8 +1065,14 @@ public T next(RandomSource random) if (!bestEffort) { T value; - while (!seen.add((value = fn.next(random)))) {} - return value; + // 10k attempts + for (int i = 0; i < 10_000; i++) + { + if (seen.add((value = fn.next(random)))) + return value; + } + + throw new IllegalArgumentException("Could not generate a unique value after 10k attempts"); } else { diff --git a/test/unit/accord/utils/Property.java b/test/unit/accord/utils/Property.java index e6d0d1fcc9f6..79c29c5a41de 100644 --- a/test/unit/accord/utils/Property.java +++ b/test/unit/accord/utils/Property.java @@ -555,7 +555,7 @@ default void checkPostconditions(State state, Result expected, default void process(State state, SystemUnderTest sut) throws Throwable { checkPostconditions(state, apply(state), - sut, run(sut)); + sut, run(sut)); } } @@ -896,6 +896,11 @@ public CommandsBuilder<State, SystemUnderTest> addIf(Predicate<State> predicate, return addIf(predicate, (rs, state) -> cmd.next(rs)); } + public CommandsBuilder<State, SystemUnderTest> addIf(Predicate<State> predicate, Command<State, SystemUnderTest, ?> cmd) + { + return addIf(predicate, (rs, state) -> cmd); + } + public CommandsBuilder<State, SystemUnderTest> addIf(Predicate<State> predicate, Setup<State, SystemUnderTest> cmd) { if (conditionalCommands == null) diff --git a/test/unit/accord/utils/RandomSource.java b/test/unit/accord/utils/RandomSource.java index 4e60d7f805c5..ddba6237adb1 100644 --- a/test/unit/accord/utils/RandomSource.java +++ b/test/unit/accord/utils/RandomSource.java @@ -20,14 +20,22 @@ import java.util.ArrayList; import java.util.Comparator; +import java.util.EnumSet; +import java.util.LinkedHashSet; import java.util.List; -import java.util.NavigableSet; import java.util.Random; import java.util.Set; -import java.util.stream.DoubleStream; -import java.util.stream.IntStream; -import java.util.stream.LongStream; +import java.util.SortedSet; +import java.util.function.BooleanSupplier; +import java.util.function.IntSupplier; +import java.util.function.LongSupplier; +import java.util.function.Supplier; +import com.google.common.collect.Iterables; + +import accord.utils.random.Picker; + +// TODO (expected): merge with C* RandomSource public interface RandomSource { static RandomSource wrap(Random random) @@ -38,14 +46,40 @@ static RandomSource wrap(Random random) void nextBytes(byte[] bytes); boolean nextBoolean(); + default BooleanSupplier uniformBools() { return this::nextBoolean; } + default BooleanSupplier biasedUniformBools(float chance) { return () -> decide(chance); } + default Supplier<BooleanSupplier> biasedUniformBoolsSupplier(float minChance) + { + return () -> { + float chance = minChance + (1 - minChance)*nextFloat(); + return () -> decide(chance); + }; + } - int nextInt(); + /** + * Returns true with a probability of {@code chance}. This is logically the same as + * <pre>{@code nextFloat() < chance}</pre> + * + * @param chance cumulative probability in range [0..1] + */ + default boolean decide(float chance) + { + return nextFloat() < chance; + } - default int nextInt(int maxExclusive) + /** + * Returns true with a probability of {@code chance}. This is logically the same as + * <pre>{@code nextDouble() < chance}</pre> + * + * @param chance cumulative probability in range [0..1] + */ + default boolean decide(double chance) { - return nextInt(0, maxExclusive); + return nextDouble() < chance; } + int nextInt(); + default int nextInt(int maxExclusive) { return nextInt(0, maxExclusive); } default int nextInt(int minInclusive, int maxExclusive) { // this is diff behavior than ThreadLocalRandom, which returns nextInt @@ -74,29 +108,47 @@ else if (delta > 0) } return result; } - - default IntStream ints() + default int nextBiasedInt(int minInclusive, int median, int maxExclusive) { - return IntStream.generate(this::nextInt); + checkBiasedUniform(minInclusive, median, maxExclusive); + + int range = Math.max(maxExclusive - median, median - minInclusive) * 2; + int next = nextInt(range) - range/2; + next += median; + return next >= median ? next < maxExclusive ? next : nextInt(median, maxExclusive) + : next >= minInclusive ? next : minInclusive == median ? median : nextInt(minInclusive, median); } - default IntStream ints(int maxExclusive) + default IntSupplier uniformInts(int minInclusive, int maxExclusive) { return () -> nextInt(minInclusive, maxExclusive); } + default IntSupplier biasedUniformInts(int minInclusive, int median, int maxExclusive) { - return IntStream.generate(() -> nextInt(maxExclusive)); + checkBiasedUniform(minInclusive, median, maxExclusive); + return () -> nextBiasedInt(minInclusive, median, maxExclusive); } - - default IntStream ints(int minInclusive, int maxExclusive) + default Supplier<IntSupplier> biasedUniformIntsSupplier(int absoluteMinInclusive, int absoluteMaxExclusive, int minMedian, int maxMedian, int minRange, int maxRange) { - return IntStream.generate(() -> nextInt(minInclusive, maxExclusive)); + return biasedUniformIntsSupplier(absoluteMinInclusive, absoluteMaxExclusive, minMedian, (minMedian+maxMedian)/2, maxMedian, minRange, (minRange+maxRange)/2, maxRange); } - - long nextLong(); - - default long nextLong(long maxExclusive) + default Supplier<IntSupplier> biasedUniformIntsSupplier(int absoluteMinInclusive, int absoluteMaxExclusive, int minMedian, int medianMedian, int maxMedian, int minRange, int medianRange, int maxRange) { - return nextLong(0, maxExclusive); + checkBiasedUniform(minMedian, medianMedian, maxMedian); + checkBiasedUniform(minRange, medianRange, maxRange); + if (minMedian < absoluteMinInclusive) + throw new IllegalArgumentException(String.format("absoluteMin (%s) should be less than or equal to minMedian (%s)", absoluteMinInclusive, minMedian)); + if (maxMedian > absoluteMaxExclusive) + throw new IllegalArgumentException(String.format("absoluteMax (%s) should be greater than or equal to maxMedian (%s)", absoluteMaxExclusive, maxMedian)); + if (minRange < 1) + throw new IllegalArgumentException(String.format("minRange (%s) should be greater than or equal to 1", minRange)); + return () -> { + int median = nextBiasedInt(minMedian, medianMedian, maxMedian); + int minInclusive = Math.max(absoluteMinInclusive, median - nextBiasedInt(minRange, medianRange, maxRange)/2); + int maxExclusive = Math.min(absoluteMaxExclusive, median + (nextBiasedInt(minRange, medianRange, maxRange)+1)/2); + return biasedUniformInts(minInclusive, median, maxExclusive); + }; } + long nextLong(); + default long nextLong(long maxExclusive) { return nextLong(0, maxExclusive); } default long nextLong(long minInclusive, long maxExclusive) { // this is diff behavior than ThreadLocalRandom, which returns nextLong @@ -125,31 +177,57 @@ else if (delta > 0L) } return result; } - - default LongStream longs() + default long nextBiasedLong(long minInclusive, long median, long maxExclusive) { - return LongStream.generate(this::nextLong); + checkBiasedUniform(minInclusive, median, maxExclusive); + + long range = Math.max(maxExclusive - median, median - minInclusive) * 2; + long next = nextLong(range) - range/2; + next += median; + return next >= median ? next < maxExclusive ? next : nextLong(median, maxExclusive) + : next >= minInclusive ? next : minInclusive == median ? median : nextLong(minInclusive, median); } - default LongStream longs(long maxExclusive) + default LongSupplier uniformLongs(long minInclusive, long maxExclusive) { return () -> nextLong(minInclusive, maxExclusive); } + default LongSupplier biasedUniformLongs(long minInclusive, long median, long maxExclusive) + { + checkBiasedUniform(minInclusive, median, maxExclusive); + return () -> nextBiasedLong(minInclusive, median, maxExclusive); + } + default Supplier<LongSupplier> biasedUniformLongsSupplier(long absoluteMinInclusive, long absoluteMaxExclusive, long minMedian, long maxMedian, long minRange, long maxRange) + { + return biasedUniformLongsSupplier(absoluteMinInclusive, absoluteMaxExclusive, minMedian, (minMedian+maxMedian)/2, maxRange, minRange, (minRange+maxRange)/2, maxRange); + } + default Supplier<LongSupplier> biasedUniformLongsSupplier(long absoluteMinInclusive, long absoluteMaxExclusive, long minMedian, long medianMedian, long maxMedian, long minRange, long medianRange, long maxRange) { - return LongStream.generate(() -> nextLong(maxExclusive)); + checkBiasedUniform(minMedian, medianMedian, maxMedian); + checkBiasedUniform(minRange, medianRange, maxRange); + if (minMedian < absoluteMinInclusive) + throw new IllegalArgumentException(String.format("absoluteMin (%s) should be less than or equal to minMedian (%s)", absoluteMinInclusive, minMedian)); + if (maxMedian > absoluteMaxExclusive) + throw new IllegalArgumentException(String.format("absoluteMax (%s) should be greater than or equal to maxMedian (%s)", absoluteMaxExclusive, maxMedian)); + if (minRange < 1) + throw new IllegalArgumentException(String.format("minRange (%s) should be greater than or equal to 1", minRange)); + return () -> { + long median = nextBiasedLong(minMedian, medianMedian, maxMedian); + long minInclusive = Math.max(absoluteMinInclusive, median - nextBiasedLong(minRange, medianRange, maxRange)/2); + long maxExclusive = Math.min(absoluteMaxExclusive, median + (1+nextBiasedLong(minRange, medianRange, maxRange))/2); + return biasedUniformLongs(minInclusive, median, maxExclusive); + }; } - default LongStream longs(long minInclusive, long maxExclusive) + static void checkBiasedUniform(long minInclusive, long median, long maxExclusive) { - return LongStream.generate(() -> nextLong(minInclusive, maxExclusive)); + if (minInclusive > median) + throw new IllegalArgumentException(String.format("Min (%s) should be equal to or less than median (%d).", minInclusive, median)); + if (median >= maxExclusive) + throw new IllegalArgumentException(String.format("Median (%s) should be less than max (%d).", median, maxExclusive)); } float nextFloat(); double nextDouble(); - - default double nextDouble(double maxExclusive) - { - return nextDouble(0, maxExclusive); - } - + default double nextDouble(double maxExclusive) { return nextDouble(0, maxExclusive); } default double nextDouble(double minInclusive, double maxExclusive) { if (minInclusive >= maxExclusive) @@ -162,21 +240,6 @@ default double nextDouble(double minInclusive, double maxExclusive) return result; } - default DoubleStream doubles() - { - return DoubleStream.generate(this::nextDouble); - } - - default DoubleStream doubles(double maxExclusive) - { - return DoubleStream.generate(() -> nextDouble(maxExclusive)); - } - - default DoubleStream doubles(double minInclusive, double maxExclusive) - { - return DoubleStream.generate(() -> nextDouble(minInclusive, maxExclusive)); - } - double nextGaussian(); default int pickInt(int first, int second, int... rest) @@ -227,13 +290,32 @@ default long pickLong(long[] array, int offset, int length) return array[nextInt(offset, offset + length)]; } - default <T extends Comparable<T>> T pick(Set<T> set) + default <T> T pickOrderedSet(SortedSet<T> set) { + int offset = nextInt(0, set.size()); + return Iterables.get(set, offset); + } + + default <T> T pickOrderedSet(LinkedHashSet<T> set) + { + int offset = nextInt(0, set.size()); + return Iterables.get(set, offset); + } + + default <T extends Enum<T>> T pickOrderedSet(EnumSet<T> set) + { + int offset = nextInt(0, set.size()); + return Iterables.get(set, offset); + } + + default <T extends Comparable<? super T>> T pickUnorderedSet(Set<T> set) + { + if (set instanceof SortedSet) + return pickOrderedSet((SortedSet<T>) set); List<T> values = new ArrayList<>(set); // Non-ordered sets may have different iteration order on different environments, which would make a seed produce different histories! // To avoid such a problem, make sure to apply a deterministic function (sort). - if (!(set instanceof NavigableSet)) - values.sort(Comparator.naturalOrder()); + values.sort(Comparator.naturalOrder()); return pick(values); } @@ -266,32 +348,13 @@ default <T> T pick(List<T> values, int offset, int length) return values.get(nextInt(offset, offset + length)); } - void setSeed(long seed); + default <T> Supplier<T> randomWeightedPicker(T[] objects) { return Picker.WeightedObjectPicker.randomWeighted(this, objects); } + default <T> Supplier<T> randomWeightedPicker(T[] objects, float[] bias) { return Picker.WeightedObjectPicker.randomWeighted(this, objects, bias); } + default <T> Supplier<T> weightedPicker(T[] objects, float[] proportionalWeights) { return Picker.WeightedObjectPicker.weighted(this, objects, proportionalWeights); } + void setSeed(long seed); RandomSource fork(); - /** - * Returns true with a probability of {@code chance}. This logic is logically the same as - * <pre>{@code nextFloat() < chance}</pre> - * - * @param chance cumulative probability in range [0..1] - */ - default boolean decide(float chance) - { - return nextFloat() < chance; - } - - /** - * Returns true with a probability of {@code chance}. This logic is logically the same as - * <pre>{@code nextDouble() < chance}</pre> - * - * @param chance cumulative probability in range [0..1] - */ - default boolean decide(double chance) - { - return nextDouble() < chance; - } - default long reset() { long seed = nextLong(); diff --git a/test/unit/accord/utils/random/Picker.java b/test/unit/accord/utils/random/Picker.java new file mode 100644 index 000000000000..f12369d35fca --- /dev/null +++ b/test/unit/accord/utils/random/Picker.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package accord.utils.random; + +import java.util.Arrays; +import java.util.function.Supplier; + +import accord.utils.Invariants; +import accord.utils.RandomSource; + +public class Picker +{ + public static float[] randomWeights(RandomSource random, int length) + { + float[] weights = new float[length - 1]; + float sum = 0; + for (int i = 0 ; i < weights.length ; ++i) + weights[i] = sum += random.nextFloat(); + sum += random.nextFloat(); + for (int i = 0 ; i < weights.length ; ++i) + weights[i] /= sum; + return weights; + } + + static abstract class Weighted + { + final RandomSource random; + final float[] weights; + + public Weighted(RandomSource random, float[] weights) + { + this.random = random; + this.weights = weights; + } + + + static float[] randomWeights(RandomSource random, float[] bias) + { + float[] weights = new float[bias.length - 1]; + float sum = 0; + for (int i = 0 ; i < weights.length ; ++i) + weights[i] = sum += random.nextFloat() * bias[i]; + sum += random.nextFloat() * bias[weights.length]; + for (int i = 0 ; i < weights.length ; ++i) + weights[i] /= sum; + return weights; + } + + static float[] normaliseWeights(float[] input) + { + float[] output = new float[input.length - 1]; + float sum = 0; + for (int i = 0 ; i < output.length ; ++i) + output[i] = sum += input[i]; + sum += input[output.length]; + for (int i = 0 ; i < output.length ; ++i) + output[i] /= sum; + return output; + } + + int pickIndex() + { + int i = Arrays.binarySearch(weights, random.nextFloat()); + if (i < 0) i = -1 - i; + return i; + } + } + + public static class WeightedObjectPicker<T> extends Weighted implements Supplier<T> + { + final T[] values; + + private WeightedObjectPicker(RandomSource random, T[] values, float[] weights) + { + super(random, weights); + this.values = values; + } + + @Override + public T get() + { + return values[pickIndex()]; + } + + public static <T> WeightedObjectPicker<T> randomWeighted(RandomSource random, T[] values) + { + return new WeightedObjectPicker<>(random, values, Picker.randomWeights(random, values.length)); + } + + public static <T> WeightedObjectPicker<T> randomWeighted(RandomSource random, T[] values, float[] bias) + { + Invariants.checkArgument(values.length == bias.length); + return new WeightedObjectPicker<>(random, values, randomWeights(random, bias)); + } + + public static <T> WeightedObjectPicker<T> weighted(RandomSource random, T[] values, float[] proportionalWeights) + { + Invariants.checkArgument(values.length == proportionalWeights.length); + return new WeightedObjectPicker<>(random, values, normaliseWeights(proportionalWeights)); + } + } +} diff --git a/test/unit/org/apache/cassandra/cql3/RandomSchemaTest.java b/test/unit/org/apache/cassandra/cql3/RandomSchemaTest.java index 1eff610c84d1..00abdc9ca166 100644 --- a/test/unit/org/apache/cassandra/cql3/RandomSchemaTest.java +++ b/test/unit/org/apache/cassandra/cql3/RandomSchemaTest.java @@ -105,13 +105,11 @@ public void test() .withMaxDepth(2) .withDefaultSetKey(withoutUnsafeEquality) .withoutTypeKinds(AbstractTypeGenerators.TypeKind.COUNTER) - .withUDTNames(udtName) - .build()) + .withUDTNames(udtName)) .withPartitionColumnsCount(1) .withPrimaryColumnTypeGen(new TypeGenBuilder(withoutUnsafeEquality) // map of vector of map crossed the size cut-off for one of the tests, so changed max depth from 2 to 1, so we can't have the second map - .withMaxDepth(1) - .build()) + .withMaxDepth(1)) .withClusteringColumnsBetween(1, 2) .withRegularColumnsBetween(1, 5) .withStaticColumnsBetween(0, 2) diff --git a/test/unit/org/apache/cassandra/repair/FailedAckTest.java b/test/unit/org/apache/cassandra/repair/FailedAckTest.java index a8e1163befd2..c77a812f92ef 100644 --- a/test/unit/org/apache/cassandra/repair/FailedAckTest.java +++ b/test/unit/org/apache/cassandra/repair/FailedAckTest.java @@ -66,7 +66,7 @@ public void failedAck() RepairCoordinator repair = coordinator.repair(KEYSPACE, irOption(rs, coordinator, KEYSPACE, ignore -> TABLES), false); repair.run(); // make sure the failing node is not the coordinator, else messaging isn't used - InetAddressAndPort failingAddress = rs.pick(repair.state.getNeighborsAndRanges().participants); + InetAddressAndPort failingAddress = rs.pickUnorderedSet(repair.state.getNeighborsAndRanges().participants); Cluster.Node failingNode = cluster.nodes.get(failingAddress); RepairStage stage = stageGen.next(rs); switch (stage) diff --git a/test/unit/org/apache/cassandra/repair/FuzzTestBase.java b/test/unit/org/apache/cassandra/repair/FuzzTestBase.java index ad5e8220e280..044711a617b6 100644 --- a/test/unit/org/apache/cassandra/repair/FuzzTestBase.java +++ b/test/unit/org/apache/cassandra/repair/FuzzTestBase.java @@ -325,7 +325,7 @@ private static void createSchema() TableId id = ids.get(i); TableMetadata tableMetadata = new CassandraGenerators.TableMetadataBuilder().withKeyspaceName(ks).withTableName(name).withTableId(id).withTableKinds(TableMetadata.Kind.REGULAR) // shouldn't matter, just wanted to avoid UDT as that needs more setup - .withDefaultTypeGen(AbstractTypeGenerators.builder().withTypeKinds(AbstractTypeGenerators.TypeKind.PRIMITIVE).withoutPrimitive(EmptyType.instance).build()).build().generate(qt); + .withDefaultTypeGen(AbstractTypeGenerators.builder().withTypeKinds(AbstractTypeGenerators.TypeKind.PRIMITIVE).withoutPrimitive(EmptyType.instance)).build().generate(qt); tableBuilder.add(tableMetadata); } KeyspaceParams params = KeyspaceParams.simple(3); diff --git a/test/unit/org/apache/cassandra/utils/ASTGenerators.java b/test/unit/org/apache/cassandra/utils/ASTGenerators.java new file mode 100644 index 000000000000..ccb5ebaff15d --- /dev/null +++ b/test/unit/org/apache/cassandra/utils/ASTGenerators.java @@ -0,0 +1,929 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.EnumMap; +import java.util.EnumSet; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.OptionalInt; +import java.util.OptionalLong; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.function.BiFunction; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import javax.annotation.Nullable; + +import com.google.common.collect.Iterables; + +import org.apache.cassandra.cql3.ast.AssignmentOperator; +import org.apache.cassandra.cql3.ast.Bind; +import org.apache.cassandra.cql3.ast.CasCondition; +import org.apache.cassandra.cql3.ast.Conditional; +import org.apache.cassandra.cql3.ast.Expression; +import org.apache.cassandra.cql3.ast.Literal; +import org.apache.cassandra.cql3.ast.Mutation; +import org.apache.cassandra.cql3.ast.Operator; +import org.apache.cassandra.cql3.ast.Reference; +import org.apache.cassandra.cql3.ast.Select; +import org.apache.cassandra.cql3.ast.Symbol; +import org.apache.cassandra.cql3.ast.TableReference; +import org.apache.cassandra.cql3.ast.Txn; +import org.apache.cassandra.cql3.ast.TypeHint; +import org.apache.cassandra.cql3.ast.Value; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.IntegerType; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.db.marshal.MapType; +import org.apache.cassandra.db.marshal.SetType; +import org.apache.cassandra.db.marshal.ShortType; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.TableMetadata; +import org.quicktheories.core.Gen; +import org.quicktheories.core.RandomnessSource; +import org.quicktheories.generators.SourceDSL; +import org.quicktheories.impl.Constraint; + +import static org.apache.cassandra.utils.Generators.SYMBOL_GEN; + +public class ASTGenerators +{ + static Gen<Value> valueGen(Object value, AbstractType<?> type) + { + Gen<Boolean> bool = SourceDSL.booleans().all(); + return rnd -> bool.generate(rnd) ? new Bind(value, type) : new Literal(value, type); + } + + static Gen<Value> valueGen(AbstractType<?> type) + { + Gen<?> v = AbstractTypeGenerators.getTypeSupport(type).valueGen; + return rnd -> valueGen(v.generate(rnd), type).generate(rnd); + } + + private static <K, V> Map<K, V> assertDeterministic(Map<K, V> map) + { + if (map instanceof LinkedHashMap || map instanceof TreeMap || map instanceof EnumMap) + return map; + if (map.size() == 1) + return map; + throw new AssertionError("Unsupported map type: " + map.getClass()); + } + + /** + * Returns a list of all columns in a deterministic order. This method is similar to {@link TableMetadata#columns()}, + * but that method uses a hash order, so the values could be different from host to host or jvm to jvm... + */ + public static List<ColumnMetadata> allColumnsInFixedOrder(TableMetadata metadata) + { + List<ColumnMetadata> columns = new ArrayList<>(metadata.columns().size()); + metadata.allColumnsInSelectOrder().forEachRemaining(columns::add); + return columns; + } + + public static Gen<AssignmentOperator> assignmentOperatorGen(EnumSet<AssignmentOperator.Kind> allowed, Expression right) + { + if (allowed.isEmpty()) + throw new IllegalArgumentException("Unable to create a operator gen for empty set of allowed operators"); + if (allowed.size() == 1) + return SourceDSL.arbitrary().constant(new AssignmentOperator(Iterables.getFirst(allowed, null), right)); + + Gen<AssignmentOperator.Kind> kind = SourceDSL.arbitrary().pick(new ArrayList<>(allowed)); + return kind.map(k -> new AssignmentOperator(k, right)); + } + + public static Gen<Operator> operatorGen(Set<Operator.Kind> allowed, Expression e, Gen<Value> paramValueGen) + { + if (allowed.isEmpty()) + throw new IllegalArgumentException("Unable to create a operator gen for empty set of allowed operators"); + Gen<Operator.Kind> kindGen = allowed.size() == 1 ? + SourceDSL.arbitrary().constant(Iterables.getFirst(allowed, null)) + : SourceDSL.arbitrary().pick(new ArrayList<>(allowed)); + Gen<Boolean> bool = SourceDSL.booleans().all(); + return rnd -> { + Gen<Value> valueGen = paramValueGen; + Operator.Kind kind = kindGen.generate(rnd); + if (kind == Operator.Kind.SUBTRACT && e.type() instanceof MapType) + { + // `map - set` not `map - map` + valueGen = valueGen.map(v -> { + // since we know E is of type map we know the value is a map + Map<?, ?> map = (Map<?, ?>) v.value(); + Set<?> newValue = map.keySet(); + SetType<Object> newType = SetType.getInstance(((MapType) e.type()).nameComparator(), false); + return v.with(newValue, newType); + }); + } + Expression other = valueGen.generate(rnd); + Expression left, right; + if (bool.generate(rnd)) + { + left = e; + right = other; + } + else + { + left = other; + right = e; + } + //TODO (correctness): "(smallint) ? - 16250" failed, but is this general or is it a small int thing? + //NOTE: (smallint) of -11843 and 3749 failed as well... + //NOTE: (long) was found and didn't fail... + //NOTE: see https://the-asf.slack.com/archives/CK23JSY2K/p1724819303058669 - varint didn't fail but serialized using int32 which causes equality mismatches for pk/ck lookups + if (e.type().unwrap() == ShortType.instance + || e.type().unwrap() == IntegerType.instance) // seed=7525457176675272023L + { + left = new TypeHint(left); + right = new TypeHint(right); + } + return new Operator(kind, TypeHint.maybeApplyTypeHint(left), TypeHint.maybeApplyTypeHint(right)); + }; + } + + public static class ExpressionBuilder<T> + { + private final AbstractType<T> type; + private final EnumSet<Operator.Kind> allowedOperators; + private Gen<T> valueGen; + private Gen<Boolean> useOperator = SourceDSL.booleans().all(); + private BiFunction<Object, AbstractType<?>, Gen<Value>> literalOrBindGen = ASTGenerators::valueGen; + + public ExpressionBuilder(AbstractType<T> type) + { + this.type = type.unwrap(); + this.valueGen = AbstractTypeGenerators.getTypeSupport(this.type).valueGen; + this.allowedOperators = Operator.supportsOperators(this.type); + } + + public ExpressionBuilder withOperators() + { + useOperator = i -> true; + return this; + } + + public ExpressionBuilder withoutOperators() + { + useOperator = i -> false; + return this; + } + + public ExpressionBuilder allowOperators() + { + useOperator = SourceDSL.booleans().all(); + return this; + } + + public ExpressionBuilder withLiteralOrBindGen(BiFunction<Object, AbstractType<?>, Gen<Value>> literalOrBindGen) + { + this.literalOrBindGen = literalOrBindGen; + return this; + } + + public Gen<Expression> build() + { + //TODO (coverage): rather than single level operators, allow nested (a + b + c + d) + Gen<Value> leaf = rs -> literalOrBindGen.apply(valueGen.generate(rs), type).generate(rs); + return rs -> { + Expression e = leaf.generate(rs); + if (!allowedOperators.isEmpty() && useOperator.generate(rs)) + e = operatorGen(allowedOperators, e, leaf).generate(rs); + return e; + }; + } + } + + public static class SelectGenBuilder + { + private final TableMetadata metadata; + private Gen<List<Expression>> selectGen; + private Gen<Map<Symbol, Expression>> keyGen; + private Gen<Optional<Value>> limitGen; + private BiFunction<Object, AbstractType<?>, Gen<Value>> literalOrBindGen = ASTGenerators::valueGen; + + public SelectGenBuilder(TableMetadata metadata) + { + this.metadata = Objects.requireNonNull(metadata); + this.selectGen = selectColumns(metadata); + this.keyGen = partitionKeyGen(metadata); + + withDefaultLimit(); + } + + public SelectGenBuilder withLiteralOrBindGen(BiFunction<Object, AbstractType<?>, Gen<Value>> literalOrBindGen) + { + this.literalOrBindGen = literalOrBindGen; + return this; + } + + public SelectGenBuilder withSelectStar() + { + selectGen = ignore -> Collections.emptyList(); + return this; + } + + public SelectGenBuilder withDefaultLimit() + { + Gen<Optional<Value>> non = ignore -> Optional.empty(); + Constraint limitLength = Constraint.between(1, 10_000); + Gen<Optional<Value>> positive = rnd -> Optional.of(valueGen(Math.toIntExact(rnd.next(limitLength)), Int32Type.instance).generate(rnd)); + limitGen = non.mix(positive); + return this; + } + + public SelectGenBuilder withLimit1() + { + this.limitGen = rnd -> Optional.of(valueGen(1, Int32Type.instance).generate(rnd)); + return this; + } + + public SelectGenBuilder withoutLimit() + { + this.limitGen = ignore -> Optional.empty(); + return this; + } + + public SelectGenBuilder withKeys(Gen<Map<Symbol, Object>> partitionKeys, Gen<Map<Symbol, Object>> clusteringKeys) + { + keyGen = rs -> { + Map<Symbol, Expression> keys = new LinkedHashMap<>(); + for (Map.Entry<Symbol, Object> e : assertDeterministic(partitionKeys.generate(rs)).entrySet()) + keys.put(e.getKey(), literalOrBindGen.apply(e.getValue(), e.getKey().type()).generate(rs)); + if (!metadata.clusteringColumns().isEmpty()) + { + for (Map.Entry<Symbol, Object> e : assertDeterministic(clusteringKeys.generate(rs)).entrySet()) + keys.put(e.getKey(), literalOrBindGen.apply(e.getValue(), e.getKey().type()).generate(rs)); + } + return keys; + }; + return this; + } + + public Gen<Select> build() + { + Optional<TableReference> ref = Optional.of(TableReference.from(metadata)); + return rnd -> { + List<Expression> select = selectGen.generate(rnd); + Conditional keyClause = and(keyGen.generate(rnd)); + Optional<Value> limit = limitGen.generate(rnd); + return new Select(select, ref, Optional.of(keyClause), Optional.empty(), limit); + }; + } + + private static Conditional and(Map<Symbol, Expression> data) + { + Conditional.Builder builder = new Conditional.Builder(); + for (Map.Entry<Symbol, Expression> e : assertDeterministic(data).entrySet()) + builder.where(e.getKey(), Conditional.Where.Inequality.EQUAL, e.getValue()); + return builder.build(); + } + + private static Gen<List<Expression>> selectColumns(TableMetadata metadata) + { + List<ColumnMetadata> columns = allColumnsInFixedOrder(metadata); + Constraint between = Constraint.between(0, columns.size() - 1); + Gen<int[]> indexGen = rnd -> { + int size = Math.toIntExact(rnd.next(between)) + 1; + Set<Integer> dedup = new LinkedHashSet<>(); + while (dedup.size() < size) + dedup.add(Math.toIntExact(rnd.next(between))); + return dedup.stream().mapToInt(Integer::intValue).toArray(); + }; + return rnd -> { + int[] indexes = indexGen.generate(rnd); + List<Expression> es = new ArrayList<>(indexes.length); + IntStream.of(indexes).mapToObj(columns::get).forEach(c -> es.add(new Symbol(c))); + return es; + }; + } + + private static Gen<Map<Symbol, Expression>> partitionKeyGen(TableMetadata metadata) + { + Map<ColumnMetadata, Gen<?>> gens = new LinkedHashMap<>(); + for (ColumnMetadata col : allColumnsInFixedOrder(metadata)) + gens.put(col, AbstractTypeGenerators.getTypeSupport(col.type).valueGen); + return rnd -> { + Map<Symbol, Expression> output = new LinkedHashMap<>(); + for (ColumnMetadata col : metadata.partitionKeyColumns()) + output.put(new Symbol(col), gens.get(col) + .map(o -> valueGen(o, col.type).generate(rnd)) + .generate(rnd)); + return output; + }; + } + } + + public static class MutationGenBuilder + { + public enum DeleteKind { Partition, Row, Column } + private final TableMetadata metadata; + private final LinkedHashSet<Symbol> allColumns; + private final LinkedHashSet<Symbol> partitionColumns, clusteringColumns; + private final LinkedHashSet<Symbol> primaryColumns; + private final LinkedHashSet<Symbol> regularColumns, staticColumns, regularAndStaticColumns; + private Gen<Mutation.Kind> kindGen = SourceDSL.arbitrary().enumValues(Mutation.Kind.class); + private Gen<OptionalInt> ttlGen = SourceDSL.integers().between(1, Math.toIntExact(TimeUnit.DAYS.toSeconds(10))).map(i -> i % 2 == 0 ? OptionalInt.empty() : OptionalInt.of(i)); + private Gen<OptionalLong> timestampGen = SourceDSL.longs().between(1, Long.MAX_VALUE).map(i -> i % 2 == 0 ? OptionalLong.empty() : OptionalLong.of(i)); + private Collection<Reference> references = Collections.emptyList(); + private Gen<Boolean> withCasGen = SourceDSL.booleans().all(); + private Gen<Boolean> useCasIf = SourceDSL.booleans().all(); + private BiFunction<RandomnessSource, List<Symbol>, List<Symbol>> ifConditionFilter = (rnd, symbols) -> symbols; + private Gen<DeleteKind> deleteKindGen = SourceDSL.arbitrary().enumValues(DeleteKind.class); + private Map<Symbol, ExpressionBuilder<?>> columnExpressions = new LinkedHashMap<>(); + + public MutationGenBuilder(TableMetadata metadata) + { + this.metadata = Objects.requireNonNull(metadata); + this.allColumns = Mutation.toSet(metadata::allColumnsInSelectOrder); + this.partitionColumns = Mutation.toSet(metadata.partitionKeyColumns()); + this.clusteringColumns = Mutation.toSet(metadata.clusteringColumns()); + this.primaryColumns = Mutation.toSet(metadata.primaryKeyColumns()); + this.regularColumns = Mutation.toSet(metadata.regularColumns()); + this.staticColumns = Mutation.toSet(metadata.staticColumns()); + this.regularAndStaticColumns = new LinkedHashSet<>(); + regularAndStaticColumns.addAll(staticColumns); + regularAndStaticColumns.addAll(regularColumns); + + for (Symbol symbol : allColumns) + columnExpressions.put(symbol, new ExpressionBuilder<>(symbol.type())); + } + + public MutationGenBuilder withDeletionKind(Gen<DeleteKind> deleteKindGen) + { + this.deleteKindGen = deleteKindGen; + return this; + } + + public MutationGenBuilder withDeletionKind(DeleteKind... values) + { + return withDeletionKind(SourceDSL.arbitrary().pick(values)); + } + + public MutationGenBuilder withLiteralOrBindGen(BiFunction<Object, AbstractType<?>, Gen<Value>> literalOrBindGen) + { + columnExpressions.values().forEach(e -> e.withLiteralOrBindGen(literalOrBindGen)); + return this; + } + + public MutationGenBuilder withoutTransaction() + { + withoutCas(); + return this; + } + + public MutationGenBuilder withCas() + { + withCasGen = SourceDSL.arbitrary().constant(true); + return this; + } + + public MutationGenBuilder withoutCas() + { + withCasGen = SourceDSL.arbitrary().constant(false); + return this; + } + + public MutationGenBuilder withCasGen(Gen<Boolean> withCasGen) + { + withCasGen = Objects.requireNonNull(withCasGen); + return this; + } + + public MutationGenBuilder withCasIf() + { + useCasIf = SourceDSL.arbitrary().constant(true); + return this; + } + + public MutationGenBuilder withoutCasIf() + { + useCasIf = SourceDSL.arbitrary().constant(false); + return this; + } + + public MutationGenBuilder withCasIfGen(Gen<Boolean> gen) + { + useCasIf = Objects.requireNonNull(gen); + return this; + } + + public MutationGenBuilder withIfColumnFilter(BiFunction<RandomnessSource, List<Symbol>, List<Symbol>> ifConditionFilter) + { + this.ifConditionFilter = Objects.requireNonNull(ifConditionFilter); + return this; + } + + public MutationGenBuilder withoutTimestamp() + { + timestampGen = ignore -> OptionalLong.empty(); + return this; + } + + public MutationGenBuilder withoutTtl() + { + ttlGen = ignore -> OptionalInt.empty(); + return this; + } + + public MutationGenBuilder withOperators() + { + columnExpressions.values().forEach(e -> e.withOperators()); + return this; + } + + public MutationGenBuilder withoutOperators() + { + columnExpressions.values().forEach(e -> e.withoutOperators()); + return this; + } + + public MutationGenBuilder withReferences(Collection<Reference> references) + { + this.references = references; + return this; + } + + private Gen<? extends Map<Symbol, Object>> partitionValueGen = null; + private Gen<? extends Map<Symbol, Object>> clusteringValueGen = null; + + public MutationGenBuilder withPartitions(Gen<? extends Map<Symbol, Object>> values) + { + this.partitionValueGen = values; + return this; + } + + public MutationGenBuilder withClusterings(Gen<? extends Map<Symbol, Object>> values) + { + this.clusteringValueGen = values; + return this; + } + + private static void values(RandomnessSource rnd, + Map<Symbol, ExpressionBuilder<?>> columnExpressions, + Conditional.EqBuilder<?> builder, + LinkedHashSet<Symbol> columns, + @Nullable Gen<? extends Map<Symbol, Object>> gen) + { + if (gen != null) + { + Map<Symbol, Object> map = gen.generate(rnd); + for (Map.Entry<Symbol, ?> e : assertDeterministic(map).entrySet()) + builder.value(e.getKey(), valueGen(e.getValue(), e.getKey().type()).generate(rnd)); + } + else + { + //TODO (coverage): support IN rather than just EQ + for (Symbol s : columns) + builder.value(s, columnExpressions.get(s).build().generate(rnd)); + } + } + + public Gen<Mutation> build() + { + Gen<Boolean> bool = SourceDSL.booleans().all(); + Map<? extends AbstractType<?>, List<Reference>> typeToReference = references.stream().collect(Collectors.groupingBy(Reference::type)); + return rnd -> { + Mutation.Kind kind = kindGen.generate(rnd); + // when there are not non-primary-columns then can't support UPDATE + if (kind == Mutation.Kind.UPDATE && regularColumns.isEmpty()) + { + int i; + int maxRetries = 42; + for (i = 0; i < maxRetries && kind == Mutation.Kind.UPDATE; i++) + kind = kindGen.generate(rnd); + if (i == maxRetries) + throw new IllegalArgumentException("Kind gen kept returning UPDATE, but not supported when there are no non-primary columns"); + } + boolean isCas = withCasGen.generate(rnd); + boolean isTransaction = isCas; //TODO (coverage): add accord support + switch (kind) + { + case INSERT: + { + Mutation.InsertBuilder builder = Mutation.insert(metadata); + if (isCas) + builder.ifNotExists(); + var ttl = ttlGen.generate(rnd); + if (ttl.isPresent()) + builder.ttl(valueGen(ttl.getAsInt(), Int32Type.instance).generate(rnd)); + var timestamp = timestampGen.generate(rnd); + if (timestamp.isPresent()) + builder.timestamp(valueGen(timestamp.getAsLong(), LongType.instance).generate(rnd)); + values(rnd, columnExpressions, builder, partitionColumns, partitionValueGen); + values(rnd, columnExpressions, builder, clusteringColumns, clusteringValueGen); + LinkedHashSet<Symbol> columnsToGenerate; + if (regularAndStaticColumns.isEmpty()) + { + columnsToGenerate = new LinkedHashSet<>(0); + } + else if (regularAndStaticColumns.size() == 1 || bool.generate(rnd)) + { + // all columns + columnsToGenerate = new LinkedHashSet<>(regularAndStaticColumns); + } + else + { + // subset + columnsToGenerate = new LinkedHashSet<>(subsetRegularAndStaticColumns(rnd)); + } + + generateRemaining(rnd, bool, Mutation.Kind.INSERT, isTransaction, typeToReference, builder, columnsToGenerate); + return builder.build(); + } + case UPDATE: + { + Mutation.UpdateBuilder builder = Mutation.update(metadata); + var ttl = ttlGen.generate(rnd); + if (ttl.isPresent()) + builder.ttl(valueGen(ttl.getAsInt(), Int32Type.instance).generate(rnd)); + var timestamp = timestampGen.generate(rnd); + if (timestamp.isPresent()) + builder.timestamp(valueGen(timestamp.getAsLong(), LongType.instance).generate(rnd)); + if (isCas) + { + if (useCasIf.generate(rnd)) + { + ifGen(new ArrayList<>(regularAndStaticColumns)).generate(rnd).ifPresent(c -> builder.ifCondition(c)); + } + else + { + builder.ifExists(); + } + } + values(rnd, columnExpressions, builder, partitionColumns, partitionValueGen); + values(rnd, columnExpressions, builder, clusteringColumns, clusteringValueGen); + + LinkedHashSet<Symbol> columnsToGenerate; + if (regularAndStaticColumns.size() == 1 || bool.generate(rnd)) + { + // all columns + columnsToGenerate = new LinkedHashSet<>(regularAndStaticColumns); + } + else + { + // subset must include a regular column + columnsToGenerate = new LinkedHashSet<>(subset(rnd, regularColumns)); + if (!staticColumns.isEmpty() && bool.generate(rnd)) + columnsToGenerate.addAll(subset(rnd, staticColumns)); + } + Conditional.EqBuilder<Mutation.UpdateBuilder> setBuilder = builder::set; + generateRemaining(rnd, bool, Mutation.Kind.UPDATE, isTransaction, typeToReference, setBuilder, columnsToGenerate); + return builder.build(); + } + case DELETE: + { + Mutation.DeleteBuilder builder = Mutation.delete(metadata); + + // 3 types of delete: partition, row, columns + DeleteKind deleteKind = deleteKindGen.generate(rnd); + // if there are no columns to delete, fallback to row + if (deleteKind == DeleteKind.Column && regularAndStaticColumns.isEmpty()) + deleteKind = DeleteKind.Row; + if (deleteKind == DeleteKind.Row && clusteringColumns.isEmpty()) + deleteKind = DeleteKind.Partition; + + values(rnd, columnExpressions, builder, partitionColumns, partitionValueGen); + + switch (deleteKind) + { + case Partition: + // nothing to do here, already handled + break; + case Row: + values(rnd, columnExpressions, builder, clusteringColumns, clusteringValueGen); + break; + case Column: + if (clusteringColumns.isEmpty()) + { + subsetRegularAndStaticColumns(rnd).forEach(builder::column); + } + else if (staticColumns.isEmpty()) + { + subset(rnd, regularColumns).forEach(builder::column); + values(rnd, columnExpressions, builder, clusteringColumns, clusteringValueGen); + } + else if (regularColumns.isEmpty()) + { + subset(rnd, staticColumns).forEach(builder::column); + } + else + { + // 2 possible states: + // 1) select a row then delete the columns + // 2) select a partition then select static columns only + if (bool.generate(rnd)) + { + // select static + subset(rnd, staticColumns).forEach(builder::column); + } + else + { + // select a row, at least 1 regular, and 0 or more statics + values(rnd, columnExpressions, builder, clusteringColumns, clusteringValueGen); + subset(rnd, regularColumns).forEach(builder::column); + if (bool.generate(rnd)) + subset(rnd, staticColumns).forEach(builder::column); + } + } + if (!clusteringColumns.isEmpty() && !staticColumns.isEmpty()) + { + if (bool.generate(rnd)) + { + // static only + subset(rnd, staticColumns).forEach(builder::column); + } + else + { + // mixed (piss + } + } + break; + default: + throw new UnsupportedOperationException(); + } + + var timestamp = timestampGen.generate(rnd); + if (timestamp.isPresent()) + builder.timestamp(valueGen(timestamp.getAsLong(), LongType.instance).generate(rnd)); + if (isCas) + { + boolean existAllowed = true; + List<Symbol> columns; + switch (deleteKind) + { + case Partition: + { + // As of this moment delete if partition exists does a full partition read, so its blocked + // due to being too costly... this query is logically correct so we should support as only + // liveness information is needed, but its not supported right now so need to work around + // see ML "[DISCUSS] CASSANDRA-20163 DELETE partition IF static column condition is currently blocked" + // I tried to enable delete partition if static column condition in CASSANDRA-20156, but was + // asked to abandon the patch for consistency reasons. + // Delete partition when there are clustering columns is unsupported, so avoid generating + if (clusteringColumns.isEmpty()) + { + // this is the same as delete row + columns = new ArrayList<>(regularAndStaticColumns); + existAllowed = true; + } + else + { + columns = Collections.emptyList(); + existAllowed = false; + } + } + break; + case Row: + { + columns = new ArrayList<>(regularAndStaticColumns); + } + break; + case Column: + { + // some column deletes support without clustering, others dont... to avoid + // relearning this, only allow conditions on the followin columns: + // 1) the columns in the query; only valid columns are present + // 2) static columns; these are always safe to include + LinkedHashSet<Symbol> uniq = new LinkedHashSet<>(builder.columns()); + uniq.addAll(staticColumns); + columns = new ArrayList<>(uniq); + } + break; + default: + throw new UnsupportedOperationException(deleteKind.name()); + } + if (!columns.isEmpty() && useCasIf.generate(rnd)) + { + ifGen(columns).generate(rnd).ifPresent(builder::ifCondition); + } + else if (existAllowed) + { + builder.ifExists(); + } + else + { + // can't do a CAS query + } + } + return builder.build(); + } + default: + throw new UnsupportedOperationException(kind.name()); + } + }; + } + + private void generateRemaining(RandomnessSource rnd, + Gen<Boolean> bool, + Mutation.Kind kind, + boolean isTransaction, + Map<? extends AbstractType<?>, List<Reference>> typeToReference, + Conditional.EqBuilder<?> builder, + LinkedHashSet<Symbol> columnsToGenerate) + { + //TODO (flexability): since expression offers visit to replace things, could also keep the expression in tact but just replace Value with the Reference? + if (!typeToReference.isEmpty()) + { + List<Symbol> allowed = new ArrayList<>(columnsToGenerate); + for (Symbol s : allowed) + { + List<Reference> matches = typeToReference.get(s.type()); + if (matches == null) + continue; + if (bool.generate(rnd)) + { + columnsToGenerate.remove(s); + builder.value(s, SourceDSL.arbitrary().pick(matches).generate(rnd)); + } + } + } + if (kind == Mutation.Kind.UPDATE && isTransaction) + { + for (Symbol c : new ArrayList<>(columnsToGenerate)) + { + var useOperator = columnExpressions.get(c).useOperator; + EnumSet<AssignmentOperator.Kind> additionOperatorAllowed = AssignmentOperator.supportsOperators(c.type()); + if (!additionOperatorAllowed.isEmpty() && useOperator.generate(rnd)) + { + Expression expression = columnExpressions.get(c).build().generate(rnd); + builder.value(c, assignmentOperatorGen(additionOperatorAllowed, expression).generate(rnd)); + columnsToGenerate.remove(c); + } + } + } + columnsToGenerate.forEach(s -> builder.value(s, columnExpressions.get(s).build().generate(rnd))); + } + + private List<Symbol> subsetRegularAndStaticColumns(RandomnessSource rnd) + { + return subset(rnd, regularAndStaticColumns); + } + + private static List<Symbol> subset(RandomnessSource rnd, LinkedHashSet<Symbol> columns) + { + if (columns.size() == 1) + return new ArrayList<>(columns); + int numColumns = Math.toIntExact(rnd.next(Constraint.between(1, columns.size()))); + List<Symbol> subset = Generators.uniqueList(SourceDSL.arbitrary().pick(new ArrayList<>(columns)), i -> numColumns).generate(rnd); + return subset; + } + + private Gen<Optional<CasCondition.IfCondition>> ifGen(List<Symbol> possibleColumns) + { + return rnd -> { + List<Symbol> symbols = ifConditionFilter.apply(rnd, possibleColumns); + if (symbols == null || symbols.isEmpty()) + return Optional.empty(); + Conditional.Builder builder = new Conditional.Builder(); + for (Symbol symbol : symbols) + builder.where(symbol, Conditional.Where.Inequality.EQUAL, columnExpressions.get(symbol).build().generate(rnd)); + return Optional.of(new CasCondition.IfCondition(builder.build())); + }; + } + } + + public static class TxnGenBuilder + { + public enum TxReturn { NONE, TABLE, REF} + private final TableMetadata metadata; + private Constraint letRange = Constraint.between(0, 3); + private Constraint ifUpdateRange = Constraint.between(1, 3); + private Constraint updateRange = Constraint.between(0, 3); + private Gen<Select> selectGen; + private Gen<TxReturn> txReturnGen = SourceDSL.arbitrary().enumValues(TxReturn.class); + private boolean allowReferences = true; + + public TxnGenBuilder(TableMetadata metadata) + { + this.metadata = metadata; + this.selectGen = new SelectGenBuilder(metadata) + .withLimit1() + .build(); + } + + public TxnGenBuilder withoutReferences() + { + this.allowReferences = false; + return this; + } + + public Gen<Txn> build() + { + Gen<Boolean> bool = SourceDSL.booleans().all(); + return rnd -> { + Txn.Builder builder = new Txn.Builder(); + do + { + int numLets = Math.toIntExact(rnd.next(letRange)); + for (int i = 0; i < numLets; i++) + { + // LET doesn't use normal symbol logic and acts closer to a common lanaguage; name does not lower + // case... it is possible that a reserved word gets used, so make sure to use a generator that + // filters those out. + String name; + while (builder.lets().containsKey(name = SYMBOL_GEN.generate(rnd))) {} + builder.addLet(name, selectGen.generate(rnd)); + } + Gen<Reference> refGen = SourceDSL.arbitrary().pick(new ArrayList<>(builder.allowedReferences())); + if (allowReferences) + { + switch (txReturnGen.generate(rnd)) + { + case REF: + { + if (!builder.allowedReferences().isEmpty()) + { + Gen<List<Reference>> refsGen = SourceDSL.lists().of(refGen).ofSizeBetween(1, Math.max(10, builder.allowedReferences().size())); + builder.addReturn(new Select((List<Expression>) (List<?>) refsGen.generate(rnd))); + } + } + break; + case TABLE: + builder.addReturn(selectGen.generate(rnd)); + break; + } + } + else + { + builder.addReturn(selectGen.generate(rnd)); + } + MutationGenBuilder mutationBuilder = new MutationGenBuilder(metadata) + .withoutCas() + .withoutTimestamp() + .withoutTtl() + .withReferences(new ArrayList<>(builder.allowedReferences())); + if (!allowReferences) + mutationBuilder.withReferences(Collections.emptyList()); + Gen<Mutation> updateGen = mutationBuilder.build(); + if (allowReferences && !builder.lets().isEmpty() && bool.generate(rnd)) + { + Gen<Conditional> conditionalGen = conditionalGen(refGen); + int numUpdates = Math.toIntExact(rnd.next(ifUpdateRange)); + List<Mutation> mutations = new ArrayList<>(numUpdates); + for (int i = 0; i < numUpdates; i++) + mutations.add(updateGen.generate(rnd)); + builder.addIf(new Txn.If(conditionalGen.generate(rnd), mutations)); + } + else + { + // Current limitation is that mutations are tied to the condition if present; can't have + // a condition and mutations that don't belong to it in v1... once multiple conditions are + // supported then can always attempt to add updates + int numUpdates = Math.toIntExact(rnd.next(updateRange)); + for (int i = 0; i < numUpdates; i++) + builder.addUpdate(updateGen.generate(rnd)); + } + } while (builder.isEmpty()); + return builder.build(); + }; + } + + private static Gen<Conditional> conditionalGen(Gen<Reference> refGen) + { + Constraint numConditionsConstraint = Constraint.between(1, 10); + return rnd -> { + //TODO support OR + Gen<Conditional.Where> whereGen = whereGen(refGen.generate(rnd)); + int size = Math.toIntExact(rnd.next(numConditionsConstraint)); + Conditional accum = whereGen.generate(rnd); + for (int i = 1; i < size; i++) + accum = new Conditional.And(accum, whereGen.generate(rnd)); + return accum; + }; + } + + private static Gen<Conditional.Where> whereGen(Reference ref) + { + Gen<Conditional.Where.Inequality> kindGen = SourceDSL.arbitrary().enumValues(Conditional.Where.Inequality.class); + Gen<?> dataGen = AbstractTypeGenerators.getTypeSupport(ref.type()).valueGen; + return rnd -> { + Conditional.Where.Inequality kind = kindGen.generate(rnd); + return Conditional.Where.create(kind, ref, valueGen(dataGen.generate(rnd), ref.type()).generate(rnd)); + }; + } + } +} diff --git a/test/unit/org/apache/cassandra/utils/AbstractTypeGenerators.java b/test/unit/org/apache/cassandra/utils/AbstractTypeGenerators.java index 257847cda622..902c37e92d60 100644 --- a/test/unit/org/apache/cassandra/utils/AbstractTypeGenerators.java +++ b/test/unit/org/apache/cassandra/utils/AbstractTypeGenerators.java @@ -41,6 +41,7 @@ import java.util.stream.Stream; import javax.annotation.Nullable; +import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Maps; @@ -243,17 +244,22 @@ public static <T> Releaser overridePrimitiveTypeSupport(AbstractType<T> type, Ty return () -> PRIMITIVE_TYPE_DATA_GENS.put(type, original); } - public static TypeGenBuilder withoutUnsafeEquality() + public static TypeGenBuilder withoutUnsafeEquality(TypeGenBuilder builder) { // make sure to keep UNSAFE_EQUALITY in-sync - return AbstractTypeGenerators.builder() - .withoutEmpty() - .withoutPrimitive(DurationType.instance) - // decimal "normalizes" the data to compare, so primary columns "may" mutate the data, causing missmatches - // see CASSANDRA-18530 - .withoutPrimitive(DecimalType.instance) - // counters are only for top level - .withoutTypeKinds(TypeKind.COUNTER); + return builder + .withoutEmpty() + .withoutPrimitive(DurationType.instance) + // decimal "normalizes" the data to compare, so primary columns "may" mutate the data, causing missmatches + // see CASSANDRA-18530 + .withoutPrimitive(DecimalType.instance) + // counters are only for top level + .withoutTypeKinds(TypeKind.COUNTER); + } + + public static TypeGenBuilder withoutUnsafeEquality() + { + return withoutUnsafeEquality(AbstractTypeGenerators.builder()); } public interface Releaser extends AutoCloseable @@ -274,6 +280,7 @@ public static class TypeGenBuilder private Function<Integer, Gen<AbstractType<?>>> defaultSetKeyFunc; private Predicate<AbstractType<?>> typeFilter = null; private Gen<String> udtName = null; + private Gen<Boolean> multiCellGen = BOOLEAN_GEN; public TypeGenBuilder() { @@ -297,6 +304,17 @@ public TypeGenBuilder(TypeGenBuilder other) typeFilter = other.typeFilter; } + public TypeGenBuilder withMultiCell(Gen<Boolean> multiCellGen) + { + this.multiCellGen = multiCellGen; + return this; + } + + public TypeGenBuilder withMultiCell(boolean multiCell) + { + return withMultiCell(i -> multiCell); + } + public TypeGenBuilder withTypeFilter(Predicate<AbstractType<?>> fn) { typeFilter = fn; @@ -456,7 +474,7 @@ else if (kinds != null) } else kindGen = SourceDSL.arbitrary().enumValues(TypeKind.class); - return buildRecursive(maxDepth, maxDepth, kindGen, BOOLEAN_GEN); + return buildRecursive(maxDepth, maxDepth, kindGen, multiCellGen); } private Gen<AbstractType<?>> buildRecursive(int maxDepth, int level, Gen<TypeKind> typeKindGen, Gen<Boolean> multiCellGen) @@ -732,6 +750,18 @@ public static Gen<UserType> userTypeGen(Gen<AbstractType<?>> elementGen, Gen<Int return userTypeGen(elementGen, sizeGen, ksGen, nameGen, BOOLEAN_GEN); } + private static ThreadLocal<String> OVERRIDE_KEYSPACE = new ThreadLocal<>(); + + public static void overrideUDTKeyspace(String ks) + { + OVERRIDE_KEYSPACE.set(ks); + } + + public static void clearUDTKeyspace() + { + OVERRIDE_KEYSPACE.remove(); + } + public static Gen<UserType> userTypeGen(Gen<AbstractType<?>> elementGen, Gen<Integer> sizeGen, Gen<String> ksGen, Gen<String> nameGen, Gen<Boolean> multiCellGen) { Gen<FieldIdentifier> fieldNameGen = IDENTIFIER_GEN.map(FieldIdentifier::forQuoted); @@ -740,7 +770,9 @@ public static Gen<UserType> userTypeGen(Gen<AbstractType<?>> elementGen, Gen<Int int numElements = sizeGen.generate(rnd); List<AbstractType<?>> fieldTypes = new ArrayList<>(numElements); LinkedHashSet<FieldIdentifier> fieldNames = new LinkedHashSet<>(numElements); - String ks = ksGen.generate(rnd); + String ks = OVERRIDE_KEYSPACE.get(); + if (ks == null) + ks = ksGen.generate(rnd); String name = nameGen.generate(rnd); ByteBuffer nameBB = AsciiType.instance.decompose(name); @@ -1089,6 +1121,49 @@ private static int normalizeSizeFromType(TypeSupport<?> keySupport, int size) return Math.min(size, uniq); } + public static boolean contains(AbstractType<?> type, AbstractType<?> searchFor) + { + return contains(type, searchFor::equals); + } + + public static boolean contains(AbstractType<?> type, Predicate<AbstractType<?>> searchFor) + { + class Found + { + boolean result = false; + } + Found found = new Found(); + visit(type, t -> { + if (searchFor.test(t)) + { + found.result = true; + return VisitAction.STOP; + } + return VisitAction.CONTINUE; + }); + return found.result; + } + + public enum VisitAction { CONTINUE, STOP} + + public static VisitAction visit(AbstractType<?> type, Function<AbstractType<?>, VisitAction> fn) + { + VisitAction action = fn.apply(type); + if (action == VisitAction.STOP) return action; + if (type.isReversed()) + { + type = type.unwrap(); + action = fn.apply(type); + if (action == VisitAction.STOP) return action; + } + for (AbstractType<?> t : type.subTypes()) + { + action = visit(t, fn); + if (action == VisitAction.STOP) return action; + } + return VisitAction.CONTINUE; + } + public static Set<UserType> extractUDTs(AbstractType<?> type) { Set<UserType> matches = new HashSet<>(); @@ -1098,12 +1173,11 @@ public static Set<UserType> extractUDTs(AbstractType<?> type) public static void extractUDTs(AbstractType<?> type, Set<UserType> matches) { - if (type instanceof ReversedType) - type = ((ReversedType) type).baseType; - if (type instanceof UserType) - matches.add((UserType) type); - for (AbstractType<?> t : type.subTypes()) - extractUDTs(t, matches); + visit(type, t -> { + if (t instanceof UserType) + matches.add((UserType) t); + return VisitAction.CONTINUE; + }); } public static String typeTree(AbstractType<?> type) @@ -1569,4 +1643,32 @@ private static <T extends AbstractType> void forEachCollectionTypeVariantsPair(T forEachTypesPair(frozenAndUnfrozen(l), frozenAndUnfrozen(r), typePairConsumer); } + public static TypeSupport<?> elementAccess(AbstractType<?> type) + { + type = type.unwrap(); + Preconditions.checkArgument(type.isCollection() || type.isUDT(), "Unexpected type: %s", type); + if (type.isUDT()) + { + // select a field + UserType ut = (UserType) type; + Gen<ByteBuffer> fieldNameGen = SourceDSL.arbitrary().pick(ut.fieldNames().stream().map(f -> f.bytes).collect(Collectors.toList())); + return new TypeSupport<>(BytesType.instance, fieldNameGen, ByteBuffer::compareTo); + } + else + { + CollectionType<?> ct = (CollectionType<?>) type; + switch (ct.kind) + { +// case SET: // set does not support element access; see org.apache.cassandra.db.marshal.MultiElementType.getElement + case LIST: + // by index + return new TypeSupport<>(Int32Type.instance, SourceDSL.integers().between(0, Integer.MAX_VALUE), Integer::compare); + case MAP: + // by key + return getTypeSupport(ct.nameComparator()); + default: throw new UnsupportedOperationException(ct.kind.name()); + } + } + } + } diff --git a/test/unit/org/apache/cassandra/utils/CassandraGenerators.java b/test/unit/org/apache/cassandra/utils/CassandraGenerators.java index 5c3fd3790c0f..ca2576cdcc17 100644 --- a/test/unit/org/apache/cassandra/utils/CassandraGenerators.java +++ b/test/unit/org/apache/cassandra/utils/CassandraGenerators.java @@ -21,23 +21,33 @@ import java.math.BigInteger; import java.net.InetAddress; import java.nio.ByteBuffer; +import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Deque; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.NavigableMap; +import java.util.Objects; import java.util.Set; +import java.util.TreeMap; import java.util.UUID; +import java.util.function.Consumer; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.Stream; - import javax.annotation.Nullable; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; import org.apache.commons.lang3.builder.MultilineRecursiveToStringStyle; import org.apache.commons.lang3.builder.ReflectionToStringBuilder; @@ -45,6 +55,7 @@ import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.Duration; import org.apache.cassandra.cql3.FieldIdentifier; +import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.ReadCommand; import org.apache.cassandra.db.SchemaCQLHelper; import org.apache.cassandra.db.SinglePartitionReadCommand; @@ -52,7 +63,9 @@ import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.ByteBufferAccessor; import org.apache.cassandra.db.marshal.CompositeType; +import org.apache.cassandra.db.marshal.CounterColumnType; import org.apache.cassandra.db.marshal.EmptyType; +import org.apache.cassandra.db.marshal.Int32Type; import org.apache.cassandra.db.rows.Cell; import org.apache.cassandra.db.marshal.UserType; import org.apache.cassandra.dht.ByteOrderedPartitioner; @@ -62,22 +75,40 @@ import org.apache.cassandra.dht.OrderPreservingPartitioner; import org.apache.cassandra.dht.RandomPartitioner; import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.ReversedLongLocalPartitioner; import org.apache.cassandra.dht.Token; +import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.gms.ApplicationState; import org.apache.cassandra.gms.EndpointState; import org.apache.cassandra.gms.HeartBeatState; import org.apache.cassandra.gms.VersionedValue; +import org.apache.cassandra.io.sstable.format.SSTableFormat; +import org.apache.cassandra.locator.AbstractReplicationStrategy; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.LocalStrategy; +import org.apache.cassandra.locator.MetaStrategy; +import org.apache.cassandra.locator.NetworkTopologyStrategy; +import org.apache.cassandra.locator.SimpleStrategy; import org.apache.cassandra.net.ConnectionType; import org.apache.cassandra.net.Message; import org.apache.cassandra.net.NoPayload; import org.apache.cassandra.net.PingRequest; import org.apache.cassandra.net.Verb; import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.schema.MemtableParams; +import org.apache.cassandra.schema.ReplicationParams; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.TableParams; +import org.apache.cassandra.schema.Tables; +import org.apache.cassandra.schema.Types; +import org.apache.cassandra.schema.UserFunctions; +import org.apache.cassandra.schema.Views; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.Epoch; +import org.apache.cassandra.utils.AbstractTypeGenerators.TypeGenBuilder; import org.apache.cassandra.utils.AbstractTypeGenerators.ValueDomain; import org.quicktheories.core.Gen; import org.quicktheories.core.RandomnessSource; @@ -87,6 +118,7 @@ import static org.apache.cassandra.utils.AbstractTypeGenerators.allowReversed; import static org.apache.cassandra.utils.AbstractTypeGenerators.getTypeSupport; +import static org.apache.cassandra.utils.AbstractTypeGenerators.withoutUnsafeEquality; import static org.apache.cassandra.utils.Generators.IDENTIFIER_GEN; import static org.apache.cassandra.utils.Generators.SMALL_TIME_SPAN_NANOS; import static org.apache.cassandra.utils.Generators.TIMESTAMP_NANOS; @@ -111,7 +143,6 @@ public final class CassandraGenerators return InetAddressAndPort.getByAddressOverrideDefaults(address, NETWORK_PORT_GEN.generate(rnd)); }; - public static final Gen<TableId> TABLE_ID_GEN = Generators.UUID_RANDOM_GEN.map(TableId::fromUUID); private static final Gen<TableMetadata.Kind> TABLE_KIND_GEN = SourceDSL.arbitrary().pick(TableMetadata.Kind.REGULAR, TableMetadata.Kind.INDEX, TableMetadata.Kind.VIRTUAL); public static final Gen<TableMetadata> TABLE_METADATA_GEN = gen(rnd -> createTableMetadata(IDENTIFIER_GEN.generate(rnd), rnd)).describedAs(CassandraGenerators::toStringRecursive); @@ -183,29 +214,304 @@ public static Gen<String> sstableFormatNames() return SourceDSL.arbitrary().pick("big", "bti"); } + public static Gen<SSTableFormat<?, ?>> sstableFormat() + { + // make sure ordering is determanstic, else repeatability breaks + NavigableMap<String, SSTableFormat<?, ?>> formats = new TreeMap<>(DatabaseDescriptor.getSSTableFormats()); + return SourceDSL.arbitrary().pick(new ArrayList<>(formats.values())); + } + + public static class AbstractReplicationStrategyBuilder + { + public enum Strategy + { + Simple(true), + NetworkTopology(true), + Local(false), + Meta(false); + + public final boolean userAllowed; + + Strategy(boolean userAllowed) + { + this.userAllowed = userAllowed; + } + } + + private Gen<Strategy> strategyGen = SourceDSL.arbitrary().enumValues(Strategy.class); + private Gen<String> keyspaceNameGen = KEYSPACE_NAME_GEN; + private Gen<Integer> rfGen = SourceDSL.integers().between(1, 3); + private Gen<List<String>> networkTopologyDCGen = rs -> { + Gen<Integer> numDcsGen = SourceDSL.integers().between(1, 3); + Gen<String> nameGen = IDENTIFIER_GEN; + Set<String> dcs = new HashSet<>(); + int targetSize = numDcsGen.generate(rs); + while (dcs.size() != targetSize) + dcs.add(nameGen.generate(rs)); + List<String> ordered = new ArrayList<>(dcs); + ordered.sort(Comparator.naturalOrder()); + return ordered; + }; + + public AbstractReplicationStrategyBuilder withKeyspace(Gen<String> keyspaceNameGen) + { + this.keyspaceNameGen = keyspaceNameGen; + return this; + } + + public AbstractReplicationStrategyBuilder withKeyspace(String keyspace) + { + this.keyspaceNameGen = i -> keyspace; + return this; + } + + public AbstractReplicationStrategyBuilder withUserAllowed() + { + List<Strategy> allowed = Stream.of(Strategy.values()).filter(s -> s.userAllowed).collect(Collectors.toList()); + strategyGen = SourceDSL.arbitrary().pick(allowed); + return this; + } + + public AbstractReplicationStrategyBuilder withRf(Gen<Integer> rfGen) + { + this.rfGen = rfGen; + return this; + } + + public AbstractReplicationStrategyBuilder withRf(int rf) + { + this.rfGen = i -> rf; + return this; + } + + public AbstractReplicationStrategyBuilder withDatacenters(Gen<List<String>> networkTopologyDCGen) + { + this.networkTopologyDCGen = networkTopologyDCGen; + return this; + } + + public AbstractReplicationStrategyBuilder withDatacenters(String first, String... rest) + { + if (rest.length == 0) + { + this.networkTopologyDCGen = i -> Collections.singletonList(first); + } + else + { + List<String> all = new ArrayList<>(rest.length + 1); + all.add(first); + all.addAll(Arrays.asList(rest)); + this.networkTopologyDCGen = i -> all; + } + return this; + } + + public Gen<AbstractReplicationStrategy> build() + { + return rs -> { + Strategy strategy = strategyGen.generate(rs); + switch (strategy) + { + case Simple: + return new SimpleStrategy(keyspaceNameGen.generate(rs), + ImmutableMap.of(SimpleStrategy.REPLICATION_FACTOR, rfGen.generate(rs).toString())); + case NetworkTopology: + ImmutableMap.Builder<String, String> builder = ImmutableMap.builder(); + List<String> names = networkTopologyDCGen.generate(rs); + for (String name : names) + builder.put(name, rfGen.generate(rs).toString()); + ImmutableMap<String, String> map = builder.build(); + return new TestableNetworkTopologyStrategy(keyspaceNameGen.generate(rs), map); + case Meta: + return new MetaStrategy(keyspaceNameGen.generate(rs), ImmutableMap.of()); + case Local: + return new LocalStrategy(keyspaceNameGen.generate(rs), ImmutableMap.of()); + default: + throw new UnsupportedOperationException(strategy.name()); + } + }; + } + } + + public static class TestableNetworkTopologyStrategy extends NetworkTopologyStrategy + { + public TestableNetworkTopologyStrategy(String keyspaceName, Map<String, String> configOptions) throws ConfigurationException + { + super(keyspaceName, configOptions); + } + + @Override + public Collection<String> recognizedOptions(ClusterMetadata metadata) + { + return configOptions.keySet(); + } + } + + public static KeyspaceMetadataBuilder regularKeyspace() + { + return new KeyspaceMetadataBuilder().withKind(KeyspaceMetadata.Kind.REGULAR); + } + + public static class KeyspaceMetadataBuilder + { + private Gen<String> nameGen = KEYSPACE_NAME_GEN; + private Gen<KeyspaceMetadata.Kind> kindGen = SourceDSL.arbitrary().enumValues(KeyspaceMetadata.Kind.class); + private Gen<AbstractReplicationStrategyBuilder> replicationGen = i -> new AbstractReplicationStrategyBuilder(); + private Gen<Boolean> durableWritesGen = SourceDSL.booleans().all(); + + public KeyspaceMetadataBuilder withReplication(Gen<AbstractReplicationStrategyBuilder> replicationGen) + { + this.replicationGen = replicationGen; + return this; + } + + public KeyspaceMetadataBuilder withReplication(AbstractReplicationStrategyBuilder replication) + { + this.replicationGen = i -> replication; + return this; + } + + public KeyspaceMetadataBuilder withName(Gen<String> nameGen) + { + this.nameGen = nameGen; + return this; + } + + public KeyspaceMetadataBuilder withName(String name) + { + this.nameGen = i -> name; + return this; + } + + public KeyspaceMetadataBuilder withKind(Gen<KeyspaceMetadata.Kind> kindGen) + { + this.kindGen = kindGen; + return this; + } + + public KeyspaceMetadataBuilder withKind(KeyspaceMetadata.Kind kind) + { + this.kindGen = i -> kind; + return this; + } + + public Gen<KeyspaceMetadata> build() + { + return rs -> { + String name = nameGen.generate(rs); + KeyspaceMetadata.Kind kind = kindGen.generate(rs); + AbstractReplicationStrategy replication = replicationGen.generate(rs).withKeyspace(nameGen).build().generate(rs); + ReplicationParams replicationParams = ReplicationParams.fromStrategy(replication); + boolean durableWrites = durableWritesGen.generate(rs); + KeyspaceParams params = new KeyspaceParams(durableWrites, replicationParams); + Tables tables = Tables.none(); + Views views = Views.none(); + Types types = Types.none(); + UserFunctions userFunctions = UserFunctions.none(); + return KeyspaceMetadata.createUnsafe(name, kind, params, tables, views, types, userFunctions); + }; + } + } + + public static class TableParamsBuilder + { + @Nullable + private Gen<String> memtableKeyGen = null; + + public TableParamsBuilder withKnownMemtables() + { + Set<String> known = MemtableParams.knownDefinitions(); + // for testing reason, some invalid types are added; filter out + List<String> valid = known.stream().filter(name -> !name.startsWith("test_")).collect(Collectors.toList()); + memtableKeyGen = SourceDSL.arbitrary().pick(valid); + return this; + } + + public Gen<TableParams> build() + { + return rnd -> { + TableParams.Builder params = TableParams.builder(); + if (memtableKeyGen != null) + params.memtable(MemtableParams.get(memtableKeyGen.generate(rnd))); + return params.build(); + }; + } + } + + public static TableMetadataBuilder regularTable() + { + return new TableMetadataBuilder() + .withTableKinds(TableMetadata.Kind.REGULAR) + .withKnownMemtables(); + } + public static class TableMetadataBuilder { private Gen<String> ksNameGen = CassandraGenerators.KEYSPACE_NAME_GEN; private Gen<String> tableNameGen = IDENTIFIER_GEN; - private Gen<AbstractType<?>> defaultTypeGen = AbstractTypeGenerators.builder() - .withDefaultSetKey(AbstractTypeGenerators.withoutUnsafeEquality()) - .withMaxDepth(1) - .build(); - private Gen<AbstractType<?>> partitionColTypeGen, clusteringColTypeGen, staticColTypeGen, regularColTypeGen; + private TypeGenBuilder defaultTypeGen = defaultTypeGen(); + private Gen<Boolean> useCounter = ignore -> false; + private TypeGenBuilder partitionColTypeGen, clusteringColTypeGen, staticColTypeGen, regularColTypeGen; private Gen<TableId> tableIdGen = TABLE_ID_GEN; private Gen<TableMetadata.Kind> tableKindGen = SourceDSL.arbitrary().constant(TableMetadata.Kind.REGULAR); private Gen<Integer> numPartitionColumnsGen = SourceDSL.integers().between(1, 2); private Gen<Integer> numClusteringColumnsGen = SourceDSL.integers().between(1, 2); private Gen<Integer> numRegularColumnsGen = SourceDSL.integers().between(1, 5); private Gen<Integer> numStaticColumnsGen = SourceDSL.integers().between(0, 2); - private Gen<String> memtableKeyGen = null; + @Nullable + private ColumnNameGen columnNameGen = null; + private TableParamsBuilder paramsBuilder = new TableParamsBuilder(); + private Gen<IPartitioner> partitionerGen = partitioners(); + + public static TypeGenBuilder defaultTypeGen() + { + return AbstractTypeGenerators.builder() + .withoutEmpty() + .withDefaultSetKey(withoutUnsafeEquality()) + .withMaxDepth(1) + .withoutTypeKinds(AbstractTypeGenerators.TypeKind.COUNTER); + } + + public TableMetadataBuilder withSimpleColumnNames() + { + columnNameGen = (i, kind, offset) -> { + switch (kind) + { + case PARTITION_KEY: return "pk" + offset; + case CLUSTERING: return "ck" + offset; + case STATIC: return "s" + offset; + case REGULAR: return "v" + offset; + default: throw new UnsupportedOperationException("Unknown kind: " + kind); + } + }; + return this; + } + + public TableMetadataBuilder withPartitioner(Gen<IPartitioner> partitionerGen) + { + this.partitionerGen = Objects.requireNonNull(partitionerGen); + return this; + } + + public TableMetadataBuilder withPartitioner(IPartitioner partitioner) + { + return withPartitioner(i -> partitioner); + } + + public TableMetadataBuilder withUseCounter(boolean useCounter) + { + return withUseCounter(ignore -> useCounter); + } + + public TableMetadataBuilder withUseCounter(Gen<Boolean> useCounter) + { + this.useCounter = Objects.requireNonNull(useCounter); + return this; + } public TableMetadataBuilder withKnownMemtables() { - Set<String> known = MemtableParams.knownDefinitions(); - // for testing reason, some invalid types are added; filter out - List<String> valid = known.stream().filter(name -> !name.startsWith("test_")).collect(Collectors.toList()); - memtableKeyGen = SourceDSL.arbitrary().pick(valid); + paramsBuilder.withKnownMemtables(); return this; } @@ -293,38 +599,44 @@ public TableMetadataBuilder withStaticColumnsBetween(int min, int max) return this; } - public TableMetadataBuilder withDefaultTypeGen(Gen<AbstractType<?>> typeGen) + public TableMetadataBuilder withDefaultTypeGen(TypeGenBuilder typeGen) { this.defaultTypeGen = typeGen; return this; } - public TableMetadataBuilder withPrimaryColumnTypeGen(Gen<AbstractType<?>> typeGen) + public TableMetadataBuilder withoutEmpty() + { + defaultTypeGen.withoutEmpty(); + return this; + } + + public TableMetadataBuilder withPrimaryColumnTypeGen(TypeGenBuilder typeGen) { withPartitionColumnTypeGen(typeGen); withClusteringColumnTypeGen(typeGen); return this; } - public TableMetadataBuilder withPartitionColumnTypeGen(Gen<AbstractType<?>> typeGen) + public TableMetadataBuilder withPartitionColumnTypeGen(TypeGenBuilder typeGen) { this.partitionColTypeGen = typeGen; return this; } - public TableMetadataBuilder withClusteringColumnTypeGen(Gen<AbstractType<?>> typeGen) + public TableMetadataBuilder withClusteringColumnTypeGen(TypeGenBuilder typeGen) { this.clusteringColTypeGen = typeGen; return this; } - public TableMetadataBuilder withStaticColumnTypeGen(Gen<AbstractType<?>> typeGen) + public TableMetadataBuilder withStaticColumnTypeGen(TypeGenBuilder typeGen) { this.staticColTypeGen = typeGen; return this; } - public TableMetadataBuilder withRegularColumnTypeGen(Gen<AbstractType<?>> typeGen) + public TableMetadataBuilder withRegularColumnTypeGen(TypeGenBuilder typeGen) { this.regularColTypeGen = typeGen; return this; @@ -343,42 +655,92 @@ public Gen<TableMetadata> build() public TableMetadata build(RandomnessSource rnd) { - if (partitionColTypeGen == null && clusteringColTypeGen == null) - withPrimaryColumnTypeGen(Generators.filter(defaultTypeGen, t -> !AbstractTypeGenerators.UNSAFE_EQUALITY.contains(t.getClass()))); + Gen<AbstractType<?>> partitionColTypeGen = withoutUnsafeEquality(new TypeGenBuilder(this.partitionColTypeGen != null ? this.partitionColTypeGen : defaultTypeGen)).build(); + Gen<AbstractType<?>> clusteringColTypeGen = withoutUnsafeEquality(new TypeGenBuilder(this.clusteringColTypeGen != null ? this.clusteringColTypeGen : defaultTypeGen)).build(); + Gen<AbstractType<?>> staticColTypeGen = (this.staticColTypeGen != null ? this.staticColTypeGen : defaultTypeGen).build(); + Gen<AbstractType<?>> regularColTypeGen = (this.regularColTypeGen != null ? this.regularColTypeGen : defaultTypeGen).build(); String ks = ksNameGen.generate(rnd); - String tableName = tableNameGen.generate(rnd); - TableParams.Builder params = TableParams.builder(); - if (memtableKeyGen != null) - params.memtable(MemtableParams.get(memtableKeyGen.generate(rnd))); - TableMetadata.Builder builder = TableMetadata.builder(ks, tableName, tableIdGen.generate(rnd)) - .partitioner(partitioners().generate(rnd)) - .kind(tableKindGen.generate(rnd)) - .isCounter(BOOLEAN_GEN.generate(rnd)) - .params(params.build()); - - int numPartitionColumns = numPartitionColumnsGen.generate(rnd); - int numClusteringColumns = numClusteringColumnsGen.generate(rnd); - int numRegularColumns = numRegularColumnsGen.generate(rnd); - int numStaticColumns = numStaticColumnsGen.generate(rnd); - - Set<String> createdColumnNames = new HashSet<>(); - for (int i = 0; i < numPartitionColumns; i++) - builder.addColumn(createColumnDefinition(ks, tableName, ColumnMetadata.Kind.PARTITION_KEY, createdColumnNames, partitionColTypeGen == null ? defaultTypeGen : partitionColTypeGen, rnd)); - for (int i = 0; i < numClusteringColumns; i++) - builder.addColumn(createColumnDefinition(ks, tableName, ColumnMetadata.Kind.CLUSTERING, createdColumnNames, clusteringColTypeGen == null ? defaultTypeGen : clusteringColTypeGen, rnd)); - for (int i = 0; i < numStaticColumns; i++) - builder.addColumn(createColumnDefinition(ks, tableName, ColumnMetadata.Kind.STATIC, createdColumnNames, staticColTypeGen == null ? defaultTypeGen : staticColTypeGen, rnd)); - for (int i = 0; i < numRegularColumns; i++) - builder.addColumn(createColumnDefinition(ks, tableName, ColumnMetadata.Kind.REGULAR, createdColumnNames, regularColTypeGen == null ? defaultTypeGen : regularColTypeGen, rnd)); + AbstractTypeGenerators.overrideUDTKeyspace(ks); + try + { + String tableName = tableNameGen.generate(rnd); + TableParams params = paramsBuilder.build().generate(rnd); + boolean isCounter = useCounter.generate(rnd); + TableMetadata.Builder builder = TableMetadata.builder(ks, tableName, tableIdGen.generate(rnd)) + .partitioner(partitionerGen.generate(rnd)) + .kind(tableKindGen.generate(rnd)) + .isCounter(isCounter) + .params(params); + + int numPartitionColumns = numPartitionColumnsGen.generate(rnd); + int numClusteringColumns = numClusteringColumnsGen.generate(rnd); + + ColumnNameGen nameGen; + if (columnNameGen != null) + { + nameGen = columnNameGen; + } + else + { + Set<String> createdColumnNames = new HashSet<>(); + // filter for unique names + nameGen = (r, i1, i2) -> { + String str; + while (!createdColumnNames.add(str = IDENTIFIER_GEN.generate(r))) + { + } + return str; + }; + } + for (int i = 0; i < numPartitionColumns; i++) + builder.addColumn(createColumnDefinition(ks, tableName, ColumnMetadata.Kind.PARTITION_KEY, i, nameGen, partitionColTypeGen, rnd)); + for (int i = 0; i < numClusteringColumns; i++) + builder.addColumn(createColumnDefinition(ks, tableName, ColumnMetadata.Kind.CLUSTERING, i, nameGen, clusteringColTypeGen, rnd)); - return builder.build(); + if (isCounter) + { + builder.addColumn(createColumnDefinition(ks, tableName, ColumnMetadata.Kind.REGULAR, 0, nameGen, ignore -> CounterColumnType.instance, rnd)); + } + else + { + int numRegularColumns = numRegularColumnsGen.generate(rnd); + int numStaticColumns = numStaticColumnsGen.generate(rnd); + for (int i = 0; i < numStaticColumns; i++) + builder.addColumn(createColumnDefinition(ks, tableName, ColumnMetadata.Kind.STATIC, i, nameGen, staticColTypeGen, rnd)); + for (int i = 0; i < numRegularColumns; i++) + builder.addColumn(createColumnDefinition(ks, tableName, ColumnMetadata.Kind.REGULAR, i, nameGen, regularColTypeGen, rnd)); + } + return builder.build(); + } + finally + { + AbstractTypeGenerators.clearUDTKeyspace(); + } } } + public static Gen<ColumnMetadata> columnMetadataGen(Gen<ColumnMetadata.Kind> kindGen, Gen<AbstractType<?>> typeGen) + { + Gen<String> ksNameGen = CassandraGenerators.KEYSPACE_NAME_GEN; + Gen<String> tableNameGen = IDENTIFIER_GEN; + return rs -> { + String ks = ksNameGen.generate(rs); + String table = tableNameGen.generate(rs); + ColumnMetadata.Kind kind = kindGen.generate(rs); + return createColumnDefinition(ks, table, kind, 0, (r, i1, i2) -> IDENTIFIER_GEN.generate(r), typeGen, rs); + }; + } + + public interface ColumnNameGen + { + String next(RandomnessSource rs, ColumnMetadata.Kind kind, int kindOffset); + } + private static ColumnMetadata createColumnDefinition(String ks, String table, ColumnMetadata.Kind kind, - Set<String> createdColumnNames, /* This is mutated to check for collisions, so has a side effect outside of normal random generation */ + int kindOffset, + ColumnNameGen nameGen, Gen<AbstractType<?>> typeGen, RandomnessSource rnd) { @@ -396,14 +758,12 @@ private static ColumnMetadata createColumnDefinition(String ks, String table, // when working on a clustering column, add in reversed types periodically typeGen = allowReversed(typeGen); } - // filter for unique names - String str; - while (!createdColumnNames.add(str = IDENTIFIER_GEN.generate(rnd))) - { - } + String str = nameGen.next(rnd, kind, kindOffset); + ColumnIdentifier name = new ColumnIdentifier(str, true); - int position = !kind.isPrimaryKeyKind() ? -1 : (int) rnd.next(Constraint.between(0, 30)); - return new ColumnMetadata(ks, table, name, typeGen.generate(rnd), position, kind, null); + int position = !kind.isPrimaryKeyKind() ? -1 : kindOffset; + AbstractType<?> type = typeGen.generate(rnd); + return new ColumnMetadata(ks, table, name, type, position, kind, null); } public static Gen<ByteBuffer> partitionKeyDataGen(TableMetadata metadata) @@ -425,27 +785,7 @@ public static Gen<ByteBuffer> partitionKeyDataGen(TableMetadata metadata) public static Gen<ByteBuffer[]> data(TableMetadata metadata, @Nullable Gen<ValueDomain> valueDomainGen) { - AbstractTypeGenerators.TypeSupport<?>[] types = new AbstractTypeGenerators.TypeSupport[metadata.columns().size()]; - Iterator<ColumnMetadata> it = metadata.allColumnsInSelectOrder(); - int partitionColumns = metadata.partitionKeyColumns().size(); - int clusteringColumns = metadata.clusteringColumns().size(); - int primaryKeyColumns = partitionColumns + clusteringColumns; - for (int i = 0; it.hasNext(); i++) - { - ColumnMetadata col = it.next(); - types[i] = AbstractTypeGenerators.getTypeSupportWithNulls(col.type, i < partitionColumns ? null : valueDomainGen); - if (i < partitionColumns) - types[i] = types[i].withoutEmptyData(); - if (i >= partitionColumns && i < primaryKeyColumns) - // clustering doesn't allow null... - types[i] = types[i].mapBytes(b -> b == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : b); - } - return rnd -> { - ByteBuffer[] row = new ByteBuffer[types.length]; - for (int i = 0; i < row.length; i++) - row[i] = types[i].bytesGen().generate(rnd); - return row; - }; + return new DataGeneratorBuilder(metadata).withValueDomain(valueDomainGen).build(); } /** @@ -573,7 +913,8 @@ public static Gen<Token> murmurTokenIn(Range<Token> range) public static Gen<Token> byteOrderToken() { - Constraint size = Constraint.between(0, 10); + // empty token only happens if partition key is byte[0], which isn't allowed + Constraint size = Constraint.between(1, 10); Constraint byteRange = Constraint.between(Byte.MIN_VALUE, Byte.MAX_VALUE); return rs -> { byte[] token = new byte[Math.toIntExact(rs.next(size))]; @@ -585,7 +926,9 @@ public static Gen<Token> byteOrderToken() public static Gen<Token> randomPartitionerToken() { - Constraint domain = Constraint.none(); + // valid range is -1 -> 2^127 + Constraint domain = Constraint.between(-1, Long.MAX_VALUE); + // TODO (coverage): handle the range [2^63-1, 2^127] return rs -> new RandomPartitioner.BigIntegerToken(BigInteger.valueOf(rs.next(domain))); } @@ -595,7 +938,7 @@ public static Gen<Token> localPartitionerToken(LocalPartitioner partitioner) return rs -> partitioner.getToken(bytes.generate(rs)); } - public static Gen<IPartitioner> localPartitioner() + public static Gen<LocalPartitioner> localPartitioner() { return AbstractTypeGenerators.safeTypeGen().map(LocalPartitioner::new); } @@ -610,9 +953,25 @@ public static Gen<Token> localPartitionerToken() }; } + public static Gen<Token> reversedLongLocalToken() + { + Constraint range = Constraint.between(0, Long.MAX_VALUE); + return rs -> new ReversedLongLocalPartitioner.ReversedLongLocalToken(rs.next(range)); + } + + public static Gen<ByteBuffer> reversedLongLocalKeys() + { + Constraint range = Constraint.between(0, Long.MAX_VALUE); + return rs -> { + long value = rs.next(range); + return ByteBufferUtil.bytes(value); + }; + } + public static Gen<Token> orderPreservingToken() { - Gen<String> string = Generators.utf8(0, 10); + // empty token only happens if partition key is byte[0], which isn't allowed + Gen<String> string = Generators.utf8(1, 10); return rs -> new OrderPreservingPartitioner.StringToken(string.generate(rs)); } @@ -625,23 +984,38 @@ public static Gen<Token> tokensInRange(Range<Token> range) private enum SupportedPartitioners { - Murmur(ignore -> Murmur3Partitioner.instance), - ByteOrdered(ignore -> ByteOrderedPartitioner.instance), - Random(ignore -> RandomPartitioner.instance), - Local(localPartitioner()), - OrderPreserving(ignore -> OrderPreservingPartitioner.instance); + Murmur(Murmur3Partitioner.class, ignore -> Murmur3Partitioner.instance), + ByteOrdered(ByteOrderedPartitioner.class, ignore -> ByteOrderedPartitioner.instance), + Random(RandomPartitioner.class, ignore -> RandomPartitioner.instance), + Local(LocalPartitioner.class, localPartitioner()), + OrderPreserving(OrderPreservingPartitioner.class, ignore -> OrderPreservingPartitioner.instance); - private final Gen<IPartitioner> partitioner; + private final Class<? extends IPartitioner> clazz; + private final Gen<? extends IPartitioner> partitioner; - SupportedPartitioners(Gen<IPartitioner> partitionerGen) + <T extends IPartitioner> SupportedPartitioners(Class<T> clazz, Gen<T> partitionerGen) { + this.clazz = clazz; partitioner = partitionerGen; } - public Gen<IPartitioner> partitioner() + public Gen<? extends IPartitioner> partitioner() { return partitioner; } + + public static Set<Class<? extends IPartitioner>> knownPartitioners() + { + ImmutableSet.Builder<Class<? extends IPartitioner>> builder = ImmutableSet.builder(); + for (SupportedPartitioners p : values()) + builder.add(p.clazz); + return builder.build(); + } + } + + public static Set<Class<? extends IPartitioner>> knownPartitioners() + { + return SupportedPartitioners.knownPartitioners(); } public static Gen<IPartitioner> partitioners() @@ -650,6 +1024,7 @@ public static Gen<IPartitioner> partitioners() .flatMap(SupportedPartitioners::partitioner); } + public static Gen<IPartitioner> nonLocalPartitioners() { return SourceDSL.arbitrary().enumValues(SupportedPartitioners.class) @@ -657,6 +1032,24 @@ public static Gen<IPartitioner> nonLocalPartitioners() .flatMap(SupportedPartitioners::partitioner); } + /** + * For {@link LocalPartitioner} it can have a very complex type which can lead to generating data larger than + * allowed in a primary key. If a test needs to filter out those cases, can just + * {@code .map(CassandraGenerators::simplify)} to resolve. + */ + public static IPartitioner simplify(IPartitioner partitioner) + { + // serializers require tokens to fit within 1 << 16, but that makes the test flakey when LocalPartitioner with a nested type is found... + if (!(partitioner instanceof LocalPartitioner)) return partitioner; + if (!shouldSimplify(partitioner.getTokenValidator())) return partitioner; + return new LocalPartitioner(Int32Type.instance); + } + + private static boolean shouldSimplify(AbstractType<?> type) + { + return AbstractTypeGenerators.contains(type, t -> t.isCollection()); + } + public static Gen<Token> token() { return partitioners().flatMap(CassandraGenerators::token); @@ -810,4 +1203,174 @@ public static Gen<Duration> duration() return Duration.newInstance(months, days, nanoseconds); }; } + + public static Gen<DecoratedKey> decoratedKeys() + { + return decoratedKeys(partitioners(), Generators.bytes(0, 100)); + } + + public static Gen<DecoratedKey> decoratedKeys(Gen<IPartitioner> partitionerGen) + { + return decoratedKeys(partitionerGen, Generators.bytes(0, 100)); + } + + public static Gen<DecoratedKey> decoratedKeys(Gen<IPartitioner> partitionerGen, Gen<ByteBuffer> keyGen) + { + return rs -> { + IPartitioner partitioner = partitionerGen.generate(rs); + Gen<ByteBuffer> valueGen = keyGen; + if (partitioner instanceof LocalPartitioner) + { + LocalPartitioner lp = (LocalPartitioner) partitioner; + valueGen = AbstractTypeGenerators.getTypeSupport(lp.getTokenValidator()).bytesGen(); + } + else if (partitioner instanceof ReversedLongLocalPartitioner) + { + valueGen = reversedLongLocalKeys(); + } + return partitioner.decorateKey(valueGen.generate(rs)); + }; + } + + public static void visitUDTs(TableMetadata metadata, Consumer<UserType> fn) + { + Set<UserType> udts = CassandraGenerators.extractUDTs(metadata); + if (!udts.isEmpty()) + { + Deque<UserType> pending = new ArrayDeque<>(udts); + Set<ByteBuffer> visited = new HashSet<>(); + while (!pending.isEmpty()) + { + UserType next = pending.poll(); + Set<UserType> subTypes = AbstractTypeGenerators.extractUDTs(next); + subTypes.remove(next); // it includes self + if (subTypes.isEmpty() || subTypes.stream().allMatch(t -> visited.contains(t.name))) + { + fn.accept(next); + visited.add(next.name); + } + else + { + pending.add(next); + } + } + } + } + + public static class DataGeneratorBuilder + { + private final TableMetadata metadata; + @Nullable + private Gen<ValueDomain> valueDomainGen = null; + + public DataGeneratorBuilder(TableMetadata metadata) + { + this.metadata = metadata; + } + + public DataGeneratorBuilder withValueDomain(@Nullable Gen<ValueDomain> valueDomainGen) + { + this.valueDomainGen = valueDomainGen; + return this; + } + + public Gen<Gen<ByteBuffer[]>> build(Gen<Integer> numUniqPartitionsGen) + { + AbstractTypeGenerators.TypeSupport<?>[] types = typeSupport(); + return rnd -> { + int numPartitions = numUniqPartitionsGen.generate(rnd); + Set<List<ByteBuffer>> partitions = Sets.newHashSetWithExpectedSize(numPartitions); + int partitionColumns = metadata.partitionKeyColumns().size(); + for (int i = 0; i < numPartitions; i++) + { + List<ByteBuffer> pk = new ArrayList<>(partitionColumns); + int attempts = 0; + do + { + attempts++; + pk.clear(); + for (int c = 0; c < partitionColumns; c++) + pk.add(types[c].bytesGen().generate(rnd)); + } + while (!partitions.add(pk) && attempts < 42); + } + List<List<ByteBuffer>> deterministicOrder = new ArrayList<>(partitions); + deterministicOrder.sort((a, b) -> { + int rc = 0; + for (int i = 0; i < a.size(); i++) + { + rc = a.get(i).compareTo(b.get(i)); + if (rc != 0) return rc; + } + return rc; + }); + + Gen<List<ByteBuffer>> pkGen = SourceDSL.arbitrary().pick(deterministicOrder); + + return next -> { + // select partition + List<ByteBuffer> pk = pkGen.generate(next); + // generate rest + ByteBuffer[] row = new ByteBuffer[types.length]; + for (int i = 0; i < pk.size(); i++) + row[i] = pk.get(i); + + for (int i = partitionColumns; i < row.length; i++) + row[i] = types[i].bytesGen().generate(rnd); + return row; + }; + }; + } + + public Gen<ByteBuffer[]> build() + { + AbstractTypeGenerators.TypeSupport<?>[] types = typeSupport(); + return rnd -> { + ByteBuffer[] row = new ByteBuffer[types.length]; + for (int i = 0; i < row.length; i++) + row[i] = types[i].bytesGen().generate(rnd); + return row; + }; + } + + private AbstractTypeGenerators.TypeSupport<?>[] typeSupport() + { + AbstractTypeGenerators.TypeSupport<?>[] types = new AbstractTypeGenerators.TypeSupport[metadata.columns().size()]; + Iterator<ColumnMetadata> it = metadata.allColumnsInSelectOrder(); + int partitionColumns = metadata.partitionKeyColumns().size(); + int clusteringColumns = metadata.clusteringColumns().size(); + int primaryKeyColumns = partitionColumns + clusteringColumns; + for (int i = 0; it.hasNext(); i++) + { + ColumnMetadata col = it.next(); + types[i] = AbstractTypeGenerators.getTypeSupportWithNulls(col.type, i < partitionColumns ? null : valueDomainGen); + if (i < partitionColumns) + types[i] = types[i].withoutEmptyData(); + if (i >= partitionColumns && i < primaryKeyColumns) + // clustering doesn't allow null... + types[i] = types[i].mapBytes(b -> b == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : b); + } + return types; + } + } + + private enum EpochConstants { FIRST, EMPTY, UPGRADE_STARTUP, UPGRADE_GOSSIP} + public static Gen<Epoch> epochs() + { + return rnd -> { + if (SourceDSL.booleans().all().generate(rnd)) + { + switch (SourceDSL.arbitrary().enumValues(EpochConstants.class).generate(rnd)) + { + case FIRST: return Epoch.FIRST; + case EMPTY: return Epoch.EMPTY; + case UPGRADE_STARTUP: return Epoch.UPGRADE_STARTUP; + case UPGRADE_GOSSIP: return Epoch.UPGRADE_GOSSIP; + default: throw new UnsupportedOperationException(); + } + } + + return Epoch.create(SourceDSL.longs().between(2, Long.MAX_VALUE).generate(rnd)); + }; + } } diff --git a/test/unit/org/apache/cassandra/utils/Generators.java b/test/unit/org/apache/cassandra/utils/Generators.java index 9675d5525ee0..5f99421240a3 100644 --- a/test/unit/org/apache/cassandra/utils/Generators.java +++ b/test/unit/org/apache/cassandra/utils/Generators.java @@ -26,8 +26,11 @@ import java.sql.Timestamp; import java.time.ZoneOffset; import java.time.ZonedDateTime; +import java.util.ArrayList; +import java.util.Comparator; import java.util.Date; import java.util.HashSet; +import java.util.List; import java.util.Random; import java.util.Set; import java.util.UUID; @@ -39,6 +42,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.cql3.ReservedKeywords; import org.quicktheories.core.Gen; import org.quicktheories.core.RandomnessSource; import org.quicktheories.generators.SourceDSL; @@ -58,9 +62,12 @@ public final class Generators private static final Constraint DNS_DOMAIN_PARTS_CONSTRAINT = Constraint.between(1, 127); + private static final char CHAR_UNDERSCORE = 95; + private static final char[] LETTER_DOMAIN = createLetterDomain(); private static final Constraint LETTER_CONSTRAINT = Constraint.between(0, LETTER_DOMAIN.length - 1).withNoShrinkPoint(); private static final char[] LETTER_OR_DIGIT_DOMAIN = createLetterOrDigitDomain(); + private static final char[] LETTER_OR_DIGIT_DOMAIN_WITH_UNDERSCORE = createLetterOrDigitDomainWithUnderscore(); private static final Constraint LETTER_OR_DIGIT_CONSTRAINT = Constraint.between(0, LETTER_OR_DIGIT_DOMAIN.length - 1).withNoShrinkPoint(); private static final char[] REGEX_WORD_DOMAIN = createRegexWordDomain(); private static final Constraint REGEX_WORD_CONSTRAINT = Constraint.between(0, REGEX_WORD_DOMAIN.length - 1).withNoShrinkPoint(); @@ -68,6 +75,12 @@ public final class Generators private static final Constraint DNS_DOMAIN_PART_CONSTRAINT = Constraint.between(0, DNS_DOMAIN_PART_DOMAIN.length - 1).withNoShrinkPoint(); public static final Gen<String> IDENTIFIER_GEN = Generators.regexWord(SourceDSL.integers().between(1, 50)); + public static final Gen<String> SYMBOL_GEN = Generators.filter(symbolGen(SourceDSL.integers().between(1, 48)), s -> !ReservedKeywords.isReserved(s)); + + public static Gen<String> symbolGen(Gen<Integer> size) + { + return string(size, LETTER_OR_DIGIT_DOMAIN_WITH_UNDERSCORE, (index, c) -> !(index == 0 && !Character.isLetter(c))); + } public static Gen<Character> letterOrDigit() { @@ -259,7 +272,23 @@ public static Gen<String> string(Gen<Integer> sizes, char[] domain) return charArray(sizes, domain).map(c -> new String(c)); } + public static Gen<String> string(Gen<Integer> sizes, char[] domain, IntCharBiPredicate fn) + { + // note, map is overloaded so String::new is ambugious to javac, so need a lambda here + return charArray(sizes, domain, fn).map(c -> new String(c)); + } + + public interface IntCharBiPredicate + { + boolean test(int a, char b); + } + public static Gen<char[]> charArray(Gen<Integer> sizes, char[] domain) + { + return charArray(sizes, domain, (a, b) -> true); + } + + public static Gen<char[]> charArray(Gen<Integer> sizes, char[] domain, IntCharBiPredicate fn) { Constraint constraints = Constraint.between(0, domain.length - 1).withNoShrinkPoint(); Gen<char[]> gen = td -> { @@ -267,8 +296,13 @@ public static Gen<char[]> charArray(Gen<Integer> sizes, char[] domain) char[] is = new char[size]; for (int i = 0; i != size; i++) { - int idx = (int) td.next(constraints); - is[i] = domain[idx]; + char c; + do + { + int idx = (int) td.next(constraints); + c = domain[idx]; + } while (!fn.test(i, c)); + is[i] = c; } return is; }; @@ -309,6 +343,14 @@ private static char[] createLetterOrDigitDomain() return domain; } + private static char[] createLetterOrDigitDomainWithUnderscore() + { + char[] domain = new char[LETTER_OR_DIGIT_DOMAIN.length + 1]; + System.arraycopy(LETTER_OR_DIGIT_DOMAIN, 0, domain, 0, LETTER_OR_DIGIT_DOMAIN.length); + domain[domain.length - 1] = CHAR_UNDERSCORE; + return domain; + } + private static char[] createRegexWordDomain() { // \w == [a-zA-Z_0-9] the only difference with letterOrDigit is the addition of _ @@ -326,6 +368,11 @@ public static Gen<ByteBuffer> bytes(int min, int max) return bytes(min, max, SourceDSL.arbitrary().constant(BBCases.HEAP)); } + public static Gen<ByteBuffer> directBytes(int min, int max) + { + return bytes(min, max, SourceDSL.arbitrary().pick(BBCases.DIRECT, BBCases.READ_ONLY_DIRECT)); + } + public static Gen<ByteBuffer> bytesAnyType(int min, int max) { return bytes(min, max, SourceDSL.arbitrary().enumValues(BBCases.class)); @@ -371,7 +418,7 @@ private static ByteBuffer directBufferFromSharedBlob(int offset, int size) { return bb; } - /** + /** * Implements a valid utf-8 generator. * * Implementation note, currently relies on getBytes to strip out non-valid utf-8 chars, so is slow @@ -379,9 +426,9 @@ private static ByteBuffer directBufferFromSharedBlob(int offset, int size) { public static Gen<String> utf8(int min, int max) { return SourceDSL.strings() - .basicMultilingualPlaneAlphabet() - .ofLengthBetween(min, max) - .map(s -> new String(s.getBytes(StandardCharsets.UTF_8), StandardCharsets.UTF_8)); + .basicMultilingualPlaneAlphabet() + .ofLengthBetween(min, max) + .map(s -> new String(s.getBytes(StandardCharsets.UTF_8), StandardCharsets.UTF_8)); } public static Gen<BigInteger> bigInt() @@ -432,6 +479,28 @@ public static <T> Gen<T> unique(Gen<T> gen) return filter(gen, dedup::add); } + public static <T> Gen<Set<T>> set(Gen<T> gen, Gen<Integer> sizeGen) + { + return rnd -> { + Set<T> set = new HashSet<>(); + int size = sizeGen.generate(rnd); + for (int i = 0; i < size; i++) + { + while (!set.add(gen.generate(rnd))) {} + } + return set; + }; + } + + public static <T extends Comparable<? super T>> Gen<List<T>> uniqueList(Gen<T> gen, Gen<Integer> sizeGen) + { + return set(gen, sizeGen).map(t -> { + List<T> list = new ArrayList<>(t); + list.sort(Comparator.naturalOrder()); + return list; + }); + } + public static <T> Gen<T> cached(Gen<T> gen) { Object cacheMissed = new Object(); @@ -466,7 +535,7 @@ private static final class LazySharedBlob static { - long blobSeed = TEST_BLOB_SHARED_SEED.getLong(System.currentTimeMillis()); + long blobSeed = TEST_BLOB_SHARED_SEED.getLong(); logger.info("Shared blob Gen used seed {}", blobSeed); Random random = new Random(blobSeed); @@ -542,4 +611,23 @@ public static <T> accord.utils.Gen<T> toGen(org.quicktheories.core.Gen<T> qt) return qt.generate(r); }; } + + public static Gen<TimeUUID> timeUUID() + { + ZonedDateTime now = ZonedDateTime.of(2020, 8, 20, + 0, 0, 0, 0, ZoneOffset.UTC); + ZonedDateTime startOfTime = now.minusYears(50); + ZonedDateTime endOfDays = now.plusYears(50); + Constraint micros = Constraint.between(toMicros(startOfTime), toMicros(endOfDays)); + return rs -> { + long nowMicro = rs.next(micros); + long lsb = rs.next(Constraint.none()); + return new TimeUUID(TimeUUID.unixMicrosToRawTimestamp(nowMicro), lsb); + }; + } + + private static long toMicros(ZonedDateTime zdt) + { + return zdt.toInstant().toEpochMilli() * 1000 + zdt.getNano() / 1000; + } } From d6f623b308b51551270ddb584fc6a8ba8766daf4 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Wed, 29 Jan 2025 10:44:59 +0100 Subject: [PATCH 111/225] Fix preparation of CLASSPATH in tools/bin/cassandra.in.sh for development builds patch by Stefan Miklosovic; reviewed by Michael Semb Wever for CASSANDRA-20257 --- tools/bin/cassandra.in.sh | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/tools/bin/cassandra.in.sh b/tools/bin/cassandra.in.sh index e1d1fe3f0542..c0c00c96e9f7 100644 --- a/tools/bin/cassandra.in.sh +++ b/tools/bin/cassandra.in.sh @@ -30,10 +30,18 @@ CLASSPATH="$CASSANDRA_CONF" # compiled classes. NOTE: This isn't needed by the startup script, # it's just used here in constructing the classpath. if [ -d $CASSANDRA_HOME/build ] ; then - #cassandra_bin="$CASSANDRA_HOME/build/classes/main" - cassandra_bin=`ls -1 $CASSANDRA_HOME/build/apache-cassandra*.jar` - cassandra_bin="$cassandra_bin:$CASSANDRA_HOME/build/classes/stress:$CASSANDRA_HOME/build/classes/fqltool" - CLASSPATH="$CLASSPATH:$cassandra_bin" + jars_cnt="`ls -1 $CASSANDRA_HOME/build/apache-cassandra*.jar | grep -v 'javadoc.jar' | grep -v 'sources.jar' | wc -l | xargs echo`" + if [ "$jars_cnt" -gt 1 ]; then + dir="`cd $CASSANDRA_HOME/build; pwd`" + echo "There are JAR artifacts for multiple versions in the $dir directory. Please clean the project with 'ant realclean' and build it again." 1>&2 + exit 1 + fi + + if [ "$jars_cnt" = "1" ]; then + cassandra_bin="`ls -1 $CASSANDRA_HOME/build/apache-cassandra*.jar | grep -v javadoc | grep -v sources`" + cassandra_bin="$cassandra_bin:$CASSANDRA_HOME/build/classes/stress:$CASSANDRA_HOME/build/classes/fqltool" + CLASSPATH="$CLASSPATH:$cassandra_bin" + fi fi # the default location for commitlogs, sstables, and saved caches From 5cbf993d96749fa8994e48f3ff2cbd9843a0a0ef Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi <bernardo.botella@apple.com> Date: Mon, 3 Jun 2024 15:41:20 -0700 Subject: [PATCH 112/225] CEP-42 - Add Constraints Framework patch by Bernardo Botella; reviewed by Yifan Cai, Stefan Miklosovic, Josh McKenzie, Maxwell Guo, Dmitry Konstantinov, Sam Tunnicliffe for CASSANDRA-19947 --- CHANGES.txt | 1 + NEWS.txt | 3 + doc/modules/cassandra/nav.adoc | 1 + .../pages/developing/cql/constraints.adoc | 93 ++ .../cassandra/pages/developing/cql/index.adoc | 1 + doc/modules/cassandra/pages/new/index.adoc | 1 + pylib/cqlshlib/cql3handling.py | 11 +- pylib/cqlshlib/test/test_cqlsh_completion.py | 11 +- src/antlr/Cql.g | 1 + src/antlr/Lexer.g | 1 + src/antlr/Parser.g | 20 +- .../org/apache/cassandra/cql3/Validation.java | 13 + .../cql3/constraints/ColumnConstraint.java | 85 ++ .../cql3/constraints/ColumnConstraints.java | 207 +++ .../cql3/constraints/ConstraintFunction.java | 48 + .../ConstraintViolationException.java | 32 + .../constraints/FunctionColumnConstraint.java | 199 +++ .../InvalidConstraintDefinitionException.java | 32 + .../cql3/constraints/LengthConstraint.java | 115 ++ .../constraints/ScalarColumnConstraint.java | 160 +++ .../statements/ModificationStatement.java | 22 + .../cql3/statements/UpdateStatement.java | 30 +- .../schema/AlterTableStatement.java | 89 +- .../schema/CreateTableStatement.java | 22 +- .../apache/cassandra/db/ClusteringPrefix.java | 6 + src/java/org/apache/cassandra/db/Columns.java | 7 +- .../db/marshal/AbstractCompositeType.java | 62 +- .../cassandra/db/marshal/AbstractType.java | 32 + .../apache/cassandra/db/marshal/MapType.java | 6 + .../cassandra/db/marshal/ReversedType.java | 6 + .../cassandra/db/marshal/TupleType.java | 6 + .../apache/cassandra/db/marshal/UserType.java | 6 + .../cassandra/schema/ColumnMetadata.java | 126 +- .../cassandra/schema/TableMetadata.java | 78 +- .../cassandra/tcm/membership/NodeVersion.java | 2 +- .../cassandra/tcm/serialization/Version.java | 4 + .../test/ColumnConstraintsTest.java | 302 ++++ .../test/cdc/ToggleCDCOnRepair.java | 98 ++ .../cdc/ToggleCDCOnRepairEnabledTest.java | 65 +- ...CDCWithConstraintsOnRepairEnabledTest.java | 38 + .../distributed/test/log/SnapshotTest.java | 1 + ...ableWithTableConstraintValidationTest.java | 208 +++ .../contraints/ColumnConstraintsTest.java | 50 + .../CqlConstraintValidationTester.java | 81 ++ ...WithColumnCqlConstraintValidationTest.java | 1264 +++++++++++++++++ .../db/marshal/AbstractTypeTest.java | 14 +- 46 files changed, 3550 insertions(+), 110 deletions(-) create mode 100644 doc/modules/cassandra/pages/developing/cql/constraints.adoc create mode 100644 src/java/org/apache/cassandra/cql3/constraints/ColumnConstraint.java create mode 100644 src/java/org/apache/cassandra/cql3/constraints/ColumnConstraints.java create mode 100644 src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java create mode 100644 src/java/org/apache/cassandra/cql3/constraints/ConstraintViolationException.java create mode 100644 src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java create mode 100644 src/java/org/apache/cassandra/cql3/constraints/InvalidConstraintDefinitionException.java create mode 100644 src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java create mode 100644 src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/ColumnConstraintsTest.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/cdc/ToggleCDCOnRepair.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/cdc/ToggleCDCWithConstraintsOnRepairEnabledTest.java create mode 100644 test/unit/org/apache/cassandra/contraints/AlterTableWithTableConstraintValidationTest.java create mode 100644 test/unit/org/apache/cassandra/contraints/ColumnConstraintsTest.java create mode 100644 test/unit/org/apache/cassandra/contraints/CqlConstraintValidationTester.java create mode 100644 test/unit/org/apache/cassandra/contraints/CreateTableWithColumnCqlConstraintValidationTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 59d478d713f3..6f09b0e0a3df 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * CEP-42 - Add Constraints Framework (CASSANDRA-19947) * Add table metric PurgeableTombstoneScannedHistogram and a tracing event for scanned purgeable tombstones (CASSANDRA-20132) * Make sure we can parse the expanded CQL before writing it to the log or sending it to replicas (CASSANDRA-20218) * Add format_bytes and format_time functions (CASSANDRA-19546) diff --git a/NEWS.txt b/NEWS.txt index dc4c766e9f21..bb8898d53b25 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -76,6 +76,9 @@ New features metadata. In the first instance, this encompasses cluster membership, token ownership and schema metadata. See https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-21%3A+Transactional+Cluster+Metadata for more detail on the motivation and design, and see "Upgrading" below for specific instructions on migrating clusters to this system. + - CEP-42 Constraints Framework provides flexibility to Cassandra users and operators by providing a set of + usable constraints at table level, that will ease validations at application level and protect + the Database from misconfigured clients. More updates and documentation to follow. - New Guardrails added: - Whether bulk loading of SSTables is allowed. diff --git a/doc/modules/cassandra/nav.adoc b/doc/modules/cassandra/nav.adoc index 6a4c7a7a2fe5..c549ba4ae5d1 100644 --- a/doc/modules/cassandra/nav.adoc +++ b/doc/modules/cassandra/nav.adoc @@ -60,6 +60,7 @@ *** xref:cassandra:developing/cql/json.adoc[JSON] *** xref:cassandra:developing/cql/security.adoc[Security] *** xref:cassandra:developing/cql/triggers.adoc[Triggers] +*** xref:cassandra:developing/cql/constraints.adoc[Constraints] *** xref:cassandra:developing/cql/appendices.adoc[Appendices] *** xref:cassandra:developing/cql/changes.adoc[Changes] *** xref:cassandra:developing/cql/SASI.adoc[SASI] diff --git a/doc/modules/cassandra/pages/developing/cql/constraints.adoc b/doc/modules/cassandra/pages/developing/cql/constraints.adoc new file mode 100644 index 000000000000..e6a76dbe7d3b --- /dev/null +++ b/doc/modules/cassandra/pages/developing/cql/constraints.adoc @@ -0,0 +1,93 @@ += Constraints + +Constraints provide a way of specifying and enforcing conditions at a +column level in a table schema definition and enforcing them at write time. + +== CREATE CONSTRAINT + +Constraints can be created within the column definition, or as part +of the table properties. + +The main syntax to define a constraint is as follows: + +[source,bnf] +---- +CREATE TABLE keyspace.table ( + name text, + i int CHECK (condition) (AND (condition))* + ..., + +); +---- + +As shown in this syntax, more than one constraint can be defined for a given column using the AND keyword. + +== ALTER CONSTRAINT + +Altering a constraint is done by following the alter column CQL syntax: +[source,bnf] +---- +ALTER TABLE [IF EXISTS] <table> ALTER [IF EXISTS] <column> CHECK <condition>; +---- + +== DROP CONSTRAINT +And DROP can be used to drop constraints for a column as well. +[source,bnf] +---- +ALTER TABLE [IF EXISTS] <table> ALTER [IF EXISTS] <column> DROP CHECK; +---- + +== AVAILABLE CONSTRAINTS + +=== SCALAR CONSTRAINT + +Defines a comparator against a numeric type. It support all numeric types supported in Cassandra, with all the regular +comparators. + +For example, we can define constraints that ensure that i is bigger or equal than 100 but smaller than 1000. + +[source,bnf] +---- +CREATE TABLE keyspace.table ( + name text, + i int CHECK i < 1000 AND i > 100 + ..., +); +---- + +Altering that constraint can be done with: + +---- +ALTER TABLE keyspace.table ALTER i CHECK i >= 500; +---- + +Finally, the constraint can be removed: + +---- +ALTER TABLE keyspace.table ALTER i DROP CHECK; +---- + +=== LENGTH CONSTRAINT + +Defines a condition that checks the length of text or binary type. + +For example, we can create a constraint that checks that name can't be longer than 256 characters: + +---- +CREATE TABLE keyspace.table ( + name text CHECK LENGTH(name) < 256 + ..., +); +---- + +Altering that constraint can be done with: + +---- +ALTER TABLE keyspace.table ALTER name LENGTH(name) < 512; +---- + +Finally, the constraint can be removed: + +---- +ALTER TABLE keyspace.table ALTER name DROP CHECK; +---- diff --git a/doc/modules/cassandra/pages/developing/cql/index.adoc b/doc/modules/cassandra/pages/developing/cql/index.adoc index 3b31a4d3817a..97204ce5687e 100644 --- a/doc/modules/cassandra/pages/developing/cql/index.adoc +++ b/doc/modules/cassandra/pages/developing/cql/index.adoc @@ -21,5 +21,6 @@ For that reason, when used in this document, these terms (tables, rows and colum * xref:developing/cql/json.adoc[JSON] * xref:developing/cql/security.adoc[CQL security] * xref:developing/cql/triggers.adoc[Triggers] +* xref:developing/cql/constraints.adoc[Constraints] * xref:developing/cql/appendices.adoc[Appendices] * xref:developing/cql/changes.adoc[Changes] \ No newline at end of file diff --git a/doc/modules/cassandra/pages/new/index.adoc b/doc/modules/cassandra/pages/new/index.adoc index 4371cebd6bab..015f484f5728 100644 --- a/doc/modules/cassandra/pages/new/index.adoc +++ b/doc/modules/cassandra/pages/new/index.adoc @@ -7,6 +7,7 @@ This section covers the new features in Apache Cassandra 5.1. * https://cwiki.apache.org/confluence/x/FQRACw[ACID Transactions (Accord)] * https://cwiki.apache.org/confluence/x/YyD1D[Transactional Cluster Metadata] +* https://cwiki.apache.org/confluence/x/8IpyEg[Constraints] == New Features in Apache Cassandra 5.0 diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py index 4d2f4dffcf14..9883543ab5ff 100644 --- a/pylib/cqlshlib/cql3handling.py +++ b/pylib/cqlshlib/cql3handling.py @@ -322,7 +322,14 @@ def dequote_value(cqlword): <userType> ::= utname=<cfOrKsName> ; -<storageType> ::= ( <simpleStorageType> | <collectionType> | <frozenCollectionType> | <vectorType> | <userType> ) ( <column_mask> )? ; +<storageType> ::= ( <simpleStorageType> | <collectionType> | <frozenCollectionType> | <vectorType> | <userType> ) ( <constraintsExpr> )? ( <column_mask> )? ; + +<constraintsExpr> ::= "CHECK" <constraint> ( "AND" <constraint> )* + ; + +<constraint> ::= <cident> <cmp> <term> + | <functionArguments> <cmp> <term> + ; <column_mask> ::= "MASKED" "WITH" ( "DEFAULT" | <functionName> <selectionFunctionArguments> ); @@ -1463,7 +1470,7 @@ def idx_ks_idx_name_completer(ctxt, cass): | "WITH" <cfamProperty> ( "AND" <cfamProperty> )* | "RENAME" ("IF" "EXISTS")? existcol=<cident> "TO" newcol=<cident> ( "AND" existcol=<cident> "TO" newcol=<cident> )* - | "ALTER" ("IF" "EXISTS")? existcol=<cident> ( <column_mask> | "DROP" "MASKED" ) + | "ALTER" ("IF" "EXISTS")? existcol=<cident> ( <constraintsExpr> | <column_mask> | "DROP" ( "CHECK" | "MASKED" ) ) ; <alterUserTypeStatement> ::= "ALTER" "TYPE" ("IF" "EXISTS")? ut=<userTypeName> diff --git a/pylib/cqlshlib/test/test_cqlsh_completion.py b/pylib/cqlshlib/test/test_cqlsh_completion.py index 2a02fcf464ba..b8c0fb149918 100644 --- a/pylib/cqlshlib/test/test_cqlsh_completion.py +++ b/pylib/cqlshlib/test/test_cqlsh_completion.py @@ -636,7 +636,7 @@ def create_columnfamily_table_template(self, name): self.trycompletions(prefix + ' new_table (col_a ine', immediate='t ') self.trycompletions(prefix + ' new_table (col_a int ', - choices=[',', 'MASKED', 'PRIMARY']) + choices=[',', 'CHECK', 'MASKED', 'PRIMARY']) self.trycompletions(prefix + ' new_table (col_a int M', immediate='ASKED WITH ') self.trycompletions(prefix + ' new_table (col_a int MASKED WITH ', @@ -1106,7 +1106,7 @@ def test_complete_in_alter_table(self): self.trycompletions('ALTER TABLE IF EXISTS new_table ADD ', choices=['<new_column_name>', 'IF']) self.trycompletions('ALTER TABLE IF EXISTS new_table ADD IF NOT EXISTS ', choices=['<new_column_name>']) self.trycompletions('ALTER TABLE new_table ADD IF NOT EXISTS ', choices=['<new_column_name>']) - self.trycompletions('ALTER TABLE new_table ADD col int ', choices=[';', 'MASKED', 'static']) + self.trycompletions('ALTER TABLE new_table ADD col int ', choices=[';', 'MASKED', 'CHECK', 'static']) self.trycompletions('ALTER TABLE new_table ADD col int M', immediate='ASKED WITH ') self.trycompletions('ALTER TABLE new_table ADD col int MASKED WITH ', choices=['DEFAULT', self.cqlsh.keyspace + '.', 'system.'], @@ -1116,12 +1116,15 @@ def test_complete_in_alter_table(self): self.trycompletions('ALTER TABLE IF EXISTS new_table DROP ', choices=['IF', '<quotedName>', '<identifier>']) self.trycompletions('ALTER TABLE IF EXISTS new_table ALTER ', choices=['IF', '<quotedName>', '<identifier>']) self.trycompletions('ALTER TABLE IF EXISTS new_table ALTER IF E', immediate='XISTS ') - self.trycompletions('ALTER TABLE IF EXISTS new_table ALTER IF EXISTS col ', choices=['MASKED', 'DROP']) + self.trycompletions('ALTER TABLE IF EXISTS new_table ALTER IF EXISTS col ', choices=['CHECK', 'MASKED', 'DROP']) + self.trycompletions('ALTER TABLE IF EXISTS new_table ALTER IF EXISTS col C', immediate='HECK ') self.trycompletions('ALTER TABLE IF EXISTS new_table ALTER IF EXISTS col M', immediate='ASKED WITH ') self.trycompletions('ALTER TABLE IF EXISTS new_table ALTER IF EXISTS col MASKED WITH ', choices=['DEFAULT', self.cqlsh.keyspace + '.', 'system.'], other_choices_ok=True) - self.trycompletions('ALTER TABLE IF EXISTS new_table ALTER IF EXISTS col D', immediate='ROP MASKED ;') + self.trycompletions('ALTER TABLE IF EXISTS new_table ALTER IF EXISTS col D', immediate='ROP ') + self.trycompletions('ALTER TABLE IF EXISTS new_table ALTER IF EXISTS col DROP ', choices=['CHECK', 'MASKED']) + self.trycompletions('ALTER TABLE IF EXISTS new_table ALTER IF EXISTS col DROP C', immediate='HECK ;') self.trycompletions('ALTER TABLE IF EXISTS new_table ALTER IF EXISTS col DROP M', immediate='ASKED ;') def test_complete_in_alter_type(self): diff --git a/src/antlr/Cql.g b/src/antlr/Cql.g index 5cf8c88a3143..fe068f4b5adb 100644 --- a/src/antlr/Cql.g +++ b/src/antlr/Cql.g @@ -38,6 +38,7 @@ import Parser,Lexer; import org.apache.cassandra.auth.*; import org.apache.cassandra.cql3.conditions.*; + import org.apache.cassandra.cql3.constraints.*; import org.apache.cassandra.cql3.functions.*; import org.apache.cassandra.cql3.functions.masking.*; import org.apache.cassandra.cql3.restrictions.CustomIndexExpression; diff --git a/src/antlr/Lexer.g b/src/antlr/Lexer.g index 34f5a80ccdf2..96a51e23c80a 100644 --- a/src/antlr/Lexer.g +++ b/src/antlr/Lexer.g @@ -129,6 +129,7 @@ K_GROUP: G R O U P; K_CLUSTER: C L U S T E R; K_INTERNALS: I N T E R N A L S; K_ONLY: O N L Y; +K_CHECK: C H E C K; K_GRANT: G R A N T; K_ALL: A L L; diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g index 940039573399..5f5908f6b0c6 100644 --- a/src/antlr/Parser.g +++ b/src/antlr/Parser.g @@ -781,11 +781,24 @@ tableDefinition[CreateTableStatement.Raw stmt] tableColumns[CreateTableStatement.Raw stmt] @init { boolean isStatic = false; } - : k=ident v=comparatorType (K_STATIC { isStatic = true; })? (mask=columnMask)? { $stmt.addColumn(k, v, isStatic, mask); } + : k=ident v=comparatorType (K_STATIC { isStatic = true; })? (mask=columnMask)? (constraints=columnConstraints)? { $stmt.addColumn(k, v, isStatic, mask, constraints); } (K_PRIMARY K_KEY { $stmt.setPartitionKeyColumn(k); })? | K_PRIMARY K_KEY '(' tablePartitionKey[stmt] (',' c=ident { $stmt.markClusteringColumn(c); } )* ')' ; +columnConstraints returns [ColumnConstraints.Raw constraints] + @init { + boolean isStatic = false; + List constraintsList = new ArrayList(); + } + : K_CHECK cc=columnConstraint { constraintsList.add(cc); } (K_AND cc=columnConstraint { constraintsList.add(cc); })* { $constraints = new ColumnConstraints.Raw(constraintsList); } + ; + +columnConstraint returns [ColumnConstraint columnConstraint] + : funcName=ident '(' k=ident ')' op=relationType t=value { $columnConstraint = new FunctionColumnConstraint.Raw(funcName, k, op, t.getText()).prepare(); } + | k=ident op=relationType t=value { $columnConstraint = new ScalarColumnConstraint.Raw(k, op, t.getText()).prepare(); } + ; + columnMask returns [ColumnMask.Raw mask] @init { List<Term.Raw> arguments = new ArrayList<>(); } : K_MASKED K_WITH name=functionName columnMaskArguments[arguments] { $mask = new ColumnMask.Raw(name, arguments); } @@ -965,7 +978,9 @@ alterTableStatement returns [AlterTableStatement.Raw stmt] | K_ALTER ( K_IF K_EXISTS { $stmt.ifColumnExists(true); } )? id=cident ( mask=columnMask { $stmt.mask(id, mask); } - | K_DROP K_MASKED { $stmt.mask(id, null); } ) + | K_DROP K_MASKED { $stmt.mask(id, null); } + | K_DROP K_CHECK { $stmt.dropConstraints(id); } + | (constraints=columnConstraints) { $stmt.alterConstraints(id, constraints); }) | K_ADD ( K_IF K_NOT K_EXISTS { $stmt.ifColumnNotExists(true); } )? ( id=ident v=comparatorType b=isStaticColumn (m=columnMask)? { $stmt.add(id, v, b, m); } @@ -2067,5 +2082,6 @@ basic_unreserved_keyword returns [String str] | K_VECTOR | K_ANN | K_BETWEEN + | K_CHECK ) { $str = $k.text; } ; diff --git a/src/java/org/apache/cassandra/cql3/Validation.java b/src/java/org/apache/cassandra/cql3/Validation.java index 27a1b4e667cd..cbffa5a52e5a 100644 --- a/src/java/org/apache/cassandra/cql3/Validation.java +++ b/src/java/org/apache/cassandra/cql3/Validation.java @@ -19,6 +19,7 @@ import java.nio.ByteBuffer; +import org.apache.cassandra.cql3.constraints.ConstraintViolationException; import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.serializers.MarshalException; @@ -64,4 +65,16 @@ public static void validateKey(TableMetadata metadata, ByteBuffer key) throw new InvalidRequestException(e.getMessage()); } } + + public static void checkConstraints(TableMetadata metadata, ByteBuffer key) + { + try + { + metadata.partitionKeyType.checkConstraints(key, metadata.partitionKeyConstraints); + } + catch (ConstraintViolationException e) + { + throw new InvalidRequestException(e.getMessage(), e); + } + } } diff --git a/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraint.java new file mode 100644 index 000000000000..ccbcdef48af1 --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraint.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.constraints; + +import java.nio.ByteBuffer; + +import org.apache.cassandra.cql3.CqlBuilder; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.tcm.serialization.MetadataSerializer; +import org.apache.cassandra.schema.ColumnMetadata; + +/** + * Common class for the conditions that a CQL Constraint needs to implement to be integrated in the + * CQL Constraints framework, with T as a constraint serializer. + */ +public interface ColumnConstraint<T> +{ + + // Enum containing all the possible constraint serializers to help with serialization/deserialization + // of constraints. + enum ConstraintType + { + // The order of that enum matters!! + // We are serializing its enum position instead of its name. + // Changing this enum would affect how that int is interpreted when deserializing. + COMPOSED(ColumnConstraints.serializer), + FUNCTION(FunctionColumnConstraint.serializer), + SCALAR(ScalarColumnConstraint.serializer); + + private final MetadataSerializer<?> serializer; + + ConstraintType(MetadataSerializer<?> serializer) + { + this.serializer = serializer; + } + + public static MetadataSerializer<?> getSerializer(int i) + { + return ConstraintType.values()[i].serializer; + } + } + + MetadataSerializer<T> serializer(); + + void appendCqlTo(CqlBuilder builder); + + /** + * Method that evaluates the condition. It can either succeed or throw a {@link ConstraintViolationException}. + * + * @param valueType value type of the column value under test + * @param columnValue Column value to be evaluated at write time + */ + void evaluate(AbstractType<?> valueType, ByteBuffer columnValue) throws ConstraintViolationException; + + /** + * Method to validate the condition. This method is called when creating constraint via CQL. + * A {@link InvalidConstraintDefinitionException} is thrown for invalid consrtaint definition. + * + * @param columnMetadata Metadata of the column in which the constraint is defined. + */ + void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException; + + /** + * Method to get the Constraint serializer + * + * @return the Constraint type serializer + */ + ConstraintType getConstraintType(); +} diff --git a/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraints.java b/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraints.java new file mode 100644 index 000000000000..3b6646aec3ca --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraints.java @@ -0,0 +1,207 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.constraints; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +import org.apache.cassandra.cql3.CqlBuilder; +import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.tcm.serialization.MetadataSerializer; +import org.apache.cassandra.tcm.serialization.Version; + + +// group of constraints for the column +public class ColumnConstraints implements ColumnConstraint<ColumnConstraints> +{ + public static final Serializer serializer = new Serializer(); + public static final ColumnConstraints NO_OP = new Noop(); + + private final List<ColumnConstraint<?>> constraints; + + public ColumnConstraints(List<ColumnConstraint<?>> constraints) + { + this.constraints = constraints; + } + + @Override + public MetadataSerializer<ColumnConstraints> serializer() + { + return serializer; + } + + @Override + public void appendCqlTo(CqlBuilder builder) + { + for (ColumnConstraint<?> constraint : constraints) + constraint.appendCqlTo(builder); + } + + @Override + public void evaluate(AbstractType<?> valueType, ByteBuffer columnValue) throws ConstraintViolationException + { + for (ColumnConstraint<?> constraint : constraints) + constraint.evaluate(valueType, columnValue); + } + + public List<ColumnConstraint<?>> getConstraints() + { + return constraints; + } + + public boolean isEmpty() + { + return constraints.isEmpty(); + } + + public int getSize() + { + return constraints.size(); + } + + // Checks if there is at least one constraint that will perform checks + public boolean hasRelevantConstraints() + { + for (ColumnConstraint c : constraints) + { + if (c != ColumnConstraints.NO_OP) + return true; + } + return false; + } + + @Override + public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException + { + if (!columnMetadata.type.isConstrainable()) + throw new InvalidConstraintDefinitionException("Constraint cannot be defined on the column " + + columnMetadata.name + " of type " + columnMetadata.type.asCQL3Type() + + " for the table " + columnMetadata.ksName + "." + columnMetadata.cfName); + + for (ColumnConstraint<?> constraint : constraints) + constraint.validate(columnMetadata); + } + + @Override + public ConstraintType getConstraintType() + { + return ConstraintType.COMPOSED; + } + + private static class Noop extends ColumnConstraints + { + private Noop() + { + super(Collections.emptyList()); + } + + @Override + public void validate(ColumnMetadata columnMetadata) + { + // Do nothing. It is always valid + } + } + + public final static class Raw + { + private final List<ColumnConstraint<?>> constraints; + + public Raw(List<ColumnConstraint<?>> constraints) + { + this.constraints = constraints; + } + + public Raw() + { + this.constraints = Collections.emptyList(); + } + + public ColumnConstraints prepare() + { + if (constraints.isEmpty()) + return NO_OP; + return new ColumnConstraints(constraints); + } + } + + public static class Serializer implements MetadataSerializer<ColumnConstraints> + { + + @Override + public void serialize(ColumnConstraints columnConstraint, DataOutputPlus out, Version version) throws IOException + { + out.writeInt(columnConstraint.getSize()); + for (ColumnConstraint constraint : columnConstraint.getConstraints()) + { + // We serialize the serializer ordinal in the enum to save space + out.writeShort(constraint.getConstraintType().ordinal()); + constraint.serializer().serialize(constraint, out, version); + } + } + + @Override + public ColumnConstraints deserialize(DataInputPlus in, Version version) throws IOException + { + List<ColumnConstraint<?>> columnConstraints = new ArrayList<>(); + int numberOfConstraints = in.readInt(); + for (int i = 0; i < numberOfConstraints; i++) + { + int serializerPosition = in.readShort(); + ColumnConstraint<?> constraint = (ColumnConstraint<?>) ConstraintType + .getSerializer(serializerPosition) + .deserialize(in, version); + columnConstraints.add(constraint); + } + return new ColumnConstraints(columnConstraints); + } + + @Override + public long serializedSize(ColumnConstraints columnConstraint, Version version) + { + long constraintsSize = TypeSizes.INT_SIZE; + for (ColumnConstraint constraint : columnConstraint.getConstraints()) + { + constraintsSize += TypeSizes.SHORT_SIZE; + constraintsSize += constraint.serializer().serializedSize(constraint, version); + } + return constraintsSize; + } + } + + @Override + public boolean equals(Object o) + { + if (this == o) + return true; + + if (!(o instanceof ColumnConstraints)) + return false; + + ColumnConstraints other = (ColumnConstraints) o; + return Objects.equals(constraints, other.constraints); + } +} diff --git a/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java b/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java new file mode 100644 index 000000000000..aa3ea2303b3a --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.constraints; + +import java.nio.ByteBuffer; + +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.schema.ColumnMetadata; + +/** + * Interface to be implemented by functions that are executed as part of CQL constraints. + */ +public interface ConstraintFunction +{ + /** + * @return the function name to be executed. + */ + String getName(); + + /** + * Method that performs the actual condition test, executed during the write path. + * It the test is not successful, it throws a {@link ConstraintViolationException}. + */ + void evaluate(AbstractType<?> valueType, Operator relationType, String term, ByteBuffer columnValue) throws ConstraintViolationException; + + /** + * Method that validates that a condition is valid. This method is called when the CQL constraint is created to determine + * if the CQL statement is valid or needs to be rejected as invalid throwing a {@link InvalidConstraintDefinitionException} + */ + void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException; +} diff --git a/src/java/org/apache/cassandra/cql3/constraints/ConstraintViolationException.java b/src/java/org/apache/cassandra/cql3/constraints/ConstraintViolationException.java new file mode 100644 index 000000000000..ea7742ac18f9 --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/constraints/ConstraintViolationException.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.constraints; + +import org.apache.cassandra.exceptions.InvalidRequestException; + +/** + * Thrown to indicate that the CQL operation did not comply with the defined Constraints + */ +public class ConstraintViolationException extends InvalidRequestException +{ + public ConstraintViolationException(String msg) + { + super(msg); + } +} diff --git a/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java new file mode 100644 index 000000000000..ad42e1e12913 --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java @@ -0,0 +1,199 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.constraints; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.function.Function; + +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.CqlBuilder; +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.tcm.serialization.MetadataSerializer; +import org.apache.cassandra.tcm.serialization.Version; +import org.apache.cassandra.utils.LocalizeString; + +public class FunctionColumnConstraint implements ColumnConstraint<FunctionColumnConstraint> +{ + public static final Serializer serializer = new Serializer(); + + public final ConstraintFunction function; + public final ColumnIdentifier columnName; + public final Operator relationType; + public final String term; + + public final static class Raw + { + public final ConstraintFunction function; + public final ColumnIdentifier columnName; + public final Operator relationType; + public final String term; + + public Raw(ColumnIdentifier functionName, ColumnIdentifier columnName, Operator relationType, String term) + { + this.relationType = relationType; + this.columnName = columnName; + this.term = term; + function = createConstraintFunction(functionName.toCQLString(), columnName); + } + + public FunctionColumnConstraint prepare() + { + return new FunctionColumnConstraint(function, columnName, relationType, term); + } + } + + private enum Functions + { + LENGTH(LengthConstraint::new); + + private final Function<ColumnIdentifier, ConstraintFunction> functionCreator; + + Functions(Function<ColumnIdentifier, ConstraintFunction> functionCreator) + { + this.functionCreator = functionCreator; + } + } + + private static ConstraintFunction createConstraintFunction(String functionName, ColumnIdentifier columnName) + { + try + { + return Functions.valueOf(LocalizeString.toUpperCaseLocalized(functionName)).functionCreator.apply(columnName); + } + catch (IllegalArgumentException ex) + { + throw new InvalidConstraintDefinitionException("Unrecognized constraint function: " + functionName); + } + } + + private FunctionColumnConstraint(ConstraintFunction function, ColumnIdentifier columnName, Operator relationType, String term) + { + this.function = function; + this.columnName = columnName; + this.relationType = relationType; + this.term = term; + } + + @Override + public void appendCqlTo(CqlBuilder builder) + { + builder.append(toString()); + } + + @Override + public MetadataSerializer<FunctionColumnConstraint> serializer() + { + return serializer; + } + + @Override + public void evaluate(AbstractType<?> valueType, ByteBuffer columnValue) + { + function.evaluate(valueType, relationType, term, columnValue); + } + + @Override + public void validate(ColumnMetadata columnMetadata) + { + validateArgs(columnMetadata); + function.validate(columnMetadata); + } + + @Override + public ConstraintType getConstraintType() + { + return ConstraintType.FUNCTION; + } + + void validateArgs(ColumnMetadata columnMetadata) + { + if (!columnMetadata.name.equals(columnName)) + throw new InvalidConstraintDefinitionException("Function parameter should be the column name"); + } + + @Override + public String toString() + { + return function.getName() + "(" + columnName + ") " + relationType + " " + term; + } + + public static class Serializer implements MetadataSerializer<FunctionColumnConstraint> + { + @Override + public void serialize(FunctionColumnConstraint columnConstraint, DataOutputPlus out, Version version) throws IOException + { + out.writeUTF(columnConstraint.function.getName()); + out.writeUTF(columnConstraint.columnName.toCQLString()); + columnConstraint.relationType.writeTo(out); + out.writeUTF(columnConstraint.term); + } + + @Override + public FunctionColumnConstraint deserialize(DataInputPlus in, Version version) throws IOException + { + String functionName = in.readUTF(); + ConstraintFunction function; + String columnNameString = in.readUTF(); + ColumnIdentifier columnName = new ColumnIdentifier(columnNameString, true); + try + { + function = createConstraintFunction(functionName, columnName); + } + catch (Exception e) + { + throw new IOException(e); + } + Operator relationType = Operator.readFrom(in); + final String term = in.readUTF(); + return new FunctionColumnConstraint(function, columnName, relationType, term); + } + + @Override + public long serializedSize(FunctionColumnConstraint columnConstraint, Version version) + { + return TypeSizes.sizeof(columnConstraint.function.getClass().getName()) + + TypeSizes.sizeof(columnConstraint.columnName.toCQLString()) + + TypeSizes.sizeof(columnConstraint.term) + + Operator.serializedSize(); + } + } + + @Override + public boolean equals(Object o) + { + if (this == o) + return true; + + if (!(o instanceof FunctionColumnConstraint)) + return false; + + FunctionColumnConstraint other = (FunctionColumnConstraint) o; + + return function.equals(other.function) + && columnName.equals(other.columnName) + && relationType == other.relationType + && term.equals(other.term); + } +} diff --git a/src/java/org/apache/cassandra/cql3/constraints/InvalidConstraintDefinitionException.java b/src/java/org/apache/cassandra/cql3/constraints/InvalidConstraintDefinitionException.java new file mode 100644 index 000000000000..7ef32f0ca1f5 --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/constraints/InvalidConstraintDefinitionException.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.constraints; + +import org.apache.cassandra.exceptions.InvalidRequestException; + +/** + * Thrown to indicate that the CQL constraint is not valid + */ +public class InvalidConstraintDefinitionException extends InvalidRequestException +{ + public InvalidConstraintDefinitionException(String msg) + { + super(msg); + } +} diff --git a/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java new file mode 100644 index 000000000000..9d6f99ca24de --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.constraints; + +import java.nio.ByteBuffer; + +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.AsciiType; +import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.utils.ByteBufferUtil; + +public class LengthConstraint implements ConstraintFunction +{ + private static final AbstractType<?>[] SUPPORTED_TYPES = new AbstractType[] { BytesType.instance, UTF8Type.instance, AsciiType.instance }; + + public static final String FUNCTION_NAME = "LENGTH"; + + private final ColumnIdentifier columnName; + + public LengthConstraint(ColumnIdentifier columnName) + { + this.columnName = columnName; + } + + @Override + public String getName() + { + return FUNCTION_NAME; + } + + @Override + public void evaluate(AbstractType<?> valueType, Operator relationType, String term, ByteBuffer columnValue) + { + int valueLength = getValueLength(columnValue, valueType); + int sizeConstraint = Integer.parseInt(term); + + ByteBuffer leftOperand = ByteBufferUtil.bytes(valueLength); + ByteBuffer rightOperand = ByteBufferUtil.bytes(sizeConstraint); + + if (!relationType.isSatisfiedBy(Int32Type.instance, leftOperand, rightOperand)) + throw new ConstraintViolationException(columnName + " does not satisfy length constraint. " + + valueLength + " should be " + relationType + ' ' + term); + } + + @Override + public void validate(ColumnMetadata columnMetadata) + { + boolean supported = false; + AbstractType<?> unwrapped = columnMetadata.type.unwrap(); + for (AbstractType<?> supportedType : SUPPORTED_TYPES) + { + if (supportedType == unwrapped) + { + supported = true; + break; + } + } + + if (!supported) + throw invalidConstraintDefinitionException(columnMetadata.type); + } + + private int getValueLength(ByteBuffer value, AbstractType<?> valueType) + { + if (valueType.getClass() == BytesType.class) + { + return value.remaining(); + } + + if (valueType.getClass() == AsciiType.class || valueType.getClass() == UTF8Type.class) + return ((String) valueType.compose(value)).length(); + + throw invalidConstraintDefinitionException(valueType); + } + + private InvalidConstraintDefinitionException invalidConstraintDefinitionException(AbstractType<?> valueType) + { + throw new InvalidConstraintDefinitionException("Column type " + valueType.getClass() + " is not supported."); + } + + @Override + public boolean equals(Object o) + { + if (this == o) + return true; + + if (!(o instanceof LengthConstraint)) + return false; + + LengthConstraint other = (LengthConstraint) o; + + return columnName.equals(other.columnName); + } +} diff --git a/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java new file mode 100644 index 000000000000..14be7d689d20 --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.constraints; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.CqlBuilder; +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.tcm.serialization.MetadataSerializer; +import org.apache.cassandra.tcm.serialization.Version; + +public class ScalarColumnConstraint implements ColumnConstraint<ScalarColumnConstraint> +{ + public final ColumnIdentifier param; + public final Operator relationType; + public final String term; + + public final static Serializer serializer = new Serializer(); + + public final static class Raw + { + public final ColumnIdentifier param; + public final Operator relationType; + public final String term; + + public Raw(ColumnIdentifier param, Operator relationType, String term) + { + this.param = param; + this.relationType = relationType; + this.term = term; + } + + public ScalarColumnConstraint prepare() + { + return new ScalarColumnConstraint(param, relationType, term); + } + } + + private ScalarColumnConstraint(ColumnIdentifier param, Operator relationType, String term) + { + this.param = param; + this.relationType = relationType; + this.term = term; + } + + @Override + public void evaluate(AbstractType<?> valueType, ByteBuffer columnValue) + { + ByteBuffer value; + try + { + value = valueType.fromString(term); + } + catch (NumberFormatException exception) + { + throw new ConstraintViolationException(param + " and " + term + " need to be numbers."); + } + + if (!relationType.isSatisfiedBy(valueType, columnValue, value)) + throw new ConstraintViolationException("Column value does not satisfy value constraint. " + + " It should be " + relationType + " " + term); + } + + @Override + public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException + { + if (!columnMetadata.type.isNumber()) + throw new InvalidConstraintDefinitionException(param + " is not a number"); + } + + @Override + public ConstraintType getConstraintType() + { + return ConstraintType.SCALAR; + } + + @Override + public String toString() + { + return param + " " + relationType + " " + term; + } + + @Override + public MetadataSerializer<ScalarColumnConstraint> serializer() + { + return serializer; + } + + @Override + public void appendCqlTo(CqlBuilder builder) + { + builder.append(toString()); + } + + private static class Serializer implements MetadataSerializer<ScalarColumnConstraint> + { + @Override + public void serialize(ScalarColumnConstraint columnConstraint, DataOutputPlus out, Version version) throws IOException + { + out.writeUTF(columnConstraint.param.toCQLString()); + columnConstraint.relationType.writeTo(out); + out.writeUTF(columnConstraint.term); + } + + @Override + public ScalarColumnConstraint deserialize(DataInputPlus in, Version version) throws IOException + { + ColumnIdentifier param = new ColumnIdentifier(in.readUTF(), true); + Operator relationType = Operator.readFrom(in); + return new ScalarColumnConstraint(param, relationType, in.readUTF()); + } + + @Override + public long serializedSize(ScalarColumnConstraint columnConstraint, Version version) + { + return TypeSizes.sizeof(columnConstraint.term) + + Operator.serializedSize() + + TypeSizes.sizeof(columnConstraint.param.toString()); + } + } + + @Override + public boolean equals(Object o) + { + if (this == o) + return true; + + if (!(o instanceof ScalarColumnConstraint)) + return false; + + ScalarColumnConstraint other = (ScalarColumnConstraint) o; + + return param.equals(other.param) + && relationType == other.relationType + && term.equals(other.term); + } +} diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java index 113135016d8a..b528133e437c 100644 --- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java @@ -26,6 +26,7 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.cql3.constraints.ConstraintViolationException; import org.apache.cassandra.db.guardrails.Guardrails; import org.apache.cassandra.dht.Token; import org.apache.cassandra.locator.Replica; @@ -800,6 +801,7 @@ final void addUpdates(UpdatesCollector collector, for (ByteBuffer key : keys) { Validation.validateKey(metadata(), key); + Validation.checkConstraints(metadata(), key); DecoratedKey dk = metadata().partitioner.decorateKey(key); PartitionUpdate.Builder updateBuilder = collector.getPartitionUpdateBuilder(metadata(), dk, options.getConsistency()); @@ -810,9 +812,11 @@ final void addUpdates(UpdatesCollector collector, } else { + // Clustering keys need to be checked on their own for (Clustering<?> clustering : clusterings) { clustering.validate(); + checkClusteringConstraints(clustering); addUpdateForKey(updateBuilder, clustering, params); } } @@ -820,6 +824,24 @@ final void addUpdates(UpdatesCollector collector, } } + private void checkClusteringConstraints(Clustering<?> clustering) + { + for (ColumnMetadata column : metadata.clusteringColumns()) + { + if (column.hasConstraint()) + { + try + { + clustering.checkConstraints(column.position(), metadata.comparator, column.getColumnConstraints()); + } + catch (ConstraintViolationException e) + { + throw new InvalidRequestException(e.getMessage(), e); + } + } + } + } + public Slices createSlices(QueryOptions options) { return restrictions.getSlices(options); diff --git a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java index 378b827a368f..d8310cdd2f8d 100644 --- a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java @@ -17,6 +17,7 @@ */ package org.apache.cassandra.cql3.statements; +import java.nio.ByteBuffer; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -26,12 +27,15 @@ import org.apache.cassandra.cql3.*; import org.apache.cassandra.cql3.conditions.ColumnCondition; import org.apache.cassandra.cql3.conditions.Conditions; +import org.apache.cassandra.cql3.constraints.ColumnConstraint; import org.apache.cassandra.cql3.restrictions.StatementRestrictions; import org.apache.cassandra.cql3.terms.Constants; import org.apache.cassandra.cql3.terms.Term; import org.apache.cassandra.db.Clustering; import org.apache.cassandra.db.Slice; import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.rows.Cell; +import org.apache.cassandra.db.rows.Row; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.ClientState; @@ -93,7 +97,9 @@ public void addUpdateForKey(PartitionUpdate.Builder updateBuilder, Clustering<?> for (int i = 0, isize = updates.size(); i < isize; i++) updates.get(i).execute(updateBuilder.partitionKey(), params); - updateBuilder.add(params.buildRow()); + Row row = params.buildRow(); + evaluateConstraintsForRow(row, metadata); + updateBuilder.add(row); } if (updatesStaticRow()) @@ -348,4 +354,26 @@ public AuditLogContext getAuditLogContext() { return new AuditLogContext(AuditLogEntryType.UPDATE, keyspace(), table()); } + + public static void evaluateConstraintsForRow(Row row, TableMetadata metadata) + { + for (ColumnMetadata column : metadata.columnsWithConstraints) + { + Cell<?> cell = row.getCell(column); + if (cell != null) + { + ColumnMetadata columnMetadata = cell.column(); + ByteBuffer cellData = cell.buffer(); + evaluateConstraint(columnMetadata, cellData); + } + } + } + + public static void evaluateConstraint(ColumnMetadata columnMetadata, ByteBuffer cellData) + { + for (ColumnConstraint constraint : columnMetadata.getColumnConstraints().getConstraints()) + { + constraint.evaluate(columnMetadata.type, cellData); + } + } } diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java index 4302919ca99b..27d996134064 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java @@ -42,6 +42,7 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.CQL3Type; import org.apache.cassandra.cql3.CQLStatement; +import org.apache.cassandra.cql3.constraints.ColumnConstraints; import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.QualifiedName; import org.apache.cassandra.cql3.functions.masking.ColumnMask; @@ -710,6 +711,67 @@ private void validateCanDropCompactStorage() } } + public static class DropConstraints extends AlterTableStatement + { + final ColumnIdentifier columnName; + + DropConstraints(String keyspaceName, String tableName, boolean ifTableExists, ColumnIdentifier columnName) + { + super(keyspaceName, tableName, ifTableExists); + this.columnName = columnName; + } + + @Override + public KeyspaceMetadata apply(Epoch epoch, KeyspaceMetadata keyspace, TableMetadata table, ClusterMetadata metadata) + { + ColumnMetadata columnMetadata = table.getColumn(columnName); + columnMetadata.removeColumnConstraints(); + + TableMetadata.Builder tableBuilder = table.unbuild().epoch(epoch); + Views.Builder viewsBuilder = keyspace.views.unbuild(); + TableMetadata tableMetadata = tableBuilder.build(); + tableMetadata.validate(); + + return keyspace.withSwapped(keyspace.tables.withSwapped(tableMetadata)) + .withSwapped(viewsBuilder.build()); + } + } + + public static class AlterConstraints extends AlterTableStatement + { + final ColumnIdentifier columnName; + final ColumnConstraints constraints; + + AlterConstraints(String keyspaceName, String tableName, boolean ifTableExists, ColumnIdentifier columnName, ColumnConstraints constraints) + { + super(keyspaceName, tableName, ifTableExists); + this.columnName = columnName; + this.constraints = constraints; + } + + @Override + public KeyspaceMetadata apply(Epoch epoch, KeyspaceMetadata keyspace, TableMetadata table, ClusterMetadata metadata) + { + TableMetadata.Builder tableBuilder = table.unbuild().epoch(epoch); + + for (ColumnMetadata column : tableBuilder.columns()) + { + if (column.name == columnName) + { + column.setColumnConstraints(constraints); + break; + } + } + + Views.Builder viewsBuilder = keyspace.views.unbuild(); + TableMetadata tableMetadata = tableBuilder.build(); + tableMetadata.validate(); + + return keyspace.withSwapped(keyspace.tables.withSwapped(tableMetadata)) + .withSwapped(viewsBuilder.build()); + } + } + public static final class Raw extends CQLStatement.Raw { private enum Kind @@ -720,13 +782,17 @@ private enum Kind DROP_COLUMNS, RENAME_COLUMNS, ALTER_OPTIONS, - DROP_COMPACT_STORAGE + DROP_COMPACT_STORAGE, + DROP_CONSTRAINTS, + ALTER_CONSTRAINTS } private final QualifiedName name; private final boolean ifTableExists; private boolean ifColumnExists; private boolean ifColumnNotExists; + private ColumnIdentifier constraintName; + private ColumnConstraints constraints; private Kind kind; @@ -748,9 +814,15 @@ private enum Kind public final TableAttributes attrs = new TableAttributes(); public Raw(QualifiedName name, boolean ifTableExists) + { + this(name, ifTableExists, null); + } + + public Raw(QualifiedName name, boolean ifTableExists, ColumnIdentifier constraintName) { this.name = name; this.ifTableExists = ifTableExists; + this.constraintName = constraintName; } public AlterTableStatement prepare(ClientState state) @@ -767,6 +839,8 @@ public AlterTableStatement prepare(ClientState state) case RENAME_COLUMNS: return new RenameColumns(keyspaceName, tableName, renamedColumns, ifTableExists, ifColumnExists); case ALTER_OPTIONS: return new AlterOptions(keyspaceName, tableName, attrs, ifTableExists); case DROP_COMPACT_STORAGE: return new DropCompactStorage(keyspaceName, tableName, ifTableExists); + case DROP_CONSTRAINTS: return new DropConstraints(keyspaceName, tableName, ifTableExists, constraintName); + case ALTER_CONSTRAINTS: return new AlterConstraints(keyspaceName, tableName, ifTableExists, constraintName, constraints); } throw new AssertionError(); @@ -811,6 +885,19 @@ public void dropCompactStorage() kind = Kind.DROP_COMPACT_STORAGE; } + public void dropConstraints(ColumnIdentifier name) + { + kind = Kind.DROP_CONSTRAINTS; + this.constraintName = name; + } + + public void alterConstraints(ColumnIdentifier name, ColumnConstraints.Raw rawConstraints) + { + kind = Kind.ALTER_CONSTRAINTS; + this.constraintName = name; + this.constraints = rawConstraints.prepare(); + } + public void timestamp(long timestamp) { this.timestamp = timestamp; diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java index 529054a1b7bf..8338b4006eaa 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java @@ -32,6 +32,7 @@ import org.apache.cassandra.auth.IResource; import org.apache.cassandra.auth.Permission; import org.apache.cassandra.cql3.*; +import org.apache.cassandra.cql3.constraints.ColumnConstraints; import org.apache.cassandra.cql3.functions.masking.ColumnMask; import org.apache.cassandra.db.guardrails.Guardrails; import org.apache.cassandra.db.marshal.*; @@ -56,6 +57,7 @@ public final class CreateTableStatement extends AlterSchemaStatement private final Map<ColumnIdentifier, ColumnProperties.Raw> rawColumns; private final Set<ColumnIdentifier> staticColumns; private final List<ColumnIdentifier> partitionKeyColumns; + private final Map<ColumnIdentifier, ColumnConstraints> columnConstraints; private final List<ColumnIdentifier> clusteringColumns; private final LinkedHashMap<ColumnIdentifier, Boolean> clusteringOrder; @@ -72,6 +74,7 @@ public CreateTableStatement(String keyspaceName, Set<ColumnIdentifier> staticColumns, List<ColumnIdentifier> partitionKeyColumns, List<ColumnIdentifier> clusteringColumns, + Map<ColumnIdentifier, ColumnConstraints> columnConstraints, LinkedHashMap<ColumnIdentifier, Boolean> clusteringOrder, TableAttributes attrs, boolean ifNotExists, @@ -84,6 +87,7 @@ public CreateTableStatement(String keyspaceName, this.staticColumns = staticColumns; this.partitionKeyColumns = partitionKeyColumns; this.clusteringColumns = clusteringColumns; + this.columnConstraints = columnConstraints; this.clusteringOrder = clusteringOrder; this.attrs = attrs; @@ -343,13 +347,13 @@ public TableMetadata.Builder builder(Types types, UserFunctions functions) for (int i = 0; i < partitionKeyColumns.size(); i++) { ColumnProperties properties = partitionKeyColumnProperties.get(i); - builder.addPartitionKeyColumn(partitionKeyColumns.get(i), properties.type, properties.mask); + builder.addPartitionKeyColumn(partitionKeyColumns.get(i), properties.type, properties.mask, columnConstraints.get(partitionKeyColumns.get(i))); } for (int i = 0; i < clusteringColumns.size(); i++) { ColumnProperties properties = clusteringColumnProperties.get(i); - builder.addClusteringColumn(clusteringColumns.get(i), properties.type, properties.mask); + builder.addClusteringColumn(clusteringColumns.get(i), properties.type, properties.mask, columnConstraints.get(clusteringColumns.get(i))); } if (useCompactStorage) @@ -360,11 +364,12 @@ public TableMetadata.Builder builder(Types types, UserFunctions functions) { columns.forEach((column, properties) -> { if (staticColumns.contains(column)) - builder.addStaticColumn(column, properties.type, properties.mask); + builder.addStaticColumn(column, properties.type, properties.mask, columnConstraints.get(column)); else - builder.addRegularColumn(column, properties.type, properties.mask); + builder.addRegularColumn(column, properties.type, properties.mask, columnConstraints.get(column)); }); } + return builder; } @@ -506,6 +511,7 @@ public final static class Raw extends CQLStatement.Raw private final Map<ColumnIdentifier, ColumnProperties.Raw> rawColumns = new HashMap<>(); private final Set<ColumnIdentifier> staticColumns = new HashSet<>(); private final List<ColumnIdentifier> clusteringColumns = new ArrayList<>(); + private final Map<ColumnIdentifier, ColumnConstraints> columnConstraints = new HashMap<>(); private List<ColumnIdentifier> partitionKeyColumns; @@ -531,6 +537,7 @@ public CreateTableStatement prepare(ClientState state) staticColumns, partitionKeyColumns, clusteringColumns, + columnConstraints, clusteringOrder, attrs, ifNotExists, @@ -559,14 +566,17 @@ public String table() return name.getName(); } - public void addColumn(ColumnIdentifier column, CQL3Type.Raw type, boolean isStatic, ColumnMask.Raw mask) + public void addColumn(ColumnIdentifier column, CQL3Type.Raw type, boolean isStatic, ColumnMask.Raw mask, ColumnConstraints.Raw constraints) { - if (null != rawColumns.put(column, new ColumnProperties.Raw(type, mask))) throw ire("Duplicate column '%s' declaration for table '%s'", column, name); if (isStatic) staticColumns.add(column); + if (null == constraints) + columnConstraints.put(column, ColumnConstraints.NO_OP); + else + columnConstraints.put(column, constraints.prepare()); } public void setCompactStorage() diff --git a/src/java/org/apache/cassandra/db/ClusteringPrefix.java b/src/java/org/apache/cassandra/db/ClusteringPrefix.java index 02f9330b430b..167d89c6a485 100644 --- a/src/java/org/apache/cassandra/db/ClusteringPrefix.java +++ b/src/java/org/apache/cassandra/db/ClusteringPrefix.java @@ -24,6 +24,7 @@ import org.apache.cassandra.cache.IMeasurableMemory; import org.apache.cassandra.config.*; +import org.apache.cassandra.cql3.constraints.ColumnConstraints; import org.apache.cassandra.db.marshal.ByteArrayAccessor; import org.apache.cassandra.db.marshal.ByteBufferAccessor; import org.apache.cassandra.db.marshal.CompositeType; @@ -752,4 +753,9 @@ public static boolean equals(ClusteringPrefix<?> prefix, Object o) return equals(prefix, (ClusteringPrefix<?>) o); } + default void checkConstraints(int clusterIndex, ClusteringComparator comparator, ColumnConstraints constraints) + { + comparator.subtype(clusterIndex).checkConstraints(accessor().toBuffer(get(clusterIndex)), constraints); + } + } \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/Columns.java b/src/java/org/apache/cassandra/db/Columns.java index 275d000dd369..60852c5c5667 100644 --- a/src/java/org/apache/cassandra/db/Columns.java +++ b/src/java/org/apache/cassandra/db/Columns.java @@ -28,6 +28,7 @@ import net.nicoulaj.compilecommand.annotations.DontInline; import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.constraints.ColumnConstraints; import org.apache.cassandra.db.marshal.SetType; import org.apache.cassandra.db.marshal.UTF8Type; import org.apache.cassandra.db.rows.ColumnData; @@ -62,7 +63,8 @@ public class Columns extends AbstractCollection<ColumnMetadata> implements Colle SetType.getInstance(UTF8Type.instance, true), ColumnMetadata.NO_POSITION, ColumnMetadata.Kind.STATIC, - null); + null, + ColumnConstraints.NO_OP); public static final ColumnMetadata FIRST_COMPLEX_REGULAR = new ColumnMetadata("", @@ -71,7 +73,8 @@ public class Columns extends AbstractCollection<ColumnMetadata> implements Colle SetType.getInstance(UTF8Type.instance, true), ColumnMetadata.NO_POSITION, ColumnMetadata.Kind.REGULAR, - null); + null, + ColumnConstraints.NO_OP); private final Object[] columns; private final int complexIdx; // Index of the first complex column diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java b/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java index dfa7496ea6e8..737954d09bdb 100644 --- a/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java +++ b/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java @@ -23,6 +23,9 @@ import java.util.List; import java.util.regex.Pattern; +import org.apache.cassandra.cql3.constraints.ColumnConstraint; +import org.apache.cassandra.cql3.constraints.ColumnConstraints; +import org.apache.cassandra.cql3.constraints.ConstraintViolationException; import org.apache.cassandra.cql3.terms.Term; import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.transport.ProtocolVersion; @@ -47,6 +50,13 @@ public boolean allowsEmpty() return true; } + @Override + public boolean isConstrainable() + { + // Constraints are not supported for composite types + return false; + } + public <VL, VR> int compareCustom(VL left, ValueAccessor<VL> accessorL, VR right, ValueAccessor<VR> accessorR) { if (accessorL.isEmpty(left) || accessorR.isEmpty(right)) @@ -281,7 +291,8 @@ public void validate(ByteBuffer bb) throws MarshalException validate(bb, ByteBufferAccessor.instance); } - public <V> void validate(V input, ValueAccessor<V> accessor) + @Override + public <V> void validate(V input, ValueAccessor<V> accessor) { boolean isStatic = readIsStatic(input, accessor); int offset = startingOffset(isStatic); @@ -316,6 +327,55 @@ public <V> void validate(V input, ValueAccessor<V> accessor) } } + @Override + public void checkConstraints(ByteBuffer input, ColumnConstraints constraints) throws ConstraintViolationException + { + // no constraints defined for the partition keys + if (!constraints.hasRelevantConstraints()) + return; + + ValueAccessor<ByteBuffer> accessor = ByteBufferAccessor.instance; + + boolean isStatic = readIsStatic(input, accessor); + int offset = startingOffset(isStatic); + + int i = 0; + List<ByteBuffer> partitionKeyValues = new ArrayList<>(); + while (!accessor.isEmptyFromOffset(input, offset)) + { + AbstractType<?> comparator = getComparator(i, input, accessor, offset); + offset += getComparatorSize(i, input, accessor, offset); + + int length = accessor.getUnsignedShort(input, offset); + offset += 2; + + ByteBuffer value = accessor.slice(input, offset, length); +// partitionKeyValues.add(comparator.compose(value)); + partitionKeyValues.add(value); + offset += length; + + accessor.getByte(input, offset++); + ++i; + } + + if (partitionKeyValues.size() != constraints.getConstraints().size()) + { + // contraints list should have the exact size of partition key values. + // Noop constraints are filled for the partition key columns w/o any constraints. + throw new IllegalStateException("The number of constraints (" + partitionKeyValues.size() + ") " + + "should be the same with the number of partition key columns (" + + constraints.getConstraints().size() + ")"); + } + + for (int k = 0; k < constraints.getConstraints().size(); k++) + { + AbstractType<?> comparator = getComparator(i, input, accessor, offset); + ByteBuffer value = partitionKeyValues.get(k); + ColumnConstraint constraint = constraints.getConstraints().get(k); + constraint.evaluate(comparator, value); + } + } + public abstract ByteBuffer decompose(Object... objects); abstract protected <V> int getComparatorSize(int i, V value, ValueAccessor<V> accessor, int offset); diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractType.java b/src/java/org/apache/cassandra/db/marshal/AbstractType.java index 3c382e781be1..29f446838949 100644 --- a/src/java/org/apache/cassandra/db/marshal/AbstractType.java +++ b/src/java/org/apache/cassandra/db/marshal/AbstractType.java @@ -31,6 +31,9 @@ import org.apache.cassandra.cql3.AssignmentTestable; import org.apache.cassandra.cql3.CQL3Type; import org.apache.cassandra.cql3.ColumnSpecification; +import org.apache.cassandra.cql3.constraints.ColumnConstraint; +import org.apache.cassandra.cql3.constraints.ColumnConstraints; +import org.apache.cassandra.cql3.constraints.ConstraintViolationException; import org.apache.cassandra.cql3.terms.Term; import org.apache.cassandra.cql3.functions.ArgumentDeserializer; import org.apache.cassandra.db.rows.Cell; @@ -204,6 +207,25 @@ public <V> void validate(V value, ValueAccessor<V> accessor) throws MarshalExcep getSerializer().validate(value, accessor); } + public void checkConstraints(ByteBuffer bytes, ColumnConstraints constraints) throws ConstraintViolationException + { + if (constraints.isEmpty()) + return; + + T value = getSerializer().deserialize(bytes); + constraints.evaluate(this, bytes); + } + + public void checkConstraints(ByteBuffer bytes, List<ColumnConstraint> constraints) throws ConstraintViolationException + { + if (constraints.isEmpty()) + return; + + T value = getSerializer().deserialize(bytes); + for (ColumnConstraint constraint : constraints) + constraint.evaluate(this, bytes); + } + public final int compare(ByteBuffer left, ByteBuffer right) { return compare(left, ByteBufferAccessor.instance, right, ByteBufferAccessor.instance); @@ -525,6 +547,11 @@ public <V> boolean isNull(V buffer, ValueAccessor<V> accessor) return getSerializer().isNull(buffer, accessor); } + public boolean isNumber() + { + return unwrap() instanceof org.apache.cassandra.db.marshal.NumberType; + } + // This assumes that no empty values are passed public void writeValue(ByteBuffer value, DataOutputPlus out) throws IOException { @@ -799,4 +826,9 @@ public Object deserialize(ProtocolVersion protocolVersion, ByteBuffer buffer) return type.compose(buffer); } } + + public boolean isConstrainable() + { + return true; + } } diff --git a/src/java/org/apache/cassandra/db/marshal/MapType.java b/src/java/org/apache/cassandra/db/marshal/MapType.java index 69ea6d17e1d3..f8ac6c00680e 100644 --- a/src/java/org/apache/cassandra/db/marshal/MapType.java +++ b/src/java/org/apache/cassandra/db/marshal/MapType.java @@ -456,4 +456,10 @@ public ByteBuffer getElement(@Nullable ColumnData columnData, ByteBuffer keyOrIn return getSerializer().getSerializedValue(((Cell<?>) columnData).buffer(), keyOrIndex, getValuesType()); } + + @Override + public boolean isConstrainable() + { + return false; + } } diff --git a/src/java/org/apache/cassandra/db/marshal/ReversedType.java b/src/java/org/apache/cassandra/db/marshal/ReversedType.java index b567fe348126..1bf60844cfa3 100644 --- a/src/java/org/apache/cassandra/db/marshal/ReversedType.java +++ b/src/java/org/apache/cassandra/db/marshal/ReversedType.java @@ -244,4 +244,10 @@ public AbstractType<T> unwrap() { return baseType.unwrap(); } + + @Override + public boolean isConstrainable() + { + return unwrap().isConstrainable(); + } } diff --git a/src/java/org/apache/cassandra/db/marshal/TupleType.java b/src/java/org/apache/cassandra/db/marshal/TupleType.java index feb9b405fc9d..d6ce2da0f48d 100644 --- a/src/java/org/apache/cassandra/db/marshal/TupleType.java +++ b/src/java/org/apache/cassandra/db/marshal/TupleType.java @@ -616,4 +616,10 @@ public int compareCQL(ComplexColumnData columnData, List<ByteBuffer> fields) { throw new UnsupportedOperationException("Multicell tuples are not supported"); } + + @Override + public boolean isConstrainable() + { + return false; + } } diff --git a/src/java/org/apache/cassandra/db/marshal/UserType.java b/src/java/org/apache/cassandra/db/marshal/UserType.java index 82e9aafcc67a..804891448345 100644 --- a/src/java/org/apache/cassandra/db/marshal/UserType.java +++ b/src/java/org/apache/cassandra/db/marshal/UserType.java @@ -618,6 +618,12 @@ protected String componentOrFieldName(int i) return "field " + fieldName(i); } + @Override + public boolean isConstrainable() + { + return false; + } + private enum ConflictBehavior { LOG { diff --git a/src/java/org/apache/cassandra/schema/ColumnMetadata.java b/src/java/org/apache/cassandra/schema/ColumnMetadata.java index cb3c879a94b9..79fad3a80e56 100644 --- a/src/java/org/apache/cassandra/schema/ColumnMetadata.java +++ b/src/java/org/apache/cassandra/schema/ColumnMetadata.java @@ -22,6 +22,7 @@ import java.util.*; import java.util.function.Predicate; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import com.google.common.annotations.VisibleForTesting; @@ -30,6 +31,8 @@ import com.google.common.collect.Lists; import org.apache.cassandra.cql3.*; +import org.apache.cassandra.cql3.constraints.ColumnConstraint; +import org.apache.cassandra.cql3.constraints.ColumnConstraints; import org.apache.cassandra.cql3.functions.masking.ColumnMask; import org.apache.cassandra.cql3.selection.Selectable; import org.apache.cassandra.cql3.selection.Selector; @@ -115,6 +118,9 @@ public boolean isPrimaryKeyKind() @Nullable private final ColumnMask mask; + @Nonnull + private ColumnConstraints columnConstraints; + private static long comparisonOrder(Kind kind, boolean isComplex, long position, ColumnIdentifier name) { assert position >= 0 && position < 1 << 12; @@ -126,42 +132,60 @@ private static long comparisonOrder(Kind kind, boolean isComplex, long position, public static ColumnMetadata partitionKeyColumn(TableMetadata table, ByteBuffer name, AbstractType<?> type, int position) { - return new ColumnMetadata(table, name, type, position, Kind.PARTITION_KEY, null); + return new ColumnMetadata(table, name, type, position, Kind.PARTITION_KEY, null, ColumnConstraints.NO_OP); } public static ColumnMetadata partitionKeyColumn(String keyspace, String table, String name, AbstractType<?> type, int position) { - return new ColumnMetadata(keyspace, table, ColumnIdentifier.getInterned(name, true), type, position, Kind.PARTITION_KEY, null); + return new ColumnMetadata(keyspace, table, ColumnIdentifier.getInterned(name, true), type, position, Kind.PARTITION_KEY, null, ColumnConstraints.NO_OP); } public static ColumnMetadata clusteringColumn(TableMetadata table, ByteBuffer name, AbstractType<?> type, int position) { - return new ColumnMetadata(table, name, type, position, Kind.CLUSTERING, null); + return new ColumnMetadata(table, name, type, position, Kind.CLUSTERING, null, ColumnConstraints.NO_OP); } public static ColumnMetadata clusteringColumn(String keyspace, String table, String name, AbstractType<?> type, int position) { - return new ColumnMetadata(keyspace, table, ColumnIdentifier.getInterned(name, true), type, position, Kind.CLUSTERING, null); + return new ColumnMetadata(keyspace, table, ColumnIdentifier.getInterned(name, true), type, position, Kind.CLUSTERING, null, ColumnConstraints.NO_OP); } public static ColumnMetadata regularColumn(TableMetadata table, ByteBuffer name, AbstractType<?> type) { - return new ColumnMetadata(table, name, type, NO_POSITION, Kind.REGULAR, null); + return new ColumnMetadata(table, name, type, NO_POSITION, Kind.REGULAR, null, ColumnConstraints.NO_OP); } public static ColumnMetadata regularColumn(String keyspace, String table, String name, AbstractType<?> type) { - return new ColumnMetadata(keyspace, table, ColumnIdentifier.getInterned(name, true), type, NO_POSITION, Kind.REGULAR, null); + return new ColumnMetadata(keyspace, table, ColumnIdentifier.getInterned(name, true), type, NO_POSITION, Kind.REGULAR, null, ColumnConstraints.NO_OP); } public static ColumnMetadata staticColumn(TableMetadata table, ByteBuffer name, AbstractType<?> type) { - return new ColumnMetadata(table, name, type, NO_POSITION, Kind.STATIC, null); + return new ColumnMetadata(table, name, type, NO_POSITION, Kind.STATIC, null, ColumnConstraints.NO_OP); } public static ColumnMetadata staticColumn(String keyspace, String table, String name, AbstractType<?> type) { - return new ColumnMetadata(keyspace, table, ColumnIdentifier.getInterned(name, true), type, NO_POSITION, Kind.STATIC, null); + return new ColumnMetadata(keyspace, table, ColumnIdentifier.getInterned(name, true), type, NO_POSITION, Kind.STATIC, null, ColumnConstraints.NO_OP); + } + + public ColumnMetadata(TableMetadata table, + ByteBuffer name, + AbstractType<?> type, + int position, + Kind kind, + @Nullable ColumnMask mask, + @Nonnull ColumnConstraints columnConstraints) + { + this(table.keyspace, + table.name, + ColumnIdentifier.getInterned(name, UTF8Type.instance), + type, + position, + kind, + mask, + columnConstraints); } public ColumnMetadata(TableMetadata table, @@ -177,7 +201,8 @@ public ColumnMetadata(TableMetadata table, type, position, kind, - mask); + mask, + ColumnConstraints.NO_OP); } @VisibleForTesting @@ -188,6 +213,19 @@ public ColumnMetadata(String ksName, int position, Kind kind, @Nullable ColumnMask mask) + { + this(ksName, cfName, name, type, position, kind, mask, ColumnConstraints.NO_OP); + } + + @VisibleForTesting + public ColumnMetadata(String ksName, + String cfName, + ColumnIdentifier name, + AbstractType<?> type, + int position, + Kind kind, + @Nullable ColumnMask mask, + @Nonnull ColumnConstraints columnConstraints) { super(ksName, cfName, name, type); assert name != null && type != null && kind != null; @@ -206,6 +244,7 @@ public ColumnMetadata(String ksName, this.asymmetricCellPathComparator = cellPathComparator == null ? null : (a, b) -> cellPathComparator.compare(((Cell<?>)a).path(), (CellPath) b); this.comparisonOrder = comparisonOrder(kind, isComplex(), Math.max(0, position), name); this.mask = mask; + this.columnConstraints = columnConstraints; } private static Comparator<CellPath> makeCellPathComparator(Kind kind, AbstractType<?> type) @@ -237,22 +276,22 @@ private static Comparator<CellPath> makeCellPathComparator(Kind kind, AbstractTy public ColumnMetadata copy() { - return new ColumnMetadata(ksName, cfName, name, type, position, kind, mask); + return new ColumnMetadata(ksName, cfName, name, type, position, kind, mask, columnConstraints); } public ColumnMetadata withNewName(ColumnIdentifier newName) { - return new ColumnMetadata(ksName, cfName, newName, type, position, kind, mask); + return new ColumnMetadata(ksName, cfName, newName, type, position, kind, mask, columnConstraints); } public ColumnMetadata withNewType(AbstractType<?> newType) { - return new ColumnMetadata(ksName, cfName, name, newType, position, kind, mask); + return new ColumnMetadata(ksName, cfName, name, newType, position, kind, mask, columnConstraints); } public ColumnMetadata withNewMask(@Nullable ColumnMask newMask) { - return new ColumnMetadata(ksName, cfName, name, type, position, kind, newMask); + return new ColumnMetadata(ksName, cfName, name, type, position, kind, newMask, columnConstraints); } public boolean isPartitionKey() @@ -275,6 +314,11 @@ public boolean isMasked() return mask != null; } + public boolean hasConstraint() + { + return columnConstraints.hasRelevantConstraints(); + } + public boolean isRegular() { return kind == Kind.REGULAR; @@ -299,6 +343,21 @@ public ColumnMask getMask() return mask; } + public ColumnConstraints getColumnConstraints() + { + return columnConstraints; + } + + public void setColumnConstraints(ColumnConstraints constraints) + { + this.columnConstraints = constraints; + } + + public void removeColumnConstraints() + { + columnConstraints = ColumnConstraints.NO_OP; + } + @Override public boolean equals(Object o) { @@ -320,7 +379,8 @@ private boolean equalsWithoutType(ColumnMetadata other) && position == other.position && ksName.equals(other.ksName) && cfName.equals(other.cfName) - && Objects.equals(mask, other.mask); + && Objects.equals(mask, other.mask) + && Objects.equals(columnConstraints, other.columnConstraints); } Optional<Difference> compare(ColumnMetadata other) @@ -351,6 +411,7 @@ public int hashCode() result = 31 * result + (kind == null ? 0 : kind.hashCode()); result = 31 * result + position; result = 31 * result + (mask == null ? 0 : mask.hashCode()); + result = 31 * result + (columnConstraints == null ? 0 : columnConstraints.hashCode()); hash = result; } return result; @@ -516,6 +577,19 @@ public void appendCqlTo(CqlBuilder builder) if (isMasked()) mask.appendCqlTo(builder); + + if (!columnConstraints.isEmpty()) + { + builder.append(" CHECK "); + Iterator<ColumnConstraint<?>> constraintIterator = columnConstraints.getConstraints().iterator(); + constraintIterator.next().appendCqlTo(builder); + + while (constraintIterator.hasNext()) + { + builder.append(" AND "); + constraintIterator.next().appendCqlTo(builder); + } + } } public static String toCQLString(Iterable<ColumnMetadata> defs) @@ -609,6 +683,13 @@ public void serialize(ColumnMetadata t, DataOutputPlus out, Version version) thr out.writeBoolean(t.mask != null); if (t.mask != null) ColumnMask.serializer.serialize(t.mask, out, version); + if (version.isAtLeast(Version.V6)) + { + boolean hasConstraints = t.hasConstraint(); + out.writeBoolean(hasConstraints); + if (hasConstraints) + ColumnConstraints.serializer.serialize(t.columnConstraints, out, version); + } } public ColumnMetadata deserialize(DataInputPlus in, Types types, UserFunctions functions, Version version) throws IOException @@ -627,11 +708,23 @@ public ColumnMetadata deserialize(DataInputPlus in, Types types, UserFunctions f boolean masked = in.readBoolean(); if (masked) mask = ColumnMask.serializer.deserialize(in, ksName, type, types, functions, version); - return new ColumnMetadata(ksName, tableName, new ColumnIdentifier(nameBB, name), type, position, kind, mask); + ColumnConstraints constraints; + if (version.isAtLeast(Version.V6) && in.readBoolean()) + constraints = ColumnConstraints.serializer.deserialize(in, version); + else + constraints = ColumnConstraints.NO_OP; + return new ColumnMetadata(ksName, tableName, new ColumnIdentifier(nameBB, name), type, position, kind, mask, constraints); } public long serializedSize(ColumnMetadata t, Version version) { + long constraintsSize = 0; + if (version.isAtLeast(Version.V6)) + { + constraintsSize += BOOL_SIZE; + if (t.hasConstraint()) + constraintsSize += t.getColumnConstraints().serializer().serializedSize(t.columnConstraints, version); + } return sizeof(t.ksName) + sizeof(t.cfName) + sizeof(t.kind.name()) + @@ -641,7 +734,8 @@ public long serializedSize(ColumnMetadata t, Version version) sizeof(t.name.toString()) + ByteBufferUtil.serializedSizeWithShortLength(t.name.bytes) + BOOL_SIZE + - ((t.mask == null) ? 0 : ColumnMask.serializer.serializedSize(t.mask, version)); + ((t.mask == null) ? 0 : ColumnMask.serializer.serializedSize(t.mask, version)) + + constraintsSize; } } } diff --git a/src/java/org/apache/cassandra/schema/TableMetadata.java b/src/java/org/apache/cassandra/schema/TableMetadata.java index b97f8ef5a754..6f9672d3c1dd 100644 --- a/src/java/org/apache/cassandra/schema/TableMetadata.java +++ b/src/java/org/apache/cassandra/schema/TableMetadata.java @@ -32,6 +32,7 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import com.google.common.base.MoreObjects; @@ -46,9 +47,12 @@ import org.apache.cassandra.auth.DataResource; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.constraints.ColumnConstraint; +import org.apache.cassandra.cql3.constraints.ColumnConstraints; import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.CqlBuilder; import org.apache.cassandra.cql3.SchemaElement; +import org.apache.cassandra.cql3.constraints.InvalidConstraintDefinitionException; import org.apache.cassandra.cql3.functions.Function; import org.apache.cassandra.cql3.functions.masking.ColumnMask; import org.apache.cassandra.db.Clustering; @@ -196,6 +200,12 @@ public enum Kind public final DataResource resource; public TableMetadataRef ref; + // We cache the columns with constraints to avoid iterations over columns + // Partition keys columns are evaluated separately, so we keep the two of them in + // two different variables. + public final List<ColumnConstraint> partitionKeyConstraints; + public final List<ColumnMetadata> columnsWithConstraints; + protected TableMetadata(Builder builder) { flags = Sets.immutableEnumSet(builder.flags); @@ -235,6 +245,22 @@ else if (isIndex()) ref = TableMetadataRef.forIndex(Schema.instance, this, keyspace, indexName, id); else ref = TableMetadataRef.withInitialReference(new TableMetadataRef(Schema.instance, keyspace, name, id), this); + + List<ColumnConstraint> pkConstraints = new ArrayList<>(this.partitionKeyColumns.size()); + for (ColumnMetadata column : this.partitionKeyColumns) + { + if (column.hasConstraint()) + pkConstraints.add(column.getColumnConstraints()); + } + this.partitionKeyConstraints = pkConstraints; + + List<ColumnMetadata> columnsWithConstraints = new ArrayList<>(); + for (ColumnMetadata column : this.columns()) + { + if (column.hasConstraint() && !column.isPartitionKey() && !column.isClusteringColumn()) + columnsWithConstraints.add(column); + } + this.columnsWithConstraints = columnsWithConstraints; } public static Builder builder(String keyspace, String table) @@ -527,6 +553,19 @@ public void validate() except("Missing partition keys for table %s", toString()); indexes.validate(this); + + for (ColumnMetadata columnMetadata : columns()) + { + ColumnConstraints constraints = columnMetadata.getColumnConstraints(); + try + { + constraints.validate(columnMetadata); + } + catch (InvalidConstraintDefinitionException e) + { + throw new InvalidRequestException(e.getMessage(), e); + } + } } /** @@ -1019,7 +1058,12 @@ public Builder addPartitionKeyColumn(ColumnIdentifier name, AbstractType<?> type public Builder addPartitionKeyColumn(ColumnIdentifier name, AbstractType<?> type, @Nullable ColumnMask mask) { - return addColumn(new ColumnMetadata(keyspace, this.name, name, type, partitionKeyColumns.size(), ColumnMetadata.Kind.PARTITION_KEY, mask)); + return addPartitionKeyColumn(name, type, mask, ColumnConstraints.NO_OP); + } + + public Builder addPartitionKeyColumn(ColumnIdentifier name, AbstractType<?> type, @Nullable ColumnMask mask, @Nonnull ColumnConstraints cqlConstraints) + { + return addColumn(new ColumnMetadata(keyspace, this.name, name, type, partitionKeyColumns.size(), ColumnMetadata.Kind.PARTITION_KEY, mask, cqlConstraints)); } public Builder addClusteringColumn(String name, AbstractType<?> type) @@ -1039,7 +1083,12 @@ public Builder addClusteringColumn(ColumnIdentifier name, AbstractType<?> type) public Builder addClusteringColumn(ColumnIdentifier name, AbstractType<?> type, @Nullable ColumnMask mask) { - return addColumn(new ColumnMetadata(keyspace, this.name, name, type, clusteringColumns.size(), ColumnMetadata.Kind.CLUSTERING, mask)); + return addClusteringColumn(name, type, mask, ColumnConstraints.NO_OP); + } + + public Builder addClusteringColumn(ColumnIdentifier name, AbstractType<?> type, @Nullable ColumnMask mask, @Nonnull ColumnConstraints cqlConstraints) + { + return addColumn(new ColumnMetadata(keyspace, this.name, name, type, clusteringColumns.size(), ColumnMetadata.Kind.CLUSTERING, mask, cqlConstraints)); } public Builder addRegularColumn(String name, AbstractType<?> type) @@ -1059,7 +1108,12 @@ public Builder addRegularColumn(ColumnIdentifier name, AbstractType<?> type) public Builder addRegularColumn(ColumnIdentifier name, AbstractType<?> type, @Nullable ColumnMask mask) { - return addColumn(new ColumnMetadata(keyspace, this.name, name, type, ColumnMetadata.NO_POSITION, ColumnMetadata.Kind.REGULAR, mask)); + return addRegularColumn(name, type, mask, ColumnConstraints.NO_OP); + } + + public Builder addRegularColumn(ColumnIdentifier name, AbstractType<?> type, @Nullable ColumnMask mask, @Nonnull ColumnConstraints cqlConstraints) + { + return addColumn(new ColumnMetadata(keyspace, this.name, name, type, ColumnMetadata.NO_POSITION, ColumnMetadata.Kind.REGULAR, mask, cqlConstraints)); } public Builder addStaticColumn(String name, AbstractType<?> type) @@ -1079,7 +1133,12 @@ public Builder addStaticColumn(ColumnIdentifier name, AbstractType<?> type) public Builder addStaticColumn(ColumnIdentifier name, AbstractType<?> type, @Nullable ColumnMask mask) { - return addColumn(new ColumnMetadata(keyspace, this.name, name, type, ColumnMetadata.NO_POSITION, ColumnMetadata.Kind.STATIC, mask)); + return addStaticColumn(name, type, mask, ColumnConstraints.NO_OP); + } + + public Builder addStaticColumn(ColumnIdentifier name, AbstractType<?> type, @Nullable ColumnMask mask, @Nonnull ColumnConstraints cqlConstraints) + { + return addColumn(new ColumnMetadata(keyspace, this.name, name, type, ColumnMetadata.NO_POSITION, ColumnMetadata.Kind.STATIC, mask, cqlConstraints)); } public Builder addColumn(ColumnMetadata column) @@ -1376,8 +1435,7 @@ public void appendCqlTo(CqlBuilder builder, if (!hasSingleColumnPrimaryKey) appendPrimaryKey(builder); - builder.decreaseIndent() - .append(')'); + builder.decreaseIndent().append(')'); builder.append(" WITH ") .increaseIndent(); @@ -1461,8 +1519,7 @@ void appendPrimaryKey(CqlBuilder builder) builder.append(", ") .appendWithSeparators(clusteringColumns, (b, c) -> b.append(c.name), ", "); - builder.append(')') - .newLine(); + builder.append(')').newLine(); } void appendTableOptions(CqlBuilder builder, boolean withInternals) @@ -1682,7 +1739,7 @@ else if (isStaticCompactTable) for (ColumnMetadata c : regularAndStaticColumns) { if (c.isStatic()) - columns.add(new ColumnMetadata(c.ksName, c.cfName, c.name, c.type, -1, ColumnMetadata.Kind.REGULAR, c.getMask())); + columns.add(new ColumnMetadata(c.ksName, c.cfName, c.name, c.type, -1, ColumnMetadata.Kind.REGULAR, c.getMask(), c.getColumnConstraints())); } otherColumns = columns.iterator(); } @@ -1900,7 +1957,6 @@ public long serializedSize(TableMetadata t, Version version) size += Triggers.serializer.serializedSize(t.triggers, version); return size; - } } -} \ No newline at end of file +} diff --git a/src/java/org/apache/cassandra/tcm/membership/NodeVersion.java b/src/java/org/apache/cassandra/tcm/membership/NodeVersion.java index 88d529635202..e5b608994aa0 100644 --- a/src/java/org/apache/cassandra/tcm/membership/NodeVersion.java +++ b/src/java/org/apache/cassandra/tcm/membership/NodeVersion.java @@ -34,7 +34,7 @@ public class NodeVersion implements Comparable<NodeVersion> { public static final Serializer serializer = new Serializer(); - public static final Version CURRENT_METADATA_VERSION = Version.V5; + public static final Version CURRENT_METADATA_VERSION = Version.V6; public static final NodeVersion CURRENT = new NodeVersion(new CassandraVersion(FBUtilities.getReleaseVersionString()), CURRENT_METADATA_VERSION); private static final CassandraVersion SINCE_VERSION = CassandraVersion.CASSANDRA_5_0; diff --git a/src/java/org/apache/cassandra/tcm/serialization/Version.java b/src/java/org/apache/cassandra/tcm/serialization/Version.java index 55bbbfb04e79..50e1792e2374 100644 --- a/src/java/org/apache/cassandra/tcm/serialization/Version.java +++ b/src/java/org/apache/cassandra/tcm/serialization/Version.java @@ -51,6 +51,10 @@ public enum Version * - PreInitialize includes datacenter (affects local serialization on first CMS node only) */ V5(5), + /** + * CEP-42 - Constraints framework. New version due to modifications in table metadata serialization. + */ + V6(6), UNKNOWN(Integer.MAX_VALUE); diff --git a/test/distributed/org/apache/cassandra/distributed/test/ColumnConstraintsTest.java b/test/distributed/org/apache/cassandra/distributed/test/ColumnConstraintsTest.java new file mode 100644 index 000000000000..7d2aeed896fe --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/ColumnConstraintsTest.java @@ -0,0 +1,302 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import java.io.IOException; +import java.util.Map; +import java.util.Set; + +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.junit.Test; + +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.assertj.core.api.Assertions; +import org.assertj.core.api.Condition; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public class ColumnConstraintsTest extends TestBaseImpl +{ + final static Map<String, String> RELATIONS_MAP = Map.of("st", "<", + "set", "<=", + "et", "=", + "net", "!=", + "bt", ">", + "bet", ">="); + + @Test + public void testInvalidConstraintsExceptions() throws IOException + { + final String tableName = KEYSPACE + ".tbl1"; + + try (Cluster cluster = init(Cluster.build(3).start())) + { + assertThrowsInvalidConstraintException(cluster, String.format("CREATE TABLE %s (pk int, ck1 text CHECK ck1 < 100, ck2 int, v int, " + + "PRIMARY KEY ((pk), ck1, ck2));", tableName), + "ck1 is not a number"); + + assertThrowsInvalidConstraintException(cluster, String.format("CREATE TABLE %s (pk int, ck1 int CHECK LENGTH(ck1) < 100, ck2 int, v int, " + + "PRIMARY KEY ((pk), ck1, ck2));", tableName), + "Column should be of type class org.apache.cassandra.db.marshal.UTF8Type or " + + "class org.apache.cassandra.db.marshal.AsciiType but got class org.apache.cassandra.db.marshal.Int32Type"); + } + } + + @Test + public void testUpdateConstraint() throws IOException + { + final String tableName = KEYSPACE + ".tbl1"; + + try (Cluster cluster = init(Cluster.build(3).start())) + { + String createTableStatement = "CREATE TABLE %s (pk int, ck1 int CHECK ck1 < 100, ck2 int, v int, PRIMARY KEY ((pk), ck1, ck2));"; + cluster.schemaChange(String.format(createTableStatement, tableName)); + + String insertStatement = "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 200, 3)"; + + cluster.coordinator(1).execute(String.format("ALTER TABLE %s ALTER ck2 CHECK ck2 < 100", tableName), ConsistencyLevel.ALL); + + // Can't insert + assertThrowsConstraintViolationException(cluster, + String.format(insertStatement, tableName), + "ck1 value length should be smaller than 100"); + + cluster.coordinator(1).execute(String.format("ALTER TABLE %s ALTER ck2 DROP CHECK", tableName), ConsistencyLevel.ALL); + + // Can insert after droping the constraint + cluster.coordinator(1).execute(String.format(insertStatement, tableName), ConsistencyLevel.ALL); + } + } + + @Test + public void testConstraintWithJsonInsert() throws IOException + { + final String tableName = KEYSPACE + ".tbl1"; + + try (Cluster cluster = init(Cluster.build(3).start())) + { + String createTableStatement = "CREATE TABLE %s (pk int, ck1 int CHECK ck1 < 100, ck2 int, v uuid, PRIMARY KEY ((pk), ck1, ck2));"; + cluster.schemaChange(String.format(createTableStatement, tableName)); + + cluster.coordinator(1).execute(String.format("INSERT INTO %s JSON '{\"pk\" : 1, \"ck1\" : 2, \"ck2\" : 2, \"v\" : \"ac064e40-0417-4a4a-bf53-b7cf145afdc2\" }'", tableName), ConsistencyLevel.ALL); + + assertThrowsConstraintViolationException(cluster, + String.format("INSERT INTO %s JSON '{\"pk\" : 1, \"ck1\" : 200, \"ck2\" : 2, \"v\" : \"ac064e40-0417-4a4a-bf53-b7cf145afdc2\" }'", tableName), + "ck1 value length should be smaller than 100"); + + assertThrowsConstraintViolationException(cluster, + String.format("INSERT INTO %s JSON '{\"pk\" : 1, \"ck1\": 100, \"ck2\" : 2, \"v\" : \"ac064e40-0417-4a4a-bf53-b7cf145afdc2\" }'", tableName), + "ck1 value length should be smaller than 100"); + } + } + + @Test + public void testScalarTableLevelConstraint() throws IOException + { + Set<String> typesSet = Set.of("int", "double", "float", "decimal"); + + try (Cluster cluster = init(Cluster.build(3).start())) + { + // Create tables + for (String type : typesSet) + { + for (Map.Entry<String, String> relation : RELATIONS_MAP.entrySet()) + { + String tableName = String.format(KEYSPACE + ".%s_tbl1_%s", type, relation.getKey()); + String createTableStatementSmallerThan = "CREATE TABLE " + tableName + " (pk int, ck1 " + type + " CHECK ck1 " + relation.getValue() + " 100, ck2 int, v int, PRIMARY KEY ((pk), ck1, ck2));"; + cluster.schemaChange(createTableStatementSmallerThan); + } + } + + for (String type : typesSet) + { + // st + String tableName = String.format(KEYSPACE + ".%s_tbl1_%s", type, "st"); + + cluster.coordinator(1).execute(String.format("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 2, 3)", tableName), ConsistencyLevel.ALL); + + assertThrowsConstraintViolationException(cluster, + String.format("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 200, 2, 3)", tableName), + "ck1 value should be smaller than 100.0"); + + assertThrowsConstraintViolationException(cluster, + String.format("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 100, 2, 3)", tableName), + "ck1 value length be smaller than 100.0"); + + // set + tableName = String.format(KEYSPACE + ".%s_tbl1_%s", type, "set"); + + cluster.coordinator(1).execute(String.format("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 2, 3)", tableName), ConsistencyLevel.ALL); + cluster.coordinator(1).execute(String.format("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 100, 2, 3)", tableName), ConsistencyLevel.ALL); + + assertThrowsConstraintViolationException(cluster, + String.format("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 200, 2, 3)", tableName), + "ck1 value should be smaller than 100.0"); + + // et + tableName = String.format(KEYSPACE + ".%s_tbl1_%s", type, "et"); + + cluster.coordinator(1).execute(String.format("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 100, 2, 3)", tableName), ConsistencyLevel.ALL); + + assertThrowsConstraintViolationException(cluster, + String.format("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 200, 2, 3)", tableName), + "ck1 value should be smaller than 100.0"); + + // net + tableName = String.format(KEYSPACE + ".%s_tbl1_%s", type, "net"); + + cluster.coordinator(1).execute(String.format("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 200, 2, 3)", tableName), ConsistencyLevel.ALL); + + assertThrowsConstraintViolationException(cluster, + String.format("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 100, 2, 3)", tableName), + "ck1 value should be smaller than 100.0"); + + // bt + tableName = String.format(KEYSPACE + ".%s_tbl1_%s", type, "bt"); + + cluster.coordinator(1).execute(String.format("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 200, 2, 3)", tableName), ConsistencyLevel.ALL); + + assertThrowsConstraintViolationException(cluster, + String.format("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 1, 2, 3)", tableName), + "ck1 value should be smaller than 100.0"); + assertThrowsConstraintViolationException(cluster, + String.format("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 100, 2, 3)", tableName), + "ck1 value should be smaller than 100.0"); + + // bet + tableName = String.format(KEYSPACE + ".%s_tbl1_%s", type, "bet"); + + cluster.coordinator(1).execute(String.format("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 200, 2, 3)", tableName), ConsistencyLevel.ALL); + cluster.coordinator(1).execute(String.format("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 100, 2, 3)", tableName), ConsistencyLevel.ALL); + + assertThrowsConstraintViolationException(cluster, + String.format("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 1, 2, 3)", tableName), + "ck1 value should be smaller than 100.0"); + + } + } + } + + @Test + public void testLengthTableLevelConstraint() throws IOException + { + Set<String> typesSet = Set.of("varchar", "text", "blob", "ascii"); + + try (Cluster cluster = init(Cluster.build(3).start())) + { + // Create tables + for (String type : typesSet) + { + for (Map.Entry<String, String> relation : RELATIONS_MAP.entrySet()) + { + String tableName = String.format(KEYSPACE + ".%s_tbl1_%s", type, relation.getKey()); + String createTableStatementSmallerThan = "CREATE TABLE " + tableName + " (pk " + type + " CHECK LENGTH(pk) " + relation.getValue() + " 4, ck1 int, ck2 int, v int, PRIMARY KEY ((pk), ck1, ck2));"; + cluster.schemaChange(createTableStatementSmallerThan); + } + } + + for (String type : typesSet) + { + String value = "'%s'"; + if (type.equals("blob")) + value = "textAsBlob('%s')"; + // st + String tableName = String.format(KEYSPACE + ".%s_tbl1_%s", type, "st"); + cluster.coordinator(1).execute(String.format("INSERT INTO " + tableName + " (pk, ck1, ck2, v) VALUES (" + value + ", 2, 2, 3)", "foo"), ConsistencyLevel.ALL); + + assertThrowsConstraintViolationException(cluster, + String.format("INSERT INTO " + tableName + " (pk, ck1, ck2, v) VALUES (" + value + ", 200, 2, 3)", "fooo"), + "ck1 value length should be smaller than 100"); + + assertThrowsConstraintViolationException(cluster, + String.format("INSERT INTO " + tableName + " (pk, ck1, ck2, v) VALUES (" + value + ", 100, 2, 3)", "foooo"), + "ck1 value length should be smaller than 100"); + + // set + tableName = String.format(KEYSPACE + ".%s_tbl1_%s", type, "set"); + cluster.coordinator(1).execute(String.format("INSERT INTO " + tableName + " (pk, ck1, ck2, v) VALUES (" + value + ", 2, 2, 3)", "foo"), ConsistencyLevel.ALL); + cluster.coordinator(1).execute(String.format("INSERT INTO " + tableName + " (pk, ck1, ck2, v) VALUES (" + value + ", 2, 2, 3)", "fooo"), ConsistencyLevel.ALL); + + assertThrowsConstraintViolationException(cluster, + String.format("INSERT INTO " + tableName + " (pk, ck1, ck2, v) VALUES (" + value + ", 100, 2, 3)", "foooo"), + "ck1 value length should be smaller than 100"); + + // et + tableName = String.format(KEYSPACE + ".%s_tbl1_%s", type, "et"); + cluster.coordinator(1).execute(String.format("INSERT INTO " + tableName + " (pk, ck1, ck2, v) VALUES (" + value + ", 2, 2, 3)", "fooo"), ConsistencyLevel.ALL); + + assertThrowsConstraintViolationException(cluster, + String.format("INSERT INTO " + tableName + " (pk, ck1, ck2, v) VALUES (" + value + ", 100, 2, 3)", "foooo"), + "ck1 value length should be smaller than 100"); + + assertThrowsConstraintViolationException(cluster, + String.format("INSERT INTO " + tableName + " (pk, ck1, ck2, v) VALUES (" + value + ", 100, 2, 3)", "fo"), + "ck1 value length should be smaller than 100"); + + // net + tableName = String.format(KEYSPACE + ".%s_tbl1_%s", type, "net"); + cluster.coordinator(1).execute(String.format("INSERT INTO " + tableName + " (pk, ck1, ck2, v) VALUES (" + value + ", 2, 2, 3)", "foooo"), ConsistencyLevel.ALL); + cluster.coordinator(1).execute(String.format("INSERT INTO " + tableName + " (pk, ck1, ck2, v) VALUES (" + value + ", 2, 2, 3)", "foo"), ConsistencyLevel.ALL); + + assertThrowsConstraintViolationException(cluster, + String.format("INSERT INTO " + tableName + " (pk, ck1, ck2, v) VALUES (" + value + ", 100, 2, 3)", "fooo"), + "ck1 value length should be smaller than 100"); + + // bt + tableName = String.format(KEYSPACE + ".%s_tbl1_%s", type, "bt"); + cluster.coordinator(1).execute(String.format("INSERT INTO " + tableName + " (pk, ck1, ck2, v) VALUES (" + value + ", 2, 2, 3)", "foooo"), ConsistencyLevel.ALL); + + assertThrowsConstraintViolationException(cluster, + String.format("INSERT INTO " + tableName + " (pk, ck1, ck2, v) VALUES (" + value + ", 100, 2, 3)", "fooo"), + "ck1 value length should be smaller than 100"); + + assertThrowsConstraintViolationException(cluster, + String.format("INSERT INTO " + tableName + " (pk, ck1, ck2, v) VALUES (" + value + ", 100, 2, 3)", "foo"), + "ck1 value length should be smaller than 100"); + + // bet + tableName = String.format(KEYSPACE + ".%s_tbl1_%s", type, "bet"); + cluster.coordinator(1).execute(String.format("INSERT INTO " + tableName + " (pk, ck1, ck2, v) VALUES (" + value + ", 2, 2, 3)", "foooo"), ConsistencyLevel.ALL); + cluster.coordinator(1).execute(String.format("INSERT INTO " + tableName + " (pk, ck1, ck2, v) VALUES (" + value + ", 2, 2, 3)", "fooo"), ConsistencyLevel.ALL); + + assertThrowsConstraintViolationException(cluster, + String.format("INSERT INTO " + tableName + " (pk, ck1, ck2, v) VALUES (" + value + ", 100, 2, 3)", "foo"), + "ck1 value length should be smaller than 100"); + } + } + } + + private void assertThrowsConstraintViolationException(Cluster cluster, String statement, String description) + { + Assertions.assertThatThrownBy(() -> cluster.coordinator(1).execute(statement, ConsistencyLevel.ALL)) + .describedAs(description) + .has(new Condition<Throwable>(t -> t.getClass().getCanonicalName() + .equals(InvalidRequestException.class.getCanonicalName()), description)); + } + + private void assertThrowsInvalidConstraintException(Cluster cluster, String statement, String description) + { + Assertions.setMaxStackTraceElementsDisplayed(100); + assertThatThrownBy(() -> cluster.schemaChange(statement)) + .describedAs(description) + .has(new Condition<Throwable>(t -> t.getClass().getCanonicalName() + .equals(InvalidRequestException.class.getCanonicalName()), description)); + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/cdc/ToggleCDCOnRepair.java b/test/distributed/org/apache/cassandra/distributed/test/cdc/ToggleCDCOnRepair.java new file mode 100644 index 000000000000..0fd09c682892 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/cdc/ToggleCDCOnRepair.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.cdc; + +import java.util.function.Consumer; + + +import org.apache.cassandra.db.commitlog.CommitLog; +import org.apache.cassandra.db.commitlog.CommitLogSegment; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.test.TestBaseImpl; + +import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows; +import static org.apache.cassandra.distributed.shared.AssertUtils.assertTrue; +import static org.apache.cassandra.distributed.shared.AssertUtils.row; + +public abstract class ToggleCDCOnRepair extends TestBaseImpl +{ + Consumer<Cluster> getRepairEnabledRepairAssertion() + { + return cluster -> { + cluster.get(2).runOnInstance(() -> { + boolean containCDCInLog = CommitLog.instance.segmentManager + .getActiveSegments() + .stream() + .anyMatch(s -> s.getCDCState() == CommitLogSegment.CDCState.CONTAINS); + assertTrue("Mutation should be added to commit log when cdc_on_repair_enabled is true", + containCDCInLog); + }); + }; + } + + Consumer<Cluster> getRepairDisabledRepairAssertion() + { + return cluster -> { + cluster.get(2).runOnInstance(() -> { + boolean containCDCInLog = CommitLog.instance.segmentManager + .getActiveSegments() + .stream() + .allMatch(s -> s.getCDCState() != CommitLogSegment.CDCState.CONTAINS); + assertTrue("No mutation should be added to commit log when cdc_on_repair_enabled is false", + containCDCInLog); + }); + }; + } + + // test helper to repair data between nodes when cdc_on_repair_enabled is on or off. + void testCDCOnRepairEnabled(boolean enabled, Consumer<Cluster> assertion, boolean constraintsEnabled) throws Exception + { + try (Cluster cluster = init(Cluster.build(2) + .withConfig(c -> c.set("cdc_enabled", true) + .set("cdc_on_repair_enabled", enabled) + .with(Feature.NETWORK) + .with(Feature.GOSSIP)) + .start())) + { + cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (k INT PRIMARY KEY, v INT) WITH cdc=true")); + + // Data only in node1 + cluster.get(1).executeInternal(withKeyspace("INSERT INTO %s.tbl (k, v) VALUES (1, 1)")); + + if (constraintsEnabled) + cluster.schemaChange(withKeyspace("ALTER TABLE %s.tbl ALTER v CHECK v != 1")); + + Object[][] result = cluster.get(1).executeInternal(withKeyspace("SELECT * FROM %s.tbl WHERE k = 1")); + assertRows(result, row(1, 1)); + result = cluster.get(2).executeInternal(withKeyspace("SELECT * FROM %s.tbl WHERE k = 1")); + assertRows(result); + + // repair + cluster.get(1).flush(KEYSPACE); + cluster.get(2).nodetool("repair", KEYSPACE, "tbl"); + + // verify node2 now have data + result = cluster.get(2).executeInternal(withKeyspace("SELECT * FROM %s.tbl WHERE k = 1")); + assertRows(result, row(1, 1)); + + assertion.accept(cluster); + } + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/cdc/ToggleCDCOnRepairEnabledTest.java b/test/distributed/org/apache/cassandra/distributed/test/cdc/ToggleCDCOnRepairEnabledTest.java index 499cf076afa0..63aa4c8e25af 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/cdc/ToggleCDCOnRepairEnabledTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/cdc/ToggleCDCOnRepairEnabledTest.java @@ -18,80 +18,21 @@ package org.apache.cassandra.distributed.test.cdc; -import java.util.function.Consumer; import org.junit.Test; -import org.apache.cassandra.db.commitlog.CommitLog; -import org.apache.cassandra.db.commitlog.CommitLogSegment; -import org.apache.cassandra.distributed.Cluster; -import org.apache.cassandra.distributed.api.Feature; -import org.apache.cassandra.distributed.test.TestBaseImpl; -import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows; -import static org.apache.cassandra.distributed.shared.AssertUtils.assertTrue; -import static org.apache.cassandra.distributed.shared.AssertUtils.row; - -public class ToggleCDCOnRepairEnabledTest extends TestBaseImpl +public class ToggleCDCOnRepairEnabledTest extends ToggleCDCOnRepair { @Test public void testCDCOnRepairIsEnabled() throws Exception { - testCDCOnRepairEnabled(true, cluster -> { - cluster.get(2).runOnInstance(() -> { - boolean containCDCInLog = CommitLog.instance.segmentManager - .getActiveSegments() - .stream() - .anyMatch(s -> s.getCDCState() == CommitLogSegment.CDCState.CONTAINS); - assertTrue("Mutation should be added to commit log when cdc_on_repair_enabled is true", - containCDCInLog); - }); - }); + testCDCOnRepairEnabled(true, getRepairEnabledRepairAssertion(), false); } @Test public void testCDCOnRepairIsDisabled() throws Exception { - testCDCOnRepairEnabled(false, cluster -> { - cluster.get(2).runOnInstance(() -> { - boolean containCDCInLog = CommitLog.instance.segmentManager - .getActiveSegments() - .stream() - .allMatch(s -> s.getCDCState() != CommitLogSegment.CDCState.CONTAINS); - assertTrue("No mutation should be added to commit log when cdc_on_repair_enabled is false", - containCDCInLog); - }); - }); - } - - // test helper to repair data between nodes when cdc_on_repair_enabled is on or off. - private void testCDCOnRepairEnabled(boolean enabled, Consumer<Cluster> assertion) throws Exception - { - try (Cluster cluster = init(Cluster.build(2) - .withConfig(c -> c.set("cdc_enabled", true) - .set("cdc_on_repair_enabled", enabled) - .with(Feature.NETWORK) - .with(Feature.GOSSIP)) - .start())) - { - cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl (k INT PRIMARY KEY, v INT) WITH cdc=true")); - - // Data only in node1 - cluster.get(1).executeInternal(withKeyspace("INSERT INTO %s.tbl (k, v) VALUES (1, 1)")); - Object[][] result = cluster.get(1).executeInternal(withKeyspace("SELECT * FROM %s.tbl WHERE k = 1")); - assertRows(result, row(1, 1)); - result = cluster.get(2).executeInternal(withKeyspace("SELECT * FROM %s.tbl WHERE k = 1")); - assertRows(result); - - // repair - cluster.get(1).flush(KEYSPACE); - cluster.get(2).nodetool("repair", KEYSPACE, "tbl"); - - // verify node2 now have data - result = cluster.get(2).executeInternal(withKeyspace("SELECT * FROM %s.tbl WHERE k = 1")); - assertRows(result, row(1, 1)); - - assertion.accept(cluster); - } + testCDCOnRepairEnabled(false, getRepairDisabledRepairAssertion(), false); } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/cdc/ToggleCDCWithConstraintsOnRepairEnabledTest.java b/test/distributed/org/apache/cassandra/distributed/test/cdc/ToggleCDCWithConstraintsOnRepairEnabledTest.java new file mode 100644 index 000000000000..3697209df77e --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/cdc/ToggleCDCWithConstraintsOnRepairEnabledTest.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.cdc; + + +import org.junit.Test; + + +public class ToggleCDCWithConstraintsOnRepairEnabledTest extends ToggleCDCOnRepair +{ + @Test + public void testCDCWithConstraintsOnRepairIsEnabled() throws Exception + { + testCDCOnRepairEnabled(true, getRepairEnabledRepairAssertion(), true); + } + + @Test + public void testCDCWithConstraintsOnRepairIsDisabled() throws Exception + { + testCDCOnRepairEnabled(false, getRepairDisabledRepairAssertion(), true); + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/SnapshotTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/SnapshotTest.java index 8e424e0bc5c0..775e7ffe7956 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/SnapshotTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/SnapshotTest.java @@ -51,6 +51,7 @@ public void testSimpleSnapshot() throws Throwable .start())) { cluster.schemaChange(withKeyspace("create table %s.tbl (id int primary key, x int)")); + cluster.schemaChange(withKeyspace("create table %s.tblconstraints (id int primary key, x int check x > 100 and x < 200, v text check LENGTH(v) > 10)")); cluster.schemaChange(withKeyspace("CREATE OR REPLACE FUNCTION %s.fLog (input double) CALLED ON NULL INPUT RETURNS double LANGUAGE java AS 'return Double.valueOf(Math.log(input.doubleValue()));';")); cluster.schemaChange(withKeyspace("CREATE OR REPLACE FUNCTION %s.avgState ( state tuple<int,bigint>, val int ) CALLED ON NULL INPUT RETURNS tuple<int,bigint> LANGUAGE java AS \n" + " 'if (val !=null) { state.setInt(0, state.getInt(0)+1); state.setLong(1, state.getLong(1)+val.intValue()); } return state;'; ")); diff --git a/test/unit/org/apache/cassandra/contraints/AlterTableWithTableConstraintValidationTest.java b/test/unit/org/apache/cassandra/contraints/AlterTableWithTableConstraintValidationTest.java new file mode 100644 index 000000000000..0229d87e25dc --- /dev/null +++ b/test/unit/org/apache/cassandra/contraints/AlterTableWithTableConstraintValidationTest.java @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.contraints; + +import org.junit.Test; + + +public class AlterTableWithTableConstraintValidationTest extends CqlConstraintValidationTester +{ + + @Test + public void testCreateTableWithColumnNamedConstraintDescribeTableNonFunction() throws Throwable + { + String table = createTable("CREATE TABLE %s (pk int, ck1 int CHECK ck1 < 100, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + + execute("ALTER TABLE %s ALTER ck1 DROP CHECK"); + + String tableCreateStatement = "CREATE TABLE " + KEYSPACE + "." + table + " (\n" + + " pk int,\n" + + " ck1 int,\n" + + " ck2 int,\n" + + " v int,\n" + + " PRIMARY KEY (pk, ck1, ck2)\n" + + ") WITH CLUSTERING ORDER BY (ck1 ASC, ck2 ASC)\n" + + " AND " + tableParametersCql(); + + assertRowsNet(executeDescribeNet(KEYSPACE, "DESCRIBE TABLE " + KEYSPACE + "." + table), + row(KEYSPACE, + "table", + table, + tableCreateStatement)); + } + + @Test + public void testCreateTableAddConstraint() throws Throwable + { + String table = createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + + execute("ALTER TABLE %s ALTER ck1 CHECK ck1 < 100 AND ck1 > 10"); + + String tableCreateStatement = "CREATE TABLE " + KEYSPACE + "." + table + " (\n" + + " pk int,\n" + + " ck1 int CHECK ck1 < 100 AND ck1 > 10,\n" + + " ck2 int,\n" + + " v int,\n" + + " PRIMARY KEY (pk, ck1, ck2)\n" + + ") WITH CLUSTERING ORDER BY (ck1 ASC, ck2 ASC)\n" + + " AND " + tableParametersCql(); + + assertRowsNet(executeDescribeNet(KEYSPACE, "DESCRIBE TABLE " + KEYSPACE + "." + table), + row(KEYSPACE, + "table", + table, + tableCreateStatement)); + } + + @Test + public void testCreateTableAddMultipleConstraints() throws Throwable + { + String table = createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + + execute("ALTER TABLE %s ALTER ck1 CHECK ck1 < 100"); + execute("ALTER TABLE %s ALTER ck2 CHECK ck2 > 10"); + + String tableCreateStatement = "CREATE TABLE " + KEYSPACE + "." + table + " (\n" + + " pk int,\n" + + " ck1 int CHECK ck1 < 100,\n" + + " ck2 int CHECK ck2 > 10,\n" + + " v int,\n" + + " PRIMARY KEY (pk, ck1, ck2)\n" + + ") WITH CLUSTERING ORDER BY (ck1 ASC, ck2 ASC)\n" + + " AND " + tableParametersCql(); + + assertRowsNet(executeDescribeNet(KEYSPACE, "DESCRIBE TABLE " + KEYSPACE + "." + table), + row(KEYSPACE, + "table", + table, + tableCreateStatement)); + } + + @Test + public void testCreateTableAddMultipleMixedConstraints() throws Throwable + { + String table = createTable("CREATE TABLE %s (pk int, ck1 int, ck2 text, v int, PRIMARY KEY ((pk), ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + + execute("ALTER TABLE %s ALTER ck1 CHECK ck1 < 100"); + + String tableCreateStatement = "CREATE TABLE " + KEYSPACE + "." + table + " (\n" + + " pk int,\n" + + " ck1 int CHECK ck1 < 100,\n" + + " ck2 text,\n" + + " v int,\n" + + " PRIMARY KEY (pk, ck1, ck2)\n" + + ") WITH CLUSTERING ORDER BY (ck1 ASC, ck2 ASC)\n" + + " AND " + tableParametersCql(); + + assertRowsNet(executeDescribeNet(KEYSPACE, "DESCRIBE TABLE " + KEYSPACE + "." + table), + row(KEYSPACE, + "table", + table, + tableCreateStatement)); + + execute("ALTER TABLE %s ALTER ck2 CHECK LENGTH(ck2) = 4"); + + tableCreateStatement = "CREATE TABLE " + KEYSPACE + "." + table + " (\n" + + " pk int,\n" + + " ck1 int CHECK ck1 < 100,\n" + + " ck2 text CHECK LENGTH(ck2) = 4,\n" + + " v int,\n" + + " PRIMARY KEY (pk, ck1, ck2)\n" + + ") WITH CLUSTERING ORDER BY (ck1 ASC, ck2 ASC)\n" + + " AND " + tableParametersCql(); + + assertRowsNet(executeDescribeNet(KEYSPACE, "DESCRIBE TABLE " + KEYSPACE + "." + table), + row(KEYSPACE, + "table", + table, + tableCreateStatement)); + } + + @Test + public void testCreateTableAddAndRemoveConstraint() throws Throwable + { + String table = createTable("CREATE TABLE %s (pk int, ck1 int, ck2 text, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + + execute("ALTER TABLE %s ALTER ck1 CHECK ck1 < 100"); + + String tableCreateStatement = "CREATE TABLE " + KEYSPACE + "." + table + " (\n" + + " pk int,\n" + + " ck1 int CHECK ck1 < 100,\n" + + " ck2 text,\n" + + " v int,\n" + + " PRIMARY KEY (pk, ck1, ck2)\n" + + ") WITH CLUSTERING ORDER BY (ck1 ASC, ck2 ASC)\n" + + " AND " + tableParametersCql(); + + assertRowsNet(executeDescribeNet(KEYSPACE, "DESCRIBE TABLE " + KEYSPACE + "." + table), + row(KEYSPACE, + "table", + table, + tableCreateStatement)); + + execute("ALTER TABLE %s ALTER ck1 DROP CHECK"); + + String tableCreateStatement2 = "CREATE TABLE " + KEYSPACE + "." + table + " (\n" + + " pk int,\n" + + " ck1 int,\n" + + " ck2 text,\n" + + " v int,\n" + + " PRIMARY KEY (pk, ck1, ck2)\n" + + ") WITH CLUSTERING ORDER BY (ck1 ASC, ck2 ASC)\n" + + " AND " + tableParametersCql(); + + assertRowsNet(executeDescribeNet(KEYSPACE, "DESCRIBE TABLE " + KEYSPACE + "." + table), + row(KEYSPACE, + "table", + table, + tableCreateStatement2)); + } + + @Test + public void testAlterWithConstraintsAndCdcEnabled() throws Throwable + { + createTable("CREATE TABLE %s (pk text, ck1 int, ck2 int, PRIMARY KEY ((pk),ck1, ck2)) WITH cdc = true;"); + // It works + execute("ALTER TABLE %s ALTER ck1 CHECK ck1 < 100"); + } + + @Test + public void testAlterWithCdcAndPKConstraintsEnabled() throws Throwable + { + createTable("CREATE TABLE %s (pk text CHECK length(pk) = 100, ck1 int, ck2 int, PRIMARY KEY ((pk), ck1, ck2));"); + // It works + execute("ALTER TABLE %s WITH cdc = true"); + } + + @Test + public void testAlterWithCdcAndRegularConstraintsEnabled() throws Throwable + { + createTable("CREATE TABLE %s (pk text, ck1 int CHECK ck1 < 100, ck2 int, PRIMARY KEY (pk));"); + // It works + execute("ALTER TABLE %s WITH cdc = true"); + } + + @Test + public void testAlterWithCdcAndClusteringConstraintsEnabled() throws Throwable + { + createTable("CREATE TABLE %s (pk text, ck1 int CHECK ck1 < 100, ck2 int, PRIMARY KEY ((pk), ck1, ck2));"); + // It works + execute("ALTER TABLE %s WITH cdc = true"); + } +} diff --git a/test/unit/org/apache/cassandra/contraints/ColumnConstraintsTest.java b/test/unit/org/apache/cassandra/contraints/ColumnConstraintsTest.java new file mode 100644 index 000000000000..45a631cca609 --- /dev/null +++ b/test/unit/org/apache/cassandra/contraints/ColumnConstraintsTest.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.contraints; + +import org.junit.Test; + +import org.apache.cassandra.cql3.constraints.ColumnConstraint.ConstraintType; + +import static org.apache.cassandra.cql3.constraints.ColumnConstraint.ConstraintType.COMPOSED; +import static org.apache.cassandra.cql3.constraints.ColumnConstraint.ConstraintType.FUNCTION; +import static org.apache.cassandra.cql3.constraints.ColumnConstraint.ConstraintType.SCALAR; +import static org.junit.Assert.assertEquals; + +public class ColumnConstraintsTest +{ + private static final ConstraintType[] EXPECTED_VALUES = { COMPOSED, FUNCTION, SCALAR }; + + @Test + public void testEnumCodesAndNames() + { + ConstraintType[] values = ConstraintType.values(); + + for (int i = 0; i < values.length; i++) + { + assertEquals("Column Constraint Serializer mismatch in the enum " + values[i], + EXPECTED_VALUES[i].name(), values[i].name()); + assertEquals("Column Constraint Serializer mismatch in the enum for value " + values[i], + ConstraintType.getSerializer(EXPECTED_VALUES[i].ordinal()), ConstraintType.getSerializer(i)); + } + + assertEquals("Column Constraint Serializer enum constants has changed. Update the test.", + EXPECTED_VALUES.length, values.length); + } +} diff --git a/test/unit/org/apache/cassandra/contraints/CqlConstraintValidationTester.java b/test/unit/org/apache/cassandra/contraints/CqlConstraintValidationTester.java new file mode 100644 index 000000000000..f2bc3dd1b464 --- /dev/null +++ b/test/unit/org/apache/cassandra/contraints/CqlConstraintValidationTester.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.contraints; + +import java.util.Map; + +import com.datastax.driver.core.ResultSet; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.cql3.CqlBuilder; +import org.apache.cassandra.schema.CompactionParams; +import org.apache.cassandra.transport.ProtocolVersion; + +public abstract class CqlConstraintValidationTester extends CQLTester +{ + ResultSet executeDescribeNet(String cql) throws Throwable + { + return executeDescribeNet(null, cql); + } + + ResultSet executeDescribeNet(String useKs, String cql) throws Throwable + { + return executeNetWithPaging(getProtocolVersion(useKs), cql, useKs, 3); + } + + private ProtocolVersion getProtocolVersion(String useKs) throws Throwable + { + // We're using a trick here to distinguish driver sessions with a "USE keyspace" and without: + // As different ProtocolVersions use different driver instances, we use different ProtocolVersions + // for the with and without "USE keyspace" cases. + + ProtocolVersion v = useKs != null ? ProtocolVersion.CURRENT : ProtocolVersion.V6; + + if (useKs != null) + executeNet(v, "USE " + useKs); + return v; + } + + static String tableParametersCql() + { + return "additional_write_policy = '99p'\n" + + " AND allow_auto_snapshot = true\n" + + " AND bloom_filter_fp_chance = 0.01\n" + + " AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'}\n" + + " AND cdc = false\n" + + " AND comment = ''\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 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(); + } +} diff --git a/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnCqlConstraintValidationTest.java b/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnCqlConstraintValidationTest.java new file mode 100644 index 000000000000..97de503d1b6c --- /dev/null +++ b/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnCqlConstraintValidationTest.java @@ -0,0 +1,1264 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.contraints; + + +import java.util.Arrays; +import java.util.Collection; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.utils.Generators; + +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static accord.utils.Property.qt; +import static org.quicktheories.generators.SourceDSL.doubles; +import static org.quicktheories.generators.SourceDSL.integers; + +@RunWith(Parameterized.class) +public class CreateTableWithColumnCqlConstraintValidationTest extends CqlConstraintValidationTester +{ + + @Parameterized.Parameter + public String order; + + @Parameterized.Parameters() + public static Collection<Object[]> generateData() + { + return Arrays.asList(new Object[][]{ + { "ASC" }, + { "DESC" } + }); + } + + @Test + public void testCreateTableWithColumnNotNamedConstraintDescribeTableNonFunction() throws Throwable + { + String table = createTable(KEYSPACE_PER_TEST, "CREATE TABLE %s (pk int, ck1 int CHECK ck1 < 100, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + + String tableCreateStatement = "CREATE TABLE " + KEYSPACE_PER_TEST + "." + table + " (\n" + + " pk int,\n" + + " ck1 int CHECK ck1 < 100,\n" + + " ck2 int,\n" + + " v int,\n" + + " PRIMARY KEY (pk, ck1, ck2)\n" + + ") WITH CLUSTERING ORDER BY (ck1 ASC, ck2 ASC)\n" + + " AND " + tableParametersCql(); + + assertRowsNet(executeDescribeNet("DESCRIBE TABLE " + KEYSPACE_PER_TEST + "." + table), + row(KEYSPACE_PER_TEST, + "table", + table, + tableCreateStatement)); + } + + @Test + public void testCreateTableWithColumnMultipleConstraintsDescribeTableNonFunction() throws Throwable + { + String table = createTable(KEYSPACE_PER_TEST, "CREATE TABLE %s (pk int, ck1 int CHECK ck1 < 100 AND ck1 > 10, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + + String tableCreateStatement = "CREATE TABLE " + KEYSPACE_PER_TEST + "." + table + " (\n" + + " pk int,\n" + + " ck1 int CHECK ck1 < 100 AND ck1 > 10,\n" + + " ck2 int,\n" + + " v int,\n" + + " PRIMARY KEY (pk, ck1, ck2)\n" + + ") WITH CLUSTERING ORDER BY (ck1 ASC, ck2 ASC)\n" + + " AND " + tableParametersCql(); + + assertRowsNet(executeDescribeNet("DESCRIBE TABLE " + KEYSPACE_PER_TEST + "." + table), + row(KEYSPACE_PER_TEST, + "table", + table, + tableCreateStatement)); + } + + @Test + public void testCreateTableWithColumnNotNamedConstraintDescribeTableFunction() throws Throwable + { + String table = createTable(KEYSPACE_PER_TEST, "CREATE TABLE %s (pk int, ck1 text CHECK LENGTH(ck1) = 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + + String tableCreateStatement = "CREATE TABLE " + KEYSPACE_PER_TEST + "." + table + " (\n" + + " pk int,\n" + + " ck1 text CHECK LENGTH(ck1) = 4,\n" + + " ck2 int,\n" + + " v int,\n" + + " PRIMARY KEY (pk, ck1, ck2)\n" + + ") WITH CLUSTERING ORDER BY (ck1 ASC, ck2 ASC)\n" + + " AND " + tableParametersCql(); + + assertRowsNet(executeDescribeNet("DESCRIBE TABLE " + KEYSPACE_PER_TEST + "." + table), + row(KEYSPACE_PER_TEST, + "table", + table, + tableCreateStatement)); + } + + // SCALAR + @Test + public void testCreateTableWithColumnWithClusteringColumnLessThanScalarConstraintInteger() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int CHECK ck1 < 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(integers().between(0, 3))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(integers().between(4, 100))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnBiggerThanScalarConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int CHECK ck1 > 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(integers().between(5, 100))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(integers().between(0, 4))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnBiggerOrEqualThanScalarConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int CHECK ck1 >= 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(integers().between(4, 100))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(integers().between(0, 3))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnLessOrEqualThanScalarConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int CHECK ck1 <= 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(integers().between(0, 4))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(integers().between(5, 100))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnDifferentThanScalarConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int CHECK ck1 != 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(integers().between(0, 3))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + qt().forAll(Generators.toGen(integers().between(5, 100))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 4, 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnMultipleScalarConstraints() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int CHECK ck1 < 4 AND ck1 >= 2, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(integers().between(2, 3))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(integers().between(-100, 1))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + qt().forAll(Generators.toGen(integers().between(4, 100))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnLessThanScalarSmallIntConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 smallint CHECK ck1 < 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(integers().between(0, 3))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(integers().between(4, 100))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnBiggerThanScalarSmallIntConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 smallint CHECK ck1 > 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(integers().between(5, 100))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(integers().between(0, 4))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnBiggerOrEqualThanScalarSmallIntConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 smallint CHECK ck1 >= 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(integers().between(4, 100))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(integers().between(0, 3))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnLessOrEqualThanScalarSmallIntConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 smallint CHECK ck1 <= 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(integers().between(0, 4))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(integers().between(5, 100))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnDifferentThanScalarSmallIntConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 smallint CHECK ck1 != 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(integers().between(0, 3))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + qt().forAll(Generators.toGen(integers().between(5, 100))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 4, 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnMultipleScalarSmallIntConstraints() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 smallint CHECK ck1 < 4 AND ck1 >= 2, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(integers().between(2, 3))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(integers().between(-100, 1))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + qt().forAll(Generators.toGen(integers().between(4, 100))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnLessThanScalarDecimalConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 decimal CHECK ck1 < 4.2, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 2, 3)"); + qt().forAll(Generators.toGen(doubles().between(0, 4.1))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(doubles().between(4.3, 100))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnBiggerThanScalarDecimalConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 decimal CHECK ck1 > 4.2, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(doubles().between(4.3, 100))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(doubles().between(0, 4.2))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnBiggerOrEqualThanScalarDecimalConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 decimal CHECK ck1 >= 4.2, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(doubles().between(4.2, 100))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(doubles().between(0, 4.1))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnLessOrEqualThanScalarDecimalConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 decimal CHECK ck1 <= 4.2, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(doubles().between(0, 4.2))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(doubles().between(4.3, 100))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnDifferentThanScalarDecimalConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 decimal CHECK ck1 != 4.2, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(doubles().between(0, 4.1))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + qt().forAll(Generators.toGen(doubles().between(4.3, 100))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 4.2, 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnMultipleScalarDecimalConstraints() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 decimal CHECK ck1 < 4.2 AND ck1 >= 2.1, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(doubles().between(2.1, 4.1))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(doubles().between(-100, 2))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + qt().forAll(Generators.toGen(doubles().between(4.2, 100))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnLessThanScalarDoubleConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 double CHECK ck1 < 4.2, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 2, 3)"); + qt().forAll(Generators.toGen(doubles().between(0, 4.1))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(doubles().between(4.3, 100))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnBiggerThanScalarDoubleConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 double CHECK ck1 > 4.2, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(doubles().between(4.3, 100))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(doubles().between(0, 4.2))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnBiggerOrEqualThanScalarDoubleConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 double CHECK ck1 >= 4.2, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(doubles().between(4.2, 100))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(doubles().between(0, 4.1))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnLessOrEqualThanScalarDoubleConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 double CHECK ck1 <= 4.2, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(doubles().between(0, 4.2))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(doubles().between(4.3, 100))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnDifferentThanScalarDoubleConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 double CHECK ck1 != 4.2, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(doubles().between(0, 4.1))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + qt().forAll(Generators.toGen(doubles().between(4.3, 100))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 4.2, 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnMultipleScalarDoubleConstraints() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 double CHECK ck1 < 4.2 AND ck1 >= 2.1, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(doubles().between(2.1, 4.1))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(doubles().between(-100, 2))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + qt().forAll(Generators.toGen(doubles().between(4.2, 100))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnLessThanScalarFloatConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 float CHECK ck1 < 4.2, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 2, 3)"); + qt().forAll(Generators.toGen(doubles().between(0, 4.1))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(doubles().between(4.3, 100))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnBiggerThanScalarFloatConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 float CHECK ck1 > 4.2, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(doubles().between(4.3, 100))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(doubles().between(0, 4.2))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnBiggerOrEqualThanScalarFloatConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 float CHECK ck1 >= 4.2, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(doubles().between(4.2, 100))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(doubles().between(0, 4.1))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnLessOrEqualThanScalarFloatConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 float CHECK ck1 <= 4.2, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(doubles().between(0, 4.2))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(doubles().between(4.3, 100))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnDifferentThanScalarFloatConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 float CHECK ck1 != 4.2, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(doubles().between(0, 4.1))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + qt().forAll(Generators.toGen(doubles().between(4.3, 100))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 4.2, 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnMultipleScalarFloatConstraints() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 float CHECK ck1 < 4.2 AND ck1 >= 2.1, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + qt().forAll(Generators.toGen(doubles().between(2.1, 4.1))) + .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(doubles().between(-100, 2))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + qt().forAll(Generators.toGen(doubles().between(4.2, 100))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } + + // FUNCTION + @Test + public void testCreateTableWithColumnWithClusteringColumnLengthEqualToConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 text CHECK LENGTH(ck1) = 4, ck2 int, v int, PRIMARY KEY ((pk), ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fooo', 2, 3)"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foo', 2, 3)"); + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foooo', 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnLengthDifferentThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 text CHECK LENGTH(ck1) != 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foo', 2, 3)"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foooo', 2, 3)"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fooo', 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnLengthBiggerThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 text CHECK LENGTH(ck1) > 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foooo', 2, 3)"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foo', 2, 3)"); + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fooo', 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnLengthBiggerOrEqualThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 text CHECK LENGTH(ck1) >= 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foooo', 2, 3)"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fooo', 2, 3)"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foo', 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnLengthSmallerThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 text CHECK LENGTH(ck1) < 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foo', 2, 3)"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fooo', 2, 3)"); + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foooo', 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnLengthSmallerOrEqualThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 text CHECK LENGTH(ck1) <= 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foo', 2, 3)"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fooo', 2, 3)"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foooo', 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringBlobColumnLengthEqualToConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 blob CHECK LENGTH(ck1) = 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fooo'), 2, 3)"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foo'), 2, 3)"); + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foooo'), 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringBlobColumnLengthDifferentThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 blob CHECK LENGTH(ck1) != 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foo'), 2, 3)"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foooo'), 2, 3)"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fooo'), 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringBlobColumnLengthBiggerThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 blob CHECK LENGTH(ck1) > 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foooo'), 2, 3)"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foo'), 2, 3)"); + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fooo'), 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringBlobColumnLengthBiggerOrEqualThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 blob CHECK LENGTH(ck1) >= 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foooo'), 2, 3)"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fooo'), 2, 3)"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foo'), 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringBlobColumnLengthSmallerThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 blob CHECK LENGTH(ck1) < 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foo'), 2, 3)"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fooo'), 2, 3)"); + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foooo'), 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringBlobColumnLengthSmallerOrEqualThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 blob CHECK LENGTH(ck1) <= 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foo'), 2, 3)"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fooo'), 2, 3)"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foooo', 2, 3)"); + } + + + @Test + public void testCreateTableWithColumnWithPkColumnLengthEqualToConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk text CHECK LENGTH(pk) = 4, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 1, 2, 3)"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foo', 1, 2, 3)"); + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foooo', 1, 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithPkColumnLengthDifferentThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk text CHECK LENGTH(pk) != 4, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foo', 1, 2, 3)"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foooo', 1, 2, 3)"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 1, 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithPkColumnLengthBiggerThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk text CHECK LENGTH(pk) > 4, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foooo', 1, 2, 3)"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foo', 1, 2, 3)"); + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 1, 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithPkColumnLengthBiggerOrEqualThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk text CHECK LENGTH(pk) >= 4, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foooo', 1, 2, 3)"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 1, 2, 3)"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foo', 1, 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithPkColumnLengthSmallerThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk text CHECK LENGTH(pk) < 4, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foo', 1, 2, 3)"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 1, 2, 3)"); + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foooo', 1, 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithPkColumnLengthSmallerOrEqualThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk text CHECK LENGTH(pk) <= 4, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foo', 1, 2, 3)"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 1, 2, 3)"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foooo', 1, 2, 3)"); + } + + + @Test + public void testCreateTableWithColumnWithRegularColumnLengthEqualToConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v text CHECK LENGTH(v) = 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fooo')"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foo')"); + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foooo')"); + } + + @Test + public void testCreateTableWithColumnWithRegularColumnLengthDifferentThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v text CHECK LENGTH(v) != 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foo')"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foooo')"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fooo')"); + } + + @Test + public void testCreateTableWithColumnWithRegularColumnLengthBiggerThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v text CHECK LENGTH(v) > 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foooo')"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foo')"); + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fooo')"); + } + + @Test + public void testCreateTableWithColumnWithRegularColumnLengthBiggerOrEqualThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v text CHECK LENGTH(v) >= 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foooo')"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fooo')"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foo')"); + } + + @Test + public void testCreateTableWithColumnWithRegularColumnLengthSmallerThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v text CHECK LENGTH(v) < 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foo')"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fooo')"); + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foooo')"); + } + + @Test + public void testCreateTableWithColumnWithRegularColumnLengthSmallerOrEqualThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v text CHECK LENGTH(v) <= 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foo')"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fooo')"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foooo')"); + } + + @Test + public void testCreateTableWithColumnMixedColumnsLengthConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk text CHECK LENGTH(pk) = 4, ck1 int, ck2 int, v text CHECK LENGTH(v) = 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 2, 3, 'fooo')"); + + // Invalid + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foo', 2, 3, 'foo')"); + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 2, 3, 'foo')"); + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foo', 2, 3, 'fooo')"); + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foooo', 2, 3, 'fooo')"); + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 2, 3, 'foooo')"); + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foooo', 2, 3, 'foooo')"); + } + + @Test + public void testCreateTableWithWrongColumnConstraint() throws Throwable + { + try + { + createTable("CREATE TABLE %s (pk text, ck1 int CHECK LENGTH(pk) = 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + fail(); + } + catch (InvalidRequestException e) + { + assertTrue(e.getCause() instanceof InvalidRequestException); + assertTrue(e.getMessage().contains("Error setting schema for test")); + } + } + + @Test + public void testCreateTableWithWrongColumnMultipleConstraint() throws Throwable + { + try + { + createTable("CREATE TABLE %s (pk text, ck1 int CHECK LENGTH(pk) = 4 AND ck1 < 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + fail(); + } + catch (InvalidRequestException e) + { + assertTrue(e.getCause() instanceof InvalidRequestException); + assertTrue(e.getMessage().contains("Error setting schema for test")); + } + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnInvalidTypeConstraint() throws Throwable + { + try + { + createTable("CREATE TABLE %s (pk int, ck1 int CHECK LENGTH(ck1) = 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + fail(); + } + catch (InvalidRequestException e) + { + assertTrue(e.getCause() instanceof InvalidRequestException); + assertTrue(e.getMessage().contains("Error setting schema for test")); + } + } + + @Test + public void testCreateTableInvalidFunction() throws Throwable + { + try + { + createTable("CREATE TABLE %s (pk text CHECK not_a_function(pk) = 4, ck1 int, ck2 int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + fail(); + } + catch (InvalidRequestException e) + { + assertTrue(e.getCause() instanceof InvalidRequestException); + assertTrue(e.getMessage().contains("Error setting schema for test")); + } + } + + @Test + public void testCreateTableWithPKConstraintsAndCDCEnabled() throws Throwable + { + // It works + createTable("CREATE TABLE %s (pk text CHECK length(pk) = 4, ck1 int, ck2 int, PRIMARY KEY ((pk), ck1, ck2)) WITH cdc = true;"); + } + + @Test + public void testCreateTableWithClusteringConstraintsAndCDCEnabled() throws Throwable + { + // It works + createTable("CREATE TABLE %s (pk text, ck1 int CHECK ck1 < 100, ck2 int, PRIMARY KEY ((pk), ck1, ck2)) WITH cdc = true;"); + } + + @Test + public void testCreateTableWithRegularConstraintsAndCDCEnabled() throws Throwable + { + // It works + createTable("CREATE TABLE %s (pk text, ck1 int CHECK ck1 < 100, ck2 int, PRIMARY KEY (pk)) WITH cdc = true;"); + } + + // Copy table with like + @Test + public void testCreateTableWithColumnWithClusteringColumnLessThanScalarConstraintIntegerOnLikeTable() throws Throwable + { + createTable(KEYSPACE, "CREATE TABLE %s (pk int, ck1 int CHECK ck1 < 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");", "liketabletame"); + + execute("create table " + KEYSPACE + ".tb_copy like %s"); + + // Valid + qt().forAll(Generators.toGen(integers().between(0, 3))) + .check(d -> execute("INSERT INTO " + KEYSPACE + ".tb_copy (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(integers().between(4, 100))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO " + KEYSPACE + ".tb_copy(pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } +} diff --git a/test/unit/org/apache/cassandra/db/marshal/AbstractTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/AbstractTypeTest.java index ed2046fe5df6..5cb051b35e77 100644 --- a/test/unit/org/apache/cassandra/db/marshal/AbstractTypeTest.java +++ b/test/unit/org/apache/cassandra/db/marshal/AbstractTypeTest.java @@ -259,6 +259,18 @@ private boolean isTestType(Class<? extends AbstractType> klass) return "test".equals(new File(src.getLocation().getPath()).name()); } + @Test + public void isConstrainedTest() + { + qt().forAll(genBuilder().build()).checkAssert(type -> { + if (type instanceof MapType || type instanceof TupleType || type instanceof AbstractCompositeType) + assertThat(type.isConstrainable()).isEqualTo(false); + else + assertThat(type.isConstrainable()).isEqualTo(true); + }); + + } + @Test public void unsafeSharedSerializer() { @@ -950,7 +962,7 @@ private static void verifyTypesCompatibility(AbstractType left, AbstractType rig assertThat(leftDecomposed.hasRemaining()).describedAs(typeRelDesc(".decompose", left, right)).isEqualTo(rightDecomposed.hasRemaining()); // serialization compatibility means that we can read a cell written using right's type serializer with left's type serializer; - // this additinoally imposes the requirement for storing the buffer lenght in the serialized form if the value is of variable length + // this additinoally imposes the requirement for storing the buffer length in the serialized form if the value is of variable length // as well as, either both types serialize into a single or multiple cells if (left.isSerializationCompatibleWith(right)) { From 8d525de550ccc934970b5f65bf3c753eac9f94de Mon Sep 17 00:00:00 2001 From: Brad Schoening <bschoeni@gmail.com> Date: Thu, 30 Jan 2025 14:31:54 -0500 Subject: [PATCH 113/225] Remove obsolete CAPI-Rowcache and updated Lucene Indexing Plugin patch by Brad Schoening; reviewed by Stefan Miklosovic for CASSANDRA-19773 --- .../pages/integrating/plugins/index.adoc | 25 ++++--------------- 1 file changed, 5 insertions(+), 20 deletions(-) diff --git a/doc/modules/cassandra/pages/integrating/plugins/index.adoc b/doc/modules/cassandra/pages/integrating/plugins/index.adoc index 8c6925ec39a1..b6cd8fb0e98f 100644 --- a/doc/modules/cassandra/pages/integrating/plugins/index.adoc +++ b/doc/modules/cassandra/pages/integrating/plugins/index.adoc @@ -3,27 +3,12 @@ Available third-party plugins for Apache Cassandra -== CAPI-Rowcache +== Cassandra Lucene Index -The Coherent Accelerator Process Interface (CAPI) is a general term for -the infrastructure of attaching a Coherent accelerator to an IBM POWER -system. A key innovation in IBM POWER8's open architecture is the CAPI. -It provides a high bandwidth, low latency path between external devices, -the POWER8 core, and the system's open memory architecture. IBM Data -Engine for NoSQL is an integrated platform for large and fast growing -NoSQL data stores. It builds on the CAPI capability of POWER8 systems -and provides super-fast access to large flash storage capacity and -addresses the challenges associated with typical x86 server based -scale-out deployments. +**NOTE** Support for the Cassandra Lucene Index Plug-in was retired by Instaclustr after the release of Cassandra 5.0 with Storage-attached Indexing (SAI). It remains support for Cassandra 4.x releases. -The official page for the -https://github.com/ppc64le/capi-rowcache[CAPI-Rowcache plugin] contains -further details how to build/run/download the plugin. - -== Stratio's Cassandra Lucene Index - -Stratio's Lucene index is a Cassandra secondary index implementation -based on http://lucene.apache.org/[Apache Lucene]. It extends +The Cassandra Lucene Index Plug-in was a Cassandra secondary index implementation +based on http://lucene.apache.org/[Apache Lucene]. It extended Cassandra's functionality to provide near real-time distributed search engine capabilities such as with ElasticSearch or http://lucene.apache.org/solr/[Apache Solr], including full text search @@ -33,5 +18,5 @@ distance. Each node indexes its own data, so high availability and scalability is guaranteed. The official Github repository -http://www.github.com/stratio/cassandra-lucene-index[Cassandra Lucene +https://github.com/instaclustr/cassandra-lucene-index/[Cassandra Lucene Index] contains everything you need to build/run/configure the plugin. From 68b74253329dc95a4b15dd23531caa2f5c439a7f Mon Sep 17 00:00:00 2001 From: Dmitry Konstantinov <konstantinov@netcracker.com> Date: Wed, 4 Dec 2024 22:51:28 +0300 Subject: [PATCH 114/225] WaitQueue.Signal.awaitUninterruptibly may block forever if invoking thread is interrupted Patch by Dmitry Konstantinov; reviewed by Benedict Elliott Smith and Josh McKenzie for CASSANDRA-20084 --- CHANGES.txt | 1 + .../cassandra/utils/concurrent/WaitQueue.java | 59 ++++++++++++++-- .../cassandra/concurrent/WaitQueueTest.java | 70 +++++++++++++++++++ 3 files changed, 125 insertions(+), 5 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 3d372fce00fe..1ba96d2a2310 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.1.8 + * Fix WaitQueue.Signal.awaitUninterruptibly may block forever if invoking thread is interrupted (CASSANDRA-20084) * Run audit_logging_options through santiation and validation on startup (CASSANDRA-20208) * Enforce CQL message size limit on multiframe messages (CASSANDRA-20052) * Add nodetool checktokenmetadata command that checks TokenMetadata is insync with Gossip endpointState (CASSANDRA-18758) diff --git a/src/java/org/apache/cassandra/utils/concurrent/WaitQueue.java b/src/java/org/apache/cassandra/utils/concurrent/WaitQueue.java index e9dcdf86e9f9..e97d4a0c74ed 100644 --- a/src/java/org/apache/cassandra/utils/concurrent/WaitQueue.java +++ b/src/java/org/apache/cassandra/utils/concurrent/WaitQueue.java @@ -74,7 +74,7 @@ * to be met that we no longer need. * <p>5. This scheme is not fair</p> * <p>6. Only the thread that calls register() may call await()</p> - * + * <p>7. A signal can be cancelled while it is within await() if the invoking thread is interrupted</p> * TODO: this class should not be backed by CLQ (should use an intrusive linked-list with lower overhead) */ @Shared(scope = SIMULATION, inner = INTERFACES) @@ -109,9 +109,9 @@ public static interface Signal extends Condition public boolean isSet(); /** - * atomically: cancels the Signal if !isSet(), or returns true if isSignalled() + * atomically: cancels the Signal if !isSet(), or returns true if isSet() * - * @return true if isSignalled() + * @return true if isSet() */ public boolean checkAndClear(); @@ -120,6 +120,23 @@ public static interface Signal extends Condition * and if signalled propagates the signal to another waiting thread */ public abstract void cancel(); + + /** + * Await indefinitely, throwing any interrupt. + * No spurious wakeups. + * Important: the signal can be cancelled if the thread executing await() is interrupted + * @throws InterruptedException if interrupted + */ + Awaitable await() throws InterruptedException; + + /** + * Await until the deadline (in nanoTime), throwing any interrupt. + * No spurious wakeups. + * @return true if we were signalled, false if the deadline elapsed + * Important: the signal can be cancelled if the thread executing await() is interrupted + * @throws InterruptedException if interrupted + */ + boolean awaitUntil(long nanoTimeDeadline) throws InterruptedException; } /** @@ -281,9 +298,24 @@ public int getWaiting() */ public static abstract class AbstractSignal extends AbstractAwaitable implements Signal { + public Signal awaitUninterruptibly() + { + boolean interrupted = false; + while (!isSet()) + { + if (Thread.interrupted()) + interrupted = true; + LockSupport.park(); + } + if (interrupted) + Thread.currentThread().interrupt(); + checkAndClear(); + return this; + } + public Signal await() throws InterruptedException { - while (!isSignalled()) + while (!isSet()) { checkInterrupted(); LockSupport.park(); @@ -292,10 +324,27 @@ public Signal await() throws InterruptedException return this; } + public boolean awaitUntilUninterruptibly(long nanoTimeDeadline) + { + boolean interrupted = false; + long now; + while (nanoTimeDeadline > (now = nanoTime()) && !isSet()) + { + if (Thread.interrupted()) + interrupted = true; + long delta = nanoTimeDeadline - now; + LockSupport.parkNanos(delta); + } + if (interrupted) + Thread.currentThread().interrupt(); + + return checkAndClear(); + } + public boolean awaitUntil(long nanoTimeDeadline) throws InterruptedException { long now; - while (nanoTimeDeadline > (now = nanoTime()) && !isSignalled()) + while (nanoTimeDeadline > (now = nanoTime()) && !isSet()) { checkInterrupted(); long delta = nanoTimeDeadline - now; diff --git a/test/unit/org/apache/cassandra/concurrent/WaitQueueTest.java b/test/unit/org/apache/cassandra/concurrent/WaitQueueTest.java index a9049a9d8103..721c9094a9be 100644 --- a/test/unit/org/apache/cassandra/concurrent/WaitQueueTest.java +++ b/test/unit/org/apache/cassandra/concurrent/WaitQueueTest.java @@ -26,6 +26,7 @@ import org.junit.*; import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -119,4 +120,73 @@ public void run() assertFalse(fail.get()); } + @Test + public void testInterruptOfSignalAwaitingThread() throws InterruptedException + { + final WaitQueue waitQueue = newWaitQueue(); + Thread writerAwaitThread = createThread(() -> { + Thread.currentThread().interrupt(); + WaitQueue.Signal signal = waitQueue.register(); + signal.awaitUninterruptibly(); + + }, "writer.await"); + + writerAwaitThread.start(); + + Thread.sleep(1_000); // wait to enter signal.awaitUninterruptibly() + waitQueue.signalAll(); + + writerAwaitThread.join(4_000); + if (writerAwaitThread.isAlive()) + { + printThreadStackTrace(writerAwaitThread); + fail("signal.awaitUninterruptibly() is stuck"); + } + } + + @Test + public void testInterruptOfSignalAwaitingWithTimeoutThread() throws InterruptedException + { + final WaitQueue waitQueue = newWaitQueue(); + Thread writerAwaitThread = createThread(() -> { + Thread.currentThread().interrupt(); + WaitQueue.Signal signal = waitQueue.register(); + signal.awaitUninterruptibly(100_000, TimeUnit.MILLISECONDS); + }, "writer.await"); + + writerAwaitThread.start(); + + Thread.sleep(1_000); // wait to enter signal.awaitUninterruptibly() + waitQueue.signalAll(); + + writerAwaitThread.join(4_000); + if (writerAwaitThread.isAlive()) + { + printThreadStackTrace(writerAwaitThread); + fail("signal.awaitUninterruptibly() is stuck"); + } + } + + private static Thread createThread(Runnable job, String name) + { + Thread thread = new Thread(job, name); + thread.setDaemon(true); + return thread; + } + + private static void printThreadStackTrace(Thread thread) + { + System.out.println("Stack trace for thread: " + thread.getName()); + StackTraceElement[] stackTrace = thread.getStackTrace(); + if (stackTrace.length == 0) + { + System.out.println("The thread is not currently running or has no stack trace."); + } else + { + for (StackTraceElement element : stackTrace) + { + System.out.println("\tat " + element); + } + } + } } From a118a704cbe31bc72699595f112dda7870c85287 Mon Sep 17 00:00:00 2001 From: Caleb Rackliffe <calebrackliffe@gmail.com> Date: Thu, 30 Jan 2025 17:28:37 -0600 Subject: [PATCH 115/225] Avoid under-skipping during intersections when an iterator has mixed STATIC and WIDE keys patch by Caleb Rackliffe; reviewed by David Capwell for CASSANDRA-20258 --- CHANGES.txt | 1 + .../index/sai/StorageAttachedIndexGroup.java | 6 ++- .../KeyRangeIntersectionIterator.java | 30 ++++++++++--- .../cql/CompositePartitionKeyIndexTest.java | 42 +++++++++++++++++++ 4 files changed, 71 insertions(+), 8 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index a28736122fdc..4553d03c0a89 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.3 + * Avoid under-skipping during intersections when an iterator has mixed STATIC and WIDE keys (CASSANDRA-20258) * Correct the default behavior of compareTo() when comparing WIDE and STATIC PrimaryKeys (CASSANDRA-20238) * Make sure we can set parameters when configuring CassandraCIDRAuthorizer (CASSANDRA-20220) * Add selected SAI index state and query performance metrics to nodetool tablestats (CASSANDRA-20026) diff --git a/src/java/org/apache/cassandra/index/sai/StorageAttachedIndexGroup.java b/src/java/org/apache/cassandra/index/sai/StorageAttachedIndexGroup.java index 30d23f5de61d..bddeacecc7cd 100644 --- a/src/java/org/apache/cassandra/index/sai/StorageAttachedIndexGroup.java +++ b/src/java/org/apache/cassandra/index/sai/StorageAttachedIndexGroup.java @@ -176,8 +176,10 @@ public Index.Indexer indexerFor(Predicate<Index> indexSelector, @Override public void insertRow(Row row) { - for (Index.Indexer indexer : indexers) - indexer.insertRow(row); + // SAI does not index deletions, as these are resolved during post-filtering. + if (row.deletion().isLive()) + for (Index.Indexer indexer : indexers) + indexer.insertRow(row); } @Override diff --git a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeIntersectionIterator.java b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeIntersectionIterator.java index e820b2fc4b73..6237aa013180 100644 --- a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeIntersectionIterator.java +++ b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeIntersectionIterator.java @@ -85,12 +85,7 @@ protected PrimaryKey computeNext() if (range.peek().compareTo(highestKey, false) < 0) { - // If we advance a STATIC key, then we must advance it to the same partition as the highestKey. - // Advancing a STATIC key to a WIDE key directly (without throwing away the clustering) would - // go too far, as WIDE keys are stored after STATIC in the posting list. - PrimaryKey nextKey = range.peek().kind() == Kind.STATIC - ? skipAndPeek(range, highestKey.toStatic()) - : skipAndPeek(range, highestKey); + PrimaryKey nextKey = skipToHighestKey(range); // We use strict comparison here, since it orders WIDE primary keys after STATIC primary keys // in the same partition. When WIDE keys are present, we want to return them rather than STATIC @@ -129,6 +124,29 @@ protected PrimaryKey computeNext() return endOfData(); } + private PrimaryKey skipToHighestKey(KeyRangeIterator range) + { + if (range.peek().kind() == highestKey.kind()) + return skipAndPeek(range, highestKey); + + if (range.peek().kind() == Kind.STATIC) + { + // If we advance a STATIC key, then we must advance it to the same partition as the highestKey. + // Advancing a STATIC key to a WIDE key directly (without throwing away the clustering) would + // go too far, as WIDE keys are stored after STATIC in the posting list. + PrimaryKey nextKey = skipAndPeek(range, highestKey.toStatic()); + + if (nextKey != null && nextKey.compareTo(highestKey, true) < 0 && nextKey.kind() == Kind.WIDE) + // This iterator may have mixed STATIC and non-STATIC postings. Advance again if we've + // landed on a WIDE key that sorts lower in the same partition. + nextKey = skipAndPeek(range, highestKey); + + return nextKey; + } + + return skipAndPeek(range, highestKey); + } + /** * Advances the iterator of one range to the next item, which becomes the highest seen so far. * Iterators pointing to STATIC keys are advanced only if no non-STATIC keys have been advanced. diff --git a/test/unit/org/apache/cassandra/index/sai/cql/CompositePartitionKeyIndexTest.java b/test/unit/org/apache/cassandra/index/sai/cql/CompositePartitionKeyIndexTest.java index f384ca74e449..5d3c9e11d75a 100644 --- a/test/unit/org/apache/cassandra/index/sai/cql/CompositePartitionKeyIndexTest.java +++ b/test/unit/org/apache/cassandra/index/sai/cql/CompositePartitionKeyIndexTest.java @@ -22,12 +22,54 @@ import org.junit.Test; import org.apache.cassandra.cql3.restrictions.StatementRestrictions; +import org.apache.cassandra.db.marshal.FloatType; import org.apache.cassandra.db.marshal.SimpleDateType; import org.apache.cassandra.db.marshal.TimeType; +import org.apache.cassandra.db.marshal.UUIDType; import org.apache.cassandra.index.sai.SAITester; public class CompositePartitionKeyIndexTest extends SAITester { + @Test + public void testIntersectionOnMixedPostingsOnDelete() throws Throwable + { + createTable("CREATE TABLE %s (pk0 boolean, pk1 uuid, ck0 date, ck1 smallint, s0 timeuuid static, v0 bigint, v1 float, PRIMARY KEY ((pk0, pk1), ck0, ck1)) WITH CLUSTERING ORDER BY (ck0 DESC, ck1 ASC)"); + + createIndex("CREATE INDEX tbl_pk0 ON %s(pk0) USING 'sai'"); + createIndex("CREATE INDEX tbl_ck0 ON %s(ck0) USING 'sai'"); + + execute("INSERT INTO %s (pk0, pk1, ck0, ck1, s0) VALUES (true, 00000000-0000-4700-8d00-000000000000, '-3038243-10-30', -12906, 00000000-0000-1900-aa00-000000000000)"); + execute("INSERT INTO %s (pk0, pk1, ck0, ck1, v0, v1) VALUES (false, 00000000-0000-4f00-a200-000000000000, '-1225324-10-07', -3223, -7318794006633168842, 8.0350916E-32 + 6.127658E28)"); + execute("DELETE FROM %s WHERE pk0 = false AND pk1 = 00000000-0000-4f00-a200-000000000000 AND ck0 = '-1111567-10-09' AND ck1 = 25967"); + execute("DELETE s0 FROM %s WHERE pk0 = false AND pk1 = 00000000-0000-4500-9200-000000000000"); + + beforeAndAfterFlush(() -> + assertRows(execute("SELECT * FROM %s WHERE pk0 = false AND ck0 = '-1225324-10-07'"), + row(false, UUIDType.instance.fromString("00000000-0000-4f00-a200-000000000000"), + SimpleDateType.instance.fromString("-1225324-10-07"), (short) -3223, null, + -7318794006633168842L, FloatType.instance.fromString("6.127658E28")))); + } + + @Test + public void testIntersectionOnMixedPostingsOnUpdate() throws Throwable + { + createTable("CREATE TABLE %s (pk0 boolean, pk1 uuid, ck0 date, ck1 smallint, s0 timeuuid static, v0 bigint, v1 float, PRIMARY KEY ((pk0, pk1), ck0, ck1)) WITH CLUSTERING ORDER BY (ck0 DESC, ck1 ASC)"); + + createIndex("CREATE INDEX tbl_pk0 ON %s(pk0) USING 'sai'"); + createIndex("CREATE INDEX tbl_ck0 ON %s(ck0) USING 'sai'"); + + execute("INSERT INTO %s (pk0, pk1, ck0, ck1, s0) VALUES (true, 00000000-0000-4700-8d00-000000000000, '-3038243-10-30', -12906, 00000000-0000-1900-aa00-000000000000)"); + execute("INSERT INTO %s (pk0, pk1, ck0, ck1, v0, v1) VALUES (false, 00000000-0000-4f00-a200-000000000000, '-1225324-10-07', -3223, -7318794006633168842, 8.0350916E-32 + 6.127658E28)"); + execute("UPDATE %s SET v1 = 2.1 WHERE pk0 = false AND pk1 = 00000000-0000-4f00-a200-000000000000 AND ck0 = '-1111567-10-09' AND ck1 = 25967"); + execute("UPDATE %s SET s0 = 00000000-0000-1900-aa00-000000000000 WHERE pk0 = false AND pk1 = 00000000-0000-4500-9200-000000000000"); + + beforeAndAfterFlush(() -> + assertRows(execute("SELECT * FROM %s WHERE pk0 = false AND ck0 = '-1225324-10-07'"), + row(false, UUIDType.instance.fromString("00000000-0000-4f00-a200-000000000000"), + SimpleDateType.instance.fromString("-1225324-10-07"), (short) -3223, null, + -7318794006633168842L, FloatType.instance.fromString("6.127658E28")))); + } + @Test public void testIntersectionWithStaticOverlap() throws Throwable { From 7c147ef80e63d99cee576f4c7ee345c8a8c1aa7e Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Tue, 4 Feb 2025 09:46:26 +0100 Subject: [PATCH 116/225] Increment version --- CHANGES.txt | 3 +++ build.xml | 2 +- debian/changelog | 6 ++++++ 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index d9a3ecbe4d28..38d003acf6f4 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,3 +1,6 @@ +3.0.32 + + 3.0.31 * Tighten up permissions on system keyspaces (CASSANDRA-20090) * Fix incorrect column identifier bytes problem when renaming a column (CASSANDRA-18956) diff --git a/build.xml b/build.xml index 7bcfb3e17c7e..07b12ed097b8 100644 --- a/build.xml +++ b/build.xml @@ -34,7 +34,7 @@ <property name="debuglevel" value="source,lines,vars"/> <!-- default version and SCM information --> - <property name="base.version" value="3.0.31"/> + <property name="base.version" value="3.0.32"/> <property name="scm.connection" value="scm:https://gitbox.apache.org/repos/asf/cassandra.git"/> <property name="scm.developerConnection" value="scm:https://gitbox.apache.org/repos/asf/cassandra.git"/> <property name="scm.url" value="https://gitbox.apache.org/repos/asf?p=cassandra.git"/> diff --git a/debian/changelog b/debian/changelog index 53060d642101..fca85e2dba24 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,3 +1,9 @@ +cassandra (3.0.32) UNRELEASED; urgency=medium + + * New release + + -- Stefan Miklosovic <smiklosovic@apache.org> Tue, 04 Feb 2025 09:43:30 +0100 + cassandra (3.0.31) unstable; urgency=medium * New release From 4f9e474bb1a2ff2e22d898f8d97e4f7532a7098f Mon Sep 17 00:00:00 2001 From: Francisco Guerrero <frankgh@apache.org> Date: Fri, 27 Dec 2024 14:12:17 -0800 Subject: [PATCH 117/225] Fix autocompletion for role names/user names patch by Francisco Guerrero; reviewed by Stefan Miklosovic, Bernardo Botella for CASSANDRA-20175 --- CHANGES.txt | 1 + pylib/cqlshlib/cql3handling.py | 18 ++++-------------- pylib/cqlshlib/test/test_cqlsh_completion.py | 5 +++++ 3 files changed, 10 insertions(+), 14 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index fb82600c97d9..09a9bee44d97 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0.17 + * Fix autocompletion for role names/user names (CASSANDRA-20175) 4.0.16 diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py index 3d1b16386ffb..a813c345f1c6 100644 --- a/pylib/cqlshlib/cql3handling.py +++ b/pylib/cqlshlib/cql3handling.py @@ -1472,9 +1472,9 @@ def alter_type_field_completer(ctxt, cass): ''' syntax_rules += r''' -<rolename> ::= <identifier> +<rolename> ::= role=( <identifier> | <quotedName> - | <unreservedKeyword> + | <unreservedKeyword> ) ; <createRoleStatement> ::= "CREATE" "ROLE" <rolename> @@ -1564,32 +1564,22 @@ def alter_type_field_completer(ctxt, cass): @completer_for('username', 'name') def username_name_completer(ctxt, cass): - def maybe_quote(name): - if CqlRuleSet.is_valid_cql3_name(name): - return name - return "'%s'" % name - # disable completion for CREATE USER. if ctxt.matched[0][1].upper() == 'CREATE': return [Hint('<username>')] session = cass.session - return [maybe_quote(list(row.values())[0].replace("'", "''")) for row in session.execute("LIST USERS")] + return map(maybe_escape_name, [row['name'] for row in session.execute("LIST USERS")]) @completer_for('rolename', 'role') def rolename_completer(ctxt, cass): - def maybe_quote(name): - if CqlRuleSet.is_valid_cql3_name(name): - return name - return "'%s'" % name - # disable completion for CREATE ROLE. if ctxt.matched[0][1].upper() == 'CREATE': return [Hint('<rolename>')] session = cass.session - return [maybe_quote(row[0].replace("'", "''")) for row in session.execute("LIST ROLES")] + return map(maybe_escape_name, [row['role'] for row in session.execute("LIST ROLES")]) syntax_rules += r''' diff --git a/pylib/cqlshlib/test/test_cqlsh_completion.py b/pylib/cqlshlib/test/test_cqlsh_completion.py index 9d902e43b31d..8585e3496e6b 100644 --- a/pylib/cqlshlib/test/test_cqlsh_completion.py +++ b/pylib/cqlshlib/test/test_cqlsh_completion.py @@ -837,3 +837,8 @@ def test_complete_in_alter_keyspace(self): self.trycompletions('ALTER KEYSPACE system_trac', "es WITH replication = {'class': '") self.trycompletions("ALTER KEYSPACE system_traces WITH replication = {'class': '", '', choices=['NetworkTopologyStrategy', 'SimpleStrategy']) + + def test_complete_in_create_role(self): + self.trycompletions('CREATE ROLE ', choices=['<rolename>']) + self.trycompletions('CREATE USER ', choices=['<username>', 'IF']) + From 69dc5d05ef18ff7c6a4433a81b8a3d59e3069579 Mon Sep 17 00:00:00 2001 From: Chris Lohfink <clohfink@netflix.com> Date: Sat, 21 Dec 2024 10:07:27 -0600 Subject: [PATCH 118/225] Add system_views.partition_key_statistics for querying SSTable metadata patch by Chris Lohfink; reviewed by David Capwell, Stefan Miklosovic for CASSANDRA-20161 --- CHANGES.txt | 1 + .../managing/operating/virtualtables.adoc | 47 +++ .../restrictions/StatementRestrictions.java | 10 +- .../cql3/statements/SelectStatement.java | 5 +- .../db/PartitionRangeReadCommand.java | 2 +- .../db/SinglePartitionReadCommand.java | 2 +- .../db/virtual/AbstractVirtualTable.java | 5 +- .../CollectionVirtualTableAdapter.java | 6 +- .../db/virtual/PartitionKeyStatsTable.java | 364 ++++++++++++++++++ .../db/virtual/SystemViewsKeyspace.java | 1 + .../cassandra/db/virtual/VirtualTable.java | 7 +- .../apache/cassandra/dht/IPartitioner.java | 9 + .../cassandra/dht/Murmur3Partitioner.java | 5 + .../cassandra/dht/RandomPartitioner.java | 5 + .../io/sstable/format/SSTableReader.java | 8 + .../sstable/format/big/BigTableKeyReader.java | 7 +- .../io/sstable/format/big/BigTableReader.java | 22 ++ .../io/sstable/format/bti/BtiTableReader.java | 10 + .../format/ForwardingSSTableReader.java | 5 + .../virtual/PartitionKeyStatsTableTest.java | 351 +++++++++++++++++ 20 files changed, 856 insertions(+), 16 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/virtual/PartitionKeyStatsTable.java create mode 100644 test/unit/org/apache/cassandra/db/virtual/PartitionKeyStatsTableTest.java diff --git a/CHANGES.txt b/CHANGES.txt index c1406fee9963..efac6cea44e7 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Add system_views.partition_key_statistics for querying SSTable metadata (CASSANDRA-20161) * CEP-42 - Add Constraints Framework (CASSANDRA-19947) * Add table metric PurgeableTombstoneScannedHistogram and a tracing event for scanned purgeable tombstones (CASSANDRA-20132) * Make sure we can parse the expanded CQL before writing it to the log or sending it to replicas (CASSANDRA-20218) diff --git a/doc/modules/cassandra/pages/managing/operating/virtualtables.adoc b/doc/modules/cassandra/pages/managing/operating/virtualtables.adoc index d3b948e3d172..362308372ce4 100644 --- a/doc/modules/cassandra/pages/managing/operating/virtualtables.adoc +++ b/doc/modules/cassandra/pages/managing/operating/virtualtables.adoc @@ -516,6 +516,53 @@ SELECT total - progress AS remaining FROM system_views.sstable_tasks; ---- +=== Virtual table for primary id's + +Since https://issues.apache.org/jira/browse/CASSANDRA-20161[CASSANDRA-20161], there is +a virtual table `system_views.partition_key_statistics` to allow users to query partition keys and related metadata for a specific table within a keyspace. This feature provides insights into SSTable-level details, such as token values, size estimates, and SSTable counts, without requiring expensive disk-based operations. + +[source,console] +---- +cassandra@cqlsh> select * from ks.tbl; + + id | cl1 | i +----+-----+----- + 1 | 202 | 999 + 1 | 200 | 999 + 1 | 101 | 600 + 2 | 101 | 600 +---- + +[source,console] +---- +cassandra@cqlsh> use system_views; +cassandra@cqlsh> select * from partition_key_statistics where keyspace_name = 'ks' and table_name = 'tbl' and key = '1'; + +@ Row 1 +---------------+---------------------- + keyspace_name | ks + table_name | tbl + token_value | -4069959284402364209 + key | 1 + size_estimate | 83 + sstables | 3 + +cassandra@cqlsh> select * from partition_key_statistics where keyspace_name = 'ks' and table_name = 'tbl' and key = '2'; + +@ Row 1 +---------------+---------------------- + keyspace_name | ks + table_name | tbl + token_value | -3248873570005575792 + key | 2 + size_estimate | 25 + sstables | 1 +---- + +The value in `sstables` column means how many SSTables a particular key is located in. + +If there is a composite partition key, you can separate the values by a colon for `key`. + === Other Virtual Tables Some examples of using other virtual tables are as follows. diff --git a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java index 71958bc61b1c..e619993c6af6 100644 --- a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java +++ b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java @@ -345,7 +345,7 @@ else if (operator.requiresIndexing()) } else { - if (!allowFiltering && requiresAllowFilteringIfNotSpecified()) + if (!allowFiltering && requiresAllowFilteringIfNotSpecified(table)) throw invalidRequest(allowFilteringMessage(state)); } @@ -356,12 +356,12 @@ else if (operator.requiresIndexing()) validateSecondaryIndexSelections(); } - public boolean requiresAllowFilteringIfNotSpecified() + public static boolean requiresAllowFilteringIfNotSpecified(TableMetadata metadata) { - if (!table.isVirtual()) + if (!metadata.isVirtual()) return true; - VirtualTable tableNullable = VirtualKeyspaceRegistry.instance.getTableNullable(table.id); + VirtualTable tableNullable = VirtualKeyspaceRegistry.instance.getTableNullable(metadata.id); assert tableNullable != null; return !tableNullable.allowFilteringImplicitly(); } @@ -568,7 +568,7 @@ private void processPartitionKeyRestrictions(ClientState state, boolean hasQueri // components must have a EQ. Only the last partition key component can be in IN relation. if (partitionKeyRestrictions.needFiltering()) { - if (!allowFiltering && !forView && !hasQueriableIndex && requiresAllowFilteringIfNotSpecified()) + if (!allowFiltering && !forView && !hasQueriableIndex && requiresAllowFilteringIfNotSpecified(table)) throw new InvalidRequestException(allowFilteringMessage(state)); isKeyRange = true; diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java index 03ff708ec5e2..1762b776a786 100644 --- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java @@ -88,6 +88,7 @@ import org.apache.commons.lang3.builder.ToStringStyle; import static java.lang.String.format; +import static org.apache.cassandra.cql3.restrictions.StatementRestrictions.requiresAllowFilteringIfNotSpecified; import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse; import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull; import static org.apache.cassandra.cql3.statements.RequestValidations.checkNull; @@ -1357,7 +1358,7 @@ private StatementRestrictions prepareRestrictions(ClientState state, boundNames, orderings, selectsOnlyStaticColumns, - parameters.allowFiltering, + parameters.allowFiltering || !requiresAllowFilteringIfNotSpecified(metadata), forView); } @@ -1583,7 +1584,7 @@ private void checkNeedsFiltering(TableMetadata table, StatementRestrictions rest { // We will potentially filter data if the row filter is not the identity and there isn't any index group // supporting all the expressions in the filter. - if (restrictions.requiresAllowFilteringIfNotSpecified()) + if (requiresAllowFilteringIfNotSpecified(table)) checkFalse(restrictions.needFiltering(table), StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE); } } diff --git a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java index c39f2cb2f3dc..4926061cb870 100644 --- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java +++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java @@ -565,7 +565,7 @@ public PartitionIterator execute(ConsistencyLevel consistency, ClientState state public UnfilteredPartitionIterator executeLocally(ReadExecutionController executionController) { VirtualTable view = VirtualKeyspaceRegistry.instance.getTableNullable(metadata().id); - UnfilteredPartitionIterator resultIterator = view.select(dataRange, columnFilter()); + UnfilteredPartitionIterator resultIterator = view.select(dataRange, columnFilter(), rowFilter()); return limits().filter(rowFilter().filter(resultIterator, nowInSec()), nowInSec(), selectsFullPartition()); } diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java index be1d5a23221c..ad6da1e88a5f 100644 --- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java +++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java @@ -1395,7 +1395,7 @@ public PartitionIterator execute(ConsistencyLevel consistency, ClientState state public UnfilteredPartitionIterator executeLocally(ReadExecutionController executionController) { VirtualTable view = VirtualKeyspaceRegistry.instance.getTableNullable(metadata().id); - UnfilteredPartitionIterator resultIterator = view.select(partitionKey, clusteringIndexFilter, columnFilter()); + UnfilteredPartitionIterator resultIterator = view.select(partitionKey, clusteringIndexFilter, columnFilter(), rowFilter()); return limits().filter(rowFilter().filter(resultIterator, nowInSec()), nowInSec(), selectsFullPartition()); } diff --git a/src/java/org/apache/cassandra/db/virtual/AbstractVirtualTable.java b/src/java/org/apache/cassandra/db/virtual/AbstractVirtualTable.java index 344369f7d75e..df2e4bc7ccb0 100644 --- a/src/java/org/apache/cassandra/db/virtual/AbstractVirtualTable.java +++ b/src/java/org/apache/cassandra/db/virtual/AbstractVirtualTable.java @@ -29,6 +29,7 @@ import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.db.filter.ClusteringIndexFilter; import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.filter.RowFilter; import org.apache.cassandra.db.partitions.AbstractUnfilteredPartitionIterator; import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.db.partitions.SingletonUnfilteredPartitionIterator; @@ -75,7 +76,7 @@ public DataSet data(DecoratedKey partitionKey) } @Override - public final UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter) + public final UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter) { Partition partition = data(partitionKey).getPartition(partitionKey); @@ -88,7 +89,7 @@ public final UnfilteredPartitionIterator select(DecoratedKey partitionKey, Clust } @Override - public final UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter) + public final UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter) { DataSet data = data(); diff --git a/src/java/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapter.java b/src/java/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapter.java index c5079d73c0f0..47aa3bd5c43c 100644 --- a/src/java/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapter.java +++ b/src/java/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapter.java @@ -51,6 +51,7 @@ import org.apache.cassandra.db.EmptyIterators; import org.apache.cassandra.db.filter.ClusteringIndexFilter; import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.filter.RowFilter; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.BooleanType; import org.apache.cassandra.db.marshal.ByteType; @@ -307,7 +308,8 @@ else if (partitionKeyTypes.size() > 1) @Override public UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringFilter, - ColumnFilter columnFilter) + ColumnFilter columnFilter, + RowFilter rowFilter) { if (!data.iterator().hasNext()) return EmptyIterators.unfilteredPartition(metadata); @@ -348,7 +350,7 @@ public UnfilteredPartitionIterator select(DecoratedKey partitionKey, } @Override - public UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter) + public UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter) { return createPartitionIterator(metadata, new AbstractIterator<>() { diff --git a/src/java/org/apache/cassandra/db/virtual/PartitionKeyStatsTable.java b/src/java/org/apache/cassandra/db/virtual/PartitionKeyStatsTable.java new file mode 100644 index 000000000000..d114e5faa763 --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/PartitionKeyStatsTable.java @@ -0,0 +1,364 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.virtual; + +import java.io.IOException; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.function.Consumer; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DataRange; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.Slice; +import org.apache.cassandra.db.Slices; +import org.apache.cassandra.db.context.CounterContext; +import org.apache.cassandra.db.filter.ClusteringIndexFilter; +import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.filter.RowFilter; +import org.apache.cassandra.db.marshal.CompositeType; +import org.apache.cassandra.db.marshal.CounterColumnType; +import org.apache.cassandra.db.marshal.IntegerType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.partitions.SingletonUnfilteredPartitionIterator; +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; +import org.apache.cassandra.db.rows.AbstractUnfilteredRowIterator; +import org.apache.cassandra.db.rows.BTreeRow; +import org.apache.cassandra.db.rows.BufferCell; +import org.apache.cassandra.db.rows.Cell; +import org.apache.cassandra.db.rows.EncodingStats; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.Rows; +import org.apache.cassandra.db.rows.Unfiltered; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.db.rows.UnfilteredRowIterators; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.Bounds; +import org.apache.cassandra.dht.LocalPartitioner; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.io.sstable.KeyReader; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.serializers.MarshalException; + +import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest; + +/** + * A virtual table for querying partition keys of SSTables in a specific keyspace. + * + * <p>This table is implemented as a virtual table in Cassandra, meaning it does not + * store data persistently on disk but instead derives its data from live metadata. + * + * <p>The CQL equivalent of this virtual table is: + * <pre> + * CREATE TABLE system_views.partition_key_statistics ( + * keyspace_name TEXT, + * table_name TEXT, + * token_value INT, + * key TEXT, + * size_estimate COUNTER, + * sstables COUNTER, + * PRIMARY KEY ((keyspace_name, table_name), token_value, key) + * ); + * </pre> + * + * <p>Note: + * <ul> + * <li>The `size_estimate` and `sstables` columns represent aggregate information about SSTable sizes and counts.</li> + * <li>Range queries across multiple tables and updates are not supported as this is a read-only table.</li> + * </ul> + */ +public class PartitionKeyStatsTable implements VirtualTable +{ + private static final Logger logger = LoggerFactory.getLogger(PartitionKeyStatsTable.class); + public static final String NAME = "partition_key_statistics"; + + private static final String TABLE_READ_ONLY_ERROR = "The specified table is read-only."; + private static final String UNSUPPORTED_RANGE_QUERY_ERROR = "Range queries are not supported. Please provide both a keyspace and a table name."; + private static final String REVERSED_QUERY_ERROR = "Reversed queries are not supported."; + private static final String KEYSPACE_NOT_EXIST_ERROR = "The keyspace '%s' does not exist."; + private static final String TABLE_NOT_EXIST_ERROR = "The table '%s' does not exist in the keyspace '%s'."; + private static final String KEY_ONLY_EQUALS_ERROR = "The 'key' column can only be used in an equality query for this virtual table."; + private static final String KEY_NOT_WITHIN_BOUNDS_ERROR = "The specified 'key' is not within the provided token value bounds."; + private static final String PARTITIONER_NOT_SUPPORTED = "Partitioner '%s' for table '%s' in keyspace '%s' is not supported."; + + private static final String COLUMN_KEYSPACE_NAME = "keyspace_name"; + private static final String COLUMN_TABLE_NAME = "table_name"; + private static final String COLUMN_TOKEN_VALUE = "token_value"; + private static final String COLUMN_KEY = "key"; + private static final String COLUMN_SIZE_ESTIMATE = "size_estimate"; + private static final String COLUMN_SSTABLES = "sstables"; + + private final TableMetadata metadata; + private final ColumnMetadata sizeEstimateColumn; + private final ColumnMetadata sstablesColumn; + + @VisibleForTesting + final CopyOnWriteArrayList<Consumer<DecoratedKey>> readListener = new CopyOnWriteArrayList<>(); + + public PartitionKeyStatsTable(String keyspace) + { + this.metadata = TableMetadata.builder(keyspace, NAME) + .kind(TableMetadata.Kind.VIRTUAL) + .partitioner(new LocalPartitioner(CompositeType.getInstance(UTF8Type.instance, UTF8Type.instance))) + .addPartitionKeyColumn(COLUMN_KEYSPACE_NAME, UTF8Type.instance) + .addPartitionKeyColumn(COLUMN_TABLE_NAME, UTF8Type.instance) + .addClusteringColumn(COLUMN_TOKEN_VALUE, IntegerType.instance) + .addClusteringColumn(COLUMN_KEY, UTF8Type.instance) + .addRegularColumn(COLUMN_SIZE_ESTIMATE, CounterColumnType.instance) + .addRegularColumn(COLUMN_SSTABLES, CounterColumnType.instance) + .build(); + sizeEstimateColumn = metadata.regularColumns().getSimple(0); + sstablesColumn = metadata.regularColumns().getSimple(1); + } + + @Override + public UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter) + { + if (clusteringIndexFilter.isReversed()) + throw new InvalidRequestException(REVERSED_QUERY_ERROR); + + ByteBuffer[] key = ((CompositeType) this.metadata.partitionKeyType).split(partitionKey.getKey()); + String keyspace = UTF8Type.instance.getString(key[0]); + String table = UTF8Type.instance.getString(key[1]); + + KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace); + if (ksm == null) + throw invalidRequest(KEYSPACE_NOT_EXIST_ERROR, keyspace); + + TableMetadata metadata = ksm.getTableOrViewNullable(table); + if (metadata == null) + throw invalidRequest(TABLE_NOT_EXIST_ERROR, table, keyspace); + + if (!metadata.partitioner.supportsSplitting()) + throw invalidRequest(PARTITIONER_NOT_SUPPORTED, metadata.partitioner.getClass().getName(), table, keyspace); + + AbstractBounds<PartitionPosition> range = getBounds(metadata, clusteringIndexFilter, rowFilter); + return new SingletonUnfilteredPartitionIterator(select(partitionKey, metadata, clusteringIndexFilter, range)); + } + + private List<SSTableReader> getSStables(TableMetadata metadata, AbstractBounds<PartitionPosition> range) + { + return Lists.newArrayList(ColumnFamilyStore.getIfExists(metadata).getTracker().getView().liveSSTablesInBounds(range.left, range.right)); + } + + private UnfilteredRowIterator select(DecoratedKey partitionKey, TableMetadata metadata, ClusteringIndexFilter clusteringIndexFilter, AbstractBounds<PartitionPosition> range) + { + List<SSTableReader> sstables = getSStables(metadata, range); + if (sstables.isEmpty()) + return UnfilteredRowIterators.noRowsIterator(metadata, partitionKey, Rows.EMPTY_STATIC_ROW, DeletionTime.LIVE, false); + + List<UnfilteredRowIterator> sstableIterators = Lists.newArrayList(); + for (SSTableReader sstable : sstables) + sstableIterators.add(getSStableRowIterator(metadata, partitionKey, sstable, clusteringIndexFilter, range)); + + return UnfilteredRowIterators.merge(sstableIterators); + } + + private UnfilteredRowIterator getSStableRowIterator(TableMetadata target, DecoratedKey partitionKey, SSTableReader sstable, ClusteringIndexFilter filter, AbstractBounds<PartitionPosition> range) + { + final KeyReader reader; + try + { + // ignore warning on try-with-resources, the reader will be closed on endOfData or close + reader = sstable.keyReader(range.left); + } + catch (IOException e) + { + logger.error("Error generating keyReader for SSTable: {}", sstable, e); + throw new RuntimeException(e); + } + + return new AbstractUnfilteredRowIterator(metadata, partitionKey, DeletionTime.LIVE, + metadata.regularAndStaticColumns(), Rows.EMPTY_STATIC_ROW, + false, EncodingStats.NO_STATS) + { + public Unfiltered endOfData() + { + reader.close(); + return super.endOfData(); + } + + public void close() + { + reader.close(); + } + + private Row buildRow(Clustering<?> clustering, long size) + { + Row.Builder row = BTreeRow.sortedBuilder(); + row.newRow(clustering); + row.addCell(cell(sizeEstimateColumn, CounterContext.instance().createUpdate(size))); + row.addCell(cell(sstablesColumn, CounterContext.instance().createUpdate(1))); + return row.build(); + } + + @Override + protected Unfiltered computeNext() + { + while (!reader.isExhausted()) + { + DecoratedKey key = target.partitioner.decorateKey(reader.key()); + + for (Consumer<DecoratedKey> listener : readListener) + listener.accept(key); + + // Store the reader's current data position to calculate size later + long lastPosition = reader.dataPosition(); + try + { + // Advance the reader to the next key for the next iteration. Also by moving to next key + // we move the dataPosition to the start of the next key for calculating size + reader.advance(); + } + catch (IOException e) + { + logger.error("Error advancing reader for SSTable: {}", sstable, e); + return endOfData(); + } + + // Calculate the size of the current key. If EOF use the length of the file + long current = reader.dataPosition() == -1 ? sstable.uncompressedLength() : reader.dataPosition(); + long size = current - lastPosition; + + String keyString = target.partitionKeyType.getString(key.getKey()); + + // Check if the current key is outside the queried range; if so, stop + if (range.right.compareTo(key) < 0) + return endOfData(); + + // Convert the token to a string and create a clustering object + String tokenString = key.getToken().toString(); + Clustering<?> clustering = Clustering.make( + IntegerType.instance.decompose(new BigInteger(tokenString)), + UTF8Type.instance.decompose(keyString) + ); + + // Check if the current clustering matches the filter; if so, return the row + if (filter.selects(clustering)) + return buildRow(clustering, size); + } + return endOfData(); + } + }; + } + + /** + * This converts the clustering token/key into the partition level token/key for the target table. Also provides an + * optimization from RowFilter when a `key` is specified with or without the clustering `token` being set. + */ + private AbstractBounds<PartitionPosition> getBounds(TableMetadata target, ClusteringIndexFilter clusteringIndexFilter, RowFilter rowFilter) + { + Slices s = clusteringIndexFilter.getSlices(target); + Token startToken = target.partitioner.getMinimumToken(); + Token endToken = target.partitioner.getMaximumToken(); + BigInteger startTokenValue = new BigInteger(endToken.getTokenValue().toString(), 10); + BigInteger endTokenValue = new BigInteger(startToken.getTokenValue().toString(), 10); + + // find min/max token values from the clustering key + for (int i = 0; i < s.size(); i++) + { + Slice slice = s.get(i); + if (!slice.start().isEmpty()) + { + startTokenValue = startTokenValue.min(IntegerType.instance.compose(slice.start().bufferAt(0))); + startToken = target.partitioner.getTokenFactory().fromString(startTokenValue.toString()); + } + if (!slice.end().isEmpty()) + { + endTokenValue = endTokenValue.max(IntegerType.instance.compose(slice.end().bufferAt(0))); + endToken = target.partitioner.getTokenFactory().fromString(endTokenValue.toString()); + } + } + + // override min/max of token if the `key` is specified + for (RowFilter.Expression expression : rowFilter.getExpressions()) + { + if (expression.column().name.toString().equals(COLUMN_KEY)) + { + if (expression.operator() != Operator.EQ) + throw new InvalidRequestException(KEY_ONLY_EQUALS_ERROR); + + String keyString = UTF8Type.instance.compose(expression.getIndexValue()); + ByteBuffer keyAsBB; + try + { + keyAsBB = target.partitionKeyType.fromString(keyString); + } + catch (MarshalException ex) + { + throw new InvalidRequestException(ex.getMessage()); + } + DecoratedKey decoratedKey = target.partitioner.decorateKey(keyAsBB); + + if (!DataRange.forKeyRange(new Range<>(startToken.minKeyBound(), endToken.maxKeyBound())).contains(decoratedKey.getToken().minKeyBound())) + throw new InvalidRequestException(KEY_NOT_WITHIN_BOUNDS_ERROR); + + return Bounds.bounds(decoratedKey, true, decoratedKey, true); + } + } + return Bounds.bounds(startToken.minKeyBound(), true, endToken.maxKeyBound(), true); + } + + private static Cell<?> cell(ColumnMetadata column, ByteBuffer value) + { + return BufferCell.live(column, 1L, value); + } + + @Override + public TableMetadata metadata() + { + return this.metadata; + } + + @Override + public UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter) + { + throw new InvalidRequestException(UNSUPPORTED_RANGE_QUERY_ERROR); + } + + @Override + public void truncate() + { + throw new InvalidRequestException(TABLE_READ_ONLY_ERROR); + } + + @Override + public void apply(PartitionUpdate update) + { + throw new InvalidRequestException(TABLE_READ_ONLY_ERROR); + } +} diff --git a/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java b/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java index 8c1412e08c01..dacf9f643a8c 100644 --- a/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java +++ b/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java @@ -56,6 +56,7 @@ private SystemViewsKeyspace() .add(new RolesCacheKeysTable(VIRTUAL_VIEWS)) .add(new CQLMetricsTable(VIRTUAL_VIEWS)) .add(new BatchMetricsTable(VIRTUAL_VIEWS)) + .add(new PartitionKeyStatsTable(VIRTUAL_VIEWS)) .add(new StreamingVirtualTable(VIRTUAL_VIEWS)) .add(new GossipInfoTable(VIRTUAL_VIEWS)) .add(new QueriesTable(VIRTUAL_VIEWS)) diff --git a/src/java/org/apache/cassandra/db/virtual/VirtualTable.java b/src/java/org/apache/cassandra/db/virtual/VirtualTable.java index 53a9f2ac7fef..770cb139830f 100644 --- a/src/java/org/apache/cassandra/db/virtual/VirtualTable.java +++ b/src/java/org/apache/cassandra/db/virtual/VirtualTable.java @@ -21,6 +21,7 @@ import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.filter.ClusteringIndexFilter; import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.filter.RowFilter; import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; import org.apache.cassandra.schema.TableMetadata; @@ -59,18 +60,20 @@ default String name() * @param partitionKey the partition key * @param clusteringIndexFilter the clustering columns to selected * @param columnFilter the selected columns + * @param rowFilter filter on which rows a given query should include or exclude * @return the rows corresponding to the requested data. */ - UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter); + UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter); /** * Selects the rows from a range of partitions. * * @param dataRange the range of data to retrieve * @param columnFilter the selected columns + * @param rowFilter filter on which rows a given query should include or exclude * @return the rows corresponding to the requested data. */ - UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter); + UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter); /** * Truncates data from the underlying source, if supported. diff --git a/src/java/org/apache/cassandra/dht/IPartitioner.java b/src/java/org/apache/cassandra/dht/IPartitioner.java index 9dba3e0d9d67..341ebc47f1d4 100644 --- a/src/java/org/apache/cassandra/dht/IPartitioner.java +++ b/src/java/org/apache/cassandra/dht/IPartitioner.java @@ -74,6 +74,15 @@ default Token getMaximumToken() throw new UnsupportedOperationException("If you are using a splitting partitioner, getMaximumToken has to be implemented"); } + /** + * + * @return true if supports splitting as per {@link IPartitioner#split(Token, Token, double)}, false otherwise. Defaults to false. + */ + default boolean supportsSplitting() + { + return false; + } + /** * @return a Token that can be used to route a given key * (This is NOT a method to create a Token from its string representation; diff --git a/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java b/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java index e2371c09376d..dfe0971f7a46 100644 --- a/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java +++ b/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java @@ -138,6 +138,11 @@ public Token split(Token lToken, Token rToken, double ratioToLeft) return new LongToken(newToken); } + public boolean supportsSplitting() + { + return true; + } + public LongToken getMinimumToken() { return MINIMUM; diff --git a/src/java/org/apache/cassandra/dht/RandomPartitioner.java b/src/java/org/apache/cassandra/dht/RandomPartitioner.java index a8fbe764d47d..9b833e3868d8 100644 --- a/src/java/org/apache/cassandra/dht/RandomPartitioner.java +++ b/src/java/org/apache/cassandra/dht/RandomPartitioner.java @@ -135,6 +135,11 @@ public Token split(Token ltoken, Token rtoken, double ratioToLeft) return new BigIntegerToken(newToken); } + public boolean supportsSplitting() + { + return true; + } + public BigIntegerToken getMinimumToken() { return MINIMUM; diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java index 8e2db41f6cbf..bf3b203e3ed9 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java @@ -903,6 +903,14 @@ public UnfilteredRowIterator simpleIterator(FileDataInput file, DecoratedKey key */ public abstract KeyReader keyReader() throws IOException; + /** + * Returns a {@link KeyReader} over all keys in the sstable after a given key. + * @param key + * @return + * @throws IOException + */ + public abstract KeyReader keyReader(PartitionPosition key) throws IOException; + /** * Returns a {@link KeyIterator} over all keys in the sstable. */ diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableKeyReader.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableKeyReader.java index e0b965057fa4..04b07af2ce70 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableKeyReader.java +++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableKeyReader.java @@ -53,7 +53,12 @@ private BigTableKeyReader(FileHandle indexFile, public static BigTableKeyReader create(RandomAccessReader indexFileReader, IndexSerializer serializer) throws IOException { - BigTableKeyReader iterator = new BigTableKeyReader(null, indexFileReader, serializer); + return create(null, indexFileReader, serializer); + } + + public static BigTableKeyReader create(FileHandle indexFile, RandomAccessReader indexFileReader, IndexSerializer serializer) throws IOException + { + BigTableKeyReader iterator = new BigTableKeyReader(indexFile, indexFileReader, serializer); try { iterator.advance(); diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java index 692cadf34df4..0864a64cee4c 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java @@ -155,6 +155,28 @@ public KeyReader keyReader() throws IOException return BigTableKeyReader.create(ifile, rowIndexEntrySerializer); } + @Override + public KeyReader keyReader(PartitionPosition key) throws IOException + { + FileHandle iFile = ifile.sharedCopy(); + RandomAccessReader reader = iFile.createReader(); + reader.seek(getIndexScanPosition(key)); + KeyReader keys = BigTableKeyReader.create(iFile, reader, rowIndexEntrySerializer); + + boolean hasMoreKeys = true; + while (hasMoreKeys) + { + ByteBuffer indexKey = keys.key(); + DecoratedKey indexDecoratedKey = decorateKey(indexKey); + if (indexDecoratedKey.compareTo(key) >= 0) + break; + + // Advance the iterator and check if more keys are available + hasMoreKeys = keys.advance(); + } + return keys; + } + /** * Finds and returns the first key beyond a given token in this SSTable or null if no such key exists. */ diff --git a/src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java index 9a65be1137bb..39160639584a 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java @@ -45,6 +45,7 @@ import org.apache.cassandra.io.sstable.CorruptSSTableException; import org.apache.cassandra.io.sstable.Descriptor; import org.apache.cassandra.io.sstable.IVerifier; +import org.apache.cassandra.io.sstable.KeyReader; import org.apache.cassandra.io.sstable.SSTable; import org.apache.cassandra.io.sstable.SSTableReadsListener; import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason; @@ -124,6 +125,15 @@ public long estimatedKeys() return partitionIndex == null ? 0 : partitionIndex.size(); } + @Override + public KeyReader keyReader(PartitionPosition key) throws IOException + { + return PartitionIterator.create(partitionIndex, metadata().partitioner, rowIndexFile, dfile, + key, -1, + metadata().partitioner.getMaximumToken().maxKeyBound(), 0, + descriptor.version); + } + @Override protected TrieIndexEntry getRowIndexEntry(PartitionPosition key, Operator operator, diff --git a/test/distributed/org/apache/cassandra/io/sstable/format/ForwardingSSTableReader.java b/test/distributed/org/apache/cassandra/io/sstable/format/ForwardingSSTableReader.java index 710a42664790..60168f65196c 100644 --- a/test/distributed/org/apache/cassandra/io/sstable/format/ForwardingSSTableReader.java +++ b/test/distributed/org/apache/cassandra/io/sstable/format/ForwardingSSTableReader.java @@ -243,6 +243,11 @@ public KeyReader keyReader() throws IOException return delegate.keyReader(); } + public KeyReader keyReader(PartitionPosition key) throws IOException + { + return delegate.keyReader(key); + } + @Override public KeyIterator keyIterator() throws IOException { diff --git a/test/unit/org/apache/cassandra/db/virtual/PartitionKeyStatsTableTest.java b/test/unit/org/apache/cassandra/db/virtual/PartitionKeyStatsTableTest.java new file mode 100644 index 000000000000..1d0a9e43e1a9 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/virtual/PartitionKeyStatsTableTest.java @@ -0,0 +1,351 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.virtual; + +import java.math.BigInteger; +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +import com.google.common.collect.ImmutableList; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Row; +import com.datastax.driver.core.exceptions.InvalidQueryException; +import org.apache.cassandra.Util; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.io.sstable.format.bti.BtiFormat; +import org.bouncycastle.util.encoders.Hex; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +@RunWith(Parameterized.class) +public class PartitionKeyStatsTableTest extends CQLTester +{ + private static final String KS_NAME = "vts"; + private String table; + private AtomicInteger scanned; + + private final boolean useBtiFormat; + + @Parameters(name = "Use BtiFormat = {0}") + public static Collection<Object[]> parameters() + { + return Arrays.asList(new Object[][]{ { false }, { true } }); + } + + public PartitionKeyStatsTableTest(boolean useBtiFormat) + { + this.useBtiFormat = useBtiFormat; + } + + @Before + public void before() + { + if (useBtiFormat) + DatabaseDescriptor.setSelectedSSTableFormat(new BtiFormat.BtiFormatFactory().getInstance(Collections.emptyMap())); + + PartitionKeyStatsTable primaryIdTable = new PartitionKeyStatsTable(KS_NAME); + scanned = new AtomicInteger(); + VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(primaryIdTable))); + + table = createTable("CREATE TABLE %s (key blob PRIMARY KEY, value blob)"); + + ByteBuffer value = ByteBuffer.wrap(new byte[1]); + for (int i = -10; i < 1000; i++) + { + ByteBuffer key = Murmur3Partitioner.LongToken.keyForToken(i); + execute("INSERT INTO %s (key, value) VALUES (?, ?)", key, value); + } + Util.flushTable(KEYSPACE, table); + primaryIdTable.readListener.add(unused -> scanned.incrementAndGet()); + } + + @Test + public void testPrimaryIdTable() + { + ResultSet rs = executeNetWithPaging("SELECT * FROM vts.partition_key_statistics WHERE keyspace_name = ? AND table_name = ?", + 10, KEYSPACE, table); + List<Row> all = rs.all(); + assertEquals(1010, all.size()); + assertResults(all, -10, 1000); + // 1010 + 100 for the 1 per 10 page, +1 for the last + assertEquals(1111, scanned.get()); + } + + @Test + public void testTokenValueGreaterThanZero() + { + ResultSet rs = executeNetWithPaging("SELECT * FROM vts.partition_key_statistics WHERE keyspace_name = ? AND table_name = ? AND token_value > 0", + 10, KEYSPACE, table); + List<Row> all = rs.all(); + assertEquals(999, all.size()); + assertResults(all, 1, 1000); + assertEquals(1099, scanned.get()); + } + + @Test + public void testTokenValueGreaterThanNegativeFive() + { + ResultSet rs = executeNetWithPaging("SELECT * FROM vts.partition_key_statistics WHERE keyspace_name = ? AND table_name = ? AND token_value > -5", + 10, KEYSPACE, table); + List<Row> all = rs.all(); + assertEquals(1004, all.size()); + assertResults(all, -4, 1000); + // 1004 + 100 for the 1 per 10 page, +1 for the last + assertEquals(1105, scanned.get()); + } + + @Test + public void testTokenValueLessThanOrEqualToFive() + { + ResultSet rs = executeNetWithPaging("SELECT * FROM vts.partition_key_statistics WHERE keyspace_name = ? AND table_name = ? AND token_value <= 5", + 10, KEYSPACE, table); + List<Row> all = rs.all(); + assertEquals(16, all.size()); + assertResults(all, -10, 5); + assertEquals(18, scanned.get()); + } + + @Test + public void testTokenValueEqualToZero() + { + ResultSet rs = executeNetWithPaging("SELECT * FROM vts.partition_key_statistics WHERE keyspace_name = ? AND table_name = ? AND token_value = 0", + 10, KEYSPACE, table); + List<Row> all = rs.all(); + assertEquals(1, all.size()); + Row row = all.get(0); + assertEquals(BigInteger.valueOf(0), row.get("token_value", BigInteger.class)); + assertEquals(2, scanned.get()); + } + + @Test + public void testTokenValueBounds() + { + ResultSet rs = executeNetWithPaging("SELECT * FROM vts.partition_key_statistics WHERE keyspace_name = ? AND table_name = ? AND token_value > 0 AND token_value < 15", + 10, KEYSPACE, table); + List<Row> all = rs.all(); + assertEquals(14, all.size()); + assertResults(all, 1, 14); + // 0->10 = 11, 10->16 = 7 + assertEquals(18, scanned.get()); + } + + @Test + public void testTokenValueBoundsWithBetween() + { + ResultSet rs = executeNetWithPaging("SELECT * FROM vts.partition_key_statistics WHERE keyspace_name = ? AND table_name = ? AND token_value BETWEEN 0 AND 15", + 10, KEYSPACE, table); + List<Row> all = rs.all(); + assertEquals(16, all.size()); + assertResults(all, 0, 15); + assertEquals(18, scanned.get()); + } + + @Test + public void testTokenValueBoundsWithIn() + { + ResultSet rs = executeNetWithPaging("SELECT * FROM vts.partition_key_statistics WHERE keyspace_name = ? AND table_name = ? AND token_value IN (1,3,6)", + 10, KEYSPACE, table); + List<Row> all = rs.all(); + assertEquals(3, all.size()); + assertEquals(BigInteger.valueOf(1), all.get(0).get("token_value", BigInteger.class)); + assertEquals(BigInteger.valueOf(3), all.get(1).get("token_value", BigInteger.class)); + assertEquals(BigInteger.valueOf(6), all.get(2).get("token_value", BigInteger.class)); + assertEquals(7, scanned.get()); + } + + @Test + public void testTokenValueBoundsWithKey() + { + ByteBuffer ten = Murmur3Partitioner.LongToken.keyForToken(10); + ResultSet rs = executeNetWithPaging("SELECT * FROM vts.partition_key_statistics WHERE keyspace_name = ? AND table_name = ? AND token_value > 0 AND token_value < 15 AND key = ?", + 10, KEYSPACE, table, Hex.toHexString(ten.array())); + List<Row> all = rs.all(); + assertEquals(1, all.size()); + Row row = all.get(0); + assertEquals(BigInteger.valueOf(10), row.get("token_value", BigInteger.class)); + assertEquals(2, scanned.get()); + } + + @Test + public void testByKey() + { + ByteBuffer ten = Murmur3Partitioner.LongToken.keyForToken(10); + ResultSet rs = executeNetWithPaging("SELECT * FROM vts.partition_key_statistics WHERE keyspace_name = ? AND table_name = ? AND key = ?", + 10, KEYSPACE, table, Hex.toHexString(ten.array())); + List<Row> all = rs.all(); + assertEquals(1, all.size()); + Row row = all.get(0); + assertEquals(BigInteger.valueOf(10), row.get("token_value", BigInteger.class)); + assertEquals(2, scanned.get()); + } + + @Test + public void testIgnoreSStableOutOfRange() + { + ByteBuffer twok = Murmur3Partitioner.LongToken.keyForToken(2000); + execute("INSERT INTO %s (key, value) VALUES (?, ?)", twok, ByteBuffer.wrap(new byte[1])); + Util.flushTable(KEYSPACE, table); + ResultSet rs = executeNetWithPaging("SELECT * FROM vts.partition_key_statistics WHERE keyspace_name = ? AND table_name = ? AND token_value > 1500", + 10, KEYSPACE, table); + List<Row> all = rs.all(); + assertEquals(1, all.size()); + Row row = all.get(0); + assertEquals(BigInteger.valueOf(2000), row.get("token_value", BigInteger.class)); + assertEquals(1L, row.get("sstables", Long.class).longValue()); + assertEquals(1, scanned.get()); + } + + @Test + public void testNoResults() + { + ResultSet rs = executeNetWithPaging("SELECT * FROM vts.partition_key_statistics WHERE keyspace_name = ? AND table_name = ? AND token_value < -1000", + 10, KEYSPACE, table); + List<Row> all = rs.all(); + assertEquals(0, all.size()); + assertEquals(0, scanned.get()); // sstables shouldn't even of been touched + } + + @Test(expected = InvalidQueryException.class) + public void testNonExistantKeyspace() + { + ResultSet rs = executeNetWithPaging("SELECT * FROM vts.partition_key_statistics WHERE keyspace_name = 'non_existent' AND table_name = ?", + 10, table); + List<Row> all = rs.all(); + assertEquals(0, all.size()); + assertEquals(0, scanned.get()); + } + + @Test + public void testNoResultsWithSSTables() + { + ByteBuffer o1 = Murmur3Partitioner.LongToken.keyForToken(10000); + ByteBuffer o2 = Murmur3Partitioner.LongToken.keyForToken(10002); + ByteBuffer value = ByteBuffer.wrap(new byte[10]); + execute("INSERT INTO %s (key, value) VALUES (?, ?)", o1, value); + execute("INSERT INTO %s (key, value) VALUES (?, ?)", o2, value); + Util.flushTable(KEYSPACE, table); + + ResultSet rs = executeNetWithPaging("SELECT * FROM vts.partition_key_statistics WHERE keyspace_name = ? AND table_name = ? AND token_value = 10001", + 10, KEYSPACE, table); + List<Row> all = rs.all(); + assertEquals(0, all.size()); + assertEquals(1, scanned.get()); + } + + @Test + public void testPrimaryIdTableDuplicates() + { + // 0xc25f118f072d6ba5cab7fb1468ace617 hashes to 1563004846366 + ByteBuffer dup = Murmur3Partitioner.LongToken.keyForToken(1563004846366L); + // -19, 68, -61 (0xed44c3) hashes to 1563004846366 + ByteBuffer dup2 = ByteBuffer.wrap(new byte[]{ -19, 68, -61 }); + ByteBuffer value = ByteBuffer.wrap(new byte[10]); + execute("INSERT INTO %s (key, value) VALUES (?, ?)", dup, value); + execute("INSERT INTO %s (key, value) VALUES (?, ?)", dup2, value); + Util.flushTable(KEYSPACE, table); + + ResultSet rs = executeNetWithPaging("SELECT * FROM vts.partition_key_statistics WHERE keyspace_name = ? AND table_name = ? AND token_value = 1563004846366", + 10, KEYSPACE, table); + List<Row> all = rs.all(); + assertEquals(2, all.size()); + assertEquals(BigInteger.valueOf(1563004846366L), all.get(0).get("token_value", BigInteger.class)); + assertEquals(BigInteger.valueOf(1563004846366L), all.get(1).get("token_value", BigInteger.class)); + assertEquals("c25f118f072d6ba5cab7fb1468ace617", all.get(0).getString("key")); + assertEquals("ed44c3", all.get(1).getString("key")); + assertEquals(2, scanned.get()); + } + + @Test + public void testCompositeType() throws UnknownHostException + { + String table = createTable("CREATE TABLE %s (key text, keytwo inet, value text, primary key ((key, keytwo)))"); + + execute("INSERT INTO %s (key, keytwo, value) VALUES (?, ?, ?)", "testkey", InetAddress.getByName("127.0.0.1"), "value"); + Util.flushTable(KEYSPACE, table); + + ResultSet rs = executeNetWithPaging("SELECT * FROM vts.partition_key_statistics WHERE keyspace_name = ? AND table_name = ? AND key = 'testkey:127.0.0.1'", + 10, KEYSPACE, table); + List<Row> all = rs.all(); + assertEquals(1, all.size()); + } + + @Test + public void testTextType() + { + String table = createTable("CREATE TABLE %s (key text PRIMARY KEY, value text)"); + + execute("INSERT INTO %s (key, value) VALUES (?, ?)", "testkey", "value"); + Util.flushTable(KEYSPACE, table); + + ResultSet rs = executeNetWithPaging("SELECT * FROM vts.partition_key_statistics WHERE keyspace_name = ? AND table_name = ? AND key = 'testkey'", + 10, KEYSPACE, table); + List<Row> all = rs.all(); + assertEquals(1, all.size()); + } + + @Test + public void testSameKeyInMultipleSSTables() + { + String table = createTable("CREATE TABLE %s (key blob PRIMARY KEY, value blob)"); + + ByteBuffer key = Murmur3Partitioner.LongToken.keyForToken(1); + ByteBuffer value = ByteBuffer.wrap(new byte[10]); + execute("INSERT INTO %s (key, value) VALUES (?, ?)", key, value); + Util.flushTable(KEYSPACE, table); + value = ByteBuffer.wrap(new byte[100]); + execute("INSERT INTO %s (key, value) VALUES (?, ?)", key, value); + Util.flushTable(KEYSPACE, table); + + ResultSet rs = executeNetWithPaging("SELECT * FROM vts.partition_key_statistics WHERE keyspace_name = ? AND table_name = ?", + 10, KEYSPACE, table); + List<Row> all = rs.all(); + assertEquals(1, all.size()); + Row row = all.get(0); + assertEquals(BigInteger.valueOf(1), row.get("token_value", BigInteger.class)); + long size = row.get("size_estimate", Long.class); + // providing a range since with timestamp delta vint encoding worried this may drift with time or in wierd + // VMs so just want to make sure it's in the right ballpark + assertTrue(size >= 110 && size < 200); + assertEquals(2L, row.get("sstables", Long.class).longValue()); + assertEquals(2, scanned.get()); + } + + private static void assertResults(List<Row> all, int start, int end) + { + for (int i = start, offset = 0; i < end; i++, offset++) + { + Row row = all.get(offset); + } + } +} From f97c945afb76b9fc49d8663c475ae931d30c3713 Mon Sep 17 00:00:00 2001 From: Jon Meredith <jonmeredith@apache.org> Date: Tue, 4 Feb 2025 17:52:24 -0700 Subject: [PATCH 119/225] Skip check for DirectIO when initializing tools. Patch by Jon Meredith; reviewed by Stefan Miklosovic for CASSANDRA-20289 --- CHANGES.txt | 1 + .../cassandra/config/DatabaseDescriptor.java | 34 +++++++++++-------- 2 files changed, 20 insertions(+), 15 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index b3cfd5947690..769e4dddbee4 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.4 + * Skip check for DirectIO when initializing tools (CASSANDRA-20289) * Avoid under-skipping during intersections when an iterator has mixed STATIC and WIDE keys (CASSANDRA-20258) * Correct the default behavior of compareTo() when comparing WIDE and STATIC PrimaryKeys (CASSANDRA-20238) Merged from 4.1: diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 4a11285e4132..6de2c7473d8d 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -1480,24 +1480,28 @@ private static Pair<DiskAccessMode, Boolean> resolveCommitLogWriteDiskAccessMode { boolean compressOrEncrypt = getCommitLogCompression() != null || (getEncryptionContext() != null && getEncryptionContext().isEnabled()); boolean directIOSupported = false; - try + // File.getBlockSize creates directories/files tools may not have permissions for + if (!toolInitialized) { - String commitLogLocation = getCommitLogLocation(); + try + { + String commitLogLocation = getCommitLogLocation(); - if (commitLogLocation == null) - throw new ConfigurationException("commitlog_directory must be specified", false); + if (commitLogLocation == null) + throw new ConfigurationException("commitlog_directory must be specified", false); - File commitLogLocationDir = new File(commitLogLocation); - PathUtils.createDirectoriesIfNotExists(commitLogLocationDir.toPath()); - directIOSupported = FileUtils.getBlockSize(commitLogLocationDir) > 0; - } - catch (IOError | ConfigurationException ex) - { - throw ex; - } - catch (RuntimeException e) - { - logger.warn("Unable to determine block size for commit log directory: {}", e.getMessage()); + File commitLogLocationDir = new File(commitLogLocation); + PathUtils.createDirectoriesIfNotExists(commitLogLocationDir.toPath()); + directIOSupported = FileUtils.getBlockSize(commitLogLocationDir) > 0; + } + catch (IOError | ConfigurationException ex) + { + throw ex; + } + catch (RuntimeException e) + { + logger.warn("Unable to determine block size for commit log directory: {}", e.getMessage()); + } } if (providedDiskAccessMode == DiskAccessMode.auto) From 0b6ae2673ec6a1f0af0bfa7c57a54f72add5a27e Mon Sep 17 00:00:00 2001 From: Sam Tunnicliffe <samt@apache.org> Date: Wed, 5 Feb 2025 12:29:00 +0000 Subject: [PATCH 120/225] Minimize expensive reads during authz flow in 3.0/3.11 Patch by Sam Tunnicliffe; reviewed by Francisco Guerrero for CASSANDRA-20293 --- CHANGES.txt | 1 + src/java/org/apache/cassandra/service/ClientState.java | 7 ++++--- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 38d003acf6f4..1ae082009cc5 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.32 +* Minimise expensive reads during authz flow in 3.0/3.11 (CASSANDRA-20293) 3.0.31 diff --git a/src/java/org/apache/cassandra/service/ClientState.java b/src/java/org/apache/cassandra/service/ClientState.java index 8ab6887c5be2..9fefb7c14126 100644 --- a/src/java/org/apache/cassandra/service/ClientState.java +++ b/src/java/org/apache/cassandra/service/ClientState.java @@ -364,9 +364,9 @@ public void ensureHasPermission(Permission perm, IResource resource) throws Unau if (((FunctionResource)resource).getKeyspace().equals(SystemKeyspace.NAME)) return; - if (resource instanceof DataResource && !(user.isSuper() || user.isSystem())) + if (resource instanceof DataResource) { - DataResource dataResource = (DataResource)resource; + DataResource dataResource = (DataResource) resource; if (!dataResource.isRootLevel()) { String keyspace = dataResource.getKeyspace(); @@ -376,7 +376,8 @@ public void ensureHasPermission(Permission perm, IResource resource) throws Unau // see the Permission enum for details. if (Schema.isSystemKeyspace(keyspace)) { - ensurePermissionOnResourceChain(perm, Resources.chain(dataResource, IResource::hasParent)); + if (!(user.isSystem() || user.isSuper())) + ensurePermissionOnResourceChain(perm, Resources.chain(dataResource, IResource::hasParent)); return; } } From c559cf89624802fdb770e6ea962b45f102e48224 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Thu, 6 Feb 2025 07:20:58 +0100 Subject: [PATCH 121/225] Prepare debian changelog for 3.0.32 --- debian/changelog | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/debian/changelog b/debian/changelog index fca85e2dba24..77e8fce50880 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,8 +1,8 @@ -cassandra (3.0.32) UNRELEASED; urgency=medium +cassandra (3.0.32) unstable; urgency=medium * New release - -- Stefan Miklosovic <smiklosovic@apache.org> Tue, 04 Feb 2025 09:43:30 +0100 + -- Stefan Miklosovic <smiklosovic@apache.org> Thu, 06 Feb 2025 07:20:16 +0100 cassandra (3.0.31) unstable; urgency=medium From ecbafb52750f1a2ab8523cbd7cc88c760dd3632b Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Thu, 6 Feb 2025 07:41:54 +0100 Subject: [PATCH 122/225] Prepare debian changelog for 3.11.19 --- debian/changelog | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/debian/changelog b/debian/changelog index f4c41c63d8b8..3912c63bdc46 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,8 +1,8 @@ -cassandra (3.11.19) UNRELEASED; urgency=medium +cassandra (3.11.19) unstable; urgency=medium * New release - -- Stefan Miklosovic <smiklosovic@apache.org> Tue, 04 Feb 2025 09:43:30 +0100 + -- Stefan Miklosovic <smiklosovic@apache.org> Thu, 06 Feb 2025 07:41:47 +0100 cassandra (3.11.18) unstable; urgency=medium From 066c489d764d54fc574d77c9006eb9d6399d4e93 Mon Sep 17 00:00:00 2001 From: Sam Tunnicliffe <samt@apache.org> Date: Thu, 6 Feb 2025 09:10:51 +0000 Subject: [PATCH 123/225] Tighten up permissions on system keyspaces * Restrict which permissions can be granted on system keyspaces * Ensure that GRANT... ON ALL KEYSPACES excludes system keyspaces * Add system_traces to the always readable set Patch by Sam Tunnicliffe and Francisco Guerrero; reviewed by Sam Tunnicliffe and Francisco Guerrero for CASSANDRA-20090 Note: this is a re-application of the commit to cassandra-4.0 only, to fix a bad merge the first time around. Co-authored-by: Francisco Guerrero <frankgh@apache.org> --- CHANGES.txt | 1 + .../org/apache/cassandra/auth/Permission.java | 7 + .../org/apache/cassandra/auth/Resources.java | 28 +- .../statements/GrantPermissionsStatement.java | 21 + .../cassandra/schema/SchemaConstants.java | 10 + .../apache/cassandra/service/ClientState.java | 61 ++- .../cassandra/auth/GrantAndRevokeTest.java | 359 ++++++++++++++++++ .../org/apache/cassandra/cql3/CQLTester.java | 2 +- 8 files changed, 470 insertions(+), 19 deletions(-) create mode 100644 test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 09a9bee44d97..db7080bc1646 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,5 +1,6 @@ 4.0.17 * Fix autocompletion for role names/user names (CASSANDRA-20175) + * Re-apply: Tighten up permission on system keyspaces (CASSANDRA-20040) 4.0.16 diff --git a/src/java/org/apache/cassandra/auth/Permission.java b/src/java/org/apache/cassandra/auth/Permission.java index d552280e64bd..11c7aeb05b70 100644 --- a/src/java/org/apache/cassandra/auth/Permission.java +++ b/src/java/org/apache/cassandra/auth/Permission.java @@ -66,4 +66,11 @@ public enum Permission public static final Set<Permission> ALL = Sets.immutableEnumSet(EnumSet.range(Permission.CREATE, Permission.EXECUTE)); public static final Set<Permission> NONE = ImmutableSet.of(); + + /** + * Set of Permissions which may never be granted on any system keyspace, or table in a system keyspace, to any role. + * (Only SELECT, DESCRIBE and ALTER may ever be granted). + */ + public static final Set<Permission> INVALID_FOR_SYSTEM_KEYSPACES = + Sets.immutableEnumSet(EnumSet.complementOf(EnumSet.of(Permission.SELECT, Permission.DESCRIBE, Permission.ALTER))); } diff --git a/src/java/org/apache/cassandra/auth/Resources.java b/src/java/org/apache/cassandra/auth/Resources.java index 653cd46e324b..28637106322c 100644 --- a/src/java/org/apache/cassandra/auth/Resources.java +++ b/src/java/org/apache/cassandra/auth/Resources.java @@ -19,6 +19,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.function.Predicate; import org.apache.cassandra.utils.Hex; @@ -27,18 +28,33 @@ public final class Resources /** * Construct a chain of resource parents starting with the resource and ending with the root. * - * @param resource The staring point. + * @param resource The starting point. * @return list of resource in the chain form start to the root. */ public static List<? extends IResource> chain(IResource resource) { - List<IResource> chain = new ArrayList<IResource>(); + return chain(resource, (r) -> true); + } + + /** + * Construct a chain of resource parents starting with the resource and ending with the root. Only resources which + * satisfy the supplied predicate will be included. + * + * @param resource The starting point. + * @param filter can be used to omit specific resources from the chain + * @return list of resource in the chain form start to the root. + */ + public static List<? extends IResource> chain(IResource resource, Predicate<IResource> filter) + { + + List<IResource> chain = new ArrayList<>(4); while (true) { - chain.add(resource); - if (!resource.hasParent()) - break; - resource = resource.getParent(); + if (filter.test(resource)) + chain.add(resource); + if (!resource.hasParent()) + break; + resource = resource.getParent(); } return chain; } diff --git a/src/java/org/apache/cassandra/cql3/statements/GrantPermissionsStatement.java b/src/java/org/apache/cassandra/cql3/statements/GrantPermissionsStatement.java index 3db20e384108..104cb8a949a2 100644 --- a/src/java/org/apache/cassandra/cql3/statements/GrantPermissionsStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/GrantPermissionsStatement.java @@ -17,16 +17,20 @@ */ package org.apache.cassandra.cql3.statements; +import java.util.Collections; import java.util.Set; import org.apache.cassandra.audit.AuditLogContext; import org.apache.cassandra.audit.AuditLogEntryType; +import org.apache.cassandra.auth.DataResource; import org.apache.cassandra.auth.IResource; import org.apache.cassandra.auth.Permission; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.RoleName; import org.apache.cassandra.exceptions.RequestExecutionException; import org.apache.cassandra.exceptions.RequestValidationException; +import org.apache.cassandra.exceptions.UnauthorizedException; +import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.service.ClientState; import org.apache.cassandra.transport.messages.ResultMessage; @@ -37,6 +41,23 @@ public GrantPermissionsStatement(Set<Permission> permissions, IResource resource super(permissions, resource, grantee); } + public void validate(ClientState state) throws RequestValidationException + { + super.validate(state); + if (resource instanceof DataResource) + { + DataResource data = (DataResource) resource; + // Only a subset of permissions can be granted on system keyspaces + if (!data.isRootLevel() + && SchemaConstants.isNonVirtualSystemKeyspace(data.getKeyspace()) + && !Collections.disjoint(permissions, Permission.INVALID_FOR_SYSTEM_KEYSPACES)) + { + throw new UnauthorizedException("Granting permissions on system keyspaces is strictly limited, " + + "this operation is not permitted"); + } + } + } + public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException { DatabaseDescriptor.getAuthorizer().grant(state.getUser(), permissions, resource, grantee); diff --git a/src/java/org/apache/cassandra/schema/SchemaConstants.java b/src/java/org/apache/cassandra/schema/SchemaConstants.java index 7b6b7de4906d..130227d84420 100644 --- a/src/java/org/apache/cassandra/schema/SchemaConstants.java +++ b/src/java/org/apache/cassandra/schema/SchemaConstants.java @@ -108,4 +108,14 @@ public static boolean isSystemKeyspace(String keyspaceName) || isReplicatedSystemKeyspace(keyspaceName) || isVirtualSystemKeyspace(keyspaceName); } + + /** + * @return whether or not the keyspace is a non-virtual, system keyspace + */ + public static boolean isNonVirtualSystemKeyspace(String keyspaceName) + { + final String lowercaseKeyspaceName = keyspaceName.toLowerCase(); + return LOCAL_SYSTEM_KEYSPACE_NAMES.contains(lowercaseKeyspaceName) + || REPLICATED_SYSTEM_KEYSPACE_NAMES.contains(lowercaseKeyspaceName); + } } diff --git a/src/java/org/apache/cassandra/service/ClientState.java b/src/java/org/apache/cassandra/service/ClientState.java index f76e7e3a4f80..adca72770966 100644 --- a/src/java/org/apache/cassandra/service/ClientState.java +++ b/src/java/org/apache/cassandra/service/ClientState.java @@ -21,11 +21,12 @@ import java.net.InetSocketAddress; import java.net.SocketAddress; import java.util.Arrays; -import java.util.HashSet; import java.util.Optional; +import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; +import com.google.common.collect.ImmutableSet; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,6 +48,7 @@ import org.apache.cassandra.exceptions.AuthenticationException; import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.exceptions.UnauthorizedException; +import org.apache.cassandra.tracing.TraceKeyspace; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.MD5Digest; @@ -58,29 +60,37 @@ public class ClientState { private static final Logger logger = LoggerFactory.getLogger(ClientState.class); - private static final Set<IResource> READABLE_SYSTEM_RESOURCES = new HashSet<>(); - private static final Set<IResource> PROTECTED_AUTH_RESOURCES = new HashSet<>(); - + public static final ImmutableSet<IResource> READABLE_SYSTEM_RESOURCES; + public static final ImmutableSet<IResource> PROTECTED_AUTH_RESOURCES; static { // We want these system cfs to be always readable to authenticated users since many tools rely on them // (nodetool, cqlsh, bulkloader, etc.) - for (String cf : Arrays.asList(SystemKeyspace.LOCAL, SystemKeyspace.LEGACY_PEERS, SystemKeyspace.PEERS_V2)) - READABLE_SYSTEM_RESOURCES.add(DataResource.table(SchemaConstants.SYSTEM_KEYSPACE_NAME, cf)); + ImmutableSet.Builder<IResource> readableBuilder = ImmutableSet.builder(); + for (String cf : Arrays.asList(SystemKeyspace.LOCAL, SystemKeyspace.LEGACY_PEERS, SystemKeyspace.PEERS_V2, + SystemKeyspace.LEGACY_SIZE_ESTIMATES, SystemKeyspace.TABLE_ESTIMATES)) + readableBuilder.add(DataResource.table(SchemaConstants.SYSTEM_KEYSPACE_NAME, cf)); // make all schema tables readable by default (required by the drivers) - SchemaKeyspaceTables.ALL.forEach(table -> READABLE_SYSTEM_RESOURCES.add(DataResource.table(SchemaConstants.SCHEMA_KEYSPACE_NAME, table))); + SchemaKeyspaceTables.ALL.forEach(table -> readableBuilder.add(DataResource.table(SchemaConstants.SCHEMA_KEYSPACE_NAME, table))); + + // make system_traces readable by all or else tracing will require explicit grants + readableBuilder.add(DataResource.table(SchemaConstants.TRACE_KEYSPACE_NAME, TraceKeyspace.EVENTS)); + readableBuilder.add(DataResource.table(SchemaConstants.TRACE_KEYSPACE_NAME, TraceKeyspace.SESSIONS)); // make all virtual schema tables readable by default as well - VirtualSchemaKeyspace.instance.tables().forEach(t -> READABLE_SYSTEM_RESOURCES.add(t.metadata().resource)); + VirtualSchemaKeyspace.instance.tables().forEach(t -> readableBuilder.add(t.metadata().resource)); + READABLE_SYSTEM_RESOURCES = readableBuilder.build(); + ImmutableSet.Builder<IResource> protectedBuilder = ImmutableSet.builder(); // neither clients nor tools need authentication/authorization if (DatabaseDescriptor.isDaemonInitialized()) { - PROTECTED_AUTH_RESOURCES.addAll(DatabaseDescriptor.getAuthenticator().protectedResources()); - PROTECTED_AUTH_RESOURCES.addAll(DatabaseDescriptor.getAuthorizer().protectedResources()); - PROTECTED_AUTH_RESOURCES.addAll(DatabaseDescriptor.getRoleManager().protectedResources()); + protectedBuilder.addAll(DatabaseDescriptor.getAuthenticator().protectedResources()); + protectedBuilder.addAll(DatabaseDescriptor.getAuthorizer().protectedResources()); + protectedBuilder.addAll(DatabaseDescriptor.getRoleManager().protectedResources()); } + PROTECTED_AUTH_RESOURCES = protectedBuilder.build(); } // Current user for the session @@ -384,9 +394,12 @@ private void ensurePermission(String keyspace, Permission perm, DataResource res preventSystemKSSchemaModification(keyspace, resource, perm); + // Some system data is always readable if ((perm == Permission.SELECT) && READABLE_SYSTEM_RESOURCES.contains(resource)) return; + // Modifications to any resource upon which the authenticator, authorizer or role manager depend should not be + // be performed by users if (PROTECTED_AUTH_RESOURCES.contains(resource)) if ((perm == Permission.CREATE) || (perm == Permission.ALTER) || (perm == Permission.DROP)) throw new UnauthorizedException(String.format("%s schema is protected", resource)); @@ -403,6 +416,24 @@ public void ensurePermission(Permission perm, IResource resource) if (((FunctionResource)resource).getKeyspace().equals(SchemaConstants.SYSTEM_KEYSPACE_NAME)) return; + if (resource instanceof DataResource && !(user.isSuper() || user.isSystem())) + { + DataResource dataResource = (DataResource)resource; + if (!dataResource.isRootLevel()) + { + String keyspace = dataResource.getKeyspace(); + // A user may have permissions granted on ALL KEYSPACES, but this should exclude system keyspaces. Any + // permission on those keyspaces or their tables must be granted to the user either explicitly or + // transitively. The set of grantable permissions for system keyspaces is further limited, + // see the Permission enum for details. + if (SchemaConstants.isSystemKeyspace(keyspace)) + { + ensurePermissionOnResourceChain(perm, Resources.chain(dataResource, IResource::hasParent)); + return; + } + } + } + ensurePermissionOnResourceChain(perm, resource); } @@ -425,7 +456,13 @@ public void ensurePermission(Permission permission, Function function) private void ensurePermissionOnResourceChain(Permission perm, IResource resource) { - for (IResource r : Resources.chain(resource)) + ensurePermissionOnResourceChain(perm, Resources.chain(resource)); + } + + private void ensurePermissionOnResourceChain(Permission perm, List<? extends IResource> resources) + { + IResource resource = resources.get(0); + for (IResource r : resources) if (authorize(r).contains(perm)) return; diff --git a/test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java b/test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java new file mode 100644 index 000000000000..4ce56721d74d --- /dev/null +++ b/test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java @@ -0,0 +1,359 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.auth; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Set; + +import com.google.common.collect.Iterables; +import org.junit.After; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.PlainTextAuthProvider; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.exceptions.UnauthorizedException; +import org.apache.cassandra.Util; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.db.SystemKeyspace; +import org.apache.cassandra.utils.Pair; + +import static java.lang.String.format; +import static org.apache.cassandra.schema.SchemaConstants.LOCAL_SYSTEM_KEYSPACE_NAMES; +import static org.apache.cassandra.schema.SchemaConstants.REPLICATED_SYSTEM_KEYSPACE_NAMES; + +public class GrantAndRevokeTest extends CQLTester +{ + private static final String user = "user"; + private static final String pass = "12345"; + private static final Pair<String, String> USER = Pair.create(user, pass); + private static final Pair<String, String> SUPERUSER = Pair.create("cassandra", "cassandra"); + private static int counter = 0; + + @BeforeClass + public static void setUpClass() + { + DatabaseDescriptor.setAuthenticator(new PasswordAuthenticator()); + DatabaseDescriptor.setAuthorizer(new CassandraAuthorizer()); + DatabaseDescriptor.setRoleManager(new CassandraRoleManager()); + DatabaseDescriptor.setPermissionsValidity(0); + CQLTester.setUpClass(); + requireNetworkWithoutDriver(); + DatabaseDescriptor.getRoleManager().setup(); + DatabaseDescriptor.getAuthenticator().setup(); + DatabaseDescriptor.getAuthorizer().setup(); + } + + @After + public void tearDown() throws Throwable + { + session(SUPERUSER).execute("DROP ROLE " + user); + Roles.clearCache(); + } + + private Session session(Pair<String, String> credentials) + { + Cluster cluster = Cluster.builder() + .addContactPoints(nativeAddr) + .withClusterName("Test Cluster " + counter++) + .withPort(nativePort) + .withAuthProvider(new PlainTextAuthProvider(credentials.left, credentials.right)) + .build(); + + return cluster.connect(); + } + + @Test + public void testGrantedKeyspace() throws Throwable + { + Session superuser = session(SUPERUSER); + superuser.execute(String.format("CREATE ROLE %s WITH LOGIN = TRUE AND password='%s'", user, pass)); + superuser.execute("GRANT CREATE ON KEYSPACE " + KEYSPACE_PER_TEST + " TO " + user); + String table = KEYSPACE_PER_TEST + '.' + createTable(KEYSPACE_PER_TEST, "CREATE TABLE %s (pk int, ck int, val int, val_2 text, PRIMARY KEY (pk, ck))"); + String index = KEYSPACE_PER_TEST + ".idx_01"; + createIndex(KEYSPACE_PER_TEST, "CREATE INDEX idx_01 ON %s (val_2)"); + String type = KEYSPACE_PER_TEST + '.' + createType(KEYSPACE_PER_TEST, "CREATE TYPE %s (a int, b text)"); + String mv = KEYSPACE_PER_TEST + ".ks_mv_01"; + superuser.execute("CREATE MATERIALIZED VIEW " + mv + " AS SELECT * FROM " + table + " WHERE val IS NOT NULL AND pk IS NOT NULL AND ck IS NOT NULL PRIMARY KEY (val, pk, ck)"); + + Session nonsuperuser = session(USER); + // ALTER and DROP tables created by somebody else + // Spin assert for effective auth changes. + final String spinAssertTable = table; + Util.spinAssertEquals(false, () -> { + try + { + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table " + spinAssertTable + "> or any of its parents", + formatQuery(KEYSPACE_PER_TEST, "INSERT INTO %s (pk, ck, val, val_2) VALUES (1, 1, 1, '1')")); + } + catch(Throwable e) + { + return true; + } + return false; + }, 10); + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table " + table + "> or any of its parents", + formatQuery(KEYSPACE_PER_TEST, "UPDATE %s SET val = 1 WHERE pk = 1 AND ck = 1")); + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table " + table + "> or any of its parents", + formatQuery(KEYSPACE_PER_TEST, "DELETE FROM %s WHERE pk = 1 AND ck = 2")); + assertUnauthorizedQuery(nonsuperuser, "User user has no SELECT permission on <table " + table + "> or any of its parents", + formatQuery(KEYSPACE_PER_TEST, "SELECT * FROM %s WHERE pk = 1 AND ck = 1")); + assertUnauthorizedQuery(nonsuperuser, "User user has no SELECT permission on <table " + table + "> or any of its parents", + "SELECT * FROM " + mv + " WHERE val = 1 AND pk = 1 AND ck = 1"); + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table " + table + "> or any of its parents", + formatQuery(KEYSPACE_PER_TEST, "TRUNCATE TABLE %s")); + assertUnauthorizedQuery(nonsuperuser, "User user has no ALTER permission on <table " + table + "> or any of its parents", + formatQuery(KEYSPACE_PER_TEST, "ALTER TABLE %s ADD val_3 int")); + assertUnauthorizedQuery(nonsuperuser, "User user has no DROP permission on <table " + table + "> or any of its parents", + formatQuery(KEYSPACE_PER_TEST, "DROP TABLE %s")); + assertUnauthorizedQuery(nonsuperuser, "User user has no ALTER permission on <keyspace " + KEYSPACE_PER_TEST + "> or any of its parents", + "ALTER TYPE " + type + " ADD c bigint"); + assertUnauthorizedQuery(nonsuperuser, "User user has no DROP permission on <keyspace " + KEYSPACE_PER_TEST + "> or any of its parents", + "DROP TYPE " + type); + assertUnauthorizedQuery(nonsuperuser, "User user has no ALTER permission on <table " + table + "> or any of its parents", + "DROP MATERIALIZED VIEW " + mv); + assertUnauthorizedQuery(nonsuperuser, "User user has no ALTER permission on <table " + table + "> or any of its parents", + "DROP INDEX " + index); + + superuser.execute("GRANT ALTER ON KEYSPACE " + KEYSPACE_PER_TEST + " TO " + user); + superuser.execute("GRANT DROP ON KEYSPACE " + KEYSPACE_PER_TEST + " TO " + user); + superuser.execute("GRANT SELECT ON KEYSPACE " + KEYSPACE_PER_TEST + " TO " + user); + superuser.execute("GRANT MODIFY ON KEYSPACE " + KEYSPACE_PER_TEST + " TO " + user); + // Spin assert for effective auth changes. + Util.spinAssertEquals(false, () -> { + try + { + nonsuperuser.execute("ALTER KEYSPACE " + KEYSPACE_PER_TEST + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}"); + } + catch(Throwable e) + { + return true; + } + return false; + }, 10); + + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "INSERT INTO %s (pk, ck, val, val_2) VALUES (1, 1, 1, '1')")); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "UPDATE %s SET val = 1 WHERE pk = 1 AND ck = 1")); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "DELETE FROM %s WHERE pk = 1 AND ck = 2")); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "SELECT * FROM %s WHERE pk = 1 AND ck = 1")); + nonsuperuser.execute("SELECT * FROM " + mv + " WHERE val = 1 AND pk = 1"); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "TRUNCATE TABLE %s")); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "ALTER TABLE %s ADD val_3 int")); + nonsuperuser.execute("DROP MATERIALIZED VIEW " + mv); + nonsuperuser.execute("DROP INDEX " + index); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "DROP TABLE %s")); + nonsuperuser.execute("ALTER TYPE " + type + " ADD c bigint"); + nonsuperuser.execute("DROP TYPE " + type); + + // calling creatTableName to create a new table name that will be used by the formatQuery + table = createTableName(); + type = KEYSPACE_PER_TEST + "." + createTypeName(); + mv = KEYSPACE_PER_TEST + ".ks_mv_02"; + nonsuperuser.execute("CREATE TYPE " + type + " (a int, b text)"); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "CREATE TABLE %s (pk int, ck int, val int, val_2 text, PRIMARY KEY (pk, ck))")); + nonsuperuser.execute("CREATE MATERIALIZED VIEW " + mv + " AS SELECT * FROM " + table + " WHERE val IS NOT NULL AND pk IS NOT NULL AND ck IS NOT NULL PRIMARY KEY (val, pk, ck)"); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "INSERT INTO %s (pk, ck, val, val_2) VALUES (1, 1, 1, '1')")); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "UPDATE %s SET val = 1 WHERE pk = 1 AND ck = 1")); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "DELETE FROM %s WHERE pk = 1 AND ck = 2")); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "SELECT * FROM %s WHERE pk = 1 AND ck = 1")); + nonsuperuser.execute("SELECT * FROM " + mv + " WHERE val = 1 AND pk = 1"); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "TRUNCATE TABLE %s")); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "ALTER TABLE %s ADD val_3 int")); + nonsuperuser.execute("DROP MATERIALIZED VIEW " + mv); + nonsuperuser.execute(formatQuery(KEYSPACE_PER_TEST, "DROP TABLE %s")); + nonsuperuser.execute("ALTER TYPE " + type + " ADD c bigint"); + nonsuperuser.execute("DROP TYPE " + type); + + superuser.execute("REVOKE ALTER ON KEYSPACE " + KEYSPACE_PER_TEST + " FROM " + user); + superuser.execute("REVOKE DROP ON KEYSPACE " + KEYSPACE_PER_TEST + " FROM " + user); + superuser.execute("REVOKE SELECT ON KEYSPACE " + KEYSPACE_PER_TEST + " FROM " + user); + superuser.execute("REVOKE MODIFY ON KEYSPACE " + KEYSPACE_PER_TEST + " FROM " + user); + + table = KEYSPACE_PER_TEST + "." + createTable(KEYSPACE_PER_TEST, "CREATE TABLE %s (pk int, ck int, val int, val_2 text, PRIMARY KEY (pk, ck))"); + type = KEYSPACE_PER_TEST + "." + createType(KEYSPACE_PER_TEST, "CREATE TYPE %s (a int, b text)"); + index = KEYSPACE_PER_TEST + ".idx_02"; + createIndex(KEYSPACE_PER_TEST, "CREATE INDEX idx_02 ON %s (val_2)"); + mv = KEYSPACE_PER_TEST + ".ks_mv_03"; + superuser.execute("CREATE MATERIALIZED VIEW " + mv + " AS SELECT * FROM " + table + " WHERE val IS NOT NULL AND pk IS NOT NULL AND ck IS NOT NULL PRIMARY KEY (val, pk, ck)"); + + // Spin assert for effective auth changes. + final String spinAssertTable2 = table; + Util.spinAssertEquals(false, () -> { + try + { + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table " + spinAssertTable2 + "> or any of its parents", + "INSERT INTO " + spinAssertTable2 + " (pk, ck, val, val_2) VALUES (1, 1, 1, '1')"); + } + catch(Throwable e) + { + return true; + } + return false; + }, 10); + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table " + table + "> or any of its parents", + "UPDATE " + table + " SET val = 1 WHERE pk = 1 AND ck = 1"); + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table " + table + "> or any of its parents", + "DELETE FROM " + table + " WHERE pk = 1 AND ck = 2"); + assertUnauthorizedQuery(nonsuperuser, "User user has no SELECT permission on <table " + table + "> or any of its parents", + "SELECT * FROM " + table + " WHERE pk = 1 AND ck = 1"); + assertUnauthorizedQuery(nonsuperuser, "User user has no SELECT permission on <table " + table + "> or any of its parents", + "SELECT * FROM " + mv + " WHERE val = 1 AND pk = 1 AND ck = 1"); + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table " + table + "> or any of its parents", + "TRUNCATE TABLE " + table); + assertUnauthorizedQuery(nonsuperuser, "User user has no ALTER permission on <table " + table + "> or any of its parents", + formatQuery(KEYSPACE_PER_TEST, "ALTER TABLE " + table + " ADD val_3 int")); + assertUnauthorizedQuery(nonsuperuser, "User user has no DROP permission on <table " + table + "> or any of its parents", + formatQuery(KEYSPACE_PER_TEST, "DROP TABLE " + table)); + assertUnauthorizedQuery(nonsuperuser, "User user has no ALTER permission on <keyspace " + KEYSPACE_PER_TEST + "> or any of its parents", + "ALTER TYPE " + type + " ADD c bigint"); + assertUnauthorizedQuery(nonsuperuser, "User user has no DROP permission on <keyspace " + KEYSPACE_PER_TEST + "> or any of its parents", + "DROP TYPE " + type); + assertUnauthorizedQuery(nonsuperuser, "User user has no ALTER permission on <table " + table + "> or any of its parents", + "DROP MATERIALIZED VIEW " + mv); + assertUnauthorizedQuery(nonsuperuser, "User user has no ALTER permission on <table " + table + "> or any of its parents", + "DROP INDEX " + index); + } + + + @Test + public void testSpecificGrantsOnSystemKeyspaces() throws Throwable + { + // Granting specific permissions on system keyspaces should not be allowed if those permissions include any from + // the denylist Permission.INVALID_FOR_SYSTEM_KEYSPACES. By this definition, GRANT ALL on any system keyspace, + // or a table within one, should be rejected. + Session superuser = session(SUPERUSER); + superuser.execute("CREATE ROLE '" + user + "'"); + String responseMsg = "Granting permissions on system keyspaces is strictly limited, this operation is not permitted"; + for (String keyspace : Iterables.concat(LOCAL_SYSTEM_KEYSPACE_NAMES, REPLICATED_SYSTEM_KEYSPACE_NAMES)) + { + assertUnauthorizedQuery(superuser, responseMsg, format("GRANT ALL PERMISSIONS ON KEYSPACE %s TO %s", keyspace, user)); + DataResource keyspaceResource = DataResource.keyspace(keyspace); + for (Permission p : keyspaceResource.applicablePermissions()) + maybeRejectGrant(superuser, p, responseMsg, format("GRANT %s ON KEYSPACE %s TO %s", p.name(), keyspace, user)); + + for (TableMetadata table : Schema.instance.getKeyspaceMetadata(keyspace).tables) + { + DataResource tableResource = DataResource.table(keyspace, table.name); + assertUnauthorizedQuery(superuser, responseMsg, format("GRANT ALL PERMISSIONS ON %s.\"%s\" TO %s", table.keyspace, table.name, user)); + for (Permission p : tableResource.applicablePermissions()) + maybeRejectGrant(superuser, p, responseMsg, format("GRANT %s ON %s.\"%s\" TO %s", p.name(), table.keyspace, table.name, user)); + } + } + } + + @Test + public void testGrantOnAllKeyspaces() throws Throwable + { + // Granting either specific or ALL permissions on ALL KEYSPACES is allowed, however these permissions are + // effective for non-system keyspaces only. If for any reason it is necessary to modify permissions on + // on a system keyspace, it must be done using keyspace specific grant statements. + Session superuser = session(SUPERUSER); + superuser.execute(String.format("CREATE ROLE %s WITH LOGIN = TRUE AND password='%s'", user, pass)); + superuser.execute(String.format("ALTER KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", SchemaConstants.TRACE_KEYSPACE_NAME)); + superuser.execute("CREATE KEYSPACE user_keyspace WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}"); + superuser.execute("CREATE TABLE user_keyspace.t1 (k int PRIMARY KEY)"); + + Session nonsuperuser = session(USER); + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table user_keyspace.t1> or any of its parents", + "INSERT INTO user_keyspace.t1 (k) VALUES (0)"); + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table system.local> or any of its parents", + "INSERT INTO system.local(key) VALUES ('invalid')"); + + superuser.execute(format("GRANT MODIFY ON ALL KEYSPACES TO %s", user)); + // User now has write permission on non-system keyspaces only + nonsuperuser.execute("INSERT INTO user_keyspace.t1 (k) VALUES (0)"); + assertUnauthorizedQuery(nonsuperuser, "User user has no MODIFY permission on <table system.local> or any of its parents", + "INSERT INTO system.local(key) VALUES ('invalid')"); + + // A non-superuser only has read access to a pre-defined set of system tables and all system_schema/traces + // tables and granting ALL permissions on ALL keyspaces also does not affect this. + maybeReadSystemTables(nonsuperuser, false); + superuser.execute(format("GRANT ALL PERMISSIONS ON ALL KEYSPACES TO %s", user)); + maybeReadSystemTables(nonsuperuser, false); + + // A superuser can still read system tables + maybeReadSystemTables(superuser, true); + // and also write to them, though this is still strongly discouraged + superuser.execute("INSERT INTO system.peers(peer, data_center) VALUES ('127.0.100.100', 'invalid_dc')"); + + } + + private void maybeReadSystemTables(Session session, boolean isSuper) throws Throwable + { + Set<String> readableKeyspaces = new HashSet<>(Arrays.asList(SchemaConstants.SCHEMA_KEYSPACE_NAME, SchemaConstants.TRACE_KEYSPACE_NAME)); + Set<String> readableSystemTables = new HashSet<>(Arrays.asList(SystemKeyspace.LOCAL, + SystemKeyspace.PEERS_V2, + SystemKeyspace.LEGACY_PEERS, + SystemKeyspace.LEGACY_SIZE_ESTIMATES, + SystemKeyspace.TABLE_ESTIMATES)); + + for (String keyspace : Iterables.concat(LOCAL_SYSTEM_KEYSPACE_NAMES, REPLICATED_SYSTEM_KEYSPACE_NAMES)) + { + for (TableMetadata table : Schema.instance.getKeyspaceMetadata(keyspace).tables) + { + if (isSuper || (readableKeyspaces.contains(keyspace) || (keyspace.equals(SchemaConstants.SYSTEM_KEYSPACE_NAME) && readableSystemTables.contains(table.name)))) + { + session.execute(format("SELECT * FROM %s.\"%s\" LIMIT 1", table.keyspace, table.name)); + } + else + { + assertUnauthorizedQuery(session, format("User %s has no SELECT permission on %s or any of its parents", user, DataResource.table(table.keyspace, table.name)), + format("SELECT * FROM %s.\"%s\" LIMIT 1", table.keyspace, table.name)); + } + } + } + } + + private void maybeRejectGrant(Session session, Permission p, String errorResponse, String grant) throws Throwable + { + if (Permission.INVALID_FOR_SYSTEM_KEYSPACES.contains(p)) + assertUnauthorizedQuery(session, errorResponse, grant); + else + session.execute(grant); + } + + private void assertUnauthorizedQuery(Session session, String errorMessage, String query) throws Throwable + { + try + { + session.execute(query); + Assert.fail("Query should be invalid but no error was thrown. Query is: " + query); + } + catch (Exception e) + { + if (!UnauthorizedException.class.isAssignableFrom(e.getClass())) + { + Assert.fail("Query should be invalid but wrong error was thrown. " + + "Expected: " + UnauthorizedException.class.getName() + ", got: " + e.getClass().getName() + ". " + + "Query is: " + query); + } + if (errorMessage != null) + { + assertMessageContains(errorMessage, e); + } + } + } +} diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index 48fddec68ab3..d568d13e1174 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -1561,7 +1561,7 @@ protected void assertInvalidSyntaxMessage(String errorMessage, String query, Obj * @param text the text that the exception message must contains * @param e the exception to check */ - private static void assertMessageContains(String text, Exception e) + protected static void assertMessageContains(String text, Exception e) { Assert.assertTrue("Expected error message to contain '" + text + "', but got '" + e.getMessage() + "'", e.getMessage().contains(text)); From 0354c915a9f8be6ab671af5ff93b8178e6f2e76f Mon Sep 17 00:00:00 2001 From: Mick Semb Wever <mck@apache.org> Date: Thu, 6 Feb 2025 10:45:07 +0100 Subject: [PATCH 124/225] Prepare debian changelog for 4.0.17 --- debian/changelog | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/debian/changelog b/debian/changelog index 9edc99c40b6b..9e7f49c8b25f 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,8 +1,8 @@ -cassandra (4.0.17) UNRELEASED; urgency=medium +cassandra (4.0.17) unstable; urgency=medium * New release - -- Stefan Miklosovic <smiklosovic@apache.org> Tue, 04 Feb 2025 09:43:30 +0100 + -- Mick Semb Wever <mck@apache.org> Thu, 06 Feb 2025 10:44:49 +0100 cassandra (4.0.16) unstable; urgency=medium From b6baa8bde368fc33f5b99c721b619deadb8f8ab7 Mon Sep 17 00:00:00 2001 From: Brandon Williams <brandonwilliams@apache.org> Date: Thu, 6 Feb 2025 06:05:44 -0600 Subject: [PATCH 125/225] Ninja fix ticket number in changes --- CHANGES.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index db7080bc1646..d8cacc666356 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,6 +1,6 @@ 4.0.17 * Fix autocompletion for role names/user names (CASSANDRA-20175) - * Re-apply: Tighten up permission on system keyspaces (CASSANDRA-20040) + * Re-apply: Tighten up permission on system keyspaces (CASSANDRA-20090) 4.0.16 From 953ab6cf64088614ada26e05d664b6a25b1d561d Mon Sep 17 00:00:00 2001 From: Caleb Rackliffe <calebrackliffe@gmail.com> Date: Tue, 7 Jan 2025 16:39:34 -0600 Subject: [PATCH 126/225] Avoid possible consistency violations for SAI intersection queries over repaired index matches and multiple non-indexed column matches patch by Caleb Rackliffe; reviewed by David Capwell for CASSANDRA-20189 --- CHANGES.txt | 1 + .../apache/cassandra/db/filter/RowFilter.java | 14 ++- .../cassandra/index/sai/plan/FilterTree.java | 74 +++++++++++----- .../cassandra/index/sai/plan/Operation.java | 87 ++++++++++++++++--- .../index/sai/plan/QueryController.java | 2 +- .../index/sai/utils/IndexTermType.java | 5 ++ .../test/sai/StrictFilteringTest.java | 57 ++++++++++++ .../index/sai/plan/OperationTest.java | 5 +- 8 files changed, 204 insertions(+), 41 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 769e4dddbee4..13ae3fddc29d 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.4 + * Avoid possible consistency violations for SAI intersection queries over repaired index matches and multiple non-indexed column matches (CASSANDRA-20189) * Skip check for DirectIO when initializing tools (CASSANDRA-20289) * Avoid under-skipping during intersections when an iterator has mixed STATIC and WIDE keys (CASSANDRA-20258) * Correct the default behavior of compareTo() when comparing WIDE and STATIC PrimaryKeys (CASSANDRA-20238) diff --git a/src/java/org/apache/cassandra/db/filter/RowFilter.java b/src/java/org/apache/cassandra/db/filter/RowFilter.java index 0742f4ee9fa8..483c1631648e 100644 --- a/src/java/org/apache/cassandra/db/filter/RowFilter.java +++ b/src/java/org/apache/cassandra/db/filter/RowFilter.java @@ -169,11 +169,21 @@ public boolean isStrict() } /** - * @return true if this filter contains an intersection on two or more mutable columns + * @return true if this filter contains an intersection on either any static column or two regular mutable columns */ public boolean isMutableIntersection() { - return expressions.stream().filter(e -> !e.column.isPrimaryKeyColumn()).count() > 1; + int count = 0; + for (Expression e : expressions) + { + if (e.column.isStatic() && expressions.size() > 1) + return true; + + if (!e.column.isPrimaryKeyColumn()) + if (++count > 1) + return true; + } + return false; } /** diff --git a/src/java/org/apache/cassandra/index/sai/plan/FilterTree.java b/src/java/org/apache/cassandra/index/sai/plan/FilterTree.java index 4107fad2d271..15ea273145e0 100644 --- a/src/java/org/apache/cassandra/index/sai/plan/FilterTree.java +++ b/src/java/org/apache/cassandra/index/sai/plan/FilterTree.java @@ -23,8 +23,6 @@ import java.util.List; import java.util.ListIterator; -import com.google.common.collect.ListMultimap; - import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.rows.Row; import org.apache.cassandra.index.sai.QueryContext; @@ -44,12 +42,12 @@ public class FilterTree { protected final BooleanOperator baseOperator; - protected final ListMultimap<ColumnMetadata, Expression> expressions; + protected final Operation.Expressions expressions; protected final List<FilterTree> children = new ArrayList<>(); private final boolean isStrict; private final QueryContext context; - FilterTree(BooleanOperator baseOperator, ListMultimap<ColumnMetadata, Expression> expressions, boolean isStrict, QueryContext context) + FilterTree(BooleanOperator baseOperator, Operation.Expressions expressions, boolean isStrict, QueryContext context) { this.baseOperator = baseOperator; this.expressions = expressions; @@ -67,7 +65,7 @@ void addChild(FilterTree child) */ public boolean restrictsNonStaticRow() { - for (ColumnMetadata column : expressions.keySet()) + for (ColumnMetadata column : expressions.columns()) if (!column.isStatic()) return true; @@ -98,7 +96,12 @@ private boolean localSatisfiedBy(DecoratedKey key, Row row, Row staticRow) BooleanOperator localOperator = (isStrict || !context.hasUnrepairedMatches) ? baseOperator : BooleanOperator.OR; boolean result = localOperator == BooleanOperator.AND; - Iterator<ColumnMetadata> columnIterator = expressions.keySet().iterator(); + // If all matches on indexed columns are repaired, strict filtering is not allowed, and there are multiple + // unindexed column expressions, isolate the expressions on unindexed columns and union their results: + boolean isolateUnindexed = !context.hasUnrepairedMatches && !isStrict && expressions.hasMultipleUnindexedColumns(); + boolean unindexedResult = false; + + Iterator<ColumnMetadata> columnIterator = expressions.columns().iterator(); while (columnIterator.hasNext()) { ColumnMetadata column = columnIterator.next(); @@ -106,38 +109,65 @@ private boolean localSatisfiedBy(DecoratedKey key, Row row, Row staticRow) // If there is a column with multiple expressions that can mean an OR, or (in the case of map // collections) it can mean different map indexes. - List<Expression> filters = expressions.get(column); + List<Expression> filters = expressions.expressionsFor(column); // We do a reverse iteration over the filters because NOT_EQ operations will be at the end // of the filter list, and we want to check them first. ListIterator<Expression> filterIterator = filters.listIterator(filters.size()); - while (filterIterator.hasPrevious()) + + if (isolateUnindexed && expressions.isUnindexed(column)) { - Expression filter = filterIterator.previous(); + // If we isolate unindexed column expressions, we're implicitly calculating the union of those + // expressions. Once we've matched on any column, we can skip the rest, if any exist. + if (unindexedResult) + continue; - if (filter.getIndexTermType().isNonFrozenCollection()) + while (filterIterator.hasPrevious()) { - Iterator<ByteBuffer> valueIterator = filter.getIndexTermType().valuesOf(localRow, now); - result = localOperator.apply(result, collectionMatch(valueIterator, filter)); + Expression filter = filterIterator.previous(); + unindexedResult = applyFilter(key, now, BooleanOperator.OR, unindexedResult, localRow, filter); } - else + } + else + { + while (filterIterator.hasPrevious()) { - ByteBuffer value = filter.getIndexTermType().valueOf(key, localRow, now); - result = localOperator.apply(result, singletonMatch(value, filter)); - } + Expression filter = filterIterator.previous(); + result = applyFilter(key, now, localOperator, result, localRow, filter); - // If the operation is an AND then exit early if we get a single false - if ((localOperator == BooleanOperator.AND) && !result) - return false; + // If the operation is an AND then exit early if we get a single false + if ((localOperator == BooleanOperator.AND) && !result) + return false; - // If the operation is an OR then exit early if we get a single true - if (localOperator == BooleanOperator.OR && result) - return true; + // If the operation is an OR then exit early if we get a single true + if (localOperator == BooleanOperator.OR && result) + return true; + } } } + + if (isolateUnindexed) + // If we had to isolate the unindexed column expressions, combine with the indexed column result. Note that + // the indexed result must be true at this point if it was evaluated with the AND operator: + return localOperator == BooleanOperator.AND ? unindexedResult : result || unindexedResult; + return result; } + private boolean applyFilter(DecoratedKey key, long now, BooleanOperator operator, boolean result, Row row, Expression expression) + { + if (expression.getIndexTermType().isNonFrozenCollection()) + { + Iterator<ByteBuffer> valueIterator = expression.getIndexTermType().valuesOf(row, now); + return operator.apply(result, collectionMatch(valueIterator, expression)); + } + else + { + ByteBuffer value = expression.getIndexTermType().valueOf(key, row, now); + return operator.apply(result, singletonMatch(value, expression)); + } + } + private boolean singletonMatch(ByteBuffer value, Expression filter) { return value != null && filter.isSatisfiedBy(value); diff --git a/src/java/org/apache/cassandra/index/sai/plan/Operation.java b/src/java/org/apache/cassandra/index/sai/plan/Operation.java index 5b1824184323..eb45722af54b 100644 --- a/src/java/org/apache/cassandra/index/sai/plan/Operation.java +++ b/src/java/org/apache/cassandra/index/sai/plan/Operation.java @@ -20,8 +20,11 @@ import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.List; +import java.util.Set; import java.util.function.BiFunction; import java.util.stream.Collectors; @@ -63,11 +66,58 @@ public boolean apply(boolean a, boolean b) } } + public static class Expressions + { + final ListMultimap<ColumnMetadata, Expression> expressions; + final Set<ColumnMetadata> unindexedColumns; + + Expressions(ListMultimap<ColumnMetadata, Expression> expressions, Set<ColumnMetadata> unindexedColumns) + { + this.expressions = expressions; + this.unindexedColumns = unindexedColumns; + } + + Set<ColumnMetadata> columns() + { + return expressions.keySet(); + } + + Collection<Expression> all() + { + return expressions.values(); + } + + List<Expression> expressionsFor(ColumnMetadata column) + { + return expressions.get(column); + } + + boolean isEmpty() + { + return expressions.isEmpty(); + } + + int size() + { + return expressions.size(); + } + + boolean isUnindexed(ColumnMetadata column) + { + return unindexedColumns.contains(column); + } + + boolean hasMultipleUnindexedColumns() + { + return unindexedColumns.size() > 1; + } + } + @VisibleForTesting - protected static ListMultimap<ColumnMetadata, Expression> buildIndexExpressions(QueryController queryController, - List<RowFilter.Expression> expressions) + protected static Expressions buildIndexExpressions(QueryController queryController, List<RowFilter.Expression> expressions) { ListMultimap<ColumnMetadata, Expression> analyzed = ArrayListMultimap.create(); + Set<ColumnMetadata> unindexedColumns = Collections.emptySet(); // sort all the expressions in the operation by name and priority of the logical operator // this gives us an efficient way to handle inequality and combining into ranges without extra processing @@ -82,17 +132,28 @@ protected static ListMultimap<ColumnMetadata, Expression> buildIndexExpressions( if (Expression.supportsOperator(expression.operator())) { StorageAttachedIndex index = queryController.indexFor(expression); - List<Expression> perColumn = analyzed.get(expression.column()); if (index == null) + { buildUnindexedExpression(queryController, expression, perColumn); + + if (!expression.column().isPrimaryKeyColumn()) + { + if (unindexedColumns.isEmpty()) + unindexedColumns = new HashSet<>(3); + + unindexedColumns.add(expression.column()); + } + } else + { buildIndexedExpression(index, expression, perColumn); + } } } - return analyzed; + return new Expressions(analyzed, unindexedColumns); } private static void buildUnindexedExpression(QueryController queryController, @@ -286,11 +347,11 @@ static FilterTree buildFilter(QueryController controller, boolean strict) static abstract class Node { - ListMultimap<ColumnMetadata, Expression> expressionMap; + Expressions expressions; boolean canFilter() { - return (expressionMap != null && !expressionMap.isEmpty()) || !children().isEmpty(); + return (expressions != null && !expressions.isEmpty()) || !children().isEmpty(); } List<Node> children() @@ -382,19 +443,19 @@ static class AndNode extends OperatorNode @Override public void analyze(List<RowFilter.Expression> expressionList, QueryController controller) { - expressionMap = buildIndexExpressions(controller, expressionList); + expressions = buildIndexExpressions(controller, expressionList); } @Override FilterTree filterTree(boolean isStrict, QueryContext context) { - return new FilterTree(BooleanOperator.AND, expressionMap, isStrict, context); + return new FilterTree(BooleanOperator.AND, expressions, isStrict, context); } @Override KeyRangeIterator rangeIterator(QueryController controller) { - KeyRangeIterator.Builder builder = controller.getIndexQueryResults(expressionMap.values()); + KeyRangeIterator.Builder builder = controller.getIndexQueryResults(expressions.all()); for (Node child : children) { boolean canFilter = child.canFilter(); @@ -412,15 +473,15 @@ static class ExpressionNode extends Node @Override public void analyze(List<RowFilter.Expression> expressionList, QueryController controller) { - expressionMap = buildIndexExpressions(controller, expressionList); - assert expressionMap.size() == 1 : "Expression nodes should only have a single expression!"; + expressions = buildIndexExpressions(controller, expressionList); + assert expressions.size() == 1 : "Expression nodes should only have a single expression!"; } @Override FilterTree filterTree(boolean isStrict, QueryContext context) { // There should only be one expression, so AND/OR would both work here. - return new FilterTree(BooleanOperator.AND, expressionMap, isStrict, context); + return new FilterTree(BooleanOperator.AND, expressions, isStrict, context); } public ExpressionNode(RowFilter.Expression expression) @@ -439,7 +500,7 @@ KeyRangeIterator rangeIterator(QueryController controller) { assert canFilter() : "Cannot process query with no expressions"; - return controller.getIndexQueryResults(expressionMap.values()).build(); + return controller.getIndexQueryResults(expressions.all()).build(); } } } diff --git a/src/java/org/apache/cassandra/index/sai/plan/QueryController.java b/src/java/org/apache/cassandra/index/sai/plan/QueryController.java index b462fa3ad5f1..614456c493dc 100644 --- a/src/java/org/apache/cassandra/index/sai/plan/QueryController.java +++ b/src/java/org/apache/cassandra/index/sai/plan/QueryController.java @@ -423,7 +423,7 @@ private ClusteringIndexFilter makeFilter(List<PrimaryKey> keys) "PrimaryKey " + firstKey + " clustering does not match table. There should be a clustering of size " + cfs.metadata().comparator.size(); ClusteringIndexFilter clusteringIndexFilter = command.clusteringIndexFilter(firstKey.partitionKey()); - + // If we have skinny partitions or the key is for a static row then we need to get the partition as // requested by the original query. if (cfs.metadata().comparator.size() == 0 || firstKey.kind() == PrimaryKey.Kind.STATIC) diff --git a/src/java/org/apache/cassandra/index/sai/utils/IndexTermType.java b/src/java/org/apache/cassandra/index/sai/utils/IndexTermType.java index 51600f9d791b..f5a00b184ef4 100644 --- a/src/java/org/apache/cassandra/index/sai/utils/IndexTermType.java +++ b/src/java/org/apache/cassandra/index/sai/utils/IndexTermType.java @@ -319,6 +319,11 @@ public boolean dependsOn(ColumnMetadata columnMetadata) return this.columnMetadata.compareTo(columnMetadata) == 0; } + public static boolean isEqOnlyType(AbstractType<?> type) + { + return EQ_ONLY_TYPES.contains(type); + } + /** * Indicates if the type encoding supports rounding of the raw value. * <p> diff --git a/test/distributed/org/apache/cassandra/distributed/test/sai/StrictFilteringTest.java b/test/distributed/org/apache/cassandra/distributed/test/sai/StrictFilteringTest.java index 301336f8629d..cd753d6e01f4 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/sai/StrictFilteringTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/sai/StrictFilteringTest.java @@ -54,6 +54,26 @@ public static void setUpCluster() throws IOException CLUSTER = init(Cluster.build(2).withConfig(config -> config.set("hinted_handoff_enabled", false).with(GOSSIP).with(NETWORK)).start()); } + @Test + public void shouldDegradeToUnionOnSingleStatic() + { + CLUSTER.schemaChange(withKeyspace("CREATE TABLE %s.single_static (pk0 int, ck0 int, ck1 int, s0 int static, v0 int, PRIMARY KEY (pk0, ck0, ck1)) " + + "WITH read_repair = 'NONE' AND CLUSTERING ORDER BY (ck0 ASC, ck1 DESC)")); + CLUSTER.schemaChange(withKeyspace("CREATE INDEX ON %s.single_static(ck0) USING 'sai'")); + CLUSTER.schemaChange(withKeyspace("CREATE INDEX ON %s.single_static(s0) USING 'sai'")); + SAIUtil.waitForIndexQueryable(CLUSTER, KEYSPACE); + + // To present the coordinator with enough data to find a row match, both replicas must degrade to OR at query + // time. The static column match from node 2 and the clustering key match from node 1 must be merged. + CLUSTER.get(2).executeInternal(withKeyspace("INSERT INTO %s.single_static (pk0, ck0, ck1, s0, v0) VALUES (0, 1, 2, 3, 4)")); + CLUSTER.get(1).executeInternal(withKeyspace("UPDATE %s.single_static SET v0 = 5 WHERE pk0 = 0 AND ck0 = 6 AND ck1 = 7")); + + // A static column predicate and ANY other predicate makes strict filtering impossible, as the static match + // applies to the entire partition. + String select = withKeyspace("SELECT * FROM %s.single_static WHERE s0 = 3 AND ck0 = 6"); + assertRows(CLUSTER.coordinator(1).execute(select, ConsistencyLevel.ALL), row(0, 6, 7, 3, 5)); + } + @Test public void shouldRejectNonStrictIN() { @@ -94,6 +114,43 @@ public void testPartialUpdates() assertRows(initialRows, row(0, 1, 2)); } + @Test + public void testPartialUpdatesOnNonIndexedColumnsAfterRepair() + { + CLUSTER.schemaChange(withKeyspace("CREATE TABLE %s.partial_updates_non_indexed_columns (k int PRIMARY KEY, a int, b int, c int) WITH read_repair = 'NONE'")); + CLUSTER.schemaChange(withKeyspace("CREATE INDEX ON %s.partial_updates_non_indexed_columns(a) USING 'sai'")); + SAIUtil.waitForIndexQueryable(CLUSTER, KEYSPACE); + + CLUSTER.coordinator(1).execute(withKeyspace("INSERT INTO %s.partial_updates_non_indexed_columns(k, a) VALUES (0, 1) USING TIMESTAMP 1"), ConsistencyLevel.ALL); + CLUSTER.get(1).nodetoolResult("repair", KEYSPACE).asserts().success(); + + // insert a split row + CLUSTER.get(1).executeInternal(withKeyspace("INSERT INTO %s.partial_updates_non_indexed_columns(k, b) VALUES (0, 2) USING TIMESTAMP 2")); + CLUSTER.get(2).executeInternal(withKeyspace("INSERT INTO %s.partial_updates_non_indexed_columns(k, c) VALUES (0, 3) USING TIMESTAMP 3")); + + String select = withKeyspace("SELECT * FROM %s.partial_updates_non_indexed_columns WHERE a = 1 AND b = 2 AND c = 3 ALLOW FILTERING"); + Object[][] initialRows = CLUSTER.coordinator(1).execute(select, ConsistencyLevel.ALL); + assertRows(initialRows, row(0, 1, 2, 3)); + } + + @Test + public void testPartialUpdateOnNonIndexedColumnAfterRepair() + { + CLUSTER.schemaChange(withKeyspace("CREATE TABLE %s.partial_updates_non_indexed_column (k int PRIMARY KEY, a int, b int) WITH read_repair = 'NONE'")); + CLUSTER.schemaChange(withKeyspace("CREATE INDEX ON %s.partial_updates_non_indexed_column(a) USING 'sai'")); + SAIUtil.waitForIndexQueryable(CLUSTER, KEYSPACE); + + CLUSTER.coordinator(1).execute(withKeyspace("INSERT INTO %s.partial_updates_non_indexed_column(k, a) VALUES (0, 1) USING TIMESTAMP 1"), ConsistencyLevel.ALL); + CLUSTER.get(1).nodetoolResult("repair", KEYSPACE).asserts().success(); + + // insert a split row + CLUSTER.get(1).executeInternal(withKeyspace("INSERT INTO %s.partial_updates_non_indexed_column(k, b) VALUES (0, 2) USING TIMESTAMP 2")); + + String select = withKeyspace("SELECT * FROM %s.partial_updates_non_indexed_column WHERE a = 1 AND b = 2 ALLOW FILTERING"); + Object[][] initialRows = CLUSTER.coordinator(1).execute(select, ConsistencyLevel.ALL); + assertRows(initialRows, row(0, 1, 2)); + } + @Test public void testPartialUpdatesWithDeleteBetween() { diff --git a/test/unit/org/apache/cassandra/index/sai/plan/OperationTest.java b/test/unit/org/apache/cassandra/index/sai/plan/OperationTest.java index 81292cbda06f..b27764115b1e 100644 --- a/test/unit/org/apache/cassandra/index/sai/plan/OperationTest.java +++ b/test/unit/org/apache/cassandra/index/sai/plan/OperationTest.java @@ -27,7 +27,6 @@ import java.util.concurrent.TimeUnit; import com.google.common.base.Preconditions; -import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import org.junit.Before; import org.junit.BeforeClass; @@ -324,10 +323,10 @@ public void testSatisfiedByWithClustering() assertTrue(node.buildFilter(controllerClustering, true).isSatisfiedBy(key, row, staticRow)); } - private Map<Expression.IndexOperator, Expression> convert(Multimap<ColumnMetadata, Expression> expressions) + private Map<Expression.IndexOperator, Expression> convert(Operation.Expressions expressions) { Map<Expression.IndexOperator, Expression> converted = new EnumMap<>(Expression.IndexOperator.class); - for (Expression expression : expressions.values()) + for (Expression expression : expressions.all()) { Expression column = converted.get(expression.getIndexOperator()); assert column == null; // sanity check From 08ddb01dd37c83ef4e8f16648f53683bf3ec66b6 Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi <bernardo.botella@apple.com> Date: Mon, 3 Feb 2025 06:20:06 -0800 Subject: [PATCH 127/225] Improve error messages for constraints patch by Bernardo Botella; reviewed by Stefan Miklosovic, Dinesh Joshi for CASSANDRA-20266 --- CHANGES.txt | 1 + .../cql3/constraints/LengthConstraint.java | 5 +- .../constraints/ScalarColumnConstraint.java | 4 +- ...WithColumnCqlConstraintValidationTest.java | 238 ++++++++++++------ 4 files changed, 167 insertions(+), 81 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index de1173b38230..74a985061c85 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Improve error messages for constraints (CASSANDRA-20266) * Add system_views.partition_key_statistics for querying SSTable metadata (CASSANDRA-20161) * CEP-42 - Add Constraints Framework (CASSANDRA-19947) * Add table metric PurgeableTombstoneScannedHistogram and a tracing event for scanned purgeable tombstones (CASSANDRA-20132) diff --git a/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java index 9d6f99ca24de..054f4f43b046 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java @@ -59,8 +59,9 @@ public void evaluate(AbstractType<?> valueType, Operator relationType, String te ByteBuffer rightOperand = ByteBufferUtil.bytes(sizeConstraint); if (!relationType.isSatisfiedBy(Int32Type.instance, leftOperand, rightOperand)) - throw new ConstraintViolationException(columnName + " does not satisfy length constraint. " - + valueLength + " should be " + relationType + ' ' + term); + throw new ConstraintViolationException("Column value does not satisfy value constraint for column '" + columnName + "'. " + + "It has a length of " + valueLength + " and it should be should be " + + relationType + ' ' + term); } @Override diff --git a/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java index 14be7d689d20..736ed5482365 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java @@ -80,8 +80,8 @@ public void evaluate(AbstractType<?> valueType, ByteBuffer columnValue) } if (!relationType.isSatisfiedBy(valueType, columnValue, value)) - throw new ConstraintViolationException("Column value does not satisfy value constraint. " - + " It should be " + relationType + " " + term); + throw new ConstraintViolationException("Column value does not satisfy value constraint for column '" + param + "'. " + + "It should be " + param + " " + relationType + " " + term); } @Override diff --git a/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnCqlConstraintValidationTest.java b/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnCqlConstraintValidationTest.java index 97de503d1b6c..a5f380307d57 100644 --- a/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnCqlConstraintValidationTest.java +++ b/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnCqlConstraintValidationTest.java @@ -124,12 +124,13 @@ public void testCreateTableWithColumnWithClusteringColumnLessThanScalarConstrain qt().forAll(Generators.toGen(integers().between(0, 3))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 < 4"; // Invalid qt().forAll(Generators.toGen(integers().between(4, 100))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -147,12 +148,13 @@ public void testCreateTableWithColumnWithClusteringColumnBiggerThanScalarConstra qt().forAll(Generators.toGen(integers().between(5, 100))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 > 4"; // Invalid qt().forAll(Generators.toGen(integers().between(0, 4))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -170,12 +172,13 @@ public void testCreateTableWithColumnWithClusteringColumnBiggerOrEqualThanScalar qt().forAll(Generators.toGen(integers().between(4, 100))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 >= 4"; // Invalid qt().forAll(Generators.toGen(integers().between(0, 3))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -193,12 +196,13 @@ public void testCreateTableWithColumnWithClusteringColumnLessOrEqualThanScalarCo qt().forAll(Generators.toGen(integers().between(0, 4))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 <= 4"; // Invalid qt().forAll(Generators.toGen(integers().between(5, 100))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -218,8 +222,9 @@ public void testCreateTableWithColumnWithClusteringColumnDifferentThanScalarCons qt().forAll(Generators.toGen(integers().between(5, 100))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 != 4"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 4, 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 4, 2, 3)"); } @Test @@ -231,23 +236,26 @@ public void testCreateTableWithColumnWithClusteringColumnMultipleScalarConstrain qt().forAll(Generators.toGen(integers().between(2, 3))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 >= 2"; // Invalid qt().forAll(Generators.toGen(integers().between(-100, 1))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { throw new RuntimeException(e); } }); + + final String expectedErrorMessage2 = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 < 4"; qt().forAll(Generators.toGen(integers().between(4, 100))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage2, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -265,12 +273,13 @@ public void testCreateTableWithColumnWithClusteringColumnLessThanScalarSmallIntC qt().forAll(Generators.toGen(integers().between(0, 3))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 < 4"; // Invalid qt().forAll(Generators.toGen(integers().between(4, 100))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -288,12 +297,14 @@ public void testCreateTableWithColumnWithClusteringColumnBiggerThanScalarSmallIn qt().forAll(Generators.toGen(integers().between(5, 100))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 > 4"; // Invalid qt().forAll(Generators.toGen(integers().between(0, 4))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -311,12 +322,13 @@ public void testCreateTableWithColumnWithClusteringColumnBiggerOrEqualThanScalar qt().forAll(Generators.toGen(integers().between(4, 100))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 >= 4"; // Invalid qt().forAll(Generators.toGen(integers().between(0, 3))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -334,12 +346,13 @@ public void testCreateTableWithColumnWithClusteringColumnLessOrEqualThanScalarSm qt().forAll(Generators.toGen(integers().between(0, 4))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 <= 4"; // Invalid qt().forAll(Generators.toGen(integers().between(5, 100))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -359,8 +372,9 @@ public void testCreateTableWithColumnWithClusteringColumnDifferentThanScalarSmal qt().forAll(Generators.toGen(integers().between(5, 100))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 != 4"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 4, 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 4, 2, 3)"); } @Test @@ -372,23 +386,26 @@ public void testCreateTableWithColumnWithClusteringColumnMultipleScalarSmallIntC qt().forAll(Generators.toGen(integers().between(2, 3))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 >= 2"; // Invalid qt().forAll(Generators.toGen(integers().between(-100, 1))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { throw new RuntimeException(e); } }); + + final String expectedErrorMessage2 = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 < 4"; qt().forAll(Generators.toGen(integers().between(4, 100))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage2, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -407,12 +424,14 @@ public void testCreateTableWithColumnWithClusteringColumnLessThanScalarDecimalCo qt().forAll(Generators.toGen(doubles().between(0, 4.1))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 < 4.2"; // Invalid qt().forAll(Generators.toGen(doubles().between(4.3, 100))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -430,12 +449,13 @@ public void testCreateTableWithColumnWithClusteringColumnBiggerThanScalarDecimal qt().forAll(Generators.toGen(doubles().between(4.3, 100))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 > 4.2"; // Invalid qt().forAll(Generators.toGen(doubles().between(0, 4.2))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -453,12 +473,13 @@ public void testCreateTableWithColumnWithClusteringColumnBiggerOrEqualThanScalar qt().forAll(Generators.toGen(doubles().between(4.2, 100))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 >= 4.2"; // Invalid qt().forAll(Generators.toGen(doubles().between(0, 4.1))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -476,12 +497,13 @@ public void testCreateTableWithColumnWithClusteringColumnLessOrEqualThanScalarDe qt().forAll(Generators.toGen(doubles().between(0, 4.2))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 <= 4.2"; // Invalid qt().forAll(Generators.toGen(doubles().between(4.3, 100))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -501,8 +523,9 @@ public void testCreateTableWithColumnWithClusteringColumnDifferentThanScalarDeci qt().forAll(Generators.toGen(doubles().between(4.3, 100))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 != 4.2"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 4.2, 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 4.2, 2, 3)"); } @Test @@ -514,23 +537,26 @@ public void testCreateTableWithColumnWithClusteringColumnMultipleScalarDecimalCo qt().forAll(Generators.toGen(doubles().between(2.1, 4.1))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 >= 2.1"; // Invalid qt().forAll(Generators.toGen(doubles().between(-100, 2))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { throw new RuntimeException(e); } }); + + final String expectedErrorMessage2 = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 < 4.2"; qt().forAll(Generators.toGen(doubles().between(4.2, 100))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage2, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -549,12 +575,13 @@ public void testCreateTableWithColumnWithClusteringColumnLessThanScalarDoubleCon qt().forAll(Generators.toGen(doubles().between(0, 4.1))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 < 4.2"; // Invalid qt().forAll(Generators.toGen(doubles().between(4.3, 100))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -572,12 +599,13 @@ public void testCreateTableWithColumnWithClusteringColumnBiggerThanScalarDoubleC qt().forAll(Generators.toGen(doubles().between(4.3, 100))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 > 4.2"; // Invalid qt().forAll(Generators.toGen(doubles().between(0, 4.2))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -595,12 +623,13 @@ public void testCreateTableWithColumnWithClusteringColumnBiggerOrEqualThanScalar qt().forAll(Generators.toGen(doubles().between(4.2, 100))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 >= 4.2"; // Invalid qt().forAll(Generators.toGen(doubles().between(0, 4.1))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -618,12 +647,13 @@ public void testCreateTableWithColumnWithClusteringColumnLessOrEqualThanScalarDo qt().forAll(Generators.toGen(doubles().between(0, 4.2))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 <= 4.2"; // Invalid qt().forAll(Generators.toGen(doubles().between(4.3, 100))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -643,8 +673,9 @@ public void testCreateTableWithColumnWithClusteringColumnDifferentThanScalarDoub qt().forAll(Generators.toGen(doubles().between(4.3, 100))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 != 4.2"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 4.2, 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 4.2, 2, 3)"); } @Test @@ -656,23 +687,26 @@ public void testCreateTableWithColumnWithClusteringColumnMultipleScalarDoubleCon qt().forAll(Generators.toGen(doubles().between(2.1, 4.1))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 >= 2.1"; // Invalid qt().forAll(Generators.toGen(doubles().between(-100, 2))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { throw new RuntimeException(e); } }); + + final String expectedErrorMessage2 = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 < 4.2"; qt().forAll(Generators.toGen(doubles().between(4.2, 100))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage2, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -691,12 +725,13 @@ public void testCreateTableWithColumnWithClusteringColumnLessThanScalarFloatCons qt().forAll(Generators.toGen(doubles().between(0, 4.1))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 < 4.2"; // Invalid qt().forAll(Generators.toGen(doubles().between(4.3, 100))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -714,12 +749,13 @@ public void testCreateTableWithColumnWithClusteringColumnBiggerThanScalarFloatCo qt().forAll(Generators.toGen(doubles().between(4.3, 100))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 > 4.2"; // Invalid qt().forAll(Generators.toGen(doubles().between(0, 4.2))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -737,12 +773,13 @@ public void testCreateTableWithColumnWithClusteringColumnBiggerOrEqualThanScalar qt().forAll(Generators.toGen(doubles().between(4.2, 100))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 >= 4.2"; // Invalid qt().forAll(Generators.toGen(doubles().between(0, 4.1))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -760,12 +797,13 @@ public void testCreateTableWithColumnWithClusteringColumnLessOrEqualThanScalarFl qt().forAll(Generators.toGen(doubles().between(0, 4.2))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 <= 4.2"; // Invalid qt().forAll(Generators.toGen(doubles().between(4.3, 100))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -785,8 +823,9 @@ public void testCreateTableWithColumnWithClusteringColumnDifferentThanScalarFloa qt().forAll(Generators.toGen(doubles().between(4.3, 100))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 != 4.2"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 4.2, 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 4.2, 2, 3)"); } @Test @@ -798,23 +837,26 @@ public void testCreateTableWithColumnWithClusteringColumnMultipleScalarFloatCons qt().forAll(Generators.toGen(doubles().between(2.1, 4.1))) .check(d -> execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 >= 2.1"; // Invalid qt().forAll(Generators.toGen(doubles().between(-100, 2))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { throw new RuntimeException(e); } }); + + final String expectedErrorMessage2 = "Column value does not satisfy value constraint for column 'ck1'. It should be ck1 < 4.2"; qt().forAll(Generators.toGen(doubles().between(4.2, 100))) .check(d -> { try { - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + assertInvalidThrowMessage(expectedErrorMessage2, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); } catch (Throwable e) { @@ -832,9 +874,10 @@ public void testCreateTableWithColumnWithClusteringColumnLengthEqualToConstraint // Valid execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fooo', 2, 3)"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foo', 2, 3)"); - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foooo', 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foo', 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foooo', 2, 3)"); } @Test @@ -842,12 +885,13 @@ public void testCreateTableWithColumnWithClusteringColumnLengthDifferentThanCons { createTable("CREATE TABLE %s (pk int, ck1 text CHECK LENGTH(ck1) != 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; // Valid execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foo', 2, 3)"); execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foooo', 2, 3)"); // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fooo', 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fooo', 2, 3)"); } @Test @@ -858,9 +902,10 @@ public void testCreateTableWithColumnWithClusteringColumnLengthBiggerThanConstra // Valid execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foooo', 2, 3)"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foo', 2, 3)"); - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fooo', 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foo', 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fooo', 2, 3)"); } @Test @@ -872,8 +917,9 @@ public void testCreateTableWithColumnWithClusteringColumnLengthBiggerOrEqualThan execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foooo', 2, 3)"); execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fooo', 2, 3)"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foo', 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foo', 2, 3)"); } @Test @@ -884,9 +930,10 @@ public void testCreateTableWithColumnWithClusteringColumnLengthSmallerThanConstr // Valid execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foo', 2, 3)"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fooo', 2, 3)"); - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foooo', 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fooo', 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foooo', 2, 3)"); } @Test @@ -898,8 +945,9 @@ public void testCreateTableWithColumnWithClusteringColumnLengthSmallerOrEqualTha execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foo', 2, 3)"); execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fooo', 2, 3)"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foooo', 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foooo', 2, 3)"); } @Test @@ -910,9 +958,10 @@ public void testCreateTableWithColumnWithClusteringBlobColumnLengthEqualToConstr // Valid execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fooo'), 2, 3)"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foo'), 2, 3)"); - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foooo'), 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foo'), 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foooo'), 2, 3)"); } @Test @@ -924,8 +973,9 @@ public void testCreateTableWithColumnWithClusteringBlobColumnLengthDifferentThan execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foo'), 2, 3)"); execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foooo'), 2, 3)"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fooo'), 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fooo'), 2, 3)"); } @Test @@ -936,9 +986,10 @@ public void testCreateTableWithColumnWithClusteringBlobColumnLengthBiggerThanCon // Valid execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foooo'), 2, 3)"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foo'), 2, 3)"); - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fooo'), 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foo'), 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fooo'), 2, 3)"); } @Test @@ -950,8 +1001,9 @@ public void testCreateTableWithColumnWithClusteringBlobColumnLengthBiggerOrEqual execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foooo'), 2, 3)"); execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fooo'), 2, 3)"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foo'), 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foo'), 2, 3)"); } @Test @@ -962,9 +1014,10 @@ public void testCreateTableWithColumnWithClusteringBlobColumnLengthSmallerThanCo // Valid execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foo'), 2, 3)"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fooo'), 2, 3)"); - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foooo'), 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fooo'), 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foooo'), 2, 3)"); } @Test @@ -976,8 +1029,9 @@ public void testCreateTableWithColumnWithClusteringBlobColumnLengthSmallerOrEqua execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foo'), 2, 3)"); execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fooo'), 2, 3)"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foooo', 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('foooo'), 2, 3)"); } @@ -989,9 +1043,10 @@ public void testCreateTableWithColumnWithPkColumnLengthEqualToConstraint() throw // Valid execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 1, 2, 3)"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'pk'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foo', 1, 2, 3)"); - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foooo', 1, 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foo', 1, 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foooo', 1, 2, 3)"); } @Test @@ -1003,8 +1058,9 @@ public void testCreateTableWithColumnWithPkColumnLengthDifferentThanConstraint() execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foo', 1, 2, 3)"); execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foooo', 1, 2, 3)"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'pk'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 1, 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 1, 2, 3)"); } @Test @@ -1015,9 +1071,10 @@ public void testCreateTableWithColumnWithPkColumnLengthBiggerThanConstraint() th // Valid execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foooo', 1, 2, 3)"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'pk'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foo', 1, 2, 3)"); - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 1, 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foo', 1, 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 1, 2, 3)"); } @Test @@ -1029,8 +1086,9 @@ public void testCreateTableWithColumnWithPkColumnLengthBiggerOrEqualThanConstrai execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foooo', 1, 2, 3)"); execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 1, 2, 3)"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'pk'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foo', 1, 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foo', 1, 2, 3)"); } @Test @@ -1041,9 +1099,10 @@ public void testCreateTableWithColumnWithPkColumnLengthSmallerThanConstraint() t // Valid execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foo', 1, 2, 3)"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'pk'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 1, 2, 3)"); - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foooo', 1, 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 1, 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foooo', 1, 2, 3)"); } @Test @@ -1055,8 +1114,9 @@ public void testCreateTableWithColumnWithPkColumnLengthSmallerOrEqualThanConstra execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foo', 1, 2, 3)"); execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 1, 2, 3)"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'pk'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foooo', 1, 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foooo', 1, 2, 3)"); } @@ -1068,9 +1128,10 @@ public void testCreateTableWithColumnWithRegularColumnLengthEqualToConstraint() // Valid execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fooo')"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foo')"); - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foooo')"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foo')"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foooo')"); } @Test @@ -1082,8 +1143,9 @@ public void testCreateTableWithColumnWithRegularColumnLengthDifferentThanConstra execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foo')"); execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foooo')"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fooo')"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fooo')"); } @Test @@ -1094,9 +1156,10 @@ public void testCreateTableWithColumnWithRegularColumnLengthBiggerThanConstraint // Valid execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foooo')"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foo')"); - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fooo')"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foo')"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fooo')"); } @Test @@ -1108,8 +1171,9 @@ public void testCreateTableWithColumnWithRegularColumnLengthBiggerOrEqualThanCon execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foooo')"); execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fooo')"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foo')"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foo')"); } @Test @@ -1120,9 +1184,10 @@ public void testCreateTableWithColumnWithRegularColumnLengthSmallerThanConstrain // Valid execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foo')"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fooo')"); - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foooo')"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fooo')"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foooo')"); } @Test @@ -1134,8 +1199,9 @@ public void testCreateTableWithColumnWithRegularColumnLengthSmallerOrEqualThanCo execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foo')"); execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fooo')"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foooo')"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foooo')"); } @Test @@ -1146,13 +1212,15 @@ public void testCreateTableWithColumnMixedColumnsLengthConstraint() throws Throw // Valid execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 2, 3, 'fooo')"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'pk'. It has a length of"; + final String expectedErrorMessage2 = "Column value does not satisfy value constraint for column 'v'. It has a length of"; // Invalid - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foo', 2, 3, 'foo')"); - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 2, 3, 'foo')"); - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foo', 2, 3, 'fooo')"); - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foooo', 2, 3, 'fooo')"); - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 2, 3, 'foooo')"); - assertInvalidThrow(InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foooo', 2, 3, 'foooo')"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foo', 2, 3, 'foo')"); + assertInvalidThrowMessage(expectedErrorMessage2, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 2, 3, 'foo')"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foo', 2, 3, 'fooo')"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foooo', 2, 3, 'fooo')"); + assertInvalidThrowMessage(expectedErrorMessage2, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fooo', 2, 3, 'foooo')"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('foooo', 2, 3, 'foooo')"); } @Test @@ -1200,6 +1268,22 @@ public void testCreateTableWithColumnWithClusteringColumnInvalidTypeConstraint() } } + @Test + public void testCreateTableWithColumnWithClusteringColumnInvalidScalarTypeConstraint() throws Throwable + { + try + { + createTable("CREATE TABLE %s (pk text CHECK pk = 4, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + fail(); + } + catch (InvalidRequestException e) + { + assertTrue(e.getCause() instanceof InvalidRequestException); + assertTrue(e.getCause().getMessage().equals("pk is not a number")); + assertTrue(e.getMessage().contains("Error setting schema for test")); + } + } + @Test public void testCreateTableInvalidFunction() throws Throwable { From f340514bbe1a8b88c5df3a3949bfc1fd937e576b Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Fri, 7 Feb 2025 12:40:40 +0100 Subject: [PATCH 128/225] Increment version --- CHANGES.txt | 3 +++ build.xml | 2 +- debian/changelog | 6 ++++++ 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 1ae082009cc5..9442561ef376 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,3 +1,6 @@ +3.0.33 + + 3.0.32 * Minimise expensive reads during authz flow in 3.0/3.11 (CASSANDRA-20293) diff --git a/build.xml b/build.xml index 07b12ed097b8..403f6d799dec 100644 --- a/build.xml +++ b/build.xml @@ -34,7 +34,7 @@ <property name="debuglevel" value="source,lines,vars"/> <!-- default version and SCM information --> - <property name="base.version" value="3.0.32"/> + <property name="base.version" value="3.0.33"/> <property name="scm.connection" value="scm:https://gitbox.apache.org/repos/asf/cassandra.git"/> <property name="scm.developerConnection" value="scm:https://gitbox.apache.org/repos/asf/cassandra.git"/> <property name="scm.url" value="https://gitbox.apache.org/repos/asf?p=cassandra.git"/> diff --git a/debian/changelog b/debian/changelog index 77e8fce50880..f945ed143ab4 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,3 +1,9 @@ +cassandra (3.0.33) UNRELEASED; urgency=medium + + * New release + + -- Stefan Miklosovic <smiklosovic@apache.org> Fri, 07 Feb 2025 07:20:16 +0100 + cassandra (3.0.32) unstable; urgency=medium * New release From 11480a8b0837f86b5870de870c383a88f9d97607 Mon Sep 17 00:00:00 2001 From: Dmitry Konstantinov <konstantinov@netcracker.com> Date: Fri, 7 Feb 2025 20:53:55 +0000 Subject: [PATCH 129/225] Wait for completion of async dumpMemtable in CommitLogTestCQL Patch by Dmitry Konstantinov; reviewed by brandonwilliams and smiklosovic for CASSANDRA-20298 --- .../org/apache/cassandra/db/commitlog/CommitLogCQLTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogCQLTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogCQLTest.java index 2fd38d8e8c14..90f9627af634 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogCQLTest.java +++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogCQLTest.java @@ -32,6 +32,7 @@ import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.utils.FBUtilities; public class CommitLogCQLTest extends CQLTester { @@ -89,7 +90,7 @@ public void run() { QueryProcessor.executeInternal(stmt, i, Integer.toString(i)); } - cfs.dumpMemtable(); + FBUtilities.waitOnFuture(cfs.dumpMemtable()); } } catch (Throwable t) From 881a1f023fb1ab304921399d983f01bfbba0e917 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Thu, 6 Feb 2025 08:06:59 +0100 Subject: [PATCH 130/225] Fix incorrect results of min / max in-built functions on clustering columns in descending order Not unwrapping returnType has consequences of not returning the correct results when using aggregate in-built functions (max / min) on clustering columns having descending order. The safest solution, is to unwrap returnType "at source" and work with unwrapped type only since then. patch by Stefan Miklosovic; reviewed by David Capwell for CASSANDRA-20295 --- CHANGES.txt | 1 + .../cql3/functions/AbstractFunction.java | 2 +- .../operations/AggregationTest.java | 80 +++++++++++++++++++ 3 files changed, 82 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 13ae3fddc29d..8763039fa6b4 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.4 + * Fix incorrect results of min / max in-built functions on clustering columns in descending order (CASSANDRA-20295) * Avoid possible consistency violations for SAI intersection queries over repaired index matches and multiple non-indexed column matches (CASSANDRA-20189) * Skip check for DirectIO when initializing tools (CASSANDRA-20289) * Avoid under-skipping during intersections when an iterator has mixed STATIC and WIDE keys (CASSANDRA-20258) diff --git a/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java b/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java index c3183f64133c..1b62daddbb96 100644 --- a/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java +++ b/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java @@ -46,7 +46,7 @@ protected AbstractFunction(FunctionName name, List<AbstractType<?>> argTypes, Ab { this.name = name; this.argTypes = argTypes; - this.returnType = returnType; + this.returnType = returnType != null ? returnType.unwrap() : null; } public FunctionName name() diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java index 8202e8031a02..24afbb7e802c 100644 --- a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java +++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java @@ -180,6 +180,86 @@ public void testCountStarFunction() throws Throwable assertRows(execute("SELECT max(b), COUNT(1), b FROM %s WHERE a = 1 LIMIT 2"), row(5, 4L, 1)); } + @Test + public void testMaxAggregationDescending() + { + createTable("CREATE TABLE %s (a int, b int, primary key (a, b)) WITH CLUSTERING ORDER BY (b DESC)"); + + execute("INSERT INTO %s (a, b) VALUES (1, 1000)"); + execute("INSERT INTO %s (a, b) VALUES (1, 100)"); + execute("INSERT INTO %s (a, b) VALUES (1, 1)"); + + assertRows(execute("SELECT count(b), max(b) as max FROM %s WHERE a = 1"), + row(3L, 1000)); + + execute("INSERT INTO %s (a, b) VALUES (2, 4000)"); + execute("INSERT INTO %s (a, b) VALUES (3, 100)"); + execute("INSERT INTO %s (a, b) VALUES (4, 0)"); + + assertRows(execute("SELECT count(b), max(b) as max FROM %s"), + row(6L, 4000)); + } + + @Test + public void testMinAggregationDescending() + { + createTable("CREATE TABLE %s (a int, b int, primary key (a, b)) WITH CLUSTERING ORDER BY (b DESC)"); + + execute("INSERT INTO %s (a, b) VALUES (1, 1000)"); + execute("INSERT INTO %s (a, b) VALUES (1, 100)"); + execute("INSERT INTO %s (a, b) VALUES (1, 1)"); + + assertRows(execute("SELECT count(b), min(b) as min FROM %s WHERE a = 1"), + row(3L, 1)); + + execute("INSERT INTO %s (a, b) VALUES (2, 4000)"); + execute("INSERT INTO %s (a, b) VALUES (3, 100)"); + execute("INSERT INTO %s (a, b) VALUES (4, 0)"); + + assertRows(execute("SELECT count(b), min(b) as min FROM %s"), + row(6L, 0)); + } + + @Test + public void testMaxAggregationAscending() + { + createTable("CREATE TABLE %s (a int, b int, primary key (a, b)) WITH CLUSTERING ORDER BY (b ASC)"); + + execute("INSERT INTO %s (a, b) VALUES (1, 1000)"); + execute("INSERT INTO %s (a, b) VALUES (1, 100)"); + execute("INSERT INTO %s (a, b) VALUES (1, 1)"); + + assertRows(execute("SELECT count(b), max(b) as max FROM %s WHERE a = 1"), + row(3L, 1000)); + + execute("INSERT INTO %s (a, b) VALUES (2, 4000)"); + execute("INSERT INTO %s (a, b) VALUES (3, 100)"); + execute("INSERT INTO %s (a, b) VALUES (4, 5)"); + + assertRows(execute("SELECT count(b), max(b) as max FROM %s"), + row(6L, 4000)); + } + + @Test + public void testMinAggregationAscending() + { + createTable("CREATE TABLE %s (a int, b int, primary key (a, b)) WITH CLUSTERING ORDER BY (b ASC)"); + + execute("INSERT INTO %s (a, b) VALUES (1, 1000)"); + execute("INSERT INTO %s (a, b) VALUES (1, 100)"); + execute("INSERT INTO %s (a, b) VALUES (1, 1)"); + + assertRows(execute("SELECT count(b), min(b) as min FROM %s WHERE a = 1"), + row(3L, 1)); + + execute("INSERT INTO %s (a, b) VALUES (2, 4000)"); + execute("INSERT INTO %s (a, b) VALUES (3, 100)"); + execute("INSERT INTO %s (a, b) VALUES (4, 0)"); + + assertRows(execute("SELECT count(b), min(b) as min FROM %s"), + row(6L, 0)); + } + @Test public void testAggregateWithColumns() throws Throwable { From f28a16740abfdc6c6c72a40d8ac2a7efc14edc0e Mon Sep 17 00:00:00 2001 From: Caleb Rackliffe <calebrackliffe@gmail.com> Date: Mon, 10 Feb 2025 16:27:33 -0600 Subject: [PATCH 131/225] Avoid indexing empty values for non-literals and types that do not allow them MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit patch by Caleb Rackliffe; reviewed by David Capwell and Andres de la Peña for CASSANDRA-20313 Co-authored-by: Caleb Rackliffe <calebrackliffe@gmail.com> Co-authored-by: Andres de la Peña <a.penya.garcia@gmail.com> --- CHANGES.txt | 1 + .../index/sai/disk/v1/SSTableIndexWriter.java | 2 +- .../index/sai/memory/MemtableIndex.java | 2 +- .../index/sai/utils/IndexTermType.java | 15 ++++- .../index/sai/cql/EmptyValuesTest.java | 65 +++++++++++++++++++ 5 files changed, 81 insertions(+), 4 deletions(-) create mode 100644 test/unit/org/apache/cassandra/index/sai/cql/EmptyValuesTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 8763039fa6b4..cbbd1767b596 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.4 + * Avoid indexing empty values for non-literals and types that do not allow them (CASSANDRA-20313) * Fix incorrect results of min / max in-built functions on clustering columns in descending order (CASSANDRA-20295) * Avoid possible consistency violations for SAI intersection queries over repaired index matches and multiple non-indexed column matches (CASSANDRA-20189) * Skip check for DirectIO when initializing tools (CASSANDRA-20289) diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/SSTableIndexWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/SSTableIndexWriter.java index 58ee69a21565..bf384c4f5f38 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v1/SSTableIndexWriter.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/SSTableIndexWriter.java @@ -201,7 +201,7 @@ else if (shouldFlush(sstableRowId)) } // Some types support empty byte buffers: - if (term.remaining() == 0 && !index.termType().indexType().allowsEmpty()) return; + if (term.remaining() == 0 && index.termType().skipsEmptyValue()) return; if (analyzer == null || !index.termType().isLiteral()) { diff --git a/src/java/org/apache/cassandra/index/sai/memory/MemtableIndex.java b/src/java/org/apache/cassandra/index/sai/memory/MemtableIndex.java index f0f2ea36ad6c..1546ede37d92 100644 --- a/src/java/org/apache/cassandra/index/sai/memory/MemtableIndex.java +++ b/src/java/org/apache/cassandra/index/sai/memory/MemtableIndex.java @@ -82,7 +82,7 @@ public ByteBuffer getMaxTerm() public long index(DecoratedKey key, Clustering<?> clustering, ByteBuffer value) { - if (value == null || (value.remaining() == 0 && !type.allowsEmpty())) + if (value == null || (value.remaining() == 0 && memoryIndex.index.termType().skipsEmptyValue())) return 0; long ram = memoryIndex.add(key, clustering, value); diff --git a/src/java/org/apache/cassandra/index/sai/utils/IndexTermType.java b/src/java/org/apache/cassandra/index/sai/utils/IndexTermType.java index f5a00b184ef4..7fa226e9582d 100644 --- a/src/java/org/apache/cassandra/index/sai/utils/IndexTermType.java +++ b/src/java/org/apache/cassandra/index/sai/utils/IndexTermType.java @@ -275,6 +275,14 @@ public boolean isValid(ByteBuffer term) } } + /** + * @return {@code true} if the empty values of the given type should be excluded from indexing + */ + public boolean skipsEmptyValue() + { + return !indexType.allowsEmpty() || !isLiteral(); + } + public AbstractType<?> indexType() { return indexType; @@ -541,6 +549,9 @@ else if (isBigDecimal()) public ByteSource asComparableBytes(ByteBuffer value, ByteComparable.Version version) { + if (value.remaining() == 0) + return ByteSource.EMPTY; + if (isInetAddress() || isBigInteger() || isBigDecimal()) return ByteSource.optionalFixedLength(ByteBufferAccessor.instance, value); else if (isLong()) @@ -560,8 +571,8 @@ else if (isFrozen()) */ public ByteBuffer asIndexBytes(ByteBuffer value) { - if (value == null) - return null; + if (value == null || value.remaining() == 0) + return value; if (isInetAddress()) return encodeInetAddress(value); diff --git a/test/unit/org/apache/cassandra/index/sai/cql/EmptyValuesTest.java b/test/unit/org/apache/cassandra/index/sai/cql/EmptyValuesTest.java new file mode 100644 index 000000000000..ad36dd11564d --- /dev/null +++ b/test/unit/org/apache/cassandra/index/sai/cql/EmptyValuesTest.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.index.sai.cql; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import org.apache.cassandra.cql3.CQL3Type; +import org.apache.cassandra.index.sai.SAITester; +import org.apache.cassandra.index.sai.StorageAttachedIndex; +import org.apache.cassandra.index.sai.utils.IndexTermType; +import org.apache.cassandra.utils.AbstractTypeGenerators; + +import static org.apache.cassandra.utils.ByteBufferUtil.EMPTY_BYTE_BUFFER; +import static org.quicktheories.QuickTheory.qt; + +/** + * Tests that empty values are only indexed for literal indexes. See CASSANDRA-20313 for more details. + */ +public class EmptyValuesTest extends SAITester +{ + @Test + public void testEmptyValues() + { + qt().forAll(AbstractTypeGenerators.primitiveTypeGen()).checkAssert(type -> { + CQL3Type cql3Type = type.asCQL3Type(); + if (type.allowsEmpty() && StorageAttachedIndex.SUPPORTED_TYPES.contains(cql3Type)) + testEmptyValues(cql3Type); + }); + } + + private void testEmptyValues(CQL3Type type) + { + createTable(String.format("CREATE TABLE %%s (k int PRIMARY KEY, v %s)", type)); + execute("INSERT INTO %s (k, v) VALUES (0, ?)", EMPTY_BYTE_BUFFER); + flush(); + createIndex(String.format(CREATE_INDEX_TEMPLATE, 'v')); + + IndexTermType termType = createIndexTermType(type.getType()); + boolean indexed = !termType.skipsEmptyValue(); + + Assertions.assertThat(execute("SELECT * FROM %s WHERE v = ?", EMPTY_BYTE_BUFFER)).hasSize(indexed ? 1 : 0); + + execute("INSERT INTO %s (k, v) VALUES (1, ?)", EMPTY_BYTE_BUFFER); + + Assertions.assertThat(execute("SELECT * FROM %s WHERE v = ?", EMPTY_BYTE_BUFFER)).hasSize(indexed ? 2 : 0); + flush(); + Assertions.assertThat(execute("SELECT * FROM %s WHERE v = ?", EMPTY_BYTE_BUFFER)).hasSize(indexed ? 2 : 0); + } +} \ No newline at end of file From c1d32e04da951dfbe40e97361de9c2117962b9ce Mon Sep 17 00:00:00 2001 From: David Capwell <dcapwell@apache.org> Date: Tue, 11 Feb 2025 14:33:39 -0800 Subject: [PATCH 132/225] To improve accord interoperability test coverage, need to extend the harry model domain to handle more possible CQL states patch by David Capwell; reviewed by Ariel Weisberg for CASSANDRA-20156 --- .../cassandra/schema/TableMetadata.java | 13 + .../nodetool/formatter/TableBuilder.java | 20 + .../distributed/test/JavaDriverUtils.java | 28 + .../test/cql3/MultiNodeTableWalkBase.java | 129 ++ .../MultiNodeTableWalkWithReadRepairTest.java | 45 + ...ltiNodeTableWalkWithoutReadRepairTest.java | 40 + .../test/cql3/MultiNodeTokenConflictTest.java | 86 ++ .../test/cql3/SingleNodeTableWalkTest.java | 594 ++++++++ .../cql3/SingleNodeTokenConflictTest.java | 449 ++++++ .../test/cql3/StatefulASTBase.java | 470 +++++++ .../apache/cassandra/harry/SchemaSpec.java | 2 +- .../cassandra/harry/dsl/HistoryBuilder.java | 4 +- .../cassandra/harry/gen/BijectionCache.java | 129 ++ .../cassandra/harry/gen/ValueGenerators.java | 54 +- .../harry/model/ASTSingleTableModel.java | 1226 +++++++++++++++++ .../harry/model/ASTSingleTableModelTest.java | 788 +++++++++++ .../harry/model/BytesPartitionState.java | 592 ++++++++ .../cassandra/harry/model/PartitionState.java | 26 +- .../cassandra/harry/util/StringUtils.java | 22 + .../org/apache/cassandra/cql3/KnownIssue.java | 53 + .../cassandra/cql3/ast/Conditional.java | 45 +- .../apache/cassandra/cql3/ast/Literal.java | 15 +- .../apache/cassandra/cql3/ast/Mutation.java | 100 +- .../org/apache/cassandra/cql3/ast/Select.java | 111 +- .../cassandra/cql3/ast/StandardVisitors.java | 10 + .../apache/cassandra/utils/ASTGenerators.java | 53 +- .../cassandra/utils/ImmutableUniqueList.java | 173 +++ 27 files changed, 5167 insertions(+), 110 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkBase.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkWithReadRepairTest.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkWithoutReadRepairTest.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTokenConflictTest.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTableWalkTest.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTokenConflictTest.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/cql3/StatefulASTBase.java create mode 100644 test/harry/main/org/apache/cassandra/harry/gen/BijectionCache.java create mode 100644 test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModel.java create mode 100644 test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModelTest.java create mode 100644 test/harry/main/org/apache/cassandra/harry/model/BytesPartitionState.java create mode 100644 test/unit/org/apache/cassandra/cql3/KnownIssue.java create mode 100644 test/unit/org/apache/cassandra/utils/ImmutableUniqueList.java diff --git a/src/java/org/apache/cassandra/schema/TableMetadata.java b/src/java/org/apache/cassandra/schema/TableMetadata.java index 6f9672d3c1dd..c2f53592482e 100644 --- a/src/java/org/apache/cassandra/schema/TableMetadata.java +++ b/src/java/org/apache/cassandra/schema/TableMetadata.java @@ -352,6 +352,19 @@ public ImmutableCollection<ColumnMetadata> columns() return columns.values(); } + /** + * Same as {@link #columns} but the list returned is in {@link #allColumnsInSelectOrder()}. + * + * This method is needed by tests that need deterministic ordering; {@link #columns()} returns in hash order + * so isn't consistent cross different jvms or hosts. + */ + public List<ColumnMetadata> columnsInFixedOrder() + { + List<ColumnMetadata> columnMetadata = new ArrayList<>(columns.size()); + allColumnsInSelectOrder().forEachRemaining(columnMetadata::add); + return columnMetadata; + } + public Iterable<ColumnMetadata> primaryKeyColumns() { return Iterables.concat(partitionKeyColumns, clusteringColumns); diff --git a/src/java/org/apache/cassandra/tools/nodetool/formatter/TableBuilder.java b/src/java/org/apache/cassandra/tools/nodetool/formatter/TableBuilder.java index a95cb9ec7e85..ab4ee0306359 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/formatter/TableBuilder.java +++ b/src/java/org/apache/cassandra/tools/nodetool/formatter/TableBuilder.java @@ -31,6 +31,8 @@ import java.util.stream.Collectors; import javax.annotation.Nonnull; +import com.google.common.collect.Iterables; + /** * Build and print table. * @@ -76,6 +78,24 @@ private TableBuilder(TableBuilder base, int[] maximumColumnWidth) this.rows.addAll(base.rows); } + public static String toString(String columnDelimiter, Iterable<List<String>> rows) + { + TableBuilder builder = new TableBuilder(columnDelimiter); + for (List<String> row : rows) + builder.add(row); + return builder.toString(); + } + + public static String toString(String columnDelimiter, List<String> columns, Iterable<List<String>> rows) + { + return toString(columnDelimiter, Iterables.concat(Collections.singleton(columns), rows)); + } + + public static String toStringPiped(List<String> columns, Iterable<List<String>> rows) + { + return toString(" | ", columns, rows); + } + public void add(@Nonnull List<String> row) { add(row.toArray(new String[0])); diff --git a/test/distributed/org/apache/cassandra/distributed/test/JavaDriverUtils.java b/test/distributed/org/apache/cassandra/distributed/test/JavaDriverUtils.java index ae257314589a..9bfde925639d 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/JavaDriverUtils.java +++ b/test/distributed/org/apache/cassandra/distributed/test/JavaDriverUtils.java @@ -23,6 +23,7 @@ import java.util.function.Consumer; import java.util.stream.Collectors; +import com.datastax.driver.core.ConsistencyLevel; import com.datastax.driver.core.ProtocolVersion; import org.apache.cassandra.distributed.api.Feature; import org.apache.cassandra.distributed.api.ICluster; @@ -35,6 +36,11 @@ private JavaDriverUtils() { } + public static com.datastax.driver.core.Cluster create(ICluster<? extends IInstance> dtest, Consumer<com.datastax.driver.core.Cluster.Builder> overrideBuilder) + { + return create(dtest, null, overrideBuilder); + } + public static com.datastax.driver.core.Cluster create(ICluster<? extends IInstance> dtest) { return create(dtest, null, null); @@ -77,4 +83,26 @@ public static List<InetSocketAddress> buildContactPoints(ICluster<? extends IIns .map(ClusterUtils::getNativeInetSocketAddress) .collect(Collectors.toList()); } + + public static ConsistencyLevel toDriverCL(org.apache.cassandra.distributed.api.ConsistencyLevel cl) + { + switch (cl) + { + case ONE: return ConsistencyLevel.ONE; + case TWO: return ConsistencyLevel.TWO; + case THREE: return ConsistencyLevel.THREE; + case LOCAL_ONE: return ConsistencyLevel.LOCAL_ONE; + case LOCAL_QUORUM: return ConsistencyLevel.LOCAL_QUORUM; + case SERIAL: return ConsistencyLevel.SERIAL; + case LOCAL_SERIAL: return ConsistencyLevel.LOCAL_SERIAL; + case QUORUM: return ConsistencyLevel.QUORUM; + case EACH_QUORUM: return ConsistencyLevel.EACH_QUORUM; + case ANY: return ConsistencyLevel.ANY; + case ALL: return ConsistencyLevel.ALL; + case NODE_LOCAL: + throw new AssertionError("NODE_LOCAL is not supported by driver and should go directly through jvm-dtest api"); + default: + throw new UnsupportedOperationException("Unknown ConsistencyLevel: " + cl); + } + } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkBase.java b/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkBase.java new file mode 100644 index 000000000000..f10d0edcf4d8 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkBase.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.cql3; + +import java.io.IOException; + +import accord.utils.RandomSource; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.reads.repair.ReadRepairStrategy; + +public abstract class MultiNodeTableWalkBase extends SingleNodeTableWalkTest +{ + /** + * This field lets the test run as if it was multiple nodes, but actually runs against a single node. + * This behavior is desirable when this test fails to see if the issue can be reproduced on single node as well. + */ + private static final boolean mockMultiNode = false; + + private final ReadRepairStrategy readRepair; + + protected MultiNodeTableWalkBase(ReadRepairStrategy readRepair) + { + this.readRepair = readRepair; + } + + @Override + protected TableMetadata defineTable(RandomSource rs, String ks) + { + TableMetadata tbl = super.defineTable(rs, ks); + return tbl.unbuild().params(tbl.params.unbuild().readRepair(readRepair).build()).build(); + } + + @Override + protected Cluster createCluster() throws IOException + { + return createCluster(mockMultiNode ? 1 : 3, c -> { + c.set("range_request_timeout", "180s") + .set("read_request_timeout", "180s") + .set("write_request_timeout", "180s") + .set("native_transport_timeout", "180s") + .set("slow_query_log_timeout", "180s"); + }); + } + + @Override + protected State createState(RandomSource rs, Cluster cluster) + { + return new MultiNodeState(rs, cluster); + } + + private class MultiNodeState extends State + { + public MultiNodeState(RandomSource rs, Cluster cluster) + { + super(rs, cluster); + } + + @Override + public boolean allowNonPartitionQuery() + { + // This is disabled to make CI stable. There are known issues that are being fixed so have to exclude for now + return false; + } + + @Override + public boolean allowNonPartitionMultiColumnQuery() + { + // This is disabled to make CI stable. There are known issues that are being fixed so have to exclude for now + return false; + } + + @Override + public boolean allowPartitionQuery() + { + // This is disabled to make CI stable. There are known issues that are being fixed so have to exclude for now + return false; + } + + @Override + protected boolean isMultiNode() + { + // When a seed fails its useful to rerun the test as a single node to see if the issue persists... but doing so corrupts the random history! + // To avoid that, this method hard codes that the test is multi node... + return true; + } + + @Override + protected IInvokableInstance selectInstance(RandomSource rs) + { + if (mockMultiNode) + { + rs.nextInt(0, 3); // needed to avoid breaking random history + return cluster.get(1); + } + return super.selectInstance(rs); + } + + @Override + protected ConsistencyLevel selectCl() + { + return ConsistencyLevel.ALL; + } + + @Override + protected ConsistencyLevel mutationCl() + { + return ConsistencyLevel.NODE_LOCAL; + } + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkWithReadRepairTest.java b/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkWithReadRepairTest.java new file mode 100644 index 000000000000..e8b01f8c712a --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkWithReadRepairTest.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.cql3; + +import org.junit.Ignore; + +import accord.utils.Property; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.service.reads.repair.ReadRepairStrategy; + +@Ignore("In order to stay stable RR tests are ignored for now. Once Single node and multi node w/o RR are stable, then this test should be enabled to include RR testing") +public class MultiNodeTableWalkWithReadRepairTest extends MultiNodeTableWalkBase +{ + public MultiNodeTableWalkWithReadRepairTest() + { + super(ReadRepairStrategy.BLOCKING); + } + + @Override + protected void preCheck(Cluster cluster, Property.StatefulBuilder builder) + { + // if a failing seed is detected, populate here + // Example: builder.withSeed(42L); + // CQL operations may have opertors such as +, -, and / (example 4 + 4), to "apply" them to get a constant value +// CQL_DEBUG_APPLY_OPERATOR = true; + // When an issue is found, it's a good idea to also run the same seed against MultiNodeTableWalkWithoutReadRepairTest; if Read Repair is given bad input, you should expect bad output! + // This test needs to make sure it shares the same random history as MultiNodeTableWalkWithoutReadRepairTest to always allow the ability to maintain this property. + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkWithoutReadRepairTest.java b/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkWithoutReadRepairTest.java new file mode 100644 index 000000000000..a18b80d68ad2 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkWithoutReadRepairTest.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.cql3; + +import accord.utils.Property; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.service.reads.repair.ReadRepairStrategy; + +public class MultiNodeTableWalkWithoutReadRepairTest extends MultiNodeTableWalkBase +{ + public MultiNodeTableWalkWithoutReadRepairTest() + { + super(ReadRepairStrategy.NONE); + } + + @Override + protected void preCheck(Cluster cluster, Property.StatefulBuilder builder) + { + // if a failing seed is detected, populate here + // Example: builder.withSeed(42L); + // CQL operations may have opertors such as +, -, and / (example 4 + 4), to "apply" them to get a constant value +// CQL_DEBUG_APPLY_OPERATOR = true; + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTokenConflictTest.java b/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTokenConflictTest.java new file mode 100644 index 000000000000..2d296277cfd4 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTokenConflictTest.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.cql3; + +import java.io.IOException; + +import accord.utils.Property; +import accord.utils.RandomSource; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.reads.repair.ReadRepairStrategy; + +public class MultiNodeTokenConflictTest extends SingleNodeTokenConflictTest +{ + @Override + protected void preCheck(Property.StatefulBuilder builder) + { + // if a failing seed is detected, populate here + // Example: builder.withSeed(42L); + // CQL operations may have opertors such as +, -, and / (example 4 + 4), to "apply" them to get a constant value + // CQL_DEBUG_APPLY_OPERATOR = true; + } + + @Override + protected TableMetadata defineTable(RandomSource rs, String ks) + { + TableMetadata tbl = super.defineTable(rs, ks); + // disable RR for now, should make RR testing its own class + return tbl.unbuild().params(tbl.params.unbuild().readRepair(ReadRepairStrategy.NONE).build()).build(); + } + + @Override + protected Cluster createCluster() throws IOException + { + return createCluster(3, c -> { + c.set("range_request_timeout", "180s") + .set("read_request_timeout", "180s") + .set("write_request_timeout", "180s") + .set("native_transport_timeout", "180s") + .set("slow_query_log_timeout", "180s"); + }); + } + + @Override + protected State createState(RandomSource rs, Cluster cluster) + { + return new MultiNodeState(rs, cluster); + } + + private class MultiNodeState extends State + { + MultiNodeState(RandomSource rs, Cluster cluster) + { + super(rs, cluster); + } + + @Override + protected ConsistencyLevel selectCl() + { + return ConsistencyLevel.ALL; + } + + @Override + protected ConsistencyLevel mutationCl() + { + return ConsistencyLevel.NODE_LOCAL; + } + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTableWalkTest.java b/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTableWalkTest.java new file mode 100644 index 000000000000..6412fdf2d9ae --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTableWalkTest.java @@ -0,0 +1,594 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.cql3; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.EnumSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.NavigableSet; +import java.util.Optional; +import java.util.TreeMap; +import java.util.stream.Collectors; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Sets; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import accord.utils.Gen; +import accord.utils.Gens; +import accord.utils.Property; +import accord.utils.RandomSource; +import org.apache.cassandra.cql3.KnownIssue; +import org.apache.cassandra.cql3.ast.Bind; +import org.apache.cassandra.cql3.ast.Conditional; +import org.apache.cassandra.cql3.ast.CreateIndexDDL; +import org.apache.cassandra.cql3.ast.FunctionCall; +import org.apache.cassandra.cql3.ast.Mutation; +import org.apache.cassandra.cql3.ast.ReferenceExpression; +import org.apache.cassandra.cql3.ast.Select; +import org.apache.cassandra.cql3.ast.Symbol; +import org.apache.cassandra.cql3.ast.TableReference; +import org.apache.cassandra.cql3.ast.Value; +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.InetAddressType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.test.sai.SAIUtil; +import org.apache.cassandra.harry.model.BytesPartitionState; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.ASTGenerators; +import org.apache.cassandra.utils.AbstractTypeGenerators; +import org.apache.cassandra.utils.AbstractTypeGenerators.TypeGenBuilder; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.CassandraGenerators.TableMetadataBuilder; +import org.apache.cassandra.utils.ImmutableUniqueList; +import org.quicktheories.generators.SourceDSL; + +import static accord.utils.Property.commands; +import static accord.utils.Property.stateful; +import static org.apache.cassandra.utils.AbstractTypeGenerators.getTypeSupport; +import static org.apache.cassandra.utils.Generators.toGen; + +public class SingleNodeTableWalkTest extends StatefulASTBase +{ + private static final Logger logger = LoggerFactory.getLogger(SingleNodeTableWalkTest.class); + + protected void preCheck(Cluster cluster, Property.StatefulBuilder builder) + { + // if a failing seed is detected, populate here + // Example: builder.withSeed(42L); + // CQL operations may have opertors such as +, -, and / (example 4 + 4), to "apply" them to get a constant value + // CQL_DEBUG_APPLY_OPERATOR = true; + } + + protected TypeGenBuilder supportedTypes() + { + return AbstractTypeGenerators.withoutUnsafeEquality(AbstractTypeGenerators.builder() + .withTypeKinds(AbstractTypeGenerators.TypeKind.PRIMITIVE)); + } + + protected List<CreateIndexDDL.Indexer> supportedIndexers() + { + // since legacy is async it's not clear how the test can wait for the background write to complete... + return Collections.singletonList(CreateIndexDDL.SAI); + } + + public Property.Command<State, Void, ?> selectExisting(RandomSource rs, State state) + { + NavigableSet<BytesPartitionState.Ref> keys = state.model.partitionKeys(); + BytesPartitionState.Ref ref = rs.pickOrderedSet(keys); + Clustering<ByteBuffer> key = ref.key; + + Select.Builder builder = Select.builder().table(state.metadata); + ImmutableUniqueList<Symbol> pks = state.model.factory.partitionColumns; + ImmutableUniqueList<Symbol> cks = state.model.factory.clusteringColumns; + for (Symbol pk : pks) + builder.value(pk, key.bufferAt(pks.indexOf(pk))); + + boolean wholePartition = cks.isEmpty() || rs.nextBoolean(); + if (!wholePartition) + { + // find a row to select + BytesPartitionState partition = state.model.get(ref); + if (partition.isEmpty()) + { + wholePartition = true; + } + else + { + NavigableSet<Clustering<ByteBuffer>> clusteringKeys = partition.clusteringKeys(); + Clustering<ByteBuffer> clusteringKey = rs.pickOrderedSet(clusteringKeys); + for (Symbol ck : cks) + builder.value(ck, clusteringKey.bufferAt(cks.indexOf(ck))); + } + } + Select select = builder.build(); + return state.command(rs, select, (wholePartition ? "Whole Partition" : "Single Row")); + } + + public Property.Command<State, Void, ?> selectToken(RandomSource rs, State state) + { + NavigableSet<BytesPartitionState.Ref> keys = state.model.partitionKeys(); + BytesPartitionState.Ref ref = rs.pickOrderedSet(keys); + + Select.Builder builder = Select.builder().table(state.metadata); + builder.where(FunctionCall.tokenByColumns(state.model.factory.partitionColumns), + Conditional.Where.Inequality.EQUAL, + token(state, ref)); + + Select select = builder.build(); + return state.command(rs, select, "by token"); + } + + public Property.Command<State, Void, ?> selectTokenRange(RandomSource rs, State state) + { + NavigableSet<BytesPartitionState.Ref> keys = state.model.partitionKeys(); + BytesPartitionState.Ref start, end; + switch (keys.size()) + { + case 1: + start = end = Iterables.get(keys, 0); + break; + case 2: + start = Iterables.get(keys, 0); + end = Iterables.get(keys, 1); + break; + case 0: + throw new IllegalArgumentException("Unable to select token ranges when no partitions exist"); + default: + { + int si = rs.nextInt(0, keys.size() - 1); + int ei = rs.nextInt(si + 1, keys.size()); + start = Iterables.get(keys, si); + end = Iterables.get(keys, ei); + } + break; + } + Select.Builder builder = Select.builder().table(state.metadata); + FunctionCall pkToken = FunctionCall.tokenByColumns(state.model.factory.partitionColumns); + boolean startInclusive = rs.nextBoolean(); + boolean endInclusive = rs.nextBoolean(); + if (startInclusive && endInclusive && rs.nextBoolean()) + { + // between + builder.between(pkToken, token(state, start), token(state, end)); + } + else + { + builder.where(pkToken, + startInclusive ? Conditional.Where.Inequality.GREATER_THAN_EQ : Conditional.Where.Inequality.GREATER_THAN, + token(state, start)); + builder.where(pkToken, + endInclusive ? Conditional.Where.Inequality.LESS_THAN_EQ : Conditional.Where.Inequality.LESS_THAN, + token(state, end)); + } + Select select = builder.build(); + return state.command(rs, select, "by token range"); + } + + public Property.Command<State, Void, ?> partitionRestrictedQuery(RandomSource rs, State state) + { + //TODO (now): remove duplicate logic + NavigableSet<BytesPartitionState.Ref> keys = state.model.partitionKeys(); + BytesPartitionState.Ref ref = rs.pickOrderedSet(keys); + Clustering<ByteBuffer> key = ref.key; + + Select.Builder builder = Select.builder().table(state.metadata); + ImmutableUniqueList<Symbol> pks = state.model.factory.partitionColumns; + for (Symbol pk : pks) + builder.value(pk, key.bufferAt(pks.indexOf(pk))); + + + Symbol symbol; + List<Symbol> searchableColumns = state.nonPartitionColumns; + if (state.hasMultiNodeAllowFilteringWithLocalWritesIssue()) + { + if (state.nonPkIndexedColumns.isEmpty()) + throw new AssertionError("Ignoring AF_MULTI_NODE_AND_NODE_LOCAL_WRITES is defined, but no non-partition columns are indexed"); + symbol = rs.pick(state.nonPkIndexedColumns); + } + else + { + symbol = rs.pick(searchableColumns); + } + + TreeMap<ByteBuffer, List<BytesPartitionState.PrimaryKey>> universe = state.model.index(ref, symbol); + // we need to index 'null' so LT works, but we can not directly query it... so filter out when selecting values + NavigableSet<ByteBuffer> allowed = Sets.filter(universe.navigableKeySet(), b -> !ByteBufferUtil.EMPTY_BYTE_BUFFER.equals(b)); + if (allowed.isEmpty()) + return Property.ignoreCommand(); + ByteBuffer value = rs.pickOrderedSet(allowed); + + EnumSet<CreateIndexDDL.QueryType> supported = !state.indexes.containsKey(symbol) + ? EnumSet.noneOf(CreateIndexDDL.QueryType.class) + : state.indexes.get(symbol).supportedQueries(); + if (supported.isEmpty() || !supported.contains(CreateIndexDDL.QueryType.Range)) + builder.allowFiltering(); + + // there are known SAI bugs, so need to avoid them to stay stable... + if (state.indexes.containsKey(symbol) && state.indexes.get(symbol).indexDDL.indexer == CreateIndexDDL.SAI) + { + if (symbol.type() == InetAddressType.instance + && IGNORED_ISSUES.contains(KnownIssue.SAI_INET_MIXED)) + return eqSearch(rs, state, symbol, value, builder); + } + + if (rs.nextBoolean()) + return simpleRangeSearch(rs, state, symbol, value, builder); + //TODO (coverage): define search that has a upper and lower bound: a > and a < | a beteeen ? and ? + return eqSearch(rs, state, symbol, value, builder); + } + + public Property.Command<State, Void, ?> nonPartitionQuery(RandomSource rs, State state) + { + Symbol symbol; + if (state.hasMultiNodeAllowFilteringWithLocalWritesIssue()) + { + symbol = rs.pickUnorderedSet(state.indexes.keySet()); + } + else + { + symbol = rs.pick(state.searchableColumns); + } + TreeMap<ByteBuffer, List<BytesPartitionState.PrimaryKey>> universe = state.model.index(symbol); + // we need to index 'null' so LT works, but we can not directly query it... so filter out when selecting values + NavigableSet<ByteBuffer> allowed = Sets.filter(universe.navigableKeySet(), b -> !ByteBufferUtil.EMPTY_BYTE_BUFFER.equals(b)); + if (allowed.isEmpty()) + return Property.ignoreCommand(); + ByteBuffer value = rs.pickOrderedSet(allowed); + Select.Builder builder = Select.builder().table(state.metadata); + + EnumSet<CreateIndexDDL.QueryType> supported = !state.indexes.containsKey(symbol) ? EnumSet.noneOf(CreateIndexDDL.QueryType.class) : state.indexes.get(symbol).supportedQueries(); + if (supported.isEmpty() || !supported.contains(CreateIndexDDL.QueryType.Range)) + builder.allowFiltering(); + + // there are known SAI bugs, so need to avoid them to stay stable... + if (state.indexes.containsKey(symbol) && state.indexes.get(symbol).indexDDL.indexer == CreateIndexDDL.SAI) + { + if (symbol.type() == InetAddressType.instance + && IGNORED_ISSUES.contains(KnownIssue.SAI_INET_MIXED)) + return eqSearch(rs, state, symbol, value, builder); + } + + if (rs.nextBoolean()) + return simpleRangeSearch(rs, state, symbol, value, builder); + //TODO (coverage): define search that has a upper and lower bound: a > and a < | a beteeen ? and ? + return eqSearch(rs, state, symbol, value, builder); + } + + public Property.Command<State, Void, ?> multiColumnQuery(RandomSource rs, State state) + { + List<Symbol> allowedColumns = state.multiColumnQueryColumns(); + + if (allowedColumns.size() <= 1) + throw new IllegalArgumentException("Unable to do multiple column query when there is only a single column"); + + int numColumns = rs.nextInt(1, allowedColumns.size()) + 1; + + List<Symbol> cols = Gens.lists(Gens.pick(allowedColumns)).unique().ofSize(numColumns).next(rs); + + Select.Builder builder = Select.builder().table(state.metadata).allowFiltering(); + + for (Symbol symbol : cols) + { + TreeMap<ByteBuffer, List<BytesPartitionState.PrimaryKey>> universe = state.model.index(symbol); + NavigableSet<ByteBuffer> allowed = Sets.filter(universe.navigableKeySet(), b -> !ByteBufferUtil.EMPTY_BYTE_BUFFER.equals(b)); + //TODO (now): support + if (allowed.isEmpty()) + return Property.ignoreCommand(); + ByteBuffer value = rs.pickOrderedSet(allowed); + builder.value(symbol, value); + } + + Select select = builder.build(); + String annotate = cols.stream().map(symbol -> { + var indexed = state.indexes.get(symbol); + return symbol.detailedName() + (indexed == null ? "" : " (indexed with " + indexed.indexDDL.indexer.name() + ")"); + }).collect(Collectors.joining(", ")); + return state.command(rs, select, annotate); + } + + private Property.Command<State, Void, ?> simpleRangeSearch(RandomSource rs, State state, Symbol symbol, ByteBuffer value, Select.Builder builder) + { + // do a simple search, like > or < + Conditional.Where.Inequality kind = state.rangeInequalityGen.next(rs); + builder.where(symbol, kind, value); + Select select = builder.build(); + var indexed = state.indexes.get(symbol); + return state.command(rs, select, symbol.detailedName() + (indexed == null ? "" : ", indexed with " + indexed.indexDDL.indexer.name())); + } + + private Property.Command<State, Void, ?> eqSearch(RandomSource rs, State state, Symbol symbol, ByteBuffer value, Select.Builder builder) + { + builder.value(symbol, value); + + Select select = builder.build(); + var indexed = state.indexes.get(symbol); + return state.command(rs, select, symbol.detailedName() + (indexed == null ? "" : ", indexed with " + indexed.indexDDL.indexer.name())); + } + + protected State createState(RandomSource rs, Cluster cluster) + { + return new State(rs, cluster); + } + + protected Cluster createCluster() throws IOException + { + return createCluster(1, i -> {}); + } + + @Test + public void test() throws IOException + { + try (Cluster cluster = createCluster()) + { + Property.StatefulBuilder statefulBuilder = stateful().withExamples(10).withSteps(400); + preCheck(cluster, statefulBuilder); + statefulBuilder.check(commands(() -> rs -> createState(rs, cluster)) + .add(StatefulASTBase::insert) + .add(StatefulASTBase::fullTableScan) + .addIf(State::hasPartitions, this::selectExisting) + .addAllIf(State::supportTokens, b -> b.add(this::selectToken) + .add(this::selectTokenRange)) + .addIf(State::hasEnoughMemtable, StatefulASTBase::flushTable) + .addIf(State::hasEnoughSSTables, StatefulASTBase::compactTable) + .addIf(State::allowNonPartitionQuery, this::nonPartitionQuery) + .addIf(State::allowNonPartitionMultiColumnQuery, this::multiColumnQuery) + .addIf(State::allowPartitionQuery, this::partitionRestrictedQuery) + .destroyState(State::close) + .onSuccess(onSuccess(logger)) + .build()); + } + } + + protected TableMetadata defineTable(RandomSource rs, String ks) + { + //TODO (correctness): the id isn't correct... this is what we use to create the table, so would miss the actual ID + // Defaults may also be incorrect, but given this is the same version it "shouldn't" + //TODO (coverage): partition is defined at the cluster level, so have to hard code in this model as the table is changed rather than cluster being recreated... this limits coverage + return toGen(new TableMetadataBuilder() + .withTableKinds(TableMetadata.Kind.REGULAR) + .withKnownMemtables() + .withKeyspaceName(ks).withTableName("tbl") + .withSimpleColumnNames() + .withDefaultTypeGen(supportedTypes()) + .withPartitioner(Murmur3Partitioner.instance) + .build()) + .next(rs); + } + + private List<CreateIndexDDL.Indexer> columnSupportsIndexing(TableMetadata metadata, ColumnMetadata col) + { + return supportedIndexers().stream() + .filter(i -> i.supported(metadata, col)) + .collect(Collectors.toList()); + } + + private static FunctionCall token(State state, BytesPartitionState.Ref ref) + { + Preconditions.checkNotNull(ref.key); + List<Value> values = new ArrayList<>(ref.key.size()); + for (int i = 0; i < ref.key.size(); i++) + { + ByteBuffer bb = ref.key.bufferAt(i); + Symbol type = state.model.factory.partitionColumns.get(i); + values.add(new Bind(bb, type.type())); + } + return FunctionCall.tokenByValue(values); + } + + public class State extends CommonState + { + protected final LinkedHashMap<Symbol, IndexedColumn> indexes; + private final Gen<Mutation> mutationGen; + private final List<Symbol> nonPartitionColumns; + private final List<Symbol> searchableColumns; + + public State(RandomSource rs, Cluster cluster) + { + super(rs, cluster, defineTable(rs, nextKeyspace())); + + this.indexes = createIndexes(rs, metadata); + + cluster.forEach(i -> i.nodetoolResult("disableautocompaction", metadata.keyspace, this.metadata.name).asserts().success()); + + List<LinkedHashMap<Symbol, Object>> uniquePartitions; + { + int unique = rs.nextInt(1, 10); + List<Symbol> columns = model.factory.partitionColumns; + List<Gen<?>> gens = new ArrayList<>(columns.size()); + for (int i = 0; i < columns.size(); i++) + gens.add(toGen(getTypeSupport(columns.get(i).type()).valueGen)); + uniquePartitions = Gens.lists(r2 -> { + LinkedHashMap<Symbol, Object> vs = new LinkedHashMap<>(); + for (int i = 0; i < columns.size(); i++) + vs.put(columns.get(i), gens.get(i).next(r2)); + return vs; + }).uniqueBestEffort().ofSize(unique).next(rs); + } + + ASTGenerators.MutationGenBuilder mutationGenBuilder = new ASTGenerators.MutationGenBuilder(metadata) + .withoutTransaction() + .withoutTtl() + .withoutTimestamp() + .withPartitions(SourceDSL.arbitrary().pick(uniquePartitions)); + if (IGNORED_ISSUES.contains(KnownIssue.SAI_EMPTY_TYPE)) + { + model.factory.regularAndStaticColumns.stream() + // exclude SAI indexed columns + .filter(s -> !indexes.containsKey(s) || indexes.get(s).indexDDL.indexer != CreateIndexDDL.SAI) + .forEach(mutationGenBuilder::allowEmpty); + } + else + { + model.factory.regularAndStaticColumns.forEach(mutationGenBuilder::allowEmpty); + } + this.mutationGen = toGen(mutationGenBuilder.build()); + + nonPartitionColumns = ImmutableList.<Symbol>builder() + .addAll(model.factory.clusteringColumns) + .addAll(model.factory.staticColumns) + .addAll(model.factory.regularColumns) + .build(); + + searchableColumns = metadata.partitionKeyColumns().size() > 1 ? model.factory.selectionOrder : nonPartitionColumns; + } + + @Override + protected Gen<Mutation> mutationGen() + { + return mutationGen; + } + + private LinkedHashMap<Symbol, IndexedColumn> createIndexes(RandomSource rs, TableMetadata metadata) + { + LinkedHashMap<Symbol, IndexedColumn> indexed = new LinkedHashMap<>(); + // for some test runs, avoid using indexes + if (rs.nextBoolean()) + return indexed; + for (ColumnMetadata col : metadata.columnsInFixedOrder()) + { + Symbol symbol = Symbol.from(col); + AbstractType<?> type = symbol.type(); + + if (col.name.toString().length() >= 48 + && IGNORED_ISSUES.contains(KnownIssue.CUSTOM_INDEX_MAX_COLUMN_48)) + continue; + + if (type.isCollection() && !type.isFrozenCollection()) continue; //TODO (coverage): include non-frozen collections; the index part works fine, its the select that fails... basic equality isn't allowed for map type... so how do you query? + List<CreateIndexDDL.Indexer> allowed = columnSupportsIndexing(metadata, col); + if (allowed.isEmpty()) continue; + CreateIndexDDL.Indexer indexer = rs.pick(allowed); + ReferenceExpression colExpression = Symbol.from(col); + if (type.isFrozenCollection()) + colExpression = new CreateIndexDDL.CollectionReference(CreateIndexDDL.CollectionReference.Kind.FULL, colExpression); + + String name = "tbl_" + col.name; + CreateIndexDDL ddl = new CreateIndexDDL(rs.pick(CreateIndexDDL.Version.values()), + indexer, + Optional.of(new Symbol(name, UTF8Type.instance)), + TableReference.from(metadata), + Collections.singletonList(colExpression), + Collections.emptyMap()); + String stmt = ddl.toCQL(); + logger.info(stmt); + cluster.schemaChange(stmt); + + //noinspection OptionalGetWithoutIsPresent + SAIUtil.waitForIndexQueryable(cluster, metadata.keyspace, ddl.name.get().name()); + + indexed.put(symbol, new IndexedColumn(symbol, ddl)); + } + return indexed; + } + + public boolean hasPartitions() + { + return !model.isEmpty(); + } + + public boolean supportTokens() + { + return hasPartitions(); + } + + public boolean allowNonPartitionQuery() + { + boolean result = !model.isEmpty() && !searchableColumns.isEmpty(); + if (hasMultiNodeAllowFilteringWithLocalWritesIssue()) + { + return hasNonPkIndexedColumns() && result; + } + return result; + } + + public boolean allowNonPartitionMultiColumnQuery() + { + return allowNonPartitionQuery() && multiColumnQueryColumns().size() > 1; + } + + private List<Symbol> multiColumnQueryColumns() + { + List<Symbol> allowedColumns = searchableColumns; + if (hasMultiNodeAllowFilteringWithLocalWritesIssue()) + allowedColumns = nonPkIndexedColumns; + return allowedColumns; + } + + private boolean hasMultiNodeAllowFilteringWithLocalWritesIssue() + { + return isMultiNode() && IGNORED_ISSUES.contains(KnownIssue.AF_MULTI_NODE_AND_NODE_LOCAL_WRITES); + } + + public List<Symbol> nonPkIndexedColumns; + public boolean allowPartitionQuery() + { + if (model.isEmpty() || nonPartitionColumns.isEmpty()) return false; + if (hasMultiNodeAllowFilteringWithLocalWritesIssue()) + return hasNonPkIndexedColumns(); + return true; + } + + private boolean hasNonPkIndexedColumns() + { + nonPkIndexedColumns = nonPartitionColumns.stream() + .filter(indexes::containsKey) + .collect(Collectors.toList()); + return !nonPkIndexedColumns.isEmpty(); + } + + @Override + public String toString() + { + StringBuilder sb = new StringBuilder(); + sb.append("\nSetup:\n"); + toString(sb); + indexes.values().forEach(c -> sb.append('\n').append(c.indexDDL.toCQL()).append(';')); + return sb.toString(); + } + } + + public static class IndexedColumn + { + public final Symbol symbol; + public final CreateIndexDDL indexDDL; + + public IndexedColumn(Symbol symbol, CreateIndexDDL indexDDL) + { + this.symbol = symbol; + this.indexDDL = indexDDL; + } + + public EnumSet<CreateIndexDDL.QueryType> supportedQueries() + { + return indexDDL.indexer.supportedQueries(symbol.type()); + } + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTokenConflictTest.java b/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTokenConflictTest.java new file mode 100644 index 000000000000..179b107eae77 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTokenConflictTest.java @@ -0,0 +1,449 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.cql3; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.stream.Collectors; + +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import accord.utils.Gen; +import accord.utils.Gens; +import accord.utils.Property; +import accord.utils.RandomSource; +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.KnownIssue; +import org.apache.cassandra.cql3.ast.Conditional.Where.Inequality; +import org.apache.cassandra.cql3.ast.FunctionCall; +import org.apache.cassandra.cql3.ast.Mutation; +import org.apache.cassandra.cql3.ast.Select; +import org.apache.cassandra.cql3.ast.Symbol; +import org.apache.cassandra.cql3.ast.Value; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.db.marshal.VectorType; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.dht.Murmur3Partitioner.LongToken; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.tools.nodetool.formatter.TableBuilder; +import org.apache.cassandra.utils.ASTGenerators; +import org.apache.cassandra.utils.AbstractTypeGenerators; +import org.apache.cassandra.utils.AbstractTypeGenerators.TypeGenBuilder; +import org.apache.cassandra.utils.AbstractTypeGenerators.TypeKind; +import org.apache.cassandra.utils.AbstractTypeGenerators.TypeSupport; +import org.apache.cassandra.utils.CassandraGenerators.TableMetadataBuilder; +import org.apache.cassandra.utils.Generators; +import org.apache.cassandra.utils.ImmutableUniqueList; +import org.quicktheories.generators.SourceDSL; + +import static accord.utils.Property.commands; +import static accord.utils.Property.stateful; +import static org.apache.cassandra.dht.Murmur3Partitioner.LongToken.keyForToken; +import static org.apache.cassandra.utils.Generators.toGen; + +public class SingleNodeTokenConflictTest extends StatefulASTBase +{ + private static final Logger logger = LoggerFactory.getLogger(SingleNodeTokenConflictTest.class); + + private static final Gen.IntGen NUM_TOKENS_GEN = Gens.pickInt(1, 10, 100); + /** + * {@code vector<bigint, 2>} is 16 bytes, which perfectly lines up with {@link LongToken#keyForToken(long)}. + */ + private static final VectorType<Long> PK_TYPE = VectorType.getInstance(LongType.instance, 2); + private static final TypeSupport<List<Long>> TYPE_SUPPORT = AbstractTypeGenerators.getTypeSupport(PK_TYPE); + private static final Symbol PK = new Symbol("pk0", PK_TYPE); + private static final TypeGenBuilder SUPPORTED_TYPES = + AbstractTypeGenerators.withoutUnsafeEquality(AbstractTypeGenerators.builder() + .withTypeKinds(TypeKind.PRIMITIVE)); + + protected void preCheck(Property.StatefulBuilder builder) + { + // if a failing seed is detected, populate here + // Example: builder.withSeed(42L); + } + + public static Property.Command<State, Void, ?> pkEq(RandomSource rs, State state) + { + ByteBuffer value = state.pkGen.next(rs); + Select select = Select.builder() + .table(state.tableRef) + .value(PK, state.pkValue(rs, value)) + .build(); + return state.command(rs, select, "pk EQ"); + } + + public static Property.Command<State, Void, ?> pkIn(RandomSource rs, State state) + { + List<Value> expressions = state.randomPksAsValue(rs); + Select select = Select.builder() + .table(state.tableRef) + .in(PK, expressions) + .build(); + return state.command(rs, select, "pk IN"); + } + + public static Property.Command<State, Void, ?> pkBetween(RandomSource rs, State state) + { + ByteBuffer left = state.pkGen.next(rs); + ByteBuffer right = state.betweenEqGen.next(rs) ? left : state.pkGen.next(rs); + int rc = PK_TYPE.compare(left, right); + if (rc > 0 && IGNORED_ISSUES.contains(KnownIssue.BETWEEN_START_LARGER_THAN_END)) + { + ByteBuffer tmp = left; + left = right; + right = tmp; + rc = PK_TYPE.compare(left, right); + } + Select select = Select.builder() + .table(state.tableRef) + .between(PK, state.pkValue(rs, left), state.pkValue(rs, right)) + .allowFiltering() + .build(); + return state.command(rs, select, "pk BETWEEN, rc=" + rc); + } + + public static Property.Command<State, Void, ?> pkRange(RandomSource rs, State state) + { + ByteBuffer value = state.pkGen.next(rs); + Inequality inequality = state.rangeInequalityGen.next(rs); + Select select = Select.builder() + .table(state.tableRef) + .where(PK, inequality, value) + .allowFiltering() + .build(); + return state.command(rs, select, "pk " + inequality.value); + } + + public static Property.Command<State, Void, ?> pkBoundRange(RandomSource rs, State state) + { + ByteBuffer left = state.pkGen.next(rs); + Inequality lefIneq = state.lessThanGen.next(rs); + + ByteBuffer right = state.pkGen.next(rs); + Inequality rightIneq = state.greaterThanGen.next(rs); + Select select = Select.builder() + .table(state.tableRef) + .where(PK, lefIneq, left) + .where(PK, rightIneq, right) + .allowFiltering() + .build(); + return state.command(rs, select, "pk " + lefIneq.value + " AND " + rightIneq.value); + } + + public static Property.Command<State, Void, ?> tokenEq(RandomSource rs, State state) + { + ByteBuffer value = state.pkGen.next(rs); + Select select = Select.builder() + .table(state.tableRef) + .where(FunctionCall.tokenByColumns(PK), Inequality.EQUAL, FunctionCall.tokenByValue(state.pkValue(rs, value))) + .build(); + return state.command(rs, select, "token EQ"); + } + + // As of 5.1 IN clause is limited to columns / fields / collections +// public static Property.Command<State, Void, ?> tokenIn(RandomSource rs, State state) +// { +// List<Value> expressions = state.randomPksAsValue(rs); +// Select select = Select.builder() +// .table(state.tableRef) +// .in(FunctionCall.tokenByColumns(PK), expressions.stream().map(FunctionCall::tokenByValue).collect(Collectors.toList())) +// .build(); +// return state.command(select, "token IN"); +// } + + public static Property.Command<State, Void, ?> tokenBetween(RandomSource rs, State state) + { + ByteBuffer left = state.pkGen.next(rs); + ByteBuffer right = state.betweenEqGen.next(rs) ? left : state.pkGen.next(rs); + LongToken start = Murmur3Partitioner.instance.getToken(left); + LongToken end = Murmur3Partitioner.instance.getToken(right); + int rc = start.compareTo(end); + if (rc > 0 && IGNORED_ISSUES.contains(KnownIssue.BETWEEN_START_LARGER_THAN_END)) + { + ByteBuffer tmp = left; + left = right; + right = tmp; + LongToken tmp2 = start; + start = end; + end = tmp2; + rc = start.compareTo(end); + } + Select select = Select.builder() + .table(state.tableRef) + .between(FunctionCall.tokenByColumns(PK), + FunctionCall.tokenByValue(state.pkValue(rs, left)), + FunctionCall.tokenByValue(state.pkValue(rs, right))) + .build(); + return state.command(rs, select, "token BETWEEN, rc=" + rc + + ", start token=" + start + + ", end token=" + end); + } + + public static Property.Command<State, Void, ?> tokenRange(RandomSource rs, State state) + { + ByteBuffer value = state.pkGen.next(rs); + Inequality inequality = state.rangeInequalityGen.next(rs); + Select select = Select.builder() + .table(state.tableRef) + .where(FunctionCall.tokenByColumns(PK), inequality, FunctionCall.tokenByValue(state.pkValue(rs, value))) + .allowFiltering() + .build(); + return state.command(rs, select, "token " + inequality.value + " " + Murmur3Partitioner.instance.getToken(value)); + } + + public static Property.Command<State, Void, ?> tokenBoundRange(RandomSource rs, State state) + { + ByteBuffer left = state.pkGen.next(rs); + Inequality lefIneq = state.lessThanGen.next(rs); + + ByteBuffer right = state.pkGen.next(rs); + Inequality rightIneq = state.greaterThanGen.next(rs); + Select select = Select.builder() + .table(state.tableRef) + .where(FunctionCall.tokenByColumns(PK), lefIneq, FunctionCall.tokenByValue(state.pkValue(rs, left))) + .where(FunctionCall.tokenByColumns(PK), rightIneq, FunctionCall.tokenByValue(state.pkValue(rs, right))) + .allowFiltering() + .build(); + return state.command(rs, select, "token " + lefIneq.value + " " + Murmur3Partitioner.instance.getToken(left) + + " AND " + rightIneq.value + " " + Murmur3Partitioner.instance.getToken(right)); + } + + protected Cluster createCluster() throws IOException + { + return createCluster(1, i -> {}); + } + + @Test + public void test() throws IOException + { + try (Cluster cluster = createCluster()) + { + Property.StatefulBuilder statefulBuilder = stateful().withExamples(10); + preCheck(statefulBuilder); + statefulBuilder.check(commands(() -> rs -> createState(rs, cluster)) + .add(StatefulASTBase::insert) + //TODO (now, coverage): this is flakey and non-deterministic. When this fails (gives bad response) rerunning the seed yields a passing test! +// .add(StatefulASTBase::fullTableScan) + .add(SingleNodeTokenConflictTest::pkEq) + .add(SingleNodeTokenConflictTest::pkIn) + .add(SingleNodeTokenConflictTest::pkBetween) + .add(SingleNodeTokenConflictTest::pkRange) + .add(SingleNodeTokenConflictTest::pkBoundRange) + .add(SingleNodeTokenConflictTest::tokenEq) + // there is no tokenIn, as of this moment token in does not compile in CQL + .add(SingleNodeTokenConflictTest::tokenBetween) + .add(SingleNodeTokenConflictTest::tokenRange) + .add(SingleNodeTokenConflictTest::tokenBoundRange) + .addIf(State::hasEnoughMemtable, StatefulASTBase::flushTable) + .addIf(State::hasEnoughSSTables, StatefulASTBase::compactTable) + .destroyState(State::close) + .onSuccess(onSuccess(logger)) + .build()); + } + } + + protected State createState(RandomSource rs, Cluster cluster) + { + return new State(rs, cluster); + } + + protected TableMetadata defineTable(RandomSource rs, String ks) + { + return toGen(new TableMetadataBuilder() + .withTableKinds(TableMetadata.Kind.REGULAR) + .withKnownMemtables() + .withKeyspaceName(ks).withTableName("tbl") + .withSimpleColumnNames() + .withDefaultTypeGen(SUPPORTED_TYPES) + .withPartitioner(Murmur3Partitioner.instance) + .withPartitionColumnsCount(1) + // this should produce vector<bigint, 2>... should make this easier... + .withPartitionColumnTypeGen(new TypeGenBuilder() + .withMaxDepth(0) + .withTypeKinds(TypeKind.VECTOR) + .withPrimitives(LongType.instance) + .withVectorSizeGen(i -> 2) + .withDefaultSizeGen(1)) + .build()) + .next(rs); + } + + class State extends CommonState + { + private final List<ByteBuffer> neighbors; + private final List<ByteBuffer> pkValues; + private final Gen<ByteBuffer> pkGen; + private final TreeMap<ByteBuffer, ByteBuffer> realToSynthMap; + private final TreeSet<ByteBuffer> order; + + private final Gen<Mutation> mutationGen; + + State(RandomSource rs, Cluster cluster) + { + super(rs, cluster, defineTable(rs, nextKeyspace())); + { + int numTokens = NUM_TOKENS_GEN.nextInt(rs); + var pkValues = Gens.lists(Generators.toGen(TYPE_SUPPORT.bytesGen())).unique().ofSize(numTokens).next(rs); + // now create conflicting values + var tokenValues = pkValues.stream().map(SingleNodeTokenConflictTest::toTokenValue).collect(Collectors.toList()); + // this is low probability... but just in case... if there are duplicates, drop them! + Set<ByteBuffer> seen = new HashSet<>(); + for (int i = 0; i < pkValues.size(); i++) + { + var real = pkValues.get(i); + var synth = tokenValues.get(i); + if (real.equals(synth) || !seen.add(real) || !seen.add(synth)) + { + // drop + pkValues.remove(i); + tokenValues.remove(i); + i--; + } + } + if (pkValues.isEmpty()) + throw new AssertionError("There are no values after filtering duplicates..."); + this.neighbors = rs.nextBoolean() ? Collections.emptyList() : extractNeighbors(pkValues); + // in case neighbors conflicts with pkValues or tokenValues, use ImmutableUniqueList which will ignore rather than fail + this.pkValues = ImmutableUniqueList.<ByteBuffer>builder() + .mayAddAll(pkValues) + .mayAddAll(tokenValues) + .mayAddAll(neighbors) + .build(); + this.pkGen = Gens.pick(pkValues); + this.order = new TreeSet<>(PK_TYPE); + realToSynthMap = new TreeMap<>(Comparator.comparing(Murmur3Partitioner.instance::getToken)); + for (int i = 0; i < pkValues.size(); i++) + { + ByteBuffer r = pkValues.get(i); + ByteBuffer s = tokenValues.get(i); + realToSynthMap.put(r, s); + order.add(r); + order.add(s); + } + } + + // double check pk0 + if (!PK.equals(Symbol.from(metadata.getColumn(new ColumnIdentifier("pk0", false))))) + throw new AssertionError("Table doesn't match what the test expects;\n" + metadata.toCqlString(false, false, false)); + + cluster.forEach(i -> i.nodetoolResult("disableautocompaction", metadata.keyspace, this.metadata.name).asserts().success()); + + List<Map<Symbol, Object>> uniquePartitions = new ArrayList<>(pkValues.size()); + pkValues.forEach(bb -> uniquePartitions.add(Map.of(PK, bb))); + + + this.mutationGen = toGen(new ASTGenerators.MutationGenBuilder(metadata) + .withoutTransaction() + .withoutTtl() + .withoutTimestamp() + .withPartitions(SourceDSL.arbitrary().pick(uniquePartitions)) + .build()); + } + + @Override + protected Gen<Mutation> mutationGen() + { + return mutationGen; + } + + private List<ByteBuffer> extractNeighbors(List<ByteBuffer> values) + { + // if the same value is added multiple times this data structure will ignore the addition, making sure the + // returned list only has unique values + ImmutableUniqueList.Builder<ByteBuffer> neighbors = ImmutableUniqueList.builder(); + for (ByteBuffer bb : values) + { + var token = Murmur3Partitioner.instance.getToken(bb); + if (token.token > Long.MIN_VALUE + 1) + neighbors.add(keyForToken(token.token - 1)); + if (token.token < Long.MAX_VALUE) + neighbors.add(keyForToken(token.token + 1)); + } + return neighbors.build(); + } + + private LinkedHashSet<ByteBuffer> randomPks(RandomSource rs) + { + int numPks = rs.nextInt(1, pkValues.size()); + // when numPks is large the cost to keep trying to find the few remaining pk values is costly... by cloning + // the set can be mutated to remove the value, making it so this logic can avoid retries + LinkedHashSet<ByteBuffer> available = new LinkedHashSet<>(pkValues); + LinkedHashSet<ByteBuffer> pks = new LinkedHashSet<>(); + for (int i = 0; i < numPks; i++) + { + ByteBuffer value = rs.pickOrderedSet(available); + pks.add(value); + available.remove(value); + } + return pks; + } + + private List<Value> randomPksAsValue(RandomSource rs) + { + LinkedHashSet<ByteBuffer> pks = randomPks(rs); + List<Value> expressions = new ArrayList<>(pks.size()); + pks.forEach(bb -> expressions.add(pkValue(rs, bb))); + return expressions; + } + + private Value pkValue(RandomSource rs, ByteBuffer bb) + { + return value(rs, bb, PK_TYPE); + } + + private String pkCQL(ByteBuffer bb) + { + return PK_TYPE.toCQLString(bb); + } + + @Override + public String toString() + { + StringBuilder sb = new StringBuilder(); + toString(sb); + sb.append("\n\n-- Value to Conflicting Token Map\n"); + sb.append(TableBuilder.toStringPiped(Arrays.asList("Real", "Synthetic"), + realToSynthMap.entrySet().stream().map(e -> Arrays.asList(pkCQL(e.getKey()), pkCQL(e.getValue()))).collect(Collectors.toList()))); + sb.append("\n\n-- Ordered values"); + order.forEach(e -> sb.append("\n\t").append(pkCQL(e)).append('\t')); + sb.append("\n\n-- Neighbors"); + neighbors.forEach(bb -> sb.append("\n\t").append(pkCQL(bb)).append('\t')); + return sb.toString(); + } + } + + private static ByteBuffer toTokenValue(ByteBuffer buffer) + { + return keyForToken(Murmur3Partitioner.instance.getToken(buffer)); + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/cql3/StatefulASTBase.java b/test/distributed/org/apache/cassandra/distributed/test/cql3/StatefulASTBase.java new file mode 100644 index 000000000000..ac5df71ddf97 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/cql3/StatefulASTBase.java @@ -0,0 +1,470 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.cql3; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.EnumSet; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import javax.annotation.Nullable; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; + +import accord.utils.Gen; +import accord.utils.Gens; +import accord.utils.Property; +import accord.utils.RandomSource; +import com.datastax.driver.core.ColumnDefinitions; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Row; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.SimpleStatement; +import com.datastax.driver.core.SocketOptions; +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.KnownIssue; +import org.apache.cassandra.cql3.ast.Bind; +import org.apache.cassandra.cql3.ast.CQLFormatter; +import org.apache.cassandra.cql3.ast.Conditional; +import org.apache.cassandra.cql3.ast.Literal; +import org.apache.cassandra.cql3.ast.Mutation; +import org.apache.cassandra.cql3.ast.Select; +import org.apache.cassandra.cql3.ast.StandardVisitors; +import org.apache.cassandra.cql3.ast.Statement; +import org.apache.cassandra.cql3.ast.TableReference; +import org.apache.cassandra.cql3.ast.Value; +import org.apache.cassandra.cql3.ast.Visitor; +import org.apache.cassandra.cql3.ast.Visitor.CompositeVisitor; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.AsciiType; +import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.api.IInstance; +import org.apache.cassandra.distributed.api.IInstanceConfig; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.test.JavaDriverUtils; +import org.apache.cassandra.distributed.test.TestBaseImpl; +import org.apache.cassandra.harry.model.ASTSingleTableModel; +import org.apache.cassandra.harry.util.StringUtils; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.AbstractTypeGenerators; +import org.apache.cassandra.utils.CassandraGenerators; +import org.apache.cassandra.utils.FastByteOperations; +import org.apache.cassandra.utils.Generators; +import org.quicktheories.generators.SourceDSL; + +import static org.apache.cassandra.distributed.test.JavaDriverUtils.toDriverCL; +import static org.apache.cassandra.utils.AbstractTypeGenerators.overridePrimitiveTypeSupport; +import static org.apache.cassandra.utils.AbstractTypeGenerators.stringComparator; + +public class StatefulASTBase extends TestBaseImpl +{ + protected static final EnumSet<KnownIssue> IGNORED_ISSUES = KnownIssue.ignoreAll(); + /** + * mutations and selects will use operators (eg. {@code 4 + 4}, the + operator), and this will be reflected in the history output. + * + * When an issue is found its common to filter out insertions to different partitions/rows but this can become a problem + * as the issue is for {@code pk=8} but the insert is to {@code 4 + 4}! + * + * Setting this to {@code true} will cause all operators to be "applied" or "executored" and the CQL in the history + * will be the output (eg. {@code 4 + 4 } is replaced with {@code 8}). + */ + protected static boolean CQL_DEBUG_APPLY_OPERATOR = false; + + protected static final Gen<Gen<Boolean>> BIND_OR_LITERAL_DISTRO = Gens.bools().mixedDistribution(); + protected static final Gen<Gen<Boolean>> BETWEEN_EQ_DISTRO = Gens.bools().mixedDistribution(); + protected static final Gen<Gen<Conditional.Where.Inequality>> LESS_THAN_DISTRO = Gens.mixedDistribution(Stream.of(Conditional.Where.Inequality.values()) + .filter(i -> i == Conditional.Where.Inequality.LESS_THAN || i == Conditional.Where.Inequality.LESS_THAN_EQ) + .collect(Collectors.toList())); + protected static final Gen<Gen<Conditional.Where.Inequality>> GREATER_THAN_DISTRO = Gens.mixedDistribution(Stream.of(Conditional.Where.Inequality.values()) + .filter(i -> i == Conditional.Where.Inequality.GREATER_THAN || i == Conditional.Where.Inequality.GREATER_THAN_EQ) + .collect(Collectors.toList())); + protected static final Gen<Gen<Conditional.Where.Inequality>> RANGE_INEQUALITY_DISTRO = Gens.mixedDistribution(Stream.of(Conditional.Where.Inequality.values()) + .filter(i -> i != Conditional.Where.Inequality.EQUAL && i != Conditional.Where.Inequality.NOT_EQUAL) + .collect(Collectors.toList())); + protected static final Gen<Gen.IntGen> FETCH_SIZE_DISTRO = Gens.mixedDistribution(new int[] {1, 10, 100, 1000, 5000}); + + static + { + // since this test does frequent truncates, the info table gets updated and forced flushed... which is 90% of the cost of this test... + // this flag disables that flush + CassandraRelevantProperties.UNSAFE_SYSTEM.setBoolean(true); + // queries maybe dumb which could lead to performance issues causing timeouts... don't timeout! + CassandraRelevantProperties.SAI_TEST_DISABLE_TIMEOUT.setBoolean(true); + + overridePrimitiveTypeSupport(AsciiType.instance, AbstractTypeGenerators.TypeSupport.of(AsciiType.instance, SourceDSL.strings().ascii().ofLengthBetween(1, 10), stringComparator(AsciiType.instance))); + overridePrimitiveTypeSupport(UTF8Type.instance, AbstractTypeGenerators.TypeSupport.of(UTF8Type.instance, Generators.utf8(1, 10), stringComparator(UTF8Type.instance))); + overridePrimitiveTypeSupport(BytesType.instance, AbstractTypeGenerators.TypeSupport.of(BytesType.instance, Generators.bytes(1, 10), FastByteOperations::compareUnsigned)); + } + + /** + * There is an assumption that keyspace name doesn't impact this test, so to get simpler names use this counter... + * if this assumption doesn't hold, then need to switch to random or rely on DROP KEYSPACE. + */ + private static final AtomicInteger COUNTER = new AtomicInteger(); + + protected static String nextKeyspace() + { + return "ks" + COUNTER.incrementAndGet(); + } + + protected static Cluster createCluster(int nodeCount, Consumer<IInstanceConfig> config) throws IOException + { + Cluster cluster = Cluster.build(nodeCount) + .withConfig(c -> { + c.with(Feature.NATIVE_PROTOCOL, Feature.NETWORK, Feature.GOSSIP) + // When drop tables or truncate are performed, we attempt to take snapshots. This can be costly and isn't needed by these tests + .set("incremental_backups", false); + config.accept(c); + }) + .start(); + // we don't allow setting null in yaml... but these configs support null! + cluster.forEach(i -> i.runOnInstance(() -> { + // When values are large SAI will drop them... soooo... disable that... this test does not care about perf but correctness + DatabaseDescriptor.getRawConfig().sai_frozen_term_size_warn_threshold = null; + DatabaseDescriptor.getRawConfig().sai_frozen_term_size_fail_threshold = null; + })); + return cluster; + } + + protected <S extends BaseState> Property.StatefulSuccess<S, Void> onSuccess(Logger logger) + { + return (state, sut, history) -> logger.info("Successful for the following:\nState {}\nHistory:\n{}", state, Property.formatList("\t\t", history)); + } + + protected static <S extends BaseState> Property.Command<S, Void, ?> flushTable(RandomSource rs, S state) + { + return new Property.SimpleCommand<>("nodetool flush " + state.metadata.keyspace + " " + state.metadata.name, s2 -> { + s2.cluster.forEach(i -> i.nodetoolResult("flush", s2.metadata.keyspace, s2.metadata.name).asserts().success()); + s2.flush(); + }); + } + + protected static <S extends BaseState> Property.Command<S, Void, ?> compactTable(RandomSource rs, S state) + { + return new Property.SimpleCommand<>("nodetool compact " + state.metadata.keyspace + " " + state.metadata.name, s2 -> { + state.cluster.forEach(i -> i.nodetoolResult("compact", s2.metadata.keyspace, s2.metadata.name).asserts().success()); + s2.compact(); + }); + } + + protected static <S extends CommonState> Property.Command<S, Void, ?> insert(RandomSource rs, S state) + { + int timestamp = ++state.operations; + return state.command(rs, state.mutationGen().next(rs).withTimestamp(timestamp)); + } + + protected static <S extends BaseState> Property.Command<S, Void, ?> fullTableScan(RandomSource rs, S state) + { + Select select = Select.builder(state.metadata).build(); + return state.command(rs, select, "full table scan"); + } + + protected static abstract class BaseState implements AutoCloseable + { + protected final RandomSource rs; + protected final Cluster cluster; + protected final com.datastax.driver.core.Cluster client; + protected final Session session; + protected final Gen<Boolean> bindOrLiteralGen; + protected final Gen<Boolean> betweenEqGen; + protected final Gen<Conditional.Where.Inequality> lessThanGen; + protected final Gen<Conditional.Where.Inequality> greaterThanGen; + protected final Gen<Conditional.Where.Inequality> rangeInequalityGen; + protected final Gen.IntGen fetchSizeGen; + protected final TableMetadata metadata; + protected final TableReference tableRef; + protected final ASTSingleTableModel model; + private final Visitor debug; + private final int enoughMemtables; + private final int enoughSSTables; + protected int numMutations, mutationsSinceLastFlush; + protected int numFlushes, flushesSinceLastCompaction; + protected int numCompact; + protected int operations; + + protected BaseState(RandomSource rs, Cluster cluster, TableMetadata metadata) + { + this.rs = rs; + this.cluster = cluster; + int javaDriverTimeout = Math.toIntExact(TimeUnit.MINUTES.toMillis(1)); + this.client = JavaDriverUtils.create(cluster, b -> b.withSocketOptions(new SocketOptions().setReadTimeoutMillis(javaDriverTimeout).setConnectTimeoutMillis(javaDriverTimeout))); + this.session = client.connect(); + this.debug = CQL_DEBUG_APPLY_OPERATOR ? CompositeVisitor.of(StandardVisitors.APPLY_OPERATOR, StandardVisitors.DEBUG) + : StandardVisitors.DEBUG; + + this.bindOrLiteralGen = BIND_OR_LITERAL_DISTRO.next(rs); + this.betweenEqGen = BETWEEN_EQ_DISTRO.next(rs); + this.lessThanGen = LESS_THAN_DISTRO.next(rs); + this.greaterThanGen = GREATER_THAN_DISTRO.next(rs); + this.rangeInequalityGen = RANGE_INEQUALITY_DISTRO.next(rs); + this.fetchSizeGen = FETCH_SIZE_DISTRO.next(rs); + + this.enoughMemtables = rs.pickInt(3, 10, 50); + this.enoughSSTables = rs.pickInt(3, 10, 50); + + this.metadata = metadata; + this.tableRef = TableReference.from(metadata); + this.model = new ASTSingleTableModel(metadata); + createTable(metadata); + } + + protected boolean isMultiNode() + { + return cluster.size() > 1; + } + + protected void createTable(TableMetadata metadata) + { + cluster.schemaChange(createKeyspaceCQL(metadata.keyspace)); + + CassandraGenerators.visitUDTs(metadata, next -> cluster.schemaChange(next.toCqlString(false, false, true))); + cluster.schemaChange(metadata.toCqlString(false, false, false)); + } + + private String createKeyspaceCQL(String ks) + { + return "CREATE KEYSPACE IF NOT EXISTS " + ks + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': " + Math.min(3, cluster.size()) + "};"; + } + + protected <S extends BaseState> Property.Command<S, Void, ?> command(RandomSource rs, Select select) + { + return command(rs, select, null); + } + + protected <S extends BaseState> Property.Command<S, Void, ?> command(RandomSource rs, Select select, @Nullable String annotate) + { + var inst = selectInstance(rs); + //TODO (coverage): don't limit this to all selects, only those doing range queries! + int fetchSize = fetchSizeGen.nextInt(rs); + String postfix = "on " + inst; + if (fetchSize != Integer.MAX_VALUE) + postfix += ", fetch size " + fetchSize; + if (annotate == null) annotate = postfix; + else annotate += ", " + postfix; + return new Property.SimpleCommand<>(humanReadable(select, annotate), s -> { + s.model.validate(s.executeQuery(inst, fetchSize, s.selectCl(), select), select); + }); + } + + protected ConsistencyLevel selectCl() + { + return ConsistencyLevel.LOCAL_QUORUM; + } + + protected ConsistencyLevel mutationCl() + { + return ConsistencyLevel.LOCAL_QUORUM; + } + + protected <S extends BaseState> Property.Command<S, Void, ?> command(RandomSource rs, Mutation mutation) + { + return command(rs, mutation, null); + } + + protected <S extends BaseState> Property.Command<S, Void, ?> command(RandomSource rs, Mutation mutation, @Nullable String annotate) + { + var inst = selectInstance(rs); + String postfix = "on " + inst; + if (annotate == null) annotate = postfix; + else annotate += ", " + postfix; + return new Property.SimpleCommand<>(humanReadable(mutation, annotate), s -> { + s.executeQuery(inst, Integer.MAX_VALUE, s.mutationCl(), mutation); + s.model.update(mutation); + s.mutation(); + }); + } + + protected IInvokableInstance selectInstance(RandomSource rs) + { + return cluster.get(rs.nextInt(0, cluster.size()) + 1); + } + + protected boolean hasEnoughMemtable() + { + return mutationsSinceLastFlush > enoughMemtables; + } + + protected boolean hasEnoughSSTables() + { + return flushesSinceLastCompaction > enoughSSTables; + } + + protected void mutation() + { + numMutations++; + mutationsSinceLastFlush++; + } + + protected void flush() + { + mutationsSinceLastFlush = 0; + numFlushes++; + flushesSinceLastCompaction++; + } + + protected void compact() + { + flushesSinceLastCompaction = 0; + numCompact++; + } + + protected Value value(RandomSource rs, ByteBuffer bb, AbstractType<?> type) + { + return bindOrLiteralGen.next(rs) ? new Bind(bb, type) : new Literal(bb, type); + } + + protected ByteBuffer[][] executeQuery(IInstance instance, int fetchSize, ConsistencyLevel cl, Statement stmt) + { + if (cl == ConsistencyLevel.NODE_LOCAL) + { + // This limitation is due to the fact the query column types are not known in the current QueryResult API. + // In order to fix this we need to alter the API, and backport to each branch else we break upgrade. + if (!(stmt instanceof Mutation)) + throw new IllegalArgumentException("Unable to execute Statement of type " + stmt.getClass() + " when ConsistencyLevel.NODE_LOCAL is used"); + if (fetchSize != Integer.MAX_VALUE) + throw new IllegalArgumentException("Fetch size is not allowed for Mutations"); + instance.executeInternal(stmt.toCQL(), (Object[]) stmt.bindsEncoded()); + return new ByteBuffer[0][]; + } + else + { + SimpleStatement ss = new SimpleStatement(stmt.toCQL(), (Object[]) stmt.bindsEncoded()); + if (fetchSize != Integer.MAX_VALUE) + ss.setFetchSize(fetchSize); + ss.setConsistencyLevel(toDriverCL(cl)); + + InetSocketAddress broadcastAddress = instance.config().broadcastAddress(); + var host = client.getMetadata().getAllHosts().stream() + .filter(h -> h.getBroadcastSocketAddress().getAddress().equals(broadcastAddress.getAddress())) + .filter(h -> h.getBroadcastSocketAddress().getPort() == broadcastAddress.getPort()) + .findAny() + .get(); + ss.setHost(host); + ResultSet result = session.execute(ss); + return getRowsAsByteBuffer(result); + } + } + + @VisibleForTesting + static ByteBuffer[][] getRowsAsByteBuffer(ResultSet result) + { + ColumnDefinitions columns = result.getColumnDefinitions(); + List<ByteBuffer[]> ret = new ArrayList<>(); + for (Row rowVal : result) + { + ByteBuffer[] row = new ByteBuffer[columns.size()]; + for (int i = 0; i < columns.size(); i++) + row[i] = rowVal.getBytesUnsafe(i); + ret.add(row); + } + ByteBuffer[][] a = new ByteBuffer[ret.size()][]; + return ret.toArray(a); + } + + private String humanReadable(Statement stmt, @Nullable String annotate) + { + // With UTF-8 some chars can cause printing issues leading to error messages that don't reproduce the original issue. + // To avoid this problem, always escape the CQL so nothing gets lost + String cql = StringUtils.escapeControlChars(stmt.visit(debug).toCQL(CQLFormatter.None.instance)); + if (annotate != null) + cql += " -- " + annotate; + return cql; + } + + protected void toString(StringBuilder sb) + { + sb.append(createKeyspaceCQL(metadata.keyspace)); + CassandraGenerators.visitUDTs(metadata, udt -> sb.append('\n').append(udt.toCqlString(false, false, true)).append(';')); + sb.append('\n').append(metadata.toCqlString(false, false, false)); + } + + @Override + public void close() throws Exception + { + session.close(); + client.close(); + cluster.schemaChange("DROP TABLE " + metadata); + cluster.schemaChange("DROP KEYSPACE " + metadata.keyspace); + } + + @Override + public String toString() + { + StringBuilder sb = new StringBuilder(); + toString(sb); + return sb.toString(); + } + + private static final class ValueWithType + { + final ByteBuffer value; + final AbstractType type; + + private ValueWithType(ByteBuffer value, AbstractType<?> type) + { + this.value = value; + this.type = type; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ValueWithType value1 = (ValueWithType) o; + return value.equals(value1.value) && type.equals(value1.type); + } + + @Override + public int hashCode() + { + return Objects.hash(value, type); + } + + @Override + public String toString() + { + return type.toCQLString(value); + } + } + } + + protected static abstract class CommonState extends BaseState + { + protected CommonState(RandomSource rs, Cluster cluster, TableMetadata metadata) + { + super(rs, cluster, metadata); + } + + protected abstract Gen<Mutation> mutationGen(); + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/SchemaSpec.java b/test/harry/main/org/apache/cassandra/harry/SchemaSpec.java index 5ab6862994ac..dbfa6e5f53b6 100644 --- a/test/harry/main/org/apache/cassandra/harry/SchemaSpec.java +++ b/test/harry/main/org/apache/cassandra/harry/SchemaSpec.java @@ -45,7 +45,7 @@ public class SchemaSpec public final List<ColumnSpec<?>> staticColumns; public final List<ColumnSpec<?>> allColumnInSelectOrder; - public final ValueGenerators valueGenerators; + public final ValueGenerators<Object[], Object[]> valueGenerators; public final Options options; public SchemaSpec(long seed, diff --git a/test/harry/main/org/apache/cassandra/harry/dsl/HistoryBuilder.java b/test/harry/main/org/apache/cassandra/harry/dsl/HistoryBuilder.java index 35631d55a0fa..c4565a0a74c6 100644 --- a/test/harry/main/org/apache/cassandra/harry/dsl/HistoryBuilder.java +++ b/test/harry/main/org/apache/cassandra/harry/dsl/HistoryBuilder.java @@ -372,7 +372,7 @@ public static IndexedValueGenerators valueGenerators(SchemaSpec schema, long see staticComparators); } - public static class IndexedValueGenerators extends ValueGenerators + public static class IndexedValueGenerators extends ValueGenerators<Object[], Object[]> { public IndexedValueGenerators(IndexedBijection<Object[]> pkGen, IndexedBijection<Object[]> ckGen, @@ -383,7 +383,7 @@ public IndexedValueGenerators(IndexedBijection<Object[]> pkGen, List<Comparator<Object>> regularComparators, List<Comparator<Object>> staticComparators) { - super(pkGen, ckGen, + super(pkGen, ckGen, ArrayAccessor.instance, (List<Bijections.Bijection<Object>>) (List<?>) regularColumnGens, (List<Bijections.Bijection<Object>>) (List<?>) staticColumnGens, pkComparators, ckComparators, regularComparators, staticComparators); diff --git a/test/harry/main/org/apache/cassandra/harry/gen/BijectionCache.java b/test/harry/main/org/apache/cassandra/harry/gen/BijectionCache.java new file mode 100644 index 000000000000..a388f195e956 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/gen/BijectionCache.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.gen; + +import java.util.Comparator; +import java.util.Set; + +import com.google.common.base.Preconditions; +import com.google.common.collect.BiMap; +import com.google.common.collect.HashBiMap; + +import org.apache.cassandra.harry.MagicConstants; + +public class BijectionCache<T> implements Bijections.Bijection<T> +{ + private final BiMap<T, Long> valueToDescriptor = HashBiMap.create(); + private final Comparator<? super T> comparator; + private long counter = 0; + + public BijectionCache(Comparator<? super T> comparator) + { + this.comparator = comparator; + } + + @Override + public T inflate(long descriptor) + { + T value = valueToDescriptor.inverse().get(descriptor); + if (value == null) + throw new IllegalArgumentException(String.format("Attempted to inflate %d, but it is undefined", descriptor)); + return value; + } + + @Override + public long deflate(T value) + { + Preconditions.checkNotNull(value, "Attempted to deflate 'null'"); + if (valueToDescriptor.containsKey(value)) + return valueToDescriptor.get(value); + long d = counter++; + valueToDescriptor.put(value, d); + return d; + } + + public long deflateOrUndefined(T value) + { + return valueToDescriptor.containsKey(value) ? valueToDescriptor.get(value) : MagicConstants.UNSET_DESCR; + } + + public Set<Long> descriptors() + { + return valueToDescriptor.inverse().keySet(); + } + + public Set<T> values() + { + return valueToDescriptor.keySet(); + } + + @Override + public int population() + { + throw new UnsupportedOperationException(); + } + + @Override + public long adjustEntropyDomain(long descriptor) + { + throw new UnsupportedOperationException(); + } + + @Override + public long minValue() + { + throw new UnsupportedOperationException(); + } + + @Override + public long maxValue() + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean unsigned() + { + throw new UnsupportedOperationException(); + } + + @Override + public Comparator<Long> descriptorsComparator() + { + return (a, b) -> comparator.compare(inflate(a), inflate(b)); + } + + @Override + public String toString(long pd) + { + throw new UnsupportedOperationException(); + } + + @Override + public int byteSize() + { + throw new UnsupportedOperationException(); + } + + @Override + public int compare(long l, long r) + { + throw new UnsupportedOperationException(); + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/gen/ValueGenerators.java b/test/harry/main/org/apache/cassandra/harry/gen/ValueGenerators.java index 28a671a09c88..9c8d62ba1d5f 100644 --- a/test/harry/main/org/apache/cassandra/harry/gen/ValueGenerators.java +++ b/test/harry/main/org/apache/cassandra/harry/gen/ValueGenerators.java @@ -21,23 +21,29 @@ import java.util.Comparator; import java.util.List; -public class ValueGenerators +import org.apache.cassandra.harry.gen.Bijections.Bijection; + +public class ValueGenerators<PartitionKey, ClusteringKey> { - protected final Bijections.Bijection<Object[]> pkGen; - protected final Bijections.Bijection<Object[]> ckGen; + protected final Bijection<PartitionKey> pkGen; + protected final Bijection<ClusteringKey> ckGen; + + protected final Accessor<ClusteringKey> ckAccessor; - protected final List<Bijections.Bijection<Object>> regularColumnGens; - protected final List<Bijections.Bijection<Object>> staticColumnGens; + protected final List<? extends Bijection<? extends Object>> regularColumnGens; + protected final List<? extends Bijection<? extends Object>> staticColumnGens; protected final List<Comparator<Object>> pkComparators; protected final List<Comparator<Object>> ckComparators; protected final List<Comparator<Object>> regularComparators; protected final List<Comparator<Object>> staticComparators; - public ValueGenerators(Bijections.Bijection<Object[]> pkGen, - Bijections.Bijection<Object[]> ckGen, - List<Bijections.Bijection<Object>> regularColumnGens, - List<Bijections.Bijection<Object>> staticColumnGens, + public ValueGenerators(Bijection<PartitionKey> pkGen, + Bijection<ClusteringKey> ckGen, + Accessor<ClusteringKey> ckAccessor, + + List<? extends Bijection<? extends Object>> regularColumnGens, + List<? extends Bijection<? extends Object>> staticColumnGens, List<Comparator<Object>> pkComparators, List<Comparator<Object>> ckComparators, @@ -46,6 +52,7 @@ public ValueGenerators(Bijections.Bijection<Object[]> pkGen, { this.pkGen = pkGen; this.ckGen = ckGen; + this.ckAccessor = ckAccessor; this.regularColumnGens = regularColumnGens; this.staticColumnGens = staticColumnGens; this.pkComparators = pkComparators; @@ -54,22 +61,22 @@ public ValueGenerators(Bijections.Bijection<Object[]> pkGen, this.staticComparators = staticComparators; } - public Bijections.Bijection<Object[]> pkGen() + public Bijection<PartitionKey> pkGen() { return pkGen; } - public Bijections.Bijection<Object[]> ckGen() + public Bijection<ClusteringKey> ckGen() { return ckGen; } - public Bijections.Bijection<Object> regularColumnGen(int idx) + public Bijection regularColumnGen(int idx) { return regularColumnGens.get(idx); } - public Bijections.Bijection<Object> staticColumnGen(int idx) + public Bijection staticColumnGen(int idx) { return staticColumnGens.get(idx); } @@ -109,6 +116,11 @@ public Comparator<Object> staticComparator(int idx) return staticComparators.get(idx); } + public Accessor<ClusteringKey> ckAccessor() + { + return ckAccessor; + } + public int pkPopulation() { return pkGen.population(); @@ -128,4 +140,20 @@ public int staticPopulation(int i) { return staticColumnGens.get(i).population(); } + + public interface Accessor<T> + { + Object access(int field, T value); + } + + public enum ArrayAccessor implements Accessor<Object[]> + { + instance; + + @Override + public Object access(int field, Object[] value) + { + return value[field]; + } + } } \ No newline at end of file diff --git a/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModel.java b/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModel.java new file mode 100644 index 000000000000..13180c061bf7 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModel.java @@ -0,0 +1,1226 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.model; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.NavigableSet; +import java.util.Objects; +import java.util.Set; +import java.util.TreeMap; +import java.util.function.IntFunction; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import javax.annotation.Nullable; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +import accord.utils.Invariants; +import org.apache.cassandra.cql3.ast.Conditional; +import org.apache.cassandra.cql3.ast.Conditional.Where.Inequality; +import org.apache.cassandra.cql3.ast.Element; +import org.apache.cassandra.cql3.ast.Expression; +import org.apache.cassandra.cql3.ast.ExpressionEvaluator; +import org.apache.cassandra.cql3.ast.FunctionCall; +import org.apache.cassandra.cql3.ast.Mutation; +import org.apache.cassandra.cql3.ast.Select; +import org.apache.cassandra.cql3.ast.StandardVisitors; +import org.apache.cassandra.cql3.ast.Symbol; +import org.apache.cassandra.db.BufferClustering; +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.harry.model.BytesPartitionState.PrimaryKey; +import org.apache.cassandra.harry.util.StringUtils; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.tools.nodetool.formatter.TableBuilder; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.ImmutableUniqueList; +import org.apache.cassandra.utils.Pair; + +import static org.apache.cassandra.harry.model.BytesPartitionState.asCQL; + +public class ASTSingleTableModel +{ + public final BytesPartitionState.Factory factory; + private final TreeMap<BytesPartitionState.Ref, BytesPartitionState> partitions = new TreeMap<>(); + + public ASTSingleTableModel(TableMetadata metadata) + { + this.factory = new BytesPartitionState.Factory(metadata); + } + + public NavigableSet<BytesPartitionState.Ref> partitionKeys() + { + return partitions.navigableKeySet(); + } + + public int size() + { + return partitions.size(); + } + + public boolean isEmpty() + { + return partitions.isEmpty(); + } + + public TreeMap<ByteBuffer, List<PrimaryKey>> index(BytesPartitionState.Ref ref, Symbol symbol) + { + if (factory.partitionColumns.contains(symbol)) + throw new AssertionError("When indexing based off a single partition, unable to index partition columns; given " + symbol.detailedName()); + BytesPartitionState partition = get(ref); + Invariants.nonNull(partition, "Unable to index %s; null partition %s", symbol, ref); + TreeMap<ByteBuffer, List<PrimaryKey>> index = new TreeMap<>(symbol.type()); + if (factory.staticColumns.contains(symbol)) + return indexStaticColumn(index, symbol, partition); + return indexRowColumn(index, symbol, partition); + } + + public TreeMap<ByteBuffer, List<PrimaryKey>> index(Symbol symbol) + { + TreeMap<ByteBuffer, List<PrimaryKey>> index = new TreeMap<>(symbol.type()); + if (factory.partitionColumns.contains(symbol)) + return indexPartitionColumn(index, symbol); + if (factory.staticColumns.contains(symbol)) + return indexStaticColumn(index, symbol); + return indexRowColumn(index, symbol); + } + + private TreeMap<ByteBuffer, List<PrimaryKey>> indexPartitionColumn(TreeMap<ByteBuffer, List<PrimaryKey>> index, Symbol symbol) + { + int offset = factory.partitionColumns.indexOf(symbol); + for (BytesPartitionState partition : partitions.values()) + { + if (partition.isEmpty()) continue; + ByteBuffer bb = partition.key.bufferAt(offset); + List<PrimaryKey> list = index.computeIfAbsent(bb, i -> new ArrayList<>()); + for (BytesPartitionState.Row row : partition.rows()) + list.add(row.ref()); + } + return index; + } + + private TreeMap<ByteBuffer, List<PrimaryKey>> indexStaticColumn(TreeMap<ByteBuffer, List<PrimaryKey>> index, Symbol symbol) + { + for (BytesPartitionState partition : partitions.values()) + indexStaticColumn(index, symbol, partition); + return index; + } + + private TreeMap<ByteBuffer, List<PrimaryKey>> indexStaticColumn(TreeMap<ByteBuffer, List<PrimaryKey>> index, Symbol symbol, BytesPartitionState partition) + { + if (partition.isEmpty()) return index; + ByteBuffer bb = partition.staticRow().get(symbol); + if (bb == null) + return index; + List<PrimaryKey> list = index.computeIfAbsent(bb, i -> new ArrayList<>()); + for (BytesPartitionState.Row row : partition.rows()) + list.add(row.ref()); + return index; + } + + private TreeMap<ByteBuffer, List<PrimaryKey>> indexRowColumn(TreeMap<ByteBuffer, List<PrimaryKey>> index, Symbol symbol) + { + boolean clustering = factory.clusteringColumns.contains(symbol); + int offset = clustering ? factory.clusteringColumns.indexOf(symbol) : factory.regularColumns.indexOf(symbol); + for (BytesPartitionState partition : partitions.values()) + indexRowColumn(index, clustering, offset, partition); + return index; + } + + private TreeMap<ByteBuffer, List<PrimaryKey>> indexRowColumn(TreeMap<ByteBuffer, List<PrimaryKey>> index, Symbol symbol, BytesPartitionState partition) + { + boolean clustering = factory.clusteringColumns.contains(symbol); + int offset = clustering ? factory.clusteringColumns.indexOf(symbol) : factory.regularColumns.indexOf(symbol); + indexRowColumn(index, clustering, offset, partition); + return index; + } + + private void indexRowColumn(TreeMap<ByteBuffer, List<PrimaryKey>> index, boolean clustering, int offset, BytesPartitionState partition) + { + if (partition.isEmpty()) return; + for (BytesPartitionState.Row row : partition.rows()) + { + ByteBuffer bb = clustering ? row.clustering.bufferAt(offset) : row.get(offset); + if (bb == null) + continue; + index.computeIfAbsent(bb, i -> new ArrayList<>()).add(row.ref()); + } + } + + public void update(Mutation mutation) + { + switch (mutation.kind) + { + case INSERT: + update((Mutation.Insert) mutation); + break; + case UPDATE: + update((Mutation.Update) mutation); + break; + case DELETE: + update((Mutation.Delete) mutation); + break; + default: + throw new UnsupportedOperationException(mutation.kind.name()); + } + } + + public void update(Mutation.Insert insert) + { + Clustering<ByteBuffer> pd = pd(insert); + BytesPartitionState partition = partitions.get(factory.createRef(pd)); + if (partition == null) + { + partition = factory.create(pd); + partitions.put(partition.ref(), partition); + } + Map<Symbol, Expression> values = insert.values; + if (!factory.staticColumns.isEmpty() && !Sets.intersection(factory.staticColumns.asSet(), values.keySet()).isEmpty()) + { + // static columns to add in. If we are doing something like += to a row that doesn't exist, we still update statics... + Map<Symbol, ByteBuffer> write = new HashMap<>(); + for (Symbol col : Sets.intersection(factory.staticColumns.asSet(), values.keySet())) + write.put(col, eval(values.get(col))); + partition.setStaticColumns(write); + } + // table has clustering but non are in the write, so only pk/static can be updated + if (!factory.clusteringColumns.isEmpty() && Sets.intersection(factory.clusteringColumns.asSet(), values.keySet()).isEmpty()) + return; + Map<Symbol, ByteBuffer> write = new HashMap<>(); + for (Symbol col : Sets.intersection(factory.regularColumns.asSet(), values.keySet())) + write.put(col, eval(values.get(col))); + partition.setColumns(key(insert.values, factory.clusteringColumns), + write, + true); + } + + public void update(Mutation.Update update) + { + var split = splitOnPartition(update.where.simplify()); + List<Clustering<ByteBuffer>> pks = split.left; + List<Conditional> remaining = split.right; + for (Clustering<ByteBuffer> pd : pks) + { + BytesPartitionState partition = partitions.get(factory.createRef(pd)); + if (partition == null) + { + partition = factory.create(pd); + partitions.put(partition.ref(), partition); + } + Map<Symbol, Expression> set = update.set; + if (!factory.staticColumns.isEmpty() && !Sets.intersection(factory.staticColumns.asSet(), set.keySet()).isEmpty()) + { + // static columns to add in. If we are doing something like += to a row that doesn't exist, we still update statics... + Map<Symbol, ByteBuffer> write = new HashMap<>(); + for (Symbol col : Sets.intersection(factory.staticColumns.asSet(), set.keySet())) + write.put(col, eval(set.get(col))); + partition.setStaticColumns(write); + } + // table has clustering but non are in the write, so only pk/static can be updated + if (!factory.clusteringColumns.isEmpty() && remaining.isEmpty()) + return; + for (Clustering<ByteBuffer> cd : clustering(remaining)) + { + Map<Symbol, ByteBuffer> write = new HashMap<>(); + for (Symbol col : Sets.intersection(factory.regularColumns.asSet(), set.keySet())) + write.put(col, eval(set.get(col))); + + partition.setColumns(cd, write, false); + } + } + } + + private enum DeleteKind + {PARTITION, ROW, COLUMN} + + public void update(Mutation.Delete delete) + { + //TODO (coverage): range deletes + var split = splitOnPartition(delete.where.simplify()); + List<Clustering<ByteBuffer>> pks = split.left; + List<Clustering<ByteBuffer>> clusterings = split.right.isEmpty() ? Collections.emptyList() : clustering(split.right); + HashSet<Symbol> columns = delete.columns.isEmpty() ? null : new HashSet<>(delete.columns); + for (Clustering<ByteBuffer> pd : pks) + { + BytesPartitionState partition = partitions.get(factory.createRef(pd)); + if (partition == null) return; // can't delete a partition that doesn't exist... + + DeleteKind kind = DeleteKind.PARTITION; + if (!delete.columns.isEmpty()) + kind = DeleteKind.COLUMN; + else if (!clusterings.isEmpty()) + kind = DeleteKind.ROW; + + switch (kind) + { + case PARTITION: + partitions.remove(partition.ref()); + break; + case ROW: + for (Clustering<ByteBuffer> cd : clusterings) + { + partition.deleteRow(cd); + if (partition.shouldDelete()) + partitions.remove(partition.ref()); + } + break; + case COLUMN: + if (clusterings.isEmpty()) + { + partition.deleteStaticColumns(columns); + if (partition.shouldDelete()) + partitions.remove(partition.ref()); + } + else + { + for (Clustering<ByteBuffer> cd : clusterings) + { + partition.deleteColumns(cd, columns); + if (partition.shouldDelete()) + partitions.remove(partition.ref()); + } + } + break; + default: + throw new UnsupportedOperationException(); + } + } + } + + private List<Clustering<ByteBuffer>> clustering(List<Conditional> conditionals) + { + if (conditionals.isEmpty()) + { + if (factory.clusteringColumns.isEmpty()) return Collections.singletonList(Clustering.EMPTY); + throw new IllegalArgumentException("No clustering columns defined in the WHERE clause, but clustering columns exist; expected " + factory.clusteringColumns); + } + var split = splitOnClustering(conditionals); + var clusterings = split.left; + var remaining = split.right; + if (!remaining.isEmpty()) + throw new IllegalArgumentException("Non Partition/Clustering columns found in WHERE clause; " + remaining.stream().map(Element::toCQL).collect(Collectors.joining(", "))); + return clusterings; + } + + private Pair<List<Clustering<ByteBuffer>>, List<Conditional>> splitOnPartition(List<Conditional> conditionals) + { + return splitOn(factory.partitionColumns.asSet(), conditionals); + } + + private Pair<List<Clustering<ByteBuffer>>, List<Conditional>> splitOnClustering(List<Conditional> conditionals) + { + return splitOn(factory.clusteringColumns.asSet(), conditionals); + } + + private Pair<List<Clustering<ByteBuffer>>, List<Conditional>> splitOn(ImmutableUniqueList<Symbol>.AsSet columns, List<Conditional> conditionals) + { + // pk requires equality + Map<Symbol, Set<ByteBuffer>> pks = new HashMap<>(); + List<Conditional> other = new ArrayList<>(); + for (Conditional c : conditionals) + { + if (c instanceof Conditional.Where) + { + Conditional.Where w = (Conditional.Where) c; + if (w.kind == Inequality.EQUAL && columns.contains(w.lhs)) + { + Symbol col = (Symbol) w.lhs; + ByteBuffer bb = eval(w.rhs); + if (pks.containsKey(col)) + throw new IllegalArgumentException("Partition column " + col + " was defined multiple times in the WHERE clause"); + pks.put(col, Collections.singleton(bb)); + } + else + { + other.add(c); + } + } + else if (c instanceof Conditional.In) + { + Conditional.In i = (Conditional.In) c; + if (columns.contains(i.ref)) + { + Symbol col = (Symbol) i.ref; + if (pks.containsKey(col)) + throw new IllegalArgumentException("Partition column " + col + " was defined multiple times in the WHERE clause"); + var set = i.expressions.stream().map(ASTSingleTableModel::eval).collect(Collectors.toSet()); + pks.put(col, set); + } + else + { + other.add(c); + } + } + else + { + other.add(c); + } + } + if (!columns.equals(pks.keySet())) + { + var missing = Sets.difference(columns, pks.keySet()); + throw new AssertionError("Unable to find expected columns " + missing); + } + + List<Clustering<ByteBuffer>> partitionKeys = keys(columns, pks); + return Pair.create(partitionKeys, other); + } + + private List<Clustering<ByteBuffer>> keys(Collection<Symbol> columns, Map<Symbol, Set<ByteBuffer>> pks) + { + //TODO (coverage): handle IN + ByteBuffer[] bbs = new ByteBuffer[columns.size()]; + int idx = 0; + for (Symbol s : columns) + { + Set<ByteBuffer> values = pks.get(s); + if (values.size() > 1) + throw new UnsupportedOperationException("IN clause is currently unsupported... its on the backlog!"); + bbs[idx++] = Iterables.getFirst(values, null); + } + return Collections.singletonList(BufferClustering.make(bbs)); + } + + private Clustering<ByteBuffer> pd(Mutation.Insert mutation) + { + return key(mutation.values, factory.partitionColumns); + } + + public BytesPartitionState get(BytesPartitionState.Ref ref) + { + return partitions.get(ref); + } + + public List<BytesPartitionState> getByToken(Token token) + { + NavigableSet<BytesPartitionState.Ref> keys = partitions.navigableKeySet(); + // To support the case where 2+ keys share the same token, need to create a token ref before and after the token, to make sure + // the head/tail sets find the matches correctly + NavigableSet<BytesPartitionState.Ref> matches = keys.headSet(factory.createRef(token, true), true) + .tailSet(factory.createRef(token, false), true); + if (matches.isEmpty()) return Collections.emptyList(); + return matches.stream().map(partitions::get).collect(Collectors.toList()); + } + + public void validate(ByteBuffer[][] actual, Select select) + { + SelectResult results = getRowsAsByteBuffer(select); + try + { + if (results.unordered) + { + validateAnyOrder(factory.selectionOrder, toRow(factory.selectionOrder, actual), toRow(factory.selectionOrder, results.rows)); + } + else + { + validate(actual, results.rows); + } + } + catch (AssertionError e) + { + AssertionError error = new AssertionError("Unexpected results for query: " + StringUtils.escapeControlChars(select.visit(StandardVisitors.DEBUG).toCQL()), e); + // This stack trace is not helpful, this error message is trying to improve the error returned to know what query failed, so the stack trace only adds noise + error.setStackTrace(new StackTraceElement[0]); + throw error; + } + } + + public void validate(ByteBuffer[][] actual, ByteBuffer[][] expected) + { + validate(factory.selectionOrder, actual, expected); + } + + private static void validate(ImmutableUniqueList<Symbol> columns, ByteBuffer[][] actual, ByteBuffer[][] expected) + { + // check any order + validateAnyOrder(columns, toRow(columns, actual), toRow(columns, expected)); + // all rows match, but are they in the right order? + validateOrder(columns, actual, expected); + } + + private static void validateAnyOrder(ImmutableUniqueList<Symbol> columns, Set<Row> actual, Set<Row> expected) + { + var unexpected = Sets.difference(actual, expected); + var missing = Sets.difference(expected, actual); + StringBuilder sb = null; + if (!unexpected.isEmpty()) + { + sb = new StringBuilder(); + sb.append("Unexpected rows found:\n").append(table(columns, unexpected)); + } + + if (!missing.isEmpty()) + { + if (sb == null) + { + sb = new StringBuilder(); + } + else + { + sb.append('\n'); + } + if (actual.isEmpty()) sb.append("No rows returned"); + else sb.append("Missing rows:\n").append(table(columns, missing)); + } + if (sb != null) + { + sb.append("\nExpected:\n").append(table(columns, expected)); + throw new AssertionError(sb.toString()); + } + } + + private static void validateOrder(ImmutableUniqueList<Symbol> columns, ByteBuffer[][] actual, ByteBuffer[][] expected) + { + StringBuilder sb = null; + for (int i = 0, size = Math.min(actual.length, expected.length); i < size; i++) + { + ByteBuffer[] as = actual[i]; + ByteBuffer[] es = expected[i]; + if (as.length != es.length) + { + if (sb == null) + sb = new StringBuilder(); + sb.append("\nExpected number of columns does not match"); + } + for (int c = 0, cs = Math.min(as.length, es.length); c < cs; c++) + { + ByteBuffer a = as[c]; + ByteBuffer e = es[c]; + if (!Objects.equals(a, e)) + { + Symbol symbol = columns.get(c); + if (sb == null) + sb = new StringBuilder(); + sb.append(String.format("\nIncorrect value for row %d column %s: expected %s but was %s", i, symbol, + e == null ? "null" : symbol.type().asCQL3Type().toCQLLiteral(e), + a == null ? "null" : symbol.type().asCQL3Type().toCQLLiteral(a))); + } + } + } + + if (sb != null) + { + sb.append("\nExpected:\n").append(table(columns, expected)); + sb.append("\nActual:\n").append(table(columns, actual)); + throw new AssertionError(sb.toString()); + } + } + + private static String table(ImmutableUniqueList<Symbol> columns, Collection<Row> rows) + { + return TableBuilder.toStringPiped(columns.stream().map(Symbol::toCQL).collect(Collectors.toList()), + // intellij or junit can be tripped up by utf control or invisible chars, so this logic tries to normalize to make things more safe + () -> rows.stream() + .map(r -> r.asCQL().stream().map(StringUtils::escapeControlChars).collect(Collectors.toList())) + .iterator()); + } + + private static String table(ImmutableUniqueList<Symbol> columns, ByteBuffer[][] rows) + { + return TableBuilder.toStringPiped(columns.stream().map(Symbol::toCQL).collect(Collectors.toList()), + () -> Stream.of(rows).map(row -> asCQL(columns, row)).iterator()); + } + + private static Set<Row> toRow(ImmutableUniqueList<Symbol> columns, ByteBuffer[][] rows) + { + Set<Row> set = new HashSet<>(); + for (ByteBuffer[] row : rows) + set.add(new Row(columns, row)); + return set; + } + + private static class SelectResult + { + private final ByteBuffer[][] rows; + private final boolean unordered; + + private SelectResult(ByteBuffer[][] rows, boolean unordered) + { + this.rows = rows; + this.unordered = unordered; + } + } + + private SelectResult getRowsAsByteBuffer(Select select) + { + if (select.where.isEmpty()) + return all(); + LookupContext ctx = context(select); + List<PrimaryKey> primaryKeys; + if (ctx.unmatchable) + { + primaryKeys = Collections.emptyList(); + } + else if (ctx.eq.keySet().containsAll(factory.partitionColumns)) + { + // tested + primaryKeys = findByPartitionEq(ctx); + } + else if (ctx.token != null) + { + // tested + primaryKeys = findKeysByToken(ctx); + } + else if (ctx.tokenLowerBound != null || ctx.tokenUpperBound != null) + { + primaryKeys = findKeysByTokenSearch(ctx); + } + else + { + // partial tested (handles many columns, tests are single column) + primaryKeys = search(ctx); + } + //TODO (correctness): now that we have the rows we need to handle the selections/aggregation/limit/group-by/etc. + return new SelectResult(getRowsAsByteBuffer(primaryKeys), ctx.unordered); + } + + private SelectResult all() + { + List<PrimaryKey> primaryKeys = new ArrayList<>(); + for (var partition : partitions.values()) + { + if (partition.staticOnly()) primaryKeys.add(partition.partitionRowRef()); + else partition.rows().stream().map(BytesPartitionState.Row::ref).forEach(primaryKeys::add); + } + return new SelectResult(getRowsAsByteBuffer(primaryKeys), false); + } + + public ByteBuffer[][] getRowsAsByteBuffer(List<PrimaryKey> primaryKeys) + { + ByteBuffer[][] rows = new ByteBuffer[primaryKeys.size()][]; + int idx = 0; + for (PrimaryKey pk : primaryKeys) + { + BytesPartitionState partition = partitions.get(pk.partition); + BytesPartitionState.Row row = partition.get(pk.clustering); + rows[idx++] = getRowAsByteBuffer(partition, row); + } + return rows; + } + + private ByteBuffer[] getRowAsByteBuffer(BytesPartitionState partition, @Nullable BytesPartitionState.Row row) + { + Clustering<ByteBuffer> pd = partition.key; + BytesPartitionState.Row staticRow = partition.staticRow(); + ByteBuffer[] bbs = new ByteBuffer[factory.selectionOrder.size()]; + for (Symbol col : factory.partitionColumns) + bbs[factory.selectionOrder.indexOf(col)] = pd.bufferAt(factory.partitionColumns.indexOf(col)); + for (Symbol col : factory.staticColumns) + bbs[factory.selectionOrder.indexOf(col)] = staticRow.get(col); + if (row != null) + { + for (Symbol col : factory.clusteringColumns) + bbs[factory.selectionOrder.indexOf(col)] = row.clustering.bufferAt(factory.clusteringColumns.indexOf(col)); + for (Symbol col : factory.regularColumns) + bbs[factory.selectionOrder.indexOf(col)] = row.get(col); + } + return bbs; + } + + private LookupContext context(Select select) + { + if (select.where.isEmpty()) + throw new IllegalArgumentException("Select without a where clause was expected to be handled before this point"); + return new LookupContext(select); + } + + private List<PrimaryKey> search(LookupContext ctx) + { + List<PrimaryKey> matches = new ArrayList<>(); + for (BytesPartitionState partition : partitions.values()) + { + if (!ctx.include(partition)) continue; + matches.addAll(filter(ctx, partition)); + } + return matches; + } + + private static boolean matches(AbstractType<?> type, ByteBuffer value, List<ColumnCondition> conditions) + { + for (ColumnCondition c : conditions) + { + int rc = type.compare(value, c.value); + switch (c.inequality) + { + case LESS_THAN: + if (rc >= 0) return false; + break; + case LESS_THAN_EQ: + if (rc > 0) return false; + break; + case GREATER_THAN: + if (rc <= 0) return false; + break; + case GREATER_THAN_EQ: + if (rc < 0) return false; + break; + default: + throw new UnsupportedOperationException(c.inequality.name()); + } + } + return true; + } + + /** + * The common case there can only be 1 value, but in the case of {@link Conditional.In} this can be multiple. When + * multiple values are found then the semantic is OR rather than AND like the other matches function {@link #matches(AbstractType, ByteBuffer, List)} + */ + private static boolean matches(ByteBuffer value, List<? extends Expression> conditions) + { + for (Expression e : conditions) + { + ByteBuffer expected = eval(e); + if (expected.equals(value)) + return true; + } + return false; + } + + private List<PrimaryKey> findKeysByToken(LookupContext ctx) + { + return filter(ctx, getByToken(ctx.token)); + } + + private List<PrimaryKey> findKeysByTokenSearch(LookupContext ctx) + { + return filter(ctx, getByTokenSearch(ctx.tokenLowerBound, ctx.tokenUpperBound)); + } + + private List<BytesPartitionState> getByTokenSearch(@Nullable TokenCondition tokenLowerBound, + @Nullable TokenCondition tokenUpperBound) + { + if (tokenLowerBound == null && tokenUpperBound == null) + throw new IllegalArgumentException("At least one bound must be defined..."); + NavigableSet<BytesPartitionState.Ref> keys = partitions.navigableKeySet(); + // To support the case where 2+ keys share the same token, need to create a token ref before and after the token, to make sure + // the head/tail sets find the matches correctly + if (tokenLowerBound != null) + { + boolean inclusive; + switch (tokenLowerBound.inequality) + { + case GREATER_THAN: + inclusive = false; + break; + case GREATER_THAN_EQ: + inclusive = true; + break; + default: + throw new UnsupportedOperationException(tokenLowerBound.inequality.name()); + } + // when inclusive=true the ref should be before the token, that way the tokens match + // when inclusive=false the ref should be after the token, that way they are excluded + keys = keys.tailSet(factory.createRef(tokenLowerBound.token, !inclusive), inclusive); + } + if (tokenUpperBound != null) + { + boolean inclusive; + switch (tokenUpperBound.inequality) + { + case LESS_THAN: + inclusive = false; + break; + case LESS_THAN_EQ: + inclusive = true; + break; + default: + throw new UnsupportedOperationException(tokenUpperBound.inequality.name()); + } + // when inclusive=true the ref should be after the token + // when inclusive=false the ref should be before the token + keys = keys.headSet(factory.createRef(tokenUpperBound.token, inclusive), false); + } + if (keys.isEmpty()) return Collections.emptyList(); + return keys.stream().map(partitions::get).collect(Collectors.toList()); + } + + private List<PrimaryKey> filter(LookupContext ctx, List<BytesPartitionState> partitions) + { + if (partitions.isEmpty()) return Collections.emptyList(); + List<PrimaryKey> matches = new ArrayList<>(); + for (BytesPartitionState p : partitions) + { + if (!ctx.include(p)) continue; + matches.addAll(filter(ctx, p)); + } + return matches; + } + + private List<PrimaryKey> filter(LookupContext ctx, BytesPartitionState partition) + { + Map<Symbol, List<? extends Expression>> values = ctx.eq; + List<PrimaryKey> rows = new ArrayList<>(partition.size()); + if (!factory.clusteringColumns.isEmpty() && values.keySet().containsAll(factory.clusteringColumns)) + { + // single row + for (Clustering<ByteBuffer> cd : keys(values, factory.clusteringColumns)) + { + BytesPartitionState.Row row = partition.get(cd); + if (row != null && ctx.include(row)) + rows.add(row.ref()); + } + } + else + { + // full partition + if (partition.isEmpty()) + { + if (ctx.testsRow()) + return Collections.emptyList(); + rows.add(partition.partitionRowRef()); + } + else + { + for (BytesPartitionState.Row row : partition.rows()) + { + if (ctx.include(row)) + rows.add(row.ref()); + } + } + } + return rows; + } + + private List<PrimaryKey> findByPartitionEq(LookupContext ctx) + { + List<PrimaryKey> matches = new ArrayList<>(); + for (Clustering<ByteBuffer> pd : keys(ctx.eq, factory.partitionColumns)) + { + BytesPartitionState partition = partitions.get(factory.createRef(pd)); + if (partition == null || !ctx.include(partition)) continue; + matches.addAll(filter(ctx, partition)); + } + return matches; + } + + private Clustering<ByteBuffer> key(Map<Symbol, Expression> values, ImmutableUniqueList<Symbol> columns) + { + // same as keys, but only one possible value can happen + List<Clustering<ByteBuffer>> keys = keys(Maps.transformValues(values, Collections::singletonList), columns); + Preconditions.checkState(keys.size() == 1, "Expected 1 key, but found %d", keys.size()); + return keys.get(0); + } + + private List<Clustering<ByteBuffer>> keys(Map<Symbol, List<? extends Expression>> values, ImmutableUniqueList<Symbol> columns) + { + if (columns.isEmpty()) return Collections.singletonList(Clustering.EMPTY); + List<ByteBuffer[]> current = new ArrayList<>(); + current.add(new ByteBuffer[columns.size()]); + for (Symbol symbol : columns) + { + int position = columns.indexOf(symbol); + List<? extends Expression> expressions = values.get(symbol); + ByteBuffer firstBB = eval(expressions.get(0)); + current.forEach(bbs -> bbs[position] = firstBB); + if (expressions.size() > 1) + { + // this has a multiplying effect... if there is 1 row and there are 2 expressions, then we have 2 rows + // if there are 2 rows and 2 expressions, we have 4 rows... and so on... + List<ByteBuffer[]> copy = new ArrayList<>(current); + for (int i = 1; i < expressions.size(); i++) + { + ByteBuffer bb = eval(expressions.get(i)); + for (ByteBuffer[] bbs : copy) + { + bbs = bbs.clone(); + bbs[position] = bb; + current.add(bbs); + } + } + } + } + return current.stream().map(BufferClustering::new).collect(Collectors.toList()); + } + + private static ByteBuffer eval(Expression e) + { + return ExpressionEvaluator.tryEvalEncoded(e).get(); + } + + private static class Row + { + private final ImmutableUniqueList<Symbol> columns; + private final ByteBuffer[] values; + + private Row(ImmutableUniqueList<Symbol> columns, ByteBuffer[] values) + { + this.columns = columns; + this.values = values; + } + + public String asCQL(Symbol symbol) + { + int offset = columns.indexOf(symbol); + assert offset >= 0; + ByteBuffer b = values[offset]; + return (b == null || ByteBufferUtil.EMPTY_BYTE_BUFFER.equals(b)) ? "null" : symbol.type().asCQL3Type().toCQLLiteral(b); + } + + public List<String> asCQL() + { + List<String> human = new ArrayList<>(values.length); + for (int i = 0; i < values.length; i++) + human.add(asCQL(columns.get(i))); + return human; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Row row = (Row) o; + return Arrays.equals(values, row.values); + } + + @Override + public int hashCode() + { + return Arrays.hashCode(values); + } + + @Override + public String toString() + { + return asCQL().toString(); + } + } + + private class LookupContext + { + private final Map<Symbol, List<? extends Expression>> eq = new HashMap<>(); + private final Map<Symbol, List<ColumnCondition>> ltOrGt = new HashMap<>(); + @Nullable + private Token token = null; + @Nullable + private TokenCondition tokenLowerBound, tokenUpperBound; + private boolean unordered = false; + private boolean unmatchable = false; + + private LookupContext(Select select) + { + addConditional(select.where.get()); + maybeNormalizeTokenBounds(); + } + + private void maybeNormalizeTokenBounds() + { + if (tokenLowerBound != null && tokenUpperBound != null) + { + int rc = tokenLowerBound.token.compareTo(tokenUpperBound.token); + if (rc > 0) + { + // where token > 10 and < 0.... nothing matches that! + unmatchable = true; + tokenLowerBound = null; + tokenUpperBound = null; + } + else if (rc == 0) + { + // tokens match... but is _EQ allowed for both cases? + if (!(tokenLowerBound.inequality == Inequality.GREATER_THAN_EQ + && tokenUpperBound.inequality == Inequality.LESS_THAN_EQ)) + { + // token < 42 and >= 42... nothing matches that! + unmatchable = true; + tokenLowerBound = null; + tokenUpperBound = null; + } + } + } + } + + private void addConditional(Conditional conditional) + { + if (conditional instanceof Conditional.Where) + { + Conditional.Where w = (Conditional.Where) conditional; + if (w.kind == Inequality.NOT_EQUAL) + throw new UnsupportedOperationException("!= is currently not supported"); + if (w.lhs instanceof Symbol) + { + Symbol col = (Symbol) w.lhs; + switch (w.kind) + { + case EQUAL: + var override = eq.put(col, Collections.singletonList(w.rhs)); + if (override != null) + throw new IllegalStateException("Column " + col.detailedName() + " had 2 '=' statements..."); + break; + case LESS_THAN: + case LESS_THAN_EQ: + case GREATER_THAN: + case GREATER_THAN_EQ: + ltOrGt.computeIfAbsent(col, i -> new ArrayList<>()).add(new ColumnCondition(w.kind, eval(w.rhs))); + break; + //TODO (coverage): NOT_EQUAL + default: + throw new UnsupportedOperationException(w.kind.name()); + } + } + else if (w.lhs instanceof FunctionCall) + { + FunctionCall fn = (FunctionCall) w.lhs; + switch (fn.name()) + { + case "token": + FunctionCall rhs = (FunctionCall) w.rhs; + List<ByteBuffer> pkValues = rhs.arguments.stream().map(ASTSingleTableModel::eval).collect(Collectors.toList()); + BytesPartitionState.Ref ref = factory.createRef(new BufferClustering(pkValues.toArray(ByteBuffer[]::new))); + switch (w.kind) + { + case EQUAL: + token = ref.token; + break; + case LESS_THAN: + case LESS_THAN_EQ: + if (tokenUpperBound == null) + { + tokenUpperBound = new TokenCondition(w.kind, ref.token); + } + else if (tokenUpperBound.token.equals(ref.token)) + { + // 2 cases + // a < ? AND a < ? - nothing to see here + // a < ? AND a <= ? - in this case we need the most restrictive option of < + if (tokenUpperBound.inequality != w.kind) + tokenUpperBound = new TokenCondition(Inequality.LESS_THAN, ref.token); + } + else + { + // given this is < semantic, the smallest token wins + if (ref.token.compareTo(tokenUpperBound.token) < 0) + tokenUpperBound = new TokenCondition(w.kind, ref.token); + } + break; + case GREATER_THAN: + case GREATER_THAN_EQ: + if (tokenLowerBound == null) + { + tokenLowerBound = new TokenCondition(w.kind, ref.token); + } + else if (tokenLowerBound.token.equals(ref.token)) + { + // 2 cases + // a > ? AND a > ? - nothing to see here + // a > ? AND a >= ? - in this case we need the most restrictive option of > + if (tokenLowerBound.inequality != w.kind) + tokenLowerBound = new TokenCondition(Inequality.GREATER_THAN, ref.token); + } + else + { + // given this is > semantic, the latest token wins + if (ref.token.compareTo(tokenLowerBound.token) > 0) + tokenLowerBound = new TokenCondition(w.kind, ref.token); + } + break; + default: + throw new UnsupportedOperationException(w.kind.name()); + } + break; + default: + throw new UnsupportedOperationException(fn.toCQL()); + } + } + else + { + throw new UnsupportedOperationException(w.lhs.getClass().getCanonicalName()); + } + } + else if (conditional instanceof Conditional.In) + { + Conditional.In in = (Conditional.In) conditional; + if (in.ref instanceof Symbol) + { + Symbol col = (Symbol) in.ref; + var override = eq.put(col, in.expressions); + if (override != null) + throw new IllegalStateException("Column " + col.detailedName() + " had 2 '=' statements..."); + //TODO (correctness): can't find any documentation saying clustering is ordered by the data... it "could" but is it garanateed? + if (factory.partitionColumns.contains(col) || factory.clusteringColumns.contains(col)) + unordered = true; + } + else + { + throw new UnsupportedOperationException(in.ref.getClass().getCanonicalName()); + } + } + else if (conditional instanceof Conditional.Between) + { + Conditional.Between between = (Conditional.Between) conditional; + if (between.ref instanceof Symbol) + { + Symbol col = (Symbol) between.ref; + List<ColumnCondition> list = ltOrGt.computeIfAbsent(col, i -> new ArrayList<>()); + list.add(new ColumnCondition(Inequality.GREATER_THAN_EQ, eval(between.start))); + list.add(new ColumnCondition(Inequality.LESS_THAN_EQ, eval(between.end))); + } + else if (between.ref instanceof FunctionCall) + { + FunctionCall fn = (FunctionCall) between.ref; + switch (fn.name()) + { + case "token": + // if the ref is a token, the only valid start/end are also token + List<ByteBuffer> start = ((FunctionCall) between.start).arguments.stream().map(ASTSingleTableModel::eval).collect(Collectors.toList()); + Token startToken = factory.createRef(new BufferClustering(start.toArray(ByteBuffer[]::new))).token; + + List<ByteBuffer> end = ((FunctionCall) between.end).arguments.stream().map(ASTSingleTableModel::eval).collect(Collectors.toList()); + Token endToken = factory.createRef(new BufferClustering(end.toArray(ByteBuffer[]::new))).token; + + if (startToken.equals(endToken)) + { + token = startToken; + } + else if (startToken.compareTo(endToken) > 0) + { + // start is larger than end... no matches + unmatchable = true; + } + else + { + tokenLowerBound = new TokenCondition(Inequality.GREATER_THAN_EQ, startToken); + tokenUpperBound = new TokenCondition(Inequality.LESS_THAN_EQ, endToken); + } + break; + default: + throw new UnsupportedOperationException(fn.toCQL()); + } + } + else + { + throw new UnsupportedOperationException(between.ref.getClass().getCanonicalName()); + } + } + else if (conditional instanceof Conditional.And) + { + Conditional.And and = (Conditional.And) conditional; + addConditional(and.left); + addConditional(and.right); + } + else + { + //TODO (coverage): IS + throw new UnsupportedOperationException(conditional.getClass().getCanonicalName()); + } + } + + boolean include(BytesPartitionState partition) + { + if (unmatchable) return false; + // did we include a bad partition? + if (partition.shouldDelete()) return false; + if (!include(factory.partitionColumns, partition.key::bufferAt)) + return false; + if (!factory.staticColumns.isEmpty() + && !include(factory.staticColumns, partition.staticRow()::get)) + return false; + return true; + } + + boolean include(BytesPartitionState.Row row) + { + if (unmatchable) return false; + if (!factory.clusteringColumns.isEmpty() + && !include(factory.clusteringColumns, row.clustering::bufferAt)) + return false; + if (!factory.regularColumns.isEmpty() + && !include(factory.regularColumns, row::get)) + return false; + return true; + } + + private boolean include(ImmutableUniqueList<Symbol> columns, IntFunction<ByteBuffer> accessor) + { + for (Symbol col : columns) + { + if (eq.containsKey(col)) + { + ByteBuffer actual = accessor.apply(columns.indexOf(col)); + if (actual == null) + return false; + if (!matches(actual, eq.get(col))) + return false; + } + if (ltOrGt.containsKey(col)) + { + ByteBuffer actual = accessor.apply(columns.indexOf(col)); + if (actual == null) + return false; + if (!matches(col.type(), actual, ltOrGt.get(col))) + return false; + } + } + return true; + } + + private boolean testsClustering() + { + return factory.clusteringColumns.stream().anyMatch(eq::containsKey) + || factory.clusteringColumns.stream().anyMatch(ltOrGt::containsKey); + } + + private boolean testsRegular() + { + return factory.regularColumns.stream().anyMatch(eq::containsKey) + || factory.regularColumns.stream().anyMatch(ltOrGt::containsKey); + } + + private boolean testsRow() + { + return testsClustering() || testsRegular(); + } + } + + private static class ColumnCondition + { + private final Inequality inequality; + private final ByteBuffer value; + + private ColumnCondition(Inequality inequality, ByteBuffer value) + { + this.inequality = inequality; + this.value = value; + } + } + + private static class TokenCondition + { + private final Inequality inequality; + private final Token token; + + private TokenCondition(Inequality inequality, Token token) + { + this.inequality = inequality; + this.token = token; + } + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModelTest.java b/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModelTest.java new file mode 100644 index 000000000000..af0aa22de904 --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModelTest.java @@ -0,0 +1,788 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.model; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.EnumSet; +import java.util.List; +import java.util.TreeMap; +import java.util.function.BooleanSupplier; +import java.util.function.Consumer; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import com.google.common.collect.ImmutableList; +import org.junit.Test; + +import org.apache.cassandra.cql3.ast.Bind; +import org.apache.cassandra.cql3.ast.Conditional.Where.Inequality; +import org.apache.cassandra.cql3.ast.FunctionCall; +import org.apache.cassandra.cql3.ast.Mutation; +import org.apache.cassandra.cql3.ast.Select; +import org.apache.cassandra.cql3.ast.Symbol; +import org.apache.cassandra.db.BufferClustering; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.AsciiType; +import org.apache.cassandra.db.marshal.BooleanType; +import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.marshal.InetAddressType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.LexicalUUIDType; +import org.apache.cassandra.db.marshal.ReversedType; +import org.apache.cassandra.db.marshal.ShortType; +import org.apache.cassandra.db.marshal.TimestampType; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.ByteBufferUtil; + +public class ASTSingleTableModelTest +{ + public static final ByteBuffer ZERO = ByteBufferUtil.bytes(0); + public static final ByteBuffer ONE = ByteBufferUtil.bytes(1); + public static final ByteBuffer TWO = ByteBufferUtil.bytes(2); + public static final ByteBuffer THREE = ByteBufferUtil.bytes(3); + public static final ByteBuffer[][] EMPTY = new ByteBuffer[0][]; + + private static final EnumSet<Inequality> RANGE_INEQUALITY = EnumSet.of(Inequality.LESS_THAN, Inequality.LESS_THAN_EQ, + Inequality.GREATER_THAN, Inequality.GREATER_THAN_EQ); + + @Test + public void singlePartition() + { + for (TableMetadata metadata : defaultTables()) + { + ASTSingleTableModel model = new ASTSingleTableModel(metadata); + ByteBuffer[][] expected = new ByteBuffer[][]{ + insert(model, ZERO), + insert(model, (kind, offset) -> kind == ColumnMetadata.Kind.CLUSTERING ? ONE : ZERO) + }; + // insert partition that shouldn't be returned + insert(model, ONE); + + Select.Builder builder = Select.builder().table(metadata); + for (var pk : metadata.partitionKeyColumns()) + builder.value(new Symbol(pk), ZERO); + Select select = builder.build(); + model.validate(expected, select); + } + } + + @Test + public void singleToken() + { + for (TableMetadata metadata : defaultTables()) + { + ASTSingleTableModel model = new ASTSingleTableModel(metadata); + ByteBuffer[][] expected = new ByteBuffer[][]{ + insert(model, ZERO), + insert(model, (kind, offset) -> kind == ColumnMetadata.Kind.CLUSTERING ? ONE : ZERO) + }; + // insert partition that shouldn't be returned + insert(model, ONE); + + Select.Builder builder = Select.builder().table(metadata); + builder.where(FunctionCall.tokenByColumns(model.factory.partitionColumns), + Inequality.EQUAL, + FunctionCall.tokenByValue(model.factory.partitionColumns.stream().map(i -> new Bind(ZERO, Int32Type.instance)).collect(Collectors.toList()))); + + Select select = builder.build(); + model.validate(expected, select); + } + } + + @Test + public void tokenSearch() + { + for (TableMetadata metadata : defaultTables()) + { + ASTSingleTableModel model = new ASTSingleTableModel(metadata); + ModelModel modelModel = new ModelModel(model); + boolean hasClustering = !model.factory.clusteringColumns.isEmpty(); + List<ByteBuffer> partitionValues = Arrays.asList(ONE, TWO, THREE); + + for (ByteBuffer value : partitionValues) + { + if (hasClustering) + modelModel.add(insert(model, (kind, offset) -> kind == ColumnMetadata.Kind.CLUSTERING ? ZERO : value)); + modelModel.add(insert(model, value)); + } + + FunctionCall tokenByColumns = FunctionCall.tokenByColumns(model.factory.partitionColumns); + + // unbound range: < / > + for (BytesPartitionState.Ref ref : modelModel.refs()) + { + FunctionCall tokenByValue = tokenFunction(ref); + for (Inequality inequality : RANGE_INEQUALITY) + { + model.validate(modelModel.allWhere(inequality, ref.token), + Select.builder(metadata) + .where(tokenByColumns, inequality, tokenByValue) + .build()); + } + } + // bound range: < and >, > and < + for (BytesPartitionState.Ref leftValue : modelModel.refs()) + { + FunctionCall leftTokenFunction = tokenFunction(leftValue); + for (Inequality left : RANGE_INEQUALITY) + { + for (BytesPartitionState.Ref rightValue : modelModel.refs()) + { + FunctionCall rightTokenFunction = tokenFunction(rightValue); + for (Inequality right : RANGE_INEQUALITY) + { + model.validate(modelModel.allWhere(left, leftValue.token, + right, rightValue.token), + Select.builder(metadata) + .where(tokenByColumns, left, leftTokenFunction) + .where(tokenByColumns, right, rightTokenFunction) + .build()); + } + } + } + } + // between (same as bound range, but different syntax) + for (BytesPartitionState.Ref left : modelModel.refs()) + { + FunctionCall leftTokenFunction = tokenFunction(left); + for (BytesPartitionState.Ref right : modelModel.refs()) + { + FunctionCall rightTokenFunction = tokenFunction(right); + model.validate(modelModel.allWhere(Inequality.GREATER_THAN_EQ, left.token, + Inequality.LESS_THAN_EQ, right.token), + Select.builder(metadata) + .between(tokenByColumns, leftTokenFunction, rightTokenFunction) + .build()); + } + } + } + } + + @Test + public void singleRow() + { + for (TableMetadata metadata : defaultTables()) + { + ASTSingleTableModel model = new ASTSingleTableModel(metadata); + ByteBuffer[] expectedRow = insert(model, ZERO); + // insert row that shouldn't be returned + insert(model, (kind, offset) -> kind == ColumnMetadata.Kind.CLUSTERING ? ONE : ZERO); + // insert partition that shouldn't be returned + insert(model, ONE); + + Select.Builder builder = Select.builder().table(metadata); + for (var col : metadata.primaryKeyColumns()) + builder.value(new Symbol(col), ZERO); + Select select = builder.build(); + model.validate(new ByteBuffer[][] {expectedRow}, select); + } + } + + @Test + public void eqNoMatches() + { + for (TableMetadata metadata : defaultTables()) + { + // this test only works when there are regular/static columns + if (metadata.regularAndStaticColumns().isEmpty()) continue; + ASTSingleTableModel model = new ASTSingleTableModel(metadata); + insert(model, ZERO); + // insert row that shouldn't be returned + insert(model, (kind, offset) -> kind == ColumnMetadata.Kind.CLUSTERING ? ONE : ZERO); + // insert partition that shouldn't be returned + insert(model, ONE); + + for (boolean includeClustering : Arrays.asList(true, false)) + { + Select.Builder builder = Select.builder().table(metadata); + for (var col : metadata.partitionKeyColumns()) + builder.value(new Symbol(col), ZERO); + if (includeClustering) + { + for (var col : metadata.clusteringColumns()) + builder.value(new Symbol(col), ONE); + } + for (var col : metadata.staticColumns()) + builder.value(new Symbol(col), TWO); + for (var col : metadata.regularColumns()) + builder.value(new Symbol(col), THREE); + Select select = builder.build(); + model.validate(EMPTY, select); + } + } + } + + @Test + public void selectWhereIn() + { + TableMetadata metadata = new Builder().pk(1).build(); + ASTSingleTableModel model = new ASTSingleTableModel(metadata); + // insert in token order to keep the logic simple + ByteBuffer[][] expected = { insert(model, ONE), + insert(model, TWO), + insert(model, ZERO) }; + + model.validate(expected, Select.builder() + .table(metadata) + .in("pk", 0, 1, 2) + .build()); + } + + @Test + public void nullColumnSelect() + { + // This example was found from a test, hence why more complex types are used. + // This test didn't end up depending on these complexities as the issue was null (delete or undefined column) + // handle, which is type agnostic. + TableMetadata metadata = defaultTable() + .addPartitionKeyColumn("pk0", InetAddressType.instance) + .addClusteringColumn("ck0", ReversedType.getInstance(ShortType.instance)) + .addRegularColumn("v0", TimestampType.instance) + .addRegularColumn("v1", LexicalUUIDType.instance) + .build(); + ASTSingleTableModel model = new ASTSingleTableModel(metadata); + + String pk0 = "'e44b:bdaf:aeb:f68b:1cff:ecbd:8b54:2295'"; + ByteBuffer pk0BB = InetAddressType.instance.asCQL3Type().fromCQLLiteral(pk0); + + Short row1 = Short.valueOf((short) -14407); + ByteBuffer row1BB = ShortType.instance.decompose(row1); + String row1V1 = "0x00000000000049008a00000000000000"; + ByteBuffer row1V1BB = LexicalUUIDType.instance.asCQL3Type().fromCQLLiteral(row1V1); + + Short row2 = Short.valueOf((short) ((short) 18175 - (short) 23847)); + ByteBuffer row2BB = ShortType.instance.decompose(row2); + String row2V0 = "'1989-01-11T15:00:30.950Z'"; + ByteBuffer row2V0BB = TimestampType.instance.asCQL3Type().fromCQLLiteral(row2V0); + String row2V1 = "0x0000000000001f00a700000000000000"; + ByteBuffer row2V1BB = LexicalUUIDType.instance.asCQL3Type().fromCQLLiteral(row2V1); + + Select selectPk = Select.builder(metadata) + .value("pk0", pk0) + .build(); + + Select selectColumn = Select.builder(metadata) + .value("pk0", pk0) + .where("v0", Inequality.GREATER_THAN, row2V0) + .build(); + + model.update(Mutation.update(metadata) + .set("v1", row1V1) + .value("pk0", pk0) + .value("ck0", row1) + .build()); + + model.validate(new ByteBuffer[][]{ new ByteBuffer[]{ pk0BB, row1BB, null, row1V1BB } }, selectPk); + model.validate(new ByteBuffer[0][], selectColumn); + + + model.update(Mutation.insert(metadata) + .value("pk0", pk0) + .value("ck0", row2) + .value("v0", row2V0) + .value("v1", row2V1) + .build()); + + model.validate(new ByteBuffer[][]{ + new ByteBuffer[]{ pk0BB, row2BB, row2V0BB, row2V1BB }, + new ByteBuffer[]{ pk0BB, row1BB, null, row1V1BB }, + }, selectPk); + + model.validate(new ByteBuffer[0][], selectColumn); + } + + @Test + public void selectStar() + { + for (TableMetadata metadata : defaultTables()) + { + ASTSingleTableModel model = new ASTSingleTableModel(metadata); + ModelModel modelModel = new ModelModel(model); + boolean hasClustering = !model.factory.clusteringColumns.isEmpty(); + for (ByteBuffer value : Arrays.asList(ONE, TWO, THREE)) + { + if (hasClustering) + modelModel.add(insert(model, (kind, offset) -> kind == ColumnMetadata.Kind.CLUSTERING ? ZERO : value)); + modelModel.add(insert(model, value)); + } + + model.validate(modelModel.all(), Select.builder(metadata).build()); + } + } + + @Test + public void simpleSearch() + { + for (TableMetadata metadata : defaultTables()) + { + ASTSingleTableModel model = new ASTSingleTableModel(metadata); + ModelModel modelModel = new ModelModel(model); + boolean hasClustering = !model.factory.clusteringColumns.isEmpty(); + List<ByteBuffer> partitionValues = Arrays.asList(ONE, TWO, THREE); + List<ByteBuffer> allValues = ImmutableList.<ByteBuffer>builder() + .add(ZERO) + .addAll(partitionValues) + .build(); + + for (ByteBuffer value : partitionValues) + { + if (hasClustering) + modelModel.add(insert(model, (kind, offset) -> kind == ColumnMetadata.Kind.CLUSTERING ? ZERO : value)); + modelModel.add(insert(model, value)); + } + + for (Symbol column : model.factory.selectionOrder) + { + // test eq + for (ByteBuffer value : allValues) + { + model.validate(modelModel.allEq(column, value), + Select.builder(metadata).value(column, value).build()); + } + // unbound range: < / > + for (ByteBuffer value : allValues) + { + for (Inequality inequality : RANGE_INEQUALITY) + { + model.validate(modelModel.allWhere(column, inequality, value), + Select.builder(metadata).where(column, inequality, value).build()); + } + } + // bound range: < and >, > and < + for (ByteBuffer leftValue : allValues) + { + for (Inequality left : RANGE_INEQUALITY) + { + for (ByteBuffer rightValue : allValues) + { + for (Inequality right : RANGE_INEQUALITY) + { + model.validate(modelModel.allWhere(column, + left, leftValue, + right, rightValue), + Select.builder(metadata) + .where(column, left, leftValue) + .where(column, right, rightValue) + .build()); + } + } + } + } + // between (same as bound range, but different syntax) + for (ByteBuffer left : allValues) + { + for (ByteBuffer right : allValues) + { + model.validate(modelModel.allWhere(column, + Inequality.GREATER_THAN_EQ, left, + Inequality.LESS_THAN_EQ, right), + Select.builder(metadata) + .between(column, new Bind(left, column.type()), new Bind(right, column.type())) + .build()); + } + } + } + } + } + + @Test + public void staticOnlyWrite() + { + TableMetadata metadata = new Builder().pk(1).ck(1).statics(1).regular(1).build(); + ASTSingleTableModel model = new ASTSingleTableModel(metadata); + model.update(Mutation.insert(metadata) + .value("pk", 0) + .value("s", 0) + .build()); + model.update(Mutation.update(metadata) + .set("s", 1) + .value("pk", 1) + .build()); + ByteBuffer[] rowZero = { ZERO, null, ZERO, null }; + ByteBuffer[] rowOne = { ONE, null, ONE, null }; + ByteBuffer[][] allExpected = { rowOne, rowZero, }; + model.validate(allExpected, Select.builder(metadata).build()); + model.validate(new ByteBuffer[][] {rowZero}, Select.builder(metadata).value("pk", 0).build()); + model.validate(new ByteBuffer[][] {rowZero}, Select.builder(metadata).value("s", 0).build()); + model.validate(new ByteBuffer[][] {rowOne}, Select.builder(metadata).value("pk", 1).build()); + model.validate(new ByteBuffer[][] {rowOne}, Select.builder(metadata).value("s", 1).build()); + } + + @Test + public void deleteRowImpactsSearch() + { + TableMetadata metadata = new Builder().pk(1).ck(1).statics(1).regular(1).build(); + ASTSingleTableModel model = new ASTSingleTableModel(metadata); + model.update(Mutation.insert(metadata) + .value("pk", 0) + .value("ck", 0) + .value("s", 0) + .value("v", 0) + .build()); + model.update(Mutation.delete(metadata) + .value("pk", 0) + .value("ck", 0) + .build()); + + model.validate(EMPTY, Select.builder(metadata) + .value("v", 0) + .build()); + + model.validate(EMPTY, Select.builder(metadata) + .value("pk", 0) + .value("v", 0) + .build()); + } + + @Test + public void tokenEqIncludesEmptyPartition() + { + // regression test; history + /* + History: + 1: INSERT INTO ks1.tbl (pk0, ck0, s0, v0, v1, v2, v3) VALUES (false, false, 'S' + '#', 0x7b, '21:54:38.042512095', -1220695853 + 487670685, 00000000-0000-1a00-b300-000000000000) -- on node1 + 10: UPDATE ks1.tbl SET s0='\u001C{c|\u001Dz' + '\u0006rO\u0007``', v0=0xfffa8e324eb60d5510, v1='05:09:16.823129832', v2=519617565, v3=00000000-0000-1e00-b100-000000000000 WHERE pk0 = true AND ck0 = true -- on node1 + 27: DELETE FROM ks1.tbl WHERE pk0 = false AND ck0 = false -- on node1 + 69: DELETE s0 FROM ks1.tbl WHERE pk0 = false -- on node1 + 72: SELECT * FROM ks1.tbl WHERE token(pk0) = token(false) -- by token, on node1, fetch size 1 + */ + TableMetadata metadata = defaultTable() + .addPartitionKeyColumn("pk", BooleanType.instance) + .addClusteringColumn("ck", BooleanType.instance) + .addStaticColumn("s", AsciiType.instance) + .addRegularColumn("v0", BytesType.instance) + .build(); + ASTSingleTableModel model = new ASTSingleTableModel(metadata); + model.update(Mutation.insert(metadata) + .value("pk", false) + .value("ck", false) + .value("s", "'first'") + .value("v0", "0x7b") + .build()); + model.update(Mutation.update(metadata) + .set("s", "'second'") + .set("v0", "0xfffa8e324eb60d5510") + .value("pk", true) + .value("ck", true) + .build()); + model.update(Mutation.delete(metadata) + .value("pk", false) + .value("ck", false) + .build()); + // when deleting static columns the check if the partition should be deleted didn't happen, and the filtering + // logic never excluded shouldDelete partitions + model.update(Mutation.delete(metadata) + .column("s") + .value("pk", false) + .build()); + + model.validate(EMPTY, Select.builder(metadata) + .where(FunctionCall.tokenByColumns(new Symbol("pk", BooleanType.instance)), + Inequality.EQUAL, + FunctionCall.tokenByValue(new Bind(false, BooleanType.instance))) + .build()); + } + + private static TableMetadata.Builder defaultTable() + { + return TableMetadata.builder("ks", "tbl") + .kind(TableMetadata.Kind.REGULAR) + .partitioner(Murmur3Partitioner.instance); + } + + private static FunctionCall tokenFunction(BytesPartitionState.Ref ref) + { + return FunctionCall.tokenByValue(Stream.of(ref.key.getBufferArray()).map(bb -> new Bind(bb, BytesType.instance)).collect(Collectors.toList())); + } + + private static ByteBuffer[] insert(ASTSingleTableModel model, ByteBuffer value) + { + return insert(model, (i1, i2) -> value); + } + + private static ByteBuffer[] insert(ASTSingleTableModel model, ColumnValue fn) + { + TableMetadata metadata = model.factory.metadata; + ByteBuffer[] expectedRow = new ByteBuffer[metadata.columns().size()]; + var builder = Mutation.insert(metadata); + int offset = 0; + int idx = 0; + for (var col : metadata.partitionKeyColumns()) + { + ByteBuffer value = fn.accept(ColumnMetadata.Kind.PARTITION_KEY, idx++); + builder.value(new Symbol(col), value); + expectedRow[offset++] = value; + } + idx = 0; + for (var col : metadata.clusteringColumns()) + { + ByteBuffer value = fn.accept(ColumnMetadata.Kind.CLUSTERING, idx++); + builder.value(new Symbol(col), value); + expectedRow[offset++] = value; + } + idx = 0; + for (var col : metadata.staticColumns()) + { + ByteBuffer value = fn.accept(ColumnMetadata.Kind.STATIC, idx++); + builder.value(new Symbol(col), value); + expectedRow[offset++] = value; + } + idx = 0; + for (var col : metadata.regularColumns()) + { + ByteBuffer value = fn.accept(ColumnMetadata.Kind.REGULAR, idx++); + builder.value(new Symbol(col), value); + expectedRow[offset++] = value; + } + model.update(builder.build()); + return expectedRow; + } + + private static List<TableMetadata> defaultTables() + { + List<TableMetadata> tables = new ArrayList<>(); + for (int pk : Arrays.asList(1, 2)) + { + for (int ck : Arrays.asList(0, 1, 2)) + { + for (int statics : Arrays.asList(0, 1, 2)) + { + for (int regular : Arrays.asList(0, 1, 2)) + { + tables.add(new Builder() + .pk(pk) + .ck(ck) + .statics(statics) + .regular(regular) + .build()); + } + } + } + } + return tables; + } + + private static class ModelModel + { + private final ASTSingleTableModel model; + private final TreeMap<BytesPartitionState.Ref, List<ByteBuffer[]>> partitions = new TreeMap<>(); + + private ModelModel(ASTSingleTableModel model) + { + this.model = model; + } + + Iterable<BytesPartitionState.Ref> refs() + { + return partitions.keySet(); + } + + ByteBuffer[] add(ByteBuffer[] row) + { + BytesPartitionState.Ref ref = createRef(row); + partitions.computeIfAbsent(ref, i -> new ArrayList<>()).add(row); + return row; + } + + private BytesPartitionState.Ref createRef(ByteBuffer[] row) + { + ByteBuffer[] pks = Arrays.copyOf(row, model.factory.partitionColumns.size()); + return model.factory.createRef(new BufferClustering(pks)); + } + + public ByteBuffer[][] all() + { + return allWhere(i -> true); + } + + public ByteBuffer[][] allEq(Symbol column, ByteBuffer value) + { + return allWhere(column, Inequality.EQUAL, value); + } + + public ByteBuffer[][] allWhere(Symbol column, Inequality inequality, ByteBuffer value) + { + int idx = model.factory.selectionOrder.indexOf(column); + return allWhere(row -> { + ByteBuffer actual = row[idx]; + if (actual == null) return false; + return include(column.type(), actual, inequality, value); + }); + } + + public ByteBuffer[][] allWhere(Symbol column, + Inequality left, ByteBuffer leftValue, + Inequality right, ByteBuffer rightValue) + { + int idx = model.factory.selectionOrder.indexOf(column); + return allWhere(row -> { + ByteBuffer actual = row[idx]; + if (actual == null) return false; + return include(column.type(), actual, left, leftValue) && + include(column.type(), actual, right, rightValue); + }); + } + + private ByteBuffer[][] allWhere(Inequality inequality, Token token) + { + return allWhere(ref -> include(ref, inequality, token), i -> true); + } + + private ByteBuffer[][] allWhere(Inequality left, Token leftToken, + Inequality right, Token rightToken) + { + return allWhere(ref -> include(ref, left, leftToken) && include(ref, right, rightToken), i -> true); + } + + private ByteBuffer[][] allWhere(Predicate<ByteBuffer[]> predicate) + { + return allWhere(i -> true, predicate); + } + + private ByteBuffer[][] allWhere(Predicate<BytesPartitionState.Ref> partitionPredicate, + Predicate<ByteBuffer[]> rowPredicate) + { + List<ByteBuffer[]> rows = new ArrayList<>(); + for (var e : partitions.entrySet()) + { + BytesPartitionState.Ref ref = e.getKey(); + if (!partitionPredicate.test(ref)) + continue; + List<ByteBuffer[]> partition = e.getValue(); + for (ByteBuffer[] row : partition) + { + if (rowPredicate.test(row)) + rows.add(row); + } + } + return rows.toArray(ByteBuffer[][]::new); + } + + private static boolean include(AbstractType<?> type, ByteBuffer actual, Inequality inequality, ByteBuffer value) + { + return include(inequality, type.compare(actual, value), () -> actual.equals(value)); + } + + private static boolean include(BytesPartitionState.Ref ref, Inequality inequality, Token token) + { + return include(inequality, ref.token.compareTo(token), () -> ref.token.equals(token)); + } + + private static boolean include(Inequality inequality, int rc, BooleanSupplier eq) + { + switch (inequality) + { + case EQUAL: + if (eq.getAsBoolean()) + return true; + break; + case NOT_EQUAL: + if (!eq.getAsBoolean()) + return true; + break; + case LESS_THAN_EQ: + if (rc == 0) + return true; + case LESS_THAN: + if (rc < 0) + return true; + break; + case GREATER_THAN_EQ: + if (rc == 0) + return true; + case GREATER_THAN: + if (rc > 0) + return true; + break; + default: + throw new UnsupportedOperationException(inequality.name()); + } + return false; + } + } + + private interface ColumnValue + { + ByteBuffer accept(ColumnMetadata.Kind kind, int offset); + } + + private static class Builder + { + private int numPk = 1; + private int numCk = 0; + private int numStatic = 0; + private int numRegular = 0; + + private Builder pk(int count) + { + numPk = count; + return this; + } + + private Builder ck(int count) + { + numCk = count; + return this; + } + + private Builder statics(int count) + { + numStatic = count; + return this; + } + + private Builder regular(int count) + { + numRegular = count; + return this; + } + + private TableMetadata build() + { + TableMetadata.Builder builder = defaultTable(); + addColumn("pk", numPk, n -> builder.addPartitionKeyColumn(n, Int32Type.instance)); + addColumn("ck", numCk, n -> builder.addClusteringColumn(n, Int32Type.instance)); + addColumn("s", numStatic, n -> builder.addStaticColumn(n, Int32Type.instance)); + addColumn("v", numRegular, n -> builder.addRegularColumn(n, Int32Type.instance)); + + return builder.build(); + } + + private static void addColumn(String prefix, int count, Consumer<String> addColumn) + { + if (count == 0) + return; + if (count == 1) + { + addColumn.accept(prefix); + } + else + { + for (int i = 0; i < count; i++) + addColumn.accept(prefix + i); + } + } + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/model/BytesPartitionState.java b/test/harry/main/org/apache/cassandra/harry/model/BytesPartitionState.java new file mode 100644 index 000000000000..6b8f61259dcb --- /dev/null +++ b/test/harry/main/org/apache/cassandra/harry/model/BytesPartitionState.java @@ -0,0 +1,592 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.harry.model; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.NavigableSet; +import java.util.Objects; +import java.util.Set; +import java.util.TreeSet; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import javax.annotation.Nullable; + +import org.apache.cassandra.cql3.ast.Symbol; +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.ClusteringComparator; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.harry.MagicConstants; +import org.apache.cassandra.harry.gen.BijectionCache; +import org.apache.cassandra.harry.gen.Bijections; +import org.apache.cassandra.harry.gen.ValueGenerators; +import org.apache.cassandra.harry.util.BitSet; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.FastByteOperations; +import org.apache.cassandra.utils.ImmutableUniqueList; + +public class BytesPartitionState +{ + private final Factory factory; + public final Clustering<ByteBuffer> key; + private final Token token; + private final PartitionState state; + + private BytesPartitionState(Factory factory, Clustering<ByteBuffer> key) + { + this.factory = factory; + this.key = key; + this.token = factory.metadata.partitioner.getToken(key.serializeAsPartitionKey()); + this.state = factory.partitionState(key); + } + + public void deleteRow(Clustering<ByteBuffer> clustering) + { + long cd = factory.clusteringCache.deflateOrUndefined(clustering); + if (MagicConstants.UNSET_DESCR == cd) + return; + state.delete(cd, MagicConstants.NO_TIMESTAMP); + } + + public void deleteColumns(Clustering<ByteBuffer> clustering, Set<Symbol> columns) + { + long cd = factory.clusteringCache.deflateOrUndefined(clustering); + if (cd != MagicConstants.UNSET_DESCR) + { + BitSet regularColumns = bitset(columns, true); + if (!regularColumns.allUnset()) + state.deleteRegularColumns(MagicConstants.NO_TIMESTAMP, cd, regularColumns); + } + deleteStaticColumns(columns); + } + + public void deleteStaticColumns(Set<Symbol> columns) + { + BitSet staticColumns = bitset(columns, false); + if (!staticColumns.allUnset()) + state.deleteStaticColumns(MagicConstants.NO_TIMESTAMP, staticColumns); + } + + private BitSet bitset(Set<Symbol> columns, boolean regular) + { + ImmutableUniqueList<Symbol> positions = regular ? factory.regularColumns : factory.staticColumns; + BitSet bitSet = new BitSet.BitSet64Bit(positions.size()); + for (int i = 0; i < positions.size(); i++) + { + Symbol column = positions.get(i); + if (columns.contains(column)) + bitSet.set(i); + } + return bitSet; + } + + public Ref ref() + { + return new Ref(factory, key, token); + } + + public PrimaryKey partitionRowRef() + { + return new PrimaryKey(ref(), null); + } + + public void setStaticColumns(Map<Symbol, ByteBuffer> values) + { + if (factory.staticColumns.isEmpty() || values.isEmpty()) + throw new IllegalStateException("Attempt to write to static columns; but they do not exist"); + long[] sds = toDescriptor(factory.staticColumns, values); + state.writeStatic(sds, MagicConstants.NO_TIMESTAMP); + } + + public void setColumns(Clustering<ByteBuffer> clustering, Map<Symbol, ByteBuffer> values, boolean writePrimaryKeyLiveness) + { + long cd = factory.clusteringCache.deflate(clustering); + long[] vds = toDescriptor(factory.regularColumns, values); + state.writeRegular(cd, vds, MagicConstants.NO_TIMESTAMP, writePrimaryKeyLiveness); + } + + private long[] toDescriptor(ImmutableUniqueList<Symbol> positions, Map<Symbol, ByteBuffer> values) + { + long[] vds = new long[positions.size()]; + for (int i = 0; i < positions.size(); i++) + { + Symbol column = positions.get(i); + if (values.containsKey(column)) + { + long vd = factory.valueCache.deflate(new Value(column.type(), values.get(column))); + vds[i] = vd; + } + else + { + vds[i] = MagicConstants.UNSET_DESCR; + } + } + return vds; + } + + private ByteBuffer[] fromDescriptor(ImmutableUniqueList<Symbol> positions, long[] values) + { + if (positions.size() != values.length) + throw new IllegalArgumentException(String.format("Attempted to extract values but expected columns didn't match; expected %s, but given %d values", positions, values.length)); + ByteBuffer[] bbs = new ByteBuffer[values.length]; + for (int i = 0; i < bbs.length; i++) + { + long vd = values[i]; + if (vd == MagicConstants.NIL_DESCR) + { + bbs[i] = null; + } + else + { + var value = factory.valueCache.inflate(vd); + Symbol column = positions.get(i); + if (!value.type.equals(column.type())) + throw new IllegalStateException(String.format("Given value descriptor %d that maps to the wrong type; expected %s, given %s", vd, column.type().asCQL3Type(), value.type.asCQL3Type())); + bbs[i] = value.value; + } + } + return bbs; + } + + public int size() + { + return state.rows().size(); + } + + public boolean isEmpty() + { + return state.rows().isEmpty(); + } + + public boolean staticOnly() + { + return isEmpty() && !factory.staticColumns.isEmpty() && !staticRow().isEmpty(); + } + + @Nullable + public Row get(Clustering<ByteBuffer> clustering) + { + long cd = factory.clusteringCache.deflateOrUndefined(clustering); + if (cd == MagicConstants.UNSET_DESCR) + return null; + PartitionState.RowState rowState = state.rows().get(cd); + if (rowState == null) + return null; + return toRow(rowState); + } + + private Row toRow(PartitionState.RowState rowState) + { + Clustering<ByteBuffer> clustering; + ByteBuffer[] values; + if (PartitionState.STATIC_CLUSTERING == rowState.cd) + { + clustering = Clustering.STATIC_CLUSTERING; + values = fromDescriptor(factory.staticColumns, rowState.vds); + } + else + { + clustering = factory.clusteringCache.inflate(rowState.cd); + values = fromDescriptor(factory.regularColumns, rowState.vds); + } + return new Row(clustering, values); + } + + public Collection<Row> rows() + { + return state.rows().values().stream().map(this::toRow).collect(Collectors.toList()); + } + + public NavigableSet<Clustering<ByteBuffer>> clusteringKeys() + { + NavigableSet<Clustering<ByteBuffer>> navigableSet = new TreeSet<>(factory.clusteringComparator); + state.rows().keySet().stream().map(factory.clusteringCache::inflate).forEach(navigableSet::add); + return navigableSet; + } + + public Row staticRow() + { + return toRow(state.staticRow()); + } + + public boolean shouldDelete() + { + return state.shouldDelete(); + } + + static List<String> asCQL(List<Symbol> columns, ByteBuffer[] row) + { + List<String> cql = new ArrayList<>(row.length); + for (int i = 0; i < row.length; i++) + cql.add(columns.get(i).type().toCQLString(row[i])); + return cql; + } + + private static void appendValues(StringBuilder sb, List<Symbol> columns, Clustering<ByteBuffer> key) + { + if (columns.isEmpty()) + { + sb.append(key == Clustering.STATIC_CLUSTERING ? "STATIC" : "EMPTY"); + return; + } + List<String> names = columns.stream().map(Symbol::toCQL).collect(Collectors.toList()); + List<String> values = asCQL(columns, key.getBufferArray()); + if (names.size() > 1) + sb.append('('); + for (int i = 0; i < names.size(); i++) + sb.append(names.get(i)).append('=').append(values.get(i)).append(", "); + sb.setLength(sb.length() - 2); // ", " = 2 chars + if (names.size() > 1) + sb.append(')'); + } + + public class PrimaryKey implements Comparable<PrimaryKey> + { + public final BytesPartitionState.Ref partition; + @Nullable + public final Clustering<ByteBuffer> clustering; + + public PrimaryKey(BytesPartitionState.Ref partition, @Nullable Clustering<ByteBuffer> clustering) + { + this.partition = partition; + this.clustering = clustering; + } + + @Override + public int compareTo(PrimaryKey o) + { + int rc = partition.compareTo(o.partition); + if (rc != 0) return rc; + if (clustering == null) return rc; // if the partition matches, and clustering is null (partition doesn't have rows) then it would be a bug if o.clustering was non-null + rc = factory.clusteringComparator.compare(clustering, o.clustering); + return rc; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + PrimaryKey that = (PrimaryKey) o; + return partition.equals(that.partition) && Objects.equals(clustering, that.clustering); + } + + @Override + public int hashCode() + { + return Objects.hash(partition, clustering); + } + + @Override + public String toString() + { + StringBuilder sb = new StringBuilder("(partition="); + sb.append(partition); + sb.append(", clustering="); + appendValues(sb, factory.clusteringColumns, clustering); + sb.append(')'); + return sb.toString(); + } + } + + public static class Ref implements Comparable<Ref> + { + private final Factory factory; + @Nullable + public final Clustering<ByteBuffer> key; + public final Token token; + // when true (null, token) > (key, token). When false (null, token) < (key, token) + private final boolean nullKeyGtMatchingToken; + + private Ref(Factory factory, Clustering<ByteBuffer> key, Token token) + { + this.factory = factory; + this.key = key; + this.token = token; + this.nullKeyGtMatchingToken = true; + } + + private Ref(Factory factory, Token token, boolean nullKeyGtMatchingToken) + { + this.factory = factory; + this.key = null; + this.token = token; + this.nullKeyGtMatchingToken = nullKeyGtMatchingToken; + } + + @Override + public int compareTo(Ref o) + { + int rc = token.compareTo(o.token); + if (rc != 0) return rc; + // when key is null, this is a token match lookup... + if (key == null) + return nullKeyGtMatchingToken ? 1 : -1; + if (o.key == null) + return nullKeyGtMatchingToken ? -1 : 1; + for (int i = 0; i < key.size(); i++) + { + ByteBuffer self = key.bufferAt(i); + ByteBuffer other = o.key.bufferAt(i); + //TODO (correctness): what is the tie breaker? + rc = FastByteOperations.compareUnsigned(self, other); + if (rc != 0) return rc; + } + return 0; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Ref ref = (Ref) o; + return Objects.equals(key, ref.key) && token.equals(ref.token); + } + + @Override + public int hashCode() + { + return Objects.hash(key, token); + } + + @Override + public String toString() + { + StringBuilder sb = new StringBuilder(); + sb.append('('); + if (key == null) + { + sb.append("null"); + } + else + { + appendValues(sb, factory.partitionColumns, key); + } + sb.append(", token=").append(token); + sb.append(')'); + return sb.toString(); + } + } + + public class Row + { + public final Clustering<ByteBuffer> clustering; + private final ImmutableUniqueList<Symbol> columnNames; + private final ByteBuffer[] columns; + + private Row(Clustering<ByteBuffer> clustering, ByteBuffer[] columns) + { + this.clustering = clustering; + this.columnNames = clustering == Clustering.STATIC_CLUSTERING ? factory.staticColumns : factory.regularColumns; + this.columns = columns; + } + + public ByteBuffer get(Symbol col) + { + return columns[columnNames.indexOf(col)]; + } + + public ByteBuffer get(int offset) + { + return columns[offset]; + } + + public PrimaryKey ref() + { + return new PrimaryKey(BytesPartitionState.this.ref(), clustering); + } + + public boolean isEmpty() + { + return Stream.of(columns).allMatch(b -> b == null ); + } + } + + public static class Factory + { + public final TableMetadata metadata; + public final ImmutableUniqueList<Symbol> partitionColumns; + public final ImmutableUniqueList<Symbol> clusteringColumns; + public final ImmutableUniqueList<Symbol> staticColumns; + public final ImmutableUniqueList<Symbol> regularColumns; + public final ImmutableUniqueList<Symbol> selectionOrder, regularAndStaticColumns; + public final ClusteringComparator clusteringComparator; + + + // translation layer for harry interop + private final BijectionCache<Clustering<ByteBuffer>> partitionCache = new BijectionCache<>(Reject.instance.as()); + private final BijectionCache<Clustering<ByteBuffer>> clusteringCache; + private final BijectionCache<Value> valueCache = new BijectionCache<>(Reject.instance.as()); + private final ValueGenerators<Clustering<ByteBuffer>, Clustering<ByteBuffer>> valueGenerators; + + public Factory(TableMetadata metadata) + { + this.metadata = metadata; + ImmutableUniqueList.Builder<Symbol> symbolListBuilder = ImmutableUniqueList.builder(); + for (ColumnMetadata pk : metadata.partitionKeyColumns()) + symbolListBuilder.add(Symbol.from(pk)); + partitionColumns = symbolListBuilder.buildAndClear(); + for (ColumnMetadata pk : metadata.clusteringColumns()) + symbolListBuilder.add(Symbol.from(pk)); + clusteringColumns = symbolListBuilder.buildAndClear(); + for (ColumnMetadata pk : metadata.staticColumns()) + symbolListBuilder.add(Symbol.from(pk)); + staticColumns = symbolListBuilder.buildAndClear(); + for (ColumnMetadata pk : metadata.regularColumns()) + symbolListBuilder.add(Symbol.from(pk)); + regularColumns = symbolListBuilder.buildAndClear(); + metadata.allColumnsInSelectOrder().forEachRemaining(cm -> symbolListBuilder.add(Symbol.from(cm))); + selectionOrder = symbolListBuilder.buildAndClear(); + metadata.regularAndStaticColumns().forEach(cm -> symbolListBuilder.add(Symbol.from(cm))); + regularAndStaticColumns = symbolListBuilder.buildAndClear(); + + clusteringComparator = new ClusteringComparator(clusteringColumns.stream().map(Symbol::rawType).collect(Collectors.toList())); + + List<Comparator<Object>> pkComparators = new ArrayList<>(partitionColumns.size()); + for (var p : partitionColumns) + pkComparators.add(compareBytes(p.type())); + List<Comparator<Object>> ckComparators = new ArrayList<>(clusteringColumns.size()); + for (var c : clusteringColumns) + ckComparators.add(compareBytes(c.rawType())); + List<Bijections.Bijection<?>> regularColumnGens = new ArrayList<>(regularColumns.size()); + List<Comparator<Object>> regularComparators = new ArrayList<>(regularColumns.size()); + for (var r : regularColumns) + { + regularColumnGens.add(valueCache); + regularComparators.add(compareValue(r.type())); + } + List<Bijections.Bijection<?>> staticColumnGens = new ArrayList<>(staticColumns.size()); + List<Comparator<Object>> staticComparators = new ArrayList<>(staticColumns.size()); + for (var s : staticColumns) + { + staticColumnGens.add(valueCache); + staticComparators.add(compareValue(s.type())); + } + + clusteringCache = new BijectionCache<>(clusteringComparator); + + ValueGenerators.Accessor<Clustering<ByteBuffer>> clusteringAccessor = (offset, clustering) -> clustering.bufferAt(offset); + valueGenerators = new ValueGenerators<>(partitionCache, clusteringCache, clusteringAccessor, + regularColumnGens, staticColumnGens, + pkComparators, ckComparators, + regularComparators, staticComparators); + } + + private Comparator<Object> compareValue(AbstractType<?> type) + { + return (a, b) -> { + Value av = (Value) a; + Value bv = (Value) b; + if (!av.type.equals(type)) + throw new IllegalArgumentException(String.format("Attempted to compare values of the wrong type; expected %s, actual %s", type.asCQL3Type(), av.type.asCQL3Type())); + if (!bv.type.equals(type)) + throw new IllegalArgumentException(String.format("Attempted to compare values of the wrong type; expected %s, actual %s", type.asCQL3Type(), bv.type.asCQL3Type())); + return type.compare(av.value, bv.value); + }; + } + + private Comparator<Object> compareBytes(AbstractType<?> type) + { + return (a, b) -> type.compare((ByteBuffer) a, (ByteBuffer) b); + } + + public BytesPartitionState create(Clustering<ByteBuffer> key) + { + return new BytesPartitionState(this, key); + } + + public BytesPartitionState.Ref createRef(Clustering<ByteBuffer> key) + { + Token token = metadata.partitioner.getToken(key.serializeAsPartitionKey()); + return new Ref(this, key, token); + } + + /** + * Define a ref where the {@link Ref#key} is {@code null}, and the ordering of this ref is that (null, token) is either before (key, token) or after; depending on {@code nullKeyGtMatchingToken} + * + * @param token for the ref + * @param nullKeyGtMatchingToken when true (null, token) > (key, token). When false (null, token) < (key, token) + */ + public BytesPartitionState.Ref createRef(Token token, boolean nullKeyGtMatchingToken) + { + return new BytesPartitionState.Ref(this, token, nullKeyGtMatchingToken); + } + + private PartitionState partitionState(Clustering<ByteBuffer> key) + { + return new PartitionState(partitionCache.deflate(key), valueGenerators); + } + } + + private static class Value + { + final AbstractType<?> type; + final ByteBuffer value; + + private Value(AbstractType<?> type, ByteBuffer value) + { + this.type = type; + this.value = value; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Value value1 = (Value) o; + return type.equals(value1.type) && value.equals(value1.value); + } + + @Override + public int hashCode() + { + return Objects.hash(type, value); + } + + @Override + public String toString() + { + return type.asCQL3Type().toCQLLiteral(value); + } + } + + private enum Reject implements Comparator<Object> + { + instance; + + <T> Comparator<T> as() + { + return (Comparator<T>) this; + } + + @Override + public int compare(Object o1, Object o2) + { + throw new UnsupportedOperationException(); + } + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/model/PartitionState.java b/test/harry/main/org/apache/cassandra/harry/model/PartitionState.java index c47f733db6f8..cf4d70bd958f 100644 --- a/test/harry/main/org/apache/cassandra/harry/model/PartitionState.java +++ b/test/harry/main/org/apache/cassandra/harry/model/PartitionState.java @@ -30,6 +30,7 @@ import java.util.*; import java.util.function.IntFunction; +import java.util.stream.LongStream; public class PartitionState implements Iterable<PartitionState.RowState> { @@ -138,12 +139,15 @@ private void filterInternal(Operations.SelectCustom select) { // TODO: inefficient; need to search for lower/higher bounds rows.entrySet().removeIf(e -> { - Map<Long, Object[]> cache = new HashMap<>(); + Map<Long, Object> cache = new HashMap<>(); for (Relations.Relation relation : select.ckRelations()) { - Object[] query = cache.computeIfAbsent(relation.descriptor, valueGenerators.ckGen()::inflate); - Object[] match = cache.computeIfAbsent(e.getValue().cd, valueGenerators.ckGen()::inflate); - if (!relation.kind.match(valueGenerators.ckComparator(relation.column), match[relation.column], query[relation.column])) + Object query = cache.computeIfAbsent(relation.descriptor, valueGenerators.ckGen()::inflate); + Object match = cache.computeIfAbsent(e.getValue().cd, valueGenerators.ckGen()::inflate); + var accessor = valueGenerators.ckAccessor(); + if (!relation.kind.match(valueGenerators.ckComparator(relation.column), + accessor.access(relation.column, match), + accessor.access(relation.column, query))) return true; // true means "no match", so remove from resultset } @@ -188,6 +192,11 @@ public boolean isEmpty() return rows.isEmpty(); } + public boolean shouldDelete() + { + return isEmpty() && staticRow.isEmpty(); + } + /** * Method used to update row state of both static and regular rows. */ @@ -223,7 +232,7 @@ private RowState updateRowState(RowState currentState, IntFunction<Bijections.Bi assert lts >= currentState.lts[i] : String.format("Out-of-order LTS: %d. Max seen: %s", lts, currentState.lts[i]); // sanity check; we're iterating in lts order - if (currentState.lts[i] == lts) + if (lts != MagicConstants.NO_TIMESTAMP && currentState.lts[i] == lts) { // Timestamp collision case Bijections.Bijection<?> column = columns.apply(i); @@ -232,8 +241,8 @@ private RowState updateRowState(RowState currentState, IntFunction<Bijections.Bi } else { + assert lts == MagicConstants.NO_TIMESTAMP || lts > currentState.lts[i]; currentState.vds[i] = vds[i]; - assert lts > currentState.lts[i]; currentState.lts[i] = lts; } } @@ -353,6 +362,11 @@ public RowState(PartitionState partitionState, this.lts = lts; } + public boolean isEmpty() + { + return LongStream.of(vds).allMatch(l -> l == MagicConstants.NIL_DESCR); + } + public RowState clone() { RowState rowState = new RowState(partitionState, cd, Arrays.copyOf(vds, vds.length), Arrays.copyOf(lts, lts.length)); diff --git a/test/harry/main/org/apache/cassandra/harry/util/StringUtils.java b/test/harry/main/org/apache/cassandra/harry/util/StringUtils.java index f6571c44784a..0b3b94b73fea 100644 --- a/test/harry/main/org/apache/cassandra/harry/util/StringUtils.java +++ b/test/harry/main/org/apache/cassandra/harry/util/StringUtils.java @@ -22,6 +22,28 @@ public class StringUtils { + /** + * When printing out UTF-8 Strings to humans there can be issues with utf-8 control chars, which leads to + * tools (such as Intellij) printing out things incorrectly. The motivating case for this was the string {@code s\u000D^vf \u001F\u000CtU}, + * this string caused Intellij to delete the content in front of the string, so the output only included the parts after this string! + * + * This method is only expected to be used when reporting to humans, and not for internal logic. + */ + public static String escapeControlChars(String input) + { + StringBuilder result = new StringBuilder(); + for (int i = 0; i < input.length(); i++) + { + char c = input.charAt(i); + if (Character.isISOControl(c)) + result.append(String.format("\\u%04X", (int) c)); + else + result.append(c); + } + + return result.toString(); + } + public static String toString(long[] arr) { if (arr.length == 0) diff --git a/test/unit/org/apache/cassandra/cql3/KnownIssue.java b/test/unit/org/apache/cassandra/cql3/KnownIssue.java new file mode 100644 index 000000000000..37ef255e5433 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/KnownIssue.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3; + +import java.util.EnumSet; + +/** + * In order to have a clean CI some known issues must be excluded from some tests until those issues are addressed. + * + * This type exists to make it easier to descover known issues and places in the code that account for them + */ +public enum KnownIssue +{ + BETWEEN_START_LARGER_THAN_END("https://issues.apache.org/jira/browse/CASSANDRA-20154", + "BETWEEN is matching values when start > end, which should never return anything"), + SAI_INET_MIXED("https://issues.apache.org/jira/browse/CASSANDRA-19492", + "SAI converts ipv4 to ipv6 to simplify the index, this causes issues with range search as it starts to mix the values, which isn't always desirable or intuative"), + CUSTOM_INDEX_MAX_COLUMN_48("https://issues.apache.org/jira/browse/CASSANDRA-19897", + "Columns can be up to 50 chars, but CREATE CUSTOM INDEX only allows up to 48"), + AF_MULTI_NODE_AND_NODE_LOCAL_WRITES("https://issues.apache.org/jira/browse/CASSANDRA-20243", + "When writes are done at NODE_LOCAL and the select is ALL, AF should be able to return the correct data but it doesn't"), + SHORT_AND_VARINT_GET_INT_FUNCTIONS("https://issues.apache.org/jira/browse/CASSANDRA-19874", + "Function inference maybe unable to infer the correct function or chooses one for a smaller type"), + SAI_EMPTY_TYPE("ML: Meaningless emptiness and filtering", + "Some types allow empty bytes, but define them as meaningless. AF can be used to query them using <, <=, and =; but SAI can not") + ; + + KnownIssue(String url, String description) + { + // don't actually care to save the values, just there to force documentation + } + + public static EnumSet<KnownIssue> ignoreAll() + { + return EnumSet.allOf(KnownIssue.class); + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ast/Conditional.java b/test/unit/org/apache/cassandra/cql3/ast/Conditional.java index 61241adc51a3..7fdcd17bea4b 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/Conditional.java +++ b/test/unit/org/apache/cassandra/cql3/ast/Conditional.java @@ -27,11 +27,14 @@ import java.util.stream.IntStream; import java.util.stream.Stream; -import accord.utils.Invariants; +import com.google.common.base.Preconditions; + +import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.BooleanType; import org.apache.cassandra.db.marshal.BytesType; import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.schema.TableMetadata; public interface Conditional extends Expression { @@ -87,6 +90,12 @@ public static Where create(Inequality kind, Expression ref, Expression expressio return new Where(kind, ref, expression); } + @Override + public String toString() + { + return toCQL(); + } + @Override public void toCQL(StringBuilder sb, CQLFormatter formatter) { @@ -160,7 +169,7 @@ class In implements Conditional public In(ReferenceExpression ref, List<? extends Expression> expressions) { - Invariants.checkArgument(!expressions.isEmpty()); + Preconditions.checkArgument(!expressions.isEmpty()); this.ref = ref; this.expressions = expressions; } @@ -304,6 +313,23 @@ default T value(String symbol, int e) } } + interface EqBuilderPlus<T extends EqBuilderPlus<T>> extends EqBuilder<T> + { + TableMetadata metadata(); + + default T value(String name, String value) + { + Symbol symbol = new Symbol(metadata().getColumn(new ColumnIdentifier(name, true))); + return value(symbol, new Bind(symbol.type().asCQL3Type().fromCQLLiteral(value), symbol.type())); + } + + default T value(String name, Object value) + { + Symbol symbol = new Symbol(metadata().getColumn(new ColumnIdentifier(name, true))); + return value(symbol, new Bind(value, symbol.type())); + } + } + interface ConditionalBuilder<T extends ConditionalBuilder<T>> extends EqBuilder<T> { @@ -383,6 +409,21 @@ default <Type> T value(String symbol, Type value, AbstractType<Type> type) } } + interface ConditionalBuilderPlus<T extends ConditionalBuilderPlus<T>> extends ConditionalBuilder<T>, EqBuilderPlus<T> + { + default T where(String name, Where.Inequality kind, String value) + { + Symbol symbol = new Symbol(metadata().getColumn(new ColumnIdentifier(name, true))); + return where(symbol, kind, new Bind(symbol.type().asCQL3Type().fromCQLLiteral(value), symbol.type())); + } + + default T where(String name, Where.Inequality kind, Object value) + { + Symbol symbol = new Symbol(metadata().getColumn(new ColumnIdentifier(name, true))); + return where(symbol, kind, new Bind(value, symbol.type())); + } + } + class Builder implements ConditionalBuilder<Builder> { private final List<Conditional> sub = new ArrayList<>(); diff --git a/test/unit/org/apache/cassandra/cql3/ast/Literal.java b/test/unit/org/apache/cassandra/cql3/ast/Literal.java index ad8e3ddc7448..3886fe91b45f 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/Literal.java +++ b/test/unit/org/apache/cassandra/cql3/ast/Literal.java @@ -21,7 +21,9 @@ import java.nio.ByteBuffer; import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.BytesType; import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.StringType; public class Literal implements Value { @@ -66,6 +68,17 @@ public Literal with(Object value, AbstractType<?> type) @Override public void toCQL(StringBuilder sb, CQLFormatter formatter) { - sb.append(type.asCQL3Type().toCQLLiteral(valueEncoded())); + ByteBuffer bytes = valueEncoded(); + if (bytes.remaining() == 0 && !actuallySupportsEmpty(type)) + { + sb.append("<empty bytes>"); + return; + } + sb.append(type.asCQL3Type().toCQLLiteral(bytes)); + } + + private static boolean actuallySupportsEmpty(AbstractType<?> type) + { + return type == BytesType.instance || type instanceof StringType; } } diff --git a/test/unit/org/apache/cassandra/cql3/ast/Mutation.java b/test/unit/org/apache/cassandra/cql3/ast/Mutation.java index 729bb8127a5a..c21508f4d21c 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/Mutation.java +++ b/test/unit/org/apache/cassandra/cql3/ast/Mutation.java @@ -30,7 +30,9 @@ import java.util.stream.Stream; import javax.annotation.Nullable; +import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.LongType; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.TableMetadata; @@ -50,6 +52,13 @@ protected Mutation(Kind kind, TableReference table) public abstract boolean isCas(); + public Mutation withTimestamp(long timestamp) + { + return withTimestamp(new Timestamp(new Literal(timestamp, LongType.instance))); + } + + public abstract Mutation withTimestamp(Timestamp timestamp); + public final Kind mutationKind() { return kind; @@ -163,6 +172,11 @@ public Using(Optional<TTL> ttl, Optional<Timestamp> timestamp) this.timestamp = timestamp; } + public Using withTimestamp(Timestamp timestamp) + { + return new Using(ttl, Optional.of(timestamp)); + } + @Override public void toCQL(StringBuilder sb, CQLFormatter formatter) { @@ -178,6 +192,18 @@ public void toCQL(StringBuilder sb, CQLFormatter formatter) timestamp.get().toCQL(sb, formatter); } } + + @Override + public Stream<? extends Element> stream() + { + int size = (ttl.isPresent() ? 1 : 0) + (timestamp.isPresent() ? 1 : 0); + switch (size) + { + case 0: return Stream.empty(); + case 1: return Stream.of(ttl.isPresent() ? ttl.get() : timestamp.get()); + default: return Stream.of(ttl.get(), timestamp.get()); + } + } } public static class Insert extends Mutation @@ -272,6 +298,14 @@ public boolean isCas() { return ifNotExists; } + + @Override + public Insert withTimestamp(Timestamp timestamp) + { + return new Insert(table, values, ifNotExists, using.isEmpty() + ? Optional.of(new Using(Optional.empty(), Optional.of(timestamp))) + : using.map(u -> u.withTimestamp(timestamp))); + } } public static class Update extends Mutation @@ -380,24 +414,33 @@ public boolean isCas() { return casCondition.isPresent(); } + + @Override + public Update withTimestamp(Timestamp timestamp) + { + var updated = using.isEmpty() + ? Optional.of(new Using(Optional.empty(), Optional.of(timestamp))) + : using.map(u -> u.withTimestamp(timestamp)); + return new Update(table, updated, set, where, casCondition); + } } public static class Delete extends Mutation { public final List<Symbol> columns; - public final Optional<Timestamp> using; + public final Optional<Timestamp> timestamp; public final Conditional where; public final Optional<? extends CasCondition> casCondition; public Delete(List<Symbol> columns, TableReference table, - Optional<Timestamp> using, + Optional<Timestamp> timestamp, Conditional where, Optional<? extends CasCondition> casCondition) { super(Mutation.Kind.DELETE, table); this.columns = columns; - this.using = using; + this.timestamp = timestamp; this.where = where; this.casCondition = casCondition; } @@ -429,10 +472,11 @@ public void toCQL(StringBuilder sb, CQLFormatter formatter) sb.append("FROM "); table.toCQL(sb, formatter); // [USING TIMESTAMP timestamp_value] - if (using.isPresent()) + if (timestamp.isPresent()) { formatter.section(sb); - using.get().toCQL(sb, formatter); + sb.append("USING "); + timestamp.get().toCQL(sb, formatter); } // WHERE PK_column_conditions toCQLWhere(this.where, sb, formatter); @@ -450,8 +494,8 @@ public Stream<? extends Element> stream() List<Element> elements = new ArrayList<>(columns.size() + 4); elements.addAll(columns); elements.add(table); - if (using.isPresent()) - elements.add(using.get()); + if (timestamp.isPresent()) + elements.add(timestamp.get()); elements.add(where); if (casCondition.isPresent()) elements.add(casCondition.get()); @@ -478,7 +522,7 @@ public Statement visit(Visitor v) updated = true; if (!updated) return this; - return new Delete(copiedColumns, table, using, copiedWhere, casCondition); + return new Delete(copiedColumns, table, timestamp, copiedWhere, casCondition); } @Override @@ -486,12 +530,18 @@ public boolean isCas() { return casCondition.isPresent(); } + + @Override + public Delete withTimestamp(Timestamp timestamp) + { + return new Delete(columns, table, Optional.of(timestamp), where, casCondition); + } } - public static abstract class BaseBuilder<T, B extends BaseBuilder<T, B>> + public static abstract class BaseBuilder<T, B extends BaseBuilder<T, B>> implements Conditional.EqBuilderPlus<B> { private final Kind kind; - private final TableMetadata metadata; + protected final TableMetadata metadata; protected final LinkedHashSet<Symbol> partitionColumns, clusteringColumns, primaryColumns, regularAndStatic, allColumns; private boolean includeKeyspace = true; private final Set<Symbol> neededPks = new HashSet<>(); @@ -509,19 +559,18 @@ protected BaseBuilder(Kind kind, TableMetadata table) this.primaryColumns.addAll(clusteringColumns); this.regularAndStatic = new LinkedHashSet<>(); this.regularAndStatic.addAll(toSet(table.regularAndStaticColumns())); - this.allColumns = toSet(safeColumns(table)); + this.allColumns = toSet(table.columnsInFixedOrder()); neededPks.addAll(partitionColumns); } - public static List<ColumnMetadata> safeColumns(TableMetadata metadata) + public abstract T build(); + + @Override + public TableMetadata metadata() { - List<ColumnMetadata> columns = new ArrayList<>(metadata.columns().size()); - metadata.allColumnsInSelectOrder().forEachRemaining(columns::add); - return columns; + return metadata; } - public abstract T build(); - protected void assertAllPksHaveEq() { if (neededPks.isEmpty()) @@ -562,7 +611,7 @@ protected TableReference tableRef() } } - public static class InsertBuilder extends BaseBuilder<Insert, InsertBuilder> implements Conditional.EqBuilder<InsertBuilder> + public static class InsertBuilder extends BaseBuilder<Insert, InsertBuilder> { private final LinkedHashMap<Symbol, Expression> values = new LinkedHashMap<>(); private boolean ifNotExists = false; @@ -616,7 +665,7 @@ public Insert build() } } - public static class UpdateBuilder extends BaseBuilder<Update, UpdateBuilder> implements Conditional.ConditionalBuilder<UpdateBuilder> + public static class UpdateBuilder extends BaseBuilder<Update, UpdateBuilder> implements Conditional.ConditionalBuilderPlus<UpdateBuilder> { private @Nullable TTL ttl; private @Nullable Timestamp timestamp; @@ -671,6 +720,12 @@ public UpdateBuilder set(String column, int value) return set(new Symbol(column, Int32Type.instance), Bind.of(value)); } + public UpdateBuilder set(String column, String value) + { + Symbol symbol = new Symbol(metadata.getColumn(new ColumnIdentifier(column, true))); + return set(symbol, new Bind(symbol.type().asCQL3Type().fromCQLLiteral(value), symbol.type())); + } + @Override public UpdateBuilder where(Expression ref, Conditional.Where.Inequality kind, Expression expression) { @@ -717,7 +772,7 @@ public Update build() } } - public static class DeleteBuilder extends BaseBuilder<Delete, DeleteBuilder> implements Conditional.ConditionalBuilder<DeleteBuilder> + public static class DeleteBuilder extends BaseBuilder<Delete, DeleteBuilder> implements Conditional.ConditionalBuilderPlus<DeleteBuilder> { private final List<Symbol> columns = new ArrayList<>(); private @Nullable Timestamp timestamp = null; @@ -747,6 +802,11 @@ public List<Symbol> columns() return Collections.unmodifiableList(columns); } + public DeleteBuilder column(String columnName) + { + return column(Symbol.from(metadata.getColumn(new ColumnIdentifier(columnName, true)))); + } + public DeleteBuilder column(Symbol symbol) { if (!regularAndStatic.contains(symbol)) diff --git a/test/unit/org/apache/cassandra/cql3/ast/Select.java b/test/unit/org/apache/cassandra/cql3/ast/Select.java index 224c8f6d5682..55f984df9182 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/Select.java +++ b/test/unit/org/apache/cassandra/cql3/ast/Select.java @@ -89,6 +89,11 @@ public static Builder builder() return new Builder(); } + public static TableBasedBuilder builder(TableMetadata metadata) + { + return new TableBasedBuilder(metadata); + } + public Select withAllowFiltering() { return new Select(selections, source, where, orderBy, limit, true); @@ -301,105 +306,84 @@ public OrderBy build() } } - public static class Builder implements Conditional.ConditionalBuilder<Builder> + public static class BaseBuilder<T extends BaseBuilder<T>> implements Conditional.ConditionalBuilder<T> { @Nullable // null means wildcard private List<Expression> selections = new ArrayList<>(); - private Optional<TableReference> source = Optional.empty(); + protected Optional<TableReference> source = Optional.empty(); private Conditional.Builder where = new Conditional.Builder(); private OrderBy.Builder orderBy = new OrderBy.Builder(); private Optional<Value> limit = Optional.empty(); private boolean allowFiltering = false; - public Builder wildcard() + public T wildcard() { if (selections != null && !selections.isEmpty()) throw new IllegalStateException("Attempted to use * for selection but existing selections exist: " + selections); selections = null; - return this; + return (T) this; } - public Builder columnSelection(String name, AbstractType<?> type) + public T columnSelection(String name, AbstractType<?> type) { return selection(new Symbol(name, type)); } - public Builder allowFiltering() + public T allowFiltering() { allowFiltering = true; - return this; + return (T) this; } - public Builder selection(Expression e) + public T selection(Expression e) { if (selections == null) throw new IllegalStateException("Unable to add '" + e.name() + "' as a selection as * was already requested"); selections.add(e); - return this; - } - - public Builder table(TableReference ref) - { - source = Optional.of(ref); - return this; - } - - public Builder table(String ks, String name) - { - return table(new TableReference(Optional.of(ks), name)); - } - - public Builder table(String name) - { - return table(new TableReference(name)); - } - - public Builder table(TableMetadata table) - { - return table(TableReference.from(table)); + return (T) this; } @Override - public Builder where(Expression ref, Conditional.Where.Inequality kind, Expression expression) + public T where(Expression ref, Conditional.Where.Inequality kind, Expression expression) { where.where(ref, kind, expression); - return this; + return (T) this; } @Override - public Builder between(Expression ref, Expression start, Expression end) + public T between(Expression ref, Expression start, Expression end) { where.between(ref, start, end); - return this; + return (T) this; } @Override - public Builder in(ReferenceExpression ref, List<? extends Expression> expressions) + public T in(ReferenceExpression ref, List<? extends Expression> expressions) { where.in(ref, expressions); - return this; + return (T) this; } @Override - public Builder is(Symbol ref, Conditional.Is.Kind kind) + public T is(Symbol ref, Conditional.Is.Kind kind) { where.is(ref, kind); - return this; + return (T) this; } - public Builder orderByColumn(String name, AbstractType<?> type, OrderBy.Ordering ordering) + public T orderByColumn(String name, AbstractType<?> type, OrderBy.Ordering ordering) { orderBy.add(new Symbol(name, type), ordering); - return this; + return (T) this; } - public Builder limit(Value limit) + public T limit(Value limit) { this.limit = Optional.of(limit); - return this; + return (T) this; } - public Builder limit(int limit) + public T limit(int limit) { return limit(Bind.of(limit)); } @@ -414,4 +398,45 @@ public Select build() allowFiltering); } } + + public static class Builder extends BaseBuilder<Builder> + { + public Builder table(TableReference ref) + { + source = Optional.of(ref); + return this; + } + + public Builder table(String ks, String name) + { + return table(new TableReference(Optional.of(ks), name)); + } + + public Builder table(String name) + { + return table(new TableReference(name)); + } + + public Builder table(TableMetadata table) + { + return table(TableReference.from(table)); + } + } + + public static class TableBasedBuilder extends BaseBuilder<TableBasedBuilder> implements Conditional.ConditionalBuilderPlus<TableBasedBuilder> + { + private final TableMetadata metadata; + + public TableBasedBuilder(TableMetadata metadata) + { + this.metadata = metadata; + source = Optional.of(TableReference.from(metadata)); + } + + @Override + public TableMetadata metadata() + { + return metadata; + } + } } diff --git a/test/unit/org/apache/cassandra/cql3/ast/StandardVisitors.java b/test/unit/org/apache/cassandra/cql3/ast/StandardVisitors.java index 2a76cd80972d..854c096c0723 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/StandardVisitors.java +++ b/test/unit/org/apache/cassandra/cql3/ast/StandardVisitors.java @@ -44,6 +44,16 @@ public Expression visit(Expression e) } }; + public static final Visitor APPLY_OPERATOR = new Visitor() + { + @Override + public Expression visit(Expression e) + { + if (!(e instanceof Operator)) return e; + return new Bind(ExpressionEvaluator.tryEval((Operator) e).get(), e.type()); + } + }; + public static final Visitor.CompositeVisitor DEBUG = Visitor.CompositeVisitor.of(UNWRAP_TYPE_HINT, BIND_TO_LITERAL); private StandardVisitors() {} diff --git a/test/unit/org/apache/cassandra/utils/ASTGenerators.java b/test/unit/org/apache/cassandra/utils/ASTGenerators.java index ccb5ebaff15d..a02bf65610c0 100644 --- a/test/unit/org/apache/cassandra/utils/ASTGenerators.java +++ b/test/unit/org/apache/cassandra/utils/ASTGenerators.java @@ -41,6 +41,7 @@ import com.google.common.collect.Iterables; +import org.apache.cassandra.cql3.KnownIssue; import org.apache.cassandra.cql3.ast.AssignmentOperator; import org.apache.cassandra.cql3.ast.Bind; import org.apache.cassandra.cql3.ast.CasCondition; @@ -74,6 +75,8 @@ public class ASTGenerators { + public static final EnumSet<KnownIssue> IGNORE_ISSUES = KnownIssue.ignoreAll(); + static Gen<Value> valueGen(Object value, AbstractType<?> type) { Gen<Boolean> bool = SourceDSL.booleans().all(); @@ -95,17 +98,6 @@ private static <K, V> Map<K, V> assertDeterministic(Map<K, V> map) throw new AssertionError("Unsupported map type: " + map.getClass()); } - /** - * Returns a list of all columns in a deterministic order. This method is similar to {@link TableMetadata#columns()}, - * but that method uses a hash order, so the values could be different from host to host or jvm to jvm... - */ - public static List<ColumnMetadata> allColumnsInFixedOrder(TableMetadata metadata) - { - List<ColumnMetadata> columns = new ArrayList<>(metadata.columns().size()); - metadata.allColumnsInSelectOrder().forEachRemaining(columns::add); - return columns; - } - public static Gen<AssignmentOperator> assignmentOperatorGen(EnumSet<AssignmentOperator.Kind> allowed, Expression right) { if (allowed.isEmpty()) @@ -155,8 +147,9 @@ public static Gen<Operator> operatorGen(Set<Operator.Kind> allowed, Expression e //NOTE: (smallint) of -11843 and 3749 failed as well... //NOTE: (long) was found and didn't fail... //NOTE: see https://the-asf.slack.com/archives/CK23JSY2K/p1724819303058669 - varint didn't fail but serialized using int32 which causes equality mismatches for pk/ck lookups - if (e.type().unwrap() == ShortType.instance - || e.type().unwrap() == IntegerType.instance) // seed=7525457176675272023L + if ((e.type().unwrap() == ShortType.instance + || e.type().unwrap() == IntegerType.instance) + && IGNORE_ISSUES.contains(KnownIssue.SHORT_AND_VARINT_GET_INT_FUNCTIONS)) // seed=7525457176675272023L { left = new TypeHint(left); right = new TypeHint(right); @@ -165,21 +158,29 @@ public static Gen<Operator> operatorGen(Set<Operator.Kind> allowed, Expression e }; } - public static class ExpressionBuilder<T> + public static class ExpressionBuilder { - private final AbstractType<T> type; + private final AbstractType<?> type; private final EnumSet<Operator.Kind> allowedOperators; - private Gen<T> valueGen; + private Gen<?> valueGen; private Gen<Boolean> useOperator = SourceDSL.booleans().all(); + private Gen<Boolean> useEmpty = SourceDSL.arbitrary().constant(false); private BiFunction<Object, AbstractType<?>, Gen<Value>> literalOrBindGen = ASTGenerators::valueGen; - public ExpressionBuilder(AbstractType<T> type) + public ExpressionBuilder(AbstractType<?> type) { this.type = type.unwrap(); this.valueGen = AbstractTypeGenerators.getTypeSupport(this.type).valueGen; this.allowedOperators = Operator.supportsOperators(this.type); } + public ExpressionBuilder allowEmpty() + { + if (!type.allowsEmpty()) return this; + useEmpty = SourceDSL.integers().between(1, 100).map(i -> i < 10); + return this; + } + public ExpressionBuilder withOperators() { useOperator = i -> true; @@ -209,6 +210,8 @@ public Gen<Expression> build() //TODO (coverage): rather than single level operators, allow nested (a + b + c + d) Gen<Value> leaf = rs -> literalOrBindGen.apply(valueGen.generate(rs), type).generate(rs); return rs -> { + if (useEmpty.generate(rs)) + return new Bind(ByteBufferUtil.EMPTY_BYTE_BUFFER, type); Expression e = leaf.generate(rs); if (!allowedOperators.isEmpty() && useOperator.generate(rs)) e = operatorGen(allowedOperators, e, leaf).generate(rs); @@ -304,7 +307,7 @@ private static Conditional and(Map<Symbol, Expression> data) private static Gen<List<Expression>> selectColumns(TableMetadata metadata) { - List<ColumnMetadata> columns = allColumnsInFixedOrder(metadata); + List<ColumnMetadata> columns = metadata.columnsInFixedOrder(); Constraint between = Constraint.between(0, columns.size() - 1); Gen<int[]> indexGen = rnd -> { int size = Math.toIntExact(rnd.next(between)) + 1; @@ -324,7 +327,7 @@ private static Gen<List<Expression>> selectColumns(TableMetadata metadata) private static Gen<Map<Symbol, Expression>> partitionKeyGen(TableMetadata metadata) { Map<ColumnMetadata, Gen<?>> gens = new LinkedHashMap<>(); - for (ColumnMetadata col : allColumnsInFixedOrder(metadata)) + for (ColumnMetadata col : metadata.columnsInFixedOrder()) gens.put(col, AbstractTypeGenerators.getTypeSupport(col.type).valueGen); return rnd -> { Map<Symbol, Expression> output = new LinkedHashMap<>(); @@ -353,7 +356,7 @@ public enum DeleteKind { Partition, Row, Column } private Gen<Boolean> useCasIf = SourceDSL.booleans().all(); private BiFunction<RandomnessSource, List<Symbol>, List<Symbol>> ifConditionFilter = (rnd, symbols) -> symbols; private Gen<DeleteKind> deleteKindGen = SourceDSL.arbitrary().enumValues(DeleteKind.class); - private Map<Symbol, ExpressionBuilder<?>> columnExpressions = new LinkedHashMap<>(); + private Map<Symbol, ExpressionBuilder> columnExpressions = new LinkedHashMap<>(); public MutationGenBuilder(TableMetadata metadata) { @@ -369,7 +372,13 @@ public MutationGenBuilder(TableMetadata metadata) regularAndStaticColumns.addAll(regularColumns); for (Symbol symbol : allColumns) - columnExpressions.put(symbol, new ExpressionBuilder<>(symbol.type())); + columnExpressions.put(symbol, new ExpressionBuilder(symbol.type())); + } + + public MutationGenBuilder allowEmpty(Symbol symbol) + { + columnExpressions.get(symbol).allowEmpty(); + return this; } public MutationGenBuilder withDeletionKind(Gen<DeleteKind> deleteKindGen) @@ -483,7 +492,7 @@ public MutationGenBuilder withClusterings(Gen<? extends Map<Symbol, Object>> val } private static void values(RandomnessSource rnd, - Map<Symbol, ExpressionBuilder<?>> columnExpressions, + Map<Symbol, ExpressionBuilder> columnExpressions, Conditional.EqBuilder<?> builder, LinkedHashSet<Symbol> columns, @Nullable Gen<? extends Map<Symbol, Object>> gen) diff --git a/test/unit/org/apache/cassandra/utils/ImmutableUniqueList.java b/test/unit/org/apache/cassandra/utils/ImmutableUniqueList.java new file mode 100644 index 000000000000..73e7da9a9121 --- /dev/null +++ b/test/unit/org/apache/cassandra/utils/ImmutableUniqueList.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils; + +import java.util.AbstractList; +import java.util.AbstractSet; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.RandomAccess; + +import com.google.common.collect.Iterators; + +import org.agrona.collections.Object2IntHashMap; + +public class ImmutableUniqueList<T> extends AbstractList<T> implements RandomAccess +{ + private final T[] values; + private final Object2IntHashMap<T> indexLookup; + private transient AsSet asSet = null; + private ImmutableUniqueList(Builder<T> builder) + { + values = (T[]) builder.values.toArray(Object[]::new); + indexLookup = new Object2IntHashMap<>(builder.indexLookup); + } + + public static <T> Builder<T> builder() + { + return new Builder<>(); + } + + public AsSet asSet() + { + if (asSet != null) return asSet; + return asSet = new AsSet(); + } + + @Override + public T get(int index) + { + return values[index]; + } + + @Override + public int indexOf(Object o) + { + return indexLookup.getOrDefault(o, -1); + } + + @Override + public int lastIndexOf(Object o) + { + // values are unique... + return indexOf(o); + } + + @Override + public boolean contains(Object o) + { + return indexLookup.containsKey(o); + } + + @Override + public int size() + { + return values.length; + } + + public static final class Builder<T> extends AbstractSet<T> + { + private final List<T> values = new ArrayList<>(); + private final Object2IntHashMap<T> indexLookup = new Object2IntHashMap<>(-1); + private int idx; + + public Builder<T> mayAddAll(Collection<? extends T> values) + { + addAll(values); + return this; + } + + @Override + public boolean add(T t) + { + if (indexLookup.containsKey(t)) return false; + int idx = this.idx++; + indexLookup.put(t, idx); + values.add(t); + return true; + } + + @Override + public boolean remove(Object o) + { + throw new UnsupportedOperationException(); + } + + @Override + public void clear() + { + values.clear(); + indexLookup.clear(); + idx = 0; + } + + @Override + public boolean isEmpty() + { + return values.isEmpty(); + } + + @Override + public boolean contains(Object o) + { + return indexLookup.containsKey(o); + } + + @Override + public Iterator<T> iterator() + { + return Iterators.unmodifiableIterator(values.iterator()); + } + + @Override + public int size() + { + return values.size(); + } + + public ImmutableUniqueList<T> build() + { + return new ImmutableUniqueList<>(this); + } + + public ImmutableUniqueList<T> buildAndClear() + { + ImmutableUniqueList<T> list = new ImmutableUniqueList<>(this); + clear(); + return list; + } + } + + public class AsSet extends AbstractSet<T> + { + @Override + public Iterator<T> iterator() + { + return ImmutableUniqueList.this.iterator(); + } + + @Override + public int size() + { + return values.length; + } + } +} From 914046ae469a43ef0ec5940ac9fa6b0eda3707b2 Mon Sep 17 00:00:00 2001 From: Bereng <berenguerblasi@gmail.com> Date: Mon, 10 Feb 2025 09:52:32 +0100 Subject: [PATCH 133/225] Long running repairs autofail prematurely patch by Berenguer Blasi; reviewed by Bernardo Botella Corbi, Michael Semb Wever for CASSANDRA-20312 --- .../org/apache/cassandra/repair/consistent/LocalSessions.java | 1 + .../apache/cassandra/repair/consistent/LocalSessionTest.java | 3 +++ 2 files changed, 4 insertions(+) diff --git a/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java b/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java index d527c962bfdb..dd60ad4a21c4 100644 --- a/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java +++ b/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java @@ -1021,6 +1021,7 @@ public void handleStatusResponse(InetAddressAndPort from, StatusResponse respons } else { + session.setLastUpdate(); logger.debug("Received StatusResponse for repair session {} with state {}, which is not actionable. Doing nothing.", sessionID, response.state); } } diff --git a/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java b/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java index 2f8dae6a1dab..10036814140f 100644 --- a/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java +++ b/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java @@ -683,8 +683,11 @@ public void handleStatusResponseNoop() throws Exception sessions.start(); LocalSession session = sessions.prepareForTest(sessionID); session.setState(REPAIRING); + int lastUpdatedOriginal = session.getLastUpdate(); + Thread.sleep(1100); sessions.handleStatusResponse(PARTICIPANT1, new StatusResponse(sessionID, FINALIZE_PROMISED)); + Assert.assertNotEquals(lastUpdatedOriginal, session.getLastUpdate()); Assert.assertEquals(REPAIRING, session.getState()); } From bd99331bf2a963f77ca7cba1d39efb985b21842f Mon Sep 17 00:00:00 2001 From: Caleb Rackliffe <calebrackliffe@gmail.com> Date: Tue, 11 Feb 2025 12:06:09 -0600 Subject: [PATCH 134/225] Avoid fetching entire partitions on unresolved static rows in RFP when no static column predicates exist patch by Caleb Rackliffe; reviewed by David Capwell for CASSANDRA-20243 --- CHANGES.txt | 1 + .../apache/cassandra/db/filter/RowFilter.java | 18 +++++++++++++++ .../index/sai/plan/QueryController.java | 6 ++++- .../cassandra/service/reads/DataResolver.java | 6 ++++- .../reads/ReplicaFilteringProtection.java | 10 ++++---- .../test/ReplicaFilteringProtectionTest.java | 19 +++++++++++++++ .../test/sai/StrictFilteringTest.java | 23 +++++++++++++++++-- 7 files changed, 74 insertions(+), 9 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index cbbd1767b596..f659a1f220af 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.4 + * Avoid fetching entire partitions on unresolved static rows in RFP when no static column predicates exist (CASSANDRA-20243) * Avoid indexing empty values for non-literals and types that do not allow them (CASSANDRA-20313) * Fix incorrect results of min / max in-built functions on clustering columns in descending order (CASSANDRA-20295) * Avoid possible consistency violations for SAI intersection queries over repaired index matches and multiple non-indexed column matches (CASSANDRA-20189) diff --git a/src/java/org/apache/cassandra/db/filter/RowFilter.java b/src/java/org/apache/cassandra/db/filter/RowFilter.java index 483c1631648e..9b83a56f23f5 100644 --- a/src/java/org/apache/cassandra/db/filter/RowFilter.java +++ b/src/java/org/apache/cassandra/db/filter/RowFilter.java @@ -394,6 +394,24 @@ public RowFilter without(ColumnMetadata column, Operator op, ByteBuffer value) return withNewExpressions(newExpressions); } + public boolean hasNonKeyExpression() + { + for (Expression e : expressions) + if (!e.column().isPrimaryKeyColumn()) + return true; + + return false; + } + + public boolean hasStaticExpression() + { + for (Expression e : expressions) + if (e.column().isStatic()) + return true; + + return false; + } + public RowFilter withoutExpressions() { return withNewExpressions(Collections.emptyList()); diff --git a/src/java/org/apache/cassandra/index/sai/plan/QueryController.java b/src/java/org/apache/cassandra/index/sai/plan/QueryController.java index 614456c493dc..8bd1e25f280e 100644 --- a/src/java/org/apache/cassandra/index/sai/plan/QueryController.java +++ b/src/java/org/apache/cassandra/index/sai/plan/QueryController.java @@ -271,7 +271,11 @@ public KeyRangeIterator.Builder getIndexQueryResults(Collection<Expression> expr private void maybeTriggerGuardrails(QueryViewBuilder.QueryView queryView) { - int referencedIndexes = queryView.referencedIndexes.size(); + int referencedIndexes = 0; + + // We want to make sure that no individual column expression touches too many SSTable-attached indexes: + for (Pair<Expression, Collection<SSTableIndex>> expressionSSTables : queryView.view) + referencedIndexes = Math.max(referencedIndexes, expressionSSTables.right.size()); if (Guardrails.saiSSTableIndexesPerQuery.failsOn(referencedIndexes, null)) { diff --git a/src/java/org/apache/cassandra/service/reads/DataResolver.java b/src/java/org/apache/cassandra/service/reads/DataResolver.java index 64e4c72b01dd..332a78570851 100644 --- a/src/java/org/apache/cassandra/service/reads/DataResolver.java +++ b/src/java/org/apache/cassandra/service/reads/DataResolver.java @@ -138,7 +138,7 @@ private boolean usesReplicaFilteringProtection() return false; Index.QueryPlan queryPlan = command.indexQueryPlan(); - if (queryPlan == null ) + if (queryPlan == null) return true; return queryPlan.supportsReplicaFilteringProtection(command.rowFilter()); @@ -275,6 +275,10 @@ private PartitionIterator resolveWithReplicaFilteringProtection(E replicas, Repa private UnaryOperator<PartitionIterator> preCountFilterForReplicaFilteringProtection() { + // Key columns are immutable and should never need to participate in replica filtering + if (!command.rowFilter().hasNonKeyExpression()) + return results -> results; + return results -> { Index.Searcher searcher = command.indexSearcher(); // in case of "ALLOW FILTERING" without index diff --git a/src/java/org/apache/cassandra/service/reads/ReplicaFilteringProtection.java b/src/java/org/apache/cassandra/service/reads/ReplicaFilteringProtection.java index 9ec02a5b2064..7c57245bb178 100644 --- a/src/java/org/apache/cassandra/service/reads/ReplicaFilteringProtection.java +++ b/src/java/org/apache/cassandra/service/reads/ReplicaFilteringProtection.java @@ -427,12 +427,12 @@ private void addToFetch(Row row) if (toFetch == null) toFetch = BTreeSet.builder(command.metadata().comparator); - // Note that for static, we shouldn't add the clustering to the clustering set (the - // ClusteringIndexNamesFilter we'll build from this later does not expect it), but the fact - // we created a builder in the first place will act as a marker that the static row must be - // fetched, even if no other rows are added for this partition. if (row.isStatic()) - unresolvedStatic = true; + // If there is an expression on a static column, the static row must be marked unresolved and the + // partition fetched, as completing the static row could produce matches across the entire partition. + // The static row itself will still be retrieved and completed if there is any unresolved non-static + // row, however, ensuring the latest static values are returned from the query. + unresolvedStatic = command.rowFilter().hasStaticExpression(); else toFetch.add(row.clustering()); } diff --git a/test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java b/test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java index fd8110cba72a..1cff326f41f4 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java @@ -27,6 +27,7 @@ import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; import org.apache.cassandra.distributed.api.IInvokableInstance; import org.apache.cassandra.distributed.api.SimpleQueryResult; import org.apache.cassandra.exceptions.OverloadedException; @@ -71,6 +72,24 @@ public static void teardown() cluster.close(); } + @Test + public void testMissingStaticRowWithNonStaticExpression() + { + cluster.schemaChange(withKeyspace("CREATE TABLE %s.single_predicate (pk0 int, ck0 int, ck1 int, s0 int static, s1 int static, v0 int, PRIMARY KEY (pk0, ck0, ck1)) " + + "WITH CLUSTERING ORDER BY (ck0 ASC, ck1 DESC) AND read_repair = 'NONE'")); + + cluster.get(1).executeInternal(withKeyspace("INSERT INTO %s.single_predicate (pk0, ck0, ck1, s0, s1, v0) " + + "VALUES (0, 1, 2, 3, 4, 5) USING TIMESTAMP 1")); + cluster.get(2).executeInternal(withKeyspace("UPDATE %s.single_predicate USING TIMESTAMP 2 SET s0 = 6, s1 = 7, v0 = 8 " + + "WHERE pk0 = 0 AND ck0 = 9 AND ck1 = 10")); + + // Node 2 will not produce a match for the static row. Make sure that replica filtering protection does not + // fetch the entire partition, which could let non-matching rows slip through combined with the fact that we + // don't post-filter at the coordinator with no regular column predicates in the query. + String select = withKeyspace("SELECT pk0, ck0, ck1, s0, s1 FROM %s.single_predicate WHERE ck1 = 2 ALLOW FILTERING"); + assertRows(cluster.coordinator(1).execute(select, ConsistencyLevel.ALL), row(0, 1, 2, 6, 7)); + } + @Test public void testMissedUpdatesBelowCachingWarnThreshold() { diff --git a/test/distributed/org/apache/cassandra/distributed/test/sai/StrictFilteringTest.java b/test/distributed/org/apache/cassandra/distributed/test/sai/StrictFilteringTest.java index cd753d6e01f4..5ef92bb9ef80 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/sai/StrictFilteringTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/sai/StrictFilteringTest.java @@ -29,7 +29,6 @@ import org.apache.cassandra.cql3.Operator; import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.api.ConsistencyLevel; -import org.apache.cassandra.distributed.shared.AssertUtils; import org.apache.cassandra.distributed.test.TestBaseImpl; import org.apache.cassandra.index.sai.plan.StorageAttachedIndexQueryPlan; @@ -54,6 +53,26 @@ public static void setUpCluster() throws IOException CLUSTER = init(Cluster.build(2).withConfig(config -> config.set("hinted_handoff_enabled", false).with(GOSSIP).with(NETWORK)).start()); } + @Test + public void testMissingStaticRowWithNonStaticExpression() + { + CLUSTER.schemaChange(withKeyspace("CREATE TABLE %s.single_predicate (pk0 int, ck0 int, ck1 int, s0 int static, s1 int static, v0 int, PRIMARY KEY (pk0, ck0, ck1)) " + + "WITH CLUSTERING ORDER BY (ck0 ASC, ck1 DESC) AND read_repair = 'NONE'")); + CLUSTER.schemaChange(withKeyspace("CREATE INDEX ON %s.single_predicate(ck1) USING 'sai'")); + SAIUtil.waitForIndexQueryable(CLUSTER, KEYSPACE); + + CLUSTER.get(1).executeInternal(withKeyspace("INSERT INTO %s.single_predicate (pk0, ck0, ck1, s0, s1, v0) " + + "VALUES (0, 1, 2, 3, 4, 5) USING TIMESTAMP 1")); + CLUSTER.get(2).executeInternal(withKeyspace("UPDATE %s.single_predicate USING TIMESTAMP 2 SET s0 = 6, s1 = 7, v0 = 8 " + + "WHERE pk0 = 0 AND ck0 = 9 AND ck1 = 10")); + + // Node 2 will not produce a match for the static row. Make sure that replica filtering protection does not + // fetch the entire partition, which could let non-matching rows slip through combined with the fact that we + // don't post-filter at the coordinator with no regular column predicates in the query. + String select = withKeyspace("SELECT pk0, ck0, ck1, s0, s1 FROM %s.single_predicate WHERE ck1 = 2 ALLOW FILTERING"); + assertRows(CLUSTER.coordinator(1).execute(select, ConsistencyLevel.ALL), row(0, 1, 2, 6, 7)); + } + @Test public void shouldDegradeToUnionOnSingleStatic() { @@ -257,7 +276,7 @@ public void testTimestampCollision() String select = withKeyspace("SELECT * FROM %s.timestamp_collision WHERE a = 2 AND b = 2"); Object[][] initialRows = CLUSTER.coordinator(1).execute(select, ConsistencyLevel.ALL); - assertRows(initialRows, AssertUtils.row(0, 2, 2)); + assertRows(initialRows, row(0, 2, 2)); } @Test From 1630ef2cec770c495dd011fefede4585b1f35fa8 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Tue, 4 Feb 2025 15:19:29 +0100 Subject: [PATCH 135/225] Implement NOT_NULL constraint patch by Stefan Miklosovic; reviewed by Bernardo Botella, Dinesh Joshi for CASSANDRA-20276 Co-authored-by: Bernardo Botella <bernardo.botella@apple.com> --- CHANGES.txt | 1 + .../pages/developing/cql/constraints.adoc | 48 +++++ src/antlr/Parser.g | 1 + .../AbstractFunctionConstraint.java | 31 +++ .../cql3/constraints/ColumnConstraint.java | 36 +++- .../cql3/constraints/ColumnConstraints.java | 16 +- .../cql3/constraints/ConstraintFunction.java | 39 +++- .../constraints/FunctionColumnConstraint.java | 30 ++- .../cql3/constraints/LengthConstraint.java | 16 +- .../cql3/constraints/NotNullConstraint.java | 73 +++++++ .../constraints/ScalarColumnConstraint.java | 30 +-- .../UnaryFunctionColumnConstraint.java | 191 ++++++++++++++++++ .../statements/ModificationStatement.java | 23 +++ .../cql3/statements/StatementType.java | 9 + .../schema/AlterTableStatement.java | 4 +- .../schema/CreateTableStatement.java | 18 +- .../cassandra/schema/ColumnMetadata.java | 17 ++ .../cassandra/schema/TableMetadata.java | 12 +- .../test/ColumnConstraintsTest.java | 22 ++ .../distributed/test/log/SnapshotTest.java | 1 + ...ableWithTableConstraintValidationTest.java | 31 ++- .../contraints/ColumnConstraintsTest.java | 3 +- ...WithColumnCqlConstraintValidationTest.java | 95 ++++++++- ...ithColumnNotNullConstraintInvalidTest.java | 84 ++++++++ ...eWithColumnNotNullConstraintValidTest.java | 66 ++++++ 25 files changed, 827 insertions(+), 70 deletions(-) create mode 100644 src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionConstraint.java create mode 100644 src/java/org/apache/cassandra/cql3/constraints/NotNullConstraint.java create mode 100644 src/java/org/apache/cassandra/cql3/constraints/UnaryFunctionColumnConstraint.java create mode 100644 test/unit/org/apache/cassandra/contraints/CreateTableWithColumnNotNullConstraintInvalidTest.java create mode 100644 test/unit/org/apache/cassandra/contraints/CreateTableWithColumnNotNullConstraintValidTest.java diff --git a/CHANGES.txt b/CHANGES.txt index aba2b957f73c..e058d9745721 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Implement NOT_NULL constraint (CASSANDRA-20276) * Improve error messages for constraints (CASSANDRA-20266) * Add system_views.partition_key_statistics for querying SSTable metadata (CASSANDRA-20161) * CEP-42 - Add Constraints Framework (CASSANDRA-19947) diff --git a/doc/modules/cassandra/pages/developing/cql/constraints.adoc b/doc/modules/cassandra/pages/developing/cql/constraints.adoc index e6a76dbe7d3b..82f454618a35 100644 --- a/doc/modules/cassandra/pages/developing/cql/constraints.adoc +++ b/doc/modules/cassandra/pages/developing/cql/constraints.adoc @@ -91,3 +91,51 @@ Finally, the constraint can be removed: ---- ALTER TABLE keyspace.table ALTER name DROP CHECK; ---- + +=== NOT_NULL constraint + +Defines a constraint that checks if a column is not null in every modification statement. + +For example, let's have this table: + +---- +CREATE TABLE ks.tb ( + id int, + cl int, + col1 int CHECK NOT_NULL(col1), + col2 int CHECK NOT_NULL(col2), + PRIMARY KEY (id, cl) +); +---- + +then this statement would fail: + +---- +INSERT INTO ks.tb (id, cl, col1) VALUES (1, 2, 3); +... [Invalid query] message="Column 'col2' has to be specified as part of this query." +---- + +as well as this statement: + +---- +INSERT INTO ks.tb (id, cl, col1, col2) VALUES (1, 2, 3, null); +---- + +A column which has `NOT_NULL` constraint has to be specified in every modification statement. + +The constraint can be removed: + +---- +ALTER TABLE keyspace.table ALTER col1 DROP CHECK; +ALTER TABLE keyspace.table ALTER col2 DROP CHECK; +---- + +We can not remove the value of a column where `NOT_NULL` constraint is present: + +---- +DELETE col2 FROM ks.tb WHERE id = 1 AND cl = 2; +... [Invalid query] message="Column 'col2' can not be set to null." +---- + +Additionally, `NOT_NULL` can not be specified on any column of a primary key, +being it a partition key or a clustering column. diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g index 5f5908f6b0c6..682ba873e46d 100644 --- a/src/antlr/Parser.g +++ b/src/antlr/Parser.g @@ -796,6 +796,7 @@ columnConstraints returns [ColumnConstraints.Raw constraints] columnConstraint returns [ColumnConstraint columnConstraint] : funcName=ident '(' k=ident ')' op=relationType t=value { $columnConstraint = new FunctionColumnConstraint.Raw(funcName, k, op, t.getText()).prepare(); } + | funcName=ident '(' k=ident ')' { $columnConstraint = new UnaryFunctionColumnConstraint.Raw(funcName, k).prepare(); } | k=ident op=relationType t=value { $columnConstraint = new ScalarColumnConstraint.Raw(k, op, t.getText()).prepare(); } ; diff --git a/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionConstraint.java new file mode 100644 index 000000000000..942d884a5e57 --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionConstraint.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.constraints; + +import org.apache.cassandra.cql3.ColumnIdentifier; + +public abstract class AbstractFunctionConstraint<T> extends ColumnConstraint<T> +{ + public AbstractFunctionConstraint(ColumnIdentifier columnName) + { + super(columnName); + } + + public abstract String name(); +} diff --git a/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraint.java index ccbcdef48af1..bba70d1f8726 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraint.java @@ -20,6 +20,7 @@ import java.nio.ByteBuffer; +import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.CqlBuilder; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.tcm.serialization.MetadataSerializer; @@ -29,19 +30,26 @@ * Common class for the conditions that a CQL Constraint needs to implement to be integrated in the * CQL Constraints framework, with T as a constraint serializer. */ -public interface ColumnConstraint<T> +public abstract class ColumnConstraint<T> { + protected final ColumnIdentifier columnName; + + public ColumnConstraint(ColumnIdentifier columnName) + { + this.columnName = columnName; + } // Enum containing all the possible constraint serializers to help with serialization/deserialization // of constraints. - enum ConstraintType + public enum ConstraintType { // The order of that enum matters!! // We are serializing its enum position instead of its name. // Changing this enum would affect how that int is interpreted when deserializing. COMPOSED(ColumnConstraints.serializer), FUNCTION(FunctionColumnConstraint.serializer), - SCALAR(ScalarColumnConstraint.serializer); + SCALAR(ScalarColumnConstraint.serializer), + UNARY_FUNCTION(UnaryFunctionColumnConstraint.serializer); private final MetadataSerializer<?> serializer; @@ -56,9 +64,9 @@ public static MetadataSerializer<?> getSerializer(int i) } } - MetadataSerializer<T> serializer(); + public abstract MetadataSerializer<T> serializer(); - void appendCqlTo(CqlBuilder builder); + public abstract void appendCqlTo(CqlBuilder builder); /** * Method that evaluates the condition. It can either succeed or throw a {@link ConstraintViolationException}. @@ -66,7 +74,19 @@ public static MetadataSerializer<?> getSerializer(int i) * @param valueType value type of the column value under test * @param columnValue Column value to be evaluated at write time */ - void evaluate(AbstractType<?> valueType, ByteBuffer columnValue) throws ConstraintViolationException; + public void evaluate(AbstractType<?> valueType, ByteBuffer columnValue) throws ConstraintViolationException + { + if (columnValue.capacity() == 0) + throw new ConstraintViolationException("Column value does not satisfy value constraint for column '" + columnName + "' as it is null."); + + internalEvaluate(valueType, columnValue); + } + + /** + * Internal evaluation method, by default called from {@link ColumnConstraint#evaluate(AbstractType, ByteBuffer)}. + * {@code columnValue} is by default guaranteed to not represent CQL value of 'null'. + */ + protected abstract void internalEvaluate(AbstractType<?> valueType, ByteBuffer columnValue); /** * Method to validate the condition. This method is called when creating constraint via CQL. @@ -74,12 +94,12 @@ public static MetadataSerializer<?> getSerializer(int i) * * @param columnMetadata Metadata of the column in which the constraint is defined. */ - void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException; + public abstract void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException; /** * Method to get the Constraint serializer * * @return the Constraint type serializer */ - ConstraintType getConstraintType(); + public abstract ConstraintType getConstraintType(); } diff --git a/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraints.java b/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraints.java index 3b6646aec3ca..7efe046b1615 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraints.java +++ b/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraints.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.Objects; +import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.CqlBuilder; import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.db.marshal.AbstractType; @@ -34,9 +35,8 @@ import org.apache.cassandra.tcm.serialization.MetadataSerializer; import org.apache.cassandra.tcm.serialization.Version; - // group of constraints for the column -public class ColumnConstraints implements ColumnConstraint<ColumnConstraints> +public class ColumnConstraints extends ColumnConstraint<ColumnConstraints> { public static final Serializer serializer = new Serializer(); public static final ColumnConstraints NO_OP = new Noop(); @@ -45,6 +45,7 @@ public class ColumnConstraints implements ColumnConstraint<ColumnConstraints> public ColumnConstraints(List<ColumnConstraint<?>> constraints) { + super(null); this.constraints = constraints; } @@ -68,6 +69,12 @@ public void evaluate(AbstractType<?> valueType, ByteBuffer columnValue) throws C constraint.evaluate(valueType, columnValue); } + @Override + protected void internalEvaluate(AbstractType<?> valueType, ByteBuffer columnValue) + { + // nothing to evaluate here + } + public List<ColumnConstraint<?>> getConstraints() { return constraints; @@ -94,6 +101,11 @@ public boolean hasRelevantConstraints() return false; } + public void checkInvalidConstraintsCombinations(ColumnIdentifier columnName) + { + // TODO check duplicities etc CASSANDRA-20330 + } + @Override public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException { diff --git a/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java b/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java index aa3ea2303b3a..cdc22f2bd486 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java +++ b/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java @@ -20,6 +20,7 @@ import java.nio.ByteBuffer; +import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.Operator; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.schema.ColumnMetadata; @@ -27,22 +28,46 @@ /** * Interface to be implemented by functions that are executed as part of CQL constraints. */ -public interface ConstraintFunction +public abstract class ConstraintFunction { - /** - * @return the function name to be executed. - */ - String getName(); + protected final ColumnIdentifier columnName; + protected final String name; + + public ConstraintFunction(ColumnIdentifier columnName, String name) + { + this.columnName = columnName; + this.name = name; + } /** * Method that performs the actual condition test, executed during the write path. * It the test is not successful, it throws a {@link ConstraintViolationException}. */ - void evaluate(AbstractType<?> valueType, Operator relationType, String term, ByteBuffer columnValue) throws ConstraintViolationException; + public void evaluate(AbstractType<?> valueType, Operator relationType, String term, ByteBuffer columnValue) throws ConstraintViolationException + { + if (columnValue.capacity() == 0) + throw new ConstraintViolationException("Column value does not satisfy value constraint for column '" + columnName + "' as it is null."); + + internalEvaluate(valueType, relationType, term, columnValue); + } + + /** + * Internal evaluation method, by default called from {@link ConstraintFunction#evaluate(AbstractType, Operator, String, ByteBuffer)}. + * {@code columnValue} is by default guaranteed to not represent CQL value of 'null'. + */ + protected abstract void internalEvaluate(AbstractType<?> valueType, Operator relationType, String term, ByteBuffer columnValue); + + /** + * Used mostly for unary functions which do not expect any relation type nor term. + */ + public void evaluate(AbstractType<?> valueType, ByteBuffer columnValue) throws ConstraintViolationException + { + evaluate(valueType, null, null, columnValue); + } /** * Method that validates that a condition is valid. This method is called when the CQL constraint is created to determine * if the CQL statement is valid or needs to be rejected as invalid throwing a {@link InvalidConstraintDefinitionException} */ - void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException; + public abstract void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException; } diff --git a/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java index ad42e1e12913..a1a3339c434b 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java @@ -34,14 +34,13 @@ import org.apache.cassandra.tcm.serialization.Version; import org.apache.cassandra.utils.LocalizeString; -public class FunctionColumnConstraint implements ColumnConstraint<FunctionColumnConstraint> +public class FunctionColumnConstraint extends AbstractFunctionConstraint<FunctionColumnConstraint> { public static final Serializer serializer = new Serializer(); - public final ConstraintFunction function; - public final ColumnIdentifier columnName; - public final Operator relationType; - public final String term; + private final ConstraintFunction function; + private final Operator relationType; + private final String term; public final static class Raw { @@ -90,12 +89,17 @@ private static ConstraintFunction createConstraintFunction(String functionName, private FunctionColumnConstraint(ConstraintFunction function, ColumnIdentifier columnName, Operator relationType, String term) { + super(columnName); this.function = function; - this.columnName = columnName; this.relationType = relationType; this.term = term; } + public String name() + { + return function.name; + } + @Override public void appendCqlTo(CqlBuilder builder) { @@ -114,6 +118,12 @@ public void evaluate(AbstractType<?> valueType, ByteBuffer columnValue) function.evaluate(valueType, relationType, term, columnValue); } + @Override + protected void internalEvaluate(AbstractType<?> valueType, ByteBuffer columnValue) + { + // evaluation is done on function + } + @Override public void validate(ColumnMetadata columnMetadata) { @@ -130,13 +140,15 @@ public ConstraintType getConstraintType() void validateArgs(ColumnMetadata columnMetadata) { if (!columnMetadata.name.equals(columnName)) - throw new InvalidConstraintDefinitionException("Function parameter should be the column name"); + throw new InvalidConstraintDefinitionException(String.format("Parameter of %s constraint should be the column name (%s)", + name(), + columnMetadata.name)); } @Override public String toString() { - return function.getName() + "(" + columnName + ") " + relationType + " " + term; + return function.name + "(" + columnName + ") " + relationType + " " + term; } public static class Serializer implements MetadataSerializer<FunctionColumnConstraint> @@ -144,7 +156,7 @@ public static class Serializer implements MetadataSerializer<FunctionColumnConst @Override public void serialize(FunctionColumnConstraint columnConstraint, DataOutputPlus out, Version version) throws IOException { - out.writeUTF(columnConstraint.function.getName()); + out.writeUTF(columnConstraint.function.name); out.writeUTF(columnConstraint.columnName.toCQLString()); columnConstraint.relationType.writeTo(out); out.writeUTF(columnConstraint.term); diff --git a/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java index 054f4f43b046..8aa9aaf99d11 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java @@ -30,27 +30,17 @@ import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.utils.ByteBufferUtil; -public class LengthConstraint implements ConstraintFunction +public class LengthConstraint extends ConstraintFunction { private static final AbstractType<?>[] SUPPORTED_TYPES = new AbstractType[] { BytesType.instance, UTF8Type.instance, AsciiType.instance }; - public static final String FUNCTION_NAME = "LENGTH"; - - private final ColumnIdentifier columnName; - public LengthConstraint(ColumnIdentifier columnName) { - this.columnName = columnName; - } - - @Override - public String getName() - { - return FUNCTION_NAME; + super(columnName, "LENGTH"); } @Override - public void evaluate(AbstractType<?> valueType, Operator relationType, String term, ByteBuffer columnValue) + public void internalEvaluate(AbstractType<?> valueType, Operator relationType, String term, ByteBuffer columnValue) { int valueLength = getValueLength(columnValue, valueType); int sizeConstraint = Integer.parseInt(term); diff --git a/src/java/org/apache/cassandra/cql3/constraints/NotNullConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/NotNullConstraint.java new file mode 100644 index 000000000000..0fb164562b34 --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/constraints/NotNullConstraint.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.constraints; + +import java.nio.ByteBuffer; + +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.schema.ColumnMetadata; + +import static java.lang.String.format; + +public class NotNullConstraint extends ConstraintFunction +{ + public static final String FUNCTION_NAME = "NOT_NULL"; + + public NotNullConstraint(ColumnIdentifier columnName) + { + this(columnName, FUNCTION_NAME); + } + + public NotNullConstraint(ColumnIdentifier columnName, String name) + { + super(columnName, name); + } + + @Override + public void internalEvaluate(AbstractType<?> valueType, Operator relationType, String term, ByteBuffer columnValue) + { + // on purpose empty as evaluate method already covered nullity + } + + @Override + public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException + { + if (columnMetadata.isPrimaryKeyColumn()) + throw new InvalidConstraintDefinitionException(format("%s constraint can not be specified on a %s key column '%s'", + name, + columnMetadata.isPartitionKey() ? "partition" : "clustering", + columnMetadata.name)); + } + + @Override + public boolean equals(Object o) + { + if (this == o) + return true; + + if (!(o instanceof NotNullConstraint)) + return false; + + NotNullConstraint other = (NotNullConstraint) o; + + return columnName.equals(other.columnName) && name.equals(other.name); + } +} diff --git a/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java index 736ed5482365..71c2a77bf9d6 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java @@ -32,14 +32,13 @@ import org.apache.cassandra.tcm.serialization.MetadataSerializer; import org.apache.cassandra.tcm.serialization.Version; -public class ScalarColumnConstraint implements ColumnConstraint<ScalarColumnConstraint> +public class ScalarColumnConstraint extends ColumnConstraint<ScalarColumnConstraint> { - public final ColumnIdentifier param; - public final Operator relationType; - public final String term; - public final static Serializer serializer = new Serializer(); + private final Operator relationType; + private final String term; + public final static class Raw { public final ColumnIdentifier param; @@ -61,13 +60,14 @@ public ScalarColumnConstraint prepare() private ScalarColumnConstraint(ColumnIdentifier param, Operator relationType, String term) { - this.param = param; + super(param); this.relationType = relationType; this.term = term; } + @Override - public void evaluate(AbstractType<?> valueType, ByteBuffer columnValue) + protected void internalEvaluate(AbstractType<?> valueType, ByteBuffer columnValue) { ByteBuffer value; try @@ -76,19 +76,19 @@ public void evaluate(AbstractType<?> valueType, ByteBuffer columnValue) } catch (NumberFormatException exception) { - throw new ConstraintViolationException(param + " and " + term + " need to be numbers."); + throw new ConstraintViolationException(columnName + " and " + term + " need to be numbers."); } if (!relationType.isSatisfiedBy(valueType, columnValue, value)) - throw new ConstraintViolationException("Column value does not satisfy value constraint for column '" + param + "'. " - + "It should be " + param + " " + relationType + " " + term); + throw new ConstraintViolationException("Column value does not satisfy value constraint for column '" + columnName + "'. " + + "It should be " + columnName + " " + relationType + " " + term); } @Override public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException { if (!columnMetadata.type.isNumber()) - throw new InvalidConstraintDefinitionException(param + " is not a number"); + throw new InvalidConstraintDefinitionException("Column '" + columnName + "' is not a number type."); } @Override @@ -100,7 +100,7 @@ public ConstraintType getConstraintType() @Override public String toString() { - return param + " " + relationType + " " + term; + return columnName + " " + relationType + " " + term; } @Override @@ -120,7 +120,7 @@ private static class Serializer implements MetadataSerializer<ScalarColumnConstr @Override public void serialize(ScalarColumnConstraint columnConstraint, DataOutputPlus out, Version version) throws IOException { - out.writeUTF(columnConstraint.param.toCQLString()); + out.writeUTF(columnConstraint.columnName.toCQLString()); columnConstraint.relationType.writeTo(out); out.writeUTF(columnConstraint.term); } @@ -138,7 +138,7 @@ public long serializedSize(ScalarColumnConstraint columnConstraint, Version vers { return TypeSizes.sizeof(columnConstraint.term) + Operator.serializedSize() - + TypeSizes.sizeof(columnConstraint.param.toString()); + + TypeSizes.sizeof(columnConstraint.columnName.toString()); } } @@ -153,7 +153,7 @@ public boolean equals(Object o) ScalarColumnConstraint other = (ScalarColumnConstraint) o; - return param.equals(other.param) + return columnName.equals(other.columnName) && relationType == other.relationType && term.equals(other.term); } diff --git a/src/java/org/apache/cassandra/cql3/constraints/UnaryFunctionColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/UnaryFunctionColumnConstraint.java new file mode 100644 index 000000000000..4ca3fb35cee6 --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/constraints/UnaryFunctionColumnConstraint.java @@ -0,0 +1,191 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.constraints; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.function.Function; + +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.CqlBuilder; +import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.tcm.serialization.MetadataSerializer; +import org.apache.cassandra.tcm.serialization.Version; +import org.apache.cassandra.utils.LocalizeString; + +import static org.apache.cassandra.cql3.constraints.ColumnConstraint.ConstraintType.UNARY_FUNCTION; + +public class UnaryFunctionColumnConstraint extends AbstractFunctionConstraint<UnaryFunctionColumnConstraint> +{ + public static final Serializer serializer = new Serializer(); + + private final ConstraintFunction function; + + public final static class Raw + { + public final ConstraintFunction function; + public final ColumnIdentifier columnName; + + public Raw(ColumnIdentifier functionName, ColumnIdentifier columnName) + { + this.columnName = columnName; + function = createConstraintFunction(functionName.toCQLString(), columnName); + } + + public UnaryFunctionColumnConstraint prepare() + { + return new UnaryFunctionColumnConstraint(function, columnName); + } + } + + private enum Functions + { + NOT_NULL(NotNullConstraint::new); + + private final Function<ColumnIdentifier, ConstraintFunction> functionCreator; + + Functions(Function<ColumnIdentifier, ConstraintFunction> functionCreator) + { + this.functionCreator = functionCreator; + } + } + + private static ConstraintFunction createConstraintFunction(String functionName, ColumnIdentifier columnName) + { + try + { + return Functions.valueOf(LocalizeString.toUpperCaseLocalized(functionName)).functionCreator.apply(columnName); + } + catch (IllegalArgumentException ex) + { + throw new InvalidConstraintDefinitionException("Unrecognized constraint function: " + functionName); + } + } + + private UnaryFunctionColumnConstraint(ConstraintFunction function, ColumnIdentifier columnName) + { + super(columnName); + this.function = function; + } + + public String name() + { + return function.name; + } + + @Override + public MetadataSerializer<UnaryFunctionColumnConstraint> serializer() + { + return serializer; + } + + @Override + public void appendCqlTo(CqlBuilder builder) + { + builder.append(toString()); + } + + @Override + public void internalEvaluate(AbstractType<?> valueType, ByteBuffer columnValue) throws ConstraintViolationException + { + function.evaluate(valueType, columnValue); + } + + @Override + public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException + { + validateArgs(columnMetadata); + function.validate(columnMetadata); + } + + @Override + public ConstraintType getConstraintType() + { + return UNARY_FUNCTION; + } + + void validateArgs(ColumnMetadata columnMetadata) + { + if (!columnMetadata.name.equals(columnName)) + throw new InvalidConstraintDefinitionException(String.format("Parameter of %s constraint should be the column name (%s)", + name(), + columnMetadata.name)); + } + + @Override + public String toString() + { + return function.name + "(" + columnName + ")"; + } + + public static class Serializer implements MetadataSerializer<UnaryFunctionColumnConstraint> + { + @Override + public void serialize(UnaryFunctionColumnConstraint columnConstraint, DataOutputPlus out, Version version) throws IOException + { + out.writeUTF(columnConstraint.function.name); + out.writeUTF(columnConstraint.columnName.toCQLString()); + } + + @Override + public UnaryFunctionColumnConstraint deserialize(DataInputPlus in, Version version) throws IOException + { + String functionName = in.readUTF(); + ConstraintFunction function; + String columnNameString = in.readUTF(); + ColumnIdentifier columnName = new ColumnIdentifier(columnNameString, true); + try + { + function = createConstraintFunction(functionName, columnName); + } + catch (Exception e) + { + throw new IOException(e); + } + + return new UnaryFunctionColumnConstraint(function, columnName); + } + + @Override + public long serializedSize(UnaryFunctionColumnConstraint columnConstraint, Version version) + { + return TypeSizes.sizeof(columnConstraint.function.getClass().getName()) + + TypeSizes.sizeof(columnConstraint.columnName.toCQLString()); + } + } + + @Override + public boolean equals(Object o) + { + if (this == o) + return true; + + if (!(o instanceof UnaryFunctionColumnConstraint)) + return false; + + UnaryFunctionColumnConstraint other = (UnaryFunctionColumnConstraint) o; + + return function.equals(other.function) + && columnName.equals(other.columnName); + } +} diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java index b528133e437c..7698c0b59c07 100644 --- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java @@ -154,6 +154,29 @@ public ModificationStatement(StatementType type, modifiedColumns = metadata.regularAndStaticColumns(); this.updatedColumns = modifiedColumns; + + if (!this.metadata.notNullColumns.isEmpty()) + { + if (this.type.isInsertOrUpdate()) + { + for (ColumnMetadata notNullColumn : this.metadata.notNullColumns) + { + if (!updatedColumns.contains(notNullColumn)) + throw RequestValidations.invalidRequest(String.format("Column '%s' has to be specified as part of this query.", + notNullColumn.name)); + } + } + else if (this.type.isDelete()) + { + for (ColumnMetadata notNullColumn : this.metadata.notNullColumns) + { + if (updatedColumns.contains(notNullColumn)) + throw RequestValidations.invalidRequest(String.format("Column '%s' can not be set to null.", + notNullColumn.name)); + } + } + } + this.conditionColumns = conditionColumnsBuilder.build(); this.requiresRead = requiresReadBuilder.build(); } diff --git a/src/java/org/apache/cassandra/cql3/statements/StatementType.java b/src/java/org/apache/cassandra/cql3/statements/StatementType.java index f9c0d3d27935..b409eb838322 100644 --- a/src/java/org/apache/cassandra/cql3/statements/StatementType.java +++ b/src/java/org/apache/cassandra/cql3/statements/StatementType.java @@ -87,6 +87,15 @@ public boolean isDelete() return this == DELETE; } + /** + * Checks if this type is an insert or an update. + * @return <code>true</code> if this type is an insert or an update, <code>false</code> otherwise. + */ + public boolean isInsertOrUpdate() + { + return this == INSERT || this == UPDATE; + } + /** * Checks if this type is a select. * @return <code>true</code> if this type is a select, <code>false</code> otherwise. diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java index 27d996134064..08dc3ea918d6 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java @@ -27,7 +27,6 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.TimeUnit; - import javax.annotation.Nullable; import com.google.common.base.Splitter; @@ -42,9 +41,9 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.CQL3Type; import org.apache.cassandra.cql3.CQLStatement; -import org.apache.cassandra.cql3.constraints.ColumnConstraints; import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.QualifiedName; +import org.apache.cassandra.cql3.constraints.ColumnConstraints; import org.apache.cassandra.cql3.functions.masking.ColumnMask; import org.apache.cassandra.db.guardrails.Guardrails; import org.apache.cassandra.db.marshal.AbstractType; @@ -758,6 +757,7 @@ public KeyspaceMetadata apply(Epoch epoch, KeyspaceMetadata keyspace, TableMetad { if (column.name == columnName) { + constraints.validate(column); column.setColumnConstraints(constraints); break; } diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java index 8338b4006eaa..518ae571a653 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java @@ -346,14 +346,16 @@ public TableMetadata.Builder builder(Types types, UserFunctions functions) for (int i = 0; i < partitionKeyColumns.size(); i++) { + ColumnConstraints constraints = validateConstraints(partitionKeyColumns.get(i)); ColumnProperties properties = partitionKeyColumnProperties.get(i); - builder.addPartitionKeyColumn(partitionKeyColumns.get(i), properties.type, properties.mask, columnConstraints.get(partitionKeyColumns.get(i))); + builder.addPartitionKeyColumn(partitionKeyColumns.get(i), properties.type, properties.mask, constraints); } for (int i = 0; i < clusteringColumns.size(); i++) { + ColumnConstraints constraints = validateConstraints(clusteringColumns.get(i)); ColumnProperties properties = clusteringColumnProperties.get(i); - builder.addClusteringColumn(clusteringColumns.get(i), properties.type, properties.mask, columnConstraints.get(clusteringColumns.get(i))); + builder.addClusteringColumn(clusteringColumns.get(i), properties.type, properties.mask, constraints); } if (useCompactStorage) @@ -363,16 +365,24 @@ public TableMetadata.Builder builder(Types types, UserFunctions functions) else { columns.forEach((column, properties) -> { + ColumnConstraints constraints = validateConstraints(column); if (staticColumns.contains(column)) - builder.addStaticColumn(column, properties.type, properties.mask, columnConstraints.get(column)); + builder.addStaticColumn(column, properties.type, properties.mask, constraints); else - builder.addRegularColumn(column, properties.type, properties.mask, columnConstraints.get(column)); + builder.addRegularColumn(column, properties.type, properties.mask, constraints); }); } return builder; } + private ColumnConstraints validateConstraints(ColumnIdentifier columnIdentifier) + { + ColumnConstraints constraints = columnConstraints.get(columnIdentifier); + constraints.checkInvalidConstraintsCombinations(columnIdentifier); + return constraints; + } + private void validateCompactTable(List<ColumnProperties> clusteringColumnProperties, Map<ColumnIdentifier, ColumnProperties> columns) { diff --git a/src/java/org/apache/cassandra/schema/ColumnMetadata.java b/src/java/org/apache/cassandra/schema/ColumnMetadata.java index 79fad3a80e56..ad44732e2b89 100644 --- a/src/java/org/apache/cassandra/schema/ColumnMetadata.java +++ b/src/java/org/apache/cassandra/schema/ColumnMetadata.java @@ -31,6 +31,7 @@ import com.google.common.collect.Lists; import org.apache.cassandra.cql3.*; +import org.apache.cassandra.cql3.constraints.AbstractFunctionConstraint; import org.apache.cassandra.cql3.constraints.ColumnConstraint; import org.apache.cassandra.cql3.constraints.ColumnConstraints; import org.apache.cassandra.cql3.functions.masking.ColumnMask; @@ -319,6 +320,21 @@ public boolean hasConstraint() return columnConstraints.hasRelevantConstraints(); } + public static boolean hasFunctionConstraint(ColumnConstraints columnConstraints, String name) + { + for (ColumnConstraint<?> constraint : columnConstraints.getConstraints()) + { + if (constraint.getConstraintType() == ColumnConstraint.ConstraintType.UNARY_FUNCTION || + constraint.getConstraintType() == ColumnConstraint.ConstraintType.FUNCTION) + { + if (((AbstractFunctionConstraint<?>) constraint).name().equals(name)) + return true; + } + } + + return false; + } + public boolean isRegular() { return kind == Kind.REGULAR; @@ -350,6 +366,7 @@ public ColumnConstraints getColumnConstraints() public void setColumnConstraints(ColumnConstraints constraints) { + constraints.checkInvalidConstraintsCombinations(name); this.columnConstraints = constraints; } diff --git a/src/java/org/apache/cassandra/schema/TableMetadata.java b/src/java/org/apache/cassandra/schema/TableMetadata.java index c2f53592482e..6e23a4bd080f 100644 --- a/src/java/org/apache/cassandra/schema/TableMetadata.java +++ b/src/java/org/apache/cassandra/schema/TableMetadata.java @@ -53,6 +53,7 @@ import org.apache.cassandra.cql3.CqlBuilder; import org.apache.cassandra.cql3.SchemaElement; import org.apache.cassandra.cql3.constraints.InvalidConstraintDefinitionException; +import org.apache.cassandra.cql3.constraints.NotNullConstraint; import org.apache.cassandra.cql3.functions.Function; import org.apache.cassandra.cql3.functions.masking.ColumnMask; import org.apache.cassandra.db.Clustering; @@ -205,6 +206,7 @@ public enum Kind // two different variables. public final List<ColumnConstraint> partitionKeyConstraints; public final List<ColumnMetadata> columnsWithConstraints; + public final List<ColumnMetadata> notNullColumns; protected TableMetadata(Builder builder) { @@ -255,12 +257,20 @@ else if (isIndex()) this.partitionKeyConstraints = pkConstraints; List<ColumnMetadata> columnsWithConstraints = new ArrayList<>(); + List<ColumnMetadata> notNullColumns = new ArrayList<>(); + for (ColumnMetadata column : this.columns()) { - if (column.hasConstraint() && !column.isPartitionKey() && !column.isClusteringColumn()) + if (column.hasConstraint() && !column.isPrimaryKeyColumn()) + { columnsWithConstraints.add(column); + if (ColumnMetadata.hasFunctionConstraint(column.getColumnConstraints(), NotNullConstraint.FUNCTION_NAME)) + notNullColumns.add(column); + + } } this.columnsWithConstraints = columnsWithConstraints; + this.notNullColumns = notNullColumns; } public static Builder builder(String keyspace, String table) diff --git a/test/distributed/org/apache/cassandra/distributed/test/ColumnConstraintsTest.java b/test/distributed/org/apache/cassandra/distributed/test/ColumnConstraintsTest.java index 7d2aeed896fe..cfd3ecb16eae 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/ColumnConstraintsTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/ColumnConstraintsTest.java @@ -22,6 +22,7 @@ import java.util.Map; import java.util.Set; +import org.apache.cassandra.cql3.constraints.ConstraintViolationException; import org.apache.cassandra.exceptions.InvalidRequestException; import org.junit.Test; @@ -283,6 +284,27 @@ public void testLengthTableLevelConstraint() throws IOException } } + @Test + public void testNotNullTableLevelConstraint() throws IOException + { + Set<String> typesSet = Set.of("varchar", "text", "blob", "ascii", "int", "smallint", "decimal", "float", "double"); + + for (String type : typesSet) + { + try (Cluster cluster = init(Cluster.build(1).start())) + { + String tableName = String.format(KEYSPACE + ".%s_tbl1_%s", type, "st"); + String createTableNotNullValue = "CREATE TABLE " + tableName + " (pk int, value int CHECK NOT_NULL(value), PRIMARY KEY (pk));"; + cluster.schemaChange(createTableNotNullValue); + + Assertions.assertThatThrownBy(() -> cluster.coordinator(1).execute(String.format("INSERT INTO " + tableName + " (pk, value) VALUES (1, null)"), ConsistencyLevel.ALL)) + .describedAs("Column value does not satisfy value constraint for column 'value' as it is null.") + .has(new Condition<Throwable>(t -> t.getClass().getCanonicalName().equals(ConstraintViolationException.class.getCanonicalName()), + "Column value does not satisfy value constraint for column 'value' as it is null.")); + } + } + } + private void assertThrowsConstraintViolationException(Cluster cluster, String statement, String description) { Assertions.assertThatThrownBy(() -> cluster.coordinator(1).execute(statement, ConsistencyLevel.ALL)) diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/SnapshotTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/SnapshotTest.java index 775e7ffe7956..628df75ca463 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/SnapshotTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/SnapshotTest.java @@ -52,6 +52,7 @@ public void testSimpleSnapshot() throws Throwable { cluster.schemaChange(withKeyspace("create table %s.tbl (id int primary key, x int)")); cluster.schemaChange(withKeyspace("create table %s.tblconstraints (id int primary key, x int check x > 100 and x < 200, v text check LENGTH(v) > 10)")); + cluster.schemaChange(withKeyspace("create table %s.tblconstraints2 (id int primary key, x int check NOT_NULL(x), v text check LENGTH(v) > 10)")); cluster.schemaChange(withKeyspace("CREATE OR REPLACE FUNCTION %s.fLog (input double) CALLED ON NULL INPUT RETURNS double LANGUAGE java AS 'return Double.valueOf(Math.log(input.doubleValue()));';")); cluster.schemaChange(withKeyspace("CREATE OR REPLACE FUNCTION %s.avgState ( state tuple<int,bigint>, val int ) CALLED ON NULL INPUT RETURNS tuple<int,bigint> LANGUAGE java AS \n" + " 'if (val !=null) { state.setInt(0, state.getInt(0)+1); state.setLong(1, state.getLong(1)+val.intValue()); } return state;'; ")); diff --git a/test/unit/org/apache/cassandra/contraints/AlterTableWithTableConstraintValidationTest.java b/test/unit/org/apache/cassandra/contraints/AlterTableWithTableConstraintValidationTest.java index 0229d87e25dc..584cd955c7dd 100644 --- a/test/unit/org/apache/cassandra/contraints/AlterTableWithTableConstraintValidationTest.java +++ b/test/unit/org/apache/cassandra/contraints/AlterTableWithTableConstraintValidationTest.java @@ -119,13 +119,30 @@ public void testCreateTableAddMultipleMixedConstraints() throws Throwable execute("ALTER TABLE %s ALTER ck2 CHECK LENGTH(ck2) = 4"); tableCreateStatement = "CREATE TABLE " + KEYSPACE + "." + table + " (\n" + - " pk int,\n" + - " ck1 int CHECK ck1 < 100,\n" + - " ck2 text CHECK LENGTH(ck2) = 4,\n" + - " v int,\n" + - " PRIMARY KEY (pk, ck1, ck2)\n" + - ") WITH CLUSTERING ORDER BY (ck1 ASC, ck2 ASC)\n" + - " AND " + tableParametersCql(); + " pk int,\n" + + " ck1 int CHECK ck1 < 100,\n" + + " ck2 text CHECK LENGTH(ck2) = 4,\n" + + " v int,\n" + + " PRIMARY KEY (pk, ck1, ck2)\n" + + ") WITH CLUSTERING ORDER BY (ck1 ASC, ck2 ASC)\n" + + " AND " + tableParametersCql(); + + assertRowsNet(executeDescribeNet(KEYSPACE, "DESCRIBE TABLE " + KEYSPACE + "." + table), + row(KEYSPACE, + "table", + table, + tableCreateStatement)); + + execute("ALTER TABLE %s ALTER v CHECK NOT_NULL(v)"); + + tableCreateStatement = "CREATE TABLE " + KEYSPACE + "." + table + " (\n" + + " pk int,\n" + + " ck1 int CHECK ck1 < 100,\n" + + " ck2 text CHECK LENGTH(ck2) = 4,\n" + + " v int CHECK NOT_NULL(v),\n" + + " PRIMARY KEY (pk, ck1, ck2)\n" + + ") WITH CLUSTERING ORDER BY (ck1 ASC, ck2 ASC)\n" + + " AND " + tableParametersCql(); assertRowsNet(executeDescribeNet(KEYSPACE, "DESCRIBE TABLE " + KEYSPACE + "." + table), row(KEYSPACE, diff --git a/test/unit/org/apache/cassandra/contraints/ColumnConstraintsTest.java b/test/unit/org/apache/cassandra/contraints/ColumnConstraintsTest.java index 45a631cca609..f43974e09eec 100644 --- a/test/unit/org/apache/cassandra/contraints/ColumnConstraintsTest.java +++ b/test/unit/org/apache/cassandra/contraints/ColumnConstraintsTest.java @@ -25,11 +25,12 @@ import static org.apache.cassandra.cql3.constraints.ColumnConstraint.ConstraintType.COMPOSED; import static org.apache.cassandra.cql3.constraints.ColumnConstraint.ConstraintType.FUNCTION; import static org.apache.cassandra.cql3.constraints.ColumnConstraint.ConstraintType.SCALAR; +import static org.apache.cassandra.cql3.constraints.ColumnConstraint.ConstraintType.UNARY_FUNCTION; import static org.junit.Assert.assertEquals; public class ColumnConstraintsTest { - private static final ConstraintType[] EXPECTED_VALUES = { COMPOSED, FUNCTION, SCALAR }; + private static final ConstraintType[] EXPECTED_VALUES = { COMPOSED, FUNCTION, SCALAR, UNARY_FUNCTION }; @Test public void testEnumCodesAndNames() diff --git a/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnCqlConstraintValidationTest.java b/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnCqlConstraintValidationTest.java index a5f380307d57..b92f7b1c4111 100644 --- a/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnCqlConstraintValidationTest.java +++ b/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnCqlConstraintValidationTest.java @@ -114,6 +114,27 @@ public void testCreateTableWithColumnNotNamedConstraintDescribeTableFunction() t tableCreateStatement)); } + @Test + public void testCreateTableWithColumnNotNullConstraintDescribe() throws Throwable + { + String table = createTable(KEYSPACE_PER_TEST, "CREATE TABLE %s (pk int, ck1 int, ck2 int, v int CHECK NOT_NULL(v), PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + + String tableCreateStatement = "CREATE TABLE " + KEYSPACE_PER_TEST + "." + table + " (\n" + + " pk int,\n" + + " ck1 int,\n" + + " ck2 int,\n" + + " v int CHECK NOT_NULL(v),\n" + + " PRIMARY KEY (pk, ck1, ck2)\n" + + ") WITH CLUSTERING ORDER BY (ck1 ASC, ck2 ASC)\n" + + " AND " + tableParametersCql(); + + assertRowsNet(executeDescribeNet("DESCRIBE TABLE " + KEYSPACE_PER_TEST + "." + table), + row(KEYSPACE_PER_TEST, + "table", + table, + tableCreateStatement)); + } + // SCALAR @Test public void testCreateTableWithColumnWithClusteringColumnLessThanScalarConstraintInteger() throws Throwable @@ -865,6 +886,46 @@ public void testCreateTableWithColumnWithClusteringColumnMultipleScalarFloatCons }); } + @Test + public void testCreateTableWithColumnWithNotNullCheckScalarIntConstraints() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v int CHECK v < 4 AND v >= 2, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v' as it is null."; + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, null)"); + } + + @Test + public void testCreateTableWithColumnWithNotNullCheckScalarSmallintConstraints() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v smallint CHECK v < 4 AND v >= 2, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v' as it is null."; + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, null)"); + } + + @Test + public void testCreateTableWithColumnWithNotNullCheckScalarDecimalConstraints() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v decimal CHECK v < 4 AND v >= 2, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v' as it is null."; + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, null)"); + } + + @Test + public void testCreateTableWithColumnWithNotNullCheckScalarDoubleConstraints() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v double CHECK v < 4 AND v >= 2, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v' as it is null."; + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, null)"); + } + + @Test + public void testCreateTableWithColumnWithNotNullCheckScalarFloatConstraints() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v float CHECK v < 4 AND v >= 2, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v' as it is null."; + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, null)"); + } + // FUNCTION @Test public void testCreateTableWithColumnWithClusteringColumnLengthEqualToConstraint() throws Throwable @@ -1204,6 +1265,38 @@ public void testCreateTableWithColumnWithRegularColumnLengthSmallerOrEqualThanCo assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'foooo')"); } + @Test + public void testCreateTableWithColumnWithRegularColumnLengthCheckNullTextConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v text CHECK LENGTH(v) <= 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v' as it is null."; + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, null)"); + } + + @Test + public void testCreateTableWithColumnWithRegularColumnLengthCheckNullVarcharConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v varchar CHECK LENGTH(v) <= 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v' as it is null."; + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, null)"); + } + + @Test + public void testCreateTableWithColumnWithRegularColumnLengthCheckNullAsciiConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v ascii CHECK LENGTH(v) <= 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v' as it is null."; + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, null)"); + } + + @Test + public void testCreateTableWithColumnWithRegularColumnLengthCheckNullBlobConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v blob CHECK LENGTH(v) <= 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v' as it is null."; + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, null)"); + } + @Test public void testCreateTableWithColumnMixedColumnsLengthConstraint() throws Throwable { @@ -1279,7 +1372,7 @@ public void testCreateTableWithColumnWithClusteringColumnInvalidScalarTypeConstr catch (InvalidRequestException e) { assertTrue(e.getCause() instanceof InvalidRequestException); - assertTrue(e.getCause().getMessage().equals("pk is not a number")); + assertTrue(e.getCause().getMessage().equals("Column 'pk' is not a number type.")); assertTrue(e.getMessage().contains("Error setting schema for test")); } } diff --git a/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnNotNullConstraintInvalidTest.java b/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnNotNullConstraintInvalidTest.java new file mode 100644 index 000000000000..73664f848b4a --- /dev/null +++ b/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnNotNullConstraintInvalidTest.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.contraints; + +import java.util.Arrays; +import java.util.Collection; +import java.util.stream.Collectors; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.cassandra.cql3.CQL3Type; +import org.apache.cassandra.cql3.constraints.InvalidConstraintDefinitionException; +import org.apache.cassandra.db.marshal.DurationType; +import org.apache.cassandra.db.marshal.EmptyType; +import org.apache.cassandra.exceptions.InvalidRequestException; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; + + +@RunWith(Parameterized.class) +public class CreateTableWithColumnNotNullConstraintInvalidTest extends CqlConstraintValidationTester +{ + + @Parameterized.Parameter + public String typeString; + + + @Parameterized.Parameters() + public static Collection<Object[]> data() + { + return Arrays.stream(CQL3Type.Native.values()) + .filter(t -> !t.getType().isCounter() + && !(t.getType().unwrap() instanceof EmptyType) + && !(t.getType().unwrap() instanceof DurationType)) + .map(Object::toString) + .distinct() + .map(t -> new Object[]{ t }) + .collect(Collectors.toList()); + } + + @Test + public void testCreateTableWithColumnNotNullCheckNonExisting() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 " + typeString + " CHECK NOT_NULL(ck1), ck2 int, v int, PRIMARY KEY (pk));"); + + // Invalid + assertInvalidThrowMessage("Column 'ck1' has to be specified as part of this query.", InvalidRequestException.class, "INSERT INTO %s (pk, ck2, v) VALUES (1, 2, 3)"); + + assertInvalidThrowMessage("Column value does not satisfy value constraint for column 'ck1' as it is null.", InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, null, 2, 3)"); + assertInvalidThrowMessage("Column 'ck1' can not be set to null.", InvalidRequestException.class, "DELETE ck1 FROM %s WHERE pk = 1"); + } + + @Test + public void testInvalidSpecificationOfNotNullConstraintOnPrimaryKeys() throws Throwable + { + assertThatThrownBy(() -> createTable("CREATE TABLE %s (pk " + typeString + " CHECK NOT_NULL(pk) PRIMARY KEY)")) + .isInstanceOf(InvalidRequestException.class) + .hasRootCauseInstanceOf(InvalidConstraintDefinitionException.class) + .hasRootCauseMessage("NOT_NULL constraint can not be specified on a partition key column 'pk'"); + + assertThatThrownBy(() -> createTable("CREATE TABLE %s (pk int, cl " + typeString + " CHECK NOT_NULL(cl), PRIMARY KEY (pk, cl))")) + .isInstanceOf(InvalidRequestException.class) + .hasRootCauseInstanceOf(InvalidConstraintDefinitionException.class) + .hasRootCauseMessage("NOT_NULL constraint can not be specified on a clustering key column 'cl'"); + } +} diff --git a/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnNotNullConstraintValidTest.java b/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnNotNullConstraintValidTest.java new file mode 100644 index 000000000000..5513a00251c4 --- /dev/null +++ b/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnNotNullConstraintValidTest.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.contraints; + +import java.util.Arrays; +import java.util.Collection; +import java.util.stream.Collectors; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.cassandra.cql3.CQL3Type; +import org.apache.cassandra.db.marshal.StringType; +import org.apache.cassandra.exceptions.InvalidRequestException; + + +@RunWith(Parameterized.class) +public class CreateTableWithColumnNotNullConstraintValidTest extends CqlConstraintValidationTester +{ + @Parameterized.Parameter(0) + public String typeString; + + @Parameterized.Parameter(1) + public Object value; + + @Parameterized.Parameters(name = "{index}: typeString={0} value={1}") + public static Collection<Object[]> data() + { + return Arrays.stream(CQL3Type.Native.values()) + .filter(t -> !t.getType().isCounter() && (t.getType().isNumber() || t.getType().unwrap() instanceof StringType)) + .map(t -> { + if (t.getType().isNumber()) + return new Object[]{ t.toString(), 123}; + return new Object[]{ t.toString(), "'fooo'"}; + }).collect(Collectors.toList()); + } + + @Test + public void testCreateTableWithColumnNotNullCheckValid() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 " + typeString + " CHECK NOT_NULL(ck1), ck2 int, v int, PRIMARY KEY (pk));"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, " + value + ", 2, 3)"); + + // Invalid + assertInvalidThrowMessage("Column 'ck1' has to be specified as part of this query.", InvalidRequestException.class, "INSERT INTO %s (pk, ck2, v) VALUES (1, 2, 3)"); + } +} From f2c6ee24ed1347519143dd94bb017ca4b38c2429 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Fri, 14 Feb 2025 19:33:39 +0100 Subject: [PATCH 136/225] Do not leak non-Java exceptions when calling snapshot operations via JMX patch by Stefan Miklosovic; reviewed by Francisco Guerrero for CASSANDRA-20335 --- CHANGES.txt | 1 + .../service/snapshot/SnapshotException.java | 32 ------------------- .../service/snapshot/SnapshotManager.java | 6 ++-- .../distributed/test/SnapshotsTest.java | 2 +- 4 files changed, 5 insertions(+), 36 deletions(-) delete mode 100644 src/java/org/apache/cassandra/service/snapshot/SnapshotException.java diff --git a/CHANGES.txt b/CHANGES.txt index e058d9745721..c0b984e5ca67 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Do not leak non-Java exceptions when calling snapshot operations via JMX (CASSANDRA-20335) * Implement NOT_NULL constraint (CASSANDRA-20276) * Improve error messages for constraints (CASSANDRA-20266) * Add system_views.partition_key_statistics for querying SSTable metadata (CASSANDRA-20161) diff --git a/src/java/org/apache/cassandra/service/snapshot/SnapshotException.java b/src/java/org/apache/cassandra/service/snapshot/SnapshotException.java deleted file mode 100644 index 39cdf7ef77dd..000000000000 --- a/src/java/org/apache/cassandra/service/snapshot/SnapshotException.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.service.snapshot; - -public class SnapshotException extends RuntimeException -{ - public SnapshotException(String message) - { - super(message); - } - - public SnapshotException(String message, Throwable cause) - { - super(message, cause); - } -} diff --git a/src/java/org/apache/cassandra/service/snapshot/SnapshotManager.java b/src/java/org/apache/cassandra/service/snapshot/SnapshotManager.java index f7fa1e4ececa..b8b81915ceca 100644 --- a/src/java/org/apache/cassandra/service/snapshot/SnapshotManager.java +++ b/src/java/org/apache/cassandra/service/snapshot/SnapshotManager.java @@ -434,7 +434,7 @@ public void takeSnapshot(String tag, Map<String, String> optMap, String... entit { takeSnapshot(SnapshotOptions.userSnapshot(tag, optMap, entities)); } - catch (SnapshotException ex) + catch (Throwable ex) { // to be compatible with deprecated methods in StorageService throw new IOException(ex); @@ -536,7 +536,7 @@ List<TableSnapshot> executeTask(TakeSnapshotTask task) } catch (Throwable t) { - throw new SnapshotException(String.format("Exception occured while executing %s: %s", task.toString(), t.getMessage()), t); + throw new RuntimeException(String.format("Exception occured while executing %s: %s", task.toString(), t.getMessage()), t); } } @@ -549,7 +549,7 @@ <T> T executeTask(AbstractSnapshotTask<T> task) } catch (Throwable t) { - throw new SnapshotException(String.format("Exception occured while executing %s", task.toString()), t); + throw new RuntimeException(String.format("Exception occured while executing %s", task.toString()), t); } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java b/test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java index d0b78112a8d5..7227d6613f85 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java @@ -297,7 +297,7 @@ public void testSnapshotInvalidArgument() throws Exception instance.nodetoolResult("snapshot", "--ttl", format("%ds", 1), "-t", "basic") .asserts() .failure() - .stdoutContains(format("ttl for snapshot must be at least %d seconds", FIVE_SECONDS)); + .stderrContains(format("ttl for snapshot must be at least %d seconds", FIVE_SECONDS)); instance.nodetoolResult("snapshot", "--ttl", "invalid-ttl").asserts().failure(); } From ebbdd8bb81cff1f120ae86a59a3934f26c30b6b7 Mon Sep 17 00:00:00 2001 From: Marcus Eriksson <marcuse@apache.org> Date: Wed, 15 Jan 2025 14:15:47 +0100 Subject: [PATCH 137/225] Make it possible to abort all kinds of multi step operations Patch by marcuse; reviewed by Sam Tunnicliffe for CASSANDRA-20217 --- CHANGES.txt | 1 + .../cassandra/service/StorageService.java | 14 ++++- .../service/StorageServiceMBean.java | 5 +- .../tcm/sequences/SingleNodeSequences.java | 57 +++++++++++++++---- .../org/apache/cassandra/tools/NodeProbe.java | 14 ++++- .../org/apache/cassandra/tools/NodeTool.java | 3 + .../tools/nodetool/Decommission.java | 14 ++++- .../apache/cassandra/tools/nodetool/Move.java | 24 ++++---- .../cassandra/tools/nodetool/RemoveNode.java | 20 +++++-- .../distributed/test/FailingMoveTest.java | 44 +++++++++++++- .../distributed/test/RemoveNodeTest.java | 2 +- .../test/ring/DecommissionTest.java | 56 ++++++++++++++++++ 12 files changed, 216 insertions(+), 38 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index c0b984e5ca67..4202555d1103 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Make it possible to abort all kinds of multi step operations (CASSANDRA-20217) * Do not leak non-Java exceptions when calling snapshot operations via JMX (CASSANDRA-20335) * Implement NOT_NULL constraint (CASSANDRA-20276) * Improve error messages for constraints (CASSANDRA-20266) diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 5e16bae9445c..252f451d85ea 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -3381,6 +3381,11 @@ public void decommission(boolean force) SingleNodeSequences.decommission(true, force); } + public void abortDecommission(String nodeId) + { + SingleNodeSequences.abortDecommission(nodeId); + } + public void shutdownNetworking() { shutdownClientServers(); @@ -3461,9 +3466,9 @@ public void resumeMove() } @Override - public void abortMove() + public void abortMove(String nodeId) { - SingleNodeSequences.abortMove(); + SingleNodeSequences.abortMove(nodeId); } public String getRemovalStatus() @@ -3533,6 +3538,11 @@ public void removeNode(String hostIdString, boolean force) SingleNodeSequences.removeNode(toRemove, force); } + public void abortRemoveNode(String nodeId) + { + SingleNodeSequences.abortRemoveNode(nodeId); + } + public void assassinateEndpoint(String address) { try diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java index f2b75195c1f2..57dfcea67338 100644 --- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java +++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java @@ -523,7 +523,7 @@ default int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, * @param force Decommission even if this will reduce N to be less than RF. */ public void decommission(boolean force) throws InterruptedException; - + public void abortDecommission(String nodeId); /** * Returns whether a node has failed to decommission. * @@ -547,7 +547,7 @@ default int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, */ public void move(String newToken) throws IOException; public void resumeMove(); - public void abortMove(); + public void abortMove(String nodeId); /** * removeToken removes token (and all data associated with @@ -555,6 +555,7 @@ default int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, */ public void removeNode(String token); public void removeNode(String token, boolean force); + public void abortRemoveNode(String nodeId); public void assassinateEndpoint(String addr); diff --git a/src/java/org/apache/cassandra/tcm/sequences/SingleNodeSequences.java b/src/java/org/apache/cassandra/tcm/sequences/SingleNodeSequences.java index 7813fb14920b..58c5f024f9c0 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/SingleNodeSequences.java +++ b/src/java/org/apache/cassandra/tcm/sequences/SingleNodeSequences.java @@ -21,6 +21,8 @@ import java.util.Collections; import java.util.EnumSet; +import javax.annotation.Nullable; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,6 +40,7 @@ import org.apache.cassandra.tcm.transformations.PrepareMove; import static org.apache.cassandra.service.StorageService.Mode.LEAVING; +import static org.apache.cassandra.service.StorageService.Mode.MOVE_FAILED; import static org.apache.cassandra.service.StorageService.Mode.NORMAL; import static org.apache.cassandra.service.StorageService.Mode.DECOMMISSION_FAILED; import static org.apache.cassandra.utils.FBUtilities.getBroadcastAddressAndPort; @@ -96,6 +99,11 @@ else if (InProgressSequences.isLeave(inProgress)) StorageService.instance.shutdownNetworking(); } + static void abortDecommission(String nodeId) + { + abortHelper(nodeId, MultiStepOperation.Kind.LEAVE, DECOMMISSION_FAILED); + } + /** * Entrypoint to begin node removal process * @@ -134,6 +142,11 @@ static void removeNode(NodeId toRemove, boolean force) InProgressSequences.finishInProgressSequences(toRemove); } + static void abortRemoveNode(String nodeId) + { + abortHelper(nodeId, MultiStepOperation.Kind.REMOVE, null); + } + /** * move the node to new token or find a new token to boot to according to load * @@ -184,7 +197,7 @@ static void resumeMove() logger.info(msg); throw new IllegalStateException(msg); } - if (StorageService.instance.operationMode() != StorageService.Mode.MOVE_FAILED) + if (StorageService.instance.operationMode() != MOVE_FAILED) { String msg = "Can't resume a move operation unless it has failed"; logger.info(msg); @@ -194,28 +207,48 @@ static void resumeMove() InProgressSequences.finishInProgressSequences(self); } - static void abortMove() + static void abortMove(String nodeId) + { + abortHelper(nodeId, MultiStepOperation.Kind.MOVE, MOVE_FAILED); + } + + /** + * + * @param nodeId node id to abort the MSO for, null for local node + * @param kind the expected kind of the multi step operation to abolt + * @param ssMode the legacy mode we want storage service to be in, null for any + */ + private static void abortHelper(@Nullable String nodeId, MultiStepOperation.Kind kind, @Nullable StorageService.Mode ssMode) { if (ClusterMetadataService.instance().isMigrating() || ClusterMetadataService.state() == ClusterMetadataService.State.GOSSIP) - throw new IllegalStateException("This cluster is migrating to cluster metadata, can't move until that is done."); + throw new IllegalStateException(String.format("This cluster is migrating to cluster metadata, can't abort %s until that is done.", kind)); ClusterMetadata metadata = ClusterMetadata.current(); - NodeId self = metadata.myNodeId(); - MultiStepOperation<?> sequence = metadata.inProgressSequences.get(self); - if (sequence == null || sequence.kind() != MultiStepOperation.Kind.MOVE) + NodeId toAbort = nodeId == null ? metadata.myNodeId() : NodeId.fromString(nodeId); + MultiStepOperation<?> sequence = metadata.inProgressSequences.get(toAbort); + if (sequence == null || sequence.kind() != kind) { - String msg = "No move operation in progress, can't abort"; + String msg = String.format("No %s operation in progress for %s, can't abort (%s)", kind, toAbort, sequence); logger.info(msg); throw new IllegalStateException(msg); } - if (StorageService.instance.operationMode() != StorageService.Mode.MOVE_FAILED) + if (toAbort.equals(metadata.myNodeId())) + { + if (ssMode != null && StorageService.instance.operationMode() != ssMode) + { + String msg = String.format("Can't abort a %s operation unless it has failed", kind); + logger.info(msg); + throw new IllegalStateException(msg); + } + StorageService.instance.clearTransientMode(); + } + else if (Gossiper.instance.isAlive(metadata.directory.endpoint(toAbort))) { - String msg = "Can't abort a move operation unless it has failed"; + String msg = String.format("Can't abort a %s operation for a node %s (%s) that is UP - run abortdecommission on that instance", + kind, toAbort, metadata.directory.endpoint(toAbort)); logger.info(msg); throw new IllegalStateException(msg); } - StorageService.instance.clearTransientMode(); - ClusterMetadataService.instance().commit(new CancelInProgressSequence(self)); + ClusterMetadataService.instance().commit(new CancelInProgressSequence(toAbort)); } - } diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index f7ac9eff3bf5..5a6e8b359821 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -1026,6 +1026,11 @@ public void decommission(boolean force) throws InterruptedException ssProxy.decommission(force); } + public void abortDecommission(String nodeId) + { + ssProxy.abortDecommission(nodeId); + } + public void move(String newToken) throws IOException { ssProxy.move(newToken); @@ -1036,9 +1041,9 @@ public void resumeMove() ssProxy.resumeMove(); } - public void abortMove() + public void abortMove(String nodeId) { - ssProxy.abortMove(); + ssProxy.abortMove(nodeId); } public void removeNode(String token) @@ -1051,6 +1056,11 @@ public void removeNode(String token, boolean force) ssProxy.removeNode(token, force); } + public void abortRemoveNode(String nodeId) + { + ssProxy.abortRemoveNode(nodeId); + } + public String getRemovalStatus(boolean withPort) { return withPort ? ssProxy.getRemovalStatusWithPort() : ssProxy.getRemovalStatus(); diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java index bd1e302ba066..49be441b89a6 100644 --- a/src/java/org/apache/cassandra/tools/NodeTool.java +++ b/src/java/org/apache/cassandra/tools/NodeTool.java @@ -106,6 +106,7 @@ public int execute(String... args) CompactionStats.class, DataPaths.class, Decommission.class, + Decommission.Abort.class, DescribeCluster.class, DescribeRing.class, DisableAuditLog.class, @@ -172,6 +173,7 @@ public int execute(String... args) ListPendingHints.class, ListSnapshots.class, Move.class, + Move.Abort.class, NetStats.class, PauseHandoff.class, ProfileLoad.class, @@ -188,6 +190,7 @@ public int execute(String... args) ReloadTriggers.class, RelocateSSTables.class, RemoveNode.class, + RemoveNode.Abort.class, Repair.class, ReplayBatchlog.class, ResetFullQueryLog.class, diff --git a/src/java/org/apache/cassandra/tools/nodetool/Decommission.java b/src/java/org/apache/cassandra/tools/nodetool/Decommission.java index de70932c5316..2c326c9b29e4 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/Decommission.java +++ b/src/java/org/apache/cassandra/tools/nodetool/Decommission.java @@ -26,7 +26,6 @@ @Command(name = "decommission", description = "Decommission the *node I am connecting to*") public class Decommission extends NodeToolCmd { - @Option(title = "force", name = {"-f", "--force"}, description = "Force decommission of this node even when it reduces the number of replicas to below configured RF") @@ -56,4 +55,17 @@ public void execute(NodeProbe probe) throw new IllegalStateException("Unsupported operation: " + e.getMessage(), e); } } + + @Command(name = "abortdecommission", description = "Abort an ongoing, failed decommission") + public static class Abort extends NodeToolCmd + { + @Option(title = "node id", name = "--node") + private String nodeId; + + @Override + protected void execute(NodeProbe probe) + { + probe.abortDecommission(nodeId); + } + } } diff --git a/src/java/org/apache/cassandra/tools/nodetool/Move.java b/src/java/org/apache/cassandra/tools/nodetool/Move.java index 87c085b86bc8..ef05bf89b6b5 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/Move.java +++ b/src/java/org/apache/cassandra/tools/nodetool/Move.java @@ -36,9 +36,6 @@ public class Move extends NodeToolCmd @Option(title = "Resume an ongoing move operation", name = "--resume") private boolean resume; - @Option(title = "Abort an ongoing move operation", name = "--abort") - private boolean abort; - @Override public void execute(NodeProbe probe) { @@ -46,20 +43,12 @@ public void execute(NodeProbe probe) { if (!newToken.isEmpty()) { - if (resume || abort) - throw new IllegalArgumentException("Can't give both a token and --resume/--abort"); - probe.move(newToken); } else { - if (abort && resume) - throw new IllegalArgumentException("Can't both resume and abort"); - if (resume) probe.resumeMove(); - else if (abort) - probe.abortMove(); else throw new IllegalArgumentException("Need to give either a token for a new move operation, or --resume/--abort for an existing one"); } @@ -68,4 +57,17 @@ else if (abort) throw new RuntimeException("Error during moving node", e); } } + + @Command(name = "abortmove", description = "Abort a failed move operation for this or a remote node") + public static class Abort extends NodeToolCmd + { + @Option(title = "node id", name = "--node") + private String nodeId; + + @Override + public void execute(NodeProbe probe) + { + probe.abortMove(nodeId); + } + } } diff --git a/src/java/org/apache/cassandra/tools/nodetool/RemoveNode.java b/src/java/org/apache/cassandra/tools/nodetool/RemoveNode.java index 89912ec6400f..94b326205851 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/RemoveNode.java +++ b/src/java/org/apache/cassandra/tools/nodetool/RemoveNode.java @@ -22,13 +22,14 @@ import io.airlift.airline.Arguments; import io.airlift.airline.Command; +import io.airlift.airline.Option; import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.tools.NodeTool.NodeToolCmd; @Command(name = "removenode", description = "Show status of current node removal, abort removal or remove provided ID") public class RemoveNode extends NodeToolCmd { - @Arguments(title = "remove_operation", usage = "<status>|<abort> <ID>|<ID>|<ID> --force", description = "Show status of current node removal, abort removal, or remove provided ID", required = true) + @Arguments(title = "remove_operation", usage = "<status>|<ID>|<ID> --force", description = "Show status of current node removal, or remove provided ID", required = true) private List<String> removeOperation = null; @Override @@ -41,15 +42,22 @@ public void execute(NodeProbe probe) break; case "force": throw new IllegalArgumentException("Can't force a nodetool removenode. Instead abort the ongoing removenode and retry."); - case "abort": - if (removeOperation.size() < 2) - probe.output().err.print("Abort requires the node id to abort the removal for."); - probe.getCMSOperationsProxy().cancelInProgressSequences(removeOperation.get(1), "REMOVE"); - break; default: boolean force = removeOperation.size() > 1 && removeOperation.get(1).equals("--force"); probe.removeNode(removeOperation.get(0), force); break; } } + + @Command(name = "abortremovenode", description = "Abort a removenode command") + public static class Abort extends NodeToolCmd + { + @Option(title = "node id", name="--node", description = "The node being removed", required = true) + private String nodeId; + + public void execute(NodeProbe probe) + { + probe.abortRemoveNode(nodeId); + } + } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/FailingMoveTest.java b/test/distributed/org/apache/cassandra/distributed/test/FailingMoveTest.java index ac60f90cd399..65e0c28c3a8e 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/FailingMoveTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/FailingMoveTest.java @@ -20,8 +20,11 @@ import java.io.IOException; import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import com.google.common.util.concurrent.Uninterruptibles; import org.junit.Test; import net.bytebuddy.ByteBuddy; @@ -32,6 +35,7 @@ import org.apache.cassandra.distributed.api.ConsistencyLevel; import org.apache.cassandra.distributed.api.Feature; import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.streaming.StreamPlan; import org.apache.cassandra.streaming.StreamResultFuture; @@ -41,6 +45,7 @@ import static net.bytebuddy.matcher.ElementMatchers.named; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; public class FailingMoveTest extends TestBaseImpl { @@ -94,12 +99,49 @@ public void testAbortMove() throws IOException BB.shouldFail.set(false); }); - cluster.get(3).nodetoolResult("move", "--abort").asserts().success(); + cluster.get(3).nodetoolResult("abortmove").asserts().success(); cluster.get(3).runOnInstance(() -> assertEquals(StorageService.Mode.NORMAL, StorageService.instance.operationMode())); assertNotEquals(moveToToken, getToken(cluster.get(3))); } } + @Test + public void testAbortMoveRemote() throws IOException, ExecutionException, InterruptedException + { + try (Cluster cluster = init(Cluster.build(3) + .withoutVNodes() + .withConfig(c -> c.with(Feature.GOSSIP, Feature.NETWORK)) + .withInstanceInitializer(BB::install) + .start())) + { + cluster.schemaChange(withKeyspace("CREATE TABLE %s.tbl(id int primary key);")); + for (int i=0; i<30; i++) + cluster.coordinator(1).execute(withKeyspace("INSERT INTO %s.tbl (id) VALUES (?)"), + ConsistencyLevel.ALL, i); + String oldToken = getToken(cluster.get(3)); + String moveToToken = "2305843009213693949"; + assertNotEquals(oldToken, moveToToken); + cluster.get(3).nodetoolResult("move", moveToToken).asserts().failure(); + int nodeId = cluster.get(3).callOnInstance(() -> { + assertEquals(StorageService.Mode.MOVE_FAILED, StorageService.instance.operationMode()); + BB.shouldFail.set(false); + return ClusterMetadata.current().myNodeId().id(); + }); + cluster.get(3).shutdown().get(); + cluster.get(2).runOnInstance(() -> { + while (Gossiper.instance.isAlive(ClusterMetadata.current().directory.endpoint(new NodeId(nodeId)))) + Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS); + }); + cluster.get(2).nodetoolResult("abortmove", "--node", String.valueOf(nodeId)).asserts().success(); + cluster.get(3).startup(); + assertNotEquals(moveToToken, getToken(cluster.get(3))); + cluster.get(3).runOnInstance(() -> { + assertEquals(StorageService.Mode.NORMAL, StorageService.instance.operationMode()); + assertTrue(ClusterMetadata.current().inProgressSequences.isEmpty()); + }); + } + } + private String getToken(IInvokableInstance instance) { return instance.callsOnInstance(() -> { diff --git a/test/distributed/org/apache/cassandra/distributed/test/RemoveNodeTest.java b/test/distributed/org/apache/cassandra/distributed/test/RemoveNodeTest.java index 2710fdc83204..cb1514b2c874 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/RemoveNodeTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/RemoveNodeTest.java @@ -79,7 +79,7 @@ public void testAbort() throws Exception // Now abort the removal. This should succeed in committing a cancellation of the removal sequence before // it can be completed, as non-CMS instance is still paused. - cmsInstance.nodetoolResult("removenode", "abort", nodeId).asserts().success(); + cmsInstance.nodetoolResult("abortremovenode", "--node", nodeId).asserts().success(); // Resume processing on the non-CMS instance. It will enact the MID_LEAVE step followed by the cancellation // of the removal process. diff --git a/test/distributed/org/apache/cassandra/distributed/test/ring/DecommissionTest.java b/test/distributed/org/apache/cassandra/distributed/test/ring/DecommissionTest.java index 9826d013d52e..6f67dac03bd8 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/ring/DecommissionTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/ring/DecommissionTest.java @@ -22,9 +22,12 @@ import java.util.HashSet; import java.util.Set; import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import javax.annotation.Nullable; +import com.google.common.util.concurrent.Uninterruptibles; import org.junit.Test; import net.bytebuddy.ByteBuddy; @@ -37,9 +40,12 @@ import org.apache.cassandra.distributed.api.NodeToolResult; import org.apache.cassandra.distributed.shared.ClusterUtils; import org.apache.cassandra.distributed.test.TestBaseImpl; +import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.service.StorageService; import org.apache.cassandra.streaming.StreamSession; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; +import org.apache.cassandra.tcm.membership.NodeId; import org.apache.cassandra.tcm.membership.NodeVersion; import org.apache.cassandra.tcm.serialization.Version; import org.apache.cassandra.tcm.transformations.Startup; @@ -94,6 +100,56 @@ public static void startStreamingFiles(@Nullable StreamSession.PrepareDirection } } + @Test + public void testAbortDecom() throws IOException + { + try (Cluster cluster = builder().withNodes(3) + .withConfig(config -> config.with(NETWORK, GOSSIP)) + .withInstanceInitializer(BB::install) + .start()) + { + populate(cluster, 0, 100, 1, 2, ConsistencyLevel.QUORUM); + cluster.get(2).nodetoolResult("decommission", "--force").asserts().failure(); + cluster.get(2).nodetoolResult("abortdecommission").asserts().success(); + cluster.get(2).runOnInstance(() -> { + assertEquals(StorageService.Mode.NORMAL, StorageService.instance.operationMode()); + assertTrue(ClusterMetadata.current().inProgressSequences.isEmpty()); + }); + cluster.get(2).nodetoolResult("decommission", "--force").asserts().success(); + } + } + + @Test + public void testAbortDecomRemote() throws IOException, ExecutionException, InterruptedException + { + try (Cluster cluster = builder().withNodes(3) + .withConfig(config -> config.with(NETWORK, GOSSIP)) + .withInstanceInitializer(BB::install) + .start()) + { + populate(cluster, 0, 100, 1, 2, ConsistencyLevel.QUORUM); + int nodeId = cluster.get(2).callOnInstance(() -> { + return ClusterMetadata.current().myNodeId().id(); + }); + cluster.get(2).nodetoolResult("decommission", "--force").asserts().failure(); + cluster.get(2).shutdown().get(); + cluster.get(3).runOnInstance(() -> { + while (Gossiper.instance.isAlive(ClusterMetadata.current().directory.endpoint(new NodeId(nodeId)))) + Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS); + }); + cluster.get(3).nodetoolResult("abortdecommission", "--node", String.valueOf(nodeId)).asserts().success(); + cluster.get(2).startup(); + cluster.get(2).runOnInstance(() -> { + assertEquals(StorageService.Mode.NORMAL, StorageService.instance.operationMode()); + assertTrue(ClusterMetadata.current().inProgressSequences.isEmpty()); + }); + cluster.get(2).runOnInstance(() -> { + BB.first.set(true); + }); + cluster.get(2).nodetoolResult("decommission", "--force").asserts().success(); + } + } + @Test public void testDecomDirectoryMinMaxVersions() throws IOException { try (Cluster cluster = builder() From 95180bab15668c29f82ab87845d74fa1d7f6d48d Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi <bernardo.botella@apple.com> Date: Fri, 7 Jun 2024 09:39:52 -0700 Subject: [PATCH 138/225] Adds new size Guardrails Patch by Bernardo Botella Corbi; Reviewed by Jordan West, Yifan Cai for CASSANDRA-19677 --- CHANGES.txt | 1 + conf/cassandra.yaml | 72 +++++ .../org/apache/cassandra/config/Config.java | 14 + .../cassandra/config/GuardrailsOptions.java | 169 +++++++++++ .../cassandra/cql3/UpdateParameters.java | 21 ++ .../apache/cassandra/cql3/terms/Lists.java | 4 +- .../org/apache/cassandra/cql3/terms/Maps.java | 4 +- .../org/apache/cassandra/cql3/terms/Sets.java | 4 +- .../db/guardrails/FallbackThreshold.java | 72 +++++ .../cassandra/db/guardrails/Guardrails.java | 200 ++++++++++++- .../db/guardrails/GuardrailsConfig.java | 75 +++++ .../db/guardrails/GuardrailsMBean.java | 150 ++++++++++ ...olumnTypeSpecificValueThresholdTester.java | 263 ++++++++++++++++++ .../GuardrailCollectionListSizeTest.java | 96 +++++++ .../GuardrailCollectionMapSizeTest.java | 121 ++++++++ .../GuardrailCollectionSetSizeTest.java | 96 +++++++ ...drailCollectionTypeSpecificSizeTester.java | 90 ++++++ .../GuardrailColumnAsciiValueSizeTest.java | 61 ++++ .../GuardrailColumnBlobValueSizeTest.java | 180 ++++++++++++ ...railColumnTextAndVarcharValueSizeTest.java | 62 +++++ 20 files changed, 1748 insertions(+), 7 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/guardrails/FallbackThreshold.java create mode 100644 test/unit/org/apache/cassandra/db/guardrails/ColumnTypeSpecificValueThresholdTester.java create mode 100644 test/unit/org/apache/cassandra/db/guardrails/GuardrailCollectionListSizeTest.java create mode 100644 test/unit/org/apache/cassandra/db/guardrails/GuardrailCollectionMapSizeTest.java create mode 100644 test/unit/org/apache/cassandra/db/guardrails/GuardrailCollectionSetSizeTest.java create mode 100644 test/unit/org/apache/cassandra/db/guardrails/GuardrailCollectionTypeSpecificSizeTester.java create mode 100644 test/unit/org/apache/cassandra/db/guardrails/GuardrailColumnAsciiValueSizeTest.java create mode 100644 test/unit/org/apache/cassandra/db/guardrails/GuardrailColumnBlobValueSizeTest.java create mode 100644 test/unit/org/apache/cassandra/db/guardrails/GuardrailColumnTextAndVarcharValueSizeTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 4202555d1103..778cb533ac02 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Add per type max size guardrails (CASSANDRA-19677) * Make it possible to abort all kinds of multi step operations (CASSANDRA-20217) * Do not leak non-Java exceptions when calling snapshot operations via JMX (CASSANDRA-20335) * Implement NOT_NULL constraint (CASSANDRA-20276) diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index 58f8945d653e..52223a12625c 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -2314,6 +2314,39 @@ drop_compact_storage_enabled: false # column_value_size_warn_threshold: # column_value_size_fail_threshold: # +# Guardrail to warn or fail when writing ascii 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 +# is probably lesser than the thresholds defined here. +# If this guardrail is enabled along with column_value_size_warn_threshold and column_value_size_fail_threshold, size +# restriction for the column type will be the smallest of the two. +# The two thresholds default to null to disable. +# Min unit: B +# column_ascii_value_size_warn_threshold: +# column_ascii_value_size_fail_threshold: +# +# Guardrail to warn or fail when writing blob 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 +# is probably lesser than the thresholds defined here. +# If this guardrail is enabled along with column_value_size_warn_threshold and column_value_size_fail_threshold, size +# restriction for the column type will be the smallest of the two. +# The two thresholds default to null to disable. +# Min unit: B +# column_blob_value_size_warn_threshold: +# column_blob_value_size_fail_threshold: +# +# Guardrail to warn or fail when writing text 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 +# is probably lesser than the thresholds defined here. +# If this guardrail is enabled along with column_value_size_warn_threshold and column_value_size_fail_threshold, size +# restriction for the column type will be the smallest of the two. +# The two thresholds default to null to disable. +# Min unit: B +# column_text_and_varchar_value_size_warn_threshold: +# column_text_and_varchar_value_size_fail_threshold: +# # Guardrail to warn or fail when encountering larger size of collection data than threshold. # At query time this guardrail is applied only to the collection fragment that is being writen, even though in the case # of non-frozen collections there could be unaccounted parts of the collection on the sstables. This is done this way to @@ -2325,6 +2358,45 @@ drop_compact_storage_enabled: false # Min unit: B # collection_size_fail_threshold: # +# Guardrail to warn or fail when encountering larger size of map data than threshold. +# At query time this guardrail is applied only to the collection fragment that is being writen, even though in the case +# of non-frozen collections there could be unaccounted parts of the collection on the sstables. This is done this way to +# prevent read-before-write. The guardrail is also checked at sstable write time to detect large non-frozen collections, +# although in that case exceeding the fail threshold will only log an error message, without interrupting the operation. +# When collection_map_size_warn_threshold and/or collection_map_size_fail_threshold are defined, they take the precedence +# over the cooresponding collection_size_warn_threshold and/or collection_size_fail_threshold. +# The two thresholds default to null to disable. +# Min unit: B +# collection_map_size_warn_threshold: +# Min unit: B +# collection_map_size_fail_threshold: +# +# Guardrail to warn or fail when encountering larger size of set data than threshold. +# At query time this guardrail is applied only to the collection fragment that is being writen, even though in the case +# of non-frozen collections there could be unaccounted parts of the collection on the sstables. This is done this way to +# prevent read-before-write. The guardrail is also checked at sstable write time to detect large non-frozen collections, +# although in that case exceeding the fail threshold will only log an error message, without interrupting the operation. +# When collection_set_size_warn_threshold and/or collection_set_size_fail_threshold are defined, they take the precedence +# over the cooresponding collection_size_warn_threshold and/or collection_size_fail_threshold. +# The two thresholds default to null to disable. +# Min unit: B +# collection_set_size_warn_threshold: +# Min unit: B +# collection_set_size_fail_threshold: +# +# Guardrail to warn or fail when encountering larger size of list data than threshold. +# At query time this guardrail is applied only to the collection fragment that is being writen, even though in the case +# of non-frozen collections there could be unaccounted parts of the collection on the sstables. This is done this way to +# prevent read-before-write. The guardrail is also checked at sstable write time to detect large non-frozen collections, +# although in that case exceeding the fail threshold will only log an error message, without interrupting the operation. +# When collection_list_size_warn_threshold and/or collection_list_size_fail_threshold are defined, they take the precedence +# over the cooresponding collection_size_warn_threshold and/or collection_size_fail_threshold. +# The two thresholds default to null to disable. +# Min unit: B +# collection_list_size_warn_threshold: +# Min unit: B +# collection_list_size_fail_threshold: +# # Guardrail to warn or fail when encountering more elements in collection than threshold. # At query time this guardrail is applied only to the collection fragment that is being writen, even though in the case # of non-frozen collections there could be unaccounted parts of the collection on the sstables. This is done this way to diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 77f7d9a34176..1b8fc6b7d1e8 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -908,8 +908,22 @@ public static void setClientMode(boolean clientMode) 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 column_ascii_value_size_warn_threshold = null; + public volatile DataStorageSpec.LongBytesBound column_ascii_value_size_fail_threshold = null; + public volatile DataStorageSpec.LongBytesBound column_blob_value_size_warn_threshold = null; + public volatile DataStorageSpec.LongBytesBound column_blob_value_size_fail_threshold = null; + public volatile DataStorageSpec.LongBytesBound column_text_and_varchar_value_size_warn_threshold = null; + public volatile DataStorageSpec.LongBytesBound column_text_and_varchar_value_size_fail_threshold = null; + public volatile DataStorageSpec.LongBytesBound column_varchar_value_size_warn_threshold = null; + public volatile DataStorageSpec.LongBytesBound column_varchar_value_size_fail_threshold = null; public volatile DataStorageSpec.LongBytesBound collection_size_warn_threshold = null; public volatile DataStorageSpec.LongBytesBound collection_size_fail_threshold = null; + public volatile DataStorageSpec.LongBytesBound collection_map_size_warn_threshold = null; + public volatile DataStorageSpec.LongBytesBound collection_map_size_fail_threshold = null; + public volatile DataStorageSpec.LongBytesBound collection_set_size_warn_threshold = null; + public volatile DataStorageSpec.LongBytesBound collection_set_size_fail_threshold = null; + public volatile DataStorageSpec.LongBytesBound collection_list_size_warn_threshold = null; + public volatile DataStorageSpec.LongBytesBound collection_list_size_fail_threshold = null; public volatile int items_per_collection_warn_threshold = -1; public volatile int items_per_collection_fail_threshold = -1; public volatile int fields_per_udt_warn_threshold = -1; diff --git a/src/java/org/apache/cassandra/config/GuardrailsOptions.java b/src/java/org/apache/cassandra/config/GuardrailsOptions.java index e1c2fe892037..1c2d5a513afe 100644 --- a/src/java/org/apache/cassandra/config/GuardrailsOptions.java +++ b/src/java/org/apache/cassandra/config/GuardrailsOptions.java @@ -83,6 +83,13 @@ public GuardrailsOptions(Config config) 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"); + validateSizeThreshold(config.collection_map_size_warn_threshold, config.collection_map_size_fail_threshold, false, "collection_map_size"); + validateSizeThreshold(config.collection_set_size_warn_threshold, config.collection_set_size_fail_threshold, false, "collection_set_size"); + validateSizeThreshold(config.collection_list_size_warn_threshold, config.collection_list_size_fail_threshold, false, "collection_list_size"); + validateSizeThreshold(config.column_ascii_value_size_warn_threshold, config.column_ascii_value_size_fail_threshold, false, "column_acii_value_size"); + validateSizeThreshold(config.column_blob_value_size_warn_threshold, config.column_blob_value_size_fail_threshold, false, "column_blob_value_size"); + validateSizeThreshold(config.column_text_and_varchar_value_size_warn_threshold, config.column_text_and_varchar_value_size_fail_threshold, false, "column_text_value_size"); + validateSizeThreshold(config.column_varchar_value_size_warn_threshold, config.column_varchar_value_size_fail_threshold, false, "column_varchar_value_size"); validateMaxIntThreshold(config.items_per_collection_warn_threshold, config.items_per_collection_fail_threshold, "items_per_collection"); validateMaxIntThreshold(config.fields_per_udt_warn_threshold, config.fields_per_udt_fail_threshold, "fields_per_udt"); validateMaxIntThreshold(config.vector_dimensions_warn_threshold, config.vector_dimensions_fail_threshold, "vector_dimensions"); @@ -630,6 +637,48 @@ public DataStorageSpec.LongBytesBound getColumnValueSizeFailThreshold() return config.column_value_size_fail_threshold; } + @Override + @Nullable + public DataStorageSpec.LongBytesBound getColumnAsciiValueSizeWarnThreshold() + { + return config.column_ascii_value_size_warn_threshold; + } + + @Override + @Nullable + public DataStorageSpec.LongBytesBound getColumnAsciiValueSizeFailThreshold() + { + return config.column_ascii_value_size_fail_threshold; + } + + @Override + @Nullable + public DataStorageSpec.LongBytesBound getColumnBlobValueSizeWarnThreshold() + { + return config.column_blob_value_size_warn_threshold; + } + + @Override + @Nullable + public DataStorageSpec.LongBytesBound getColumnBlobValueSizeFailThreshold() + { + return config.column_blob_value_size_fail_threshold; + } + + @Override + @Nullable + public DataStorageSpec.LongBytesBound getColumnTextAndVarcharValueSizeWarnThreshold() + { + return config.column_text_and_varchar_value_size_warn_threshold; + } + + @Override + @Nullable + public DataStorageSpec.LongBytesBound getColumnTextAndVarcharValueSizeFailThreshold() + { + return config.column_text_and_varchar_value_size_fail_threshold; + } + public void setColumnValueSizeThreshold(@Nullable DataStorageSpec.LongBytesBound warn, @Nullable DataStorageSpec.LongBytesBound fail) { validateSizeThreshold(warn, fail, false, "column_value_size"); @@ -643,6 +692,45 @@ public void setColumnValueSizeThreshold(@Nullable DataStorageSpec.LongBytesBound x -> config.column_value_size_fail_threshold = x); } + public void setColumnAsciiValueSizeThreshold(@Nullable DataStorageSpec.LongBytesBound warn, @Nullable DataStorageSpec.LongBytesBound fail) + { + validateSizeThreshold(warn, fail, false, "column_ascii_value_size"); + updatePropertyWithLogging("column_ascii_value_size_warn_threshold", + warn, + () -> config.column_ascii_value_size_warn_threshold, + x -> config.column_ascii_value_size_warn_threshold = x); + updatePropertyWithLogging("column_ascii_value_size_fail_threshold", + fail, + () -> config.column_ascii_value_size_fail_threshold, + x -> config.column_ascii_value_size_fail_threshold = x); + } + + public void setColumnBlobValueSizeThreshold(@Nullable DataStorageSpec.LongBytesBound warn, @Nullable DataStorageSpec.LongBytesBound fail) + { + validateSizeThreshold(warn, fail, false, "column_blob_value_size"); + updatePropertyWithLogging("column_blob_value_size_warn_threshold", + warn, + () -> config.column_blob_value_size_warn_threshold, + x -> config.column_blob_value_size_warn_threshold = x); + updatePropertyWithLogging("column_blob_value_size_fail_threshold", + fail, + () -> config.column_blob_value_size_fail_threshold, + x -> config.column_blob_value_size_fail_threshold = x); + } + + public void setColumnTextAndVarcharValueSizeThreshold(@Nullable DataStorageSpec.LongBytesBound warn, @Nullable DataStorageSpec.LongBytesBound fail) + { + validateSizeThreshold(warn, fail, false, "column_text_value_size"); + updatePropertyWithLogging("column_text_and_varchar_value_size_warn_threshold", + warn, + () -> config.column_text_and_varchar_value_size_warn_threshold, + x -> config.column_text_and_varchar_value_size_warn_threshold = x); + updatePropertyWithLogging("column_text_and_varchar_value_size_fail_threshold", + fail, + () -> config.column_text_and_varchar_value_size_fail_threshold, + x -> config.column_text_and_varchar_value_size_fail_threshold = x); + } + @Override @Nullable public DataStorageSpec.LongBytesBound getCollectionSizeWarnThreshold() @@ -657,6 +745,48 @@ public DataStorageSpec.LongBytesBound getCollectionSizeFailThreshold() return config.collection_size_fail_threshold; } + @Override + @Nullable + public DataStorageSpec.LongBytesBound getCollectionMapSizeWarnThreshold() + { + return config.collection_map_size_warn_threshold; + } + + @Override + @Nullable + public DataStorageSpec.LongBytesBound getCollectionMapSizeFailThreshold() + { + return config.collection_map_size_fail_threshold; + } + + @Override + @Nullable + public DataStorageSpec.LongBytesBound getCollectionSetSizeWarnThreshold() + { + return config.collection_set_size_warn_threshold; + } + + @Override + @Nullable + public DataStorageSpec.LongBytesBound getCollectionSetSizeFailThreshold() + { + return config.collection_set_size_fail_threshold; + } + + @Override + @Nullable + public DataStorageSpec.LongBytesBound getCollectionListSizeWarnThreshold() + { + return config.collection_list_size_warn_threshold; + } + + @Override + @Nullable + public DataStorageSpec.LongBytesBound getCollectionListSizeFailThreshold() + { + return config.collection_list_size_fail_threshold; + } + public void setCollectionSizeThreshold(@Nullable DataStorageSpec.LongBytesBound warn, @Nullable DataStorageSpec.LongBytesBound fail) { validateSizeThreshold(warn, fail, false, "collection_size"); @@ -670,6 +800,45 @@ public void setCollectionSizeThreshold(@Nullable DataStorageSpec.LongBytesBound x -> config.collection_size_fail_threshold = x); } + public void setCollectionMapSizeThreshold(@Nullable DataStorageSpec.LongBytesBound warn, @Nullable DataStorageSpec.LongBytesBound fail) + { + validateSizeThreshold(warn, fail, false, "collection_map_size"); + updatePropertyWithLogging("collection_map_size_warn_threshold", + warn, + () -> config.collection_map_size_warn_threshold, + x -> config.collection_map_size_warn_threshold = x); + updatePropertyWithLogging("collection_map_size_fail_threshold", + fail, + () -> config.collection_map_size_fail_threshold, + x -> config.collection_map_size_fail_threshold = x); + } + + public void setCollectionSetSizeThreshold(@Nullable DataStorageSpec.LongBytesBound warn, @Nullable DataStorageSpec.LongBytesBound fail) + { + validateSizeThreshold(warn, fail, false, "collection_set_size"); + updatePropertyWithLogging("collection_set_size_warn_threshold", + warn, + () -> config.collection_set_size_warn_threshold, + x -> config.collection_set_size_warn_threshold = x); + updatePropertyWithLogging("collection_set_size_fail_threshold", + fail, + () -> config.collection_set_size_fail_threshold, + x -> config.collection_set_size_fail_threshold = x); + } + + public void setCollectionListSizeThreshold(@Nullable DataStorageSpec.LongBytesBound warn, @Nullable DataStorageSpec.LongBytesBound fail) + { + validateSizeThreshold(warn, fail, false, "collection_list_size"); + updatePropertyWithLogging("collection_list_size_warn_threshold", + warn, + () -> config.collection_list_size_warn_threshold, + x -> config.collection_list_size_warn_threshold = x); + updatePropertyWithLogging("collection_list_size_fail_threshold", + fail, + () -> config.collection_list_size_fail_threshold, + x -> config.collection_list_size_fail_threshold = x); + } + @Override public int getItemsPerCollectionWarnThreshold() { diff --git a/src/java/org/apache/cassandra/cql3/UpdateParameters.java b/src/java/org/apache/cassandra/cql3/UpdateParameters.java index a68fd1d736cd..51df331b8dc7 100644 --- a/src/java/org/apache/cassandra/cql3/UpdateParameters.java +++ b/src/java/org/apache/cassandra/cql3/UpdateParameters.java @@ -163,8 +163,12 @@ public Cell<?> addCell(ColumnMetadata column, ByteBuffer value) throws InvalidRe public Cell<?> addCell(ColumnMetadata column, CellPath path, ByteBuffer value) throws InvalidRequestException { + // General column value size Guardrails.columnValueSize.guard(value.remaining(), column.name.toString(), false, clientState); + // Check specific sizes per column type + validateColumnSize(column, value); + if (path != null && column.type.isMultiCell()) Guardrails.columnValueSize.guard(path.dataSize(), column.name.toString(), false, clientState); @@ -175,6 +179,23 @@ public Cell<?> addCell(ColumnMetadata column, CellPath path, ByteBuffer value) t return cell; } + private void validateColumnSize(ColumnMetadata column, ByteBuffer value) + { + CQL3Type cql3Type = column.type.asCQL3Type(); + if (cql3Type.equals(CQL3Type.Native.ASCII)) // Ascii size specific guardrail + { + Guardrails.columnAsciiValueSize.guard(value.remaining(), column.name.toString(), false, clientState); + } + else if (cql3Type.equals(CQL3Type.Native.BLOB)) // Blob size specific guardrail + { + Guardrails.columnBlobValueSize.guard(value.remaining(), column.name.toString(), false, clientState); + } + else if (cql3Type.equals(CQL3Type.Native.TEXT)) // text and varchar size specific guardrails + { + Guardrails.columnTextAndVarcharValueSize.guard(value.remaining(), column.name.toString(), false, clientState); + } + } + public void addCounter(ColumnMetadata column, long increment) throws InvalidRequestException { assert ttl == LivenessInfo.NO_TTL; diff --git a/src/java/org/apache/cassandra/cql3/terms/Lists.java b/src/java/org/apache/cassandra/cql3/terms/Lists.java index 17f39ce26caf..1e95fcfd83c4 100644 --- a/src/java/org/apache/cassandra/cql3/terms/Lists.java +++ b/src/java/org/apache/cassandra/cql3/terms/Lists.java @@ -413,13 +413,13 @@ static void doAppend(Term.Terminal value, ColumnMetadata column, UpdateParameter Cell<?> cell = params.addCell(column, CellPath.create(uuid), buffer); dataSize += cell.dataSize(); } - Guardrails.collectionSize.guard(dataSize, column.name.toString(), false, params.clientState); + Guardrails.collectionListSize.guard(dataSize, column.name.toString(), false, params.clientState); } else { Guardrails.itemsPerCollection.guard(type.collectionSize(elements), column.name.toString(), false, params.clientState); Cell<?> cell = params.addCell(column, value.get()); - Guardrails.collectionSize.guard(cell.dataSize(), column.name.toString(), false, params.clientState); + Guardrails.collectionListSize.guard(cell.dataSize(), column.name.toString(), false, params.clientState); } } } diff --git a/src/java/org/apache/cassandra/cql3/terms/Maps.java b/src/java/org/apache/cassandra/cql3/terms/Maps.java index 70871e2569cb..4355e8a3bf5c 100644 --- a/src/java/org/apache/cassandra/cql3/terms/Maps.java +++ b/src/java/org/apache/cassandra/cql3/terms/Maps.java @@ -352,13 +352,13 @@ static void doPut(Term.Terminal value, ColumnMetadata column, UpdateParameters p Cell<?> cell = params.addCell(column, CellPath.create(iter.next()), iter.next()); dataSize += cell.dataSize(); } - Guardrails.collectionSize.guard(dataSize, column.name.toString(), false, params.clientState); + Guardrails.collectionMapSize.guard(dataSize, column.name.toString(), false, params.clientState); } else { Guardrails.itemsPerCollection.guard(type.collectionSize(elements), column.name.toString(), false, params.clientState); Cell<?> cell = params.addCell(column, value.get()); - Guardrails.collectionSize.guard(cell.dataSize(), column.name.toString(), false, params.clientState); + Guardrails.collectionMapSize.guard(cell.dataSize(), column.name.toString(), false, params.clientState); } } } diff --git a/src/java/org/apache/cassandra/cql3/terms/Sets.java b/src/java/org/apache/cassandra/cql3/terms/Sets.java index deb4adc45bb4..1686f907699f 100644 --- a/src/java/org/apache/cassandra/cql3/terms/Sets.java +++ b/src/java/org/apache/cassandra/cql3/terms/Sets.java @@ -289,13 +289,13 @@ static void doAdd(Term.Terminal value, ColumnMetadata column, UpdateParameters p Cell<?> cell = params.addCell(column, CellPath.create(bb), ByteBufferUtil.EMPTY_BYTE_BUFFER); dataSize += cell.dataSize(); } - Guardrails.collectionSize.guard(dataSize, column.name.toString(), false, params.clientState); + Guardrails.collectionSetSize.guard(dataSize, column.name.toString(), false, params.clientState); } else { Guardrails.itemsPerCollection.guard(type.collectionSize(elements), column.name.toString(), false, params.clientState); Cell<?> cell = params.addCell(column, value.get()); - Guardrails.collectionSize.guard(cell.dataSize(), column.name.toString(), false, params.clientState); + Guardrails.collectionSetSize.guard(cell.dataSize(), column.name.toString(), false, params.clientState); } } } diff --git a/src/java/org/apache/cassandra/db/guardrails/FallbackThreshold.java b/src/java/org/apache/cassandra/db/guardrails/FallbackThreshold.java new file mode 100644 index 000000000000..1a9f12cea4b5 --- /dev/null +++ b/src/java/org/apache/cassandra/db/guardrails/FallbackThreshold.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.guardrails; + +import javax.annotation.Nullable; + +import org.apache.cassandra.service.ClientState; + + +/** + * Guardrail type that defines a fallback guardail in case the primare guardrail is either not enabled or + * is valid. + */ +public class FallbackThreshold<T extends Threshold> extends Threshold +{ + private final T primary; + private final T fallback; + + public FallbackThreshold(T primary, T fallback) + { + super(primary.name, primary.reason, primary.warnThreshold, primary.failThreshold, primary.messageProvider); + this.primary = primary; + this.fallback = fallback; + } + + @Override + public void guard(long value, String what, boolean containsUserData, @Nullable ClientState state) + { + if (primary.enabled(state)) + { + primary.guard(value, what, containsUserData, state); + } + else + { + fallback.guard(value, what, containsUserData, state); + } + } + + @Override + protected boolean compare(long value, long threshold) + { + throw new UnsupportedOperationException("Guardrail fallback wrapper does not support calling this method"); + } + + @Override + protected long failValue(ClientState state) + { + throw new UnsupportedOperationException("Guardrail fallback wrapper does not support calling this method"); + } + + @Override + protected long warnValue(ClientState state) + { + throw new UnsupportedOperationException("Guardrail fallback wrapper does not support calling this method"); + } +} diff --git a/src/java/org/apache/cassandra/db/guardrails/Guardrails.java b/src/java/org/apache/cassandra/db/guardrails/Guardrails.java index df463d8b0f9f..764dd277872b 100644 --- a/src/java/org/apache/cassandra/db/guardrails/Guardrails.java +++ b/src/java/org/apache/cassandra/db/guardrails/Guardrails.java @@ -361,7 +361,7 @@ public final class Guardrails implements GuardrailsMBean what, value, isWarning ? "warning" : "failure", threshold)); /** - * Guardrail on the size of a collection. + * Guardrail on the size of a simple type column. */ public static final MaxThreshold columnValueSize = new MaxThreshold("column_value_size", @@ -372,6 +372,45 @@ public final class Guardrails implements GuardrailsMBean 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. */ @@ -384,6 +423,45 @@ public final class Guardrails implements GuardrailsMBean 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. */ @@ -991,6 +1069,66 @@ public void setColumnValueSizeThreshold(@Nullable String warnSize, @Nullable Str DEFAULT_CONFIG.setColumnValueSizeThreshold(sizeFromString(warnSize), sizeFromString(failSize)); } + @Override + @Nullable + public String getColumnAsciiValueSizeWarnThreshold() + { + return sizeToString(DEFAULT_CONFIG.getColumnAsciiValueSizeWarnThreshold()); + } + + @Override + @Nullable + public String getColumnAsciiValueSizeFailThreshold() + { + return sizeToString(DEFAULT_CONFIG.getColumnAsciiValueSizeFailThreshold()); + } + + @Override + public void setColumnAsciiValueSizeThreshold(@Nullable String warnSize, @Nullable String failSize) + { + DEFAULT_CONFIG.setColumnAsciiValueSizeThreshold(sizeFromString(warnSize), sizeFromString(failSize)); + } + + @Override + @Nullable + public String getColumnBlobValueSizeWarnThreshold() + { + return sizeToString(DEFAULT_CONFIG.getColumnBlobValueSizeWarnThreshold()); + } + + @Override + @Nullable + public String getColumnBlobValueSizeFailThreshold() + { + return sizeToString(DEFAULT_CONFIG.getColumnBlobValueSizeFailThreshold()); + } + + @Override + public void setColumnBlobValueSizeThreshold(@Nullable String warnSize, @Nullable String failSize) + { + DEFAULT_CONFIG.setColumnBlobValueSizeThreshold(sizeFromString(warnSize), sizeFromString(failSize)); + } + + @Override + @Nullable + public String getColumnTextAndVarcharValueSizeWarnThreshold() + { + return sizeToString(DEFAULT_CONFIG.getColumnTextAndVarcharValueSizeWarnThreshold()); + } + + @Override + @Nullable + public String getColumnTextAndVarcharValueSizeFailThreshold() + { + return sizeToString(DEFAULT_CONFIG.getColumnTextAndVarcharValueSizeFailThreshold()); + } + + @Override + public void setColumnTextAndVarcharValueSizeThreshold(@Nullable String warnSize, @Nullable String failSize) + { + DEFAULT_CONFIG.setColumnTextAndVarcharValueSizeThreshold(sizeFromString(warnSize), sizeFromString(failSize)); + } + @Override @Nullable public String getCollectionSizeWarnThreshold() @@ -1011,6 +1149,66 @@ public void setCollectionSizeThreshold(@Nullable String warnSize, @Nullable Stri DEFAULT_CONFIG.setCollectionSizeThreshold(sizeFromString(warnSize), sizeFromString(failSize)); } + @Override + @Nullable + public String getCollectionMapSizeWarnThreshold() + { + return sizeToString(DEFAULT_CONFIG.getCollectionMapSizeWarnThreshold()); + } + + @Override + @Nullable + public String getCollectionMapSizeFailThreshold() + { + return sizeToString(DEFAULT_CONFIG.getCollectionMapSizeFailThreshold()); + } + + @Override + public void setCollectionMapSizeThreshold(@Nullable String warnSize, @Nullable String failSize) + { + DEFAULT_CONFIG.setCollectionMapSizeThreshold(sizeFromString(warnSize), sizeFromString(failSize)); + } + + @Override + @Nullable + public String getCollectionSetSizeWarnThreshold() + { + return sizeToString(DEFAULT_CONFIG.getCollectionSetSizeWarnThreshold()); + } + + @Override + @Nullable + public String getCollectionSetSizeFailThreshold() + { + return sizeToString(DEFAULT_CONFIG.getCollectionSetSizeFailThreshold()); + } + + @Override + public void setCollectionSetSizeThreshold(@Nullable String warnSize, @Nullable String failSize) + { + DEFAULT_CONFIG.setCollectionSetSizeThreshold(sizeFromString(warnSize), sizeFromString(failSize)); + } + + @Override + @Nullable + public String getCollectionListSizeWarnThreshold() + { + return sizeToString(DEFAULT_CONFIG.getCollectionListSizeWarnThreshold()); + } + + @Override + @Nullable + public String getCollectionListSizeFailThreshold() + { + return sizeToString(DEFAULT_CONFIG.getCollectionListSizeFailThreshold()); + } + + @Override + public void setCollectionListSizeThreshold(@Nullable String warnSize, @Nullable String failSize) + { + DEFAULT_CONFIG.setCollectionListSizeThreshold(sizeFromString(warnSize), sizeFromString(failSize)); + } + @Override public int getItemsPerCollectionWarnThreshold() { diff --git a/src/java/org/apache/cassandra/db/guardrails/GuardrailsConfig.java b/src/java/org/apache/cassandra/db/guardrails/GuardrailsConfig.java index d2814a5d7865..5bfbe1b0f153 100644 --- a/src/java/org/apache/cassandra/db/guardrails/GuardrailsConfig.java +++ b/src/java/org/apache/cassandra/db/guardrails/GuardrailsConfig.java @@ -279,6 +279,45 @@ public interface GuardrailsConfig @Nullable DataStorageSpec.LongBytesBound getColumnValueSizeFailThreshold(); + /** + * @return The threshold to warn when writing ascii column values larger than threshold. + */ + @Nullable + DataStorageSpec.LongBytesBound getColumnAsciiValueSizeWarnThreshold(); + + + /** + * @return The threshold to prevent writing ascii column values larger than threshold. + */ + @Nullable + DataStorageSpec.LongBytesBound getColumnAsciiValueSizeFailThreshold(); + + /** + * @return The threshold to whan when writing blob column values larger than threshold. + */ + @Nullable + DataStorageSpec.LongBytesBound getColumnBlobValueSizeWarnThreshold(); + + + /** + * @return The threshold to prevent writing column blob values larger than threshold. + */ + @Nullable + DataStorageSpec.LongBytesBound getColumnBlobValueSizeFailThreshold(); + + /** + * @return The threshold to warn when writing text column values larger than threshold. + */ + @Nullable + DataStorageSpec.LongBytesBound getColumnTextAndVarcharValueSizeWarnThreshold(); + + + /** + * @return The threshold to prevent writing text column values larger than threshold. + */ + @Nullable + DataStorageSpec.LongBytesBound getColumnTextAndVarcharValueSizeFailThreshold(); + /** * @return The threshold to warn when encountering a collection with larger data size than threshold. */ @@ -291,6 +330,42 @@ public interface GuardrailsConfig @Nullable DataStorageSpec.LongBytesBound getCollectionSizeFailThreshold(); + /** + * @return The threshold to warn when encountering a map collection with larger data size than threshold. + */ + @Nullable + DataStorageSpec.LongBytesBound getCollectionMapSizeWarnThreshold(); + + /** + * @return The threshold to prevent map collections with larger data size than threshold. + */ + @Nullable + DataStorageSpec.LongBytesBound getCollectionMapSizeFailThreshold(); + + /** + * @return The threshold to warn when encountering a set collection with larger data size than threshold. + */ + @Nullable + DataStorageSpec.LongBytesBound getCollectionSetSizeWarnThreshold(); + + /** + * @return The threshold to prevent set collections with larger data size than threshold. + */ + @Nullable + DataStorageSpec.LongBytesBound getCollectionSetSizeFailThreshold(); + + /** + * @return The threshold to warn when encountering a list collection with larger data size than threshold. + */ + @Nullable + DataStorageSpec.LongBytesBound getCollectionListSizeWarnThreshold(); + + /** + * @return The threshold to prevent list collections with larger data size than threshold. + */ + @Nullable + DataStorageSpec.LongBytesBound getCollectionListSizeFailThreshold(); + /** * @return The threshold to warn when encountering more elements in a collection than threshold. */ diff --git a/src/java/org/apache/cassandra/db/guardrails/GuardrailsMBean.java b/src/java/org/apache/cassandra/db/guardrails/GuardrailsMBean.java index 5af045610293..421ef341bf45 100644 --- a/src/java/org/apache/cassandra/db/guardrails/GuardrailsMBean.java +++ b/src/java/org/apache/cassandra/db/guardrails/GuardrailsMBean.java @@ -552,6 +552,78 @@ public interface GuardrailsMBean */ void setColumnValueSizeThreshold(@Nullable String warnSize, @Nullable String failSize); + /** + * @return The threshold to warn when encountering ascii 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. + */ + @Nullable + String getColumnAsciiValueSizeWarnThreshold(); + + /** + * @return The threshold to prevent ascii 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. + */ + @Nullable + String getColumnAsciiValueSizeFailThreshold(); + + /** + * @param warnSize The threshold to warn when encountering ascii 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. + * @param failSize The threshold to prevent ascii 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. + */ + void setColumnAsciiValueSizeThreshold(@Nullable String warnSize, @Nullable String failSize); + + /** + * @return The threshold to warn when encountering blob 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. + */ + @Nullable + String getColumnBlobValueSizeWarnThreshold(); + + /** + * @return The threshold to prevent blob 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. + */ + @Nullable + String getColumnBlobValueSizeFailThreshold(); + + /** + * @param warnSize The threshold to warn when encountering blob 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. + * @param failSize The threshold to prevent blob 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. + */ + void setColumnBlobValueSizeThreshold(@Nullable String warnSize, @Nullable String failSize); + + /** + * @return The threshold to warn when encountering text and varchar 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. + */ + @Nullable + String getColumnTextAndVarcharValueSizeWarnThreshold(); + + /** + * @return The threshold to prevent text and varchar 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. + */ + @Nullable + String getColumnTextAndVarcharValueSizeFailThreshold(); + + /** + * @param warnSize The threshold to warn when encountering text or varchar 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. + * @param failSize The threshold to prevent text 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. + */ + void setColumnTextAndVarcharValueSizeThreshold(@Nullable String warnSize, @Nullable String failSize); + /** * @return The threshold to warn when encountering larger size of collection data than threshold, as a string * formatted as in, for example, {@code 10GiB}, {@code 20MiB}, {@code 30KiB} or {@code 40B}. A {@code null} value @@ -578,6 +650,84 @@ public interface GuardrailsMBean */ void setCollectionSizeThreshold(@Nullable String warnSize, @Nullable String failSize); + /** + * @return The threshold to warn when encountering larger size of map collection data than threshold, as a string + * formatted as in, for example, {@code 10GiB}, {@code 20MiB}, {@code 30KiB} or {@code 40B}. A {@code null} value + * means that the threshold is disabled. + */ + @Nullable + String getCollectionMapSizeWarnThreshold(); + + /** + * @return The threshold to prevent map collections with larger data size than threshold, as a string formatted as in, + * for example, {@code 10GiB}, {@code 20MiB}, {@code 30KiB} or {@code 40B}. A {@code null} value means that the + * threshold is disabled. + */ + @Nullable + String getCollectionMapSizeFailThreshold(); + + /** + * @param warnSize The threshold to warn when encountering larger size of map collection data 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. + * @param failSize The threshold to prevent map collections with larger data size 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. + */ + void setCollectionMapSizeThreshold(@Nullable String warnSize, @Nullable String failSize); + + /** + * @return The threshold to warn when encountering larger size of set collection data than threshold, as a string + * formatted as in, for example, {@code 10GiB}, {@code 20MiB}, {@code 30KiB} or {@code 40B}. A {@code null} value + * means that the threshold is disabled. + */ + @Nullable + String getCollectionSetSizeWarnThreshold(); + + /** + * @return The threshold to prevent set collections with larger data size than threshold, as a string formatted as in, + * for example, {@code 10GiB}, {@code 20MiB}, {@code 30KiB} or {@code 40B}. A {@code null} value means that the + * threshold is disabled. + */ + @Nullable + String getCollectionSetSizeFailThreshold(); + + /** + * @param warnSize The threshold to warn when encountering larger size of set collection data 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. + * @param failSize The threshold to prevent set collections with larger data size 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. + */ + void setCollectionSetSizeThreshold(@Nullable String warnSize, @Nullable String failSize); + + /** + * @return The threshold to warn when encountering larger size of list collection data than threshold, as a string + * formatted as in, for example, {@code 10GiB}, {@code 20MiB}, {@code 30KiB} or {@code 40B}. A {@code null} value + * means that the threshold is disabled. + */ + @Nullable + String getCollectionListSizeWarnThreshold(); + + /** + * @return The threshold to prevent list collections with larger data size than threshold, as a string formatted as in, + * for example, {@code 10GiB}, {@code 20MiB}, {@code 30KiB} or {@code 40B}. A {@code null} value means that the + * threshold is disabled. + */ + @Nullable + String getCollectionListSizeFailThreshold(); + + /** + * @param warnSize The threshold to warn when encountering larger size of list collection data 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. + * @param failSize The threshold to prevent list collections with larger data size 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. + */ + void setCollectionListSizeThreshold(@Nullable String warnSize, @Nullable String failSize); + /** * @return The threshold to warn when encountering more elements in a collection than threshold. */ diff --git a/test/unit/org/apache/cassandra/db/guardrails/ColumnTypeSpecificValueThresholdTester.java b/test/unit/org/apache/cassandra/db/guardrails/ColumnTypeSpecificValueThresholdTester.java new file mode 100644 index 000000000000..56de5bcbbe11 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/guardrails/ColumnTypeSpecificValueThresholdTester.java @@ -0,0 +1,263 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.guardrails; + +import java.util.function.Function; +import java.util.function.ToLongFunction; + +import org.junit.Test; + +import static java.nio.ByteBuffer.allocate; + + +public abstract class ColumnTypeSpecificValueThresholdTester extends ValueThresholdTester +{ + + private static final int WARN_THRESHOLD = 1024; // bytes + private static final int FAIL_THRESHOLD = WARN_THRESHOLD * 4; // bytes + + protected ColumnTypeSpecificValueThresholdTester(String warnThreshold, + String failThreshold, + Threshold threshold, + TriConsumer<Guardrails, String, String> setter, + Function<Guardrails, String> warnGetter, + Function<Guardrails, String> failGetter, + Function<Long, String> stringFormatter, + ToLongFunction<String> stringParser) + { + super(warnThreshold, + failThreshold, + threshold, + setter, + warnGetter, + failGetter, + stringFormatter, + stringParser); + } + + @Override + protected int warnThreshold() + { + return WARN_THRESHOLD; + } + + @Override + protected int failThreshold() + { + return FAIL_THRESHOLD; + } + + protected abstract String columnType(); + + @Test + public void testSimplePartitionKey() throws Throwable + { + createTable("CREATE TABLE %s (k " + columnType() + " PRIMARY KEY, v int)"); + + // the size of primary key columns is not guarded because they already have a fixed limit of 65535B + + testNoThreshold("INSERT INTO %s (k, v) VALUES (?, 0)"); + testNoThreshold("UPDATE %s SET v = 1 WHERE k = ?"); + testNoThreshold("DELETE v FROM %s WHERE k = ?"); + testNoThreshold("DELETE FROM %s WHERE k = ?"); + } + + @Test + public void testCompositePartitionKey() throws Throwable + { + createTable("CREATE TABLE %s (k1 " + columnType() + ", k2 " + columnType() + ", v int, PRIMARY KEY((k1, k2)))"); + + // the size of primary key columns is not guarded because they already have a fixed limit of 65535B + + testNoThreshold2("INSERT INTO %s (k1, k2, v) VALUES (?, ?, 0)"); + testNoThreshold2("UPDATE %s SET v = 1 WHERE k1 = ? AND k2 = ?"); + testNoThreshold2("DELETE v FROM %s WHERE k1 = ? AND k2 = ?"); + testNoThreshold2("DELETE FROM %s WHERE k1 = ? AND k2 = ?"); + } + + @Test + public void testSimpleClustering() throws Throwable + { + createTable("CREATE TABLE %s (k int, c " + columnType() + ", v int, PRIMARY KEY(k, c))"); + + // the size of primary key columns is not guarded because they already have a fixed limit of 65535B + + testNoThreshold("INSERT INTO %s (k, c, v) VALUES (0, ?, 0)"); + testNoThreshold("UPDATE %s SET v = 1 WHERE k = 0 AND c = ?"); + testNoThreshold("DELETE v FROM %s WHERE k = 0 AND c = ?"); + testNoThreshold("DELETE FROM %s WHERE k = 0 AND c = ?"); + } + + @Test + public void testCompositeClustering() throws Throwable + { + createTable("CREATE TABLE %s (k int, c1 " + columnType() + ", c2 " + columnType() + ", v int, PRIMARY KEY(k, c1, c2))"); + + // the size of primary key columns is not guarded because they already have a fixed limit of 65535B + + testNoThreshold("DELETE FROM %s WHERE k = 0 AND c1 = ?"); + testNoThreshold("DELETE FROM %s WHERE k = 0 AND c1 > ?"); + testNoThreshold("DELETE FROM %s WHERE k = 0 AND c1 < ?"); + testNoThreshold("DELETE FROM %s WHERE k = 0 AND c1 >= ?"); + testNoThreshold("DELETE FROM %s WHERE k = 0 AND c1 <= ?"); + + testNoThreshold2("INSERT INTO %s (k, c1, c2, v) VALUES (0, ?, ?, 0)"); + testNoThreshold2("UPDATE %s SET v = 1 WHERE k = 0 AND c1 = ? AND c2 = ?"); + testNoThreshold2("DELETE v FROM %s WHERE k = 0 AND c1 = ? AND c2 = ?"); + testNoThreshold2("DELETE FROM %s WHERE k = 0 AND c1 = ? AND c2 = ?"); + testNoThreshold2("DELETE FROM %s WHERE k = 0 AND c1 = ? AND c2 > ?"); + testNoThreshold2("DELETE FROM %s WHERE k = 0 AND c1 = ? AND c2 < ?"); + testNoThreshold2("DELETE FROM %s WHERE k = 0 AND c1 = ? AND c2 >= ?"); + testNoThreshold2("DELETE FROM %s WHERE k = 0 AND c1 = ? AND c2 <= ?"); + } + + @Test + public void testRegularColumn() throws Throwable + { + createTable("CREATE TABLE %s (k int PRIMARY KEY, v " + columnType() + ")"); + + testThreshold("v", "INSERT INTO %s (k, v) VALUES (0, ?)"); + testThreshold("v", "UPDATE %s SET v = ? WHERE k = 0"); + } + + @Test + public void testStaticColumn() throws Throwable + { + createTable("CREATE TABLE %s (k int, c int, s " + columnType() + " STATIC, r int, PRIMARY KEY(k, c))"); + + testThreshold("s", "INSERT INTO %s (k, s) VALUES (0, ?)"); + testThreshold("s", "INSERT INTO %s (k, c, s, r) VALUES (0, 0, ?, 0)"); + testThreshold("s", "UPDATE %s SET s = ? WHERE k = 0"); + testThreshold("s", "UPDATE %s SET s = ?, r = 0 WHERE k = 0 AND c = 0"); + } + + @Test + public void testBatch() throws Throwable + { + createTable("CREATE TABLE %s (k " + columnType() + ", c " + columnType() + ", r " + columnType() + ", s " + columnType() + " STATIC, PRIMARY KEY(k, c))"); + + // partition key + testNoThreshold("BEGIN BATCH INSERT INTO %s (k, c, r) VALUES (?, '0', '0'); APPLY BATCH;"); + testNoThreshold("BEGIN BATCH UPDATE %s SET r = '0' WHERE k = ? AND c = '0'; APPLY BATCH;"); + testNoThreshold("BEGIN BATCH DELETE r FROM %s WHERE k = ? AND c = '0'; APPLY BATCH;"); + testNoThreshold("BEGIN BATCH DELETE FROM %s WHERE k = ?; APPLY BATCH;"); + + // static column + testThreshold("s", "BEGIN BATCH INSERT INTO %s (k, s) VALUES ('0', ?); APPLY BATCH;"); + testThreshold("s", "BEGIN BATCH INSERT INTO %s (k, s, c, r) VALUES ('0', ?, '0', '0'); APPLY BATCH;"); + testThreshold("s", "BEGIN BATCH UPDATE %s SET s = ? WHERE k = '0'; APPLY BATCH;"); + testThreshold("s", "BEGIN BATCH UPDATE %s SET s = ?, r = '0' WHERE k = '0' AND c = '0'; APPLY BATCH;"); + + // clustering key + testNoThreshold("BEGIN BATCH INSERT INTO %s (k, c, r) VALUES ('0', ?, '0'); APPLY BATCH;"); + testNoThreshold("BEGIN BATCH UPDATE %s SET r = '0' WHERE k = '0' AND c = ?; APPLY BATCH;"); + testNoThreshold("BEGIN BATCH DELETE r FROM %s WHERE k = '0' AND c = ?; APPLY BATCH;"); + testNoThreshold("BEGIN BATCH DELETE FROM %s WHERE k = '0' AND c = ?; APPLY BATCH;"); + + // regular column + testThreshold("r", "BEGIN BATCH INSERT INTO %s (k, c, r) VALUES ('0', '0', ?); APPLY BATCH;"); + testThreshold("r", "BEGIN BATCH UPDATE %s SET r = ? WHERE k = '0' AND c = '0'; APPLY BATCH;"); + } + + @Test + public void testCASWithIfNotExistsCondition() throws Throwable + { + createTable("CREATE TABLE %s (k " + columnType() + ", c " + columnType() + ", v " + columnType() + ", s " + columnType() + " STATIC, PRIMARY KEY(k, c))"); + + // partition key + testNoThreshold("INSERT INTO %s (k, c, v) VALUES (?, '0', '0') IF NOT EXISTS"); + + // clustering key + testNoThreshold("INSERT INTO %s (k, c, v) VALUES ('0', ?, '0') IF NOT EXISTS"); + + // static column + assertValid("INSERT INTO %s (k, s) VALUES ('1', ?) IF NOT EXISTS", allocate(1)); + assertValid("INSERT INTO %s (k, s) VALUES ('2', ?) IF NOT EXISTS", allocate(WARN_THRESHOLD)); + assertValid("INSERT INTO %s (k, s) VALUES ('2', ?) IF NOT EXISTS", allocate(WARN_THRESHOLD + 1)); // not applied + assertWarns("s", "INSERT INTO %s (k, s) VALUES ('3', ?) IF NOT EXISTS", allocate(WARN_THRESHOLD + 1)); + + // regular column + assertValid("INSERT INTO %s (k, c, v) VALUES ('4', '0', ?) IF NOT EXISTS", allocate(1)); + assertValid("INSERT INTO %s (k, c, v) VALUES ('5', '0', ?) IF NOT EXISTS", allocate(WARN_THRESHOLD)); + assertValid("INSERT INTO %s (k, c, v) VALUES ('5', '0', ?) IF NOT EXISTS", allocate(WARN_THRESHOLD + 1)); // not applied + assertWarns("v", "INSERT INTO %s (k, c, v) VALUES ('6', '0', ?) IF NOT EXISTS", allocate(WARN_THRESHOLD + 1)); + } + + @Test + public void testCASWithIfExistsCondition() throws Throwable + { + createTable("CREATE TABLE %s (k " + columnType() + ", c " + columnType() + ", v " + columnType() + ", s " + columnType() + " STATIC, PRIMARY KEY(k, c))"); + + // partition key, the CAS updates with values beyond the threshold are not applied, so they don't come to fail + testNoThreshold("UPDATE %s SET v = '0' WHERE k = ? AND c = '0' IF EXISTS"); + + // clustering key, the CAS updates with values beyond the threshold are not applied, so they don't come to fail + testNoThreshold("UPDATE %s SET v = '0' WHERE k = '0' AND c = ? IF EXISTS"); + + // static column, only the applied CAS updates can fire the guardrail + assertValid("INSERT INTO %s (k, s) VALUES ('0', '0')"); + testThreshold("s", "UPDATE %s SET s = ? WHERE k = '0' IF EXISTS"); + assertValid("DELETE FROM %s WHERE k = '0'"); + testNoThreshold("UPDATE %s SET s = ? WHERE k = '0' IF EXISTS"); + + // regular column, only the applied CAS updates can fire the guardrail + assertValid("INSERT INTO %s (k, c) VALUES ('0', '0')"); + testThreshold("v", "UPDATE %s SET v = ? WHERE k = '0' AND c = '0' IF EXISTS"); + assertValid("DELETE FROM %s WHERE k = '0' AND c = '0'"); + testNoThreshold("UPDATE %s SET v = ? WHERE k = '0' AND c = '0' IF EXISTS"); + } + + @Test + public void testCASWithColumnsCondition() throws Throwable + { + createTable("CREATE TABLE %s (k int PRIMARY KEY, v " + columnType() + ")"); + + // updates are always accepted for values lesser than the threshold, independently of whether they are applied + assertValid("DELETE FROM %s WHERE k = 0"); + assertValid("UPDATE %s SET v = ? WHERE k = 0 IF v = '0'", allocate(1)); + assertValid("UPDATE %s SET v = '0' WHERE k = 0"); + assertValid("UPDATE %s SET v = ? WHERE k = 0 IF v = '0'", allocate(1)); + + // updates are always accepted for values equals to the threshold, independently of whether they are applied + assertValid("DELETE FROM %s WHERE k = 0"); + assertValid("UPDATE %s SET v = ? WHERE k = 0 IF v = '0'", allocate(WARN_THRESHOLD)); + assertValid("UPDATE %s SET v = '0' WHERE k = 0"); + assertValid("UPDATE %s SET v = ? WHERE k = 0 IF v = '0'", allocate(WARN_THRESHOLD)); + + // updates beyond the threshold fail only if the update is applied + assertValid("DELETE FROM %s WHERE k = 0"); + assertValid("UPDATE %s SET v = ? WHERE k = 0 IF v = '0'", allocate(WARN_THRESHOLD + 1)); + assertValid("UPDATE %s SET v = '0' WHERE k = 0"); + assertWarns("v", "UPDATE %s SET v = ? WHERE k = 0 IF v = '0'", allocate(WARN_THRESHOLD + 1)); + } + + @Test + public void testSelect() throws Throwable + { + createTable("CREATE TABLE %s (k " + columnType() + ", c " + columnType() + ", r " + columnType() + ", s " + columnType() + " STATIC, PRIMARY KEY(k, c))"); + + // the guardail is only checked for writes; reads are excluded + + testNoThreshold("SELECT * FROM %s WHERE k = ?"); + testNoThreshold("SELECT * FROM %s WHERE k = '0' AND c = ?"); + testNoThreshold("SELECT * FROM %s WHERE c = ? ALLOW FILTERING"); + testNoThreshold("SELECT * FROM %s WHERE s = ? ALLOW FILTERING"); + testNoThreshold("SELECT * FROM %s WHERE r = ? ALLOW FILTERING"); + } +} diff --git a/test/unit/org/apache/cassandra/db/guardrails/GuardrailCollectionListSizeTest.java b/test/unit/org/apache/cassandra/db/guardrails/GuardrailCollectionListSizeTest.java new file mode 100644 index 000000000000..b118f954f3d1 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/guardrails/GuardrailCollectionListSizeTest.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.guardrails; + + +import java.nio.ByteBuffer; + +import com.google.common.collect.ImmutableList; +import org.junit.Test; + +import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.marshal.ListType; + +import static java.nio.ByteBuffer.allocate; + +/** + * Tests the guardrail for the size of list collections, {@link Guardrails#collectionSize}. + * <p> + * This test doesn't include the activation of the guardrail during sstable writes, these cases are covered by the dtest + * {@link org.apache.cassandra.distributed.test.guardrails.GuardrailCollectionSizeOnSSTableWriteTest}. + */ +public class GuardrailCollectionListSizeTest extends GuardrailCollectionTypeSpecificSizeTester +{ + public GuardrailCollectionListSizeTest() + { + super(Guardrails.collectionListSize, + Guardrails::setCollectionListSizeThreshold, + Guardrails::getCollectionListSizeWarnThreshold, + Guardrails::getCollectionListSizeFailThreshold); + } + + @Test + public void testListSize() throws Throwable + { + createTable("CREATE TABLE %s (k int PRIMARY KEY, v list<blob>)"); + + assertValid("INSERT INTO %s (k, v) VALUES (0, null)"); + assertValid("INSERT INTO %s (k, v) VALUES (1, ?)", list()); + assertValid("INSERT INTO %s (k, v) VALUES (2, ?)", list(allocate(1))); + assertValid("INSERT INTO %s (k, v) VALUES (3, ?)", list(allocate(WARN_THRESHOLD / 2))); + + assertWarns("INSERT INTO %s (k, v) VALUES (4, ?)", list(allocate(WARN_THRESHOLD))); + assertWarns("INSERT INTO %s (k, v) VALUES (5, ?)", list(allocate(WARN_THRESHOLD / 2), allocate(WARN_THRESHOLD / 2))); + + assertFails("INSERT INTO %s (k, v) VALUES (6, ?)", list(allocate(FAIL_THRESHOLD))); + assertFails("INSERT INTO %s (k, v) VALUES (7, ?)", list(allocate(FAIL_THRESHOLD / 2), allocate(FAIL_THRESHOLD / 2))); + } + + @Test + public void testListSizeFrozen() throws Throwable + { + createTable("CREATE TABLE %s (k int PRIMARY KEY, v frozen<list<blob>>)"); + + assertValid("INSERT INTO %s (k, v) VALUES (0, null)"); + assertValid("INSERT INTO %s (k, v) VALUES (1, ?)", list()); + assertValid("INSERT INTO %s (k, v) VALUES (2, ?)", list(allocate(1))); + assertWarns("INSERT INTO %s (k, v) VALUES (4, ?)", list(allocate(WARN_THRESHOLD))); + assertFails("INSERT INTO %s (k, v) VALUES (5, ?)", list(allocate(FAIL_THRESHOLD))); + } + + @Test + public void testListSizeWithUpdates() throws Throwable + { + createTable("CREATE TABLE %s (k int PRIMARY KEY, v list<blob>)"); + + assertValid("INSERT INTO %s (k, v) VALUES (0, ?)", list(allocate(1))); + assertValid("UPDATE %s SET v = v + ? WHERE k = 0", list(allocate(1))); + + assertValid("INSERT INTO %s (k, v) VALUES (1, ?)", list(allocate(WARN_THRESHOLD / 2))); + assertValid("UPDATE %s SET v = v + ? WHERE k = 1", list(allocate(WARN_THRESHOLD / 2))); + + assertWarns("INSERT INTO %s (k, v) VALUES (2, ?)", list(allocate(FAIL_THRESHOLD / 2))); + assertWarns("UPDATE %s SET v = v + ? WHERE k = 2", list(allocate(FAIL_THRESHOLD / 2))); + } + + private static ByteBuffer list(ByteBuffer... values) + { + return ListType.getInstance(BytesType.instance, true).decompose(ImmutableList.copyOf(values)); + } +} diff --git a/test/unit/org/apache/cassandra/db/guardrails/GuardrailCollectionMapSizeTest.java b/test/unit/org/apache/cassandra/db/guardrails/GuardrailCollectionMapSizeTest.java new file mode 100644 index 000000000000..b4b99c1f1cd1 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/guardrails/GuardrailCollectionMapSizeTest.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.guardrails; + + +import java.nio.ByteBuffer; +import java.util.Collections; + +import com.google.common.collect.ImmutableMap; +import org.junit.Test; + +import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.marshal.MapType; + +import static java.nio.ByteBuffer.allocate; + +/** + * Tests the guardrail for the size of map collections, {@link Guardrails#collectionSize}. + * <p> + * This test doesn't include the activation of the guardrail during sstable writes, these cases are covered by the dtest + * {@link org.apache.cassandra.distributed.test.guardrails.GuardrailCollectionSizeOnSSTableWriteTest}. + */ +public class GuardrailCollectionMapSizeTest extends GuardrailCollectionTypeSpecificSizeTester +{ + public GuardrailCollectionMapSizeTest() + { + super(Guardrails.collectionMapSize, + Guardrails::setCollectionMapSizeThreshold, + Guardrails::getCollectionMapSizeWarnThreshold, + Guardrails::getCollectionMapSizeFailThreshold); + } + + @Test + public void testMapSizeWithUpdates() throws Throwable + { + createTable("CREATE TABLE %s (k int PRIMARY KEY, v map<blob, blob>)"); + + assertValid("INSERT INTO %s (k, v) VALUES (0, ?)", map(allocate(1), allocate(1))); + assertValid("UPDATE %s SET v = v + ? WHERE k = 0", map(allocate(1), allocate(1))); + + assertValid("INSERT INTO %s (k, v) VALUES (1, ?)", map(allocate(1), allocate(WARN_THRESHOLD / 2))); + assertValid("UPDATE %s SET v = v + ? WHERE k = 1", map(allocate(2), allocate(WARN_THRESHOLD / 2))); + + assertValid("INSERT INTO %s (k, v) VALUES (2, ?)", map(allocate(WARN_THRESHOLD / 4), allocate(1))); + assertValid("UPDATE %s SET v = v + ? WHERE k = 2", map(allocate(WARN_THRESHOLD * 3 / 4), allocate(1))); + + assertValid("INSERT INTO %s (k, v) VALUES (3, ?)", map(allocate(WARN_THRESHOLD / 4), allocate(WARN_THRESHOLD / 4))); + assertValid("UPDATE %s SET v = v + ? WHERE k = 3", map(allocate(WARN_THRESHOLD / 4 + 1), allocate(WARN_THRESHOLD / 4))); + + assertWarns("INSERT INTO %s (k, v) VALUES (4, ?)", map(allocate(1), allocate(FAIL_THRESHOLD / 2))); + assertWarns("UPDATE %s SET v = v + ? WHERE k = 4", map(allocate(2), allocate(FAIL_THRESHOLD / 2))); + + assertWarns("INSERT INTO %s (k, v) VALUES (5, ?)", map(allocate(FAIL_THRESHOLD / 4), allocate(1))); + assertWarns("UPDATE %s SET v = v + ? WHERE k = 5", map(allocate(FAIL_THRESHOLD * 3 / 4), allocate(1))); + + assertWarns("INSERT INTO %s (k, v) VALUES (6, ?)", map(allocate(FAIL_THRESHOLD / 4), allocate(FAIL_THRESHOLD / 4))); + assertWarns("UPDATE %s SET v = v + ? WHERE k = 6", map(allocate(FAIL_THRESHOLD / 4 + 1), allocate(FAIL_THRESHOLD / 4))); + } + + @Test + public void testMapSizeFrozen() throws Throwable + { + createTable("CREATE TABLE %s (k int PRIMARY KEY, v frozen<map<blob, blob>>)"); + + assertValid("INSERT INTO %s (k, v) VALUES (0, null)"); + assertValid("INSERT INTO %s (k, v) VALUES (1, ?)", map()); + assertValid("INSERT INTO %s (k, v) VALUES (2, ?)", map(allocate(1), allocate(1))); + assertWarns("INSERT INTO %s (k, v) VALUES (4, ?)", map(allocate(1), allocate(WARN_THRESHOLD))); + assertWarns("INSERT INTO %s (k, v) VALUES (5, ?)", map(allocate(WARN_THRESHOLD), allocate(1))); + assertWarns("INSERT INTO %s (k, v) VALUES (6, ?)", map(allocate(WARN_THRESHOLD), allocate(WARN_THRESHOLD))); + assertFails("INSERT INTO %s (k, v) VALUES (7, ?)", map(allocate(1), allocate(FAIL_THRESHOLD))); + assertFails("INSERT INTO %s (k, v) VALUES (8, ?)", map(allocate(FAIL_THRESHOLD), allocate(1))); + assertFails("INSERT INTO %s (k, v) VALUES (9, ?)", map(allocate(FAIL_THRESHOLD), allocate(FAIL_THRESHOLD))); + } + + @Test + public void testMapSize() throws Throwable + { + createTable("CREATE TABLE %s (k int PRIMARY KEY, v map<blob, blob>)"); + + assertValid("INSERT INTO %s (k, v) VALUES (0, null)"); + assertValid("INSERT INTO %s (k, v) VALUES (1, ?)", map()); + assertValid("INSERT INTO %s (k, v) VALUES (2, ?)", map(allocate(1), allocate(1))); + assertValid("INSERT INTO %s (k, v) VALUES (3, ?)", map(allocate(1), allocate(WARN_THRESHOLD / 2))); + assertValid("INSERT INTO %s (k, v) VALUES (4, ?)", map(allocate(WARN_THRESHOLD / 2), allocate(1))); + + assertWarns("INSERT INTO %s (k, v) VALUES (5, ?)", map(allocate(WARN_THRESHOLD), allocate(1))); + assertWarns("INSERT INTO %s (k, v) VALUES (6, ?)", map(allocate(1), allocate(WARN_THRESHOLD))); + assertWarns("INSERT INTO %s (k, v) VALUES (7, ?)", map(allocate(WARN_THRESHOLD), allocate(WARN_THRESHOLD))); + + assertFails("INSERT INTO %s (k, v) VALUES (8, ?)", map(allocate(FAIL_THRESHOLD), allocate(1))); + assertFails("INSERT INTO %s (k, v) VALUES (9, ?)", map(allocate(1), allocate(FAIL_THRESHOLD))); + assertFails("INSERT INTO %s (k, v) VALUES (10, ?)", map(allocate(FAIL_THRESHOLD), allocate(FAIL_THRESHOLD))); + } + + private ByteBuffer map() + { + return MapType.getInstance(BytesType.instance, BytesType.instance, true).decompose(Collections.emptyMap()); + } + + private ByteBuffer map(ByteBuffer key, ByteBuffer value) + { + return MapType.getInstance(BytesType.instance, BytesType.instance, true).decompose(ImmutableMap.of(key, value)); + } +} diff --git a/test/unit/org/apache/cassandra/db/guardrails/GuardrailCollectionSetSizeTest.java b/test/unit/org/apache/cassandra/db/guardrails/GuardrailCollectionSetSizeTest.java new file mode 100644 index 000000000000..2914e2ef6af2 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/guardrails/GuardrailCollectionSetSizeTest.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.guardrails; + + +import java.nio.ByteBuffer; + +import com.google.common.collect.ImmutableSet; +import org.junit.Test; + +import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.marshal.SetType; + +import static java.nio.ByteBuffer.allocate; + +/** + * Tests the guardrail for the size of set collections, {@link Guardrails#collectionSize}. + * <p> + * This test doesn't include the activation of the guardrail during sstable writes, these cases are covered by the dtest + * {@link org.apache.cassandra.distributed.test.guardrails.GuardrailCollectionSizeOnSSTableWriteTest}. + */ +public class GuardrailCollectionSetSizeTest extends GuardrailCollectionTypeSpecificSizeTester +{ + public GuardrailCollectionSetSizeTest() + { + super(Guardrails.collectionSetSize, + Guardrails::setCollectionSetSizeThreshold, + Guardrails::getCollectionSetSizeWarnThreshold, + Guardrails::getCollectionSetSizeFailThreshold); + } + + @Test + public void testSetSizeWithUpdates() throws Throwable + { + createTable("CREATE TABLE %s (k int PRIMARY KEY, v set<blob>)"); + + assertValid("INSERT INTO %s (k, v) VALUES (0, ?)", set(allocate(1))); + assertValid("UPDATE %s SET v = v + ? WHERE k = 0", set(allocate(1))); + + assertValid("INSERT INTO %s (k, v) VALUES (1, ?)", set(allocate(WARN_THRESHOLD / 4))); + assertValid("UPDATE %s SET v = v + ? WHERE k = 1", set(allocate(WARN_THRESHOLD * 3 / 4))); + + assertWarns("INSERT INTO %s (k, v) VALUES (2, ?)", set(allocate(FAIL_THRESHOLD / 4))); + assertWarns("UPDATE %s SET v = v + ? WHERE k = 2", set(allocate(FAIL_THRESHOLD * 3 / 4))); + } + + @Test + public void testSetSizeFrozen() throws Throwable + { + createTable("CREATE TABLE %s (k int PRIMARY KEY, v frozen<set<blob>>)"); + + assertValid("INSERT INTO %s (k, v) VALUES (0, null)"); + assertValid("INSERT INTO %s (k, v) VALUES (1, ?)", set()); + assertValid("INSERT INTO %s (k, v) VALUES (2, ?)", set(allocate(1))); + assertWarns("INSERT INTO %s (k, v) VALUES (4, ?)", set(allocate(WARN_THRESHOLD))); + assertFails("INSERT INTO %s (k, v) VALUES (5, ?)", set(allocate(FAIL_THRESHOLD))); + } + + @Test + public void testSetSize() throws Throwable + { + createTable("CREATE TABLE %s (k int PRIMARY KEY, v set<blob>)"); + + assertValid("INSERT INTO %s (k, v) VALUES (0, null)"); + assertValid("INSERT INTO %s (k, v) VALUES (1, ?)", set()); + assertValid("INSERT INTO %s (k, v) VALUES (2, ?)", set(allocate(1))); + assertValid("INSERT INTO %s (k, v) VALUES (3, ?)", set(allocate(WARN_THRESHOLD / 2))); + + assertWarns("INSERT INTO %s (k, v) VALUES (4, ?)", set(allocate(WARN_THRESHOLD))); + assertWarns("INSERT INTO %s (k, v) VALUES (5, ?)", set(allocate(WARN_THRESHOLD / 4), allocate(WARN_THRESHOLD * 3 / 4))); + + assertFails("INSERT INTO %s (k, v) VALUES (6, ?)", set(allocate(FAIL_THRESHOLD))); + assertFails("INSERT INTO %s (k, v) VALUES (7, ?)", set(allocate(FAIL_THRESHOLD / 4), allocate(FAIL_THRESHOLD * 3 / 4))); + } + + private static ByteBuffer set(ByteBuffer... values) + { + return SetType.getInstance(BytesType.instance, true).decompose(ImmutableSet.copyOf(values)); + } +} diff --git a/test/unit/org/apache/cassandra/db/guardrails/GuardrailCollectionTypeSpecificSizeTester.java b/test/unit/org/apache/cassandra/db/guardrails/GuardrailCollectionTypeSpecificSizeTester.java new file mode 100644 index 000000000000..073e72d5741d --- /dev/null +++ b/test/unit/org/apache/cassandra/db/guardrails/GuardrailCollectionTypeSpecificSizeTester.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.guardrails; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.function.Function; + +import org.junit.After; +import org.apache.cassandra.config.DataStorageSpec; +import static org.apache.cassandra.config.DataStorageSpec.DataStorageUnit.BYTES; + +/** + * Tests the guardrail for the size of collections, {@link Guardrails#collectionSize}. + * <p> + * This test doesn't include the activation of the guardrail during sstable writes, these cases are covered by the dtest + * {@link org.apache.cassandra.distributed.test.guardrails.GuardrailCollectionSizeOnSSTableWriteTest}. + */ +public abstract class GuardrailCollectionTypeSpecificSizeTester extends ThresholdTester +{ + protected static final int WARN_THRESHOLD = 1024; // bytes + protected static final int FAIL_THRESHOLD = WARN_THRESHOLD * 4; // bytes + + public GuardrailCollectionTypeSpecificSizeTester(Threshold threshold, + TriConsumer<Guardrails, String, String> setter, + Function<Guardrails, String> warnGetter, + Function<Guardrails, String> failGetter) + { + super(WARN_THRESHOLD + "B", + FAIL_THRESHOLD + "B", + threshold, + setter, + warnGetter, + failGetter, + bytes -> new DataStorageSpec.LongBytesBound(bytes, BYTES).toString(), + size -> new DataStorageSpec.LongBytesBound(size).toBytes()); + } + + @After + public void after() + { + // immediately drop the created table so its async cleanup doesn't interfere with the next tests + if (currentTable() != null) + dropTable("DROP TABLE %s"); + } + + @Override + protected String createTable(String query) + { + String table = super.createTable(query); + disableCompaction(); + return table; + } + + protected void assertValid(String query, ByteBuffer... values) throws Throwable + { + assertValid(execute(query, values)); + } + + protected void assertWarns(String query, ByteBuffer... values) throws Throwable + { + assertWarns(execute(query, values), "Detected collection v"); + } + + protected void assertFails(String query, ByteBuffer... values) throws Throwable + { + assertFails(execute(query, values), "Detected collection v"); + } + + protected CheckedFunction execute(String query, ByteBuffer... values) + { + return () -> execute(userClientState, query, Arrays.asList(values)); + } +} diff --git a/test/unit/org/apache/cassandra/db/guardrails/GuardrailColumnAsciiValueSizeTest.java b/test/unit/org/apache/cassandra/db/guardrails/GuardrailColumnAsciiValueSizeTest.java new file mode 100644 index 000000000000..c6d93f2ad46b --- /dev/null +++ b/test/unit/org/apache/cassandra/db/guardrails/GuardrailColumnAsciiValueSizeTest.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.guardrails; + +import org.apache.cassandra.config.DataStorageSpec; +import static org.apache.cassandra.config.DataStorageSpec.DataStorageUnit.BYTES; + +/** + * Tests the guardrail for the size of ascii column values, {@link Guardrails#columnAsciiValueSize}. + */ +public class GuardrailColumnAsciiValueSizeTest extends ColumnTypeSpecificValueThresholdTester +{ + private static final int WARN_THRESHOLD = 1024; // bytes + private static final int FAIL_THRESHOLD = WARN_THRESHOLD * 4; // bytes + + public GuardrailColumnAsciiValueSizeTest() + { + super(WARN_THRESHOLD + "B", + FAIL_THRESHOLD + "B", + Guardrails.columnAsciiValueSize, + Guardrails::setColumnAsciiValueSizeThreshold, + Guardrails::getColumnAsciiValueSizeWarnThreshold, + Guardrails::getColumnAsciiValueSizeFailThreshold, + bytes -> new DataStorageSpec.LongBytesBound(bytes, BYTES).toString(), + size -> new DataStorageSpec.LongBytesBound(size).toBytes()); + } + + @Override + protected int warnThreshold() + { + return WARN_THRESHOLD; + } + + @Override + protected int failThreshold() + { + return FAIL_THRESHOLD; + } + + @Override + protected String columnType() + { + return "ascii"; + } +} diff --git a/test/unit/org/apache/cassandra/db/guardrails/GuardrailColumnBlobValueSizeTest.java b/test/unit/org/apache/cassandra/db/guardrails/GuardrailColumnBlobValueSizeTest.java new file mode 100644 index 000000000000..12e5ca2395f9 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/guardrails/GuardrailColumnBlobValueSizeTest.java @@ -0,0 +1,180 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.guardrails; + +import org.junit.Test; + +import org.apache.cassandra.config.DataStorageSpec; + +import static java.nio.ByteBuffer.allocate; +import static org.apache.cassandra.config.DataStorageSpec.DataStorageUnit.BYTES; + +/** + * Tests the guardrail for the size of blob column values, {@link Guardrails#columnBlobValueSize}. + */ +public class GuardrailColumnBlobValueSizeTest extends ColumnTypeSpecificValueThresholdTester +{ + private static final int WARN_THRESHOLD = 1024; // bytes + private static final int FAIL_THRESHOLD = WARN_THRESHOLD * 4; // bytes + + public GuardrailColumnBlobValueSizeTest() + { + super(WARN_THRESHOLD + "B", + FAIL_THRESHOLD + "B", + Guardrails.columnBlobValueSize, + Guardrails::setColumnBlobValueSizeThreshold, + Guardrails::getColumnBlobValueSizeWarnThreshold, + Guardrails::getColumnBlobValueSizeFailThreshold, + bytes -> new DataStorageSpec.LongBytesBound(bytes, BYTES).toString(), + size -> new DataStorageSpec.LongBytesBound(size).toBytes()); + } + + @Override + protected int warnThreshold() + { + return WARN_THRESHOLD; + } + + @Override + protected int failThreshold() + { + return FAIL_THRESHOLD; + } + + @Override + protected String columnType() + { + return "blob"; + } + + @Override + @Test + public void testBatch() throws Throwable + { + createTable("CREATE TABLE %s (k text, c " + columnType() + ", r " + columnType() + ", s " + columnType() + " STATIC, PRIMARY KEY(k, c))"); + + // partition key + testNoThreshold("BEGIN BATCH INSERT INTO %s (k, c, r) VALUES (?, textAsBlob('0'), textAsBlob('0')); APPLY BATCH;"); + testNoThreshold("BEGIN BATCH UPDATE %s SET r = textAsBlob('0') WHERE k = ? AND c = textAsBlob('0'); APPLY BATCH;"); + + // static column + testThreshold("s", "BEGIN BATCH INSERT INTO %s (k, s) VALUES ('0', ?); APPLY BATCH;"); + testThreshold("s", "BEGIN BATCH INSERT INTO %s (k, s, c, r) VALUES ('0', ?, textAsBlob('0'), textAsBlob('0')); APPLY BATCH;"); + testThreshold("s", "BEGIN BATCH UPDATE %s SET s = ? WHERE k = '0'; APPLY BATCH;"); + testThreshold("s", "BEGIN BATCH UPDATE %s SET s = ?, r = textAsBlob('0') WHERE k = '0' AND c = textAsBlob('0'); APPLY BATCH;"); + + // clustering key + testNoThreshold("BEGIN BATCH INSERT INTO %s (k, c, r) VALUES ('0', ?, textAsBlob('0')); APPLY BATCH;"); + testNoThreshold("BEGIN BATCH UPDATE %s SET r = textAsBlob('0') WHERE k = '0' AND c = ?; APPLY BATCH;"); + testNoThreshold("BEGIN BATCH DELETE r FROM %s WHERE k = '0' AND c = ?; APPLY BATCH;"); + testNoThreshold("BEGIN BATCH DELETE FROM %s WHERE k = '0' AND c = ?; APPLY BATCH;"); + + // regular column + testThreshold("r", "BEGIN BATCH INSERT INTO %s (k, c, r) VALUES ('0', textAsBlob('0'), ?); APPLY BATCH;"); + testThreshold("r", "BEGIN BATCH UPDATE %s SET r = ? WHERE k = '0' AND c = textAsBlob('0'); APPLY BATCH;"); + } + + @Override + @Test + public void testCASWithIfNotExistsCondition() throws Throwable + { + createTable("CREATE TABLE %s (k text, c " + columnType() + ", v " + columnType() + ", s " + columnType() + " STATIC, PRIMARY KEY(k, c))"); + + // partition key + testNoThreshold("INSERT INTO %s (k, c, v) VALUES (?, textAsBlob('0'), textAsBlob('0')) IF NOT EXISTS"); + + // clustering key + testNoThreshold("INSERT INTO %s (k, c, v) VALUES ('0', ?, textAsBlob('0')) IF NOT EXISTS"); + + // static column + assertValid("INSERT INTO %s (k, s) VALUES ('1', ?) IF NOT EXISTS", allocate(1)); + assertValid("INSERT INTO %s (k, s) VALUES ('2', ?) IF NOT EXISTS", allocate(WARN_THRESHOLD)); + assertValid("INSERT INTO %s (k, s) VALUES ('2', ?) IF NOT EXISTS", allocate(WARN_THRESHOLD + 1)); // not applied + assertWarns("s", "INSERT INTO %s (k, s) VALUES ('3', ?) IF NOT EXISTS", allocate(WARN_THRESHOLD + 1)); + + // regular column + assertValid("INSERT INTO %s (k, c, v) VALUES ('4', textAsBlob('0'), ?) IF NOT EXISTS", allocate(1)); + assertValid("INSERT INTO %s (k, c, v) VALUES ('5', textAsBlob('0'), ?) IF NOT EXISTS", allocate(WARN_THRESHOLD)); + assertValid("INSERT INTO %s (k, c, v) VALUES ('5', textAsBlob('0'), ?) IF NOT EXISTS", allocate(WARN_THRESHOLD + 1)); // not applied + assertWarns("v", "INSERT INTO %s (k, c, v) VALUES ('6', textAsBlob('0'), ?) IF NOT EXISTS", allocate(WARN_THRESHOLD + 1)); + } + + @Override + @Test + public void testCASWithIfExistsCondition() throws Throwable + { + createTable("CREATE TABLE %s (k text, c " + columnType() + ", v " + columnType() + ", s " + columnType() + " STATIC, PRIMARY KEY(k, c))"); + + // partition key, the CAS updates with values beyond the threshold are not applied, so they don't come to fail + testNoThreshold("UPDATE %s SET v = textAsBlob('0') WHERE k = ? AND c = textAsBlob('0') IF EXISTS"); + + // clustering key, the CAS updates with values beyond the threshold are not applied, so they don't come to fail + testNoThreshold("UPDATE %s SET v = textAsBlob('0') WHERE k = '0' AND c = ? IF EXISTS"); + + // static column, only the applied CAS updates can fire the guardrail + assertValid("INSERT INTO %s (k, s) VALUES ('0', textAsBlob('0'))"); + testThreshold("s", "UPDATE %s SET s = ? WHERE k = '0' IF EXISTS"); + assertValid("DELETE FROM %s WHERE k = '0'"); + testNoThreshold("UPDATE %s SET s = ? WHERE k = '0' IF EXISTS"); + + // regular column, only the applied CAS updates can fire the guardrail + assertValid("INSERT INTO %s (k, c) VALUES ('0', textAsBlob('0'))"); + testThreshold("v", "UPDATE %s SET v = ? WHERE k = '0' AND c = textAsBlob('0') IF EXISTS"); + assertValid("DELETE FROM %s WHERE k = '0' AND c = textAsBlob('0')"); + testNoThreshold("UPDATE %s SET v = ? WHERE k = '0' AND c = textAsBlob('0') IF EXISTS"); + } + + @Test + public void testCASWithColumnsCondition() throws Throwable + { + createTable("CREATE TABLE %s (k int PRIMARY KEY, v " + columnType() + ")"); + + // updates are always accepted for values lesser than the threshold, independently of whether they are applied + assertValid("DELETE FROM %s WHERE k = 0"); + assertValid("UPDATE %s SET v = ? WHERE k = 0 IF v = textAsBlob('0')", allocate(1)); + assertValid("UPDATE %s SET v = textAsBlob('0') WHERE k = 0"); + assertValid("UPDATE %s SET v = ? WHERE k = 0 IF v = textAsBlob('0')", allocate(1)); + + // updates are always accepted for values equals to the threshold, independently of whether they are applied + assertValid("DELETE FROM %s WHERE k = 0"); + assertValid("UPDATE %s SET v = ? WHERE k = 0 IF v = textAsBlob('0')", allocate(WARN_THRESHOLD)); + assertValid("UPDATE %s SET v = textAsBlob('0') WHERE k = 0"); + assertValid("UPDATE %s SET v = ? WHERE k = 0 IF v = textAsBlob('0')", allocate(WARN_THRESHOLD)); + + // updates beyond the threshold fail only if the update is applied + assertValid("DELETE FROM %s WHERE k = 0"); + assertValid("UPDATE %s SET v = ? WHERE k = 0 IF v = textAsBlob('0')", allocate(WARN_THRESHOLD + 1)); + assertValid("UPDATE %s SET v = textAsBlob('0') WHERE k = 0"); + assertWarns("v", "UPDATE %s SET v = ? WHERE k = 0 IF v = textAsBlob('0')", allocate(WARN_THRESHOLD + 1)); + } + + @Override + @Test + public void testSelect() throws Throwable + { + createTable("CREATE TABLE %s (k " + columnType() + ", c " + columnType() + ", r " + columnType() + ", s " + columnType() + " STATIC, PRIMARY KEY(k, c))"); + + // the guardail is only checked for writes; reads are excluded + + testNoThreshold("SELECT * FROM %s WHERE k = ?"); + testNoThreshold("SELECT * FROM %s WHERE c = ? ALLOW FILTERING"); + testNoThreshold("SELECT * FROM %s WHERE s = ? ALLOW FILTERING"); + testNoThreshold("SELECT * FROM %s WHERE r = ? ALLOW FILTERING"); + } +} diff --git a/test/unit/org/apache/cassandra/db/guardrails/GuardrailColumnTextAndVarcharValueSizeTest.java b/test/unit/org/apache/cassandra/db/guardrails/GuardrailColumnTextAndVarcharValueSizeTest.java new file mode 100644 index 000000000000..dfbb2217203e --- /dev/null +++ b/test/unit/org/apache/cassandra/db/guardrails/GuardrailColumnTextAndVarcharValueSizeTest.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.guardrails; + +import org.apache.cassandra.config.DataStorageSpec; + +import static org.apache.cassandra.config.DataStorageSpec.DataStorageUnit.BYTES; + +/** + * Tests the guardrail for the size of text and varchar column values, {@link Guardrails#columnTextAndVarcharValueSize}. + */ +public class GuardrailColumnTextAndVarcharValueSizeTest extends ColumnTypeSpecificValueThresholdTester +{ + private static final int WARN_THRESHOLD = 1024; // bytes + private static final int FAIL_THRESHOLD = WARN_THRESHOLD * 4; // bytes + + public GuardrailColumnTextAndVarcharValueSizeTest() + { + super(WARN_THRESHOLD + "B", + FAIL_THRESHOLD + "B", + Guardrails.columnTextAndVarcharValueSize, + Guardrails::setColumnTextAndVarcharValueSizeThreshold, + Guardrails::getColumnTextAndVarcharValueSizeWarnThreshold, + Guardrails::getColumnTextAndVarcharValueSizeFailThreshold, + bytes -> new DataStorageSpec.LongBytesBound(bytes, BYTES).toString(), + size -> new DataStorageSpec.LongBytesBound(size).toBytes()); + } + + @Override + protected int warnThreshold() + { + return WARN_THRESHOLD; + } + + @Override + protected int failThreshold() + { + return FAIL_THRESHOLD; + } + + @Override + protected String columnType() + { + return "text"; + } +} From ba16a082f31ce2bde1780542bac87436f43dcb69 Mon Sep 17 00:00:00 2001 From: Mick Semb Wever <mck@apache.org> Date: Sun, 9 Feb 2025 22:38:36 +0100 Subject: [PATCH 139/225] Adjust 5.0 and trunk Jenkinsfile's splits configuration More splits for dtest-upgrades and all variants of the (unit) "test" type. Less splits for long-test, dtest-larges, and jvm-dtest-upgrade. And some debug on xml test files produced in each split. patch by Mick Semb Wever; reviewed by Brandon Williams, Maxim Muzafarov, Dmitry Konstantinov for CASSANDRA-20311 --- .jenkins/Jenkinsfile | 33 ++++++++++++++++----------------- 1 file changed, 16 insertions(+), 17 deletions(-) diff --git a/.jenkins/Jenkinsfile b/.jenkins/Jenkinsfile index d7c84eba8050..808d3cb44952 100644 --- a/.jenkins/Jenkinsfile +++ b/.jenkins/Jenkinsfile @@ -165,25 +165,25 @@ def tasks() { 'cqlsh-test': [splits: 1], 'fqltool-test': [splits: 1, size: 'small'], 'test-cdc': [splits: 8], - 'test': [splits: 8], - 'test-latest': [splits: 8], - 'test-compression': [splits: 8], + 'test': [splits: 16], + 'test-latest': [splits: 16], + 'test-compression': [splits: 16], 'stress-test': [splits: 1, size: 'small'], 'test-burn': [splits: 2], - 'long-test': [splits: 8], - 'test-oa': [splits: 8], - 'test-system-keyspace-directory': [splits: 8], + 'long-test': [splits: 4], + 'test-oa': [splits: 16], + 'test-system-keyspace-directory': [splits: 16], 'jvm-dtest': [splits: 12], - 'jvm-dtest-upgrade': [splits: 8], + 'jvm-dtest-upgrade': [splits: 6], 'simulator-dtest': [splits: 1, size: 'large'], 'dtest': [splits: 64, size: 'large'], 'dtest-novnode': [splits: 64, size: 'large'], 'dtest-latest': [splits: 64, size: 'large'], - 'dtest-large': [splits: 8, size: 'large'], - 'dtest-large-novnode': [splits: 8, size: 'large'], - 'dtest-large-latest': [splits: 8, size: 'large'], - 'dtest-upgrade': [splits: 64, size: 'large'], - 'dtest-upgrade-novnode': [splits: 64, size: 'large'], + 'dtest-large': [splits: 6, size: 'large'], + 'dtest-large-novnode': [splits: 6, size: 'large'], + 'dtest-large-latest': [splits: 6, size: 'large'], + 'dtest-upgrade': [splits: 128, size: 'large'], + 'dtest-upgrade-novnode': [splits: 128, size: 'large'], 'dtest-upgrade-large': [splits: 32, size: 'large'], 'dtest-upgrade-novnode-large': [splits: 32, size: 'large'], ] @@ -379,8 +379,7 @@ def test(command, cell) { fetchDockerImages(['ubuntu2004_test']) def cell_suffix = "_jdk${cell.jdk}_python_${cell.python}_${cell.cython}_${cell.arch}_${cell.split}_${splits}" def logfile = "stage-logs/${JOB_NAME}_${BUILD_NUMBER}_${cell.step}${cell_suffix}_attempt${attempt}.log.xz" - // pipe to tee needs pipefail - def script_vars = "#!/bin/bash \n set -o pipefail ; " + def script_vars = "#!/bin/bash \n set -o pipefail ; " // pipe to tee needs pipefail script_vars = "${script_vars} python_version=\'${cell.python}\'" script_vars = "${script_vars} m2_dir=\'${WORKSPACE}/build/m2\'" if ("cqlsh-test" == cell.step) { @@ -394,7 +393,6 @@ def test(command, cell) { def status = sh label: "RUNNING TESTS ${cell.step}...", script: "${script_vars} .build/docker/run-tests.sh -a ${cell.step} -c '${cell.split}/${splits}' -j ${cell.jdk} 2>&1 | tee >( xz -c > build/${logfile} )", returnStatus: true dir("build") { archiveArtifacts artifacts: "${logfile}", fingerprint: true - copyToNightlies("${logfile}", "${cell.step}/${cell.arch}/jdk${cell.jdk}/python${cell.python}/cython_${cell.cython}/" + "split_${cell.split}_${splits}".replace("/", "_")) } if (0 != status) { error("Stage ${cell.step}${cell_suffix} failed with exit status ${status}") } } finally { @@ -411,9 +409,9 @@ def test(command, cell) { find test/output -name nosetests.xml -execdir mv nosetests.xml ${cell.step}/nosetests${cell_suffix}.xml ';' """ junit testResults: "test/**/TEST-*.xml,test/**/cqlshlib*.xml,test/**/nosetests*.xml", testDataPublishers: [[$class: 'StabilityTestDataPublisher']] - sh "find test/output -type f -name *.xml -exec sh -c 'xz -f {} &' ';' ; wait " + sh "find test/output -type f -name *.xml -exec sh -c 'xz -f {} &' ';' ; wait ; find test/output -type f -name *.xml.xz | wc -l" archiveArtifacts artifacts: "test/logs/**,test/**/TEST-*.xml.xz,test/**/cqlshlib*.xml.xz,test/**/nosetests*.xml.xz", fingerprint: true - copyToNightlies("test/logs/**", "${cell.step}/${cell.arch}/jdk${cell.jdk}/python${cell.python}/cython_${cell.cython}/" + "split_${cell.split}_${splits}".replace("/", "_")) + copyToNightlies("${logfile}, test/logs/**", "${cell.step}/${cell.arch}/jdk${cell.jdk}/python${cell.python}/cython_${cell.cython}/" + "split_${cell.split}_${splits}".replace("/", "_")) } cleanAgent(cell.step) } @@ -551,6 +549,7 @@ def generateTestReports() { // TODO parallelised for loop // TODO results_details.tar.xz needs to include all logs for failed tests sh """${script_vars} ( + find build/test/output -type f -name *.xml.xz | wc -l find build/test/output -name *.xml.xz -exec sh -c 'xz -f --decompress {} &' ';' ; wait for target in \$(ls build/test/output/) ; do From 1a6b8e06288bbb62e2f816c8d7b24569a0a839a0 Mon Sep 17 00:00:00 2001 From: Sam Tunnicliffe <samt@apache.org> Date: Tue, 11 Feb 2025 11:01:27 +0000 Subject: [PATCH 140/225] Invalidate affected prepared stmts on every table metadata change Where MVs are also affected by a table alteration make sure we correctly set the epoch of the TableMetadata enclosed in the ViewMetadata Patch by Sam Tunnicliffe and Marcus Eriksson; reviewed by Marcus Eriksson for CASSANDRA-20318 Co-authored-by: Sam Tunnicliffe <samt@apache.org> Co-authored-by: Marcus Eriksson <marcuse@apache.org> --- CHANGES.txt | 1 + .../cassandra/schema/TableMetadata.java | 13 ++--- .../org/apache/cassandra/schema/Views.java | 7 ++- .../tcm/transformations/AlterSchema.java | 30 ++++++++--- .../PreparedStatementInvalidationTest.java | 54 +++++++++++++++++++ 5 files changed, 89 insertions(+), 16 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/test/PreparedStatementInvalidationTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 778cb533ac02..04274279d715 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Invalidate relevant prepared statements on every change to TableMetadata (CASSANDRA-20318) * Add per type max size guardrails (CASSANDRA-19677) * Make it possible to abort all kinds of multi step operations (CASSANDRA-20217) * Do not leak non-Java exceptions when calling snapshot operations via JMX (CASSANDRA-20335) diff --git a/src/java/org/apache/cassandra/schema/TableMetadata.java b/src/java/org/apache/cassandra/schema/TableMetadata.java index 6e23a4bd080f..4dcec20f92a1 100644 --- a/src/java/org/apache/cassandra/schema/TableMetadata.java +++ b/src/java/org/apache/cassandra/schema/TableMetadata.java @@ -691,19 +691,14 @@ public String indexTableName(IndexMetadata info) } /** - * @return true if the change as made impacts queries/updates on the table, - * e.g. any columns or indexes were added, removed, or altered; otherwise, false is returned. + * @return true if the change as made impacts queries/updates on the table, effectively this is + * true if the metadata has changed in any way, as replicas compare metadata epochs + * when performing reads & writes. * Used to determine whether prepared statements against this table need to be re-prepared. */ boolean changeAffectsPreparedStatements(TableMetadata updated) { - return !partitionKeyColumns.equals(updated.partitionKeyColumns) - || !clusteringColumns.equals(updated.clusteringColumns) - || !regularAndStaticColumns.equals(updated.regularAndStaticColumns) - || !indexes.equals(updated.indexes) - || params.defaultTimeToLive != updated.params.defaultTimeToLive - || params.gcGraceSeconds != updated.params.gcGraceSeconds - || ( !Flag.isCQLTable(flags) && Flag.isCQLTable(updated.flags) ); + return epoch.isBefore(updated.epoch); } /** diff --git a/src/java/org/apache/cassandra/schema/Views.java b/src/java/org/apache/cassandra/schema/Views.java index 054db8bbcac5..dc5e7556d06d 100644 --- a/src/java/org/apache/cassandra/schema/Views.java +++ b/src/java/org/apache/cassandra/schema/Views.java @@ -68,6 +68,11 @@ public static Views none() return NONE; } + public static Views of(Iterable<ViewMetadata> views) + { + return builder().put(views).build(); + } + public Iterator<ViewMetadata> iterator() { return views.values().iterator(); @@ -232,7 +237,7 @@ static ViewsDiff diff(Views before, Views after) return ViewsDiff.diff(before, after); } - static final class ViewsDiff extends Diff<Views, ViewMetadata> + public static final class ViewsDiff extends Diff<Views, ViewMetadata> { private static final ViewsDiff NONE = new ViewsDiff(Views.none(), Views.none(), ImmutableList.of()); diff --git a/src/java/org/apache/cassandra/tcm/transformations/AlterSchema.java b/src/java/org/apache/cassandra/tcm/transformations/AlterSchema.java index 93692f55ba93..cec1d42ca86d 100644 --- a/src/java/org/apache/cassandra/tcm/transformations/AlterSchema.java +++ b/src/java/org/apache/cassandra/tcm/transformations/AlterSchema.java @@ -43,6 +43,8 @@ import org.apache.cassandra.schema.SchemaTransformation; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.Tables; +import org.apache.cassandra.schema.ViewMetadata; +import org.apache.cassandra.schema.Views; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.Epoch; @@ -163,8 +165,9 @@ public final Result execute(ClusterMetadata prev) if (!keyspacesByReplication.containsKey(newKSM.params.replication)) affectsPlacements.add(newKSM); - Tables tables = Tables.of(normaliseEpochs(nextEpoch, newKSM.tables.stream())); - newKeyspaces = newKeyspaces.withAddedOrUpdated(newKSM.withSwapped(tables)); + Tables tables = Tables.of(normaliseTableEpochs(nextEpoch, newKSM.tables.stream())); + Views views = Views.of(normaliseViewEpochs(nextEpoch, newKSM.views.stream())); + newKeyspaces = newKeyspaces.withAddedOrUpdated(newKSM.withSwapped(tables).withSwapped(views)); } // Scan modified keyspaces to check for replication changes and to ensure that any modified table metadata @@ -175,12 +178,20 @@ public final Result execute(ClusterMetadata prev) affectsPlacements.add(alteredKSM.before); Tables tables = Tables.of(alteredKSM.after.tables); - for (TableMetadata created : normaliseEpochs(nextEpoch, alteredKSM.tables.created.stream())) + for (TableMetadata created : normaliseTableEpochs(nextEpoch, alteredKSM.tables.created.stream())) tables = tables.withSwapped(created); - for (TableMetadata altered : normaliseEpochs(nextEpoch, alteredKSM.tables.altered.stream().map(altered -> altered.after))) + for (TableMetadata altered : normaliseTableEpochs(nextEpoch, alteredKSM.tables.altered.stream().map(altered -> altered.after))) tables = tables.withSwapped(altered); - newKeyspaces = newKeyspaces.withAddedOrUpdated(alteredKSM.after.withSwapped(tables)); + + Views views = Views.of(alteredKSM.after.views); + for (ViewMetadata created : normaliseViewEpochs(nextEpoch, alteredKSM.views.created.stream())) + views = views.withSwapped(created); + + for (ViewMetadata altered : normaliseViewEpochs(nextEpoch, alteredKSM.views.altered.stream().map(altered -> altered.after))) + views = views.withSwapped(altered); + + newKeyspaces = newKeyspaces.withAddedOrUpdated(alteredKSM.after.withSwapped(tables).withSwapped(views)); } // Changes which affect placement (i.e. new, removed or altered replication settings) are not allowed if there @@ -236,7 +247,7 @@ private static Map<ReplicationParams, Set<KeyspaceMetadata>> groupByReplication( return byReplication; } - private static Iterable<TableMetadata> normaliseEpochs(Epoch nextEpoch, Stream<TableMetadata> tables) + private static Iterable<TableMetadata> normaliseTableEpochs(Epoch nextEpoch, Stream<TableMetadata> tables) { return tables.map(tm -> tm.epoch.is(nextEpoch) ? tm @@ -244,6 +255,13 @@ private static Iterable<TableMetadata> normaliseEpochs(Epoch nextEpoch, Stream<T .collect(Collectors.toList()); } + private static Iterable<ViewMetadata> normaliseViewEpochs(Epoch nextEpoch, Stream<ViewMetadata> views) + { + return views.map(vm -> vm.metadata.epoch.is(nextEpoch) + ? vm + : vm.copy(vm.metadata.unbuild().epoch(nextEpoch).build())) + .collect(Collectors.toList()); + } static class Serializer implements AsymmetricMetadataSerializer<Transformation, AlterSchema> { diff --git a/test/distributed/org/apache/cassandra/distributed/test/PreparedStatementInvalidationTest.java b/test/distributed/org/apache/cassandra/distributed/test/PreparedStatementInvalidationTest.java new file mode 100644 index 000000000000..fe58dc91a93b --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/PreparedStatementInvalidationTest.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import org.junit.Test; + +import com.datastax.driver.core.PreparedStatement; +import com.datastax.driver.core.Session; +import org.apache.cassandra.distributed.api.ICluster; +import org.apache.cassandra.distributed.api.IInvokableInstance; + +import static org.apache.cassandra.distributed.api.Feature.GOSSIP; +import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL; +import static org.apache.cassandra.distributed.api.Feature.NETWORK; + +public class PreparedStatementInvalidationTest extends TestBaseImpl +{ + @Test + public void testInvalidation() throws Exception + { + try (ICluster<IInvokableInstance> c = init(builder().withNodes(2) + .withConfig(config -> config.with(GOSSIP, NETWORK, NATIVE_PROTOCOL)) + .start())) + { + try (com.datastax.driver.core.Cluster cluster = com.datastax.driver.core.Cluster.builder() + .addContactPoint("127.0.0.1") + .build(); + Session s = cluster.connect()) + { + s.execute(withKeyspace("CREATE TABLE %s.tbl (pk int primary key)")); + PreparedStatement prepared = s.prepare(withKeyspace("select pk from %s.tbl where pk = ?")); + s.execute(prepared.bind(1)); + s.execute(withKeyspace("alter table %s.tbl with speculative_retry='none'")); + s.execute(prepared.bind(2)); + } + } + } +} \ No newline at end of file From 8cd52f18a69c86044154c02065716fc853223f73 Mon Sep 17 00:00:00 2001 From: Brad Schoening <bschoeni@gmail.com> Date: Fri, 21 Feb 2025 21:26:42 +0100 Subject: [PATCH 141/225] Improve documentation in cql_singlefile.adoc patch by Brad Schoening; reviewed by Stefan Miklosovic for CASSANDRA-20354 --- .../pages/developing/cql/cql_singlefile.adoc | 730 ++++-------------- 1 file changed, 159 insertions(+), 571 deletions(-) diff --git a/doc/modules/cassandra/pages/developing/cql/cql_singlefile.adoc b/doc/modules/cassandra/pages/developing/cql/cql_singlefile.adoc index d00661338e92..5b39524b380a 100644 --- a/doc/modules/cassandra/pages/developing/cql/cql_singlefile.adoc +++ b/doc/modules/cassandra/pages/developing/cql/cql_singlefile.adoc @@ -16,7 +16,7 @@ between the different versions of CQL v3. CQL v3 offers a model very close to SQL in the sense that data is put in _tables_ containing _rows_ of _columns_. For that reason, when used in this document, these terms (tables, rows and columns) have the same -definition than they have in SQL. But please note that as such, they do +definition as they have in SQL. But please note that as such, they do *not* refer to the concept of rows and columns found in the internal implementation of Cassandra and in the thrift and CQL v2 API. @@ -62,18 +62,18 @@ A number of such identifiers, like `SELECT` or `WITH`, are _keywords_. They have a fixed meaning for the language and most are reserved. The list of those keywords can be found in link:#appendixA[Appendix A]. -Identifiers and (unquoted) keywords are case insensitive. Thus `SELECT` -is the same than `select` or `sElEcT`, and `myId` is the same than +Identifiers and (unquoted) keywords are case-insensitive. Thus `SELECT` +is the same as `select` or `sElEcT`, and `myId` is the same as `myid` or `MYID` for instance. A convention often used (in particular by the samples of this documentation) is to use upper case for keywords and lower case for other identifiers. There is a second kind of identifiers called _quoted identifiers_ defined by enclosing an arbitrary sequence of characters in -double-quotes(`"`). Quoted identifiers are never keywords. Thus +double-quotes(`"`). Quoted identifiers are never keywords. Thus, `"select"` is not a reserved keyword and can be used to refer to a column, while `select` would raise a parse error. Also, contrarily to -unquoted identifiers and keywords, quoted identifiers are case sensitive +unquoted identifiers and keywords, quoted identifiers are case-sensitive (`"My Quoted Id"` is _different_ from `"my quoted id"`). A fully lowercase quoted identifier that matches `[a-zA-Z]``[a-zA-Z0-9_]``*` is equivalent to the unquoted identifier obtained by removing the @@ -82,7 +82,7 @@ different from `"myId"`). Inside a quoted identifier, the double-quote character can be repeated to escape it, so `"foo "" bar"` is a valid identifier. -*Warning*: _quoted identifiers_ allows to declare columns with arbitrary +*Warning*: _quoted identifiers_ allow for declaring columns with arbitrary names, and those can sometime clash with specific names used by the server. For instance, when using conditional update, the server will respond with a result-set containing a special result named @@ -98,7 +98,7 @@ function call (like `"f(x)"`). CQL defines the following kind of _constants_: strings, integers, floats, booleans, uuids and blobs: -* A string constant is an arbitrary sequence of characters characters +* A string constant is an arbitrary sequence of characters enclosed by single-quote(`'`). One can include a single-quote in a string by repeating it, e.g. `'It''s raining today'`. Those are not to be confused with quoted identifiers that use double-quotes. @@ -110,10 +110,10 @@ be confused with quoted identifiers that use double-quotes. case-insensitivity (i.e. `True` is a valid boolean constant). * A http://en.wikipedia.org/wiki/Universally_unique_identifier[UUID] constant is defined by `hex{8}-hex{4}-hex{4}-hex{4}-hex{12}` where `hex` -is an hexadecimal character, e.g. `[0-9a-fA-F]` and `{4}` is the number +is a hexadecimal character, e.g. `[0-9a-fA-F]` and `{4}` is the number of such characters. -* A blob constant is an hexadecimal number defined by `0[xX](hex)+` -where `hex` is an hexadecimal character, e.g. `[0-9a-fA-F]`. +* A blob constant is a hexadecimal number defined by `0[xX](hex)+` +where `hex` is a hexadecimal character, e.g. `[0-9a-fA-F]`. For how these constants are typed, see the link:#types[data types section]. @@ -123,7 +123,7 @@ section]. CQL has the notion of a _term_, which denotes the kind of values that CQL support. Terms are defined by: -[source, bnf] +[source,bnf] ---- include::cassandra:example$BNF/term.bnf[] ---- @@ -220,12 +220,12 @@ will be valid in practice. Most notably, `<variable>` and nested A `<variable>` can be either anonymous (a question mark (`?`)) or named (an identifier preceded by `:`). Both declare a bind variables for link:#preparedStatement[prepared statements]. The only difference -between an anymous and a named variable is that a named one will be +between an anonymous and a named variable is that a named one will be easier to refer to (how exactly depends on the client driver used). The `<properties>` production is use by statement that create and alter keyspaces and tables. Each `<property>` is either a _simple_ one, in -which case it just has a value, or a _map_ one, in which case it’s value +which case it just has a value, or a _map_ one, in which case its value is a map grouping sub-options. The following will refer to one or the other as the _kind_ (_simple_ or _map_) of the property. @@ -263,7 +263,7 @@ multiple times with different concrete values. In a statement, each time a column value is expected (in the data manipulation and query statements), a `<variable>` (see above) can be used instead. A statement with bind variables must then be _prepared_. -Once it has been prepared, it can executed by providing concrete values +Once prepared, it can executed by providing concrete values for the bind variables. The exact procedure to prepare a statement and execute a prepared statement depends on the CQL driver used and is beyond the scope of this document. @@ -282,7 +282,7 @@ bind markers are used, the names for the query parameters will be _Syntax:_ [source,bnf] -::= CREATE KEYSPACE (IF NOT EXISTS)? WITH +include::cassandra:example$BNF/create_ks.bnf[] _Sample:_ @@ -302,8 +302,7 @@ The `CREATE KEYSPACE` statement creates a new top-level _keyspace_. A keyspace is a namespace that defines a replication strategy and some options for a set of tables. Valid keyspaces names are identifiers composed exclusively of alphanumerical characters and whose length is -lesser or equal to 32. Note that as identifiers, keyspace names are case -insensitive: use a quoted identifier for case sensitive keyspace names. +lesser or equal to 32. Note that as identifiers, keyspace names are case-insensitive: use a quoted identifier for case sensitive keyspace names. The supported `<properties>` for `CREATE KEYSPACE` are: @@ -342,12 +341,12 @@ will be a no-op if the keyspace already exists. _Syntax:_ [source,bnf] -::= USE +include::cassandra:example$BNF/use_ks.bnf[] _Sample:_ [source,sql] -USE myApp; +include::cassandra:example$CQL/use_ks.cql[] The `USE` statement takes an existing keyspace name as argument and set it as the per-connection current working keyspace. All subsequent @@ -361,14 +360,12 @@ another USE statement is issued or the connection terminates. _Syntax:_ [source,bnf] -::= ALTER KEYSPACE (IF EXISTS)? WITH +include::cassandra:example$BNF/alter_ks.bnf[] _Sample:_ [source,sql] -ALTER KEYSPACE Excelsior -WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : -4}; +include::cassandra:example$CQL/alter_ks.cql[] The `ALTER KEYSPACE` statement alters the properties of an existing keyspace. The supported `<properties>` are the same as for the @@ -385,13 +382,13 @@ _Syntax:_ _Sample:_ [source,sql] - DROP KEYSPACE myApp; +DROP KEYSPACE myApp; A `DROP KEYSPACE` statement results in the immediate, irreversible removal of an existing keyspace, including all column families in it, and all data contained in those column families. -If the keyspace does not exists, the statement will return an error, +If the keyspace does not exist, the statement will return an error, unless `IF EXISTS` is used in which case the operation is a no-op. [[createTableStmt]] @@ -400,42 +397,15 @@ unless `IF EXISTS` is used in which case the operation is a no-op. _Syntax:_ [source,bnf] ----- -::= CREATE ( TABLE | COLUMNFAMILY ) ( IF NOT EXISTS )? -`(' ( `,' )* `)' -( WITH ( AND )* )? - -::= ( STATIC )? ( PRIMARY KEY )? -| PRIMARY KEY `(' ( `,' )* `)' - -::= -| `(' (`,' )* `)' - -::= -| COMPACT STORAGE -| CLUSTERING ORDER ----- +include::cassandra:example$BNF/create_table.bnf[] _Sample:_ [source,sql] ---- -CREATE TABLE monkeySpecies ( -species text PRIMARY KEY, -common_name text, -population varint, -average_size int -) WITH comment='Important biological records'; - -CREATE TABLE timeline ( -userid uuid, -posted_month int, -posted_time uuid, -body text, -posted_by text, -PRIMARY KEY (userid, posted_month, posted_time) -) WITH compaction = { 'class' : 'LeveledCompactionStrategy' }; +include::cassandra:example$CQL/create_table.cql[] ---- + The `CREATE TABLE` statement creates a new table. Each such table is a set of _rows_ (usually representing related entities) for which it defines a number of properties. A table is defined by a @@ -523,16 +493,10 @@ partition (having the same partition key). For instance, in: [source,sql] ---- -CREATE TABLE test ( -pk int, -t int, -v text, -s text static, -PRIMARY KEY (pk, t) -); -INSERT INTO test(pk, t, v, s) VALUES (0, 0, 'val0', 'static0'); -INSERT INTO test(pk, t, v, s) VALUES (0, 1, 'val1', 'static1'); -SELECT * FROM test WHERE pk=0 AND t=0; +include::cassandra:example$CQL/create_static_column.cql[] + +include::cassandra:example$CQL/insert_static_data.cql[] +include::cassandra:example$CQL/select_static_data.cql[] ---- the last query will return `'static1'` as value for `s`, since `s` is @@ -632,14 +596,14 @@ supported by the default classes are: enabled or not. |`tombstone_threshold` |_all_ |0.2 |A ratio such that if a sstable has -more than this ratio of gcable tombstones over all contained columns, +more than this ratio of GC eligable tombstones over all contained columns, the sstable will be compacted (with no other sstables) for the purpose of purging those tombstones. |`tombstone_compaction_interval` |_all_ |1 day |The minimum time to wait after an sstable creation time before considering it for ``tombstone compaction'', where ``tombstone compaction'' is the compaction triggered -if the sstable has more gcable tombstones than `tombstone_threshold`. +if the sstable has more GC eligible tombstones than `tombstone_threshold`. |`unchecked_tombstone_compaction` |_all_ |false |Setting this to true enables more aggressive tombstone compactions - single sstable tombstone @@ -649,7 +613,7 @@ successful. |`min_sstable_size` |SizeTieredCompactionStrategy |50MB |The size tiered strategy groups SSTables to compact in buckets. A bucket groups SSTables that differs from less than 50% in size. However, for small sizes, this -would result in a bucketing that is too fine grained. `min_sstable_size` +would result in a bucketing that is too fine-grained. `min_sstable_size` defines a size threshold (in bytes) below which all SSTables belong to one unique bucket @@ -677,7 +641,7 @@ given partition key are never split into 2 sstables |`timestamp_resolution` |TimeWindowCompactionStrategy |MICROSECONDS |The timestamp resolution used when inserting data, could be MILLISECONDS, -MICROSECONDS etc (should be understandable by Java TimeUnit) - don’t +MICROSECONDS etc. (should be understandable by Java TimeUnit) - don’t change this unless you do mutations with USING TIMESTAMP (or equivalent directly in the client) @@ -711,7 +675,7 @@ Use `'enabled' : false` to disable compression. Custom compressor can be provided by specifying the full class name as a link:#constants[string constant]. | | | -|`enabled` |true |By default compression is enabled. To disable it, set +|`enabled` |true |By default, compression is enabled. To disable it, set `enabled` to `false` |`chunk_length_in_kb` |64KB |On disk SSTables are compressed by block (to allow random reads). This defines the size (in KB) of said block. Bigger values may improve the compression rate, but @@ -750,28 +714,15 @@ haven’t) when creating a table. _Syntax:_ [source,bnf] ----- -::= ALTER (TABLE | COLUMNFAMILY) (IF EXISTS)? +include::cassandra:example$BNF/alter_table.bnf[] -::= ADD (IF NOT EXISTS)? -| ADD (IF NOT EXISTS)? ( ( , )* ) -| DROP (IF EXISTS)? -| DROP (IF EXISTS)? ( ( , )* ) -| RENAME (IF EXISTS)? TO (AND TO)* -| WITH ( AND )* ----- _Sample:_ [source,sql] ---- -ALTER TABLE addamsFamily - -ALTER TABLE addamsFamily -ADD gravesite varchar; - -ALTER TABLE addamsFamily -WITH comment = 'A most excellent and useful column family'; +include::cassandra:example$CQL/alter_table_add_column.cql[] +include::cassandra:example$CQL/alter_table_with_comment.cql[] ---- The `ALTER` statement is used to manipulate table definitions. It allows @@ -789,13 +740,13 @@ cannot be added to tables defined with the `COMPACT STORAGE` option. If the new column already exists, the statement will return an error, unless `IF NOT EXISTS` is used in which case the operation is a no-op. * `DROP`: Removes a column from the table. Dropped columns will immediately become unavailable in the queries and will not be included -in compacted sstables in the future. If a column is readded, queries +in compacted sstables in the future. If a column is read, queries won’t return values written before the column was last dropped. It is assumed that timestamps represent actual time, so if this is not your case, you should NOT read previously dropped columns. Columns can’t be dropped from tables defined with the `COMPACT STORAGE` option. If the dropped column does not already exist, the statement will return an error, unless `IF EXISTS` is used in which case the operation is a no-op. -* `RENAME` a primary key column of a table. Non primary key columns cannot be renamed. +* `RENAME` a primary key column of a table. Non-primary key columns cannot be renamed. Furthermore, renaming a column to another name which already exists isn't allowed. It's important to keep in mind that renamed columns shouldn't have dependent seconday indexes. If the renamed column does not already exist, the statement will return an error, unless `IF EXISTS` is used in which case the operation is a no-op. @@ -853,12 +804,12 @@ conversions are allowed. _Syntax:_ [source,bnf] -::= DROP TABLE ( IF EXISTS )? +include::cassandra:example$BNF/drop_table.bnf[] _Sample:_ [source,sql] - DROP TABLE worldSeriesAttendees; +DROP TABLE worldSeriesAttendees; The `DROP TABLE` statement results in the immediate, irreversible removal of a table, including all data contained in it. As for table @@ -873,12 +824,12 @@ If the table does not exist, the statement will return an error, unless _Syntax:_ [source,bnf] -::= TRUNCATE ( TABLE | COLUMNFAMILY )? +include::cassandra:example$BNF/truncate_table.bnf[] _Sample:_ [source,sql] - TRUNCATE superImportantData; +TRUNCATE superImportantData; The `TRUNCATE` statement permanently removes all data from a table. @@ -893,27 +844,12 @@ desired. _Syntax:_ [source,bnf] ----- -::= CREATE ( CUSTOM )? INDEX ( IF NOT EXISTS )? ( )? -ON `(' `)' -( USING ( WITH OPTIONS = )? )? - -::= -| keys( ) ----- +include::cassandra:example$BNF/create_index_statement.bnf[] _Sample:_ [source,sql] ----- -CREATE INDEX userIndex ON NerdMovies (user); -CREATE INDEX ON Mutants (abilityId); -CREATE INDEX ON users (keys(favs)); -CREATE INDEX ON users (age) USING 'sai'; -CREATE CUSTOM INDEX ON users (email) USING 'path.to.the.IndexClass'; -CREATE CUSTOM INDEX ON users (email) USING 'path.to.the.IndexClass' WITH -OPTIONS = {'storage': '/mnt/ssd/indexes/'}; ----- +include::cassandra:example$CQL/create_index.cql[] If data already exists for the column, it will be indexed asynchronously. After the index is created, new data for the column is @@ -927,7 +863,7 @@ If it is used, the statement will be a no-op if the index already exists. The `USING` keyword optionally specifies an index type. There are two built-in types: - legacy_local_table - (default) legacy secondary index, implemented as a hidden local table -- sai - "storage-attched" index, implemented via optimized SSTable/Memtable-attached indexes +- sai - "storage-attached" index, implemented via optimized SSTable/Memtable-attached indexes To create a custom index, a fully qualified class name must be specified. @@ -946,7 +882,7 @@ the map values. _Syntax:_ [source,bnf] -DROP INDEX ( IF EXISTS )? ('.')? +include::cassandra:example$BNF/drop_index_statement.bnf[] _Sample:_ @@ -960,7 +896,7 @@ The `DROP INDEX` statement is used to drop an existing secondary index. The argument of the statement is the index name, which may optionally specify the keyspace of the index. -If the index does not exists, the statement will return an error, unless +If the index does not exist, the statement will return an error, unless `IF EXISTS` is used in which case the operation is a no-op. [[createMVStmt]] @@ -969,29 +905,15 @@ If the index does not exists, the statement will return an error, unless _Syntax:_ [source,bnf] ----- -::= CREATE MATERIALIZED VIEW ( IF NOT EXISTS )? AS -SELECT ( `(' ( `,' ) * `)' | `*' ) -FROM -( WHERE )? -PRIMARY KEY `(' ( `,' )* `)' -( WITH ( AND )* )? ----- +include::cassandra:example$BNF/create_mv_statement.bnf[] _Sample:_ [source,sql] ----- -CREATE MATERIALIZED VIEW monkeySpecies_by_population AS -SELECT * -FROM monkeySpecies -WHERE population IS NOT NULL AND species IS NOT NULL -PRIMARY KEY (population, species) -WITH comment='Allow query by population instead of species'; ----- +include::cassandra:example$CQL/create_mv_statement.cql[] The `CREATE MATERIALIZED VIEW` statement creates a new materialized -view. Each such view is a set of _rows_ which corresponds to rows which +view. Each such view is a set of _rows_ corresponding to rows which are present in the underlying, or base, table specified in the `SELECT` statement. A materialized view cannot be directly updated, but updates to the base table will cause corresponding updates in the view. @@ -1020,7 +942,7 @@ _Syntax:_ [source,bnf] ::= ALTER MATERIALIZED VIEW WITH ( AND )* -The `ALTER MATERIALIZED VIEW` statement allows options to be update; +The `ALTER MATERIALIZED VIEW` statement allows options to be updated; these options are the same as `CREATE TABLE`’s options. [[dropMVStmt]] @@ -1029,7 +951,7 @@ these options are the same as `CREATE TABLE`’s options. _Syntax:_ [source,bnf] -::= DROP MATERIALIZED VIEW ( IF EXISTS )? +include::cassandra:example$BNF/drop_mv_statement.bnf[] _Sample:_ @@ -1039,7 +961,7 @@ DROP MATERIALIZED VIEW monkeySpecies_by_population; The `DROP MATERIALIZED VIEW` statement is used to drop an existing materialized view. -If the materialized view does not exists, the statement will return an +If the materialized view does not exist, the statement will return an error, unless `IF EXISTS` is used in which case the operation is a no-op. @@ -1049,14 +971,7 @@ no-op. _Syntax:_ [source,bnf] ----- -::= CREATE TYPE ( IF NOT EXISTS )? -`(' ( `,' )* `)' - -::= ( `.' )? - -::= ----- +include::cassandra:example$BNF/create_type.bnf[] _Sample:_ @@ -1141,13 +1056,12 @@ is used, in which case the operation is a no-op. _Syntax:_ [source,bnf] -::= CREATE TRIGGER ( IF NOT EXISTS )? ( )? ON USING +include::cassandra:example$BNF/create_trigger_statement.bnf[] _Sample:_ [source,sql] -CREATE TRIGGER myTrigger ON myTable USING -'org.apache.cassandra.triggers.InvertedIndex'; +include::cassandra:example$CQL/create_trigger.cql[] The actual logic that makes up the trigger can be written in any Java (JVM) language and exists outside the database. You place the trigger @@ -1163,12 +1077,12 @@ the transaction. _Syntax:_ [source,bnf] -::= DROP TRIGGER ( IF EXISTS )? ( )? ON +include::cassandra:example$BNF/drop_trigger_statement.bnf[] _Sample:_ [source,sql] -DROP TRIGGER myTrigger ON myTable; +include::cassandra:example$CQL/drop_trigger.cql[] `DROP TRIGGER` statement removes the registration of a trigger created using `CREATE TRIGGER`. @@ -1180,35 +1094,15 @@ _Syntax:_ [source,bnf] ---- -::= CREATE ( OR REPLACE )? -FUNCTION ( IF NOT EXISTS )? -( `.' )? -`(' ( `,' )* `)' -( CALLED | RETURNS NULL ) ON NULL INPUT -RETURNS -LANGUAGE -AS +include::cassandra:example$BNF/create_function_statement.bnf[] ---- _Sample:_ [source,sql] -CREATE OR REPLACE FUNCTION somefunction -( somearg int, anotherarg text, complexarg frozen, listarg list ) -RETURNS NULL ON NULL INPUT -RETURNS text -LANGUAGE java -AS $$ -// some Java code -$$; -CREATE FUNCTION akeyspace.fname IF NOT EXISTS -( someArg int ) -CALLED ON NULL INPUT -RETURNS text -LANGUAGE java -AS $$ -// some Java code -$$; +---- +include::cassandra:example$CQL/create_function.cql[] +---- `CREATE FUNCTION` creates or replaces a user-defined function. @@ -1258,19 +1152,12 @@ information. _Syntax:_ [source,bnf] -::= DROP FUNCTION ( IF EXISTS )? -( `.' )? -( `(' ( `,' )* `)' )? +include::cassandra:example$BNF/drop_function_statement.bnf[] _Sample:_ [source,sql] ----- -DROP FUNCTION myfunction; -DROP FUNCTION mykeyspace.afunction; -DROP FUNCTION afunction ( int ); -DROP FUNCTION afunction ( text ); ----- +include::cassandra:example$CQL/drop_function.cql[] `DROP FUNCTION` statement removes a function created using `CREATE FUNCTION`. + @@ -1287,16 +1174,7 @@ if it exists. _Syntax:_ [source,bnf] ----- -::= CREATE ( OR REPLACE )? -AGGREGATE ( IF NOT EXISTS )? -( `.' )? -`(' ( `,' )* `)' -SFUNC -STYPE -( FINALFUNC )? -( INITCOND )? ----- +include::cassandra:example$BNF/create_aggregate_statement.bnf[] _Sample:_ @@ -1365,21 +1243,12 @@ information. _Syntax:_ [source,bnf] ----- -::= DROP AGGREGATE ( IF EXISTS )? -( `.' )? -( `(' ( `,' )* `)' )? ----- +include::cassandra:example$BNF/drop_aggregate_statement.bnf[] _Sample:_ [source,sql] ----- -DROP AGGREGATE myAggregate; -DROP AGGREGATE myKeyspace.anAggregate; -DROP AGGREGATE someAggregate ( int ); -DROP AGGREGATE someAggregate ( text ); ----- +include::cassandra:example$CQL/drop_aggregate.cql[] The `DROP AGGREGATE` statement removes an aggregate created using `CREATE AGGREGATE`. You must specify the argument types of the aggregate @@ -1403,30 +1272,14 @@ _Syntax:_ [source,bnf] ---- -::= INSERT INTO -( ( VALUES ) -| ( JSON )) -( IF NOT EXISTS )? -( USING ( AND )* )? - -::= `(' ( `,' )* `)' - -::= `(' ( `,' )* `)' - -::= TIMESTAMP -| TTL +include::cassandra:example$BNF/insert_statement.bnf[] ---- _Sample:_ [source,sql] ---- -INSERT INTO NerdMovies (movie, director, main_actor, year) -VALUES ('Serenity', 'Joss Whedon', 'Nathan Fillion', 2005) -USING TTL 86400; - -INSERT INTO NerdMovies JSON '{"movie": "Serenity", "director": -"Joss Whedon", "year": 2005}' +include::cassandra:example$CQL/insert_statement.cql[] ---- The `INSERT` statement writes one or more columns for a given row in a @@ -1443,7 +1296,7 @@ creation or update happened. It is however possible to use the `IF NOT EXISTS` condition to only insert if the row does not exist prior to the insertion. But please note -that using `IF NOT EXISTS` will incur a non negligible performance cost +that using `IF NOT EXISTS` will incur a non-negligible performance cost (internally, Paxos will be used) so this should be used sparingly. All updates for an `INSERT` are applied atomically and in isolation. @@ -1461,54 +1314,14 @@ _Syntax:_ [source,bnf] ---- -::= UPDATE -( USING ( AND )* )? -SET ( `,' )* -WHERE -( IF ( AND condition )* )? - -::= `=' -| `=' (`+' | `-') ( | | ) -| `=' `+' -| `[' `]' `=' -| `.' `=' - -::= -| CONTAINS (KEY)? -| IN -| `[' `]' -| `[' `]' IN -| `.' -| `.' IN - -::= `<' | `<=' | `=' | `!=' | `>=' | `>' -::= ( | `(' ( ( `,' )* )? `)') - -::= ( AND )* - -::= `=' -| `(' (`,' )* `)' `=' -| IN `(' ( ( `,' )* )? `)' -| IN -| `(' (`,' )* `)' IN `(' ( ( `,' )* )? `)' -| `(' (`,' )* `)' IN - -::= TIMESTAMP -| TTL +include::cassandra:example$BNF/update_statement.bnf[] ---- _Sample:_ [source,sql] ---- -UPDATE NerdMovies USING TTL 400 -SET director = 'Joss Whedon', -main_actor = 'Nathan Fillion', -year = 2005 -WHERE movie = 'Serenity'; - -UPDATE UserActions SET total = total + 2 WHERE user = -B70DE1D0-9908-4AE3-BE34-5573E5B09F14 AND action = 'click'; +include::cassandra:example$CQL/update_statement.cql[] ---- The `UPDATE` statement writes one or more columns for a given row in a @@ -1532,7 +1345,7 @@ In an `UPDATE` statement, all updates within the same partition key are applied atomically and in isolation. The `c = c + 3` form of `<assignment>` is used to increment/decrement -counters. The identifier after the `=' sign *must* be the same than the +counters. The identifier after the `=' sign *must* be the same as the one before the `=' sign (Only increment/decrement is supported on counters, not the assignment of a specific value). @@ -1567,48 +1380,13 @@ TTL of 0 will remove the TTL for the inserted or updated values. _Syntax:_ [source,bnf] ----- -::= DELETE ( ( `,' )* )? -FROM -( USING TIMESTAMP )? -WHERE -( IF ( EXISTS | ( ( AND )*) ) )? - -::= -| `[' `]' -| `.' - -::= ( AND )* - -::= -| `(' (`,' )* `)' -| IN `(' ( ( `,' )* )? `)' -| IN -| `(' (`,' )* `)' IN `(' ( ( `,' )* )? `)' -| `(' (`,' )* `)' IN - -::= `=' | `<' | `>' | `<=' | `>=' -::= ( | `(' ( ( `,' )* )? `)') - -::= ( | `!=') -| CONTAINS (KEY)? -| IN -| `[' `]' ( | `!=') -| `[' `]' IN -| `.' ( | `!=') -| `.' IN ----- +include::cassandra:example$BNF/delete_statement.bnf[] _Sample:_ [source,sql] ---- -DELETE FROM NerdMovies USING TIMESTAMP 1240003134 WHERE movie = -'Serenity'; - -DELETE phone FROM Users WHERE userid IN -(C73DE1D3-AF08-40F3-B124-3FF3E5109F22, -B70DE1D0-9908-4AE3-BE34-5573E5B09F14); +include::cassandra:example$CQL/delete_statement.cql[] ---- The `DELETE` statement deletes columns and rows. If column names are @@ -1640,31 +1418,12 @@ sparingly. _Syntax:_ [source,bnf] ----- -::= BEGIN ( UNLOGGED | COUNTER ) BATCH -( USING ( AND )* )? -( `;' )* -APPLY BATCH - -::= -| -| - -::= TIMESTAMP ----- +include::cassandra:example$BNF/batch_statement.bnf[] _Sample:_ [source,sql] ----- -BEGIN BATCH -INSERT INTO users (userid, password, name) VALUES ('user2', 'ch@ngem3b', -'second user'); -UPDATE users SET password = 'ps22dhds' WHERE userid = 'user3'; -INSERT INTO users (userid, password) VALUES ('user4', 'ch@ngem3c'); -DELETE name FROM users WHERE userid = 'user1'; -APPLY BATCH; ----- +include::cassandra:example$CQL/batch_statement.cql[] The `BATCH` statement group multiple modification statements (insertions/updates and deletions) into a single statement. It serves @@ -1728,66 +1487,14 @@ _Syntax:_ [source,bnf] ---- -::= SELECT ( JSON )? -FROM -( WHERE )? -( GROUP BY )? -( ORDER BY )? -( PER PARTITION LIMIT )? -( LIMIT )? -( ALLOW FILTERING )? - -::= DISTINCT? - -::= (AS )? ( `,' (AS )? )* -| `*' - -::= -| -| WRITETIME `(' `)' -| MAXWRITETIME `(' `)' -| COUNT `(' `*' `)' -| TTL `(' `)' -| CAST `(' AS `)' -| `(' ( (`,' )*)? `)' -| `.' -| `[' `]' -| `[' ? .. ? `]' - -::= ( AND )* - -::= -| `(' (`,' )* `)' -| IN `(' ( ( `,' )* )? `)' -| `(' (`,' )* `)' IN `(' ( ( `,' )* )? `)' -| TOKEN `(' ( `,' )* `)' - -::= `=' | `<' | `>' | `<=' | `>=' | CONTAINS | CONTAINS KEY | NOT CONTAINS | NOT CONTAINS KEY -::= (`,' )* -::= ( `,' )* -::= ( ASC | DESC )? -::= `(' (`,' )* `)' +include::cassandra:example$BNF/select_statement.bnf[] ---- _Sample:_ [source,sql] ---- -SELECT name, occupation FROM users WHERE userid IN (199, 200, 207); - -SELECT JSON name, occupation FROM users WHERE userid = 199; - -SELECT name AS user_name, occupation AS user_occupation FROM users; - -SELECT time, value -FROM events -WHERE event_type = 'myEvent' -AND time > '2011-02-03' -AND time <= '2012-01-01' - -SELECT COUNT (*) FROM users; - -SELECT COUNT (*) AS user_count FROM users; +include::cassandra:example$CQL/select_statement.cql[] ---- The `SELECT` statements reads one or more columns for one or more rows @@ -1800,7 +1507,7 @@ more details. [[selectSelection]] ===== `<select-clause>` -The `<select-clause>` determines which columns needs to be queried and +The `<select-clause>` determines which columns need to be queried and returned in the result-set. It consists of either the comma-separated list of or the wildcard character (`*`) to select all the columns defined for the table. Please note that for wildcard `SELECT` queries @@ -1903,7 +1610,7 @@ relation using the tuple notation. For instance: SELECT * FROM posts WHERE userid='john doe' AND (blog_title, posted_at) > ('John''s Blog', '2012-01-01') -will request all rows that sorts after the one having ``John’s Blog'' as +will request all rows that sort after the one having ``John’s Blog'' as `blog_tile` and `2012-01-01' for `posted_at` in the clustering order. In particular, rows having a `post_at <= '2012-01-01'` will be returned as long as their `blog_title > 'John''s Blog'`, which wouldn’t be the case @@ -1931,12 +1638,12 @@ applies to the map keys. The `ORDER BY` option allows to select the order of the returned results. It takes as argument a list of column names along with the order for the column (`ASC` for ascendant and `DESC` for descendant, -omitting the order being equivalent to `ASC`). Currently the possible +omitting the order being equivalent to `ASC`). Currently, the possible orderings are limited (which depends on the table link:#createTableOptions[`CLUSTERING ORDER`] ): * if the table has been defined without any specific `CLUSTERING ORDER`, -then then allowed orderings are the order induced by the clustering +then allowed orderings are the order induced by the clustering columns and the reverse of that one. * otherwise, the orderings allowed are the order of the `CLUSTERING ORDER` option and the reversed one. @@ -1944,7 +1651,7 @@ columns and the reverse of that one. [[selectGroupBy]] ===== `<group-by>` -The `GROUP BY` option allows to condense into a single row all selected +The `GROUP BY` option allows aggregating values into a single row all selected rows that share the same values for a set of columns. Using the `GROUP BY` option, it is only possible to group rows at the @@ -1968,7 +1675,7 @@ returned. The `LIMIT` option to a `SELECT` statement limits the number of rows returned by a query, while the `PER PARTITION LIMIT` option limits the number of rows returned for a given partition by the query. Note that -both type of limit can used in the same statement. +both type of limits can be used in the same statement. [[selectAllowFiltering]] ===== `ALLOW FILTERING` @@ -1992,30 +1699,20 @@ their year of birth (with a secondary index on it) and country of residence: [source,sql] ----- -CREATE TABLE users ( -username text PRIMARY KEY, -firstname text, -lastname text, -birth_year int, -country text -) - -CREATE INDEX ON users(birth_year); ----- +include::cassandra:example$CQL/allow_filtering.cql[] Then the following queries are valid: [source,sql] ---- SELECT * FROM users; -SELECT firstname, lastname FROM users WHERE birth_year = 1981; +SELECT firstname, lastname FROM users WHERE birth_year = 1981 ALLOW FILTERING; ---- because in both case, Cassandra guarantees that these queries performance will be proportional to the amount of data returned. In particular, if no users are born in 1981, then the second query -performance will not depend of the number of user profile stored in the +performance will not depend on the number of user profile stored in the database (not directly at least: due to secondary index implementation consideration, this query may still depend on the number of node in the cluster, which indirectly depends on the amount of data stored. @@ -2050,26 +1747,12 @@ country = 'FR' ALLOW FILTERING; _Syntax:_ [source,bnf] ----- -::= CREATE ROLE ( IF NOT EXISTS )? ( WITH ( AND )* )? - -::= PASSWORD = -| LOGIN = -| SUPERUSER = -| OPTIONS = ----- +include::cassandra:example$BNF/create_role_statement.bnf[] _Sample:_ [source,sql] ----- -CREATE ROLE new_role; -CREATE ROLE alice WITH PASSWORD = 'password_a' AND LOGIN = true; -CREATE ROLE bob WITH PASSWORD = 'password_b' AND LOGIN = true AND -SUPERUSER = true; -CREATE ROLE carlos WITH OPTIONS = { 'custom_option1' : 'option1_value', -'custom_option2' : 99 }; ----- +include::cassandra:example$CQL/create_role.cql[] By default roles do not possess `LOGIN` privileges or `SUPERUSER` status. @@ -2084,7 +1767,7 @@ If a role has the `LOGIN` privilege, clients may identify as that role when connecting. For the duration of that connection, the client will acquire any roles and privileges granted to that role. -Only a client with with the `CREATE` permission on the database roles +Only a client with the `CREATE` permission on the database roles resource may issue `CREATE ROLE` requests (see the link:#permissions[relevant section] below), unless the client is a `SUPERUSER`. Role management in Cassandra is pluggable and custom @@ -2108,10 +1791,7 @@ condition unless the `IF NOT EXISTS` option is used. If the option is used and the role exists, the statement is a no-op. [source,sql] ----- -CREATE ROLE other_role; -CREATE ROLE IF NOT EXISTS other_role; ----- +include::cassandra:example$CQL/create_role_ifnotexists.cql[] [[alterRoleStmt]] ==== ALTER ROLE @@ -2119,19 +1799,12 @@ CREATE ROLE IF NOT EXISTS other_role; _Syntax:_ [source,bnf] ----- -::= ALTER ROLE (IF EXISTS)? ( WITH ( AND )* )? - -::= PASSWORD = -| LOGIN = -| SUPERUSER = -| OPTIONS = ----- +include::cassandra:example$BNF/alter_role_statement.bnf[] _Sample:_ [source,sql] -ALTER ROLE bob WITH PASSWORD = 'PASSWORD_B' AND SUPERUSER = false; +include::cassandra:example$CQL/alter_role.cql[] If the role does not exist, the statement will return an error, unless `IF EXISTS` is used in which case the operation is a no-op. @@ -2152,7 +1825,7 @@ link:#permissions[permission] on that role _Syntax:_ [source,bnf] -::= DROP ROLE ( IF EXISTS )? +include::cassandra:example$BNF/drop_role_statement.bnf[] _Sample:_ @@ -2165,7 +1838,7 @@ DROP ROLE IF EXISTS bob; `DROP ROLE` requires the client to have `DROP` link:#permissions[permission] on the role in question. In addition, client may not `DROP` the role with which it identified at login. -Finaly, only a client with `SUPERUSER` status may `DROP` another +Finally, only a client with `SUPERUSER` status may `DROP` another `SUPERUSER` role. + Attempting to drop a role which does not exist results in an invalid query condition unless the `IF EXISTS` option is used. If the option is @@ -2177,12 +1850,12 @@ used and the role does not exist the statement is a no-op. _Syntax:_ [source,bnf] -::= GRANT TO +include::cassandra:example$BNF/grant_role_statement.bnf[] _Sample:_ [source,sql] -GRANT report_writer TO alice; +include::cassandra:example$CQL/grant_role.cql[] This statement grants the `report_writer` role to `alice`. Any permissions granted to `report_writer` are also acquired by `alice`. + @@ -2208,12 +1881,12 @@ GRANT role_c TO role_a; _Syntax:_ [source,bnf] -::= REVOKE FROM +include::cassandra:example$BNF/revoke_role_statement.bnf[] _Sample:_ [source,sql] -REVOKE report_writer FROM alice; +include::cassandra:example$CQL/revoke_role.cql[] This statement revokes the `report_writer` role from `alice`. Any permissions that `alice` has acquired via the `report_writer` role are @@ -2225,24 +1898,24 @@ also revoked. _Syntax:_ [source,bnf] -::= LIST ROLES ( OF )? ( NORECURSIVE )? +include::cassandra:example$BNF/list_roles_statement.bnf[] _Sample:_ [source,sql] -LIST ROLES; +include::cassandra:example$CQL/list_roles.cql[] Return all known roles in the system, this requires `DESCRIBE` permission on the database roles resource. [source,sql] -LIST ROLES OF 'alice'; +include::cassandra:example$CQL/list_roles_of.cql[] Enumerate all roles granted to `alice`, including those transitively -aquired. +acquired. [source,sql] -LIST ROLES OF 'bob' NORECURSIVE +include::cassandra:example$CQL/list_roles_nonrecursive.cql[] List all roles directly granted to `bob`. @@ -2252,47 +1925,24 @@ List all roles directly granted to `bob`. Prior to the introduction of roles in Cassandra 2.2, authentication and authorization were based around the concept of a `USER`. For backward compatibility, the legacy syntax has been preserved with `USER` centric -statments becoming synonyms for the `ROLE` based equivalents. +statements becoming synonyms for the `ROLE` based equivalents. _Syntax:_ [source,bnf] ----- -::= CREATE USER ( IF NOT EXISTS )? ( WITH PASSWORD )? ()? - -::= SUPERUSER | NOSUPERUSER ----- +include::cassandra:example$BNF/create_user_statement.bnf[] _Sample:_ [source,sql] ----- -CREATE USER alice WITH PASSWORD 'password_a' SUPERUSER; -CREATE USER bob WITH PASSWORD 'password_b' NOSUPERUSER; ----- +include::cassandra:example$CQL/create_user.cql[] `CREATE USER` is equivalent to `CREATE ROLE` where the `LOGIN` option is `true`. So, the following pairs of statements are equivalent: [source,sql] ---- -CREATE USER alice WITH PASSWORD 'password_a' SUPERUSER; -CREATE ROLE alice WITH PASSWORD = 'password_a' AND LOGIN = true AND -SUPERUSER = true; - -CREATE USER IF NOT EXISTS alice WITH PASSWORD 'password_a' SUPERUSER; -CREATE ROLE IF NOT EXISTS alice WITH PASSWORD = 'password_a' AND LOGIN = -true AND SUPERUSER = true; - -CREATE USER alice WITH PASSWORD 'password_a' NOSUPERUSER; -CREATE ROLE alice WITH PASSWORD = 'password_a' AND LOGIN = true AND -SUPERUSER = false; - -CREATE USER alice WITH PASSWORD 'password_a' NOSUPERUSER; -CREATE ROLE alice WITH PASSWORD = 'password_a' AND LOGIN = true; - -CREATE USER alice WITH PASSWORD 'password_a'; -CREATE ROLE alice WITH PASSWORD = 'password_a' AND LOGIN = true; +include::cassandra:example$CQL/create_user_role.cql[] ---- [[alterUserStmt]] @@ -2301,19 +1951,12 @@ CREATE ROLE alice WITH PASSWORD = 'password_a' AND LOGIN = true; _Syntax:_ [source,bnf] ----- -::= ALTER USER (IF EXISTS)? ( WITH PASSWORD )? ( )? - -::= SUPERUSER | NOSUPERUSER ----- +include::cassandra:example$BNF/alter_user_statement.bnf[] _Sample:_ [source,sql] ----- -ALTER USER alice WITH PASSWORD 'PASSWORD_A'; -ALTER USER bob SUPERUSER; ----- +include::cassandra:example$CQL/alter_user.cql[] If the user does not exist, the statement will return an error, unless `IF EXISTS` is used in which case the operation is a no-op. @@ -2323,7 +1966,7 @@ If the user does not exist, the statement will return an error, unless `IF EXIST _Syntax:_ [source,bnf] -::= DROP USER ( IF EXISTS )? +include::cassandra:example$BNF/drop_user_statement.bnf[] _Sample:_ @@ -2339,7 +1982,7 @@ DROP USER IF EXISTS bob; _Syntax:_ [source,bnf] -::= LIST USERS; +include::cassandra:example$BNF/list_users_statement.bnf[] _Sample:_ @@ -2411,7 +2054,7 @@ used and the identity does not exist the statement is a no-op. ==== Permissions Permissions on resources are granted to roles; there are several -different types of resources in Cassandra and each type is modelled +types of resources in Cassandra and each type is modelled hierarchically: * The hierarchy of Data resources, Keyspaces and Tables has the @@ -2422,7 +2065,7 @@ structure `ALL KEYSPACES` -> `KEYSPACE` -> `TABLE` * Resources representing JMX ObjectNames, which map to sets of MBeans/MXBeans, have the structure `ALL MBEANS` -> `MBEAN` -Permissions can be granted at any level of these hierarchies and they +Permissions can be granted at any level of these hierarchies, and they flow downwards. So granting a permission on a resource higher up the chain automatically grants that same permission on all resources lower down. For example, granting `SELECT` on a `KEYSPACE` automatically @@ -2625,27 +2268,12 @@ wildcard pattern | | | _Syntax:_ [source,bnf] ----- -::= GRANT ( ALL ( PERMISSIONS )? | ( PERMISSION )? ) ON TO - -::= CREATE | ALTER | DROP | SELECT | MODIFY | AUTHORIZE | DESCRIBE | UNMASK | SELECT_MASKED -EXECUTE - -::= ALL KEYSPACES -| KEYSPACE -| ( TABLE )? -| ALL ROLES -| ROLE -| ALL FUNCTIONS ( IN KEYSPACE )? -| FUNCTION -| ALL MBEANS -| ( MBEAN | MBEANS ) ----- +include::cassandra:example$BNF/grant_permission_statement.bnf[] _Sample:_ [source,sql] -GRANT SELECT ON ALL KEYSPACES TO data_reader; +include::cassandra:example$CQL/grant_perm.cql[] This gives any user with the role `data_reader` permission to execute `SELECT` statements on any table across all keyspaces @@ -2664,15 +2292,14 @@ This gives any user with the `schema_owner` role permissions to `DROP` `keyspace1.table1`. [source,sql] -GRANT EXECUTE ON FUNCTION keyspace1.user_function( int ) TO -report_writer; +include::cassandra:example$CQL/grant_execute.cql[] This grants any user with the `report_writer` role permission to execute `SELECT`, `INSERT` and `UPDATE` queries which use the function `keyspace1.user_function( int )` [source,sql] -GRANT DESCRIBE ON ALL ROLES TO role_admin; +include::cassandra:example$CQL/grant_describe.cql[] This grants any user with the `role_admin` role permission to view any and all roles in the system with a `LIST ROLES` statement @@ -2698,34 +2325,12 @@ the new resource. _Syntax:_ [source,bnf] ----- -::= REVOKE ( ALL ( PERMISSIONS )? | ( PERMISSION )? ) ON FROM - -::= CREATE | ALTER | DROP | SELECT | MODIFY | AUTHORIZE | DESCRIBE | UNMASK | SELECT_MASKED -EXECUTE - -::= ALL KEYSPACES -| KEYSPACE -| ( TABLE )? -| ALL ROLES -| ROLE -| ALL FUNCTIONS ( IN KEYSPACE )? -| FUNCTION -| ALL MBEANS -| ( MBEAN | MBEANS ) ----- +include::cassandra:example$BNF/revoke_permission_statement.bnf[] _Sample:_ [source,sql] ----- -REVOKE SELECT ON ALL KEYSPACES FROM data_reader; -REVOKE MODIFY ON KEYSPACE keyspace1 FROM data_writer; -REVOKE DROP ON keyspace1.table1 FROM schema_owner; -REVOKE EXECUTE ON FUNCTION keyspace1.user_function( int ) FROM -report_writer; -REVOKE DESCRIBE ON ALL ROLES FROM role_admin; ----- +include::cassandra:example$CQL/revoke_perm.cql[] [[listPermissionsStmt]] ===== LIST PERMISSIONS @@ -2733,32 +2338,18 @@ REVOKE DESCRIBE ON ALL ROLES FROM role_admin; _Syntax:_ [source,bnf] ----- -::= LIST ( ALL ( PERMISSIONS )? | ) -( ON )? -( OF ( NORECURSIVE )? )? - -::= ALL KEYSPACES -| KEYSPACE -| ( TABLE )? -| ALL ROLES -| ROLE -| ALL FUNCTIONS ( IN KEYSPACE )? -| FUNCTION -| ALL MBEANS -| ( MBEAN | MBEANS ) ----- +include::cassandra:example$BNF/list_permissions_statement.bnf[] _Sample:_ [source,sql] -LIST ALL PERMISSIONS OF alice; +include::cassandra:example$CQL/list_perm.cql[] Show all permissions granted to `alice`, including those acquired transitively from any other roles. [source,sql] -LIST ALL PERMISSIONS ON keyspace1.table1 OF bob; +include::cassandra:example$CQL/list_all_perm.cql[] Show all permissions on `keyspace1.table1` granted to `bob`, including those acquired transitively from any other roles. This also includes any @@ -2769,7 +2360,7 @@ the `NORECURSIVE` switch restricts the results to only those permissions which were directly granted to `bob` or one of `bob`’s roles. [source,sql] -LIST SELECT PERMISSIONS OF carlos; +include::cassandra:example$CQL/list_select_perm.cql[] Show any permissions granted to `carlos` or any of `carlos`’s roles, limited to `SELECT` permissions on any resource. @@ -2820,7 +2411,7 @@ Cassandra). The syntax of types is thus: Note that the native types are keywords and as such are case-insensitive. They are however not reserved ones. -The following table gives additional informations on the native data +The following table gives additional information on the native data types, and on which kind of link:#constants[constants] each type supports: @@ -2849,7 +2440,7 @@ See link:#usingdates[Working with dates] below for more information. |`inet` |strings |An IP address. It can be either 4 bytes long (IPv4) or 16 bytes long (IPv6). There is no `inet` constant, IP address should be -inputed as strings +inputted as strings |`int` |integers |32-bit signed int @@ -2959,12 +2550,11 @@ formats: ==== Counters The `counter` type is used to define _counter columns_. A counter column -is a column whose value is a 64-bit signed integer and on which 2 -operations are supported: incrementation and decrementation (see +is a column whose value is a 64-bit signed integer and which can be +incremented or decremented (see link:#updateStmt[`UPDATE`] for syntax). Note the value of a counter cannot be set. A counter doesn’t exist until first -incremented/decremented, and the first incrementation/decrementation is -made as if the previous value was 0. Deletion of counter columns is +increment or decrement operation, which uses the initial value of 0. Deletion of counter columns is supported but have some limitations (see the http://wiki.apache.org/cassandra/Counters[Cassandra Wiki] for more information). @@ -3000,10 +2590,10 @@ details). * While insertion operations on sets and maps never incur a read-before-write internally, some operations on lists do (see the section on lists below for details). It is thus advised to prefer sets -over lists when possible. +to lists when possible. Please note that while some of those limitations may or may not be -loosen in the future, the general rule that collections are for +loosened in the future, the general rule that collections are for denormalizing small amount of data is meant to stay. [[map]] @@ -3096,9 +2686,11 @@ Adding and removing values of a set can be accomplished with an `UPDATE` by adding/removing new set values to an existing `set` column. [source,sql] +---- UPDATE images SET tags = tags + { 'cute', 'cuddly' } WHERE name = 'cat.jpg'; UPDATE images SET tags = tags - { 'lame' } WHERE name = 'cat.jpg'; +---- As with link:#map[maps], TTLs if used only apply to the newly inserted/updated _values_. @@ -3145,8 +2737,8 @@ UPDATE plays SET players = 5, scores = [ 12 ] + scores WHERE id = ---- It should be noted that append and prepend are not idempotent -operations. This means that if during an append or a prepend the -operation timeout, it is not always safe to retry the operation (as this +operations. This means that if an append or prepend operation + timesout, it is not always safe to retry the operation (as this could result in the record appended or prepended twice). Lists also provides the following operation: setting an element by its @@ -3196,7 +2788,7 @@ several native functions, described below: [[castFun]] ==== Cast -The `cast` function can be used to converts one native datatype to +The `cast` function can be used to convert one native datatype to another. The following table describes the conversions supported by the `cast` @@ -3284,7 +2876,7 @@ username text, then the `token` function will take a single argument of type `text` (in that case, the partition key is `userid` (there is no clustering columns -so the partition key is the same than the primary key)), and the return +so the partition key is the same as the primary key)), and the return type will be `bigint`. [[uuidFun]] @@ -3304,7 +2896,7 @@ executed). Note that this method is useful for insertion but is largely non-sensical in `WHERE` clauses. For instance, a query of the form [source,sql] -SELECT * FROM myTable WHERE t = now() +include::cassandra:example$CQL/timeuuid_now.cql[] will never return any result by design, since the value returned by `now()` is guaranteed to be unique. @@ -3318,8 +2910,7 @@ _smallest_ (resp. _biggest_) possible `timeuuid` having for timestamp `t`. So for instance: [source,sql] -SELECT * FROM myTable WHERE t > max_timeuuid('2013-01-01 00:05+0000') AND -t < min_timeuuid('2013-02-02 10:00+0000') +include::cassandra:example$CQL/timeuuid_min_max.cql[] will select all rows where the `timeuuid` column `t` is strictly older than `2013-01-01 00:05+0000' but strictly younger than `2013-02-02 @@ -3397,16 +2988,13 @@ The `count` function can be used to count the rows returned by a query. Example: [source,sql] ----- -SELECT COUNT (*) FROM plays; -SELECT COUNT (1) FROM plays; ----- +include::cassandra:example$CQL/count.cql[] It also can be used to count the non null value of a given column. Example: [source,sql] -SELECT COUNT (scores) FROM plays; +include::cassandra:example$CQL/count_nonnull.cql[] [[maxMinFcts]] ==== Max and Min @@ -3415,7 +3003,7 @@ The `max` and `min` functions can be used to compute the maximum and the minimum value returned by a query for a given column. [source,sql] -SELECT MIN (players), MAX (players) FROM plays WHERE game = 'quake'; +include::cassandra:example$CQL/min_max.cql[] [[sumFct]] ==== Sum @@ -3424,7 +3012,7 @@ The `sum` function can be used to sum up all the values returned by a query for a given column. [source,sql] -SELECT SUM (players) FROM plays; +include::cassandra:example$CQL/sum.cql[] [[avgFct]] ==== Avg @@ -3433,7 +3021,7 @@ The `avg` function can be used to compute the average of all the values returned by a query for a given column. [source,sql] -SELECT AVG (players) FROM plays; +include::cassandra:example$CQL/avg.cql[] [[udfs]] === User-Defined Functions @@ -3499,7 +3087,7 @@ User-defined functions can be used in link:#selectStmt[`SELECT`], link:#insertStmt[`INSERT`] and link:#updateStmt[`UPDATE`] statements. The implicitly available `udfContext` field (or binding for script UDFs) -provides the neccessary functionality to create new UDT and tuple +provides the necessary functionality to create new UDT and tuple values. [source,sql] @@ -3777,7 +3365,7 @@ Reserved keywords cannot be used as identifier, they are truly reserved for the language (but one can enclose a reserved keyword by double-quotes to use it as an identifier). Non-reserved keywords however only have a specific meaning in certain context but can used as -identifer otherwise. The only _raison d’être_ of these non-reserved +identifier otherwise. The only _raison d’être_ of these non-reserved keywords is convenience: some keyword are non-reserved when it was always easy for the parser to decide whether they were used as keywords or not. @@ -3964,7 +3552,7 @@ https://issues.apache.org/jira/browse/CASSANDRA-10707)[CASSANDRA-10707]. * link:#updateOptions[`INSERT/UPDATE options`] for tables having a default_time_to_live specifying a TTL of 0 will remove the TTL from the inserted or updated values -* link:#alterTableStmt[`ALTER TABLE`] `ADD` and `DROP` now allow mutiple +* link:#alterTableStmt[`ALTER TABLE`] `ADD` and `DROP` now allow multiple columns to be added/removed * New link:#selectLimit[`PER PARTITION LIMIT`] option (see https://issues.apache.org/jira/browse/CASSANDRA-7017)[CASSANDRA-7017]. @@ -4005,7 +3593,7 @@ link:#createAggregateStmt[`CREATE AGGREGATE`] and link:#dropAggregateStmt[`DROP AGGREGATE`]. * Allows double-dollar enclosed strings literals as an alternative to single-quote enclosed strings. -* Introduces Roles to supercede user based authentication and access +* Introduces Roles to supersede user based authentication and access control * link:#usingdates[`Date`] and link:usingtime[`Time`] data types have been added @@ -4035,7 +3623,7 @@ keyspace partition using an `IN` clause on combinations of clustering columns. See link:#selectWhere[SELECT WHERE] clauses. * `IF NOT EXISTS` and `IF EXISTS` syntax is now supported by -`CREATE USER` and `DROP USER` statmenets, respectively. +`CREATE USER` and `DROP USER` statements, respectively. ==== 3.1.6 @@ -4061,9 +3649,9 @@ indexes (see link:#createIndexStmt[CREATE INDEX reference] ). ==== 3.1.2 -* `NaN` and `Infinity` has been added as valid float contants. They are -now reserved keywords. In the unlikely case you we using them as a -column identifier (or keyspace/table one), you will noew need to double +* `NaN` and `Infinity` has been added as valid float constants. They are +now reserved keywords. In the unlikely case you we're using them as a +column identifier (or keyspace/table one), you will now need to double quote them (see link:#identifiers[quote identifiers] ). ==== 3.1.1 @@ -4079,10 +3667,10 @@ of `?`). ==== 3.1.0 -* link:#alterTableStmt[ALTER TABLE] `DROP` option has been reenabled for +* link:#alterTableStmt[ALTER TABLE] `DROP` option has been re-enabled for CQL3 tables and has new semantics now: the space formerly used by dropped columns will now be eventually reclaimed (post-compaction). You -should not readd previously dropped columns unless you use timestamps +should not read previously dropped columns unless you use timestamps with microsecond precision (see https://issues.apache.org/jira/browse/CASSANDRA-3919[CASSANDRA-3919] for more details). @@ -4121,9 +3709,9 @@ added. * Type validation for the link:#constants[constants] has been fixed. For instance, the implementation used to allow `'2'` as a valid value for an `int` column (interpreting it has the equivalent of `2`), or `42` as a -valid `blob` value (in which case `42` was interpreted as an hexadecimal +valid `blob` value (in which case `42` was interpreted as a hexadecimal representation of the blob). This is no longer the case, type validation -of constants is now more strict. See the link:#types[data types] section +of constants is now stricter. See the link:#types[data types] section for details on which constant is allowed for which type. * The type validation fixed of the previous point has lead to the introduction of link:#constants[blobs constants] to allow inputing @@ -4134,7 +3722,7 @@ types] section does not list strings constants as valid blobs) and will be removed by a future version. If you were using strings as blobs, you should thus update your client code ASAP to switch blob constants. * A number of functions to convert native types to blobs have also been -introduced. Furthermore the token function is now also allowed in select +introduced. Furthermore, the token function is now also allowed in select clauses. See the link:#functions[section on functions] for details. ==== 3.0.1 From 486a6276034df40f8ac0ec9bd1d1f727b39f62c7 Mon Sep 17 00:00:00 2001 From: Brad Schoening <bschoeni@gmail.com> Date: Sun, 26 Jan 2025 15:51:23 -0500 Subject: [PATCH 142/225] Fix escaping and correct some language syntax in definitions.adoc patch by Brad Schoening; reviewed by Stefan Miklosovic for CASSANDRA-20203 --- .../pages/developing/cql/definitions.adoc | 25 ++++++++----------- 1 file changed, 11 insertions(+), 14 deletions(-) diff --git a/doc/modules/cassandra/pages/developing/cql/definitions.adoc b/doc/modules/cassandra/pages/developing/cql/definitions.adoc index 3e0251cf4c52..9d494c85b490 100644 --- a/doc/modules/cassandra/pages/developing/cql/definitions.adoc +++ b/doc/modules/cassandra/pages/developing/cql/definitions.adoc @@ -7,16 +7,15 @@ conventions in this document: * Language rules will be given in an informal http://en.wikipedia.org/wiki/Backus%E2%80%93Naur_Form#Variants[BNF -variant] notation. In particular, we'll use square brakets (`[ item ]`) -for optional items, `*` and `+` for repeated items (where `+` imply at -least one). +variant] notation. In particular, we'll use square brackets (`[ item ]`) +for optional items, star (`*`) and plus(`+`) for repeated items where star matches zero or more, and a plus sign matches one or more. * The grammar will also use the following convention for convenience: non-terminal term will be lowercase (and link to their definition) while terminal keywords will be provided "all caps". Note however that -keywords are `identifiers` and are thus case insensitive in practice. We +keywords are `identifiers` and are thus case-insensitive in practice. We will also define some early construction using regexp, which we'll indicate with `re(<some regular expression>)`. -* The grammar is provided for documentation purposes and leave some +* The grammar is provided for documentation purposes and leaves some minor details out. For instance, the comma on the last column definition in a `CREATE TABLE` statement is optional but supported if present even though the grammar in this document suggests otherwise. Also, not @@ -35,8 +34,8 @@ A number of such identifiers, like `SELECT` or `WITH`, are _keywords_. They have a fixed meaning for the language and most are reserved. The list of those keywords can be found in xref:cassandra:developing/cql/appendices.adoc#appendix-A[Appendix A]. -Identifiers and (unquoted) keywords are case insensitive. Thus `SELECT` -is the same than `select` or `sElEcT`, and `myId` is the same than +Identifiers and (unquoted) keywords are case-insensitive. Thus `SELECT` +is the same as `select` or `sElEcT`, and `myId` is the same as `myid` or `MYID`. A convention often used (in particular by the samples of this documentation) is to use uppercase for keywords and lowercase for other identifiers. @@ -47,7 +46,7 @@ double-quotes(`"`). Quoted identifiers are never keywords. Thus `"select"` is not a reserved keyword and can be used to refer to a column (note that using this is particularly ill-advised), while `select` would raise a parsing error. Also, unlike unquoted identifiers -and keywords, quoted identifiers are case sensitive (`"My Quoted Id"` is +and keywords, quoted identifiers are case-sensitive (`"My Quoted Id"` is _different_ from `"my quoted id"`). A fully lowercase quoted identifier that matches `[a-zA-Z][a-zA-Z0-9_]*` is however _equivalent_ to the unquoted identifier obtained by removing the double-quote (so `"myid"` @@ -93,7 +92,7 @@ single-quote(`'`). A single-quote can be included by repeating it, e.g. `identifiers` that use double-quotes. Alternatively, a string can be defined by enclosing the arbitrary sequence of characters by two dollar characters, in which case single-quote can be used without escaping -(`$$It's raining today$$`). That latter form is often used when defining +(`+++$$It's raining today$$+++`). That latter form is often used when defining xref:cassandra:developing/cql/functions.adoc#udfs[user-defined functions] to avoid having to escape single-quote characters in function body (as they are more likely to occur than `$$`). @@ -137,8 +136,8 @@ and should generally be preferred. A comment in CQL is a line beginning by either double dashes (`--`) or double slash (`//`). -Multi-line comments are also supported through enclosure within `/*` and -`*/` (but nesting is not supported). +Multi-line comments are also supported through enclosure within `+++/*+++` and +`+++*/+++` (but nesting is not supported). [source,cql] ---- @@ -181,6 +180,4 @@ multiple times with different concrete values. Any statement that uses at least one bind marker (see `bind_marker`) will need to be _prepared_. After which the statement can be _executed_ -by provided concrete values for each of its marker. The exact details of -how a statement is prepared and then executed depends on the CQL driver -used and you should refer to your driver documentation. +by provided concrete values for each marker. The process of preparing and executing a statement varies depending on the CQL driver you are using. Be sure to consult the documentation for your specific driver. From 027689ae48e124dae66a064cc1766b1eef9ef20b Mon Sep 17 00:00:00 2001 From: Daniel Lenski <daniel.lenski@sandboxaq.com> Date: Mon, 27 Jan 2025 18:18:33 -0800 Subject: [PATCH 143/225] Fix unparseable YAML in default cassandra.yaml when uncommented for downstream tooling The presence of an extra space the `:` in `key: value` pairs makes the default `conf/cassandra.yaml` file included in this repository unparseable and non-functional for some downstream tooling. patch by Daniel Lenski; reviewed by Stefan Miklosovic, Brandon Williams for CASSANDRA-20359 --- CHANGES.txt | 1 + conf/cassandra.yaml | 12 ++++++------ conf/cassandra_latest.yaml | 14 +++++++------- 3 files changed, 14 insertions(+), 13 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index f659a1f220af..2bd01bead52e 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.4 + * Fix unparseable YAML in default cassandra.yaml when uncommented for downstream tooling (CASSANDRA-20359) * Avoid fetching entire partitions on unresolved static rows in RFP when no static column predicates exist (CASSANDRA-20243) * Avoid indexing empty values for non-literals and types that do not allow them (CASSANDRA-20313) * Fix incorrect results of min / max in-built functions on clustering columns in descending order (CASSANDRA-20295) diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index 9258d5bb3282..6f70cd669c7e 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -199,8 +199,8 @@ authenticator: AllowAllAuthenticator # MutualTlsAuthenticator can be configured using the following configuration. One can add their own validator # which implements MutualTlsCertificateValidator class and provide logic for extracting identity out of certificates # and validating certificates. -# class_name : org.apache.cassandra.auth.MutualTlsAuthenticator -# parameters : +# class_name: org.apache.cassandra.auth.MutualTlsAuthenticator +# parameters: # validator_class_name: org.apache.cassandra.auth.SpiffeCertificateValidator # Authorization backend, implementing IAuthorizer; used to limit access/provide permissions @@ -1005,13 +1005,13 @@ listen_address: localhost # Internode authentication backend, implementing IInternodeAuthenticator; # used to allow/disallow connections from peer nodes. #internode_authenticator: -# class_name : org.apache.cassandra.auth.AllowAllInternodeAuthenticator -# parameters : +# class_name: org.apache.cassandra.auth.AllowAllInternodeAuthenticator +# parameters: # MutualTlsInternodeAuthenticator can be configured using the following configuration.One can add their own validator # which implements MutualTlsCertificateValidator class and provide logic for extracting identity out of certificates # and validating certificates. -# class_name : org.apache.cassandra.auth.MutualTlsInternodeAuthenticator -# parameters : +# class_name: org.apache.cassandra.auth.MutualTlsInternodeAuthenticator +# parameters: # validator_class_name: org.apache.cassandra.auth.SpiffeCertificateValidator # trusted_peer_identities: "spiffe1,spiffe2" # node_identity: "spiffe1" diff --git a/conf/cassandra_latest.yaml b/conf/cassandra_latest.yaml index 764193650e6f..0c7a792c8f40 100644 --- a/conf/cassandra_latest.yaml +++ b/conf/cassandra_latest.yaml @@ -199,12 +199,12 @@ batchlog_endpoint_strategy: dynamic_remote # Please increase system_auth keyspace replication factor if you use this authenticator. # If using PasswordAuthenticator, CassandraRoleManager must also be used (see below) authenticator: - class_name : AllowAllAuthenticator + class_name: AllowAllAuthenticator # MutualTlsAuthenticator can be configured using the following configuration. One can add their own validator # which implements MutualTlsCertificateValidator class and provide logic for extracting identity out of certificates # and validating certificates. -# class_name : org.apache.cassandra.auth.MutualTlsAuthenticator -# parameters : +# class_name: org.apache.cassandra.auth.MutualTlsAuthenticator +# parameters: # validator_class_name: org.apache.cassandra.auth.SpiffeCertificateValidator # Authorization backend, implementing IAuthorizer; used to limit access/provide permissions @@ -998,13 +998,13 @@ listen_address: localhost # Internode authentication backend, implementing IInternodeAuthenticator; # used to allow/disallow connections from peer nodes. #internode_authenticator: -# class_name : org.apache.cassandra.auth.AllowAllInternodeAuthenticator -# parameters : +# class_name: org.apache.cassandra.auth.AllowAllInternodeAuthenticator +# parameters: # MutualTlsInternodeAuthenticator can be configured using the following configuration.One can add their own validator # which implements MutualTlsCertificateValidator class and provide logic for extracting identity out of certificates # and validating certificates. -# class_name : org.apache.cassandra.auth.MutualTlsInternodeAuthenticator -# parameters : +# class_name: org.apache.cassandra.auth.MutualTlsInternodeAuthenticator +# parameters: # validator_class_name: org.apache.cassandra.auth.SpiffeCertificateValidator # trusted_peer_identities: "spiffe1,spiffe2" # node_identity: "spiffe1" From 0034a133fa93b82ddfafd7123bdfcc9f01d1f597 Mon Sep 17 00:00:00 2001 From: mck <mck@apache.org> Date: Wed, 5 Feb 2025 23:12:20 +0100 Subject: [PATCH 144/225] Heap and GC jvm flags improvements G1 can rely on `-XX:G1NewSizePercent=50` to always floor the young generation's size to 50% of the heap. (In production this can be raised to 66% for optimal performance.) Fixing the young generation size with `-Xmn` is only applicable to CMS, so move it to the appropriate section. When using G1, we can automatically set `-XX:ParallelGCThreads` and `-XX:ConcGCThreads` to the number of system cpu cores. The auto-generated heap size is now half the server's physical RAM, capped at 16G for CMS and 31G for G1. Increase MaxTenuringThreshold from 1 to 2, as evidence has shown it has no cost (over values of zero or one), but can have significant benefits in keeping objects in the young generation. While, values above 2 don't have any noticeable benefit. Always set CASSANDRA_HEAPDUMP_DIR, default it to $CASSANDRA_LOG_DIR, to avoid hprof filling up unexpected disk volumes. Assumption here is that the logs directory is large enough to handle these dumps, and/or operators are monitoring these directories more than other randon/unknown directories. patch by Mick Semb Wever; reviewed by Jon Haddad for CASSANDRA-20296 --- conf/cassandra-env.sh | 101 ++++++++++++++++++-------------------- conf/jvm-server.options | 35 ++----------- conf/jvm11-server.options | 28 +++++++---- conf/jvm17-server.options | 22 +++++---- 4 files changed, 85 insertions(+), 101 deletions(-) diff --git a/conf/cassandra-env.sh b/conf/cassandra-env.sh index ba9f9d459641..209816197955 100644 --- a/conf/cassandra-env.sh +++ b/conf/cassandra-env.sh @@ -44,45 +44,24 @@ calculate_heap_sizes() esac # some systems like the raspberry pi don't report cores, use at least 1 - if [ "$system_cpu_cores" -lt "1" ] - then + if [ "$system_cpu_cores" -lt "1" ] ; then system_cpu_cores="1" fi - # set max heap size based on the following - # max(min(1/2 ram, 1024MB), min(1/4 ram, 8GB)) - # calculate 1/2 ram and cap to 1024MB - # calculate 1/4 ram and cap to 8192MB - # pick the max - half_system_memory_in_mb=`expr $system_memory_in_mb / 2` - quarter_system_memory_in_mb=`expr $half_system_memory_in_mb / 2` - if [ "$half_system_memory_in_mb" -gt "1024" ] - then - half_system_memory_in_mb="1024" - fi - if [ "$quarter_system_memory_in_mb" -gt "8192" ] - then - quarter_system_memory_in_mb="8192" - fi - if [ "$half_system_memory_in_mb" -gt "$quarter_system_memory_in_mb" ] - then - max_heap_size_in_mb="$half_system_memory_in_mb" + # Heap size: min(1/2 ram, CMS ? 16G : 31G) + # CMS Young gen: 1/2 * heap size + if [ $USING_CMS -eq 0 ] ; then + heap_limit="15872" else - max_heap_size_in_mb="$quarter_system_memory_in_mb" + heap_limit="31744" fi - MAX_HEAP_SIZE="${max_heap_size_in_mb}M" - - # Young gen: min(max_sensible_per_modern_cpu_core * num_cores, 1/4 * heap size) - max_sensible_yg_per_core_in_mb="100" - max_sensible_yg_in_mb=`expr $max_sensible_yg_per_core_in_mb "*" $system_cpu_cores` - - desired_yg_in_mb=`expr $max_heap_size_in_mb / 4` - - if [ "$desired_yg_in_mb" -gt "$max_sensible_yg_in_mb" ] - then - HEAP_NEWSIZE="${max_sensible_yg_in_mb}M" + half_system_memory_in_mb=`expr $system_memory_in_mb / 2` + if [ "$half_system_memory_in_mb" -gt "$heap_limit" ] ; then + CALCULATED_MAX_HEAP_SIZE="${heap_limit}M" + CALCULATED_CMS_HEAP_NEWSIZE="8G" else - HEAP_NEWSIZE="${desired_yg_in_mb}M" + CALCULATED_MAX_HEAP_SIZE="${half_system_memory_in_mb}M" + CALCULATED_CMS_HEAP_NEWSIZE="`expr $half_system_memory_in_mb / 4`M" fi } @@ -108,35 +87,44 @@ echo $JVM_OPTS | grep -q Xmx DEFINED_XMX=$? echo $JVM_OPTS | grep -q Xms DEFINED_XMS=$? +echo $JVM_OPTS | grep -q ParallelGCThreads +DEFINED_PARALLEL_GC_THREADS=$? +echo $JVM_OPTS | grep -q ConcGCThreads +DEFINED_CONC_GC_THREADS=$? echo $JVM_OPTS | grep -q UseConcMarkSweepGC USING_CMS=$? echo $JVM_OPTS | grep -q +UseG1GC USING_G1=$? +calculate_heap_sizes + # Override these to set the amount of memory to allocate to the JVM at # start-up. For production use you may wish to adjust this for your # environment. MAX_HEAP_SIZE is the total amount of memory dedicated # to the Java heap. HEAP_NEWSIZE refers to the size of the young -# generation. Both MAX_HEAP_SIZE and HEAP_NEWSIZE should be either set -# or not (if you set one, set the other). +# generation when CMS is used. # -# The main trade-off for the young generation is that the larger it -# is, the longer GC pause times will be. The shorter it is, the more -# expensive GC will be (usually). +# When using G1 only MAX_HEAP_SIZE may be set, and HEAP_NEWSIZE must be +# left unset. # -# The example HEAP_NEWSIZE assumes a modern 8-core+ machine for decent pause -# times. If in doubt, and if you do not particularly want to tweak, go with -# 100 MB per physical CPU core. +# When using CMS both MAX_HEAP_SIZE and HEAP_NEWSIZE should be either set +# or not (if you set one, set the other). -#MAX_HEAP_SIZE="4G" -#HEAP_NEWSIZE="800M" +#MAX_HEAP_SIZE="20G" +#HEAP_NEWSIZE="10G" # Set this to control the amount of arenas per-thread in glibc #export MALLOC_ARENA_MAX=4 -# only calculate the size if it's not set manually +# Warn on an erroneously set HEAP_NEWSIZE when using G1 +if [ "x$HEAP_NEWSIZE" != "x" -a $USING_G1 -eq 0 ]; then + echo "HEAP_NEWSIZE has erroneously been set and will be ignored in combination with G1 (see cassandra-env.sh)" +fi + +# Only use the calculated size if it's not set manually if [ "x$MAX_HEAP_SIZE" = "x" ] && [ "x$HEAP_NEWSIZE" = "x" -o $USING_G1 -eq 0 ]; then - calculate_heap_sizes + MAX_HEAP_SIZE="$CALCULATED_MAX_HEAP_SIZE" + HEAP_NEWSIZE="$CALCULATED_CMS_HEAP_NEWSIZE" elif [ "x$MAX_HEAP_SIZE" = "x" ] || [ "x$HEAP_NEWSIZE" = "x" -a $USING_G1 -ne 0 ]; then echo "please set or unset MAX_HEAP_SIZE and HEAP_NEWSIZE in pairs when using CMS GC (see cassandra-env.sh)" exit 1 @@ -157,8 +145,7 @@ elif [ $DEFINED_XMX -ne 0 ] || [ $DEFINED_XMS -ne 0 ]; then fi # We only set -Xmn flag if it was not defined in jvm-server.options file -# and if the CMS GC is being used -# If defined, both Xmn and Xmx should be defined together. +# and CMS is being used. If defined, both Xmn and Xmx must be defined together. if [ $DEFINED_XMN -eq 0 ] && [ $DEFINED_XMX -ne 0 ]; then echo "Please set or unset -Xmx and -Xmn flags in pairs on jvm-server.options file." exit 1 @@ -166,13 +153,22 @@ elif [ $DEFINED_XMN -ne 0 ] && [ $USING_CMS -eq 0 ]; then JVM_OPTS="$JVM_OPTS -Xmn${HEAP_NEWSIZE}" fi -# We fail to start if -Xmn is used with G1 GC is being used -# See comments for -Xmn in jvm-server.options +# We fail to start if -Xmn is used with G1 if [ $DEFINED_XMN -eq 0 ] && [ $USING_G1 -eq 0 ]; then + # It is not recommended to set the young generation size if using the + # G1 GC, since that will override the target pause-time goal. + # Instead floor the young generation size with -XX:NewSize + # More info: http://www.oracle.com/technetwork/articles/java/g1gc-1984535.html echo "It is not recommended to set -Xmn with the G1 garbage collector. See comments for -Xmn in jvm-server.options for details." exit 1 fi +if [ $USING_G1 -eq 0 ] && [ $DEFINED_PARALLEL_GC_THREADS -ne 0 ] && [ $DEFINED_CONC_GC_THREADS -ne 0 ] ; then + # Set ParallelGCThreads and ConcGCThreads equal to number of cpu cores. + # Setting both to the same value is important to reduce STW durations. + JVM_OPTS="$JVM_OPTS -XX:ParallelGCThreads=$system_cpu_cores -XX:ConcGCThreads=$system_cpu_cores" +fi + if [ "$JVM_ARCH" = "64-Bit" ] && [ $USING_CMS -eq 0 ]; then JVM_OPTS="$JVM_OPTS -XX:+UseCondCardMark" fi @@ -183,10 +179,11 @@ JVM_OPTS="$JVM_OPTS -XX:CompileCommandFile=$CASSANDRA_CONF/hotspot_compiler" # add the jamm javaagent JVM_OPTS="$JVM_OPTS -javaagent:$CASSANDRA_HOME/lib/jamm-0.4.0.jar" -# set jvm HeapDumpPath with CASSANDRA_HEAPDUMP_DIR -if [ "x$CASSANDRA_HEAPDUMP_DIR" != "x" ]; then - JVM_OPTS="$JVM_OPTS -XX:HeapDumpPath=$CASSANDRA_HEAPDUMP_DIR/cassandra-`date +%s`-pid$$.hprof" + +if [ "x$CASSANDRA_HEAPDUMP_DIR" = "x" ]; then + CASSANDRA_HEAPDUMP_DIR="$CASSANDRA_LOG_DIR" fi +JVM_OPTS="$JVM_OPTS -XX:HeapDumpPath=$CASSANDRA_HEAPDUMP_DIR/cassandra-`date +%s`-pid$$.hprof" # stop the jvm on OutOfMemoryError as it can result in some data corruption # uncomment the preferred option diff --git a/conf/jvm-server.options b/conf/jvm-server.options index 343da4f54365..547a06dce3f7 100644 --- a/conf/jvm-server.options +++ b/conf/jvm-server.options @@ -160,39 +160,12 @@ # HEAP SETTINGS # ################# -# Heap size is automatically calculated by cassandra-env based on this -# formula: max(min(1/2 ram, 1024MB), min(1/4 ram, 8GB)) -# That is: -# - calculate 1/2 ram and cap to 1024MB -# - calculate 1/4 ram and cap to 8192MB -# - pick the max +# Heap size is automatically calculated by cassandra-env.sh based on the +# formula: min(1/2 ram, CMS ? 16G : 31G) # # For production use you may wish to adjust this for your environment. -# If that's the case, uncomment the -Xmx and Xms options below to override the -# automatic calculation of JVM heap memory. -# -# It is recommended to set min (-Xms) and max (-Xmx) heap sizes to -# the same value to avoid stop-the-world GC pauses during resize, and -# so that we can lock the heap in memory on startup to prevent any -# of it from being swapped out. -#-Xms4G -#-Xmx4G - -# Young generation size is automatically calculated by cassandra-env -# based on this formula: min(100 * num_cores, 1/4 * heap size) -# -# The main trade-off for the young generation is that the larger it -# is, the longer GC pause times will be. The shorter it is, the more -# expensive GC will be (usually). -# -# It is not recommended to set the young generation size if using the -# G1 GC, since that will override the target pause-time goal. -# More info: http://www.oracle.com/technetwork/articles/java/g1gc-1984535.html -# -# The example below assumes a modern 8-core+ machine for decent -# times. If in doubt, and if you do not particularly want to tweak, go -# 100 MB per physical CPU core. -#-Xmn800M +# If that's the case, see MAX_HEAP_SIZE (and HEAP_NEWSIZE for CMS) in cassandra-env.sh + ################################### # EXPIRATION DATE OVERFLOW POLICY # diff --git a/conf/jvm11-server.options b/conf/jvm11-server.options index f71f6287ffb5..857e07857a05 100644 --- a/conf/jvm11-server.options +++ b/conf/jvm11-server.options @@ -50,23 +50,32 @@ ## some JVMs will fill up their heap when accessed via JMX, see CASSANDRA-6541 ##-XX:+CMSClassUnloadingEnabled +# Young generation size +# The main trade-off for the young generation is that the larger it is, the longer GC pause +# times will be. The smaller it is, the more expensive GC will be (usually). +# +# If left unset MAX_HEAP_SIZE and HEAP_NEWSIZE in cassandra-env.sh take precedence. +#-Xmn800M ### G1 Settings ## Use the Hotspot garbage-first collector. -XX:+UseG1GC -XX:+ParallelRefProcEnabled --XX:MaxTenuringThreshold=1 +-XX:MaxTenuringThreshold=2 -XX:G1HeapRegionSize=16m -# -## Have the JVM do less remembered set work during STW, instead -## preferring concurrent GC. Reduces p99.9 latency. +# Floor the young generation size to 50% of the heap size +-XX:+UnlockExperimentalVMOptions +-XX:G1NewSizePercent=50 + +# Have the JVM do less remembered set work during STW, instead +# preferring concurrent GC. Reduces p99.9 latency. -XX:G1RSetUpdatingPauseTimePercent=5 -# -## Main G1GC tunable: lowering the pause target will lower throughput and vise versa. -## 200ms is the JVM default and lowest viable setting -## 1000ms increases throughput. Keep it smaller than the timeouts in cassandra.yaml. + +# Main G1GC tunable: lowering the pause target will lower throughput and vise versa. +# 200ms is the JVM default and lowest viable setting +# 1000ms increases throughput. Keep it smaller than the timeouts in cassandra.yaml. -XX:MaxGCPauseMillis=300 ## Optional G1 Settings @@ -77,9 +86,10 @@ # For systems with > 8 cores, the default ParallelGCThreads is 5/8 the number of logical cores. # Otherwise equal to the number of cores when 8 or less. # Machines with > 10 cores should try setting these to <= full cores. -#-XX:ParallelGCThreads=16 # By default, ConcGCThreads is 1/4 of ParallelGCThreads. # Setting both to the same value can reduce STW durations. +# When leaving both unset then cassandra-env.sh will set them both to the number of your cores. +#-XX:ParallelGCThreads=16 #-XX:ConcGCThreads=16 diff --git a/conf/jvm17-server.options b/conf/jvm17-server.options index 1a0f8f9c565f..9a695aa351d7 100644 --- a/conf/jvm17-server.options +++ b/conf/jvm17-server.options @@ -32,17 +32,20 @@ ## Use the Hotspot garbage-first collector. -XX:+UseG1GC -XX:+ParallelRefProcEnabled --XX:MaxTenuringThreshold=1 +-XX:MaxTenuringThreshold=2 -XX:G1HeapRegionSize=16m -# -## Have the JVM do less remembered set work during STW, instead -## preferring concurrent GC. Reduces p99.9 latency. +# Floor the young generation size to 50% of the heap size +-XX:+UnlockExperimentalVMOptions +-XX:G1NewSizePercent=50 + +# Have the JVM do less remembered set work during STW, instead +# preferring concurrent GC. Reduces p99.9 latency. -XX:G1RSetUpdatingPauseTimePercent=5 -# -## Main G1GC tunable: lowering the pause target will lower throughput and vise versa. -## 200ms is the JVM default and lowest viable setting -## 1000ms increases throughput. Keep it smaller than the timeouts in cassandra.yaml. + +# Main G1GC tunable: lowering the pause target will lower throughput and vise versa. +# 200ms is the JVM default and lowest viable setting +# 1000ms increases throughput. Keep it smaller than the timeouts in cassandra.yaml. -XX:MaxGCPauseMillis=300 ## Optional G1 Settings @@ -53,9 +56,10 @@ # For systems with > 8 cores, the default ParallelGCThreads is 5/8 the number of logical cores. # Otherwise equal to the number of cores when 8 or less. # Machines with > 10 cores should try setting these to <= full cores. -#-XX:ParallelGCThreads=16 # By default, ConcGCThreads is 1/4 of ParallelGCThreads. # Setting both to the same value can reduce STW durations. +# When leaving both unset then cassandra-env.sh will set them both to the number of your cores. +#-XX:ParallelGCThreads=16 #-XX:ConcGCThreads=16 From 291cbcccf6e472b455a9d3ef2602b68ae0b7797c Mon Sep 17 00:00:00 2001 From: Sam Tunnicliffe <samt@apache.org> Date: Tue, 14 Jan 2025 17:02:05 +0000 Subject: [PATCH 145/225] Fix SimpleClient ability to release acquired capacity patch by Sam Tunnicliffe; reviewed by Dmitry Konstantinov, Mick Semb Wever for CASSANDRA-20202 --- CHANGES.txt | 1 + .../org/apache/cassandra/transport/CQLMessageHandler.java | 5 ----- src/java/org/apache/cassandra/transport/SimpleClient.java | 4 ++-- 3 files changed, 3 insertions(+), 7 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index c5ec1056c4c5..ae4744091559 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.1.9 + * Fix SimpleClient ability to release acquired capacity (CASSANDRA-20202) * Fix WaitQueue.Signal.awaitUninterruptibly may block forever if invoking thread is interrupted (CASSANDRA-20084) Merged from 4.0: * Fix autocompletion for role names/user names (CASSANDRA-20175) diff --git a/src/java/org/apache/cassandra/transport/CQLMessageHandler.java b/src/java/org/apache/cassandra/transport/CQLMessageHandler.java index 792f6bf7b638..c2d0b4297446 100644 --- a/src/java/org/apache/cassandra/transport/CQLMessageHandler.java +++ b/src/java/org/apache/cassandra/transport/CQLMessageHandler.java @@ -373,11 +373,6 @@ private Envelope composeRequest(Envelope.Header header, ShareableBytes bytes) return new Envelope(header, body); } - protected boolean processRequest(Envelope request) - { - return processRequest(request, Overload.NONE); - } - protected boolean processRequest(Envelope request, Overload backpressure) { M message = null; diff --git a/src/java/org/apache/cassandra/transport/SimpleClient.java b/src/java/org/apache/cassandra/transport/SimpleClient.java index a7227c89a55a..504df79c987a 100644 --- a/src/java/org/apache/cassandra/transport/SimpleClient.java +++ b/src/java/org/apache/cassandra/transport/SimpleClient.java @@ -518,9 +518,9 @@ public void release() errorHandler, ctx.channel().attr(Connection.attributeKey).get().isThrowOnOverload()) { - protected boolean processRequest(Envelope request) + protected boolean processRequest(Envelope request, Overload overload) { - boolean continueProcessing = super.processRequest(request); + boolean continueProcessing = super.processRequest(request, overload); releaseCapacity(Ints.checkedCast(request.header.bodySizeInBytes)); return continueProcessing; } From 2daa0f0ba0647ee76a2b3ffc349407751031d0dd Mon Sep 17 00:00:00 2001 From: Dmitry Konstantinov <netudima@gmail.com> Date: Sun, 16 Feb 2025 11:50:20 +0000 Subject: [PATCH 146/225] Fix flaky ReadSpeculationTest: with TCM the second replica in a read plan can be different, it depends on an order of adding nodes to a cluster. To fix it NetworkTopologyProximity implementation is adjusted using ByteBuddy to make a predictable order of nodes to read (the same idea as in python read repair dtests is used). Also, dynamic snitch is disabled to avoid re-ordering of replicas in a read plan by it. Logging of actual configuration in java dtests is added to simplify troubleshooting (the default logic does not print it because logging is initialized in tests later). Patch by Dmitry Konstantinov; reviewed by Brandon Williams,Stefan Miklosovic for CASSANDRA-20251 --- .../cassandra/distributed/impl/Instance.java | 1 + .../distributed/test/ReadSpeculationTest.java | 86 +++++++++++++++++-- 2 files changed, 79 insertions(+), 8 deletions(-) diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java index cf4b6cdd7a1a..b98b36c47c93 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java @@ -713,6 +713,7 @@ protected void partialStartup(ICluster<?> cluster) throws IOException, NoSuchFie startJmx(); LoggingSupportFactory.getLoggingSupport().onStartup(); logSystemInfo(inInstancelogger); + Config.log(DatabaseDescriptor.getRawConfig()); FileUtils.setFSErrorHandler(new DefaultFSErrorHandler()); DatabaseDescriptor.createAllDirectories(); diff --git a/test/distributed/org/apache/cassandra/distributed/test/ReadSpeculationTest.java b/test/distributed/org/apache/cassandra/distributed/test/ReadSpeculationTest.java index fce1de26ead7..445315f34388 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/ReadSpeculationTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/ReadSpeculationTest.java @@ -18,43 +18,92 @@ package org.apache.cassandra.distributed.test; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.util.List; +import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import org.junit.Assert; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import net.bytebuddy.ByteBuddy; +import net.bytebuddy.dynamic.loading.ClassLoadingStrategy; +import net.bytebuddy.implementation.MethodDelegation; +import net.bytebuddy.implementation.bind.annotation.SuperCall; import org.apache.cassandra.concurrent.ScheduledExecutors; import org.apache.cassandra.concurrent.ScheduledThreadPoolExecutorPlus; import org.apache.cassandra.config.Config; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.IInstanceInitializer; import org.apache.cassandra.distributed.impl.CoordinatorHelper; import org.apache.cassandra.exceptions.ReadTimeoutException; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.NetworkTopologyProximity; +import org.apache.cassandra.locator.ReplicaCollection; +import org.apache.cassandra.locator.ReplicaPlan; +import org.apache.cassandra.locator.ReplicaPlans; import org.apache.cassandra.service.CassandraDaemon; import org.apache.cassandra.transport.Dispatcher; +import static net.bytebuddy.matcher.ElementMatchers.named; +import static org.apache.cassandra.db.ConsistencyLevel.QUORUM; +import static org.apache.cassandra.utils.ByteBufferUtil.bytes; import static org.assertj.core.api.AssertionsForClassTypes.assertThat; public class ReadSpeculationTest extends TestBaseImpl { + private static final Logger logger = LoggerFactory.getLogger(ReadSpeculationTest.class); + private static final String TABLE = "tbl"; + private static final String PK_VALUE = "1"; @Test public void speculateTest() throws Throwable { try (Cluster cluster = builder().withNodes(3) + .withConfig(config -> config.set("dynamic_snitch", false)) + .withInstanceInitializer(new FixNodeOrderForReads()) .start()) { - cluster.get(1).runOnInstance(() -> { + cluster.forEach(instance -> instance.runOnInstance(() -> { // Disable updater since we will force time ((ScheduledThreadPoolExecutorPlus) ScheduledExecutors.optionalTasks).remove(CassandraDaemon.SPECULATION_THRESHOLD_UPDATER); - }); + })); cluster.schemaChange("CREATE KEYSPACE IF NOT EXISTS " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': " + 3 + "}"); - cluster.schemaChange("CREATE TABLE IF NOT EXISTS " + KEYSPACE + ".tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck)) WITH speculative_retry = '2000ms';"); + cluster.schemaChange("CREATE TABLE IF NOT EXISTS " + KEYSPACE + "." + TABLE + " (pk int, ck int, v int, PRIMARY KEY (pk, ck)) WITH speculative_retry = '2000ms';"); - // force speculation; rely on IP order + List<InetAddress> readPlanEndpoints = cluster.get(1).applyOnInstance((none) -> { + Keyspace keyspace = Keyspace.openIfExists(KEYSPACE); + ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(TABLE); + DecoratedKey dk = cfs.decorateKey(bytes(PK_VALUE)); + ReplicaPlan.ForTokenRead plan = ReplicaPlans.forRead(keyspace, dk.getToken(), null, + QUORUM, cfs.metadata().params.speculativeRetry); + return plan.contacts().endpointList().stream().map(InetSocketAddress::getAddress).collect(Collectors.toList()); + }, null); + logger.info("Replicas provided in a read plan contacts: {}", readPlanEndpoints); + logger.info("Cluster instances: {}", cluster.stream().map(instance -> instance.broadcastAddress().getAddress()).collect(Collectors.toList())); + int firstReplica = 0; + int secondReplica = 0; + for (int i = 1; i <= 3; i++) + { + if (match(cluster, i, readPlanEndpoints, 0)) + firstReplica = i; + if (match(cluster, i, readPlanEndpoints, 1)) + secondReplica = i; + } + logger.info("1st replica to read from: {}, 2nd replica: {}", firstReplica, secondReplica); + Assert.assertEquals(1, firstReplica); + Assert.assertEquals(2, secondReplica); + // force speculation by dropping all messages sent to the 2nd read replica cluster.filters().allVerbs().from(1).to(2).drop(); @@ -157,12 +206,12 @@ private void assertWillSpeculate() DatabaseDescriptor.setReadRpcTimeout(rpcTimeoutMs); DatabaseDescriptor.setCQLStartTime(cqlStartTime); - ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore("tbl"); + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE); long speculatedBefore = cfs.metric.speculativeRetries.getCount(); long before = System.nanoTime(); cfs.sampleReadLatencyMicros = speculationTimeoutMicros; - CoordinatorHelper.unsafeExecuteInternal("SELECT * FROM " + KEYSPACE + ".tbl WHERE pk = 1", + CoordinatorHelper.unsafeExecuteInternal("SELECT * FROM " + KEYSPACE + "." + TABLE + " WHERE pk = " + PK_VALUE, ConsistencyLevel.QUORUM, ConsistencyLevel.QUORUM, new Dispatcher.RequestTime(before - enqueuedNsAgo, @@ -180,14 +229,14 @@ private void assertWillNotSpeculate() DatabaseDescriptor.setReadRpcTimeout(rpcTimeoutMs); DatabaseDescriptor.setCQLStartTime(cqlStartTime); - ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore("tbl"); + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE); long speculatedBefore = cfs.metric.speculativeRetries.getCount(); long before = System.nanoTime(); cfs.sampleReadLatencyMicros = speculationTimeoutMicros; try { - CoordinatorHelper.unsafeExecuteInternal("SELECT * FROM " + KEYSPACE + ".tbl WHERE pk = 1", + CoordinatorHelper.unsafeExecuteInternal("SELECT * FROM " + KEYSPACE + "." + TABLE + " WHERE pk = " + PK_VALUE, ConsistencyLevel.QUORUM, ConsistencyLevel.QUORUM, new Dispatcher.RequestTime(before - enqueuedNsAgo, @@ -206,4 +255,25 @@ private void assertWillNotSpeculate() } } + private static boolean match(Cluster cluster, int instanceId, List<InetAddress> readCandidates, int positionInThePlan) + { + return cluster.get(instanceId).broadcastAddress().getAddress().equals(readCandidates.get(positionInThePlan)); + } + + public static class FixNodeOrderForReads implements IInstanceInitializer + { + @Override + public void initialise(ClassLoader cl, ThreadGroup group, int node, int generation) + { + new ByteBuddy().rebase(NetworkTopologyProximity.class) + .method(named("sortedByProximity")).intercept(MethodDelegation.to(FixNodeOrderForReads.class)) + .make() + .load(cl, ClassLoadingStrategy.Default.INJECTION); + } + + public static <C extends ReplicaCollection<? extends C>> C sortedByProximity(final InetAddressAndPort address, C replicas, @SuperCall Callable<C> real) throws Exception + { + return replicas.sorted(java.util.Comparator.naturalOrder()); + } + } } From b854fa3e228ec41b5561e672b60b309b8cf7d37d Mon Sep 17 00:00:00 2001 From: Sarma Pydipally <sarma1807@users.noreply.github.com> Date: Wed, 11 Dec 2024 14:49:22 -0500 Subject: [PATCH 147/225] Replace some of existing images with ones having better look and resolution patch by Sarma Pydipally; reviewed by Bernardo Botella, Stefan Miklosovic, Mick Semb Wever for CASSANDRA-20169 --- .../assets/images/Figure_1_backups.jpg | Bin 38551 -> 64854 bytes .../assets/images/Figure_1_data_model.jpg | Bin 17469 -> 35510 bytes .../assets/images/Figure_1_guarantees.jpg | Bin 17993 -> 31418 bytes .../assets/images/Figure_1_read_repair.jpg | Bin 36919 -> 38223 bytes .../assets/images/Figure_2_data_model.jpg | Bin 20925 -> 53211 bytes .../assets/images/Figure_2_read_repair.jpg | Bin 45595 -> 60618 bytes .../assets/images/Figure_3_read_repair.jpg | Bin 43021 -> 53091 bytes .../assets/images/Figure_4_read_repair.jpg | Bin 43021 -> 53143 bytes .../assets/images/Figure_5_read_repair.jpg | Bin 42560 -> 54451 bytes .../assets/images/Figure_6_read_repair.jpg | Bin 57489 -> 58945 bytes 10 files changed, 0 insertions(+), 0 deletions(-) diff --git a/doc/modules/cassandra/assets/images/Figure_1_backups.jpg b/doc/modules/cassandra/assets/images/Figure_1_backups.jpg index 160013d76fe287ff97d965d8e2cdd8a5f683d63d..05d7a56a28158b164e2686c8597519fbfd8c0eb2 100644 GIT binary patch literal 64854 zcmeFZ3p`X`+c!Ribdt&`im4<b>7W#{JNzVsattv)NlwX6$W&u?5kgEUDaGV0(&Su{ z$+`37oFQXoikz}1jLppK`LCYmectE(zjxh-_w)Xr`+na4t@Z~qW9_xBz1DSoukZC; zOZ-vXk6UVS!2AGi&YU^8gWw-d+=JWy<F~*1HFxfuIpF`Cx&QI?pFhq0(a4`P@c;VQ ze|-F}eiy&MEt^00<=j4rIcsrqm(7t_Hb>lm!{cysr2f5a-2eERGgm@VYM%7`1q&B} z50o#(&7C75F;`MTN=i}^d^#MwkCR*`wS3K%z4PRbcu21elHYpidd7V9eZ{p3wk^oI zZO6|<ELf<hv_e@$W4)%<25tTAI}CR2GTi^0=>fBY<`#DL4vtPoon1Unc%Ah2@%0N1 z2|XJ|Ja<0wa#VE8mDsqX8#j}0rQE)gnwgcIlbe_SprE9*th}P~$<wO3`i91)=9jNt zx3;x+bas90X0kcl&tC=xzYY!azE4a}@u$(5*&pMYgOm85Cj0k+{V(HM2F5j4Qc^-v z`p3BD%nbvV#4<^#HCyH_-+M&bBS>!T)=Tr{_g&8@u3ezM%@$EOex_xi;yQh{2Jgqv z{%vG`+rT3Jw?_6q2lk)iV&WD{%mEKiVi^v?VVG8C9PVG&zijYdF$Z)w3<?!F!uY7e zKgw$*Z5CzG{l&QM@&3F9vk#ME+;WPf@Bo^@4qQyUqp#}X|5E>*qZ{6~eK~RCw{>mf z#wj6$&3l~BWXti>$g6yz@5vNYiz2#4K{Z*}HLN|fCxvbDJU%CipZ<m!e4%roa^taz zs^?<??+=~T89wUrgz7yOo}%}R>^fm?cEQMH`s1uRhIbX?Bq?HCb^`e&9wiyeiE(c% zAodg;IVi@>9_$*uB*w+m*FwUwM}2X#(3B09CB}VE5#vHkE{J@@xO}{@P>kz+|97?o zbwIP=uCEEA4^Ss%7Tn}DUG$do(;H{~ypv;JvqW~F17e&oM~wR<@vq+e=L7t2ZyN_O zE}}z>t6is@Y$lWs<U>+d4!d$wF175{lnAaVn45TR-9^J2abtyS7}b^*W)ZN3Qexam zC&6JcZulmz+|*i;A2Mt|J2kWR)AL(=r%K+UyoQ?A97EW)(m07;#{AZWDGNFRg95_> zr9NdNpIkoeS7<DMSN&l7^Tz6zCHsx#4m5g2alO2}G@OqvJ-B^)5J3k1oJN$!8fXi} zxS_bZXD&19!~5d2?$7;axadptsWTSMLU}XnrI}D}T#P#i`x%OwY4r)3m~F8bC$&tB zD*>0JK#Xg}GZUYu<Hfk^6-upn*xfsy2dUq68sQ%5>>25<$s?xZMDlt0Bc`~&bSWTo zp@IZ<ga+E4E5?=Ti1Ohn3nH`TD=kWl>(9r3Pk>P^CD9$i*nTmtz^<tyQC-wWM>T-6 zDLn-V8fAISVw{E97^|Kzqv<|Jj4Rfuqt70H{HXH@Et>w_x>`bvJMI((Pd0kE6_G8- z*?*}w|CR44ph4NdVL3wmnGk;;)K7(H_Y016ecSz0uRMwX50&t^CK;OSWoQ}$qr2;1 zy<%M3H4}PfoX%wM#D~l{E!>Zb)*P4!mraj}9)YR9mlP~e#a<R;&9i4?k!ijSdO={? z$0E0oWl%=ph?-_+wHr^pDP8(0Qj`*ltt1^Pu0%TTc_93B-Y4_>!?v!~Z3lUu9vrme zucu34dO}Mcee?}@SP^IuuW6d^Dx#TeCqcZy(M3oYJM3~ft9y`tp2uj^U33JE%_8}u z6pjvSp`Xdt$7PXwk;cjM4qtATniu$0{NdVsVR6b<(^XgAHOAiajDO9wEw@;4{zl_O zmG5io!>=b6J|X7k+oV;y85>5Km=~1q+VI4Nn&g7b?lcZRw0(tBL)&c1MsNQ!Hu^7Y z8!f&a`EsO|vGj|HPt_j-`8i5Uyrx*Sf#?QCt&JS^+KE+`GB)p(z{t;C60PJ=y5Dfb zxcWCU+bMW|wUwfbY9-DCp=u$?0NG8wtiPa~5U13|yn5Kp=kbu?p{giRg78du)VetK zTD}+;1y#n3KIc`wz$=p13iWw|S4gJ0q=l{&)U`$qzD!?H247W^NL5{Lbw+2mXa9kx z{VM}iUcq6x`;AjiDb7lMAEtX>*Kc@sUD=FN)LaYonDI}DD3h!!-fSvGA0G>sLqhq= zMpk*=Sa2>$5)JF8GR?SMqs_<-&KOp-F{Xb!Iy6w07-`d0qcL>e<Wj#3WZOO&zS23x z3j5fxsqC3A^V8{(KCpalt0|#rVw?;QJTM8A#-Yx`KAvSQ!8QmtiIQUK#JD(oxfoZ6 z*5HZF9F4rlGBGZe>0Yf)Ie@swdgWPqg<bG_knMi-C^4e-{V+4Y&iZ2SZ;TZr>-Eox z?1Cowpsn?-2@Y?3J~ThflTK`C)NstC?LA)Lxp~r4<z|}czU&7b`Hz#`tb5&8O6^PB zwk*2P5cS(WY!QFPi57V)$Z5_#ZQgUkkwv3Hb~i$)>mIz$QGuN)C9_6+;8r5ageyg* zBsVS#=YlG6=ohOA_k|bGVkEwH$236-_7Pdu^=&{H^-%^cVF|{rJUK^HA^R(SzHooO z?+15dYz1#gj(?$V*fLYkGiu)l`ZnyQG*c0Ky^o%3YJ<q1t(jo5Gxr9Eun1+CqrqHV zGCqc&xE;QXNurk??Y?*3JE9uqF-#HN=7{P=oU=C$mzkfM3LwAtcS=)#c642cBTB~> zba5di>>Xj{zk#q|PV?xLz}lCriB<|=?1LDmQiB&BZmE$ZPB&0X`^u4K&fD3rSfMT_ zo|<U5BW2U2%zYJ|vH|C0#(I?$yGomVT9m6B<0s_!L*=3pvMvckiHHP}JP)d)_F!<l z`x>;C%R}P(%|^!sn~2(a-Lj3;Ig|7iXxI^i(4fPLGsM@$$Ft(p^g2ie9=aH9mAmc} zp4a>0FTS1U*Sa!|Enk;vUweUIYhOPry2a-0%x%kKxXW*Cj?a4EW4}fIOB`+Ee*U&R ztCj{wzP=W_;INzj#`=zT!Kc>cBJG~H+v^V}C8g%Y6vj4h+@X_9>+^6^kq=MV?qluV zny~A*Uq{1%2cpQiE9TiMAD#Pf{_2RW^VRbkb%WDoC!{Y|Z`6nw=p>+K`ZimceC7q6 zH|L;I-LTp)?@$#>-!loxNgI}(NVu6*@OX7d=y+$|sJ~;*yLY2GS}eMbpD3&{cmU~e zBkfSD&KUjTvh;4E@X-;gM|a(ORs#>H{>Q)hzF!25X;YFOSOa^+xSIeh*Y2T@z82&B z@FLsv9S?)>-=@H8dLoUx9%5Xd7Muvp+Crx>6*2CWAtEb^byom>Rw3$ac_V4N7?%{< zj~6Vj))C`!brz2Kf47?gxBqv~`;Y$q%P>t38bP?`VDJy&+TY;O%W$oU7&jTi&!Tz9 zh;g3-q9z}J;S8MHH7JbZ5%`1Hy-QOf`6GlZYI>CXkv&p>>H4K%X;nJEz*U}C#JDgT zwg4X5*dvSa_6vgmnz$|qL%J~Yu8Z)f1wzAaKh!CIq{F5OZ_sdJ+{a`#LA2$llRqJe z`rt2i=Pz~j|MYu5jno1PA;u5@$OMql)c&@_P&gkfVRG;Y76qbvX+6k$cGS(<gBcZU zd~X6XCDH`JVx0A!+1WTPS_^}*@25C0&NrhX+CM%vv$C=>&O9&Oce$2F^t!E<`mHbb zH80OB-L?fMweFe}$&Dw6Ei@7=AnA6d)b;6z3a~{Cu0dD=X-+?5#ZloQG|}BST7unG z?^)=7ucYJw#;7mZx_-b`(!S6-X`s+npWA1pG5t(wVW_r=OUM^nx6HfnC4RXAVnrRB z$Rj>mPErvig20lAUpcDL{IQ&|h};A#)Tp5Tc9MV0ddJdzjr8UYO+P*@dG5g(kC$Sc z^HsT4SCS(zK`gPjnl%@$)iGI5gl+D9E2fJwAV0X<i5>Z7!TWB#5K5H02^tUQ&`&11 z+O54_S2E^VYs}i1M?Q5*m7e?*9@>!h_;slYb;%Ncp(V&2EqnwPaWmR@02^zMS066i zO?^Q}HQ_oVU-Kh@92S$tk3^q6a$US(Lh~^~9lgVTwjp}KjGg^^`-naD6%XN3f=~g3 zb2Ye`j;<e%+E2OdioK(j4JP(2PW<hRmEbUX&f6nhIcrZ)(rqLE)MNXToBdj;*&4Ii zA<e7betE7{-x(xYX>$KpKy_p(xsia_yEa)SSYiv61?oeKkItl?jRfeKi$;d~6<(@* zP49&J@2_|3i50Y@E4B092!RKR@Fw@-)oNr&PN))32U{qI94AN$z0d$I97W!eQb&L6 zo_F6rmQBH<{<g@Y=j-EPSm5cm?4w`gp^noD)g^lk&9uyj*$!I0Q%~YIrD;Eor|NUp zJ?|^p-upG{+R3Gb7H1BZX()=m#pcf9i{U@ZNM0h?b$raEjO^ZM?<euUwd#Lbs}rkW zYtC&ul~H1zCV%bGoB7A*ywJVmGKHA+{8b0>e|pvZsy)~URx{a&;`x@qhPSYfCJ+jo zn$RB6tEXIrCFq`(gcw*p0Fw}&;Zf^~P7By&gfdlHV#o+RPHiq4THvF$szbk<WXB#a za~V!eNN#%PvccBiY6`wZL!l$GvAy=VFUQx^B4pf4r^lYd_Z^+=3cjlp9lcG~PR7=H z^8^1U%vp1Dm!YE8n!ETb@M3yk;71Bq|GJA%S5zd%dF!nlK6CK6;gT30<%J!Z-uFkN z%=7W6>@K?_ZQsHojGvBq-dLSbK6QDUX^^~K?Z)6*V_*Gk`$pE;nXZjVJes}lx9cUF zwO?G@E4j+4m8A%IF}hQ@?PpAR41NeI$IIMs%}k%5XtIn1q4BGVFUp&(CDzw>U25}@ zkX&{t4s`__O@t~~f7_jNpSI9X*eoRJRiz%dI|ku|K|Ftc$SvjD-zJLa?B9q08W)~s zKR(d7==z65ouLi9V_m#I%xide36b$aM*4&LWpG`S{xb_Dp?%8d^UoC~fd7urAStwZ z1tSxDu<9eyNxH?-A~{opHP*qiYphbcm$kxDSC6ZLW@_xc)DUrqJMhk@5v+tll+JGa zE(Cd-lqdMROTO2)MkV2|7|M<s@dEb9KG~>bqAf~n9nh$|-`gd-VWw2`-K7q-4RsJj z#@6H>ZQ`c2GxB}|X+_+bsR2UL^+_p(eobsay})Bnus3Ma3mq~slnLe^6I~75E5<#d zeq>39d}1N0&o9`41eeNxfAjKFa9^76Xtdn)=T_*%c(-;dDm&|CQrN%ps~skFN+kWn zP>d^5hJ~}>AzlOy{Fbs9R|t$WGXrMYT&ftiI(&}lx%Jn<aGvn2Ms%ASA#{WE*c5L) z`|+lYA>m<sUsU71q1M6&fdR-izCVIx`B_lzG(CK8^LE7!^GTgRblv#qku$IKD8`z+ z7_L|C>Rt+kRc<zB#;ZK=?DsSKQ9X`Xld9RC(#qAiPJXJx^U^biG}7L9nQW*l6c}#@ zkv+Bvx^eyCPVeR5$18bw(+y-l%_Ne8C=c)%j7#V6Ds&%w1Z#oU%v@KJ8Jf{go9E70 z*JxU;c*|r>=h#4XRdToDisK1i?tHe7RqeK)*-aUvBZ8*fon%SjI#B`SYk5bE<93ep zca6T|`PoQ(`V!InndFC3*sLW)$?@Z7zwvB^1t$WEg;TR>$OZlZ(M>SpbC;8(M5!io zNQ%_OYU6A|EuaB0TuoZGq?K;RbW}Cuzx`9g<>FiBZl9CWQV*OwZu6mIu!D2Y{!u<V z!yFno9f*7(#Jv2LQ@@kg-PvHBLr>PCHQ&KBxe8Gfp?ns{g(h4Bmk?^lyT=bb`8+}> zw;5B&8^QG3)V)@&B58_l8*k&vs2_VGRl+l92-xoAvB)?odPf#`;%CSmgyo6jOj&H9 z8Bb?_+ot*HBehVmcWq!awe)P%^BRx5SKPojTj62e)r#wdTa^K|y40ubn1bA?%9qj5 zM?;<0w0i?b;~#KxR@7W8Q^G7yVgc!CtFks_crV^~LhycqTr<@=vyN=<OF&EkyIS2= zpHDi961d|`imUl@*ECoDQ&hm~;-^&9+}a_le7BaaShHr>ePwyn!Id6mEdl)wa}-y{ z6+nAy6iG`^9YpUpG48T|e+n>S6ib@4t3n_A4zs<%j9ehx-Lhlf{3xiTS|hu=wtS9} zSxOXR1?@;?^^k=fsqMy1s^d2?PVU21PpLO6@*6`e-XIILGPCdhqY>-hdVktmF|LPT z*ZvSb1s01;&aI4aZ{I@fBM9Fc%l%zlpT%DYw|$RCHTA}_@=bm7>la@N47i28Zw*a6 z{Ib-gdG(x@qvYF81F~blTACmwmiz;dO&`!1;1igl3Ll9{0Kfh@IJm#3AN0-eY5a`w zII12aN{?w~h|YG6l@jtgW+DY(e6I%K+Evi#q=^YUQb7+$#feP*R(t<5eU3Ik0_$}e zpgeb|{{f*G5-PYfvoD}G*d%Nj@Gk3mu@*r4^%dp|HCsO48n1&^*32gg*cLKpJi=BI zAN5##sL>)>A#cnLe{1zim-GMBT4wPb&Q^sX7UBSGcPjxz3Q18Fy+4Ie*5-Tu5L0$^ zjCbL1n;7T2?y0MN70;#?l5rk}luhtoyWXvFE>~3`YooT@`tL3;22A6*0iQlPB=-3p zmYAaR4XaQ=J_fSOQmjZ|$MSU&<H}AxNAhZG)}k$<n2cKXbM-@LBu7`R=NL~mMqi}z zQL3z>m9|~FO}<~B-fa>>t-fA3r2XGuHeSEu;Cv^dAmZ)YJr0MrI>&y+Nw2>}SD^Q2 zrElU9#<ub3wJ>63BY{Q|5NiEyZ+018O^iD&#wnpLv*GG))u==fG2RvNt{d;B@8hXP z##3ecNR~YKvXHo4aM!z{I=+@Jv;FQ0&4lu=*;bDywU7JTWta!s7mU*1&1e*UKj1K8 zSKso$y|8RUA^p31z}M>T7NP)Kc!v{s1=H*@mICX1M3hG5KZRw6Q)$uF4uS({7iTGo z7pz3R8<=jBsn%K9#0#7c2<6IlIB_<enqM1I6+v8CUsZEM@)pI*;EFDd^wQ&tY!emT z==N&D2~*9=j2`-(WF$C(-fLvxncOB9Go}#*{T`ni^j^BfWI2)&RV~G)$fBAruvIO_ zYx>6*`YNj7k>;rC?K$0FYw!pbljCaUojatm<9goyRVruGq#hn=mfy1Wz?Dl5dlWu& z^?IQGbF5PxW(wPfXYM%&lPc1Ss7iz@#cg*(&iUAVeV9Eq@adaqxe3T3qkun?x+=!q zuN2Dd2Tb4K7ytu|lN9tAgdAneE`n?IA*7GoP8IU~hpTkpsSDa-92KC17lYVId`~x3 zh!Idp=;e$ZrX(v=+=fN45Gz@s0kvFHjN6?Gv4J0{^@P6a0#kkfxbU377M|#)6@apt z`)m4O10C5V#@&VkPr?&3aQ{*<PW_9`PwwsW%bwO0W>WsNQy!w{0BJD30I0QJNJ;)h z2eN+J+u28PP{(K?Mgq7@f;tn!=K~A~rC@VpzyJNV(h}pGN@z1YJ>e2~;BT>15d6bX zIzJ3$!a``UpDCka9CaBC?2=ZC{wu?5<BtFz7b{$PJ(R-K<m;hsJTZ>_Fi@U39S?aB zVll0DOb4~G-q+ISo;Ui1S6Vk4LSM?+!?XH~!j030&&*a8yL_!|oAK2g#*a=54hR)R z1>_aL>?>;K5+^Lt$bQ}917h4Etfe?wjI-*{*IW9vO>~2R#9Sq9DPp?%k>~Sj?vPYi zb}gY7jCAd+pX`_UlD(BwKA}PWLl|I#{!U1|1tCXlqI^PRqsi;Kglqa4s8)0R^Y!wT z$aI5EWL3_jyHX3}JEZ>-mk7nv7f-3W)E};CZ`~GJh$?UYwCS7qiPBaXIwcjEGSv5u zt=&Z)GSD(-eR6&BeA1<CJhZf`bgJa-Y>wZDdhT)&x9nu{^TCoNCgViM!)Z6CNh?!a z#MUKyKQH@0anboqDFFE8;y=|T)JCB{(pQUW06$XLw;_ECe+<u^9_1p@?|m0sh(ekp zpyPqvg60ZE*H>#cwlxw~R0jsj`(LCL9{L=ZJTmP_pAUJnW_IhLI#f|Qy^bylqmMed zk1xrduy7hu2IWG{c#v$otf1Y?5aV*x$IfeI()XK4{z`NJ>R^7*0$+C%5m-<@6bb)Z zAKZOvwxY^B|2k%QPWy4!3u#6KM3J83UT^#xiMRcE@5RLauBxs(LO(!r?&5i4uc#Dg zO0mOFdmUT9qs`{K_Fca}{12oG86d6r-9}}Yu$=AzeJtc`DqUAHj6tF#x;I{BxP+l@ zWt^H4SF;2)aH~o3y?tY2?l5Ni)_kM+x_e!n<w&J)`2=?42MQYlaZ*u~1WoM`LbNLX z@9I+y1$#~94{!db0w8s=-P4#2AaMpkPD4XLt<r5=pwRVAL|#f}(7&H2AWo2AHyj1e zj>8`xJ?~ou%5ymoV)Nbjm>7qdmP3uQq78We7E^^;Wn>K)-khIZ?%upY?BHM{Rj{6H zEXK9FI?xW$wxJzj+`3>6Lv#aDgNJns+8O;RS;DxOX1v2pdKWNZHK299;aVejw2^*- zB03Doj(kf+U9kn{#5k?@z^j}E>X3NyEBx$zGQR86_L&8<56`+^`oG)Anu9Sy`3azu zA**3;mB@3@Eu2Lx84@hYH@?sx*zMncvND_H=F4QokOFu&j#|k&@H($w@X!xaG)PxQ z7Q1idsr2q&7kHYqv1H@*$By_k7Gg{9fvtf#{m7k!#|Bq~Fwc_nZm`y7G5HNtK1RP% zgKy?i6^1JzHS%iItR9{pLWnv`)_MG_Jv*|g&*Vv0V6Gyw;&J8jZpK4*!mFSa{xN|q ztSeMW6HTEzXg69o4^8IK7O%AFZst3pMqHK>s^s#tv)U+BIsx*f$ESF{plozsQQq;n z{ghc0S?;Jm<}fvNbE#zzPdBN~S*`DLkt~H?ocoNI#}|m=VLzRZCMqNN#n4GsG^>mi zMRH=(l{BWN<Gu;ac+}sJ)A8&8yE;Q7^!SMNQ@|^?gnk?!sA}Kn9j&|9k*P!5;b60- zrP(pJkX<OUQJ~NgErR38^|JftP+qWa4p|J@6Px22J%)F_-oAnRNANxEefQR$<!DC6 z<?VZ}eEjS09+CGltSz<{y4QGIZ0tJnurl*;$m5$q%dFmC{py(^t&)^v@;kbc*V6!% z$d0P>l(2=6;3VlJFQtjGg8X3%aX)u0$p|q>ppo2=(ifXa2g6tHK{W)suy=hcZ@7lb zT2{yrWA+x)JAa@K|KYGomq!oI=~ge$o&^sD)e38bm4sepqI=j*@>_QWbQdp+gRf;M z3kj%1E^5ZVjZ`+V<_ZtMb8^#3N06Q-HGO3IV%boQvT*O?fD3yrTvl-}&3)q?6dLA{ z?;f5|^m(QfVEKpxqyt@auW@D<f8I0nM!yl<y}tooE;QoB^Hra>7fl7Kp!9zC2mTxZ zKDH{RmU@LQTRy(TG#kz9&!JHCXV$iP#=AUy_N0ZhH{2^s^3db00)x^0f~BN`s2&eK zgx=t-<O3iVVM}JH*Q-f%R2v%G2vkT~U39h|v90m^(~^pst|DVk8VN#ZLT#$ljyqD{ z-(;>Ore7${pRx11u2EH8WtV&&jop{}AlZxVc-6=5MuNGA&%>Iayaf)0^M*Y)-)?{4 zQ0S85M|@(N<)U9`wv4d!lh;CfO<TL1+nzPW8ypI5J}CH3tlAcwply5A7gPz&cG=}v zEG0Ty7cF4OZrnY8*IjI3bB;S!2QPvB=(40G?b`zrSj#beA42K8Sb;{baBqwiPor|o zAdI(lkWGs-kt3iylG3htftr<OgUiF&iyzEyn73f<r!t&lQs}%uY@s?2wi4x$H<Dk| zOT$LX*|bE$GBQrM8PRKC#qrg4h%(E>xJy+v&y*;W&n=I&8gE0S>np{$MWH>Et1eg1 zpG?s{#yZ4QO!3cmbM@^~3VOFqpCE(1Vo92S2}MVoWSKhL<`<0E9nf;RC;pP!h6=PJ z*B6s>)3itT8VKc$xSI6=5PPs;?81w5y_Ve6wx%0H@WM&U3nJ&SFG)uDWx@lew)SQ> z)?|ZZ(SyNM=O>|2+^`fjepE)_NYdmvMTLko*;n!9j(0fJXjlpMDmoX^eX3kQ2cS!y zyj-Z!6$-CF&xmpMKA}N!e(tIr8v}n|>N;$-Xyb=qhlZ2MXL^?yrYWtmhSE#2ER*+b zIl8kYYUk^R{*b5Z=4egj$igGvIhz%>&NwuDNU$gf|4{#>EO_zhc@LjH@K9fI!Qzoc zwx5l%%DeW<s#@=J7FXCm{+Mdtt6{mB_{}9WJW9nZQ|JjMR|1&vf$=~XLx`p>65~7x zDrg%gZ>$&*<2bbVNx+jU)TsO7an)u>q%-QwX3R$^j>veUPPlhnY8`XG0{u$4JpD<r zf8pKW%nfa2W1eHzugu~>o_Cn0EIfj03J#$W)BM9|q^H1^{90{y73#z@;2%P~uL`$u z-+9~eI{NNrj-)S1sfCxPcUiVg!E=W(JWugX{-)w!?-K#Ky5r8Hxfgs|2Pa@ITn96q z_#{yw09Mf~X`wVvww9$v_kp4abKz4qi~7)UE|$f$QmUuRjhE>~5oA{L7N?G15$;wi zeMmg(A20PZuK93M*RfOyN!fG0q$b<&XKug>1gwoN2YJc%)SIDEJa4w*CBZRrAASW% zj_2RRL$!#rgl;j;3}Mx;M|M!yHW!_k)wcj&%9FlWt~UqW#5Lp1d%$0G;qL8fhp<ZP zbgjB$p~tn4uPRk$=j~$WEIxmkE1A5j?cER?X{&cv&^TfyvFcdP!kaw1(-8xa4-A*l z-dLo~Z+jX3+%V{Y?NReC+nX~v)<;Pv3Y*I8*(>tgR~*b+zy^fwCYORO1z$zZ0zgCq zMNa^i!L)Q0B||=pOD3`?p0}MW?S_58WIKOM^z(XSO2~KvUN(dvcNDd2JZm_3Wz1ND zeU~ncHd}U-2W;M*%?xPF>DAM1ET38S@XR?{9RcWMn#Ql5rOVXpAVT~FXaWsuASl(W zZ$_qH0FKRAgQwQq1g&tYpYGFqinzz~XpK;*Vtl`2^@kdnUSiy*z6)QgO~dkssx07B zEZ-oeYT+Qh)IcbAmzxp^1tj44X=p5x!e4?eL+S#%bvT-%T{Pix(e+b8kG%PMBy%n_ zAC0%>>GnOy7iu9QYNAvpHFMap?aszB-;mqtrDai}7rm+|1)_V<2?n|W8s+xGk%SWb zB}||EUQJnai@bn1{6?V39k`Q+QX4=SFWv;_Q=?wRn|BxSwnHts5YCMVfrRPP)sc*0 zHSI{ZXPbxO`R@!4R<`qBsj|J}Zhmi$wM{U4n<urRV4LNR#<rWqTLaVYCOR#a@jIPx zbXURc%27X#BYv^xm1s469%Bio^{UHhyU;b7DYhw_l-FfAqSNkjq4qbuR~g&iQgIk= z8z=9AYb}j~*o;J=1R+b2-%?9Y#|wV*qn9ehkPM2*A6&-c={|uA$PB)r$urK_oNCwm zq;05M00)}y_nl;g#%o>{J{!sm&KgoI3JvrMDr^qjb+kq(x@WQ*mE&gvYZmgO(G#=r zL?zC6q&`I%C99t7G}86Ascg@Hd|RdpcX}u5hThfPXY|bI3HR2~R?cgOMCYYCm0a4` z4|*hrm&_Oo;*9`L<}3iBfLbHS?~?>SHNK9?imP5J2t(y;QyQq!<aMZ1-7Fk=qk*uv zI*dIYRiks)4N=R>8M@_UNQ*gLRQAf`laEG8#~L|HLJ3?tUT;obB9uqHk=KI<P$#y4 zB~7%kY7O0u?~l1?RyTh6mdld^PdnOP;AO?Qay^?Up&>J^vG-cf_Vyw{=k?d6H=U9# zm5fz=PvXa|TB>|ESy0FD&QUwEqzsk#CUw?wN!b|;_Z;7j!_HTI%;fg|-b<#vu{-+I zVCQ%C^P9PMsT#kDstQ-OS!X3h8640ru>4r~7TXTLT7{O1aVHqf)Y*8y@}cAbScU#M zk>m_ijow^_$h6#Llv=B2HnZvj${nU}V*-zpLj8Vq9MudIfyWO@JO$EhQFt2x5G#7n z7iAV=%@S-0>{bUJUABH!kV{_!IV8UM0CKW<|CM6GzjiMLbnlO+`)UZT1u4ns6b2+M zBhNtqs1mVgzesPM)l<Lg=V1#lvwu%J^Uyy}J6A8=i;LL2_w&;85k_%Z*{#00kJ7&` z4v?r%S3`2|%!_4*FFJtqC7hZt56JdWaZ9A+y1okD)_crYQ6*$qqueXwB%|a#Zl6z{ zDqdgVlSv(#J*Db8JT>#&b*AJd^K@F!Iq&@5qs}(@zHk1h3Kw=hwmf&d(x>>7ZdzND zd$(7D!-@H?S82&y(;oYioa$eB)Gv}%d7;UrY8k*6t_Q*Ip3tNN`Vd&Nju^vt6cre2 zc<xwM9w>n}^G_o&qeJi`-?uMT7W!aous3~r6;Btsj()#VIjXcd^4lLHmipOuEA>yN z&~)G=<5MDoTMi}kjrUo3TL};Np+Gj2iT|bMSK-fB+~n(-Nd8CkMlK02q5#UB6?^{S zJ9)LGpr*awt}Ib2l{0d<v!`~pJTFGy&7@<mL28j0cV$z^8=Zslr_%>8t(Z;P+OH0z znVU1W#0R#l9m><b$w}_bPMSJ3EBIv^JV713fK)o5s)BXoH?Y+9kBVKcC?wb=T!y+n zbY0idn^RE;;1HV{ZLFEUr*hdA=Z;O2u{RBq9xDB@ZW-Z#(`U2cSZ9ro|8y&T2IT!x zOGUS_O(@MC^%QuJUolVKDZ<(T)pC}~v06@uc1fE`zm$!&L#5+aH<hK?aBN0-g~Zh1 zgB)Z!?yfDuk9~xyJ3SA3+*)mwQS^M+G{^sZ+0bl^shjqVEL#QNH%U1=QvI_CU2o%V zvR&>C*;9<Q*QbWtK`QV`F^);=XN_606ZRiNy&kzUi8tz|Cx|d7tj<PdXq=5tz)J~z zJ&ilh(=P%!v3`3F)`A$sjp>a&${;>k+S7Z*q%F_Q{o^aCcDk8k%nv$S^0nwH>@UVC ze`P7u=#n-*Zv`?BZ<~N^^-!5oEkjMX2bJY<e=Bi_3OLo=XcLL8ZwU<uwyM~i<#Kem zbXZaGt#d~CfiT?(0bHw5zVg6k6l5E|z)Cxi_epf#cowaJ(jDCNCg9Z&M>Y<G0)s1| zh4d6ao|<=PLhH%AlYx)xc~&OB6(9AzVEQa7m3;Pw<K(j`6Kne7_Q61zEAf#-`fY@r zd9bT`uNG|yC~_Yr(y=5!l1UkWkjfg}AjW-IV<t3=Yy`5qWS|b9W8hOBHrS<wU&T0= zKcEQ??UO8as9xmK1T$s{Bp@!3==J$Kn}Ry;$qLcJX^=kNke%{Z2Hf{h_yo}R$p&Rs z8)2$@f#!MEf-00}iE+~KW6-@lQFvrnRtc!@Kq;X=SByiAL{ZSJB;daPvr|yVLvJA< z9H$v#+_l|JamY9tDh-l{A#E73gN3SC9aS_}9<I+fCIE}s0pD^LG&atc7vsJz%@X+n z0Lh4>hd?u%#W=@e>I9a;g0MZ)DmwLacDtG$5b{3&MzRv)kTAhQcyj&`F>c9y9xT$Z z!4?LJagp?opw0XHC{uY7VjNI|d;lZ*y9~twR+qB<(XWze!WiqLTiBA3D(IuR^BJSU zd8${8TYo)T&jxpOWcsqcu2ZAh#;&r!MX3MY;wSA>jAg`l`{EXT-IZ2l`!zZ<3Wk|C zd9Fj9TbdS`Z){!XbN|-wI{oxW=1GvBN*zWSKuTdaFQvejj#xo}0%j^Td?bZF^0oWu zO70`5lGXfU?9mIkB*V7x&*q^}w`Xn5w${l{<mm<_#ewFg^mPc9lK1shSf(#CZH&>* zbP_JJ<8SE@nz6@~$Sq)wl}V4$9uaNq%Q~wKyafajkw<SN#GH43;H^hh9TaL6?cg3n z;ry3vlHRV2g}oV@bLAxuL(UH$KAh|^vNEzRt9aZts(ZhrxVR)j$JI#3=xEX+e@jcB zWww3Ho$9|O>O1^qzwkaz>YOrlseO~}ExfGNhpgS}daARZ`DWXV6bAT)+_mX=Q=;Xe zZMMukv&d1UQNyQky34W4Dyi>XccK0fQxbdfm(=Ia=+J+}YlJ&+#k;AgyfFauBuSO! z)m0Uh?eHloWk=Oz#c8?yGNLjVvDWLc{$_OHW!4{+hjlxiS7)r+8oSy|;>-hlZu0+K zh4;7ko*2x=2%8U*Q11*La3yZ#f>T{XQ{>;VUg-Nd7GfqV*ey$;&W_P~PF{p;^8Xv| zgVhmsLj80YaloTG(xbrDNV!5Xo@gU`9NiCj1EgU`ZY30lil%ERQKKP@1;W#So25r3 zmL5Asbb8B6OHa-K`z;IEa+`8bSNE`FUXk8UU&|iBZcd?cps}OC?-+dr#E4$c4QxS> z;Lw{~-{n5k(wHVUi#Gn%#4oz(gN?8h$tZW2plm!Rb69a&!#eLit&u)YR@N$Tam0y8 zZo-%DIG=!QA`Wt-9|xKj9|zG5*!wPp`KDn5Cy?oCkJ|K)!QAL9rRYh*{CZy=zTx~K zmFd`O1L>G*$iv^)V_pKB|0;V;VxMA~{`Mz#B)`8RSpSusS${31_^)_h0ZrrLdjk0) z9(Bw}aE!D@l!_^lo158#qtAGRT4?@VH<ZhzB^Ux;=|~Zp!oPrS<GnfDkz#`=MU0<1 zHTi_jK5go`aHcN^d*vN85doj_M<V&*AV31AAMq4aMn;4>VqDh+tTA->1ZuIfH$=<H zsx9b7OZO+8q9X$XA5)(kk{@aDwZEEK?&R`pw{j0$zZb3rk?!j~xON8}`AlvDJF~`g z8jSG*5?t$8?agCxy<f;cY$c%_v#SAAqK!8ofsK9ZI`vG}l3PaZ4_81cZJd?p`JUgB zWasRz|HP!ks8sg+#z)qQDp)VimFlI;t|6pBbHunW8erm#3)aGqHL)wA0-fQ4KQLG2 z4;J@|Y^|X1u~)muL1u657{SW>o`tHP53;%tPbQjuKQ&E2eLxjaj1$RG(Dhx|Dhd-8 z<I-~Q$OllW0w6#Wk66)!yYPTe72}L7W~1Y}M8|+;Wv~}S;T>Ms<CBDm?+`lI1QaHs zBY>soVqD8k@I3%#zx<&ooMD0}EdvMyDbR3c5b%~DRj$7-uo2@X%4qpMS%1<Teq^2( zfFdA8zNr3X{O|4=)bW;vg#+v%;qkA2{l}C1ok-(ng6Hv!XjzA?<2j7i1MQp4J8E=@ zjAw(v*-rX0kVw(XdR=|0Lf_{YD`J85p*FiY6`@vd3!g?0NUX1U@#m$*)|4e*;<ozT zdvrr^Dt~b1#!b5iD|gh?nRJS7nr!RCT397rqaA|%el7~Ms3|-J!jYGdyy-|s!sn`l zl$b!Wme2?pOx#qavwI)WGikQ7HT;Yvae&A@XWMJE_LlbU*&0RAWq0`XHhO;=oqwEF zVut18W2UC9$mheE9Y40wtT<vSkNao2oW75HXJ9?4VCw^z=<av<UMN+dLsYN5j&(r` zGCL<(iLAL~U7;OscUNrlsH{M>dYb8239?51RV1`1IwB;mHOtbuOmH{qN#&E<efq&> zlK6JhpBIg5@Nwy0^DA1vq`%9n3_VGl8NIi4ILX||Vr6z$LzPk1MIiPx!xnY{LK-YP z$ssYW!B~_-pWRi+CPccg0EWE-jm-#xdcLw&(mh-Q)oPI&{EhI+VJ3C7oQF-<spQ@; zCX|H<*WAJskSW?&RN#gC!#nDrWw4)B*n{CMgq3gfO7`Z7QsHut7%0rNA$ZKz0Sp{u zDkYkNx$svCGqa}ek(x=HjNo<ZJFvz76+re?>E2DYG|j~}Pi0ouG{A#jwm(Y!p$Mu2 z>s<;YZCO9uM+m4ZsF7P)qE+p`dZS}j*EDd72wHqiMYqDj>y%Cn0QbQ-`=7wqXMmHP zPZ286u*5~!0`SX~a{^0vG8lNFpS|q#wOl^j9h&%_hb^Xp`Z@k+ickRDRs<U}8?+Iw zg%C&j>`pyY(*|2JIHn7n+dH$LzDme!Kk_G?x)y$%F!7CwEsB}Jr$NXSkhi#iNYD@F zGla%a%K+Lp+m8Rm>*V+&W*uDM3AN#VV7c#mr($776^w4ee<lM1{qGnI??e#_AAlT5 z6L5Rd|Fs5lhk4$005&y%(vI#_Acc<DxF&!-l<2E+E&cIsIsUcMcFne~`qb;y(M{h5 z`Mv2zu1yuoH`<ynewp~1As@F8-{Cg_B@n)wnQR1scpGmJ%VZWELU><N#<fp@;P+XK zdz<f+q96Y9H2q7jaWw_VQX<?=gY+7(47ElEOI@)QPJFXo`=yKEx{GLd(i}|*lYyFH zS`<)n{*<-7iO<u$2HeZyUukCk5wGj~5|Ie#gv<ESabbFjP=SFl2y6oOjK$ZXg~dRh zeQkttLW&Xuq$UA2y!}7+z%!?QzG&&Dg%f%GkTAOtTc`@yZyUZOnDO{>)9^2W91nN| z%*i$f*(+H>V<*uC!tB{QU@6Kw4*W}`g8N@pNWY@W|6V0f*PzV{c*fib)SR)G)X@b1 zLqUFVB1PJJD)95)lW0J{Ai55HloR8ABX=lFC(~o2#5fuSDEI75)>052VQLH>VtNBN zcf|GfPxnKfqc+iI)TxT_=p7V<uOk7D&$m`HxK6_BjChEDNC;TM{E9+5D8ilcT^qkD z+-cwf;a9ROb|9)TD9kd#7W9DaB)Woi@bD!RCWLf-)*LSGKL6wXb$^d+{r~IF{!~au zcsBLRBFHo`7&~wiI2Gt6FBqU(#dc7xhbFVbX+Ou|lP+{avIIE2!GhkO!Qz9B8Nb_z zmI>^g`W&(KN>~jf+I|>~gX$aME)Y!fe5oLen4bP!f#)>F6&ysn_yMQ^yJ!6Jc_`Z0 zlE>xBHpxc8I82J@-{k00k;e|V;V3n*6kL2;ht<^ELS+1{Yn~%~O8-cp!bs(if4IM1 zxAC@3npeGInB_Azob^x{Rj3eLTeIST5^D+6pT?4URI|?i>t*v3fEr?q?e@(_?Oxn6 zurPPtt}5GqHlC~V1-D=|v|Kk^gVgZpouln7P2IXWYaIpmO<CRH?w7sd8qKJf+v^II zsAJm~QoR-2Bl-@0&AjQuEw<8}ve^1Zm92Z<!;I*<%6ZhXyj2e8x6!0()V8~p#QR?Z zK01~B0h~}o=$2zs8YoPo{@kYmg<<ME<HhtvAtNBywM8v?40n_Y7oyyz^d;!If<7xw z-~!><mXQ3ik5git!pcITlJ(Y{w7g`EhBkxSCU*d+YNB0*N|owl@gz{finu}Y5Z#)j zF8(@8iCK$uUBTv(b^+zh$ov%N8-lHROZrgnCKD4LxsXd(!5XP_Ii+HCD+bT}mVUA_ zu8DJUzq>EMCsz^KL=42vTBDj>s3w~hQSHi;Z8~4`6xkG@Q)e;{87Stk<c5QM@nxEI z&;1=!Y(D5OGKzopaV#Xj=<u^AJ{%9J@~dsH9ktRr2ZaiFXVHD3kzfaEU@xK?uLN|f zyx&=>s);N$&V^Pl8&$IcW$+5wjM}{qL87!s2?&y8qto>etjL!G#hq2$*QyGsIOYWT zs^tY=2=rc8@!BcLt}wEmt6i6sX7Pk+udZErZ-X6eqr$NMp!Mb)i_ij#&|_Q`eNUru z>$|Zm8(A|YOP%;5d0L_%#;s#X_x^a|x9Of+)-LND%l5;Rb%X9@nOVyAUn(ucxXQs= zy#%;aj2k0x#=EJU?Vq|DSlttBy+o4vBh;pezJzoRY2#?tgKbayf%OSIC?b>~N76>$ z^^f75Vh%jJdq~c?x7U}FM~L5*73W<2;5x|wTQCI73Pun4%0}Z$!s;2Sq=51c$d47z zk|b$zRQ*~f*o=f~B{rl+GH(+q=lMf&lMoK&aBCgiWqD??`tSBxg5G_dUGg9}`1j7X zJ)JN4l?#*x2WzQwg~xd*V@lu#2IwW(`tcrG1GWe)257jom9z|r=itW(hzqy2&j7v4 z(OoKZFRel<WAtf@g@zxJ>VyRU3QG^R$j-7&XlLkFKGa6Ad0O%$<sRyrNjilJ5Law| zkznl*T;eo#1c~8zkH+vIz6;9cu{coUctjPeo*vCwB3y)QW2fi%b5wO~P|5>pv47t> z-l5dS<g}WYJjVK>QzxEQ@MNM>E_jFf&wagR#Z~LyZ)q6nmnxrV&#?|l`&v6tZFaHw z_>hI8eTJn>bp6220^7@<zt$yeaIm-+w7Ag&X{qhH>b-)Y=bU`7!`9Y8b5)jRv&SD( z@^Uvn9dqz|vdTEKYuwvlg+=x!^+^f-W#05yB?5M_X$n+^0tHS!CYwYlKoCbs1ZE&X z-j@VMsM4NJ6TFYR9IDI-T-iiLs-mA>C?FOO@HAsO16UqyUWrCj&3D)3WM>x%%&?F2 z(!9|-f&F}Rh&v-RC?M@?4=kh4LHE~_7xTs&0QM$td)8W%G;uU=G06a^gTnpYJ4YWD zX+&F&6o&5_JC#Ia4*Oib{!MrttkYQRgA3TaRWUufGK@{{w&V}<uuFn9<QBrxYA>L( zfn<RGh?<zb8?j=<kQ|VejcUtUx8vs!pVg}^fBKl$Yv#(^ktR?7J>6zqv*D|XgR0WY z1VKV$wD|$nOE2{vo47CAGxCBu$Jke(`U(j?+zFo;kE~GvjFB9IZ>$0v@w3nrNxVGW zdmP~P1;(yLL+-0O_5B5wx-M4g_3s`IspLhemv*QuI=RJi!0XA|V_Kg5*<UUt25ZX) zDcmg3KB{~|nY=zS)XzRo`&z?E`{LU!q5e*!CvhF8hAmPvb{GEs6`XpJ6Zok!(Rtyc zWc`e>pd)X2_I?L8U)EUh{??M5#fR9|)a@4fo(ivrwXchY(G9$Mw!xS((o@e;s#!qf z9zbJ|k;V;YL*V5;xx+h|T<dD8b%*{tGwb$%`_&)Hdv;qe2bcnzn#zirlBX3(eZzgq z*RP!~C;P*-o5)LipaqyT>0oEbN?9heffOLRPhLkl)u|yW6+#HNmY`J5SV}Bm)iY1h zI3F=QX$Ppt%U&eyD*Qww4R7&H&86gwScc`h_p(0L@^4=fZJ-q<m2GbIt;=y)aXxwV z`1w%3!+EylI<Gh1;BWUbbV@S}PSF^&zIXRBX~UU<Cs*p+Thp{VG8VY>PO=^U$lC#k zgL~@ACeGGMfl0wpxv&2m<e&HYb9hqnZ^8=&*+i4ViQV9E2mnxiCSU#w;&HjDqodE~ z0&wB+kEk8|>YhJ`@SkRdt3_1!^CZ3h5zId?#w`@%rux7(Hn$7ayTC3Q3%H;9nFt6< zVZ0w4ho7}I>G}mYoYnm~Mj6+E<rV?IjmJ{+Nv3!<hd#mtK);O9;-{5ax%sDCC;ql4 zl?zKg$A7jr**Qyl%Lhks{S+G3u@Fnf%z-eWTqX9_m48r->ySXOnjzXtn-0&CNpSm1 z|1>8D{5VnIr>qP;Y~+{8**9cRn6LqOIQTke5P){w2bdN-bWq^_3uySYoE##Q24ekW zxOP8%G!}b7omJ=O)2fSvhx9}wc-Ak2ClANO0h|wL6YDm}6jW1sU;Hd1)%n|`Om~FK zC*kXd|9_-p8VJNxZuF0jmJ31ws5?A~PYk0jtzHT9B=`UUbH}C3eG!Nob9Is?N8C;K zpmb~2!9C0a&-<Ymmz=1M`#jAk;r<i-$7W(pZfrH&;2!=kqR4R9RZYii-@OOUzsfIi z3OjL5`HNxlnb~VO)(-2xsII*WD|wwIKXUqMMhA)vF<IagkjcX}>M&<32Daf1=x`{P z36-?bcWg!vmEVHWiik0s080IM?7ukyPnIky*FX{MB)8hua%4Pz98$3y?R&ja2fj=R z_2El5nW*%2ZQhpz&M(sIW(IB|=#BYtA6pYo7T-+0+5=Zd$9K0oogXT+?8xFi_`Nu8 z#Z4sH?hkYFxASygtD(FTtW4<aR~mzko;Mf;^q%P=jsd2w_d{p#{y61lFILQA(paPq zAmU!a-w!}Xnt+%=l^|4tzC(Mp{+@;?w0`*O4hpt7<_9JBuXg^-`oFf%PgeYQNVt0? zUmH*dQM{F6+_DsYHYnh1Ii}gDhi-@&4W*H`pa~g6YP-<u7Yas9Bzfc;xz2f`48dXb zPs1vdn5jiZIw#K6$Y`sZ2kRePv^8Jv*6&;WyeKc8y#B~^g=WslVh8j4g9?t+EO|77 zxiN&4j`3gk`bG$`3XpR~x`c9Bd`V>RN?DraUf#S*0(tE9?D}UJ{yPj@+>0|u44q!k z=N=tCZ(Wqz*Kos^nOr_9S8pu0)^hRV>NDC^*GxKTZ)96LUjm=^Po<ueKVl!c+s3Lz z|I>cw&d`;r>IYj-Ju26Aa%lfHHmd(=n|1M}Z-IeRi$<O<*D=xx$V@W#y5707bw}9m z_fv7t-S_jFxzn|Q-Fa+&9?y=;YGBPLZ9wKFZ-1VRsq><X=G_fESKxlj%3#lju?7Cr z<-~Np*OsHcE+hk&1mWmoXC`CYM(=(rOK_w%Rd9GXr3MrLB0NdkK44pYH@o!?nW&2@ zu}$!4BFxu9OgkR7_ON7Q!kCu(i?A;D^_58rfXZ6Fvrlp2b$vc@+$#CR<@59(M<sP# zGG4@!wG;tar;m*)n#XSx6@{RWln$X2yv#S|RuKDY*w@1FOfum5rOSvh(Le=2mDxO@ zuA9lEUkS{(mP_aCK7H)%9+PZTffwD(?W(tu<J?N_*NdYr(GOgH%y00wA9*`0K>)`j zPIboT4Oga;Q#c^Mn>%8sYFY1cy3i|NNrp#2;a-nI-CG%R+CzF4WobNgfBrwJKu#cY zPyxLg35+BtL!O`n5{z2FOU(t&LQmd!osQdiUBBJijTaeQ0$sx=KxJ#X9_e?r4w|fu z8^A}F_^$orU9!3IisJP9n=W@<DY-fm(tI{3{=NqW+zEGoB>|yTR{*(3f6i?9pWrv? zXrYoQr$z$%@SXU*tDgaKp2g^hBkIhRaeV>Fk+ym4Zj8A~Rz>R%)I59q>;oyd|G6Kt z#}P^UoOHC;@?_xXeahjo@8b{E244G|#`hL-^_UP}0o@=lCEhSc9}EyH`}4*$*t>TQ z)kugQU=sd8q}}Kk+jZ%PwvPd}77!F~I5rnk9oEAOQMz@Pa7*yO)4K<K4<?g+0-1UI zN21u9L-fk7i%y`@oS-is$)5x#oh&GFb0vmZT=ocsn1y_|)^+XWjHnv7>aA~JaPXr! zhqB<2%lMH}B<l8-19N6ynFbM-kk^4ks<xl9gk*^x;ZhfnEYNdY8ADw{UCmBJ?@9<w zfoRj<a^mgF6y^?krkjm_e$TnswTf|$3wb3wRF5v{{O8(%tMi$~jOX`=(g4Y=qqn4% z7uYzRLRd<=px=pKXflsv^dU4b7Q)wT^*<7`C{w6|?q)CYPxEiQaqe|7BTTjIygPN^ zVO5SOp=KzBdlXZ`SWsnY_%nVaPYI-s{poN?Zua|5prZr=0F|R|SU0Q?vf${<W)e}9 zRlOl}d&~=6?QeZMT?Xq3OV&T@@{F5xb*19>uD;c=XLlaAQYcS&p$4`&6q4UlOW}(o zXMr!dOXYY&YUK-Rgi<OvCQ|V^>vlzI-k1)r&=E!tK;GdJXXb4gF|bABzO9$ZM1!5K zJ+vBdA&g1@Y9gcKq45PkhU`JpGK79@0W{rESO#mrFHky$jE|c2TRwBnc=9+9KXm?a zysPMu@sU8?YU{x2(`%5Iclz9nUNBEY>@s6f=ls~y8i6mBt}oJI8;lANdM!wk$!b2| ztwICXw0T2IeJiHVr+wU?Kf?Sn_%4`ZBiCoH@+!Q*Z`&t7gSM0YZypSs3?98X$Lvs> z{wF1?w#k5%HpQ(W3F}{XS+$lGZEam7V_?<x@b|>d*Y!5V<q2y`OQM-tS#eq&o(byq zrVDXO`G2yLTUaZ40)L*M_b1}{=ka)Gs)tbVj!gY~j&6~CJAMTS^S_eTaG_<wV9_P= z`ZoeaQYcR?NAF6tKl{_7`Uf0(H^l9%Xk7Q?2A47Val4L;Vfgt8&umN&N$!dgo>ck# zF8JZ_;f&&xZM9(AdpsqoPZyc41!oaA*UyJNZQ`&s?PEfNerjARF;++rD3L<DG(IWh z(Ceto^@CQ$F}0X>O_H)Zt4__h6k%NX`0}yu^^t;H9-!iNZt@rxh0DO1o@%R6n5S9W zr-WYWw^8&h{R+-AB_xE%=ryV-@YHUc!98i)&XOHDTgvpSkSz~0HMYK-uW?2S7rG~U z&drGIx!`=w`7c)|X33Y&tkD|{7aYbKH3h-gf>*qSjAoY9=pvhZaMUC0%Th*x^JafZ zOjDM#%EMj*R(SWQVcr$k`x$GKLE@WwX0M4Bvi14vS}zYaQQ{aS13F1Zwe)HOnTwjj z=#r%UxkLv4jHs}BcWzA%ht+Mwk8B+ULr8pI$9nI|SdJ=tw4W}cB*064yVX|az8*Q5 z_vq_8MMl{%X74uF)zY6r{?Te9r!Qtg1uC|XEii&>)iJ5B1bJ+WX&5(9l_+SqRiljB zAvX0Yxp%#|6#SdO(Bs(Vt|F<K4S7V@eILGI9WZQIL}Nah{G|0xp|8-Fx;q|QSo{F% zM2@0P*aAHyj==13?wB0g8Xm~MgT~`yF$Ho1U6OkAaYfBi;vh#CyO!GANW~xB4<GNU zS%mD4&UoPZ`n&UpAvI<&+H<M&Fri5=0h$jMBRwX44O-3v5g?*wxlkX;p)rMfxiB7$ zKZG>L=(nGGxb#*2aOJY>%854>I{rRydOz8$-5VfZYN$^3ppC5&{6X#;oedi`?+4!D z;{^ZzVeh@8n%dfLVLTQDM3gQF@ks9$iUN|?0tg5Q2q*|Sh)5G73KBGApF^)=0a03x zBE7^yQAz@Yauksc5~L&%sgZ;dwj|l!<!_8Te)qfgy!X3x+&jMS{fic|ch+8OJ<poY zob&OcSX`vk$+1k>-=+~g{)xUXx>T2+_FR`KGJ2KW<h;;eZsb*91FdH`wk;mMcpg4^ zG3`|59BLah^#oDICVDFh3Mkjm$qFN^QJxF##$vGYE)KdaZ)fLmScI*3?#j-W$Op=6 zue9!tKiK_@t#|vR+T3R@a`LDbL}obhhYWQ#;{Jh>L%Pr}9Dz55ItEVfAn)jJwBd*` zDd~SE8zfU|ggcfbz4%Uo0)kexmBx8<cg|sNM++*oRP{g(R$}&Lpts_SWv4KK@{qd> zH|{@Cd;G_+jrjoar4J)wdjK3f%r;@sd7E+8?LW0e@r`JPBN&Gd@R|ur?Rk)e7+fqf z(mJau04J`XrI;5NgU++Hyn-e$iO5m*qr4s2E*RCxv3nDzdVI)pP%%Jejx51PXAlu5 zra=+}Y*j{u+aO`X_*x*Ft6BPkAQNO-Q%j62^PHt9SSZZ{&*gAXHCT{!ds*S8ytfv% zF6EmHRmuOAt_|PBG~LULj+EXk5GHKrUxj^m)_4uJL8n19qLtn;DT=7ghT^iC$@&Sl zBZi*P?f?_1o~^azfkp7cGq0QCKf2Z3e;Qc%Y~`7jp(p);3HBVx0=8#UTPRzANtiKg z#9pwomuN<b00$aWJb8Ox!S2|8zS>mOW=D2ULKxd8Nha09;YmPjndYU~xIOK+#s3ST zhRjM9ON*yQ=z(fRVixgxI6QjoLuRm-u#;Cri?>?xj5B+r%`rY-Gqz(m{~)OSA8uW* z1^5*eEq{XDdTDUg-l}79JHIybnSUBN?b+{W)v#F*JOBJ4^rCRpEo)u*CuEXE_=|h# zxB9Y9`Zx*^-Octt6)mDH!4f+KIQ_(0bOOt#Hp$!5hfYELRkvd<kg0cS`wWlCg#<-} zt;k^h&NrOSW@2K!!QIug!_GO!ITm?aA~p<C@mm=w7X=M1lah^_7&>P5H|eCb%EbLM zRwH2ov~%55S5Vn|#|HOyO11FfVRkXoKb1u=!nt^G(iazaPP6rwJ=_zZK)_R5--w7K zXyhI0eO{|93#a&&1ZHKUI?)LSdlXQx6+gHzxc-c1zUOvbtgW}zCT^z8_unRKQLN28 zWfLwC65G+a#B#jat5B>j4^Q|=kmtx2vV(NCVAaRY3JMX?%8#WC^k~l6&}qe{E5nd9 zENXoJ8y0U(|1@E?eS;zbunP~gkE0S{Y{P$X>(tU+Z%K}ICy*cS{h3Z<gfY0A9)00U z31q_o{_&8gs+Osq9k>yj!J?X>#@pt?JF^%8i~IHUSH-WHE!Y(8^0MA8Jg4^0Y%~9d ze-9uJ)(p>!A0o)bJM#4T2L$<uBwrUU2DwQNoWK!oRk#3maU9zzqUKxaX(sf#R!ACu zZa7^0<OQ~MD@iU$E@DL1dv9>?`GNuSm%1_C_njL5n%|XY)0%f=#UVuX5v%Sp`RbF8 zhIH+RS-SgjLKYtkQSJkB0zn_lH$eSq@uJKE1JztKoDB>P0Bgw)#U6y%?$`|@BCH{S zr~~~u04w%*CvYwN!Jl1tS>JO}w9yE`R_YS;h0y1ONum7he^~R4P<)v&B+Va~Ndg6K zCC?rxH?}zm(w~4glt!izA3<dn)O*ol9kFJ~$iNMN4rvXNF<`b)Z~@Hl>wxhTOc;O! zn=+t(a;u-d69t<bzhS=q>yG}<jFIopq{yXzRmtDEFH9#L?AQukg0GqME<2XlnYSY~ zigyD2UiI%Athtj|8TMUYU-n4sh@|Sl$c8V!l?8J~5QTI^l=VZ%3KUVdfKz4Lu%HNV zYd32G&TIjg0u4}K1(>5pAdV3wkb?aF9{q9!z3~h2Gvx!i=GD@MSOUQN|3U!uD+ma0 zj6rS!X4=*Zg3?AGo(o|dgs!^9K6WJDMI~@T7>;0n#U>FIds@&5#&RYGbZmU~&h%28 zd#mZ?+D8W*4`2N38T5Sq_T7TWit(dOO>eD`=g`yx6ws$ift%2WE~lw*ZHE9W)INfn z#h#4gyD|)R!#Z9gWYh(`DQ0URqWjhh<IRpp2(v@W1*;o~p_9K3*WfO_X?pV}e!XoY z{+eI>>HT#a5*&nvb?O8PJ$xwu<VksQa{vWyN5MZG4x|&D;%7yn8$W~&Aj4ghS$z~p zOjNkZ$pCT?5|rTmdESJNR7PDJHUW``{g?s?rUXlz?b4Ai#dc2($DnW)5|T&{aWAqD zFFmy8C<c?f*mvXZl+C)Y6Sh9!j#jPa=)~UTTkC4S4`v0eWQShUyU;$mQP68|qoon9 z7CsBuDbEi)qwGP{2;-ExZIwJ%!XR3Ra>Zb@i|fL{#6nk4l6W2VwYN-kI;xt~J_#JI zI2KJDjj8nCEsXZ8lEm;9;DqfJ9Q@qlJiI~c#M2&UUkDpjU%s#E>hbqmh}Yn&4^cs( z<#u9P1No)p7goQ8p8Ig!FvvFZ;u#-j1;eFqIFeX>&YiLJPxhLBx}#(k|NC|ZwxeLb z$yGUtCO`z^S^jr#jn@DERtel%uOC9kuk$9qqyC5PD+BWV3axHMzl9z{|LIM6-(;d= zi90QRg^THqX^sALJq-*8(+mWCK@(`$wF+~>158n}-iY~TCMsTmTv_-bb>~7l=$X(i zd&5#4_GI<-Obd#JhXWw6w&*1wov4~z4MwvaP<(qV4GYv^htmJatmL5%gFkBty7*L{ zCkPvVzKL4mr_;1`I1&GJ!v1%yU%|6y6glW$H}(I%)BPN&Uo4El3r~S3SVVaZQ4My3 z1~HYu@2d570+sknH=Re4_rM<$yo3V8Jv;gX>g$rKu@V<e;;Sbm3GJqSY2Qi`+^tW4 zBe5>+KV0wBj3%$WOKfEvc$IokI#0{SOJgYHW?8&?yt+Q_c3H)xMcORuPSghgCpUlY z9(H$$QgOAH2^$$k*^|B0jP75joeT;I)_mdR8X}xc3#>mV(uQd~C_B1xiPC+NImuT5 zi)(8x8|rl2^;6`rjjTubfZj#cm;gx(;oz)7XA1@#%BC=>cr^1Z-bEwrYE93MS!GqW zamq9Jm(y$CV`(AetSXyrd3yE0qF{%U4Q2wgc9G-0fQxxru)&lWUk3J_l1L^>K>-Fd zyH@CRXIvPRjNWZ-$?j~<)|*TuD+97rQYZtn3zyRhNtip8h~cgh<S&k?%F<4KSf20u z>tm^u4xjYT_ll8(W(LRUCY3(zw)<8{*e!?2o7Q^xYjWahqH9cBXLlQJ6KS1j2Yez4 zhX^!ChT^X$cLqeSSi;BpI?&AIJl>qA5l2>Smo1)3-wg{hhz~b+76GD$a3Wa|7vs`b zkZ|s9KfUIdvSLQo>B0ROYSkWo6Pc<%gq~*p4`fByf6uV4{3|c|Ka&^z|1z7!xPy%e zSY!lECJ}2ODkaq;s{<XMzXiU)F_?;A$$=8`ZO*j>b`L{1(LxQDa~;I*{u9<|(_t9w zgo*7ILhO%E)iSsfhS4fAA0KW>QqNy{@JhQ#K6R&on0(Lf6XyyvE>F5QrOo<YfAi#s z&q7DVv#S2t21H{5Kw~$Qj{ip-<G=D%cxHqzgJv4EV}3`yz;yN`Pj12c8BlwYy-~Gr zcoA&Z!49yH<5>2@eFN00o6Q#6DZ{#}5(=~Rj~<pzv=_kXP6w<@&U=lC+%ms+S}ug( zi?<fUqk|k%2!~*M&Sr)jU&fAG#1T#*Z(H!z=E${<dm_?~ixP!Ut3d^Btc6DU-P|#f zD=x)7TU#@X4AJ$lF4fCx^4Ay%EQ_dm%NzGU5Pt>3XiAyNMZGlWq<m83o`;`sa;7BW z5mk7FVch~uh)A6Tb&mI2roE#R$34kNGyZ8*#lkj><YGPf5iIiLy~Kp(_02-TtR6Uw z#Fr`tXE2Z|z$P5Z4<REX?ptr$*;b<k=i<anOOck9kCO3O{k^y6^9K3~n6Gu|ZaD?E zg|@qkW<;t^+dFy&<vz+jW;%Hmg!JQr7ogt^s>TG72j0Bkw0Oc2#r_LjD<M#e;ey1S z3V`sRZS7`H7a&SH;QP6sM>#ec5$A&pquU<Ep2Gma2*`W?03tao6GDQ5KXYX$Gn0rH zm^)t}(h~;nHl_#k2N;CD@B80##eo0E1s~t}*Y*DAp5VX6^MCD%_`iKT{XieYrB0A7 zA>b%ni&|XZJmjWxidh=q{2qrEr(B*y;iTFo&}@kogUz^ymQ8TG2}>e`DXy0_l}>fs zaUtk-Fw|Pj9%#PC^3RvGsXZg<=D`l}<(?MgL4Fv}sgf@>zXW8+enoh_oX#bl+br4e zXPX8$j_y4;%eIfOJq=^~a;UP#K8T@{b0^A0Zuc)l-I6K`KWB?nSn-FDmJvw4Vt^j1 z2~=*-t90Jwa|8CSMMOh2(JJvf9+~?Gp3QJ8>MNNA?)o97uBe71?;m6n-sh?kYg19J zIkOw{`JJE8Y%9ul!=)q&7b3Eqfesi8(PSy^<;w}k_K_b#>UkU!q-O;Y#el{pOS}^! z^J|1)47^6^H0UNi?M1et*w)L(F=?nefND6>(wzj=pjxmrT0Et}a%85D2Df@26XZg{ zs8q7q2tkQ|SU^O6#ig{HuD3#om{-JDbH$>I=+4PlM7Op-EK!%B57)aZgf~4O=r*^C z^gCT}Y6d+;LhUr)!zo}I0M*IVdfT{Cqbs|s6H_DHxtS@k?Q@U^nfM%+j_gkVRKQK< zSSNIpWginXd`@xP6?jYTdNgcPnOT1sC7XkYD1ptI@undfVRFy0rPqgFT=W?8?k<J< zJsa+L*3~_lr-@~8j^qA(WM}7gUnaW2)9SorO1i(sdyTZ|c5kTy+hX0P8miNNvTIJa z9G@s#1!q&YO~=_@uXGl5$~}kC8}l@-TJ`_+In;ut1S*L!Q0<gtV8J1y!Y!)dtap0d zX(g+}r6j2Jo*-rhJnZ6&Z1>;!N7-Fe6Ut}=`c?U80PzYN9ovZZ1zq@tuXhWG+?>Se z#z%LO%pBp%s4-#WK?~0XqW=yaeH6eMRROlSGZXv(u%W&R7zNHAh{+G3uSMy98xjw+ ze#}sD$lL0$n|82m*ygs{maMgLqK5goAe)bc{ztoRu~$6_mG2kP^n1`%)P|if3xJvN zn-P(9fMwpd<{y$^feB~xo1m1!ZU(asfLM~?pq*g*Gn7NK!*~gxFoeL|Tme0EOhLH| z9EPl3PeFV|7=+R6Y?6;B0&aVH2tuqkzTaofL1@Gie1tDC$R1&C3ov+jK^UBbe?x;F zrvedah`JB?gkF6lLCXQW3gLfiGy8P^-+eSgh*}5%Y2|9Lg)q1mN*a*nloot)0}l4k z_i!+I?;~WNe_ifhgDGV6|Lv(_1ZMu0^4>6tFn<&LZpv|Ii%~)>8ATWeMAF~+a?hGr zYXVAwxf0!gMr9PG&`CRia?IqgopP36p~qOumMd1D?r+Nd^1zT*E*kEuxl(G#Osq6W z$g(0DL@f=3Sj}rVxN5yWbUoDGRNA+R8=O;zeX{J|hD|~3LV$&(useg&p1zwX_tQAK zRbtb;@&S%rpn`0SMxs#)Hq}CLKz-<IU*o}|(Lh>w+nn#-2bO+;Ay;3SMIJcy__!e4 zol{s0y>H0h1C-<kr$9LJzHJHl)o@06H}WxBCR%n9+s1}!#~QE=7O(f)9_;MU?sNU} zz(e(c$39XDXeTl$$A@@mu>C?I#JiBFQAwi-$BZt#Sjm}TSW_o=+ZWDxe7-G_X0F8U zX{%q44=?JgS<uN*@?5NMPs?S1R-=r5U!l8SMV^A1WFKkTAN;QYPm?eNjDWF|dyziW zwpGUzZ*5>|419cHbuah!`p)EUPxvZ-;&L1Zs<GlZM~q+q^}npFP{r+!dA~fYGR^!= zDDv?R@iVOrPi;eEGcJ;JXzvR9EM^Msb__H-0UP^|dL=Yta^1HP-(loeLOWE0VTl#; zm4^trP|G<e2rc;;ug958ARoaWJCAH3G;i$31v9+TWB5*VJBeNU*z1c=0|iz0aS@$l z?ONFga5Q}xTKc`;^H`hgG8M3p{&pL^2ekm`Wh!_67t-R(DL+>s?KIDhW}$6DLb&2T zk}-c5Kh{^TTeRvMUhgZA{Wf?|=2=dkSK$`YQ}$43Rbxi^B{kPdsZ3Glz*`Lu2cWrx ze-<tN{(SXc9GL?CKCXE1L?>64ehBH8113Fs<U`k3KLYV<K^f(p@c*7GTED0S_5$E) z^Au2w5a6l*y5he@{r|b=_^(m@`&{b(+EESul1^iCcL{);;L00{pJOum7xIo-qPRH} z$Ri-s3FVkJL}|G3Vs`W_L0eE@F7^~ow+V0L?09MKbEja!%k!cY=T(<DwWGOww~J}G zR{O_ule*!z+D5L?LU84R1&+SVug*@fSKbyrBP*woX3F`*r>hBz>m`If;tt9+A`0oA zlmS~?Itm%^-Uf}DG*2q7>QItUJ1vB8C=YgL@){=9;wW3W5L@hp7l$3`;gtTbeXC-! z0ZaX?2d;Wzr^L4L91#(aB<};1HAlg7Dsvqv$Nb^C{3s6`r$RUhf`S4%1*xXAdu-JG z-3zBUqP5#HB6Bl>-{}V)HM`MOkpta1)|hfJ#NAI$d)KvhmR`$~rylSnMzl4bk&kTj z{^zy;fAjO^??C+<Z4JC`(3cE&CZtW!Ts_4yc%}T`jw_lU1c{WziAF%=E}9p}kcJNs zzdgomtOG0xm34_S5sF${hBjP)NUDCvFUdL#EQ-*tY!<_vfp_y%`G);?t&%y9tX6aq z289j~-0>P%OA)xYOzt#n=tehFL=7iJF9z1HRgyCqUJ%0-d4}&kF@ebT^QFpEjo+DB zhwd=F@A~3X)WW6dBLm434tWZ1lGFS1L}kjlcp1~d4H`j`VGF~wM+(|54@YmFi&F?B z-z^NPE@|aD6rNV^kHB2*!^9G|fH*^fM9epFu-vck9nietdJNT<F2{P!l!&G5_)-JA zGL@|V0YmKgV-n4^h{Rhs2h{hF%?$&1nSzTrirWaQa*ZVG{i$&9(aOFC+~U^&$PV<W z1cqP!g2%;dD<TKW5MKoZ@U|Q!M|ZYH3=l7(L5GBbPJ&i%*lE40aFSG@_g2`5E)Kb8 zwLZG%?P2t>Y}l^&)U{{UZzXLMO!W44Zp9chU%B5l5HE4CAih$>_WDJN2i2v0pulP6 zMn_|yYkGC@htHRvKg@7HJWKkc-G}q0VoS;U^QmgKtF+Y1-<#qtIOUzz(X!1l#O@i@ z*kf6p^KvajX1Sg)Hu)-y_!`YOKE%Jt{)Zr&@@n!NT1NTnhMwltA;sfSGSx}CRz>6k z%KKf4ARk2(i^MuE$kw$ON7B6yP}pZNf1vjQy5AD5yaW9Te8#zOgCx`@VCc?l&n}J! zT`$%<F+0tprt&VKgcc^$*j5cye0{hJsIT3S<_4mJ&DC#T2S6!H_Er1dR~eZySB}P` zbX&jTa!=lfv{J58uhG0PoThzl+y;Gx7R2#hI-Kemv~2SkXTDQK5hHveH&WKL@c_Hj za0nf2dcW}f*N5(@Gcu|qRhnCa4U*7@$-%tRVD(Iq<bJBwqM<<aUb4#AJf;T4isYHY ziVU|ry1`DI$;F`>q7(-GM;NpVsQEqO697kv8evS7FyAe{wpd0hDlu%5J^I}9bX6n& zB*v2VQ(v4m6GI7fM2+DsSIpIT06OSYmJ&P*$HJ%Zw}@<#`ElU6FKcm_Q<(5cmEHW7 z<NvAAq-6Bs_xio2xpK-84+B-#ML7n9gMH16gYa!Q+r7(MW6Z|>#)%`g)w7Hsu!aW; zoGUe5b2D6zx)mzqzABae{$jP9Q4kk)eMm#s|K&NxQw^=I-Fx20sz~Q#%y;BjI~;5% zU7UIM;i*oPktOH=V1S6;<VoUH-oRKNW_qeSH@l|;6+?^YB*o?L?M0;V{+#qA{!a_x z{EuRKcc|V9><DTXk9G2_R@C0*x&50eJTG{TW5&;)^!1Aj6@KP{jALHeFD-V-vC@J$ zovnwE?twAYKD^!T1-mU-4}wopL*Y>Tp^dY{6uL21^3YuQpI}S=ArxZ#XKUMe#b^5~ z{Jc|T)jn2Vdf}X*W1`xWL3az&5Uqn6GIaV&t(-D*g9h@9Gw7<h0mH<h?^?Fd#Z)z` zJ}j-H(8x`)*l(#LtWe=zXnp2Ut@bz7JryM+05JFy$6UDA1f}6bc^CtDW;LEbZ^1{` z^^+R?!SsW65;X98L8C>MhL@z0gf|%u=cKgM@2tfijhsHMS0lgZ-Qn({@OjVi<DEDj z_a5gnst(-@RCV;Chje>u+d=YfzAgNnXT3oCMy$ZhgN9I{g%g3uR@kWsv`@4jsSH^q zNiO^lk|eE3eGQUbXV+#;<vIOYYeO?j45nH32s;>L<JOI=Xe(u5<?f}LO}T-6)RA&^ za<>+3*~aLv@1ZV)#p|~E<&e+Eu(^TM2HQ%H!9bU$hj|_mL5IFtj6O@&{_CgG6>SOy z62GH~u)4b-j&fB(aQ2snQ&|4<5kwS#8&}rA@d^gw*aX&nJ07ThzT*S9;Sgm}kM|oi z|GPACGc>IZRxAWIa#*7i$wgmHp293i0#0MPV8}Xx4~RRTB?O}u0PBc00{>o&w+AW# z@Fvk&z-Rpq(p$69gvt14sN9O_524OV<S5~)KsJF5ke6um12hUfRft*+2c-rZ3EolQ zs%-W{=)4a13F!_AY`oDK(7O?EkE-ZJfH(y#34SBdt~7lkPsRb*1>=cOfX3!@5#C=C zq5!al1i+a7B4F4U_3H}#zfxfOH^(1L9xw-#_^vitO%goEAfg5UBTV4jpja-f6d)o` zfpo8d3#LwW3b?!_5G}tz^xe&W2u%RS8@&{Os)|&>1m&kyd>ty^G=g9LmQLiUD1%WR zntATd^~7}i49nWzaQ?_&od9-Pi=2}YJQ|_;k4;m!;u3?rGwD~UNUO%iRzUS^LoF){ z$Pi0yvQ`5ry6s2UaG$~~9{(Zq>(bZ^((lE)1kL*wxnA$#r~3U6@-F<|map<d=z6*U zpZ=w%0o46@EA(g(Khq@$eTXTihqb_w;QF&~J){Q`cJoDGVLD2Ie}&`sm0126S7nfj zk>y(gF6$0+Vfwi3nD^b*j#gAczC1^nRNi^B`13fL`!$^#Sy^!Cdml@M#r_mkb}7#l zHSPUA?&_)B!fKbeJty?$v|kn5+*#Y7cf;-zLkGB|H6ALOo)`-5r*?Ej5|*ovR<fzt z*ZN|!dEUvem^XNqPmxw+5B#JIuRV+$seDxT6#tJDIy%uDEJo28Mt5fmw3B}}dw!xB ze~cr+5)PiSW~;|Bw+vqUvpT<ZsqJh}2(V4N^!9w!51~*`v?4V19P_exk;5eI<sX2{ zIYB%d_&%FSwUFhIQureDp14VQz{RPewjc*m+=!<+_K%X-OU<Sg8%qh#Vos5x9F|-3 zD_32EZ&tkX;dY_XBEByUJhhfG_tT`ml)PT-G#DNGrSwjOybbw+jgw%-rnZ!@{5hZV zs`XOd?Z<xcB^>Do8FR~a`3Ex{FBNKzWor!y`a@Lb?+|4ZK4-Z9<*h4dLSIhb$`^%0 zcs79hwr4eX63m1>`i6F>*lYlxuA=4OrHgP*IWAuD$!gQ)W}*atH=T8|&0CA31zNcr zo~ylmZiEYs)}7!^d>(U9=>jnau)nz$9Ft~1r>N#;a-z_52fp~*p{o{$f@oQP5Y%S# z4Pc+rS)#9L(S0Tqr_dAg8wZ=P_Lw=S`&>3lLE)mE_<P44H%_$L_Sb4iB;aO8NTL!t zsnba=8JF0cqCaP}`Q3LkTh0}Ioxf31cJD@=IX2JKDA-y=`&^;hk#+gr(3A(2T1Pd@ z>@`?>bJd?cjnEMB;8#rs^kJm=+reW_LOM~^b?xHTQzI>;)O)xb#<8q;#f}ZaAEmGc z104;_O`?4pcw6!h!+}+}m6nYwM2=%?aM+330(w`abFYeS2WJ22X@c+n>N#_++&SY# zZ`%3yxGtXbbCUa`vI2uEq{!vrpp@yRvKo6H>sM1(>4P##M>TyU!~Oh&g3bx5y{xOH zONw1@daMp)uZLR*01FBq4EQoeY|7#S_$n-@BugOu7-`IL7{wl&l31K!0@T?;iapZ` zScCZEoM*xLhoHm<d<V`JQ<GNW*7*sw5%Z&gzE5!kPhM%|^nl^zH)-G0YdlNV1p|zw zu}-cP8@(u)!q6VJp%fbrSqS5uLPRCvz4iO9Hd->kew0{Woa6=s)8epPi-mk2)@Vr< zN5;OKjPI*3e7km9(Z1d;<w*AMlx1i12R3d|nzLF~a<<T$tbqTfHBf#MmD2S#$ix2l z2q~t5Jx30z{(kwv!wi+0tLppjs`;qdyswyRtvKouRCtsgaA{gIAh*Ew(f;g8)whA& zHYV>gh$DvXe?bzIYfGay8Fej0C4M9v$`QW6DgK=-H@}{*1g8|k5p@6daYqI*8gJth z5#k>~CraW)YS@Z6rX6Shy9n&Q7Z0h=Sgb<1&{pn~$M|Uwm;_+%l;Twd4Xcks441ax z&%hBJ(S;6_?5Yl0e3GvY3rcXxUYqH~DQn}~IxiBL*P7Tar`VP8btM7y_Dzd4X<Bm7 z?U13**{3)*P>8l0ryVGP;yo3T&(j_UGMHO($jXuhaxPEwJ?$2xryWebmfO3QI~*=N zQ9U<fY7|J%KNR-)Kws61kJwY?H71MWYt9b21$ug-ck_-KnN~9m2D5dh^zI;{2|O{_ zpK+y>UG51pYCR_6IKLRgBKtU4z78?P1lazYaZUu6-`ND(g`%sKNPnQ3v*l8yoa&?d z?}$-$pFCE>6eLeyY7n&F$zp}_jt631)h+gLUUTDyDmqG0tvv}FOG#msyS7D|+cDzm zcycj~df2@;3cAC~a8)Z-1B_GQ5f7bRPrgace)IX5?$)dC4{vlxrm9&4+0j;(TaAte zmSFO4#7;4{PdK}3jT#-9w8fcQo(bRkmimdQW4r5YDX%v7>s)(J_UHREsn4c$K-0@{ zrcRWcWhAlX&Qj7wajQYRX>msbM(jKuGsqVPDGuXW-c*W`yFHtb!z^b>zF%nDz{XdE zcl(}gtm~^>OS(2yCudT3Sboawf!n3v0i9lw?KvH{r$?=v3$%{EsnM*H9SD8U?-UX+ zoECIf+sM~DQ@fX<7SLfk*G{uB3R@g_J!F`D`vB=93D&aaKkONeK>9&U0U5v#q2Z@+ zB<5Mo_aU&_K+^Cak_Y&WG)riT2nslZXbHe*MwZ<9x(igO5!3)B6#)k8iT45N!Yqsh zLEV7kMg|yIv^8KMgKSp{aDxxPV6v5sOu6EJ0=ECnFYeq8y#+B_4ODvm+BlOSSS}XC zP(kX7zF6`s86H?T@!w$hCNY!pw?|{}$yFox1&6u&K}L0(Er;Zkuewprw?g&6x$rZ3 zHG|-l{6i?R4H|g|S`wVmB&onu4mg=QF0TCf2`*sHbU<;KEx^VIlcr-gLe|9}XBSh) zRa}O4gqU!z<(1%GyHzJ|fFW?dx#P_87ONW#$M>(Lcl5-V5?kpUsy$rC?m7$WhF##F zwlKJWTW_tyi1j3dc%S|jz63k|Ipk@Jblg2%ctM{$@<qj}C0Ax&z=E?A$;3@t-t&(8 zhe`*|jCbEeZ)4PSH#Wkn=2z#kap}@`jaC^$MDkKsK;Y6H+MT))`yTr%=iDNGEBo{H z;az2A1!|A;g@pDJgMONOMu#lHlT7TGFn5*GJp^>aoU#=<sl}|5dW!NIy+{o$$Lq7j zzdk<yP$vE@vjow=CCdD&;V3!w;m=Ylv=7dm`%#e2zd0qbhwskzXSwo@z$7|EjR~^Y zkGEp4cTC3DQ~qfD1)HbxQrU>(m7e6?*_w}kVD&2x&qV_7<D16fae8P()CSDE#UavS zSfvgZ7Ni&*b8%^`9atSVaTJsiwoOdV%dNjwtI11C%d5MaZP)Pbn(Eyn&(bb@iIj1* z+of}ar}p?nac$~?xq0b8VR#tFNb|iBgVfDZb4Z<ae&Ck5*Gk=D-E{ixN`-iUse^P= zwT4?E(Lincxf@T-s$6B<fQ{0(?@wJMPI%}{vZ<IMM4x{cz6s`jxqujs`t*?71Sc^_ zV`y%kprD#G$~?~|HcRXVp=bvmPtf42vi+$|QeK0NA-chVbP3@=T6s~SKBYqMQ9@`i z_Ig;FEyATb)xBP)B;9srIr0qQa?q{JlF{WzE9wNbf66C!5Vyn|!@-862ydc8>8L$; zSKb+%#bhh87n6{Ggkw?8e%)T5^*dSmUBTRX9oPK;ZEjfMZe>YK$$V%_5FPz2XgP&+ z7cE*rHkw1fE*wc=U4r|n_(z8<^yVzK8Y@t}C-7#VF{D3e%-Qy?R6X~qZ*2l*P`5H$ zqu%quh-jCXb=F?o8R0?p=oQRf&peNFHx{+~EJq3+`km*iZC|!5nsJEN*^`#z;N+^= zS8jUkk#a@Q#*x&IxKgH$oiNH*`|;w*fhT?CPnZXcf*ol`8cw*h`n>2(u_^R;l{(~6 zt#<Nz8t4Od80lvsA{6$>`9X_Kcxb56k7oq6c=7Fc-ux4S{KiPSPKX8*+ddhauK<rR zdUj75-+u<U)pJ0i<V>?pt`-nFd9q)lBrcNKtM>wy$AJV?U^Rqjjw8T`Oi<H^0r?tN z3NBzHlZgCC2Stp3kY|bag&nK-s<42Q)kdQvH2RfjG%s!Aqd5M*>osD^VPU(bhMKqO zXX_i1KQ4I==l9t5f6G%Rqn}}_9a$RelU#e)tBfBgc-k1{!|tKSCzJ%x+{zi_j`kzr zo@RO{+Z19=-O{mY#010gv7*2=vBXPm=l8fi-#Y4^9kIyaoyJQE$`RRsNrj&xOyy5N z4DZEIWCc6)HF_1z(Y|H+LwD#b=NZf4-;#UUZhPhRKQ^*6dwd~(t6PB0iLU6#nMXb^ zBi($c8834`;poTGPPe#(2Yigw7gNa$d{cO&g?<3`xL5p*uo37`c;=XYLUz0_Mm)LY zYt8lH^0iXe;-g19@=oiHN#&LuOlj0O!q8^?s-2$i-EJa(qDH~B6avkf?TDjP?me(4 z+{C4Doq@<ov3XK~d<>_-9TwzSh~jnNy2=H#2;Q**j}^od)NmS9chiJF$chWuJ5%WI z*09?NA5PjfC0LQ~ay5VJ4QGjj+q+iEe)9r5;BFS-FkhPO&zD``dOzg4fCl2~@B+7i zLt$;=KvNP)h!kFyBaw>8;s#nVauok@Vd_0=jkbkb=~jJso9FWzI}63XrG-Z*zmC+5 z9<dx)v*R^*9;Fo1bjlBR6+Es8JJwj@(&@9JE9GHN-TLIxflu%HoF9D6m}Ok=4O{4b zf8(i^b4j6@n@iS}bb~gZP}}HocL~hqlxyF#agAHFor+zbXQyg|j<XIK(!Q+yW$hiy zF$l)ofMdMC^%OJ$-^-28bn-6H{N&hrJY_10hK<Ezrh4Lt+wm7TC<Zph$PTy;>ZhEI zilApDEKYi#XN89x2pqXjn_D+Ac6aCGbm8=T>)+PSl9d1~y-Nd(GY;uj)_~i2Dxk6A zw}(7SsO1zsigyXNo6?|CcD1l)xFpbGb`Orou!^a|sW7mAG#*we?4c11akpluFW@KU z%AG8ekdiLnLtRIsEF4D+((-i$zav`-gUaFpa7qrqrceHYh!A>T@TErx(y%EeWfH|# zVs}K@(beBFbqpjIHrunYsd!J{ZKTaO(y2NiM@#m!Q4I;=E|`{vp&jrl@EqIjXL7xS z_hqA)a;-gS`MU>dQqm=Yl(k`NAjHE+nw$8LXcs=s%Xuh$dpkAm%^e;&pG<Wb@P9Bm zeW73Jgvf<o>=lhVB$W1sM-COc%b7lTUMOEOSeQ_p%1C}Po#S8>S>TbsEw`*AOvh(l zyZ5QHQPkc2`QCj~bH5#^o_+V*>`A@#qPIb=rO|(ZD(|y_ptGrJ4rFb7Zq2V3?(8|2 z^1Q6K^!qUoLO8=vfTJWbNRcz@ksbva*m9m(AZ80*=NV3gn<IFQzC0;T=$fo?txJ<? z&#F^MZwWTzIVLDe3?kZi&AL7nFYhSlb?T%0Co3*K22U$g#RMFDTa4HFSWX=)22e1+ z&JTcghC`9;k*Pc;{TXJI$n|+MM3%#CTREd^lT5wH8HA)7Pl?`pSez8{_3XI<R4*v3 zUxA%C63y9JZJ<I4JTLhwOj1JqDMY7c(A;Ng{90GL!VWpYHp*ZaMeah1VCRMY;BK^Y zf4HW;Pil1%;CfZR5aO@%w0@d<`8tlpICJvv^K|exddUZZ&=R^~Ue+CMG?N5=aDkF& zSKHxvS(@RxB#uNi7CDk1rE}KxyoVLa?-KpEa~Rb&efU5aJ^6gu>h*9z?%s@fTlq%o z5oHtWZuQa=HKIn&uUpTNSpFV2N-xk8U85eE-k>%bX@9CPEn%r`k8(?Htx$1p^zBL9 zAn$)}pnTCwJ1Eoi?3Y4RH2_2wjz)uIOyVPv=8rf6Ni#rle6VAiLHk0X=YLN>b4CWx z%MhA<hQt#id_ryfxmBZi^1BU*FAZup3y26vwu&Xj_P>$iwNVyTeh7s-jRT?A521DN zrneklKvQYZa^R#Cv<RHfrwT!8Isw2VJp##n8XW3Lh`kvDD<IDBr9m0ibCMfJ8k*;S zq;M9XyO<ob<x{>WdMf=(dS}ni{5KEcOP)kNLfpF``xO-w1y3xr{BxMBjt+g719fsz zKR~~<h<n<IORE5l8-!L%?|12*wNza>egEVRtnRD7-rsMR4)V59y|r(@cC?L#(2i{W zJ-~R2{)@lm&C5PS^&)<!plo$=)Vd8NsYj5TCDwKnut!?26S_z0+re@~({~P-JF30M zTViGBCdKs|z3R@yv_*JJCD;bP^>xeJf8P&XPGfa9<tJ5)kI#*GG*wn*mR$TiXKWmE zmOP|4Rh@mIdco|MX76!Fg=(kP(~{;-51F_xlYmJqwPo_9^<o1ZOY`S$6;v5I-o?DM zUus*D2oOd@(s{>m*p>-ZIH70~ZwWUQ2jpwQfozSov-W{Z^d<lcSq+7Ejg3ic$K4(A zuB-Em)W{0b{ytGxm)(8+aiqPj`E-1YU1k;DN?ZC;xlY-OIM;5gI1lY-#oo5zQ*%~5 zF%t1{SC#w~uHl~8+A6Q44QdYNW@;JEnZC?YomN;b7I8|m6e*pj1R#gN8a&OHVUIAD z+MsC4PPD%xjmRzKB($K#0|;91w8rAt6nYEbRKoUwj1WyYfm*LlDaEZb+!MA@n4U}G zF(tIwY|)^N)JKKqn_ropnz#d4ilR(*PBYjwswHUH7SwBq70x?B=+5qG8MQFR@NdGu zPC3R!Ol)N*#8-b`>w+X_$p;b`=nkTI5X~`V^$HuDXSg*F0&nU)9YYaSyL3bGR8Fux z><95Ykm1SQfYxX*mXG~}=2F-eMD{#Hi=mS`9mQ~p7vW7DiWPe)dWr%=%${9%G)FL= zxiL$kd!`)IhW5aA?yv8hE;!)0rLI)<W+48gpkU3B#ze#y;dS1rVB*fOo;cHyHqxgQ zZrq>|BphJ;`T~4K&(XHe{3d;j1+^adT#|+C5?q-%{m{KX+gG)+c;OLCci?T{^A#Q6 zm)HNgncE?gHN5<OcUxhAtyJn9v(SDZV{Zn#CMx#a(WXb*xwEdWE@|g)aOrGZ(A#*| zgfMJ;=Wt<s#@pgO9XIbPxXi>3`YCi}S3F~~t}hojE1q-uD$o%;LI)qS=02?*GTMXo zO9zkWcAs7~Y9}uJA0w0iNDQyb=4LW0&Dvgmb_@Z{6@YG!Yydy=Oq!}rrUhp<C3jxE zov}^j@E$jV3InY3aO=kRUq0())_k6p{Kw{+=Qqq+FK?<fo_(J$;*y)1>i%Vh)xxoE zp4<+!sITDyhQ&i>8r15M)zb5O{#HtF<0g(iHpZjlbtp%2g$`C+*to$t_+?07D3CR5 zb1Ye?eHl!9e*Fx0!!YYpd`O2Je_D{lM?d3>fY*{1&|AoRa2W1I_#pjf6T22Ob5ABI z2@-~U(^JV>a1Q(RSq>?Itd29Y2c4(QLyK@REgRAv?ro>}M=R$0JSVJ+3Wt!KlBlA{ z&lhEs^FzJQ@1+{PKfrqX+pv#q{;xFW@E5opxBPgtYrBgmq4BYDuQR^PI3LQ>JUL0c z|HyfR>+*Z|OhA|1UZV?VYo3+5k^X2U+vYBp^3R-^UTQs9z1PRhImAUUDoB8W($L%? zL6(IyVZbyZri!3U<GTzr9xoY#jn}=vW>IhGA(2u8oOfPzR~#?8P`Q+&9y{O{Qm|eB z6m-ywwPL~oaPZcxa5qm2=|gWJD8MOP{Sp329-4m(=CG}~I5v@G8O&kQi-?7!Hi^`_ zOG8(S;D0>I@>V-FV9`}+{-9gDDX+IbROyQqE&?*52;rr82XIn|$W7j6q<?bL+N78V z7)0v|P>%13^A=${)X>wDBC?5Ki`zYadx>RslTK8`%MCRen_HVN<2E6?g6a#~XA;CK z6#SYhs7?n?U7OBcST{k$96=<N1(5$1MlTkeJQwsLs1&xF!6#Ci^1i-neEVl(K{Vx< zkULDp@-*!Efug1h)!qBef>R=uzlNQ5)v)##eedXXw3QtAu+-iMoD~*!VjhhYigXkz ze8St~D_v^>t&AcqDxci6y3RFHaeh$Rt!>}oowh?s`0mH+hhodD9=ShZYB@M%nr4*I zeY5Y#yf4%=E}t&T9UQmR4lfkm?lzqZ-F%WMlQ-%lV|VzVXj$2z4<B%%LaGnv>_!Tq zBa`g&7<u5!=+06EpJE1f+6>U@IG!fovjLtgUzpef&u~Z}QvkT`6~{D4zEP(TcLN=l zzYR_T%(SKGMyvX^dnrAVlL2Jq`q~gPufau+iBR3)-ZsL|z+#05a+cO(I^0qiDupMx zL2qFH=XmVEH$)yUBFMHt!vOVS*DW@0)J0dP^z6m!X0=%yVq3*U(2waeF#$DAX2(4p zU(Xe^MTGw2rFCr8)ty>QZRWn&dP6ocZrQMM)Mugh>5+4(1v_nQhQb4vZC{1SXg>@a z$F*F)#9k1$6EQMz!x690UQM4ZcW65O@sM+txy!RsoA(F2X|7uB?|Soqb1YWh_f>U$ zV}_Y!AokdY6nPgF>n-*=u{)aOv=jF1^Dk8jUbdS8O<7udxJhhc3}HVWg#YN<P>YrM zPbzmrI+nhaB>OWI?53Ngm_^-~q0-`CEkjgi#g;<PAX)d;TXrUo^>XtsN**|e9TGsu zKRiYR(cvh8N;V}Ig7;fDQ^qu01qnZ$-x4hp1q~2g23A!1`nJNJ2TD%V1ALtk@KtmU zlE-`Ct*4e?Pm_)M1<=t(uc!!JeF_!$5*hv=I&mtp>`=MOp*;03FREpCzoRLCw4ha~ z9}f2tPz>~Bus$O1)Q%m^m99Ehs#{fgEud4%^lq*}PmiC~O##g*?CskOD3z)7=<~1u zfpa)w-t5hMw7oDLWCO_w9h1BHN2k0lG=gkrdty5MYf6>gVs=6sMS+rncPUKdOB8lq z5uba(yY;*~V#lrQs1NDY_)%U2G6*SBE~m%c<a@zRJoR8y4JwYW@f64GK+9e3Lkxkj zwxtGDVl%8r?QF}M(8xAGUCZgvQcJ5XPdT<<F=fc~maK?EKnCIlwfK`22=58X@DiA+ zE3m*glTA7;D3U@%U3pS)M2GUR`r+yZOrkMSMAs-?a^4`gQte}r>HuG9N3Fs|mf^k5 zU*sQH-MiLB01v(8B%*;gas=jptWf;VG<e4&@56!&C}06CJYkYzxV47kMdNF6ymCC? zps9I9qp|xDhECEohh98zJ#+0hiJT{wUK+yg7Tt61=dWIKP3`QDj)^<PjBWP&^#0(( zT*>RCwI?SkjqF{EUB(8Ald{va2G(yhmW#V5;;c6%CDgU}ufBLy(zJ0inUvn~+kBTz z!@1`rJX>dHu5)yOX?4{63t~U<b$W*mmWJX+!AhKOJ;#Zy!$bu{5^n}q2Geu$P8{=K zQn}#X%{89GWlFM9&N`>%oU(_Hx%vbSZ+CNxeUr2~MEQM1m57zwrYAPe^4qMm=TDm- z`0cpTad{(tAO1Q#&cy5hV^NRVZj?US#n_rjNohRB(?Qym#o;Fm^j5gYb(p*dqy(}# z>^dq%#z@K6m7P{pP>PcugH5r!tfL_D+^h{(;+R1t{V+{`+yK^?X%?KET4Lp}N$qHn zYm_M>MhMvj&v37EzL1ZBHpJ?LPm_`edSSBKJG~fcHMzrHV&YK}cy~5*GQn+R)yp<G z@mizHj&+lb%;FUP=&!*=qB^rTli%P?q&Pd>;|msF@kuhtg?o<H4hA}rlAi-qv`fS( z&-8ksLtSmgw3FM7qSD?>8+^Nhk+Re89q%2gG-O0?d~tG%s5}w7mU6>&--2o8zS8AZ z<*w|&kvPh8%0ZLwo&z7Ys~-HO*eZF>;=|09DXZI;FXdr6iet=CY|p6&`27HGIZqU) z1cj({plgR8OeldZ-~rN&sowdP-GzwmiDZzJ`TkA4tDx`04Oon0(2NP-RK<i;2A#zP zw|UDCgbyd;B5dSTsC{}3^^jk9`yW-D$MTm7x7rq0b6uiugr*KFiX~=5mYvA7Vy<OW z%~>6ajyrGbRFZM~@DB6q=jT7!h7(>oh&spFY4T3%TJ4X{PR@9H4ypQ5pk8+T-9YJf z2k}D`Q}k3eYT1qwh~b^403$wd*pOgM6H-QItoc$W5t|4WfYk#*5)lpRNDDsvi_f5? zBM^J=htSye1YQ&Lv%}6cw0YRlclVoAG}p=;^+Tw<=QL@tmXfAthPZxa_XzH%3;z(B zU+LH3>5<}l>bjss1ym{biSlTIAl7kJlrJVoM1MKP1IvnLN4e@KIJ5d*uu5vCe2PI% zPS*7Zo;PqoNEz%oQ${C#m+wNF>{gP@tr0qE<2&jd@s^Y2b&8~gCf*P>nG(1#<k}|_ zy|=@jQ+g8MXf<YG1mMp~u@-_<f7)jbjiz6~K<%C!^SIDh=iDxBr~3Z#1J|_8W<msR z_^-D5;p(NWW^WB2%$48Pb4-lYaCVA2V*2n=lBMz3vYp#CTyYx>72SA{ftH7J&ervt zo6wJmxpNE>jXdji)8?|(<BC5AWmgJ5q_rLwPpeoydQA3E@Ja$AVogtj+(7ROcb+S5 z$>Hq)S(PwOku}HgT!U{#@W|uOV2KXYrC#!8oK4qj!vle!r^C^y?feI!O6}e6Xcdb( z-=H{(5<vtX%8u%6Gl>2nByS;)_hc&-4zbc^(kv7J06%e*U77bSPtvjE!bC8#XXsi; z$hE2H(-Q&DGrd*sUl?-z!yHA$j;U}h;ddptVjI|+=S=trdU1q1Zk{ut{SZoBT}l8> zU3=m8>#f<_9xrA;YGCMKBye}>t5K5UL$T`-znI|?sejcZb8+}9I#O=0;@o4uB!^g! z(hHsCD_m_#O}a(X#iMhzJw#&<tVPXHcdI$N_qd2Sq-vQs$Y&mX=e6RNX8QQH^njaF z@Pd4t>5ZG*8V$=rk+XuCl^Tu-6}6Y3WY2-<J&RNxnm_}NWLZoS&F(zMyNM@)zaYqK zG~)ft#S9Unaa;<+wW_I#Z}ke$FTL2W6b)j@+edsL*ydZdfx_Iw0FP?)Wk-q5aD|g5 z^F1wqV22(+PgM}>{M+(^>Fdd1B+yF%w0OT~ZbZi@dEtC7)UwfuYfC%F@g9&2E~0fp z_w7J`*Tq4@6{t{9J4_2yul4Nm4V5Nbj$8-A+sI<RR0Q%sjJFF|8oB@Ef_>$Z-}Tmv zOSGG+r!%lS7EIj}8;>%S^r@b`=Et7~8E2D|ssidOE8J<`q=NZ+eE;#%xd>yI7Ne%N zL*_1)hinvI9nk2y>u6gmr&+kyrF=(nq5mFB%Z^W}S#l3rGcyGlDkmBjPGnpz=?MF- z>0Ds7I^u1jXgbhgg$qxJO#T`;`Q+V|BNsagt?%5ix#O{`X#Sn)a@;A7(Zb-lY1Fg} z7vBqAl^6xsL;;A1z3k_d?NDHPJOR~<SRj3&$e`&+D{-_Il{i^5*@E2^M3EvKUO-DM znG3UDcT#p5v0SLx7ImfGvtp(5!^#Q`<hnlnr`g>FcjryAy39j?fFWta;Af{;;hi{U zC$d#g!Z-U9f4&#lg`?2xQYk@`3HJd%POgn8=7HbGMxr;S2WBVs^slDm%<5Nj#M|R4 zYsgOvI=Ho_tjxPrb#4m+Z+M*vwf4kY9D?oGyiS6C4SPM+!W2hzXJcE_B=3k{`i2Ed z@dSHqJ$Wre{o{<oKv8sS0RC9fq}SmIJFJx2>n*qNzpgBeX;3jjcnnV$kLyJTrPRF; z<W`Xt;rIpSF@8k;z5!Y%9eEfcc*zhOh@_tiu^r!E{UUHrMP`+8$En|5w%-Uf-uiXa zI#D#<m^gg+gRF1#wMt6`dE3Vo$`}*T*S6O(0z1Xy+m4j9w?5J?ywF-)U8uQ%ra5(l z%Gv`!*SWy-LiUG2&7x_i!Okl#>-+k{WnO;DHS`jE_bIVD6jSIXgI42fR03GJUGRo7 zl|7-)MlDYAtoSPIh(&kMZ)QD-Y{Ykh>FkHzfa^P!lb(zbuI`CPlwdP9&tn+fN6;Sc zuC&})8WsNM#UKw=<J%#HALQ1z`5oln|KGT<Kx(xJ@h%bGqiWXw2)B3V+Q2Uh`RCaf zs#%jpGwowR;hcqj|A6J-{yx^XuyOq<xZ%W$>~A-pC+=+w-cOHT-w}%O9O=Il&{5=T zeW2LGM*B@NYt(Z1uE8E_rDu%=3WvCcDvN5Y7i;@7gYK0lO}81y38&`mP}-q7+ktT| zOnB;G-d*M>vcul^d4Fo5t84ouKNfxQ;5pB8B|`<ahYAheqodI~E%wbpY&Y~+9mAq# zN<wDNTZogeNaCU3o2?T^>e(M_LZ+~i!HD8p_|1eSdmWfS-7GG=+fXtGLXs(VETppN zL9lC6oyqRbubsrdP@}<9XqF6Hg6j_AqA{U2-$;;SVG0+V7TmQsL>N2Cq%=oq(9=^5 zERMp~$6*1}AlX8JE!^HDH&sdVm`5}e_PrgbB6kNCO$MPymGw;Ha{@S8dFaYA02-#E zs|}h_qcdmv)+p(ddkGSJ@xI2Bka6apfD$B6P{$e2^Pf}&-5UP-jcMPv#1E_-wJe?N z4A?>1p}(x_dq>kdxVzOy>kBMnD__!oy}d_DA@f}1InExk?~^;~&-+7jS8N}juhAT` zaVfPbG`Chez25tw?0vk+*1aaRcYiCf!c*F~VTa7EY>X5w+mF&0-Zxy6j14-W)p~9q zh5tqHJO40qDG^g^z*^x2P2hip)7Tnd>9aP=Ev`?Y;wGfwx~XwTdU0ENVjb#dZzFr1 zcJv-|Ct#0g{9-<TcWu*Kj?CRB&(j<fgBQBd)P<Wy6}FdTz7p9s6h+Xf$~c9w^ywjE z1XP|Jafcb#9`fu6L;2b%jIBR}n$wl-c;@rSE>3Yfb}#=dyEti;nEt3f7!I0op!9~P z9MLN<>55R}2{Jqrwxka^#t^I5L2p(|8xoagf%HypCYW~7PYpchoWTfN=;4GJVfhN_ z=Yxm(RYwL|EcTI2O}&isO2mAxt4QUD#CkRskBz~0j09@B@&dt}JrZpmK}XBN33O~C zot@B@CP%I3=2Qd`wsGt`GX1Jf0;opj!l%!T+3Hm^hXTJN(R63~hv`|r{@x#tNuc^J zf3{b6aHG}lQ=_Wy(VOoJcX%5Zw2zRw3oBDv&jy~bXw5Y35x2_ICbSyDp+~YGC*%Hd zPuH{kRG}A43O`Y#xE0WoAEq=sIB#3Ipyqt@dAWmglO_7A6ayP?afEN0!vJSL;%EYw zAgIR6aSS>!)sJ2>n(lBV;nzGfTrd~FDA9rZ;ChZ+Te;UAFF4JQXN7B^UpB^w;CEMn zGA{e!xS_YvfO>CD0O7BE)ZKZ*c<c~%Eb<B<f%2RPt&=a68f}dKFiE>2cuffiR<seL zjp20#Pe_L4#CSw`@S(QafDwfz<;mW6dVG7n#8&ULr%s6=TE=47&W@kbnVp?W(nuFk z#Nybj#bJ1Ts+^*59=Kn1XC;`2`<4c*Lp#^3`|$qa9fcJ*k4eXSpMny5|MG1JzaQ^& zsC0{f>Zng0DE{M}ik;-k^X4*FEwnS3{(CBvw9mF^A9iqxQ#ewgu25atNfg)YFOMs* zmarv#<90HN-KCyrKZeRL>710Ydan2&L+x%v+xutL{@PV>9`_5~F21P>>?b9Xk~1~c zoh)T$m2vJfpa)?AVl%VY0McI>Fm4b?f6wyGNqrvtdGI6PnVJi-Al81+-V+G-f=~&W z@fv~&5`2NIC}3p4_r}c-=P{-P&)S*_2?C+ynl&t65{(>(*k_46n}p6Kfg9zQ@@bGG z|2Mqs>yZ?}M-+Frfk555|1x?h6TPU6;stI@TS~ik5Iq2c^}p~d{m*{~nPb?em%dy1 zFIxU2_D&fHOx$NH3E{x>momMF_)<=gGx9rSbsR8fHaesP17d(T3a0b(g5^bb-bWRL z3_hl<Ka<FM&ECvB2G)BX`zE(pK*aB11Gt7$*8=@MVFm{>FrOBf8tGw_O$$Brp1rto zn~@MeE|KRrRQ@^7Ro)55>3w@(n37X@rGwVGaz-zpK2`rTN}-c&GUdp+&)dtlhr@a5 zIQP?n`^W*DJ1{||_atM$1Mpp<-TnR)c8%h>;13Uiyb2_){7oE_IR4SHAO$={q|vF! zvS8l$KeYGe;ZXPg|ELy8(v&^hRQ5etlguq4gb+f^Es3!-Z89;<REn%~Bb03_BqZ6h zPK@P7$et0i-I6Uc$#`d$`@H)6ajtWn^F7z^_q(oh&UOCy{xO&3nwj^^`~7;qp4($# zcAlF>HFi{PnAeTz_;(Sjkag%4Ezm3D+=HIc;b|0jH!KxEW(cc0Xv1Q&&>k`qy|Tub z-9ff8oNQO5h8+tx;#sf(781D|iu;3I6&K%fnDl_^KJr$*wslvmPV$vIX21N2Jc$I( z#blat3+sgEIL;80>BrXSF$Z%?z8@!bQlLh~lb!px_P#daZ`0Y6y$oDtAWAuvIU}YJ z9v*I7GZy(1Q+XuZZnTgR{`l3pCM|qyA)wDLTiK(`DT_SgQK7pgma7G3x@zwEZCPhM zu~x@j@@wTW7E+WCWBJ1_7>YaQj`r$!MyY7y!5fbGH@<j@_OzL3pll~f6H5F<H9aqA zKg}I=l~t2mfKTUm0snUq4wbDD2bMGbvp?c8awcPw6WLPto;8=p06Koo{6g^IsnZKK z(EXwwfyyV7;oUydr~r!zW2CJ0^BM}2X<ftK^$1AacW(>7{Zx(cnM>>KSSCPPz+<t( zg-Ew2`}_p_{BUBm4iwXg-VOygofKXpg}B|tZkbdp+8Q_HZg`>2iXF`Gl(4#ZrM>6( z5&v}EiQNbFR^UdI^&nuPjT<8=RbfFJdr#wOD35E8>Fo?SG|m%c%{e;``;24ZG=S)% z-oqNb{AL=x)t*=CA7*BMHqX7a@Y6v9Y-5HxhY%0QC-($0KzzO!ABu*B&FQxSB@5dq z)~mJNt$<oQpx7YoGeO9DfS_+<jY3xaz1EE_W21NBn~j~E&2z`T@Hk&54Zchj#l6rr z{LudfRX%<c`rBcQpnHD#E5f&U{I;it1NWlGgH(;w!XxKjmJOLP3y<Xa44P;;7V68L z8Fj}F{mkCx>0C#wO_*33Bd@QY->TC*Rh&QO#H!`2@!x}s0FSpItd_C8X;rHU%t1Tk zw2@m+GV26wYQ_2v+m(YF(YMqBE+vPiaEhQORu__JC4X@M&0IY-xtQmP7Kccu6FNDR zl9vN}X5XZm0XCNdn8yEs|LwaRoIL!Hq+o!!f|tTjfjTo|O)h-SoCUPi71%mmzE@M< z@IqMX718vyJr}SJHLW8n21p;$Xd{xk7*nYG=IWvNNu9i>RgqRrP!%W?q9Hij9Zm7J zLlS`Y<}jJ?kXT>(*wTYZOhs&Km&1CnN)vE0%Or``pjXy`RlGlHoNefBoW`Ise}7+} zxj9sVZw|RRb>fLb*|)~*Z7Y|55)(nFIZI8zAHb>bG@-sC!6!t$MyEkC62$nB3y3Sk z9hgmUukys`1-4lE-1e-B+2z6`qxG<OGYUG%(*73ZTs%W;M$7n6W8}?9&^7mJj^Yyt zaUD#$ZbO6Vv8oLXm!gN^LnbP|Z|w_d*N3t~%)F;v4<>(z)f}oXNr=lUwY%9hb{EPI zfaM)@5;k!uJW>!7=25cG$L@HVYhnl2_j-bMerRCz+oIItH#~{j1((atM*;X;4j9r# z_rM86(ROo;2b0rOjG1C=u2gWncvtzEz<uMyDb^}6?k6>A9jh^3*JMn59Ml=}r%a(D zFbg!;kE)18T`5B;&F6oeGk+PJXk`zUVGu8%o=+svo*?UBy?(1ln8-RH>l7~7qs2bO zJXQb1`g)~34Vgxi<DI3PHF^?=R_o=UGgP%t<8J@hz0-THP?&$ORfI+01m*xMb{#g6 zlL%Rv7|4YI!8|wiTF0Cz>(vYHY0ONo;K<bQV;&fR%3-E@aoZ+u&7%FM419{l3x3)X zR=9_2k_H142EeUx*#J^U+(QKM;XPJ+M)0A-6e2Ua6RCuPAGo!O2PPfc7{rHG+7QyJ zh>h$*vNAF2-8&puAsw!;xdDdwsjUUyG&JH|Zr^jZ-VcQ%KWtSkg03CURymrcBkn5~ z;lEF*d!FcX+pX-`l#lX}3-3Kr&H7HgGSKa+m$fyw7$@B*E7J^iJ&<kZJoIoW_{UHg zGh4c^Iembx+n;}}rXtp&njEBMw{*j4dN9_f&(4Ea$j?F0o+BEgld*0fikutY`2CSf zasVn&gf7TpuhXR0c#>#_eJ6E#lNL?Jr<l^;V53=RPDUL4<QqzR4sB+cFJ85Q0E&Ws zW0uy%ZJZYooDRL>s-<HjlXhWzhmoZ$*D74FEx7|;ud!MY#X{1p8cmy6Eu70!ICF|C z3)vAgP^hOvPL3|Ym=5<S>#@|IDXug}Cv1SUaEW5+R^ELw>!X$;YRe;FRrw74qO_L# zjV{kJ7o*Jl#lU%2nEZu1gG0ry7RO5iu<MnN%M2p#c!y`F!i^*iP>Jp?$J|PG4*|PU zZBNheeos#mIXlJi1xL$wkIZeYPZ{%jy{gt*xL1+bKLE9bR*T$z)k;Cok5k~kYyAUY zfJl-Ou8#zoY$|yzp&9qA)T|dk0Fw!F-EVmgqXrRbE%i_QD3z_(#hyv}MW=RM?^oE6 z?EnJ1Gi)7BaPx>Ud=GfR&LOCmf!Kd?wY#cw27iyLh{mePWL5dT7(Wuew(|?*+9hj2 zjsN-gp=WQ6o8Ia^Z~L5<bIw*fCZLF`k41p4VK{QHVA^9@+0Yd@0YC?U-1oA}rx)YK zo-D;VF*YK#j-X6kif;CU?$nD~p3Ch?4p%f%FH?Fs<8b8UlZ>oqC#q6OVjBi2_<i*R zp+&f?l`dACH1XCr?=eo5CozH5c*(OJvCAp<4LdgTZBt;zx=MQ0i;kAArT8cQY<V!* znDIt$Z_n|f^`jRYpPqeHC|^+!GxRo4!^J4o#Kk;+O(Cs#ar{-8i}wrP{BpZW$K>27 z6lLT=!c`+_SLw7=Uu<1jXZAkJAIU_cqHH>=Ug`w`wKL22rrWiL@D@?3&rR%MaF${K zy~Gt{C@X$}3`9zyA&|pKGe)H0>l(@2DrpKv<pW|LbJCcaKvJ@dqU#;`ffVH#u_n`t zP@5SSQ0MxtO->rttq-V<8P}fqVFqJ~2s{9BN*HvSYF+q`Nk5tF^0sJ>3D`R#plFU- zE!*l5YZKVtylchNsw6%`ysCRvy3iDTS1la@!!$Cy6Q{Zb8i26&xmAgUIMrIuo>{I5 zyU}gD*=Xv(Z@TT<G5s>Mb(5aJlZZVF`EhOhabvd;jio#($!??sSiX?A7h*pw#p<wN z7%0AX!}eg0PUvqS_F^P^y?hN;h67IyKkVhCRiZ6#%lh7(nOWKtQ~JmKSbzT0h+To& zh00EFFB8M#!t!{E^w#;VfkPjUMcAI{F?4akyg7TlCL`%<(Q(x3Sg~`=(-7B0`YG(Y zj}?v=LVW{P)pF@K_neEQ=47WH3><B7%MSu#2$b|+sccul9(!&e?>xXsg@ej?2lfuv zfp@%-pDi#1)e0@)gX@Fewh^>~_T&85IB9@_LU%03T35~Tu+Y?Vz$to*JudsW)qTo0 zvb})L4<b)dcID}{m)yE~Im0Yxbr|>wW%BFLKVhFMdSmP|x%RjLMD1obG0q742me(o znkCCYjge3=iY%?S`))W{S$Yzy$7<{*iUK$>psj8S#96krVF6n5Jyjxc8&2O0ZfQnn zL>hcN=J2zTfTukr!7Q!WevHy?<Rg5Y_i8v-e}KJqu3=JO3Bfrxpc4A9C<ewTO$@AS z7`w^~ihYHt_#i(^y>E%WTRcsngvs|ltZGHP4zS&w?L=t!SEz(=7McQ70|bMVh&zb2 zVI}ksDH=d?tBF<gp%Ecg2TMloYCp6XZDlyst}xe@+C2qklh*9G&UW3=c9jbaC8fSd zwYln7vtjR+c2E7aBx+n+*O<`Bq-O`BE@@ai?Z45%^(t>1c2AR)c0K3;Jy~*zH<0#* z`FmW*N7SZDEe3w;e8R%aeRR<<D0@0t5nm!N=kvz9wpF&t<qd0C+|j$#T>E{P@v^&T z+ar|A%_rra`OD>yBM>Ymq~LTRCfCddGtf<~gLPI>a9^i-1i+D%z$VZW;!W^E0hnZR zuKJrlM<NBf*khB%N```i*wBW$^g#B$MCsJZw)#$cRT)-srwO;A9Sv%;U{X9l+#$)U zK_8I&)>@Ga9fB+fQzU8J5s1rBZh<FS9Rj2gBD3Y6aZ7(D_23dG_in)(%KQpzCc#NT zK|wC4KJ7%fx!Y}GvCb@4E=s-N!pIBxRIGkoXs1a&QdjZiQrY{#&iqmZm-CDLE(cho z`LBn21Vf$`{RI=nWsY_8wvLt)QkTojwN9C6$T_keE=Ymt)Bod?%k`=M9b!iP|2@R) z5q1l4nKbQ>(}NR|xyHQhaK*vJ@!=jfRxJG+^g7Duz`Z3r>zzmfsUBaK+GLnVsQ=Z2 z_^W+L=bN_9x2B*~t;zX-$*f;$!Gj7EUd^0YO<QhTI_;)Y#;sq(+Op!JyQ<k=213wA zomK{d0c5Sgs>th5i^krn=^0u`&!9(VpK5^oc)K1}odxgm<8MD0%su@sbgIzc6IyjF za*igdZ0>WQz@{c!h~PBUDojI5K{_5#I0G*Xf*XiBRWMh&QpgAS%>?7S$PVAeNBR*i z&u-FFT|1i}<!74-lrx>xXIDz@uNLl@H?~7$H}`94pP#U(%Q7$b%zMHN9ysn`qM^B} zK89+@L$#!i=M`9_MGj@usYPRo#~ckX#iO^qCiHuCDlw8D1gfma1#;B^j9F*yCVAEV zwDQ4mOR-8!?3oEmg}e0n1}8&0cX%(-r!P3_rI<PMv9G1LNcIDSLFo3@QhMV`0ap*J z&<b3mAL466KDa)j5E6b??GN>pDPnEXMaXNrmMlyY83Pq3)}%wX`~kkR^(0y^KBsJM zk%FZ>YIUg&a0(RrqM9ldjJx+Tajew6c-du7RJp#7bMd?;m*VK^)%NS#Y}+rqQFBU( z{Pm)P<|A>=)+cOD2Cfv<glC(g{BQKuSlwjhti*EFFu<YnAe2CJosQ;2v%Cpd*&?$9 ztP&ISo@5|2og@u;{DH6EHG(x`h%1n<&4$MK$WDP_Q?_Px?9rLT%i;yVKSQ0L(ryYh z|CTBQqDhzW)1X;MFj$R<K(|J+5j2g?DmbQ@GjO)i&)IZnt*cOnRWdt2SlL#tdd_bB zn63Q76zzLDixlnPQz_#r7KZt#evhQ<&UQCRmZRYgPPrFHeFvV+w$x07jXWupuSuDg z>!?@Hd#Km9Yb{#tM{o8C`z#+FmF_5m=|;}&Nh=8-?6wKqcAngLyD+NZ)AS-qc&%L) zI<i2wD+rH_(p#->C$fG-?p>5Q+8t{-W>@X=JPqZ0p0aWQgif3UFj~`~D;Nl2<)LsI z{NbGN*sJ&N-vK@bQUt<(n3Jh(YPqg9JJ@iaUSKsjsqn0LTS@vQm=1R6<Cm%DBfhFn zmc|_?bY>_{t1gtc(XDSdZ7U%lIxYfIQ_gv--TbHiK0kUxXHhSGLyw(dW}79Qces=# ze_SQqlbK=W{x&UT_bu%+hf?-m3xC3blh?KZj9Xzk%^==tHw&na@4(<rJiY}NhG1l0 zvxk-%FKwdlTz@^=U5tq)bG$e)5jU1(qi_EGuJiua<_s_4*{2)h9u6AnS^1Ut69H3e zS5bcTvyDM5^;IvU8=Y41excZ5P}l15({RV3x30?A2!1C13La>f<aQ1?=JnvE1Us>M zP#mM_*5Gk}W4@{D0Ze0OV>BUf`@o@iNyD_mDmy)QjteyTsiezrMT~3*T#Sd}YU2jy z44@EJA6k}9B9o+m6iOfiww-6?e>(29!2?T`3QRyGqBH!s&_`XpRo)jYw_;IVM>w9W zeY5x->wLK@#nCc-KFe0a<hYK{aG^U97)FfVXqyS<hV@@7FCU<IBtH!P_^U`%?Vz3h zrrY(DON=^{>O_X6E8Tl#glo<d9kFzx|MrOkV@ZQJoFY4~xbWyFD`~7YOM^Tp*?q$n zW%3Zs@Ym_I5YLxiu1p>5(!N^s<ndc&!yhNKDp;V4#0PrnA%Mjz-}nJPy+8wu67TLj zkt7p?hoB9D6=SWRV@vcJU9^>HyhGR9GwSRj^*%aX;>s=i+fnB}I&6mN1k06`muq>R z<AgcH_7_z4-TkFks<#@cYfG)o(h)4VFI?y9N8eZo57ECuHL1Ei`yIO(%vHtp=c2ox zwVzz1)*?QL!P2~CYr{NvK{a5a#F=6|p%i>npIh+A?EQp^it-*WZU|D$*|j``0qEr+ zkNK(>nlB&Q887x^nZI^_^y^aA%c{1F5oG0G3&+Rv%G_N_9DI&DG^%G$p}nsbJosOS z8NAUDtS10okt_e*B>X$bK)Yf5_dNC(Y2JSrKv1UTSa?Cz$s?p1iom#z=E_rOq347D zcRyv}f#*)lZy7b#H0KwZA7`^+HKsCi6lfA83hpEC<Euq@2_!V=!A7rwHbhzBGh}-f zB+K28|2vlR3sm<xxX;9`XNy@Eje!2uLHtx1?z+Zfpktr{kY6gljK25t>OtO}g&R_~ zfYvK;eeL>?^Iet`00T<nf!C%5p$9JpSQ2&0bz9zKJZ0DDR7Y-a{6?E$aorfO%%C3p z*5KPWUZ-?^(Kq$&v)9@K(TZ+oLI@k$?SL-^`kcggczqU;JTNS?Og$h_;GKa^Fv5rB zz$P)v3hGX#5tEU7csAvWRWHLEtz;&@*UfMh22Z$un$I3^AB&1K^B7%Q{j4|mBvsKB zJ{|PBzupp|dU-yfE3l$eMA+O37EpjW|Ek>Xe5r+}R!=d?GoZhMP@9)>`JZ-s5ihB7 zuZ4^hopW6ys50|DZ}LQu_RVWg!t4qzjNe*t&*an<nR(@hnA=J4V#h2GvWz%yS$Lk* zFfE`73?{*8LE>icvVuKWOBP{Og<d6rVUnANJ6mQ|kO^HDAszn;C0j|cVGm65(0h3{ ztfA6dQK>Mm3BAa=wy?$ErqTM{mLuD6ar9dO(yVA;w<U!=$ZsY6iQgd@0)hkzvPn24 z!Qa%n(v=aWo=z8{*1LQ5$_yP>R0=Wasr<Kz#ouDq?*DA_>u_6I|BP|%-g5q$hl$Xa z7c08C_lj$)CAfw=>-qVWSRJliC|=%VEtyo0PDF&QiGK?agXnA{&SQ3EVLg+yAEVKE z<twvePMq}*5!Q;J4;s-I5`Nxw=X?hq<jy<Jj=eoHuI4v=^~(DPb7z7eAMnQg5S|8< z0r%kav3C3-d^nnd_kgxHw-UEm8iGmgITk4iR9{5T$y>>UeP1v|j$>FQ*4iKM*$F9^ zSo5nKVq`h=a?QW3v7uQ{@g+;FnJo4oOE*BK#8Ik9)P8rx>u{SKaioHmeo8Pr(jH9r zn$z+gwpVyd2GmyW{#0UjY(_Gv)I%)gkwv}62}WDOiTVJno$L<#&DxX#i}zvm*UF^> z-QgEX9=x!2%deIWH^<Jt;YmWrNGv~KFrx^Ri}rex0Vklt5U0NE{>|n?fV8}E?ngu& zm@4_nMWKwVi+{n&N}i0Js(o_ojeA{Sr=c*hp{nO!p`>;=#C^+?f;71PSUX53oYhM+ zB`{`B(0W(Wxe^AeCz+9A=f|&1JO_nQBJSieWv!DS@eK(RC^+v*S9+WL{Ih9Xd4P4$ z9Sl14-Q$ieR20_;l33%s9U(<%nV@B=x&~o1nKrJ4Zzu|11l!(6BOU-7(J6=!TYeFh z`n<rJcfNTc;mymWsPWN}NVP2G?yr{%tYRTDxGw)dcYwR_a0)5(qzIT_n6PxZkusK& zbk_u=VjcWG@qpz?I$|$2j74~WsC&s8%t<CS@WjiPaPaZUx<RCSwX+jhzVLpTTER-0 z^cRnoi{4fSq69%_-wmE5=R<dV-f*nKrgc_UXd=*1f-;kLLeb{a)+5`y=F)C%?2@ zN;Lgy<Ua1LRlK6z%y|-Qqn~QtpC2-)Ep@El{58elN0SXK=^Tn&h)s$*>{2`p(V0{* zpHM_xH2~!c3tigtH#Yb=kd$3u*D@Bm_%<HwulH#)a2b@9?9_80Fs$(nr^41qz)CZm zdTMh846?&`>AOZ!c#D|M>?xV1hVe+fxl!hZ2ABd4b5B`wodgtOwfCwd+YJ$_6vPS1 zqm`!N;MiL&tdNIRaHzH$0kg8igdis)uzF1DULInkZgoE<^YnzJ(QCgFPC-zl62D~p z*s#P!og^D@dQ%`_kq`e0eFmj=00Br|<*<4(xv+5DQ8xSoNeS{`z#d|cv-OAl*jKU` zk=hDt2QbMF&`l}%=VL1DljLb@;jEJSVGh=m^2MU<VdG?UVK?9`ZfBW&zzgF(`G&;f zjB%fGH+vA2+p8OhWOzL?nTMqDk(|e*iItxstl@_SKlQG_6Iht&KN83>Cw}<~1LaMN z$Xo)|XZC6)4vH{|P`}wD*MB}|y!Ika=isT_oEG=2vyG~qfs1x|*rzc+mxH0ip>n59 z<)NgkGgHKOKVGQmR;YC{mdZc#)M29W1($PgEsypFRWQQ)FIH#8m|TMhA$#;W72Y%N zJ<3k$BhBy~$DeAfg6k%nely=-HJ!DZw#!A969)b1B69OS@=2}%a46u+L2Y!ntY8u^ zvVo>#`semC;fWFL(ztHrZD8+215KcG){b7{E*^}%XQ4wr{yxK%l4lFqMwgj&SGt}V z+3-K*@8QDMdz=0;EIn17>ChD&N~$H++VdnfPxGHznL<^}9I$F40PZb;v9F%3&|TV# zkg<dVmpuHA4fJ;G1@`HH{mP<aQAou_ffS!;9L{|f@iWwTETX|8D=G0-xQAx8T85S? z$YvgR^NzB50l)4w4+z|mBGC7#qLz5}9wvj3ioZgLqsm7BtfQ+JJHD)<Ahv7$TZX}d z1vZ*opyD^F+Wdaf7?5-@6{l;PW3$gN8WT3{71hO*?D@W>+Cv4pjRxdEZOwjmD%SFv z6u*wf?;l@qg*`s^_0d&ZM2My3<Pi_@rls9Bn-e#SHj>_dj9{Nb&weFO7FtDJ>%Z7O zTHtK~`NTR#Y5Gm~^&XIwmM6ZtZKf9+9D<o#r<lxEwhj2s9!~o@UHVAPbn^hQ$wHRG ztkshKy-y1#zGs_Mt%`u9OMK6Y?zpSLNnjfz>fzlvqgH#D3sqRX4{uc|U4%_Ddlgm; zEbW;m6bg(SFX(-oD&M?zCS@zE=bj_`9e>7iclo{QJuj?phg^~p5<8JG^jR^(K_kPV zDj#Qr8xROV^K1`J2-L<HME1x>C*p&FW2icm#p3jmo8yyFFP1IVl6az(yp!XRmLZX4 z7~FTJ+o*@_RFzshIHE;xbDGATe-eE~rYt)=KceP;vWBO1t->=e6|c$AZG`h^)mih( zn4x3&n&)2A5B50TbW3(w*oX96de<ErvRZ!r1M{`ctky);{jPJRgMZVQ2R+M_Lu6&; z8pgaVlR4N?&?XzB6S`Ym_X`G;<-zri;+>g%v%sRp@vLbuDIN?OsX$O}g}-=(+4y}< zn4|jobp@B$b5!TaO0?eBj3dGEPIpvO+oQp2FJYL>HjsW1A<f1PY8qZ%-^vn+6#{)S z4O-P+p7ppumIsIY7~(-^rvR!C-aVCLbz}@D4yo8wPi)xSsknm#T%A~=`jBQ<g}iR~ zRt}T6_X}pMqfncbTAIZ3r1@1zV({LdDqF*8;cdsrJvd3?`Rc-cg98|^P2W+2l~t_u zm36)D;v6rt>}AW{SLzFojx;XVuZ)1}+%|%^0!wYzf#&HXK!m45ag|s*Hi|vfP1ucf zV^cry{XP(7EIni&iU6jKcdrHh5h7$zUhviLq_Vj2jF^&R_vvPnvL{XweNqMI7bnxg z<r~6xvE0dhO#xQZjH45QZXPXe8IKfCm1$omoaj1bdhNma{xpe!U!G_9SL90l?K4Ji zP3!@Z0G_tTZHG#vT};%}gzeljJ#vg8SQ_F-9_+J@;Nui$0Ki;}Xx-Ru?w`0hgwjUU zOa&9ZQpBod5LXX!PePKMNXVZx6)=Sj<Qn71%7+2a1I?CX5W82^ffb!FF*)5z1}bs^ z+Je1~pCaD$K51z1&2GDMl4nYJV>o^G9Y2^2^vw-l@of0dBS@&&wV5@+9@*zSmrsY- zf^=FqQJiPXFRZMqN(k5oWxk;-2dJ=Q<0_+Er!&5XT55bnQO@o!S?ykB%q*HRoTh`l zviOZmJWq1$16~Gq^n&11yC_8E+6z7smHa+T!o@9v*(W%+*`$Yc;;xL+`*k@@nWsl^ z8W=P~<esI&b4=l~OYXlyZaL25_?kl#y2WirbUOB)vyshy_bl#1@cFhS1M$ZnHGK26 z&HH*~{S3R#0qlmb`AAvvOQ0ic^;6APyQWB9j&Q0VLfkSS+&$6@(=2!4(`#BjYMgAZ zSK^UtRO$hZAN~B=x<|a!U9>;*YB6wvv0nwiE#ZS~FcD&MkPr}-(t=2=QU);*r)L(2 z4<EcqUt5`vW*`V8nAHJ{hd;YCegl>Q7a;`?C+D7T%#DA+5Nq4f>D^cAADTP=HKzpA zz=-vi;dLlh-~AmE$3cT}Errd)ecGs^kU$br9}?fLh#4ct*gvc%NMjFA;zLN3iVrFt zpT54I;wf>Ru(ItCimv0lopUl^(uU4=>9whG_U$>5c*QP5X<~WqB>z5mgEhKAM^*eP zJA{+YT1c@H3LnKZ(nD6IOR#^AgCi-NW?y-Me(eioUYux+WX>XdYS<~h-ES+)Y|2nE zJ(+i3^a*!bqf02{@XOh=$dzuc0^k;3VT*L*9~)v%0dwVAra`YlmFOVbo9WuKiYMm| zr?q+n()3KZXIO}Ymo-4GzfN3)Jf|=kVdCtjr*~^me)wkCF#x&xxcZe41l#lJqfuB^ z$?8!kx%tKou8W*pRfsbv6FY|D)r(5yje}i0OEsqoEQZCL;#*l8N6pWSCe&29Og?b^ zg1pc*Se|*+Q@h_hX1q+hw$jIS`jjNOr_!8BoXYyYS91ZXC%{(+Xu`Q7f~njyz#8Yb z2#jw6ecsZQ+lFM-9Hp*Fk-Fj+sXwx*a(Ie`6`(FHJMlcLD)6*)Aao3nE)PJZO*w0Y zz%SK9(nu}TfB95e>TflebOJP;&^rd<JHb;2vH!11ypSMOOfOH;jB^COfezTdO8Au! z<Nd7!R}DVyEdbY=dXHb!<h4i()<J<<l?mv)PXT=C)1Uum?9YGx_6Dx+kFB;tH%ki8 z4B1rPVVbkloo5rg5Qe`(=hlTES-uyS<Y!k7V9Wdc7HCH`4pybqlOXMAo@9&tMDQn& zQ-o6=2#&LNibS+4L)Hu=<S#hvV<z<j@veQ}OgI?Cw@MrCK70SZdrQ2IMb_(4f!>8N z0Mg0eQH&qS*W(i$JrQIXAd`3Zmo4@Hj3=e>0e6T3W{?1mR0RZTy9a!>^NHty&VuAR zr)K1q*sDK_^+<_U8a!Rr_d@1Z`0?fg>FWuaHcFxRa6)R#yB(cG)>#Ck2{=%>f1KX6 zN$*|Zv%bp&6~u=?ja)~}-PIrsh2-8Q>!D6NoeZD1e&XypYH=p-ZIr$4M~#Mkr9*nS z+g9|K?MIUt!$2kN{GVGOk-Evenitj)aNIJQ>n7;35FhKJ9m@=+8xR5xA03ESC`wO% z@<3s)p}%ld3W@g8TW%eob#n*P@C!^%BR{{mcWNc3i}Wg7wqV|2W*IE64_E#b(#(C6 zXM|thEr6<q?E8TNpnW%71gp%i`Hl2F&YtaQi=+?0_Twf^y(J)Y2hNx+sN|%&(k^;= zp+o>a8^WI-U)%r1@&25G*_kKL)I*tG*Q>bdz+Air2~e1~a9@xL?Z*Xw&%vM#)+Cv> zw&DtA_;GVb`DOTE@qm+7u>GiLM7<KV+gXRcQMxL^njF2UdX36Eao7LNAw#o*S9f20 ze*DLl868xA=ote>&T13)5H=ht;hLi0+uId!LtwsCr?;7Ca^hd1w1AyZVnuikUfN0q zzCBd_O7ct0v5A6Ug8N_2mbHC~PTLLlUZxPZ(MYZQE6I`;P|QA)qxBkllDsl<j+M&Q zSNNiMjq6rK%>iX)<LlDnWeil$x%t+McP@U(9Nk*xQB{W9Wq{s^Cny0ZW@PvS{N8pr zRv8MX_lzr$sW6-j1l+J;t>Q0CtIFoWeJeueBw5*csKAo?$?Dh0*)MW>V?#~txp&N* zbXsD>rM!m=*%0XL{1$`C1b~(YXYkX*f)VOg2eNK3b{3=Y8|mjl5{MoKZ8!wi9=|M% z8#7sp+RQijSLnzVbz=*;Wotx-GWh}`KZYR4I{Gq6XAwsHt)KaSeN+|T)NxZ_@PzIN zDKtod1N`YGt0wlwqFz-OB89rIRRL3@esj+>l+bniN8hzGA5zb?0G{QA`yN`Id)6}^ ze5W9vxuv{X-Q8r>XZ8zj>xL`@W8V&MAYav-cMN0CNgQq0cz?y#^?_g~RL1Uj6>P2( zLE4QKtITT{7P4*495bmE;Yo>*<+&k%M?w4@=pbSQWJ)Q#&lSRr3HFbmtyQ(^r>qRI zCt2NjJMOtGPKEx73_a2m^7PNcY^Ih0H&KB<)ikEK%i{KY#{_EQpR9K71ui8CN;06t za-L5<fs5cfH1V|O7M)F2p1p}z@t4-(Cr8svGi8AQL4?AIfxy1M3x|%JgqgR=cB3K9 zl66=$zC&GLh<fdcl)+5^!vN)iLl;>-mOAX-i`%L{A6MRp8@lm+Erl@fOyH3Brs!B~ zpxSy3Z<=!#rvYH6^=qKQeB=vTbpQ>79$BIkg04A+r@e%;&L!uyAot*gk#z=L&W4x9 zN#TT~&4zd&xMK<Vm7^MODBLO1Pxo)>>gqjKob)%sdIjqTPRW7h84dwbpDTH_aF81U zRMk%QHBKR>H`f<fgLlEGedhoq$&Ll$d>c%EF(x7YCwdo#%P?DY=KArl#7=pGBwbzz z?9op+u*Eg#f+bnWd|GR@e?{pIFAHN~@LkO@vnKCCy(0V5KAZozsk`;a?9|)s#fH)< zA9FJbGEm{1n70e1IsZhJ$A)2+ZoY7F)H8qYUpGe#FIvIsfWrf1+=Gj_E)-W(&}U@@ z{Z3*x__;Zp2TiPf&lqdcP#$AprHb_r_D@2$$MRNT^{eo9l~#7|eIPU%fj!%>`s$*s z-c-&VnZwuIvd-{1KyECd0^P(3Kb`aEf5oh-qB{OpG^{P<gTPECg|s|QGA|T4@UPH& zdqKPXkC53XKr{6htK+}aOqEWpf(Ya}sWFYoR=29qIfaqRYgeNp@#m{Yehyr-Ich{* zGU}B=XjlKrzg9XLKRVHJh#GVy2@2|gvpj%5Ir_fA%o#QuY(YnD->SyTvIOaD;)g;_ zw*1*`RyLT}&aH|0rJ+B}ZFa4vZF_S94QUOpym<R0nf{Yh=h~yw_t@_Q;sL%IznaL7 zptdL;@mW&d2}$1#J{*)<^48N&1$F!_ih1<bZ@HLKm`^t@;#<6=EFw<K9rsrzFMUhI z7OREkhC$4W2U!T7WTRk@AM+S>$&1Mes)Zb{49_0qzYrY882c^=3iqDC6VN8Fh{~j^ zN`7BiQ#=VHPG^QXw+8BEg4eEh_*s7&^Bd#_&Yf+iA4!joi82|~rRpZv_H=BSf>1d0 zc-|+ap<^(n0v8F7@ctUYleXz<c8$Y{35MJCrwX@DN|ND?3o9yBOFvLCrnL+`2jf8% zP8Lg=zmD1xNr=#|+GrVx+!?Rqq~n*|k{~ACD$)Nf^}7ra>>0G8oCq;|haQ0C^~@sb zN#VoOZXwqH3OT<V;h|!Z=K6Hw=H3&5EWp1)zy`~y3to#%z}j$?7KyvCMyxf!M0p}O z3HA2i4)KcwGEj%{nh0+%G(krs2B_E9fEzK6UbyTrsc%ex&03@6rS0D58mgxEHce%M z;5RJp@7~oDby2OqDdbJ%OUP?&PziIEs9bHVbkXRMn%c)TCFzaWww&~N{I2$0xB)oJ z4Gw9+(*&wKZ+^BhkpiQU?g8~y7$`&ldy4Cf>zmVSWZh0=TQHf!U6ogXF%#<}yiS%Z zrdh%k!+3EfQMo-@3M->@DzKLS10jMF=1K4icnWZG)jhZfh$#CYyi7^#n_e3FAsoij z0zxHXZ^BL;C<|kfapI#9Z$Or=FI-O|UjOIU*d5WYCnDuG9Chf17ZMq4>Fd%*iY(r) z+Ht1!^k){oa3f512fp@rv1@*hV_yq$;c^ZQ<XAWJ9!a_VA^bmhz=sGt37GH527uW0 zt^@^@E>VK{-?O*~XuG)kQ2*BC*JbB|y02p5{JLYc)BTQCGJh1WSY8YNnOlbirS{I> zt(ogQyGebXGsEC>4kx0Hoyw~jzQbG-=4k`gE%NcE<d9oQE)wK~`yz}hqe}GTP1jcU z#`}MW8^jcXm+!m-?GTe>5jhX}r1ov6_v^}|$yGCWCMm`8<mHZnO;q~<{&SuK^R9}s z>oGfZP2He-tah(%PdGB35p!$Ny~JBoZG|Z}cg%(35HM&rmxC+qW0+s9bgb9*=(goL z9QSn#n~JU<!{2SUuN*<@E)iKy!!xqv4`dcl!vstMdmkTrV$Em4CuD8HN-BGN`!3nl z=Eg@>vKU$SRz>j+^vhv$ic7cz?DLvH3`W-*Sww*RV?~V}qo#10Btg35nhd_%+Ld$G zb866auHUQ;%24cqbL+R3Ar;l_C+J<%w<-tapYSgc=hd&fCK$4zV3+WIXWVuB&&J+; zWZD8x#=ZM=WGRTN$6C)W=g$(9aN~kqJ_&LvuAmcqI<ke$DXU&WCD4a?G6Cuot9|$m z!?h{m9GpVsl#R+MAH@=(z~wIs#9ydhrSKQmcX>|jpW-(mMs+*>IZkv0@Z}jc-`A?{ zWxI>(Bhbb5ar)sVQ2fu~bW7)D9k_Pjcvd>;g1XPdX?wCC%i)*Rr=}UfyO30K#N}Z~ z#~`UUi(x=n=g2~cP^Lak5!XwUx6)o730YH>p8`1Ca6Or>cXHDz-%_OG7vlfGRv%;N zzSUlOQ@SXMYtTNvD_EmalRSByuV7X9`c<w%=#}^Hb9G{4F~zxezNR0Em%C)6TI;O) zQ1`mFj*-!RA$JEM?>LZ*cjya_W6C@PvaUR715WqUi-RGb1iK&upd(gbiBS3bMTiZ) zh$lYDbIJY(_Lo88<eA69SNbl^J|gfA+!ToZ!bX0_>GMpXWUhPIyXFDYBJM@pClVZi za?&Ytx7L<nAwj9Rl@q^m$$LkHX^dXo9wE=inx2YL=B0b8R^E-h{#U4#2_GPCeTy2S z=Q}mlcbyXBmb>GV64$4?;Gns*V#XTwQ<h@Ou3EEl&a*r8%UD$U!egL=+kM>E^s^WI z5oyoi$yOV~W5H7DPiJNw)xYlCs{^MDGeLLC-U&S&ClDRr2Jz0NF*mc@&7f&I5OW8! zJF)gI=BE6UO=?|XeKFL&)bGz~#ZMRR9-qftKKA&rklCt1C-FX2*lK6!j!x1J0gR^y z>=mL8lmqC3D=6?qd6u7~FYP?Ni$GU8GW11w?+iZr$3INzCC~Xu2KLtX*JF=_=Z4(5 zGmQernDZF4uo#%q_;Z?*xY?v%hEQgSU=YFb8crHmfCiYZE1q1%Z=0$v&9F{tJXI1T zRfxyge;rrTy#l1|B)jn9JVY?A`lv)R=EV7XPVn!P3r$zQ_6kIG5XsjOzoj{ZxAWWD z&GO<>oyY-_Pzh~q+UA3q>96DO8prULNeOfNri#WW$AZ%JvCel?M|ibPhfyHXBSWjy z--bN(Ce`(vNBwlaj)e9`kamoh<op>rGfn?9cZvB?@XKGIv0z4CgH&C4k_PmK=eb|g zxsurJZ2P2hMFwd&1LzoxfIMLN_vf0k)B`wkXNJ=Cg)bMLDV)=Hs@8Qf#ci>s9$D(q z2DQb-It22cEVW+x-fN%$EAz+`r$#HsHP0Oy``Hru;-RqTj<9`F`yMoYd#gS((NbHl znw3!l?10^c{tvbyJJ^|mJ*f;Oa82r{JPG7QwpCLRbknwhUxo<P)Q`Fw>rpfEv*peP z?6V&yV{GLx@!YsZnr2C%V2QHZGymX~_i3tozn9|(@u6>zBw^IjFdC`q@uB{glR}Hn zu?M6cpUE(_6TTp^`%qlc7MIZrM*Ik6p1wc?NXrrEgdT#h1A=mymC#K95J`sFqq&8? zZ0afu)KPLMv+PvF^g`~*uSzrO!9$aWc+P-Bv=g-FA7H-Uu{^*%+09bEG3tOutkkm2 zK$|ar(&)@rSF0Y_bRz@y=tcc?!#U|oKL%jW<SxVUT0_k2kpj`c*$YEgvHf0QOWKnt zD*bz-rS6#eokyWlX)4o>hHkZ~W3^W|n;ZDgNVb@Q1)hYL(*9?H{rHYkcv=y1&(%O( zjwz+IXBbS@>+F&U@uJEvPyo3OXAgcYjc7^ekCL0el-zvDY}fiKxSQ9(IyyRrKGF=` z1;2T+tjdBScPHLLr%>N%kk=#zU)Qv}C%OR{JyJ@c_>Zd>%<WAnSU2fJQ)`8_=*_@x zIZ*>#mM?l#@9vT}4<>^(OC@x8<xjoI<Tgb@M6VH~3YlvT_MG6K5-K3Q?*(UP6neJv zf|%ZB$1o+DzN_f=y&qHF^7tN@oFCb6;c2DmJ}a9CXIEKO0x>O-Sr#xA)$#1zY)9b6 zw|1=sh^&yy?qKn4gqUxH1HqC0lTYpj8jKhf`s}(~kfk{=dL3-Z)i2x6f@8&kFO<_m zh1}t+vnfB<@$?#eSccKq#~%p#gF743U-7jhsQe>hC!SUv(>Hzn^=}(^#a+uc#i^?f z?AKi|e6>|N4OSO4wtC>|hA;Z@jpDku^NXT_>OX1}2U{$(Vus*FArt4lhd{&QT>?+) zX1DWKhzpcQSi>LOTyE;;am@r|Wd#<0@GsTxX#+s_8QyE~c;riZdi~c?Oz%fR-$^}? zCdhWgtm|NJ@f&{EI`BjJ@e{XBI&m^xr~eh|+<J<KwrVpS_-U51thvjh#eE(-{Qo|s z)E@XxVOYM~eFAIo{A<37^k$P8q$CBQ6$}`jrOq);h$bDJ29hnp$RQg%Y*ql3L{%4Z zJIMIH;6k1;V2`czf2VfF#-JhOzt#A7Lo8G_tS0+o{;r)&6E+sX@&M=y`|d_FdE^Ii z8^+Y?{VMC5YOlu-L8SX|S$t@l-q%NSS}EYxSBC5sIxRe#9rFUh*Y;C7o4#EB{B~gL z0BcMj3fc(2(c+cX45FBz2fuv-eUMUfvXLNA$JiO9A`Z~9_RSZ<ACu(1PSyYY-R$zS zUC)&g)t+YD+!6)!si5%4prA-}NN@;Ryt1I+jo}Q1LYe7#H#6hs*ZJa5W16b%>2uPa zr(O$f3y8d`)#bPpFC)eK=L?bEPcr&cL^hT#z@9C)8gbNNf5oRmZY5BKU{4e~znrJY zf6%JM+Cknd8mwlotqyblfRN<eR<FBO<D?y+QI!mbhk4AJeRLl_&lX8|r6BZ-zW3nW zePn?spNoTjs#3u&N-LQ}?u#_Y5vT{q@WDv%GBti~TSljrsKjR!X+Iu?Jb?9RX}S&< zi)r?J_@Q86Z?H9@_Lg_1pT);zYM!j7#;8mJQvuOdJ5lr`csz38Q3_@G#4uprHFGoU z|7in}HkM;G^+zV>{_iNrMf%^fu?P9_I8qMlL;JNO<Eo5nT%i~np1|9^s5R5(`iMi{ z{`&kc*UxZKd?4X-1x@>dl_>6SQZ1s>Cvw8;1StW(10eHk+^)D0Oqm{^T8)IeCpwl! z2Xy}aC*z}odi(2_t&8IY_JhZp`ZgjPH)}sVJnVd9?7_oW)eP=m6P~u`ZO@-R_d02N zd3lO<`Ta-dgoO4zEp@ZC2AlP{|5#`I{)rsMub46L;;zhLNE80(Pxih9vn1W$o(%Qh zi|*3j*N2Oma8Ft-_rzSQZ*LwMrG4?6{@ab$6mXPKw;B|O{PYNWoehFFohyRHvBlR? z+K&l(k+sM!1EnR&wuz}O<bJFIOOYx;S+Y*lr$dBY3Cd03w?7}u*0WYsv)22|@LTXx zkVLMyPM*wZXuTMT^QT0&5wG^{3c4@&L@rmJn)Y?{ZZw@*jG{IzjC(&_MzHJ&)BZx} zM@Udif%fP0*K_{ausC?b=IZf4dfgQ{uLL@kfK$c9b#gj{e#x9XuzzoAv4qaYdL*QV zUR9&hsH_sO9Nr=t%o1K64<2hb<LUGNB8BQ5_bcz3@`i82o~ST+XU^oFKWi_3Ts3#= z@pTtaV@ry=E9cknzPRQR_cC%=Y^nL>L3rPpt*c4UcZ1I$-fhs2xPYuuSe&0G(0yrW zXv+@bh_HL()YFZA^hQMkirM=_U+4GYs<+wkDmOast;jpDTm@n}uc~s?Ks#BZ#|1IZ zHo;ShK=0E6Ak~{%O&DYufTzeQoe@m=E)x#SXUyG9cmrR&nR^lOv^Vq=E)!Wv0M<B} zFftm_ZQUshIe;7Z@9~~%7X%addb@|$<JNb=s{@LLuJ?RTSaYu;BO%?&vE5X<JviQ} z03!^k-fY#4(IZrNX3TXZ_+|WuzFtcF3R5Q~?_Z${$10G^uz!VwK(p$IJ!fACFd&o_ zh`<DAw)&HP;b7AVf<23jYllJKD0CRoXzbwLNmC<sl4@O7McKVxwIXemQ;7(a!A{DT z=6`^Id-b``iW)~6Bk{Cnpr+GEV#9l2kTz*mO84Tf7SKw$vUVR?GyEZ3>mQH5-YqMe zK!NNX0o4slZ{96C9QHDQYVII;kge}8{k=R3{se$2M{`XR_I;Ir3M_CnXi#^a-P+gg z29dc@f*zt!l?#E5=M+F{G=ju0#w1)g48Fhh#2xmot#1t-;U&D2qv)$9ZNAGNJAd7# zV5^$4ZhsFJnfM7P6)Lb5pRyGq^D?!Lu<G$zNP3?C3IUi56fuT<Lhva=U1wGUwU>gA zJMzHpSD|_9amCm~O1Y@WDnJ|7*aP_jqv>S$^bF@JB+WWT?K=uC$(|g#{rz_M7k?E^ zA78<G=a$kN%EQrZ&n_1_sAeUi4nEDmZhi4=ei8R;2+57`7JP|zr>E7k@tXD<;EP2f z4*jIgBzME5szt!t?-P7M*4+#UyoHwjg5BTDe}<Ew7K#NB6N&ZTD`;m7j6JuX1yY?c zEob@~<_;$B`KhNv8|Fzf8PGAR1NQ)~t5B&;E=^$pD&t1-eBa$W%AcPo%(}^{>TEVG z?{At`0*(9COA^xS>W5C^s3QT+xI-V8{k6*<BG1?wbgw&o4=J-hGVpr3*`+SWr;pU( zi?|Q12Z43>24MF0tMOluXs-|rIfN(oYM>BW0#A!DZP4K?`RAaN8FIJaAn_nMTjVm4 zm1jIsv0qIi+@KU={~wp{ogY~}C6QIj#2QyJH8nt|e%n+JzFui984+$EXEJg*+ixCp zpR^cosV|h-wocvRnO5x#?K%hLF0rlaws)egn<stWr~vx=#q+;_-^#n6niXzEdyw=b zWQMuxc2E}tvszr7;dCOIxE*UTs`A)IIfsVajlp+X+`3lg{;mX|u(_D*LvBZ3Tr*8G nm!2K5y%h1E5zYSujr+f!_dlk(|Mxli?{oD3)f&M6JN>@_#*?Zq literal 38551 zcmeEu1zc5Km-o4Jilmf;G)Q+|5GfT2=@1c+?#@dnN{57kgaXo_bR&WwAe{ozT>{c@ z@BI$=gg!I#zBBX9yzk8SdHfw*cb|QBueI0uueA?y6gdrCydke74?rLg;2!u7K+fXn zD9Oqisj92UE8UU<9{>QJ;w?)%dz6a+U~A{%q^@v{PDfXd4zn9T0k8oKfENIaO`IK+ z@5pNa;H4xdOXmz`I-~!*_J;wmBmfxTR8*y-`!oIDMG=}fIJp1-L>0`<Z))aj0@4pb z+Q!|*;VgX%q)F~upV262XS5U8K#(Rsqs_n5SI_f&r=`zmQ(HSzFwa?=9ZYRa&*&DA ze&Xh82GXdvKswmX(###C2SJ+A#?{smqz^%w#MaE%8354m&eC1XOe{c}2c+?wG}L85 zS^@yD@XUXp@Bcu%n0bKh1OQok2TvzUa|;(bRueWlUQtmIIz=;g8#5OdPE}(QYhx!< zI$3*L2V*-=0QlbLv$p`Ev$3TE2U&<$R7i-Ej|&|BKd!&|@Q+-7n>d@>?>WAyozEGB z?9-oVf4=)?n*A#P5ZMFACi>5``_BQO@(}=#kNug(m<|904*{T}=ZF4~ovjxO7Z(RH zZf<vXcP>ja6RxuX{p0$(8UD!mZxcV%$8}cUA7e*%-Ryy}tBniY*`S)(+t|B0(K$O9 zo0!pY{&f-mr(gV`SwFPns=C<&Gbb}UuqrKZm08-EgVSwiYUyHWZ%1co_ixScfAX~- z+HgjGU)LZ(oQVKPjJSY{JtP3?#wP%c7#Bb_NClrj{+PF0*jm6D=V{X}eP8z=4L(2r z`_mtkDDWqWv!yxRS+cCU2AzqkliL{$^2FI6bN~k+07w7|fCgXySOHFe4-f{#0ck)E zxB;jD8h|cv7cc=V09(Kba0h&V03a9$1EPU=APGnVGJ#y62q*z6fjXcGXazcfUSJ3q z2d04qU=7#;_JI>nkYPgbA;b_02rYyO!U5rfh(M$ua*$gPb%-wH9>g4C2XTe?K!PA) zkXT3(<Rv5*@)lAJX@ayv`XHYnGmsU?4&)dG6$KB41m!XcGYU6~2#O5KO%x3jLlkoq zM-(rVhbWOK&rn{W6rfa~G@*2&453V;tfB0o0;ssCWT^D0oTwtGvZyMkdZ-UjolyNy zLs1h@U!fME)}Xeb4xmn>uA?5IVW1JC(V=moiK8i^X`-2+IimTYg`*{*<)BrdwV(~4 z&7f_eA<*&BsnI#m#n6?|b<oYx-O+>56VP9ym!p3`A3|S1-^akfAj4q65XMl%(8aLA z@Wu$oNWmz^c!$x4F^92-iHS*p$$=?}se);Y>4F)I`5dzVvk|i&a{==Z3m1zPiw{d4 zOApHqD-bIID-Wvys~>9#>je8EHWRiOwhFc>wkLKJb|!WWb`SOf_6ZIl4hxPXjs}hu zP5{m`oFbeLI1@PEaB*?3;ELj^;acMQ<37WEgZmM82KNY$2#+047S90B6)zGm8}A+7 zXS{Ft`1s8D()fD#F8Go7x%kcallX@hh%azmP`Y4x0d^tzLdAvt3!4Pk1WW`n1a}F% z2oeZN33>?DFJfP0z9@Uq_@dv%<crl8M=tIW5)<+gsu0=|h7;xywi7NBVG=PD$q|_n zJtWE?Y9^W^MkQt-mL)bJ4kFGZ{y@Ayf<eMUqC{dv5=K%;(oM2ON<=C^szvHSnoL?x zI!%U3#!RL}W<wT5RzfyRc6{mbC7DZRmqIQTUFyGdKu$$2O>RaWN?uGpM1D*`M<GXH zO%X#;N%4gejgp;GjnbVmmGT4SIu!|(IF$)iC{+p77&QttJGDBsH+2T}C+fY+G?x`F zJ6wK#x%u)s4H=CzjTKECO#{seEitVWttD+7Z6oa}9Vwj*oekYHx)!=EdTM$_dRO|F z^xgC)S6HrSUJ1NXeB}!R4uc581BN(;_Y7N%w2aD(K8*Q{V@%jgB1{%ciA-%w2h1$Y z+RTrcE18#B$XS$Fyjb#CCRp)Vp{$OquUH4!FxW)ctl3i7df8Ff1=%gwli7PXP&kA* zEICp*`mUm16}@VA_0`o8PCQN-PB+ec&KWK;u3KDzT$Nm#+>G3M+%ep3+z1{)9vhxa zo^f6x-W$AuyfwT#eC&KCe93%+{P_Ix{4oA1{%rvc0W*PAfl)zXL1n?mg6{?4Lc&5$ zLPbI=!i>WAgkK1ch>(b=iiC@N6h#-kCJGa+6Fm|W5_1tN5&J66DQ+#EC%!DfB4H+x zDKRH`MbcRErR1~}ozy+4bg5}5J=7Tb3OXyzAZ;rBT6#%_O~y*5KxXqA&o!rO<<|~m z#bkYD-(5$&u5dl{dbb>roTglo+@$;!d2{)E`7H%O1#g9Sis*_r6=M`flrAfoDCH_` z-4MRvd!ywh-pxBVlWxx5V!!2dt40|``G)dS<%!!&x9x6MssJiVDo<6usIsU!s@AGu zsHv(Yt1YVYs{5$7-66hXbSLl5zQ#3;D2;JVR!vvU_gVy623omV2in)QW3{JsxOIGW zI&~>^Ep@B(F!i+bUhD1Y%jw7KFBk|K1RD$+vKV?8w%?_^Yjd~Wh``9$sPrECJ?(pW z_fCyfj9(e=-B-H*;{KM2tVyEDnyIvDoau_0q*<)l(gVo{u@9EbCC#6juUJ4W5-ip& zWi68|x2+Vd(yR`wRjjkEkv7^k#kN?s_iU@}i0rKGTI^};J?#4&I2?i<rX9r`;~lr0 zZaTeohBzBKSGkaY%G4)UR@X<aGj39D$!>@4TJEJDgdTPtpFG(;Lp+zf<h`=I(Y#H( z-}}(}1o+JO%J{zYgZSO|dk<rPJ%lazEBNOG-~?C)bOrJR#s=;NX$RFjy!_Dr;oKvI zNBO}Qf}Mg#9*aLtdxG}F@=13Hf5`I?M5t+KM;K37LfC1zNq9#DZ^W|*WaNX$?kJ(C z)M$)oyXfH<XiQElVXRl|+*9SJ6>)TNp>ezMM)B<l0tsn}IEk)_)6Z@_t9s7(Jo@=* zl10)`^0nmR7t}98UL2&Dru3&urxvAMP76yrPPa@SeW~!W;uZ6&gba)fw~WP1t<1J8 zk*wU;l&`~H!?W$Or*qVET5^SRbMvV4qViGlUGrB83<`P+uNPJoaTdKSCMynm1H5s4 zv-<Yl+u@R%CGSf`N{h;v%3hQcmxoqBDm*K;DlICft8}aSs&7=kuMw{)ujQ)EuA{F@ zt|zUJYQSlD*nn*GYTSG0@NT`yylL*e(fjdco#w$7wU+J=H$QxAm2YislWA-GDEYCj zU97#jL!_g!Q>e4zli;WFF2SzyZlUgq9^sy<UeVr~K8e1DerSKw!1aOFL8ZaYq1!`! z!<xgRBL*YWqo$+FpKU&GkGYMVjQdZZPlSHC_$6VIdh*p2+f?zi&~*LG^_fqzcV@@u zOy<_+o##&%9xmc9CM?k|<u3Cr*R9B}^sef!F047Mova6M5N^EKWZf+LD*g4-md@6~ zw$nCpC;S`5x9nYk-S>N{dz1UN`zHsXhZKi7M<PdU$J)nBC+?@1r_bPQ@EXKT#01h7 ziA2uO`^b4%J^(EQC3XM{R0qHtf_AP*V55NKZ}b$XSx`dI&(p`iTvWf&vmnh1K|N0g zvje;k6i|f#fK~8zevAnyg4#F{_^##S?`7J7>EYmw0-&V-SqtiK^jR%u7Y2a(TbUT( zuixv{!~e71`yl;ey%5mEID6R&Oh@^R#vwT?3pg{N&Y%7MzpAmNoy#2ydj~pICwn?x zVJ@CCGvQyme%V;ysQj|A{<1Xw(7}OUHrC%X&VCrTUpCe+8|#;i^~=WkWn=xav3}WD zzih1kpKL5eP!Bq@O@99$If@(yWC08`bPRMf3=DJ(OiT<coQpU(*w{EE7YOh!ULqkU zzeGYtMtPZumXeBrnv9I@Djfqe3mZEd1uZ8JCo2yVD;w)sArKHv#KFQL#=#+Gr6i+d z{SRNrcK{(4@DxZ#h0p^igb-9h2(ksB1sjYGg0|rGyAB6IK?U23iG_`W3uY+42%tbv zQBlxP(b3UB6dB?RJ_pbU(TV7JWiW`<j4`h`lJEt@yu@O-_O^*sy>E+=|GrZoHV)Y( zatcZ&W)@a9b^$>lVG&U=+3Rxh3W`cM?r3OgX@jb@iK*EGa|=r=XBSsDcMngmpofow zA3q5RjeQyypOE<MdD5$l%&gbhIk|ZyrDf$6l~vU>@0(jbw6=Y0@8};G92y=O{X8}^ zJ2$_uxU{^oy1nylcW?jT@aXugUl7o+{Qdggv%mC<5bPHU8X77Z=2^cWDDL2mN{EI| z&x=7MqlRhhNPLAa0E^^W%*(e;*bMyYTcr1$`f$h?1!kDG&$@QrvwvU50{>Gz`=euj z_G=8lLxq5ohe`-Qf#W#g#OC?bGW@=sC59}3mRG##$ZIz6R+>XhWEnvAP2^V4aKtFp z1$UP&0C<KtVv%I0YUa~ZVZ0DVC(oOx-a$o81fig4qoCEh$RBUrLrjbW=r7ax3ar2m z;rQOO_o4F)8BwoOxRC&DAw25&<BA=Eio;|(XqgMFc;BV*)USA1xTe+$gHwIA)dTO= zJN+?<TjZph{cl>a=V&9?)Jy6+BJ^u@mgSd^oMZZ<o~2Bm);sDX1u&~rDYoeeo7zx( z84v(4b`E`>4hPgo8O*z+qb_24Lb(l8-IY41FT??1!4yx68%elyW^bpSssc__xCgBE ziE!<?NDIJ*^^>n}b^5&`{ja@1qww)mvxJ7WTtg2J)Y-&j=XE9V=xq6h)S7BZf;iQ| z+bZ4|l8&spYGDqJcs${I#a|?N7^wqgddCp=Q;-0iI?i0~940j2bS;n9O1bi7CyPgB z^l^D6y4b7DmH9@*!#IsYotbS0SEn{2mxI}$=ux9;gU71EorwJHHUVn%4)`%}36Gz} z$dA?FZA*&eVLD)1LA9y+`OcGLZ)a%?iK*mlEB?jPdhuRJN@MLQzrt~`PcI~jV{vUP z+z!4p7Yaa!PwqdLG7VIWGYgFtUHc$d^}H*`tvZ3=%Cv+6p57%Kzi@e~E#~JOw^Hxj z)DJfG=nW1}7a!%46+$%;6=ANa&$RV^F%er|V1duRuwiO)m8UC~c!7_LzX|HT1GgL# zig!l~v>yM+;_wLSY<HhJ+PSkb%Y?mBOE5y*%Er9w$jd!L9Oi!bphS>h@9t3s70mN+ z)*sKaQTX=Y`O|b%@ka1*_>mF!-CaOPXHhD3LUq*m$?=J!H1N(qc<QB}(OzS)XF9|w zo6qH7F(~%4QMELx&(vxowWCav0rylo#36&o^<Xh~RrjU~pUbxy>TgJZlgytBMh06* z!A}s4%B~~<gvjqpHB6}TT*5zC28<Hn_A{v#6!-}zaet{3a#rNa?<M-<#m(|o`MKO< z;Z14#mW~RXw#>j;PmsWxl!42JG{((sx-lp_pF#)5one21YDDp3&)vPtExlvooz4=u zB#vyc3Ne9stQ3q-dm%|cw^-2;tqU!k$TeE@V6m7;(H@#-$2fYMZ{Bx6cXdsXfKny- z6EX&Vx^7^ZKh_&in%0_ZQ~u;zRC@^tl=~SrsB|I@kbqt?O;M*KC6&*$d9FRCQBgWb z*@e`jxkd^)7n@?oyh^vX`Tj3j-D$hSQNIyC-Yrm;{fYz%jvNQdDOeU5u-Bw9Za9)p zK*e~7pLNe2^+Nom?l~di?m_u3XYZt=N@;hF)RXTt*wyC%tzReoL^9y^6|6u{^3Lv8 zc{jw@tuBCewf>u$Vfx9{PoC&hQ8{NkvB1rZo=XupYZO<#lfn52UFOu!W@A-a1T9=c zi}Fbw*=Si^P)~*a3l(hT=WQP+Bl$Z<^yAAb_oj|dBd-_3VMZf4op4Q?%yNa&#`q!@ zMa&qxeyjt6hIxk^G2*d|Z4=V1ijocoxwsbELr7k1MPOv!lIIW0d`4XMteM?kzaS|7 z(ub+DKG4{A4U_}8kH|s37vR!Z1o^%faMI@h`9ArK?^yvGo)%x<s`PtYRA+q8ea82L z+A<A2ynpchl10%#I0wszsU3D}qrC-TM3|L|abfq~56jX(;hu2U8Sj!%lfFOWT^wGJ zcP$KZKcz?>fxIiE5Av?bvtw^ZX_P1)vzp!6M^eVkE^8kEv<>^KE-SKrYn$KF^Z;^C z6ONH=0Z6q^^GG3pdooquD#_0Ant9T6ov=|x*^cFHcb{Q4xt2;;)mCbGYTErYBp_GL z@@9C+EBlj(U5ZC~O5s9e3dfUm4EV^GC8;vPa-Sx!BpivRgtdWClB37?1I3_gxVo-% zAPdi@^wG*l5?JYDj)80e-ftL}itkzgm{v#T#WyY;7<s=qRs}LTMeR~V(j&SSFQixi z_LJ~Q-nCm*k>bggoMwjYSK=JJF3Mx2U$8;~w}&}>a@L7DcF~VAtE()iim9*KV7Kyl zEJ@&e8s?Nxriu6H$zE={&K)brrgV~xNLho~&YP^)bsCv&OLa|s9^-<pvw)ie!*IgW z*HfR8%XMsCHDA0Rtd(NuI1zWtIkNR>q3auVE$`%YpQcA^BT1042Hmy(w?gRfk;x^g zI7oS2^FSehC(#_gGLS@m6oflcEC3K!`@9{=!tr?<yfzXFP_aAGs@PflChLK7VgFs$ zs{+*~A!6R!G9PWYC(<DeMteq)qUqCI+Tg0FGJY<0vaKQ54|}sm%b#8m{jF^)31~dk zK>`UbLK?H6;OYfN)t()nI7>GfRDyEJ&mNRq9_i;Y8~84?WsT)c);78TOo&sX-=k){ z_K~SjfPcZK-n;L7J^SOG%k49<?T>fV$bEa)+<p@Y;PpZf`BMTW_FtjYsu)t`xyshw zw$wK%bm^01tB9N)@lL@@k2RI8ECp?^9H_*Pk#D~Apwz+Ey~r1THTwlmx6EGQOJya) z=eIM$N;r{#Wml=)==LsjPEY7gLWROTzRwcAj&L@O*RP5KiJja`56=a_IsX?#{Cil7 z0=p&1@pQTAwyF5zC20)oqk#8bYM`(hI;~n8^T&HU%Ug!fFeQi!Z!838o#EzPsVU1% zw7jA!H-N>u+cHBop<7bz&TXeSlug@Q&%Ir2zY2Ii4pXae;m!ZjPJJQu{!u`$nLb+` zibk{|U2gM*C(Ls%d+yBg)v;wB7ORXrc&a-zUE5D1D)dgLoxS}e`mx^!DrqGtRSaCy zM8-F}8@%y0858^?M@e-z)rW{5D_b0J-pl5@-25#A?eyJ@BuDfrK_jAgq3!OVDQz4c zv=I19hp>)9I=s{ZPg~uhI{eYUPur!djw~vEtbi!~qm#Jb_H00i<N>(N^zvwPo^3P9 zXWI-b1!wR{9Hh%5168X8_rR~+840|UQ|r9)fw-OiJ>Au|UYwPztrtUn^h5PokA!%? zw)r@3S`=<YCM?8vQY#Hig<g;~_FHhi3`}t!7q?OOI$Yua3pe2Tl;V5%^1JHb4~ZSM zua-5t&9DM4F7j`Fb}2dMGFuwsTxg%kY*1e)PA<0MtvyHs1s)R6-v|v{+@PmXLtkXS zas3(BA)SL%mJCZkOe{KZXV9fe5Zld4Rlf_xyz>tE1G-%NSb6Oiw27GWm=e<(!)2Vz zmuZe#6YnVoY$GCrXxa9&-8im%?iFy#>W()K8r4rn#6RfovdxI3Qk)I#!x)Otrt*sQ z-BH}#bh4;5HVvVx)=zoT(R)kVGxU}nHQug>$OMEgCUsBfyuI{~K~1JFqOtDO@h@u5 z6sYFN`xGY@djV8eok2Ec{Y`WO=d$~|=<Wro?>rat{-)L?;3(leuR$a<sP16kuj4QN zf`j_PyIxd-n?lWmJVmHHfL*st(<-mOOsm(HJpb)L^7OFIM-HcEAxzKwF?6FCr!Oe= zkU*}?$TO~e?J_?Ms4GOh<M{-uq9j&lOG;LnlJ-N|k-4l{q{+(02LL~qJc?p%MUy_z z+BTPD(&ireXVI%`2S7{8ZQ#DU9b@Grpo*4opjH>m7Nj675~&s}f}?c#8U89Z1%*hr zm1Fd0dr-CcUf5Y##h~7DzLWg6kpLzOFSu{l^)`1>+^rW)2k<)=Y3i*R?le|$g+189 zub#NeH@(}+(>z&3xM+IYAg8OLeuOzvu%d3@rFfH@HH*`oDx=A8;g_N`t(k$})Png! z1Ye@PWtxeJVtm^E^tIBvWlU6*@_p8xu@Y;}8;3^va{Oa;&Z4p<%<4P^USFkB;~V6M zzG2=ikDKdQx5jqnc+nH4yqEzgeN(G1ago_K)=q@I*_!c*c9m2*!Y6(zSP-{U*S2Uc zICavLx1FTqZi!e-U?x?;MHvQ+@<)Q!rED7RB}_=*Axx_kTKS+2?pBWk=JVcd6R(Pi z#?o$Bw=?b-9w@lFB@@IrEWaynMv;~I$PrFymvrrp&HgZrPRHI#m$MXCb%S>@r_}oy zPQ67@7SGVqi&s1v)XL)`nm)1!KGggYk-1*v%maG7uV$2m?$Fy(-)D-tgnli+Uo`DC zTSY4EU^>5N&3dAgO~En~QPfr1X~UAJxO;o;Yq8dO!=gn@Ls?yKLQ+q<BW179vMUV+ zGPtciFm5t1Azc>ZZhxI|A#0xY@I=d1GyZuNi`!iWtD~S|Jrd6#&XH`=ZoA+191>4+ z&Vg(afUvr*X`s-9CTr$U(AN+5kU8?3Uirx_$K1gLf#(D5Az=?I>7Qs~T+?_IovGk~ z!!R8GHG6Mut#W3tyC619rFbtp&8TI5H*5-LbzfCS=0c@?mc+HDJ7#rH`3>^)LLyNG zOdl`adgL2cR_#ex&vAFV<4|j@ePkWy1LI51`O?I0E{gpOMG8U_#y}M&1zO&&+hl67 zxgr~|Mme}2+AITW$3^e5#^SZXH3`97nLLG;R$8@)E)S_bvYFtqZE?KFHjG+2p~`1o zu6YH8LeDiCh{Lw=^a_%`Ow{;JU;`1F#<5bxJ7iZUQZ-1`c`nA@Xphd<`)F;V(Qsn< zM$Ca@UzWmNAkfdA=OWN16SP;TEqtZY^+5j=4!g|fw&=cbICYGu%$g;a(7ftSkbZYX zwV`F_e)Xe|T+E$h(X=&vK3UhyS?Gf~k`DtvB@R%0#yl3{6QSK+%kerH(pa%=oGCnh zDC((+;C{EiK%?8~vhu;fh|reP-bZ0(s!}&Bhx9?TiQnN1CHoRNEbNa?)r|h>r9>#- zx|GKhCt<jv)eC{vhI}6+#62CeXS9jJ9z|Y$)6=9C<)}Kmozt;nCXdQ!=NG_RxqW$% z*W)f?u+z~Vt>)o?M4%>F6rEEW<GS$z(QRBQcOu0??hU%!vZMREQ(a!C<wv^Y@)Gfk zo5f7|yTenk^vbb?y`Co+i9(M(Z7p#p&>G@Hl-$PY$>(T!pSQ6fbZN33RMh)l#t!#s zxsHveg)ik3L2t0v7T(CLNq9FlX}mHVOk>=!yV7;f4)<SLvmA+9crW7uGffF6D>DXL zD`oJ49OVcYiY78G4E;^Yh%gi>dL%_y(gm%ksH*SBhhL!}4b8hXZI7CPU;BojQEDhN zkM()$1^I;J`_V1Iivr)8zNEfX>UN98EaKOeziHgC;MD$QVkNZXZTH&sF@sKi8D~ay z5uAGgi%=I=ENt2c?GjJ!__49!^>tTyqIX-Viu0ZUPJpOLkIG$+SAthw8M7*ttgQ%_ z=vH)wd3)El0j&dEU)s>ZU$z@iyRkIz8D<|k4Rfkmxfd=?<z2Oyc*Xo>&gr$vr|U)h zJ(x7r-L9f#0ufS%%H`kp$Ci<@&gA?x>v3XYC40O#1?q3F+w*2HCIT^0+`UVZ`69LA ziA=7>k8d|%T;(i9QFtDrzi=4@c8)cl?FQAlRNfjloyes`@7QP^?p9i2VX!wIUDDNB zH={IskB_PnrFL_+lGZXJjLy6I1{+;2J+AGt;>u*X_WAzL!8|^cl$r33Papq))h4&a zSF$>jm*r$`<#yU80jRtqvbO}HVWa>ftq+{lesAhYI(pinr9NC`niUAzq%9O8Xz+^z z8h55nkJVKU+WP%3LnmnUE`iT2Aan?JIV50sm<0S;AU;?iiOd1Pus;;}`F|J=2$@2` zMulAYg#!ZR|NC)3=T6$GL<B<ka44!Y`ydMmG!`NRX@(Iir+KXlh^^CQ6zHGnMt@}P z>V(f64fz(r2aZ(H{B#hfjR)I}tp&a*(0Xn-+<rS}zw$t3K}(qX05lW`;lC&6A%4$V zWM}l1pgVuVelxAZpJAu%U;(T;0wHL!z5-u?eO-e^D3mwWz}Vkb!|I@7K-3?}+JEMC zc0vLkey&E=Fc*Xdgf;=%$PGWW-_EBsSNVDq3BdGEtt8J{s2OqA+Fmr+pXrT%WKK>z zY5taXqJIM7B3P$y;YYBqbFc`V6ET>V(aG`DoZ5L43(lMC&+tdG<{w!NCK@McQ?$pX zzChyvg@vfEonX~er&evB2TuBtz|=4jpo}_eqNd4tdtD&3KRhY^^M%E;@m_5>8~Yp- z=vA1X(aGV|oR(B~+9Wtbm4}1gXN@e<{?N2}s{DG)w`hk7_O^#j-#V9WqG=Y04aa_G zGBTW`?MCdTE%x<anY9rrc*Lbs5Z5v~6skr7b8%DaBG9E1T_n&{*n<S93njzD;d$aJ z|FY^4O!#9=@J(yzq2T$OU)L=fmq3&Xe6{@S)r`@Pl6m6A|A|V!zxhA3&UlmNC;L)7 zH%;pr5+?PrO--xo5*({d!%>9aHZ{|vmSrh_FmTLCemnNZ!vEJ-c&WgzC4R;e|Gp)j zPj$wV=~#&0SN;&}_a&))E>SekrHeG-k;%yp$YHG@hrQi*LF_gj>@~J#oO7biIadM- ze<bJskh%I?s1%<I7$E8oq4sY`K!{bL?vFgqq?b2-#)-c*6Fslo{6|3q#9QUto{6>J zWCmH@P|6R|5=Aqw@JGp}m#^i1CMAFC6Sh`f#ShZ=|F-U`LU)9Mq&TAKfcBV=a^rmT z3?9^AZKtM0E1cfkgzlOOx*!;@Bgk}Kg|==#cq<i_Ei0IksqM}`(QHla)NQQXFQR%A z{XWohpzLa}C}dH9&!Jy$t6A3N^?tH->z$pkwRMIgNlY%z6#tcr9lB09b9h)R(ew@5 zBf*7b;$3>2<|=iiW$aAWH*+$_bY|q;?-6f$Q*sqLyzTGPRX>cn9(+93VQ8*=volLO z5TizhF>lJ{)zVc08`na;`#7wx{Lk-fV-PFss)Pq+BZ0m*Lmj!<fj-W*0cnScDo7y1 zxM0=iYJXabYxz$@Uwu2~fAQ4)GndVOY2D6Jj7T7+M;Qsc@ohtd2iKgd4cff3d<5aU z6Bl8#MvmYd(&uMyUt!rQxv?bSY3=EprCX6=&bdB(us_0yb~|)EgziC;n`Sqr6Gz6Y zn!?J2M#rvshq08R(jsNooCrcrz46;sW#w;&DB1k_7_+b^-_~+{lFvZjar!(4uwcLl z4ny}j{X7c=DNJhzUInAodkZG|IF*vWdyE6Y-&sg*P%PGdmDL1o+ipCHXfN-CyMXKM zlY~;%;Zx1J8XcF;WN-JHidvoN6G}O;mmyK9pY-IP+0xmLRcTVyrm4INwbW6rAW@T3 z4Gd*XM@jF8&3jdX{k1^?$3m2;`7756F6+<FA(%>$!1AcISQ@TC_V(p0&3y4cwU44Z znK#NcD}K<Ma5b&Ef6xga>_b{V$~MlQUQb8M*VO+}%>c@^3O0XGvWN;bZGX^)P=@qM ze-wki>OR+1aSgO;!KqIL9lQ5VqTm$xZs!VO5x*gtN6+n#3<#!kNm>IVx&p(!swPJ| z?DU94O&Xm<olLdqCeW7N!|YEM%J#I2wC;h@ykf>>-BDKBkaWyGU8UK}LN;NdnGFmG zpO@Zga5BnF{gcn~wq=*K)HdIK09oL&^37Po8*271Gc?wr^PUyp0s@yIxPUBd9jld5 z<o07XFT&&nr5<Om_RPu+3gVJ)&`0(w0>s!9&r_)0OaqU<6|5?Ye7=NLw%Psqr|BLM z?ftb}&UO5MDw`BJe@HmUE!MFj8@_ZYVSB6#Emgk%qwd^2w^(diww0$FhRlC%{wvw$ zj)JZxJ@eZbX&{h)J60b>3|@g*iMs88Ig77XK3HJ)PAIWrAZSulvoWS<{JRno;hDbT zo|)Xm-Z7o}THh45$f2pbCx1J3^TB{wtVmp0QBFoJHO`Ae{^so0LD`VyiVolu$agrr z)9lc2xGTwIia-LNO}-q>3BW6(y=nQA?Jrv1wT~xV*Cuw^DJ&HS^)0ri$6A{6$0<u8 zyRbKw{LJ^O#typav}kMF+dj?Ftct-MSHXj3@(hnmA~5Bo0b2hecMUBN|E;|?jRbOQ zHa0Hb%o`nI40ZJIR;z9w+;fW{I4ybeL|*FAxHE0exZbcIN$-Z4u6zo~0dBu{K@Z)r zc&x&b-nsenUFj;ANS^4E;{?wr1LUaXKgq#N-d|$!%pe8+AL94UA=tcKFKjJ-Waq}v z`4(upTF|(auq+X^=Pn7Z*Bijp|70!vzf_(2Uu1p!clNu;L=)61i)E0269`1e=&e+q z?cWB&1%*feTPvXHq*Ai+vmP+<ODj49$z1;TYlCaJElJ@?W`R^L+I3aY`r`~iWgnfK zeB)(#X+j;PvGdmw#%%KKOZ4-H4LhyvUZ&AxGon+|=ma#i3Cfhiy=0%c6iSh-GuJ(^ zZ@KD08z6`u@k;OdJANgB$36p6!X1ZA4;UIzN6*4N*+00*$F4fp6u+e1A9>Qa)TxLB z-i1I9L&HwATa3P~WP*F(5%{xIPP^nueP-y?Si==0kfzK6Zk_E&phZh7x4?R-rG`Q9 zbsRPJet^U?C~LRhjmcu3?Wv$WW&PR8hl17|wadGs?!IsD#|LsJxED!F*t1w3tGmqB z;A=cx^)mW96+_Jky_HBC%YEdKfK-wS90@!{0#9;b6?JC`dI|_MC9okHacb+YX!>O@ z8EoT3>#OveUVc{_12wujxKG0nsWewHxba}1h>_le@8Y&%%QSklW2_g(n94mXF|!t4 zgC<5{!pt`xv+fG3cD&m|qj~i-vj$(&_jW>|Hp@(Y`>Xn7g&OzxjHsUS_;yZ4-6nPK zI3{@&Mm}>EFrM%X@E8%-a(l4y#qeh{CKvCQrSaX;_=j3|i>K<7=ApYDQ4^8Fi$1W* z8Yavwyt(b7#B-DJlSg>aoTM-s?P1~sZ26<r%Bf<=>DAa!_M&`Wn&+=4CcBHan<%0) z?~&OM)TBrAd0p^IS`)40R5MsRtoh*5&P_&@?dZ*%Mw-X|yog1?ls(9efc>@-z{)s{ z8u<v@3@uk(-oY!%j*Z3D1z%wO!>drXC#<hRf#2_J22UyGt=;@)$(DqT%?PxJsFPIz z=p)ymw!75U4OBfK^qdA~Nvh>MGhnqjjh=v^G$B7BWo!I0zp$Qj7sNjlp0o}ffz3M) zG_KR0V>m?v-;Iz~>+~p!mCtCfmWDYbATO(}GJ9QR#tcLl8TszQ9(>Mp5>TI%|0Wjn z5DcsHZePc}QhbV2VNq22Y$t+6Zb;x}?N<^;?GY#0+$j#rE)4zk*xO1Lmw>My2~ypo z64MedS{*-l=|0ENnM?m+aD=Bb+W-j|t4M6WCl$GwCP)N{VP2l#*7|5WDpfCc-MAxm z1MZg=#(u}i_5jgX9P@r`{PrYsT|eF(->BPst2|}qc&!fC&{@KRg(X~QnH<%0+D&F) z7xl(0`*H7>V|gatO1zn}K;%z)K+9#R$imMIF!UBlCUn1_?ACvNPx~8}(Wm%Hc_)?h ztG9vm48V<K?L<TnSD~Ud>QlgWwUfMq40WVefYST51^cz*l{J?yNT7J_*!<ZS-SK9V zf`*Wp>Vf71lwqM#Yl>-Eo|x0m$|X6mnmG3-UOLhl5QoP+&$8%3uglH1C>DN<QMey+ zb)(GF_|dGjoQ7*@XGI?ilP7({{rpJD*P%s5YhV~5_nJ(#(J`U{wzU-R(RMp_#Yd{= zb<9hquq$J$V%LvLW!c6nebm~kq~1w8*V6KQ<m^fJP^U_^$G420lIU@!#wr@%8u>`u zBHSp|$nUFc)u+s1kgy4&Tf7uwQK!LmqTpboB7w|r+TI3a>HLL<&a~eUu-#Ovuix?I zazBRGiSK{5KMK13sfPFeuUcS{c}v=k=t|>0K@|v`M~|qSyB-!L<Yl%~lLHBU@&G8w zk!b67fYS{8^;QERvFL+f&~pfH1gCkOBeG6p#78*o2nj%sRRrDi&mEI`jf4X?XMTnm zhaUv~!Wqx}*uXzj>lq;}LINL`vSFiA=knL^^%;)57Cs_F!#_}hg#<#*k@%vtmM3ZN zJHhF8eeqI`L>2Ov8^bLFJl0cu0X8a)1Xd)5CP&b7@YejO+kW4V&%*#lzVF9jKj}01 z{4&3An14JD!|{BZ<uuaE9v*b=+8a0zUZ_qZ+MdH_t9kskT2&z@7By;Q_cy1XR48jv zY25Ex7;DZ|yO#}Rp^Kj@o9Q|#us$ABrha$7fo?cfaM$|i-3;49Or5bjY1>4^B>mP{ zWTUOS)H{9Uo>j~_!^ku6GC!fX#ijr7`p+WQ^?CX)f5dsP%g+L`_rKW>^U0&6L$|Q7 z*p`9MP^WY0s+DLA7W=B?eysFnNBO`HK{!cngUUv-&ni+m@;#O+U1KRo_)2!^2IDqv zN>}TLaP8`<W1HIUjDN~(=5V(Ym6!bZQBsj5DZQugy<1nhGRKA2!>gC>>~ejC@o>Br zymBD=MJTLHlb~#MYWM1D+5ut!2}Jenrx+m~`RXGOV@P0o)n|K?OUaG9PeW=T9|`mp z=fkGPjS!LwW3KkcWUeaU0W3=-(6+o3<uIRkKRXCIuMHNmmbNEYZqOLCDJV0Oe=(JM zFjMt0Ua6kHpUE1JVCl~lkj&XOy*CWoJ!=LM*yUKA0=<b~JJj}5^bwCz?ttd{7;O7k zbZ$bFX|z#Jjd%^up@FLL&KlE)9NNrGMErex0*yhH)7z9BH)RhBZ4FcAX(H%RQ3qL( zK!CdImEhd=h+cX`-|*BO>M_06;r)#<h5Hh%G|d*Ou&JW{JUOgT#z@xp^8W03MJNHi zkWtOTbGuj|jla5&)Xey2gN6QQgJ${v*7g723BLUTTffJg1x?!@kqRJz1AQcLO>kg@ zf7;dlP|ij~lB)8=){yHwfbQ2keh-cNhuWn-vlj4=tQ!^gUE>b>D4Z4v$XX2+6r=JX zf%lwdAHnluDWDdWp>-bLlk>})|2;zMXEX7CWfVixHghg>lqFh>9@X>glzWkU2%UDc z`HBk|kPT_da;qv%g%4ikkKUx(Vh{x@&K}txb=IVVtHG~;Ll!}#Y>n_2LIQ_8NMOU& zAa-3v!aK#*{O%Y1(J^ILPXP!0Ymep=<Z*H`VwEJ(!6>NrN`<gqj4335K?^U+u@Xzj z&RIFo()OdRh#gyG4%QGx0-ib-6^<KaRbmZ;QYG6FlvZPY`7=8Dx9Y(Fmb=%Zo9TQb z>RwR0h|nPf$40_^82emR9#0a4$%QGFS0#4GdcM}*XfWZO3w&XGE$V%3@u8An@i8!t z1g4w86Uu$<TunVROZuADFF=>>Q!oNF9lsHQ2R$*Ag*Spt!vUKHf&(Xp)UoT-+ncU} zR@Jry1~m;sS#RnG5{5N{dbE^Idz;<*2|9SZwAIc-P!b9=DBVYYj)YUZ{|^n6`?Xu1 zg>wJB;6#I^mXK6jd}hUsg=*J9lNL2nyeW>|3F*@=m*g`Hpn!mehsR=+4)i^=LgA2= zHWxOVZjKLL^>ZsYRL#e)2+GG^l#b7ZJvp4c3+FI8d2oPu)TNI&0YhEp20a=P(qQbg zQ9|0W+GSg9^E+`L5TEP8Q<TnN@Ib>x<KZ6dxNzexm(|q47cjVrrZA6{FbrPm4Q{=Q zpwrKH9Jaj*t+0nJ8w@pmt+hu;>T9TCaPN10h2thw!VhdgWisb41=^}fQ#26OC4Frj zpx8ahzfuE3nQnFW+j1G*IG(M&u~2pmGh&>P;4qOEPRWsXzK=N#9zGvX`3uZ3NcR7L z82c?A_8$sV{P%)GA3dhvcOyWwk{ne3F^g)ygVXIEGm2sIU?>+eFBpwSpn17BtCaWD z2>g<V83@dFs=!$zEAlUWKmt1$M#raRNTC18npMs5i<Gm2?;B%9*}3DNtq#0g$ZirP zmMiLpXX%d8@(GiV_3iiUecY<Z&l@|mR%ooX^=?T_N_nA;kEc5JN@fS+J{2J=+8c=O z^C;tH$~A#6BA=VRXw<G&-f8D=FSVZ(Q43G2d8sYAzZAOg)dy>YQY$FCd?V{aMY2Uj zY!l3E{|U=GwMDGDclmT<4qc5@QJ0;{mBxLcpe6Ig^%8=Z;=y3S>w&Tj71fg30rXkD zCKBr_KBuLZ%z%J^N;jqD3)s0C+b-9TfNHf8Q{6yBR$PT@6ZsU!r+(m)$ika64}adL z0Z-L*JjB)eZFlGU_NwT;ZxUuNT0^eXU`$Y1xHbp)LmwA+ct8>xFBF&$n3fPTysOxE zc$cV(9iul<edkInTIl$e_QIS9-Si_GJ45%z8TVtomRci~HM!$`F<%00LoK!gJwu&| z@%9B}TNe@ubI4=XPX$<XfdvY7=>b_fG#9q!{VtbQMGz}1u{{0cI#9M-`tTkANGol` z>OPx)aqk}f@mo@X;-kww@SrF}1nKw)aTbb?odY%qXo3ERzE`z|=LQy&N=S+Tg79Z0 zQ0CW$^8GB$&-<+Oe{sb5e{3VwxhshT)G}bB^5Ax{=6&vr447-hz^4Lzk<Z#d<w8=! z{aV7Orft}1Kq8`%6T16KY^=cO5(q|7z}9v^MBoNPVcy!RlM5Ip{_H&Z7L35fa)BPv z&?1_^0^~ziITHG?XoPK|p|b|CU5ldp^IF>P%$jRLAO0-uRCRuVRR08!>Td@skI>J- z_Pm7Q4^2Rnh^A)aJB!afy243C7xm*JY5VsslJ2dA`O0IatfAH^A&pL3a*FmsO4+mZ z6Pl!b8oBh_y4hF@=?{<+7C{5G8PI#O+-F#Hj-tnAw{+*huzp@C>2H6*-T!#F`|pf- zUMgcB5~wJc39-v8Pry#_n3uy3rl1ZQyq6>>DL-;6gl6A}+jE=}{pybJff{hNZ4||- z_<6q7l(U#*41#&=;I`yr4%)e@6SnjV)-7ICjJ#t;9<lC83QG4Y0^4aUaYT$CQHXSe zrv_D%ay@@}7&>Dbd!l)A<%Or3Ct;nx`=mk#!EK1~rnejA-RCB+IpDE8jM0m_hGFe& zd+wrF{a(L5S1Gk6YM-{#v~eN<tj%07mhAuxH&R@+Ka>}Q9+QJ-C*CQp{dlN}dS{b5 zX5DinNJ2QLb0$J^;I51MW88fWk@?~5Q-+#sI3{R<K^O|5C?%k|$dc_OHS-nGS9l)$ zuhzpg16IZ0I@4vDZfreE_Y{Wh&1iNieWlc|D^I7Q=b}~r%3ON2zQue6bbkQV{1lJ- zt>7}%nR1_xmWxJ|Nh!K8A<8hibP7$EjC^$*cbq(>#E3#7rT6Ml_xkpTgjW3nsi4dG zWi|MH157NAA(wnh)AnC|M4fnf%{!?OUh0^A+CW!gtwFnUgw3a)<|gumwqd3W=Fx9= z62W0(r<s?HTd>Xe=1nXMRebL}&mNlQc`#8w`Op03|Br>y|F(qs@6}@esdZ;nR|gJb zrY?HCe1x$StALXysU*pfis)|~L>OKe%)~?(74;=R3mmt4_QW;z@GMZ(5&PpR71m*& z=K={^kEl>t(rGe}-Sc2JL#}oLqZd*hz!p|Fp_6Zrz<uJ6$L)d8DmgMr5}^YQcbEPt zqsT;Iawh&kI#ery1L+>0;mvah6%~4`uTbDf&y_%OEnLY}+&;eT^=30X#$OyK^*?gP zrD+R#8d^cSS&4XPon2)9ZgpxaOlQjrYQH~LbFMl5J#P&8fQKJzmglbJ)U7I$>bht= zK3eT3r`eJpo4tp)H@xe`7e;J4TA&L%y%blDOW?bwp^+PgLT7#W{ai1V^%J|oesA53 zBy0?I#ZkUDduR?;aOV*znuimT!^w<ewsR>xOcf{}VJn%Yq_AA7x4oh6`{jt_oi(>W z9fi||Q4TTRyIh7){Au?q{=%z;VjcU#$Dg^bZW^azo|Igli9=j@dJazik2;6=AJYnd zitPI5I70G;U>6w&BpX5q_TxKJ3OY{co*;orZiZD7eu5jo*aHpq?rS*|j4uF==VB9; z^VyF#K8!BgGcs`|E`Kpa0{$rCWO^kvZ1u+6Pq-2egJP^Q%t~kv_}tUnSq9Myr5^$) z(U{WcfQi0^;<v2fVsR^B3wQb$w-1DSky1gcPE#A&3jc*?!61*OjgCldVAHT;Dy=Lt z4~j$tp*5F+t~e4HkpI3j{9WwlZ%>Np-DRx-1IaHLoyI;B<sWi!z=*7hOj6k4MXAwj zBw%5cWKLP0J<i#7T0ct9NTB^Vfp!aJN=EM-Hv0X*ufWNgq5P<Uj)9ybHluw+5TKyy zi?t~hYI|o;;3h}yQ+Hw|Ge2os>vD%!Y8U(=0|Es+$$aRBSnKoFm)7tM8|cDf7goH& ziR!nsJR+3?IRM;FItPk3Tv_odh%=Wj43Bx<S#HjeYD*gUe+GR1+o;VNjiJe{Q29th z`7Xw5n(^7HGMArp0Y>?nA@J%<P@Pr>EhH4;Q>&hR1c;N!*h0adcTO!(Z{!WZHsDl< zz%dvCE&FhUuaVG)6FNq$GF13=lm2<(7_K>bSh#uYwMR=|V=q-$pcdu1*Q^s7)Te~2 zTU}afmaz|7KmsEQP<B5SmN$ak@M^)^u7-B49fL@qGbLsAt1)8YGv=0Z72+GtWXX_( zq_IuHzV@-kMV2p$OA*qikaP@~so^S2sL|myF%sY+E6}3NXvAkhh-KFd9o~<`QfFmQ zoRuk8BLOmcA^jPMslLW#PH>@3-+)DEAtw4^%WR;z5P*#kfMjUp6Q4LMfwp8;6atOq za5y40!_2#J-92H1inq2S-pJ0&9COT{Q}f?<ATs|=ILbViuSYt1zUI=%Nacawl_9zy zgVc`6(UzDt9BG*j48}xx(e}OU%^~Qrbp!Za3$3(MtHL|B29MQPOd1Moy;fH{uMcvS zotV|s1!BgMQ&fZ)u+cjQg~x<0R$8U)3QobM@>LLmw?5~>dPiVLz^@Q?I0QAL_ttsd z2uLXV&K3hqa70oCRwdsDEF!XRB-9D>cs?H<pJLSCuIo|i&+5O*@OD5#&fV6M;(m+* zevanEhe)XFL@bQ5zE%T7REtg!a6Q;|o@89mSj*SELPzg%m1D}UjbDerA*6h5GuXIn z-FVjU##5^(GrIk^;$G!<eM{BPUPS`nyP*F*FB)d`7UHAnr?gHFf@E_V-fWoVA(KOT zbLllh1+HBbEW?|{bT!o$jG-R)R27~wEU{5WI|bE$j~2^`GcbJa7#myjvmW;^)x1(4 znc@`Y{qBA$?5|v0CJCB4Ur5+nRf%pAPRhZpD#N+Cuey9x4iCc{FqFf5*4}9G0Ei}_ z_~1NJV^`>zqkmVqNI1owB!-#O1l2l@Gp$t+ZZ&h^K8uukcb3n{K-kln&J*t;a{YYo z9FsVLYqCDg@Pv>1dC?xpEeSx_6v2cQ*(x7%sE+zQ=$Q1V$!68Q9&F#O803}VGsS!v z(LDs3hGJ8P*q5!>W75jaQp8(JtawXxmn->fL|jg;-`Gx*SO0uxU0N^;JeZ<<0LLAH z9TE~C0aAlPBoJ)(HB0IB@MSv_a;4cyodvb!%MOmKVl>ku_=N_is(q(sl=@7q#WR_j z3Q4TC!w<T#9ja;hcH4k}@j@15y8Vb$kb!?HRTcJL30yMQ_NEYMvPj^Amg>Ns69h5g z=W9!pcp|P1vcVc`n$ZOwd#U_E_9<f`2#s<Zf{b$=da4N?rvIrhowUzmjc`o;GkYTA zhnKw>X@-(vTX9pc-g<YC%`SrLLG#A94<kZaB@<tkg+uI#5@Q~2z16&YTi97QQAn>v ztN&xo4AaFo82wW3Uk=-u)*X5}Itxt+*+M_rW}seuE^@l;@=8!GHR7jjX6?nq{{*zk zAh<Ets&B2&^4K6jmHTtufp@Zd+|ybMRGW~<)ZPAuC(7emAXmCn3>Mj>4+v7VRj@O9 zpy|t);bDGS^&%XvcdWT7S)bZjjC;54wXQ+MxJgPs{{^3(Yiuqddz?2w1gv&oub$of z#fPgICtPN@&)yk;pPtjitha)vwHWOZ_rtafS<b$sz0%sT{NB0Ykx^MT9LK%!U~?P^ zM5^t_fqZ%U009R*ARVQQcj+Q1rZoDm<uInLuZY2>j)R-_YEP)VJC*g7Vkc(>6NC|6 zadui1m>52h1YW&CP`yA&?VHta?JunE=Zmdd+REiAW;6_QIi@~e)0xEGN(h^tucCxL z+>0Hd@)@p5ptxxwBEmOKqatJY`17e8$O@8R<IwxTP#US8S?*)(73d+?)3bOVyO(}{ ztEs-9lD?D;zg5n+<Y4*~%=&oWG}{^d0Yk{)jL!UaTU9RNr_y@S|3<IN_hE#vUKLQt zH@T#p`sG^@uhrP7P8i&^eEQywE@bW|8^e9Zc-5KkHZ;2)S@82)=FVn9FG*8cA922j z&5(HD@sfSqF=mI~`Qw_N9)2B<NB^`8i6d%u(KS{B#1{C?RRx56$|XcKsMRnQ;!_jP z>295Y&R!tzH>=&5F&j9HEcyAsHSb>!T;pw+YnAs+hw3+q9Q9*mD890!NJf8y66nPS zedH*`KDY}vF&bb>QyAo0pRTBqC$=nVGE60^BVBvs2WzdNr3_@=+Lxi-2BQhaJ#+WU zMU3vS$5uS1FKvF#f*`cygRSDh4nu_w-4TyGp<A~=dTqII-NZS1d}ivHbRHZS(AzP~ z`qM9t;#XlfOZ1h^jk_G}NFWg0F<aDr`lW3oM~3?%c<RDgfEEh^I|>O9%K!As(3eDq z%W+*oC{MSJYZ?*@dwZ7W_OMGAg^M>Xod5RQC-n>AG0LPh8Mf9>YN%282-XbczMgva zAOVB%BW+&p(<gcaC&%W9sNkm*)j1%rYu%i-fAK7^*uA{T3RP_CG1zkG68Pop#iDQD zT{u13v%*<h|03d)gS)MueY(&w=DCh;XYidssXLS9Zf74<(4~Qzf!=Bc!+BWS3(%$b zSpd=neN?6d5Lx!}VA2WQCMec}&XDy%moH<kAp}n9-THe}YlHW7Yzu8U`)9p#7qF?l zKgY|kM9oOTB`qIp(}ZB8ke-fpCtnWFyDOepBSV!#M?@vUz=(Gey52iKyw3nur+*z3 zwl1{Ahad=rRS|*&q?e&a+qw32vaeS5$qq)28&a&zoFIX-)^UmpA3u(@%#?`Em0$9s z@3WaCtsNFvrt}S_=j>p6>gQ(LzT`F#?%LFL!|hrvc-ZXUC_7($P0T>7OwV8}KF;e! zeuKDt%JuHL9b)k()BEez;?mo9lk;y42d{SH-%@rYFA-)R;QPSQ)@Y9(^d|L%*9$dM zNn0n_p<J2DF{aEh5{RJP3bRPty%4kqY6Wo~kL4|kUn=XXc&-y>)`Zxp*5QaHQgj4- z4b8~%7-=4hOw61tDm1gJc@<<$`r(r$V?;C#q&5M~U%H!sghMh&z324G7z-RV9S&=a z`k@m2C{-eB=am&2pJzmtj6K)Zo3ufHopF?rie9~3bWyjxObA_H`I9SD#-WQd47cXl zj|uFos2Y!$p$E7|a5DDp7|?aM;f`q7WG|mOeM)=ss2CAOQbUltI`k8CQq}!W#b`MK z4x?O|UKhIUuahk*UzLGKN_=J>O*{O?>3(tsuN5h@Bwif%s4wDTie$YqfZNXSMpKT6 zNQuQC_^OF3T-4J3*}~JUaUY*WV`S#BMs4X3xHw0Y%7PbWi9fwHCN(j@Zl{;IMh+;T zy4||VHe%qB5>dGENqe~FQKqvT*PCq#$HXU6os8;H*xI78>KAmr`Ij#rtAy+psxK67 z%vXmxQp7QR$sEq=bv6B_=AQGh-5?+)_0{v&houUGMg5cg-hn$D5}Zt#eF_dhJFhsM zuDs|GuBnsAVvV}xB;5XRVyQm9z*&l{z9RKRUBB@DvC2h4_x*a&Me|5TU0w=a>L(4# zwTk0)yij4ao@KF!z2b+&C+dsVw%72bIu<|MZql}&;FRf4a8sBy+cOeD$m#V-r#>H! zDAo<4hZC;RS}{Qs`ihAk%w9|2yohswfH9zHDRg3x+jjTF#L=B45zFk_xYEKi_ASYe zBzk>Q$0J~rdVZKY+!L0#XY#2|X7;A_$@u4j^3Sai`<+;ms~t0SZ>g{)z7%Cc9C*63 z!-=D{pA>)f?_(Ak+oHttO4q-c+@(GXeq);_uzi1*X%PG7#(f=L<rn+Awr=8V7n1jn zQaLMQ%BV0W6I1d$BwTlohqGAQX7}I>aG%rn`<2<N-BeA&T}1{f&XuV|txcs%x0omU z71azBfk~nFAB=#KDwd~)T_)C2Oao=!qqf{qV+Z{5{h3-7{9lAV(2-qXw84uOZPQCU z@oLe*ZTB0(ua-W^E={@uvCUxbz!F`TckJfvG8}qnrNtcveo>cz7Vh^Mj4->US+3K- z4W8JLfbE@_BY`Yl)qV6TE9j2Ea^qgd(h-x@GIU4W05&QL+chpI{BdVf#tN9T4hJ1& zcN>p+hFzt|ELCO|%zh5+f42Oy%HE5~J=eIGN&|kaBoQ&*qeGU;(YLvWwLVjo{3$+E zmSe0V`cpIGlSMPa>US(SZFIz=!(XGw?Y0n7Ch&}28kJ*-z4;`$b+SrI$?V61LxZ*n z9&<2!D>eB8Ep}51Gj_C?X93VP;qVLf`1W$^b!TU%c6jnC&3&QZaYjMfx1EkWj;sN% zPp-cHV#)%cbZF4AiRRH4rE?X@;XXCX9JOL@os~4&(7l&W6Xj0lTh*?m^W+|%C|acT zlVNV#b+f^zG9xH=lXK@2-zOFD-rR3vjJ<N33DVPDK=Jg&MyvFu;GHN+51(k*+}QfG zGZK)<J4y;Q)pgdH`gZ5zx5x$MT7(&0K==*;!-~d~&KqU4HBKpkKKuO)A@I9onw4%F zRXw*I!$;zl#a6!@zHYtAv6c|H%@ymhDHy_KHnmXtFlpTWsoiw$`wUe7W@;f;1y^fu zyMP2K4%vOpw8!so6=AIQ)S7s5sdM7-LO^LW3Hk(|gJX=N`zh3D2dnj)nlETlpZ2(* zUBp_p7B(EY>cUEwsOI@DRSoSXdfC(3YV}cU%2FEAsM|w}Uak3g-2bP&^9*WgYxj6W z5RSqDkq#PZq99-cG}4hKLLidRMMMON7*TqQNJk(P0Tl#9ItfJxfdGc0ROv{BP@)i# zB2_|7xZC49bB<^3JMX7^&&-|sVSn0t)?Rz<*?T?fdH(<3idhs#b|P)sN?zJ<+!}HY zn~HEf^twyJ>4oL9J&uVNXkILkp2(pUJ6j)^kiv9D!kQ<G4~ro5yYJ?n52=)RSKpE( zJL?@cri0K#P#2RcZN#hA8O__yY8`zdzLlAfZ>y=b-$u$HTALa;+2Ye}BKzfqKdSIH zoje5dO(C4d_Nn))NOFdRv{Bbf9IcGHl&71Odx|Em38QFMPMMv0$w8eL*&IZ&<2l$D z)<ERvu_jC|$xFeD*UK*T&QpwUdp^TH?r_|Uao0ceIE9nP*R8gUvbt)VQ?f&{Uja*# z*T!3G#sRTU`Yvd#i<JC;7YHfB0Kpw85B+rmnY4Tar<D1)LOn;O#;fX5*<;xAA)jVY z32$F&XH5+4X`7~#Xp5Fs+-v(yo=Wig_Z|kAOTHbCMUDN<|M1^AQ`PJv$5WerRhweF z*h=hMNh{LG%(ZDn<|AbOAGxOo>S}<jDOApu`lGA$Bp@Vti-f4rx-*MS=&4ujjDwPz zQV0iIRSg9`=*6Z@$<P$>ZC9&9X5gwy*|V)z`xTXt&hP3{U(4Cb6T35Rb>*3HFLY*( z-+xh_@c=qh2lvZ3R@;9EfB5&N*@;d8x^-e%s@chx5l@&m!dPi%1g`KH9xj6NZWMiP zMN|$<nD(oxu8iOs(kx=y5^HYc@kv;c?=P7MF+K&46O#c{bfD7Qh8RwRytTax!d<C& zg`^|=;JD<_-lE+1ZtmBs!hO8EYRsMo6g63}h9d41P#xJNH3vN9A-*1k=eP;ixVeUJ zuO3g~5tlCETRf?GTI>PAof?pzHLt2uu#rL5sv2~gxLlSllwFOluJSK^LHjK^!3moB zrH$>uTwFhf$I`_cB^aqdO+%>LdJB3%Y-e~+IMT2soR2i>pxu*;&L9z9>qgR?{i-9( z72+GjrP14%+Pw84qulLCCd!YJ(N!XITRyKm{p~?(Ltf~FRIq6<%{^J$#M@E3xj8j{ zcqV!%I!UM0&fdUVTu!uZ+y25Vd?2_-4Kfs#8FQzlrf`NBlVKijFOR*FvLe*rGU>!? zu;<b(=c9M!GC-_rMqnWu{ajp2i7ixmZ1kXxGxFp@J7;%VU3IfE2tN9;LCT|z-=rUN zHZXJQ+op@O|NUIrh7$9ZNmTZ1aNPw5_dKP7KGhx-Q(#vR@243WV_`RTK^ISXjSD6( z_+q7(=p^=t$~JkA*vk`FF_(=YdeQzDu0Lrp>2v4H!N^IKxvbEvf<X_tvymhH6dx6{ zq3?u8R2@g@v0Cu+3#WyhR1e1AgNRI@gYva|=G)sFPI4cx*FoMIcwZ%O_uF0f_-+AO zyIIJ}!{6q04}7N0?Si;>L996xi$y@r2~4DIGnEG}K6M^kC>=bxZXvv%L)=l<+}8ha zZkZoSsnqsukZkx!p!2p_RVy;6%BvH_n%Jp)J?;ue@7c2f^Fd}!B~WeG_e_Ppmu2|c z)rDixbLf;5dXlkX+r8~qmlSHDT-99pAtCP&?Exf-3zl7yOs$bcM&Kyirf2gQJF^k^ zC1$@f&VurH$6|X>5}llcLqv#JIP46E`qKiG#?({rHX9K2@hfWT=Y9AEus8{M_fNVN zN|lyJy;fbH$5;#zRJ<^+AjWKE$E<j2IQdw9JKtjxDA(`(2e$-)<yd|_b#NEN`w1X{ z8JqKuMWwy2nf6o8c|E(-oWJo1EIboFt|h_o$-h&ambr-(=v7G|APa6l$Ekzr;5XrF zNS%t_0R4US*JH}7lKnYRzIn2*LXLgkznt9hlDAMbp;kU*_K5`y;SE@zJ9FcZ#a3b_ z`94)56EO;-V_^}OtM0q;TRUJX?+3rJ8-F%Qj;4lZ-Fi*ViZJsU7#Z(y$(XQAgYO9& z8uCaq-=3s(D9~8BuLX8T!yYG1Yz^|^7=Gvk+7Q{)xL8-<ae;B6i`lHeEzu~2@oxbu z(yL~-ylyn?5tfu<ftu)=FR9~-?0hmB?=F0^fF&A+=Q=aW)!r9msq7o*SozjecIEta zj)qP_;kAzAF}^@@3YW;(Q@abgMw(tv*CRQ)qnE+ZI5NC`K-i=3W4miv#i`XB)Xnnh z*518~@BkMwPkmQ<>Vr|e=Xa!@afpuE9#?>C+O>Y<ZmQoGEvAsTwu54#A7&6%37=3a zs<C)C^Nn%KyjpzKpoRHCUF5NHji>b5hRr(&i5T1woA*7eiB{V2vS`;+%iJF+Fp(z% z_W92p25HWb2dbkthhUsGVcWs+G3hr0nveASJ@k6jk(#>0J+pqIwrmhRb-NFs81<}N z?(Nr@^2}s50^ivyE+-_+7#!ts73Hi3EQrCip;>pMdb&F_tUoR>@?OeBE-02%QqqpA zx0RJ<EGKr=B|VHA+XtD%>FQzo%m;|hSO1nX`}#NJDEsM2=o<510a_RAs;VnTTJsNc ze&;U8?Mpt{7#uoXTn(`73ggXm`9S|V_Hg~VNLM_)w*hic2BiKBNE4JSk<I)8{Vfm+ zk(HV74$B-ODfGK2Yd`9X9>m|k2`?cLC3R&gg&JEFgy+32$Tv*zS3QG#BX%i88BIv4 zJSD?a`%<cn^*26bT=CS%InS!`_}68dqOu)LT;NZG6z%<jhn{Lt)31f~#87N$BzQ#% zN%}dF{=^+7Ow#Q)w3Wv7cYHf17)hF98j;^|j%G-TFt*azS+CZ~31j9cjHj|FM~)gj zQenBwV{*1P-1rp|VeM9u=1RO<pLalL{tN!Y%~K0Eh0X{bY_B`t0*?^t_LU%R$-N#? z8l9|Z*ZO4T7H~))!tLb6e$QuqSohKq);HE`;^qNGYM>dbpmR%vNKZ<=mIN_uehQA% z-U_A9?PFPp{S@WZA^haULKnfCL$leBi*LU4-kX&3X(Vps13OoGkw;(dtoMw^>N9Y6 z6u$(Uw%f?tr)}L-!K0su2)F6rHJ+hVQwY?ZlR(KbMto_!@?caLUiVRNKKP+|A-h_T zS*AgDsx0%(r$QU(+}7Zi5>>U+8~(moHkESjN-;wn@1}14KHDsvUXhS8YpBtI-3T$v zG+VM`ChUDnP!UkPgPjWHyy_wqobEhAkQFVi^*2}_8`WV!`&sPQIBV=eVUloKhQ&7d zJKQVhjZux>>lUdNWtTCEbj---VAOO4uXt6Q0M|(!zf;Ez8m{ZJmuTB=Ko}me>gVsw zI}@X=^6t#8ml+*0WOO#ZK}?3TX$Ji+C<;ZW8s=zhJ^r|Wo*TACF+d_o+qZc47|A;{ zeg*Q7lB+AT_KIsOfb+P-k-IU4tbC1+1lZP9yzjN2d|-Rzb+ze%pkq}Y_8dKhKKdr% zbJjHi1H+e&oTtVF<nFVOIy6wP7`CIB?<znbJTHkQ+N8DLH1pnrIFZf*uQ_}%dNJGb zWt3#3#1C5@m2GbpOZf}E@$lX-0{GW;j^saTV3tcIYfSk-x7oQFBFMs@%5&l{d&|rZ z(|!aq_J@I27aEq-_a}jJ*eOnE#PEe<te(|V>YfE4m~>Fs=$F5{c6J!&_}-a0QVygn zF)%mzmLz`>5BSZ2I(^S}I}H;Z24h`uBe-nu2Hpm`U-e-i!G7?j+7MWlf9F&hX?eEm zL$o|Be&p@)Bg11cQF|pWldBx82FuvLoxZOrsqw-?r*Y);g=GA17{<B55)MGB?>+fF zC!|T_<XK(WneCj|p-X2B7Q@+(eu|pn;r;}hlM^Vs7=L?<Yu@hzp22*H(nt`COlWq! zL8DJ4+L3Cv$Tokid|OtHV?Rh${rXFT3H=SSd;xtJ*#2&5oe@f+kNY{<E{A7&7ii_T z=O6HusUQj-vF(4_jFIns*dPYZRJ(`}wR4R}w@yT?@wYBFI6i|8Z~?Q+Vcd+N^eso2 zeC;dk=uXHehejZ>s3}lt;4b@I1z+Fi1ACho<(TZ@E9gW-U1v?I+&t3Es<5y}oh=1Z zY<q|?hpxcv#K5rn`5V!zSTD#;=h~E-B<7~5Ie6m`U>7CEYC~VTR@@|?TT5@`1Kh3E z?E6H`Z;ygKH4S!<CW;)^hTS;%@q)3TnPtI&4pScH(M-P?H;1aqU<Z7(koYxzNz)?r zhlslbM_r?u7AazVLswp~FQH<zT}&3u^^T;JD-a}=BvjRZlzp2FGtj4r?1CWQ)_=qo zKk3q@vu(L1GU5uD&nIqpAjib_vg8Z&E1v}-JpQ4tbo=2E-PpKY(AoK3sf)|86m!nW zrTo8;tY5Zw-XDbhN1M(URS%3zh-YQ2@aTw9n5m)0iLY!Pls`GQS-(R%Ee25C6uE6H zO5R5rLUL(eDWaa4;7}0bDiVn6Z8@zfy9;^`7g$0?plQjEY_@GKFJan+5VHi28(%k* z3#34X=UK4mro-7&4+`8V+psxs>a#1*eP1=1t448?q%E`C0GT2<2})6DqAS*56zcPG zaGVwrU#RyS@K;qsW($DLORU4W?r$^IJigAq+6840RyZdI^+J=rpUfjZ_P6|&ebA;M zK@TG3!pGeJ|ATTc2*J%;=cG)Z77UFUoh9Pey3w>JTy&S#0q360nqq-#Vtv@0Rr$(V z-L$&Aqy6CG^rX!p=AY7^Vr9x`W#UH{>UED|&tE{j;s4X#5sgZFbPt}uWK{t&^JaH} zigczwYMVK+qYJVshCd^2GXuYo7PryM{E8!rr0Ed~AOa8taAH{pJ|JFg>#px?+n}jB zz-fWG#m#lVt3&z*6bek)p5v|3N=eh>6eb{z0h)#}vVo>>q$P3EMl`72cA-2=t8lE< zlb%*<RNdPmO^9c)xX{SCKO<1rHSjN&KK2A|K|xQ@fda(|99vBfA!03(G)cCL1L%Ix z6s`6h$D2&^$sC^q=%L%Z7|%ovrYwN~JX1Y^u?s>mC364TN!yhLsFc4LwO`d>^c+$3 z8r4W<W~E;y0&hr~T(?^pq?cD(_DqW>Z3Xf7|GQospH96ug>h7%G!d_yIaxu$&(F1` zti7KC!#NG?H#qnQf`fzmD^&hn7o<~*aQgJei<@*{uo?hv%4b=5^2F+*8?ZA3Fu0L9 b|7qypuK&<G`~MsNuYDf>_lBB%clf^miYnX@ diff --git a/doc/modules/cassandra/assets/images/Figure_1_data_model.jpg b/doc/modules/cassandra/assets/images/Figure_1_data_model.jpg index a3b330e7a391146bcc8ce6098fa44f566f8a80bd..781f6831cd408680d7ebb931e64894d9037298b4 100644 GIT binary patch literal 35510 zcmeFZcU)6zw=Nn*L@6R2L5Yg=Cejp;EDK#gdK01`(gdVQl#nRBqexL&q&G3r1%XIH zS%8Ri0VO1eNRvb%Oi4S_v+v&L+;6Xa&-Xj`yMNsCThSR!n9TXU;~nK0V?6I<e`GH} z4x1U78bS8#*#j{F|3KJtkW0Tm{V!j8_wLyP{@t_p-(UaZe|vvl<bS%r|LwGYKmC9C zFZ(S-;J_Xej^iAAPD1ty?BNjD!|s7VA&@<s|8{N2|Ml9lmt!C2ey#%txp}}7nh!(v z?&09ryN`pDbKgGj>?rW(kbMH2f+y53>=&}W!F4iN_{_t{#RsG>)_058j8aZ%c!WeB z<Q5eZKO!L`D<`j@sHvr`bM~C>rOSp!#wMm_ws!Uoj!w=lo;SU2dHeYKg@%RSjX>PH z9}^oFpOBc8ocSaxJ16()v%Hehvhs?`s_L4C#-`?$*4J(ActTHaU;m#2gJa_plT*_l zXJ$#{#V<==m%pv7Qh)s1+NSLQzjlAmYY&9uKScI#6Z?mG34rt3yKf)IKCa*M+OszT zJU9gQah_1$FL=S4>qfB9$ukcR2w!|$T;F|ATEm7S;t?{+EqY3mBt!i@wf{b|e{N#Y z|64QrkBR+{dCfw2Ire~y#~}cLL0ByLl4Qu=$KNveTL=Dm9r!Vl0vFZHh}Bh1da*hW zr5Y!pNhhf1F|k{p)$k6xDt#5Z+25bluGX8LT$}xl*>~52XxQ!$r2qP!^xiY&&-7zh z5<`}3NQn%Uz=l9jqHM^|scM)%>=%R~&4x70kk}CBCH+WubGY%7n;((={0_~2PhyJ= z>`KVlZ63S5YqbVwo#FZ;QTM~DZDSYDh8lP&5ud%xba9rdV?+47D$%2Y0C<rHFdp=; zHS7QUm1EYZd;f0TLpB7njRMd=xSp^Y7GcZ1YVByovsEHx$|Zs+pq35=I4G@bNKrHN zKYer0LKt215&DY_aokK`Lykk&R5e&g8+s)hGEnqS55o9=8pEEyhXAg`-%|LkkAF+y zZz=q{Z~9vo{!i2ej0-~ss4ru<k80HgnSRi>CKW$N<n7?|(w~v%qC|<d7G?y?v1Wy` zw6J3lSEJ1}4%OeTnMZ_BO?YoyY}_qGmeXa}kUu-Ao~$8w5_(&NViDyE7fGzBcEp;g zydEzo!FHC+I{y4X=RI%i5GT=KYE~vK7l1vL-<nfwDC{8HZ76(o&%gNK8iX}t7!Oba zY{<-Q3LC;IE;^5u!MZSCupwVKC>g9FZ!lJF^HMkNsYFflpxZoPYin%C*m0zz5W4Fs z8-g`wL(Cpo8;Wf8zvmRT{^KtX5rCiW&W3c~WSwK|1E%RA3@Jc|&TT=RAEAi>_b8R* z%^pnRIO>CxIDmdWhdfIic}Sw-lC%PBFHChv%WU0mj_bgRq@_P$Hp&(s%17VRaB6Pu z%w|KJ=^*^*zK%M{`had$dz<kPF3g5_2Eb`(ir7Xl9kvQ@g08nwPtq54Vo@j7q_inr z3FU-3xFMOy1vnZVT$p}6mbTQ}K8%_^T_!pZeA}Jcn|p}=4E~l-u@#SvZH`?2%+HJE zS_USbiU!VhsUDw7a~BH)115GWPFye@%X53Z{zfckJUsYO+Sxu~KRvrj9dVBnG1aC6 zwpj&1yacXu3a3{31uQ14+vv{>QD=~*Y)Eth>r>YWpq2XOEgMqb$`F$4WkV#99Q5=4 zj#83T?_Ovs@rWj7rec~Lk<_%QN4GTuj*y6AfcEzwi4~6ZpD6+8_inmN#V4eBRPsMF zd4B6vw>gRnHG!!=)JhK?@v1}0QFI8O)ySYUjxf~8JJCN~0r*R#LQ~OB|9zFu!^w(k zrhEnJpPVg@ULCp@gbwPOnqB$Qlvw`kdJT!{RPH)6+D&4$T2G`_pC!32ITe{bPW^o7 zR&KL(@U^XcUx$*fHA_Idebv=$srw)E)_4x!t*cpMiwZCF?u1ysSUl2fX<<s&t(H^N zvV$2`VZXj%LDT5h(FARPNLz&iGGb{<%%nPf7<P@wZ*U5?Ax-%)LYA#BbN+PA*FcP# zo5B(`C-qHeszFvIe$4bnp|7{+`)cP3`e~2)=U5+HVv17H3}JtJ%Je!1MT|B~*<2r> z+LCuN&^{Uak#dtAoOE5}aj`D7$?0CAsJ;$Y;_a{~biXws-~?qlI&Ayi^i`_N_k59# zgGU2s+C_+fyeZw7(&Re7q=h08`-&BNdy?|)+DklK@pov9?ec`0Vor;Jv$JL@y;|GK zm#U2l>||@{DAT?=1?x<LQ9(WboI<;zrl-xJ&`@jBHn(1FRN*K*k#zuE?NZhl<`CKs zIzy-VfFB>A2o4}(UPMjqv{%GY<j4^Z*bpBmH}Wneo0#4-<f_z$l0wYR8B^vlY6;t3 z0mhciV)f?1`*!YU?VbO;&Kivj+Cqc64H{mzVV3~%Y6GQpy_yZ#H{8O+8XOrW@hLfV z0+l)&85G9_C1*s#ru9_awiQ>#KE{P{uddgw76wDcZS?!NnPt|jx)I0RKpvtU=2)Jb zao$BlaiaKs*s9%~I@3`5N(aqF1)D6P6x67pmQ7a4Cx4sJjG9T>55fSx(B4dB`n=hl z)Nd16#oSp-=Zv(Z74X43NAt3E>oaZ2t&O)T+D3$ce!Rhf`sPV}Uo!R(!uSeqvZ-@# zzctmMt-?oCC+~Y(gcPRaTTAQxx(Vw~e$%->dO7PF)bq%`qGum}ii4%XDOYfS;&MM5 zGPkox>_^8MXfhL24E7<+X^4!gl${>66c?@y8B#`i*WnhR*}wMV$61Ez*f#tCz&&J2 zRjrzW&b2x^x>(j%8G5wq`-ap61$<<5vLT!iOl+4DQuobxQ^xEZ>mUayC>>hg8jbVC z#0wx#0vlwjIk~sH_AhbNy}oiW-6*x7n0X;e?@nS^@BYo6(?gYhzA|y|P7fS<F2NIb z2{T<~<}6gH>UWMoc^_x#Og}e@O(~AInyqTLTXUcdt=dtA%D1b}niX`WWIKI(6)vqm zR_OD?ez4eiGWt$vnwitR=~n5?EYczS?PoUjG8efcQfmxtbGHbF$ThkX@?OwP49ykT zrW^N??5L&4!@z~bjO1WItGSfaLp?$9PEZS9`jFn$Zm~nt{UoU_HfxwYl67kS_!CKf zazr9*p#|C^fg0*M$ZSVSf~rzWMK#h-=saNPEMk(OhqXp#tw>V|i^tR~JDpS*H(mf7 zq#k$Z>&XuDh4a#OnLc&l`D<>v79Ns!ak{s1%O6%f5qP>lY3NZlgnJ!5<!kxklh^vn z#l{!a6SdZc_(E;Uc3b5+`AfOP;I9dqq*qP_*G~T|=F~|EO_TrGZTVuQ#`0?AK`q@U z&3@PFQ?Hc|m)y<9m9{JW#LymrCi0B2pIHR+7)c<u6tqYwP&ebdw7;<-hoB2`1Pq_c z#UcZsevv<=UD<^QW}_Uw+CnC4UbR;K${F`R7R(TCgsg~6)c5Zr$^sfJZY<r3p$SY< z#7J0%uo~;&&l*-g>j1Nw;aSNz2dGi~2{<T&k2*q_4eA-b3xyFdhYS>)5TV8mxp<e( z<pE*G$Y<_kw_2|+uEb9>VP+lD&+G=hhk3&#DEPrMsaev=)j{5I;VyPLvXz?W<U{Gs zx>+_ETK5$zf{Ses;<p4$Ni{Y_Ck=kkG6~}r6ECZXv+bH2s`onz9)GEhpx1O@+U%?i z{XB#nfBszH2b7^3L39&8VzbUe+)XGTn^RBC?}s;RZXBU%EMSthExc(@C|rb*ct;oh zhgwQ?N|&bt7#53a{7#?!z=h23)UlB!INY_J^SxVd7!gT^Zl(C&V@)vQ*^q_uE<R>G z8*&q!NNgHrL-4p`jMG$|x46SdQwppf7Jb<DGx7oz7mGU1P##;DLK>2+cnwQ^wWkeq zI$w`(D#IEVlku~w<kfF8l3qpHBYeF52NR1O1uRQxny()9>&RR3EqtxkN{saqQOp|0 z8fo0Ip@l2r3nsi}gc>S1L@eucRr3m6p613j$#56E&fb89>Krt+E$Ga4<kXQ0x3tu8 z2-+G<W+C(gKnqP{wxEO=$5Uv1KqIKK<y6cAoJZJ{=JpiBqK5Tho<D(>M#*@n6(ryS zgkAv>$mj>TMTesUv?@9~n=X}@R~ZHe{^V-IrT<yJ#*74mPq&2@@c~Mm8OxBQ@YBLO zm^I<Bxv3Gd%m{%eK#rUqLunv?Lb$$_5~q&Dy)0VnZRedj<0}z|OIE>LecjesAEk)L zlUF69Y?eI6m#|n;CkBRlU;6>yjvB<arb`PJyxOX~^knX$eP(Xs*Jzt8#R~@a?WMWn z-I8lvG_M!S>56w;@!s%_JaYBzczAubwA;wX=7fr@RT&+bBjcGC_GQbvy@qHtsbg$N zOiY&n8ZQ(;H7D~kgwjD*&xUZbj<6>E=_go|D4iB&8AGHFjjshNL&+wZ>Ngg!yt4?q zRVkROr0b9~w&F*fveeR8-x$kjN#_F#!<xfTtUm|QB7j~>KFxxekM^z$rL{5ZQJM&} z4HcJQaM;)V?1!!+2ygICd|}T5H`G|tNK+t=%%fDLuT>J&GWcE3r07#r^VTmi-J+>t zCQ$ZVjv&JCY)xpkA0n^5*y5s4p;@+fTya|L4Kwv{i6VXhW3eMYLi9D6IGu*nit>eE z^`0sGM5?bn!8Y3^Jv8=)Wr5NJe$cjBLD4@~z${GS>sMMw)5geIBY3s2;}U$`5mW;S zDz4uu10#s`M9)^zZcX8CTFq{fDj#kUll>{IphWj$0CoYM2#TlqyqSt9>=@%b@Q^fQ z?t>*IIJ|C-|319nd3)L{CH#Dk7{wV?;}~$xsE%7a&(>bv!(+)ZIKsxdV8wjwYOQ6t z;-l<J%er&p1)gWklyb^s%$jeXH+k*woLni^{xw$d0xh^u_fhph!`JVHQpulg=7R;u zn{odI=9OywE3~hG8bJS0Lpp%~V44hP8X|?@4%18G#GQ9=W)v91%QdzbV)P;*R;Mip zvaf=$`d!$6b%(!1+W*C9{_$L>JZlQ9Spddth*u-%LwDm5w2quke;R0MkNEL0x=In0 z4VlpBZ~oG}b4`V{1j-kSWkXCJQm<LCeqCa9sxcKW7z#U%L(d9Z8$$l-KucqqHW>x| zjANW2z@Y(rY{-jYvR}|TcpbS~He~--He^;=Ka>rr37o>cbWs`@3c;<jA?s)&J!rkU znxWq{HgUEntxF*UJzzYHj%1GH5I4k%SC@}qXie&;X;8=AUf%Q26in_aa65s2n9|2! zLp~NT+C!NBIX`Inljzj*dbAcmrJ6NOY{A6f7G4si(1C2oL1fqmv=3YeL=hIuOcW0y zf%cjLeW-6sF*~Tb!4kX1P@*iazg9j$HBR2{91!$#=_(ImmSMfr+p{_+9^XY%%%IUM zS!NmNQ)7t~>x`LA(r{62DdRG=zb|NZ1doDJ5|Xrd5*ha8h>Au0JkLZJloQeCPapWM zqSdw>Ki0Q3Z*eMhI#d70>sya;uN>lKAItib`g-z8XFtC3IDX%0V`;?$a%c6wR;_6} z=A~!UtnA*c3xDn>HXE>AnEu8y=te|j$^5lBbH0{3U;i4Hk$}+fAdRxo{$So$dU#Q% zqBDa?P?AztO10jS74>g6@}_rL+0Gc+6nZ&S>@V%Ln5fpWJ>s|;VqfEcIkoCYdlV+k z{iv?<%8cE+)8aLD%xL_rFb(mV#_$ssW92DjuPAmTE^{8v<PL?g1kD?!7uvCzS6T9m zNYFr?nNOS5fD;j^=<9{HmOwvgWbP6vcfGs}O!Z>u=iv+i;Caz0ibE;o3lNEyI!+x) z(Z7sTY!{113h*_jx}?2poh|I#coC5lr1j{H5)adxZo%+omY@!zyLr$Hxx1kA!w4}P zfnW;3g_66TK_bphlQ-WEN!Dg0vmrdVTXbUl-Hb&wLBQ7TRde1lJ~XOCQv7|LU;Rjs zhU?sm(eh8=c{a%FAhZAjOSdD;7Z%^1u%1Ev$>Nryz}=YH9fr)5a>ixg8Ods9EQp^q z2^HxOD)poI$#t{jU~?O!4N?D;NG0*gNo6mM!4H9NT05i7hg^yC{PDO$3^~dZ(#Qjr zFd$~(CJp9vd>uzfSO{QjGC$6zN01U2z1_rjsJk3>OWij|^iStDkJuuMhz<h{4UE98 zr?ZbspL&Nkn66n`$N4NFQb$A*<8EvEw_5Qf<=j}8F<;XtbQ%)QkK?>Fdy9HjpJVXO zQ?roT_@ODUyESsU{zuF?jbbX@?d-hVlL><_6qV1|m2(Rsx7`e_fOw#i>Pq7P;^?=* zX`k%4MCWH*X4ay#fJ-LK<_?u|q!GY|P<Q&+kY-#mO7a>|X*0n%-LCU+%fgseG<l*v zfvlMy_;S<pLs)*m)U;@=R!<x`4sDKJ=z>w~RpdZG(!m49<~2%CpJc|&NDunhR+^dv zvzQI>9q3S5v=T()(*ZuRWStFu6}r!-i)-v<Gh6^ccWoWr_P^<WAmL-(XHT8GwqS9< zX9Yg7FW8t{`A*+sqqe!DK2<51P8)_0-g>!J->n+2&>UChv!j>YC0e{}p^338t!AAP zuk}Tx5Mq{vW|l=#_NT22m<8k27effcW;WG{wS`Cfgc}dsQjINlkKZqDc4pbKTsMm< z-cIRlo2az5!*Zr%CN#yQ`q{=^&hnJ7W@v-zVS$=MQ%q)%T9h5ye(>{shU;spL*2s2 zet#)hH>sZ_SfAQKe*@(xrS(w69GO3vVZ9iX;9{R%PLA9#qA7egeeEb4*4p?`MTHy> zeaD6XXK^fUYq~fvw6HzB9!w&pG{RA%a4wu5F~)@r>4ELhArO;jk9~9<W4=ykK?SZ= zrl{#lN~P3FMyj~<(wtYt1kJx@KfD?41)Uxi8Gm}U-*a_o(krFh(n%LU+FO0nv%|*p zk*H7Mw@O_EN-x43?P^A?^9#K0+n-)ll+)&ZDYn1<k<-2RMY7p`hnbt55{Lq=E9WE5 z4Hgu-jD>sKXu(y=877kvPZ(#J6>LZ#b|#dz4Nm<ss4~U|CsBhq!CRnR@N=@>!;W_F zlieJGI1%)2zDh))f#9{Zv`QZSL@DvH??Ga84To1x-bX83XVOM=-Z|R;Dh0z!wiD>) zn?lwwyB-rtyY3}zKb33gmpFRP>cD5rVcphvGi1SFR<&(E-rnwBuBNwgXqIoAVYjVy zxa{(<<l99DE}_TMX>vA4WKT9DdI#T^`B`zwcMs)^W=Y4G*;@MrmS)BZ-}7QbvSx9O z#=Y1y9OW_&I0fqsibEX(N*7(xGp=BOYasT)lMDm^=~{O%lMr79e7W$$(pFTtWCKyR zm8tP-uE3@-^w#T!w&-)RuS}OGM77PGA&XrihM;<<f-p%P^&WPp>lkYmLvg~)nUn1k z`k)ZRrUhWJ2tA4n1|bJ1Gj}=4<TqRkkVd_w_S-dWXsYo8k%paTW>)iZ@~hk}cq7Kk z8;5x7q|x45!gnqSnNTkY2WbRmFQy5U-+t0=bd{&Yx$PXif9j3@jT?@M8rPQO`_coR z_G=XCh$+6pir=XIVf%9F&+6CLDLY-qCQ}#BjaOURW}H&KmSY`16!mQqe|`0Osg{y$ zwqklyL5<B)NY<jVp{x8<+PUsN5rtueb3=wGf7BEkLNP%Da?=~5RHLCqosCNr#bvT_ zPm~^B!s?-l{ssPCt5g(^Q-?~4pYBv$g>R(^MJ!`$NI^o64LR&LvfNz%8r{s9>0q*G z*GfMQYKjc{y46QLvQaZNh~@+w3#E+rvrbSzZ~H(iDkLZwCdRnk>Mk^~UC0CP9P7(n zWJ9D@bM9!Y&P$Az|6FV}kE5Tsob8!u7j^nak7ZM7N;$W9_(p(YX!xyPcFZ)AVpt|E z({2(z5A!HLXFPY-NK2#k=$W!YiScTe4;@~Xr_+dPO=f;&bJiVg(38tcj&{E=YG0Qb zl2kTChJ(e!4>}|rV44l7><sCEu@2rK2c^05GUL$;tp>`>0!`nJOVl+2rd4jE^#wzE zw65vT<~MVjQXdh<gdM448J44RX^T_a>Zr%(>k1zKnHwR{A4vVPK{#l&FZrjty;gL* zXwjK4#Y-oO%q3n~YNX^3=Jj2ucGrj*Qj;+KtUFLsJA?Z9vgn4mhu?IOgjTVmmUDik zo*AFQVj1O!rHa`Zr+H_KS}lut49*8#lnY`AHJS=A1ghR^pzy{EBp7C+V^HJZs@c?C zBH5=mQ%AZ{XQ-_k*%2hQm?E`isskpJ3bmp3$8vssMj5{hn3kC3wzRym%uD_QYLygV z_;o3U7SpGblb6sR1ZWk}@T>E*_sgWpp0<GR8?xzNXUWp^6G(NQQyTbQuha!n)}u%X zR*8*x>Ah3w@?)kZXWCEJ4mzH=l3J?zhuo3(8a569a&KvJj_TIRV1=GszpkAxUGeZe ziz{KwGzsL-FLB|i*2`Hnb`>k<%<N0g+7_c{%`_}7mV5CYx)*I*=Q!(-OPk_O7nrPd z%+%lHU5PY2fG<!~I4fgjX)nXzMVhc7Spn$otGJEph2Ct413ad76ACE7x?SPXXo}rt z0;Ywy!MBK=;~{aayHbs5L1Qc)#2Z=~Gp)mSG3WsBW&xeiaqfjhlvgFv1V`DQfUdqo z8L@2@6=dc~-s9WY7#azDGE%A4fthZWR%~z3tu0k>aM+Wb9e>2xiqquO;VA$3BIET} zfVfeIpW{&d+1rNa>?##(0~7cb&W4mJ+NL>~nbq;iT)TYDuY0W8{qW8tmu^p^0WwfW zNYN%vIZ9&7TiinNv3;o7OqsW3S+Tr=ZMBVXUZFzYo!G*XRT-;amEfv6fF+89Br2LV zJPq?9&ZsTo7{YK`8#RG|jh?NHR>k>3XZ@Xl$7F0Y#ivVpy3T@#SJCLji0gBftl0vC zv*ZgkmnttDZb+6)$uEz1Txhk`cqLt)WSA~KCO@s`nI<9-Vt%aZ`JS9pObdg~LE*(q zeDQ<}z85SvFBjA}RJ&NJo67g+m73}(=!IH_7L@CJohqpH2tJ#n{W*e1cFs@p+gPda z7na<Rsot!*<5H~Wfj%(pQT&rzzU0AQ9p{5+U}5rcB)OlNgA-&!R+Sd=c3Zk^n3-Th z!5sk04_??F+-@=;+f|1hRL5;T#qH9;WXH^V1yI}3mt)=s#ZvaXBw6X{%3odo|Lu1M z)_>L7xNp3XU7?C4sl~ZOy05x~d_wwBcJn(=(%kf}Ca~m*P~-<U!q3a|<gQz6NStdB z8*+)*3uUM_TT!shq-XZ&huY&*!#CVJ2BMyBH5%w8E?PRZ9}2$*ZFC7~rImlZpC+<I zq|}1`@LMEM3SMHWnk9_J*MU@p;!Sb}OU$Jba2$)gV$9^gGWgJsf3n^ufXNIR;9x^E zCVa4}|9IC@mPi1O$&VIBQ`#qC_zYh#L3<v~hCIan;P<}7LgN^F*pO$dnvS{ua2IPZ zQEnl!5^%{dz|9XsGH&WYf6!$^-cJAGY6Ni`jL3qpA@8;&l!kC5hdDOn!wwY;<2-e? z4Eg^wh*=NVw`1r9e@4vA2eA3M2~tDQ|I}QqH`Jk9hj0`v)`j(N=#ib@#E+lnjgo(C z0`@Iz$kurVI8OYU9XMFnKaPU-G-H<w4oH9$4q6y&nGTK}`m3na@LmK%aSpvJ#RA7k z3a>%OLH|+U{~_N;k-E^IOfY-nXN*{vbiDZoVg9EvoUf2(J)CCk$NhP}kpCA8^ut^% z@2XLEw|^AoKa4<XT!0PvV-2iP!oNuUy`q1w=-)Q_4;J!&qK$5QujjHMCwF=?=qe}~ zu#D!|t+gG1d`OpANhw3<k>iV6+jB}@A$_SQQ###gZWmc_pd5Kv$0|Z+SSfg7cY1pJ z*WEL0h-<H#Gac*~uUuiu9rTLDmwnGZ>redryX<!E!@pJB;QXv97{wYl!$CVq4I=RG zL5)ELI_ybS5{z5pNY58YO$s)~U_T=Yn8P9}%>h)H`l94V&5?%~9)_tZhBJ{Wn>;4j z+d7{cM;1I~s~NH+B7b2(!9KuH=S5y>#e~B~)kRz10P#Vq%d>ZpjfI`FcSE91mdoQ+ zI<uGMyW#iD)Loh|pM)KUaSb~gXu39QR%#Cpct$_Na2Pd^2W*X~qQxYB!F;`AXhM-J zB`Bq-JZT$x5cv5Pb!JR+BjT;Qe7wTOmnW7}VRMqb$>eiLjOyiEyJpSRmgNc|_LYjZ zO<uloV!noEs_);ArP!p*^j0+aB)%82z?=E%NLCcp2cB^WPkoZ&T9J1jL=6ku8Odl~ z1`JFv_A}(CyG|hF41r>*m^oFub%8&@1)HpC0qd929_<hjiDjLh?2se5)=qR$59OC% zs`15Ejri`XpLto#b4}CZvaC%=W~%*?s=v&erPN9#6{51V!cdc^!g(RjtkMH*@6AqI zDzvOWDJVUl)TyR0csiMGljZSp*7%;`9X3SB-~<p#k|W@bBCV+prbglnq!705R7_%j ze1`@ZdIX65)^&6|Fx<_=r%Q-Zlz*=p$gri$yy&vXj;rgP?8*0xm%9u3hVbl@kW0}Q z1G7u0??#m`>nK$&J`5-33`^RIa>TpT<)qZE-0>IPB=d3fOx<mrm%yd>k@zOdnVp@J z1&LHQL*e>%<<IgCRkK$KKXfz7L=3IF2R1~sjB3+%XrZn6b=GHzyF1jQjFcD5{EEuK zR^-b%8Kv8+7ugU&-DIUCAegE|BKAS|AtK1$_@H9|j2uM3h<$9SBw0xzIC}D6DU4%e zw{wAVP5w--o;=Q_-XiQR)w-7_&s@+<JoRu~FYDp(9MYd+NbKixq}`8JI|5D2(;-+W zk%t+Zl%0Nc#6f+T7XaAJZH~Rlu+Akc5?bBm18lE=$!BkMS53}nRNV$=i<g9MsD0PV zg+;fvmqvNnEBJ6lRmVD#WufVS>*v%{^GS~-nwAVbJZ;Vpn)g@a-SK-tvdp(}(Xt6w z!1L*u*(N&%NHrS@+ldNW+immEEfn!|J6!Ic_2aj{2W@))Sjrt}YK;4?>h+`8=$<w~ zvasq!f1rrzaXY(e?yH3^jdm;JmNr>=)~Pmcvho{DW$w-=`3=U(+1#nLJXCI$W~6S~ zD*HLp)Nk(ospi{lhDbRySQ%AgCT<aDVn}iukHe|$0`&8YNGgAvzBC2io5P0icPY*T z7;;21BZzFCaS-8(XYm5&UJJhZH(nx=H?K8z7}h8J;a|1TP(<@7ZHRY`rB^73IxS{j zipv%>6zn~9hBKUZ-=lZQRD%P9?X?P|2gj4=!)nIEOKq}>m9?BdR28^{t>I4>d>hvK zZXc^SUa&Gap6YkeLgs91oZ^JPEn%wEL95zh#i&!k$jrL-VxV=LUx-EZSV8gGzB1R? zf}2kQRYyP|u5q;_SswCF9vn;1*%8(sumfQExdj1A;p&Lgbmnv&>M!~YU2Ac;Pxa7k zu-vqA*gz=>N0diTmxEmp6EMZt^<3VY%m7hdbP>AS3LR(#$VCFn<lE3!7y>a7*HD?j zFkV1&1Kx|;#dK3&sbf@V;&#Rn-`y||9&CJ$fjDkL^C^j{KU<#pC+YFxE7u(P%qnOZ zx_+dWf0hj)V`7o^wWyD<MzvVjEi0iJ{~QMv9|i2Zu{3<(M>h1WNN>{4DCMUGZaREe z512M;%eZ>)xaio4yLN){-EWt=$=?=pWSznduATa<n;y1ol~WVPo&IDw6@9<jL*AZ- zhBj176m+DStn`u;7ji_V@3d&0v;9)dT`+3;)BICbop|%WIX!v1=5t!wB`kbNhwI`_ z&q|jlLjo9Gg!8-k=on}Lk(3CsT2B<urO74wWk(&9Af=-I#Y)4fi+$aen4^S}pTpsP z^<)!sr8B2S+jtE?w-XI#O~4N00{A}~(;}G_EX9=~fzb{$)s--Re6qv10wH8M*@QVb z`)LKGP1Q+Id15@iJse^4rscs$AKDAD|62T+H?O>Gs@lh}md(gOsr<l-z2bh^dLiWH zlsHw#$)DG2tJNBPh~#m_rgEcn!KSTv?eG->PFqS?(XRblTd-yUeD=+H6cr?_RowWO zlGSdF)cMdFoD4&2eOrF4Hfpux9^Ik68mPW@TxMfk(DXvbQ;4Yv;WXrn<JTRA$PR4) zD5Z0-y5XWYKmHgOorlevi3XfgT}OP|4W3b<X46{OgH~LrU8ew7yhN>^L_~tVY1!vt zMc=&1xwoEOiKkX1+y8u68h9m8JD!72-{Az#)ADM{l4C>x$x@L>E(n^@D-^$`a?J~I zqpEzvPeCif%csvKkoEZPPY2kQZ!cZ(F7!2bK1yZcwqs*c+cffAvEQ_Zwl>Z8UXAbF z>G9s%s<O9CE@%EJ;QTqdYDj%wQ6<)FOKG%7s`K`FbHyw<d9(Lxc`Cg#yQS%APL8EU zoqBX#YU*Otv9c#F_qFbThADjsEact9{Sxa^M$h<D)mSB62eGyOOKLM}EW^zdD9wcd z=HgY@X#dq&MyVw0R|?q0YldxApfJ&FNGKjXU4z~=12gKj^hdwV8@p`9;&vp%(&t4$ zO&Ugb%V=fH43WGafV?%U6dKr3ijE`PSU@MCMZ;myMIs$z%O6+=W$U9t7e7mrFe#Pw zZ(5t)J$l=I3cu8!HSQYoHRD@DhfXi1F{nG}C?f*6M85)5c4KBSl3g5tKiN2jC5)J3 z395-Yqw!#rNy(X&!Stx;u>?Bx+!9QtVNEKGbERs9)lTkKzBK)hL$}Aw++S_Zcv74y z<s=YZsWwYXA!w(e;MZqQl@;7x^|-O2omsvi!?htS^tFF9M@hAZbmD+zx!>!9Sf8)Y zbHp^?F8Y~K{I0F$NCYNN31Zh5ts~{Mar+oil$b@G^&1p17Pm7!h&6>fqQyV+b8(s> z%tgkoFOfcWfH-a+^8xY-RgB=%bc2L>2<J99hCEKS>a&m|INWDL`e7nz!Rz5@5u(#K zhx@m8pLaF3wX#N9I^#ix-yE51Lx2*=;@%+>KaTXQ(LDf-g^YCcLg6daIGR$03&@zw zy~wQBM;9|}!Jt?NijyJhegLfgfp^f65~O@BFZP$n(oK(_opT+kUcpB36`anGKod7- z#b|c{CAtGcn(F-)Gb==5Wl(iUR%ylc89fF<Q@18wlokajSvOB<rkPt#Zt1PLT_Wr* z<<xn4hkI4|?%ZviiqNhf2?!RhdSMXEEW#~3CDt3SH&a8@GKl*Pg#Fuf`bd;D7r?3t z=}7hN3rfcbGs4L*h(GK2sJ>Mc2u#HN*$@!}8Gjp~@9cI4-?soygQ~#qVXrm6G>m<1 ztp9xb?vQ^!UwYa#dl7vR<?~lgDfG6f)s7GNEu0qqQt_)?SzGHv?>W0#hkUzYheB}= zlJ4Yguy$>1k4<uT@Y!d5jn%TH@{T!P@x6{!1z@q8lI0awX({~ey<*wQ`wq&uoiFP= zy8ArN8<vIxiw8ZhIcai#9~zKP*!Wcp*wVs4y?w=+KnVgDC~%TkKUmaAVC`$AOcR2V zboVy~kf4e8W7}^Pzd&9D?y1zetT=RuKLJ^FwCpmT`~a9mGWm!9V9*AUehlF02XlWn zuyU^TvW^--Yoq`Gkz!0+r>eHIIGL|tKDdNw1HmqeP7I5$yV4i7f0>fe6R87ek>>s^ z&$RG2r?ibLH@C)lO<sS4m`vD`r0CEB0X4ezEb>Y{pt3LmTv9=)BR#0g{lvskG~SSm z69C_-{^Wqqbvx<$&bZ#Z4Z1Hhu)VPb{mMzw?abWJoVRyi`NJPzGo3)=VP@k3x+(MM zSv^woPV)9h%<R`{vRYh`76sE|l|(#%l16aYP2YY#-u#t<>&b|jKl35{v*IADZ)!?! z&S`s(f0**Zz2bW+3kb?{QatkP%4iyVy+B0&yqBD}?)Dj{rPqVh{M03_?Xvo?NkcJf z>vFDj$xeOhL>Er=giVh*-_h~YwJD|Y7fkVS`rn-^`1uu)6Qo0aJgeaarzW96E-z5^ zLqm>>L_l$8ovw-jt!f^?k%RW3MxiZoc#JR$lyJjxA5;>lK}F-yv#pDc5A^-XSOI{G zr1iN!L5n|4O8Y~HvlNz^NmA|5^49;cHr*QPc5UA6^EGps;df6Qs~Kl0r*KVd2t%-u zh8j1JWf%jWeQOc@w95c=F-RDB0f?Zsk`wT3NQ&-q(n-LAgcF4O>E`yr_aI6L@HbJ$ z_CMhc{8gtXZB`Lj*67i(TChTO47QL5tMA{Cqj-`k*MCv-LDkPd+Jn7c)rS$Jp#6YH zKP`n>j+6{xL2+ZsTY5~TK9mAQr*}ig2`Kb8cy=S;D4}OI{-u}Ky)qTehCf+xba_+{ zx-n?oc%d$cL|p&802&ft+?~gqDhx-LAl@uO`*5kFe`uyDXu6#t7`*}JsQf)3M|kwa zxU|y~tMQ8F`lGlRMxXata&=S?XYXs>?4y;1BRsbJ_Ah-EUn3Svb+@xOEz5)IMhiL% zR%e}bO$U*KcI^uGVZr)J$#F8nPx_A>y&_z9#I|Q>k8YUnuwnb75=wYUp^5Fdq<x?I zQgH1rF))>yV+|J_rUWsBjA`e<mT?8rgju|e6GVU|G^57S)gmrx#D+ArvJbO&OZ>Zm z^qSx>>>-K+c+%JEE=W04d$bGMYVtiS?c{s1a7S0#P5Z7lLV>7v^SspDbrAe<z60v% zdMogzgbiORd|?^nE9URQ`bU5ZFrT1VS8{*9+94mPkiH?^!cR&BH-(fg0q8vu4k*zi zO}>-!Q}aqnh`0_R3tpKUkNV*y=tdYrNR6h-OlKWm1JfAvt<AWulfaD^h&ZI(6NJA7 zFiIVF1guN2Y0Z&t5xvsg#C=Fps^U)fJePl`%iz4ttWn%e3g_uLP3`qjD9<a<e}I&% zZmqD9lt+w+2H-r{-b=X(cX6*wCDMkOt*UA_lL<XAO16tngtiW>$#rHPaY2jmUh`Z4 z(O54}w*GXZQPh0^AiqxGDX=SFztfO@y=Kg+a5P;vUOVf~R$#HjM3qCYZEy9^>i2M; zQ`Hxm9Sj_V)DTh<Jzws;q(044Qi<ceytw|^`$6yZT5}<1BYStLmnYLjUrN}m3<;K6 z32CpsR~W~1OY|q8<XNBLQn-aWmNIo`V~B3WdUue{iM*IVg08PDWX$X=1|@d!AS6i+ zxg_jEDSn_I+K)+<5*)p(jgxS7r%E-}JUc<PS#?pPi9U|3bai<Va&Lu%;m7${&3e;* z{H|W_6BtKHBazJK2K!csEjsIbbXQ=9mi2)JL0+C<@$2+#MBt?)0rki-#3Oz_lL3<J z&=9t1RX4D<tS<dcsWVWq$|qz!#`ozXv!4lGZy_80%6r3$DrXP0QHK_879FQ<CNcC% zfJ^<et#MuQjGKVHC)E@0eyre{kGnW9N*aj;`;bi*8jIR-J_EvcToENB4rwaS&5s-O z7`#WtbTXgdyz@6x;ConSG4c1^5_&K*8E=bp056KP!-s8{03yN=2ws3Dw!;&etV990 z3B~B)3s)b<`!{!u<Md%cd*^*yRVsTcYF37QBrI4CR-Cbfg_T?I4HQ03!d}|ztY!1j z+Oyr$eFom*ZC?@Z?(!c=DP3RW_UBvwe8tX(H1_SibaJny?NBY#y|ngy#Riwenc<Mz zH}4GPXk<DmOYt0w#ZTH=hV;~QJiRwvCyxSIN!`cvt*9{bDfB`qy3shDrGaz?)<|kI zVzgDNF&-A<t_iFyDzPC+Fi~Zegg=z0OM+q8K*1)Xr2UVtN@=86P9a@ud%eP|2F`!F zi#^t8`GLS2%<5!Z0Q)gqeQXF#9s$;+uCzA4ManV@x_Cj>t5x5o1VaTh?2{m!sT3uv z(+5qR#V}48D1yk6bOuOeNq%6QoIgp2CN>YfLGybP<|Ri1SOevTo}R3@ugu($Oy`?4 zm5cas`&V6Ko8+JC;Yxb>JRkK3Jbu=xorsKl+VRoN{rrbN7emgx`ka0J4=qAsaa!LU z7IaSvR^nIf;H%eH3~v!Z6s_NtBPrlEU<f9niE(Bx%bwTR+0dRuS1VlJqS*V8G@Lya zxp+^7aszpHl1=fk)NG}(O7YJ|s={94@`gVy^k)iwF~z*yWC-Xv0o4oVBUuNQsTnM8 zF%mzo+<G>7B!&$U01*Np>)u#_c(gjt59rZNSEV$kwDyCfEj0hU4;yl8+&~>^G^(O+ zQI62LYW@q|Sf_80qYw}jbx;2lDvtF5bkDfko3ooNZWYjL@quC83e0o64Lb}MFdwM~ zhZ&~<A=1c4Nm?_&M_2j?aP)v`iV~hd8g>BVRE!bujJk~1&FBm8w4M&Xa401p3JgnT z=jHxrZ^-L&9SBTA|ItdMSVBL(v0-L(Xwdf&k-{x)%q)gTP;oaH9O5$(%j6(hLR>jz zBpFT8q|mCtAe0o-7t`BroT%Jk>9RU!8{?L7HTKQr+>4^>ML{&sP@=D)2q43UKYa_6 zj27z>Sj$BVZQ=IyQ@{nYdeEi82m^Ge9KFOtr2-E@aBrXqIyj|^6J7Ga&BR>o`IBuE z(`qO79r03IzZHHWt<OWzHnlh*4f)>gRK<#e|1oEemBrdY?ud6)Q$<SrBSv;BFU}*+ zo7q%4avYeZ{<y7m<&D}~((?L>W4xGZx%=UaW<sBngkS6{gwNzs=FqB5yp*3-+v@u| z32Kk-L?icV?owqBEEU>nygoo_WF1rkMa{u{sqaQ6a(qUZZ4!u$Y7F5Wl2&C;l+^M9 zj&TyKjj7$B@W-Hi>(B&uE~+6hrkE<1z)~<=h%(!B7|RQ4^er;zYDgyCR(WU51dU)H zhML=(G3&8RYhreGgj0FQQni#)+H+vzg%7$x2jA)4J1^Y{gf75l5*GA#6YxY%;AeNy z!H!PWdkU>}aX8eJ3QcNO8ZtNeI-!DXYRM!c#<c4|%qh42n3_NiTEF)kR1965$0n$F zdl)<vG4gUOd?5%_h+_!CH4ErPW*%z)tSM>Ebq*U3<5KQ8f-;Z;zHbubdLDf%aWghf zFr89<(X98TdZ_6UD+ehLcrJ`Y<9H2@fqSRIOXmReTNsbN5DA^o#q(jqTDo<tLv4X6 zI`qefbx*si{JE7*uGMFUM3nEzzSOFI|1v!FYU&MFq2$V#=!)u)t9|waGWBlph>ods z&sA{4RPBguQIYpEQdXgau$PL+!7==ok5<0tCgOPYK&~Jyz3f#mHi1BQDR^b?ooH`I zJ$$EpKpCsj6REk@hyydxtkr+EFZ*BgU#B;`e{ac-MbljMVPMYco#8*=|NXTqa|rC9 z!QrgO9OT#e%pVEN`|w@6=wx|pg~1&bm~|~I!+#kjB!R`IH%M-<)ZSpm>;$S8T{q*2 z99`0k%VS%#$!6FdqyiP4rn#XaeJ|gpi-L;{=L;F@_DGG-8V>e68fo|C*VCiZi^~k* zO4dOgx+}WdPzxpr=z%@R=%*u;H#AqOYUcu02!W=7emWio+*|ZP%G4or%*IR8w~&r8 zWcMWOS{FJ=rCG&l=-FbjbWK!Nzw&7#t8ei2halIy2spO@lK{+VubK538*R0S6o;e^ z7#VctGe1c?Z_nzhkgV2ywuXoYEl8Vt{B@>Sr=lYt>u-jKt;vh+=PQS3{oHyrlXeZ0 za>uUiM*%7{rN@KPTt-s6xw>@D!Oe^{SZ-7^j*YH`TZPWZYuU~Y&0xKS967Co@RQd_ z&#n%xycbUnwwHcaLv~Q;+dg4Qy=oadJ3TuPqA=EJ&v=LGMN=$-X2xkX;Eb&pR^WRE z>O(H>aMvl;98uB$iW<iq*+lz%)+2-Yl^cH_ihCL<r`Ch!^SRF-_v9j#v!yVo;z9-D zx+~n--FN5FfK+LC`Rk}NJbM0Uu-FD1hcQZ|X`m(7EA_#5o00e6T8IeN0W@_SRA2)| z)_WY-bY7@Li=%kKijD9#zZCHk>Yc7mw_1GHajJLRI_o$gY<47;p+u#_C%#N+@*g<A z(Z5-@MORV?ygeUzufl)mO`)aLB<3SO%@SNA3Fadf4{H!d!JzS=+m5?b;;##1$YCfT zEekIMQh@nsxa93kA!aE{01cRIqt`KN^q@ayn5Ch(^{&7r;!<UKj%}sc%OIC{S7yS4 zz8gR8hn+T`aY=oZqT!UtBHLD~nvhRVOx~Tma^;4^6Eoy!f|S35Q_^)$TZy07EO_00 z9V%qUu1y51aZ~D>4G;DW7CL-0G8j6vG+=3y_KbE+*xIfCu(hEu<X>?(DlKAv3F>>X zAs*O7^w#J-H~Ww-T`=xT_rYn3Kmr?LA(p+gjQJt+9=0w9np8za#1j06<`8=QWCp7s ziQ$CqR)#HP66=W@V9j=)C-i97X+%)(Ze@SmD9i^AIKoAifpV(o6g)-{2?LGkyS}s- zfynEEC{hu+@PRbF5vb+5c5A#`Y-hH2qj&s`KFoU+Bwax>FyP&MtpLqABd`zS?SiA~ zWJs)$a-*9wguxA867lt9Cj&{=l#B6$rl?St3DtpI>#537<oU|{H_zObUbi`W=NM-i zd;WIXQJ`L;S<|>?Z-!t7iIqaBWpUr2O4IHzi@Hv)!6cCC)W`)x%v)?cT7X3?$0W|~ z#50Wf3Gn3m@x>L01X6Yd_W8%tUCq&7zjJj?1gTOy+KYzS5Y&Gk;i+i?I)zBNq6C5j zC?1w5hS}F8^{w5C;3iFA8B8!__<K4%I4S%YnpRLh-Is08=YR~t=$gy-=33v}AM+!} zmI{2s>&%0XatZ`wh~9Y6>p1w+ghY9n{EpJ0yYeBVQcM@{(*4*Gdi6<3;r;OHu6q_9 zN}t<5q?rXL5##!=RzAJ%mDoeS|N3ZP$TEB<lZLq8T6&3Vv)3+H#e|rn9_ABys@n2u ztt?87g@+4i;sB)thA@vIv{eekH*rNrm{}+>1Q$7X-GaU!d4<yaHLetC@<Hpo?EDyX z7Tt%QyJO7XQ|KpuVmz?!g=e(6l2fz4cSF(F_d_<3o9oa@Ge#w|8La<k!<t1l*WXZG zNfCW1n3+~u12a<#4)W`9w6sx_lr!-4>raAi_|v9oB-bMUzG3pPG)oY6V3pT9yeZc~ z0JL|Xc|r?$o<&mtY1d7EZi|O~HaO<-W8~^2xxOjG4mbWncC3lMVW_g`x;Au7b<w)x ziA7`17c;7B=jU;JdVO=xdWFL-w?nFWSiw0(n?8^Hs%EE;lhqqHG6HZuwf2?r);;4d zN>(PGmx7w@>2%)?DN(dV$@bXfetJbSs)%S4c4K`hI%6hV+yZ0?XMO4{!RjOi32i7& zcNz-Ya!i6GBrzX`IyK~Sym>;#A}smX@z|b*_X8vt=NW>KmE!ToiQMI%g_L#9q&{Z6 z1Oxd^+yWU2D2lB+R;_jM0KT9yc2TF-M@YdTn!{?e9A^3YbD)}32j7SgboqlNkiRh1 z8nNKYFAyKemCTbN24q!t*uNQj62ir*#{FZ(wsQNthQ(CV+J?0A_>1rWIosx<ew(pv zJ@ZJW-qnV-w34>Qu7>tB@|)&NSueyF2V)1tx6y_u>ohh5*~^AZ&ti6MVOHJfHWF-Y z62_E)_Fx#2)gWpDA#L<i);R7E%wH`LF50yZH3seiii%>XA&kWJtwF4Zs|Vd7h8wBS zg26{PS}ic4Z`lx0))~a1IpgS6XTB*-K1J6vC#^$IKeJcb)dL;tEVC82!iR1Kw~C+2 zWe9=Hty0Ub*C5=LLA@h3ERFOiVP*&m-UO?M^$h!Iu=FOx_+yMEig2}{SS55Al8>Dk z^XIX+6~ybKexcIY59amynODx#)c%5&U+>a6_>0$`uE0*iNG-U4&D~4z-_9uMcj<Kk zWKNBjvLyX!^B_G;i!}`fl}tFr!itrq%Y>CY`1^PMe=38QBP1fY<IxR)>vtET;8I}= z+303|Jcj4@V~EOsmv1_3scW=-(~NTqQL5j=lQ(<Xr;;dOHrp_{-DzvtoRsH$eXMDS zz9PasH#_)6fyw*V#Kiv!p#BR1>HilGq*BN<J~}$UJ6cw6mT|P+B$+xOS?!p9`TC_| zn+y96U+AtJH7y&ra62y%si`^ZTY0PH7Bm5=M#6Hf1p9#f!tZgU0MUc8d3;bpyG49b zIYH}Pylaxj_|_1jB;8czmiP4sje<^3h6Ak)4dzYNG9AoERm2{>`L*`*KCbiMcb@&d zm+i0ixBc&acc1u|^q{|dI!Kl$ny8N6rd6`CM#d@d?+B~iZ4jNC{^kIwkNxkC;Q#6O zkfn5I@F^u+Bq%dygdgPZ9$B3I;j)eWtET?vJFWB_X;v4Gd6KI)ScksEzq^Ioss=Y3 z!ml7nzySRmy4#I$j{1bfji!Uc?$+w?q|mQW#1dJ2X+~CYD4mqv20vr+8${sTl?kMM z*{5$hE?Tcgwf@u$aMxQyJa6j|a``qJo3ZpuQF0!36FxWO^m~trKkj#)-Cu>&9!E-h z(uEL<q}*6doY)MIv*!2)h!g8Mrqd^p+BzGbW{|NLan~FU7@agU`L5bSynRbW>TMl^ z0N@rP(iq0{P;gWMFn`>``^a2a(+ER|PAg+R#rdM+9?vYP={qhkxxhAoi0xt!el9n8 z-Z@Y+3Qg+K&B*r`+<epaVSVAqA$!~f+(H6OiY3MRKy2w}h+=7}6s|<iX0Xek2WRFQ zob;Jx5+6(YxHwcUz)~D{0%~MFsLv67pQubYJ|5&YG{`?=`lzA{9<OoqmfW69D~;L> zs^(eeTZn<ktVNfXs?LYxI8sBHpLHF}Q-)6)44fVN+HHrmQ7XzR-CVvGps(CGbHme* zU%i|3YB<#V<A7b9{w>LD1vB%RDZHa?u4d(SrDgW#BUK<5Bh=sJd{=2e=}4{bE5Duk zj+Xn^rfKzlI>DC@X1e-|Wtr>y?NR=-j5ow@E3;R0MB9n{^O$GGyKsvi+IRc%6F)6; zK|+Gz`2N(cy{um7QE*9Se=g>B^M4$oZ8KZ-ojDWwHk0lWDfSsDC~b-uq4VO;e)J*D z4S$<g)SYNzFI<wLSB;1vO($TRzL6sX)bPGqpLCB-1(biDpD{iPH)D;ut5U5tEWnTt z62_ws`=ce07UUoyL<TtnEM}hsT>ITsV|wzwrYu?na$?9q(W>F4i^rr`e3pdX&=qa< zA=%o1aK7DnJ$GaM2=PbvUtbAFUXdTq5wYoa=((9?ceF;);eEXz!4`AgOmhuIzLWoC z7uQ(FdBmoA(@gW%npaj8Ia@C238CG2(lXGbJ$K;rR_#=Rvf^N7xc&X$tJ&cWsrmNR zvbj36y;>W*eVJ+AX6okEEh$H1tQ*w_UMOeq_{x)S8&t@qUp~$8HU8q8GY9H)ybQhX z+FiDJ|NUZ3O|<i~BzzSl(~3ic6^W5zLvCPZa=RU%lsl*-5_<2Ngh3LWA4sA3QnBl` zrLIV0ATKN<rb`(pgkbumPJU==Ltg*jmp8ekYw+auv`az6w1A86tGVt|!F|tB8R%|X za1V(D!-|^A5Y(XQ00|`iS*t~zw;AamJ6jW%0d2zK;WL@@uDDcqQ}g>Y%8w!IaeW?c zLvo==GnD^A&A0Z7@|S<Kr#13em}1GE_io&D%*+W}we9XEO{jkazDz0@zPHS6cz^or zC%gu@rD6H3zBW{)&$8)+S!s3^KC8wuz>s|A_0@hyg5p@UF%R!pxQw`QAsS@P-)a@V zXzzJ261{Qz*x7Rv6{U`Ip|b_jHt6ZTXEiJLCrjYpcC+K_Jf%$@&#&a>?9kGw=hE%W z^PP_!$?kU8({SMU!sIUNGRy<J2a&q~J^<z?Epi%n9mIli;0GbF<@2~p2?6gf;$qe# zx#$-Gjh?)S?zuXbeuB;}22V_qIB7-I=~Z8mI@vn);(62S_Sdgl?9#njn$pls+hXXo zuFe-K_Q~B>d#l<5%WB38PQ4)MY8`dMdMkz>UzsYm&C&l*RVv?Crg+idmh2OP$115H z+fdx5`iSA@v-W8lO1=_V%SXnm_bqvg$jBY+ZAg}{;{~f#&-eZd4}RCb+)I)5pYn=Y z{~xJA^sj2HE)YGBJ=;a`Fh>ruA?{#mup$p)m9M%N|AoW)_bKvcq5tBsknYWY6;|jk zMcI&$9`rN_=ga;8Udn<QO))saTvqmRu&TI{Q2^7$d7(e<htAZ}YM5O(-$+cdI=~{* zmVWP$5gu7M*a0T#38Wr_y<;72<wIpdWHD-diGJ|0ABgG{ldui>Ka`WjD+Ry8z==mk zg3ey5O9Fg{b@<am=K)8u5-?4&+Q_c^_R|-`4Sh?%S5q?Z5;t)|I)E&(_b{Rl0Q!1X zKoly|6dhc-!XLChLtOp+kDQF{n5)#~hfVndTN{pjDB*D}o?CWpuO{>jOK0X?-<15B za=vbu+;PP-?D~-liVNunUX~v_bvaYVS^Ba4=xB^|DP-{5e{1hMqnb?Hb!~_U7*Kl4 zC?FkFs>qO0Mi3AfdQ&n8NEc}mBqU1j83DneMZ^%P5u^wL5kg=THA+=bQV^9U321yJ z#r=HWI%l2z<NMA!YoGmZ&o8pbS}Awl_b%6U-PiN<@>7qrJ&g2o9vgg#%afPyQ$|pJ z$D`tM26>$1grKZ~%MU)pac{T|1<w5J9MN*hyOP_P%!|0F@Wv{rI@#8T@1zpqRh(a@ z1&aytGSsoK2q#$kJRPez){dy6i(&^QvEfG9^xf!Ko(f7`xECxHfl_uqCf9_Nt7MkW z7IhHRV5CX7FHTx4mEKkYSTD!IiIYg5DJzpLE-Bf%DI_HFE~RO9{@v@$`;JdOAnfL_ zg9ks)aDN7|yoz~hj_)`pzaKM*?7)H-#<ih;NoQ=%lBg$KvaB^YWsgD`Gwe#fN4Zd{ z;d2YxTeh#_*5ubf<)y4xq6Pj*Hn*$~$ndhM_)I{*$~OR#evG61n7-E<9@f6Zj#RUN zaIAjVV8H(Ik5irE(RvQqzj5Sq)>~Ti7kehx4Gb8*n(q4Fjt*b1o^l$m%^vJT)&z{p zUN-G-iFnXt?rqqj`&!YtCHzHvLZs8fh9`~X?;9I!Umf;oc0SZ?={Rc<bi2;E^q{QD zNHL*kk?1U)hRVB)>xj(@r#c%N=N0pFAOfgDu>d?Q1qJN`n0*U`VBF5GAr5+s@^y{Y z;E)@fO<enIunCq7tznQaauyLr<BtJN)*IZH=J!|E|5?Xc*jS($DB*g)`hqVQFoU#s z4ww-o`K%~1fV^ibnFBXNV7YzfO<^_iFZss-0vicg|Jde=N?hN)+;Thon|y`wvUFE8 zwRagFTE;Q_vCU)cQcFxrG^x4u_rCVN?yTkU&(l61yLvkx7GCV`>8?4fX97D=r{19Z z^l{qB-F=Y@c56Dk6T%mAesTofNq!<^!#?56IX$(8%OVL4(Y9<H<&gpAn{$?U;VGJJ zFXs{|tfG2S>fBsk+t3Tn-KWpC{T`|GP-VATMW&Cy(>=@o)G=-e!4muCXp^0R$MWU~ zLHA)|PSYNbj5@!%2HqQw;=-y}pS}P?i-+Wv3mR?3<I+{3DvA2~N2?}#&RdqQxIJc; z)@eIP7iwv1Z%)QT-CQ3*{*P@@>)8l-6FPX*N$1PZK*5iUXQu8%vcIJZl~M7GZi>;H zI;MUss_ev>uae%Q128Do;3?m3=BG#t<k^Rh%HDN+d>gd6vll<Hyot!5D`8;#6m~I+ z$4H_gvhfON$-{Lib;JUU0&0vhe}esJc{b*zptjzypILZnd5Xl?WNg;%|9<29Sidc^ zrkkp!$0lF@BI!cKm~akXXTrq8%D|5^jdo-e206b3%7d9_X9gx3R+KuQnQ`TWQ8!w@ z2cLd-s5S1F*7tR%3boNP4I`YG=8NwFlATtpF-g|RX74K=EG~9y8MnB_;mB@rixE1$ z4zM>5ZKb~jJzaRplxflU+|wHK%9UbbE=7%v(T^Xr3O6}0w4!fpqnlmibN7_jnKz0K z_cf$WippyGJDF8H@W12kbhvr~Ot*gw{WF_C2BsB!BSz{@cMh`9%KkRWW@uKEJo^>D zAFA`2=#bobrgL+@QOEMtk(PML{!DVEM&y96bkdoosz3BDDb_yB(#uZ{su5eTzg;8m z8(5*}VHHO`w7=c<ucw~Ze@fe9QET$HI4QQ1Pf9^{uCC3$m^<>o-mICBk~}-pO_<`H zM<&?+Q}!vF-V6~bYATTl99Yg5Rk6?0ww+F)-V<NCdVCzj{NBa1N<C`~E;xE_ZQi&b z&x!D;bEG6iuX55`D{z6nW4Zf}@GbE9wHlD`jygsv!=yibD=**I_`*#*GG!#$=g`rE z9#uL&Z*Og{m^vRpCT*U&dg5n=|00yWeFfW&<mc1n5UZUK0GeHqyJn_3H_HFjoc8Zw zH!${`8$I)1{0RRy7GV1Z?q9KE)8v0qXOW3R^v^!Htu3%bmjjm_rmvSGR>nd3zYB)G z)T5i(8>;M&+~a~gawHuxp!4P6PJ(u!DoT&F?<ONeP)6d5$rtnmk|0|HUEVp!jxYzM zMT&L6Wx@+l8*||mK7X7U&l%q1aqxow9D;G9_gfZbw~$U|yU~Dcz^|NvZjPN$jLvo+ z4A{&0cr6M(autUI>}TDAH6buGmPN{<!_cV>5q{|b;ojyusY`LYzDXQr_x?GXQkmRJ zvLZ9St?KX&3xPzB?GM^I{fi|$55CeX%#m;)m_NYvoJ1=|H?IU3yedD{&ygQ;GMn?O z5d|7seI?-0P(suboApGtr(L(dfF&R^@ekmS*Wau#4O`eL+yuy-0Z$K*7?}8)fIG=H z&yF%t7H>hiPXr8hR$ubB$e8`m`pID0f9#1??~R{MC*L}BeWT}R2`}aBV^K5W3gayj zb*g*Z32Na%y3=L4Hx)0`Y2WM4Y*Lt_jFxr0^Ru|p=$_p{FuMJ0g}HL^`^fp6(#DAf z0O{*^j9WM)c<t*)Hd%C=>YvuMlno}>>-rR&J#Sfjx#GJq7fjh{InQ|!pf9n;t<j8z zv<H-*n&1k0YoS1R(H!+v3A@~bV^(z^I}|(*o7f}2Kp7mvh5$Ei0qN3Q;h)$M8l<zn zhSOT(0jbs-h|F!TWX_^>N?61^j9M961)~?ETn~`)JxOaZ1V|#9V~mQow=AEPr|eh} zo@6K6etRPLSpI>WhaEhMje(^B1s+gPky#}EfdedIKIaOJ`vr=O=Xs#i=D0C02PFAt zIih*|pk!7u-<>51_Z{y$Nk9c4VV^LHkxr?>mA_4!rdp3HpAa)WT)MR3|JI2WaM%kA z8PkTe^L*)4FWtu7j>Z_e<$cecJsNq&mL%gRn-!36^Ci(`w7NUY@=BGOk+Q$C{?&q7 zOhoZb72O8=?p00MJ*O&57YwV&V*0zyMfQHFfh#>YM~9het*3gGDNC`ZojR5V0wO%` zwD{@$=~0!42pHwdu5il*70uXCoUCvL!FEO{@^=XGg?bUd%s~;A#Leh~KC-`2g{Evz zT5iXfV~iM7$@(^g*yn4nb=_ocE=)KTA8B2$HE}S{C&0h|q)57O4w%zalS4_fTK41m z7;CgRWXzdxVo%VQtrZ?XWcKJF{6KSp_;cG??U<MjWvt?-@>fgV#)C1J+Q%-bC%egb zU-bM9agzk;WUq19Kyhu%(?IKjYaj?SrsP7U$qL`lWrT;&qHJ;@MgmpIn4!Y9K_!f{ zYC8tT6X}YP$ODFb=b6C9QCZc@$<U;?+=blVoi*=kM0n`vlSKaF#Ph|><J-OY2Z)cc z5+4Q47`?aX<KundlJpF;a)TXk%Mx0qrg*aJ3yIeEiftHh@rVf@*{(;OCC?uD+;}#u zU$p%AYR`b4IMR?+WoslW;kCaeD5o}^>XGm^_KzC4xL4JJ+g^pM4~mZUHH<f8XQ;aY zV?Vo^d@_x}hAQHjTMGK74hqZP<C=0Q?w-f*zc|n&^4b1rzr_^gR9v;?L{;71V@*17 zUWj8@#_OC5-QdfI(f#MBDpDtbTOTS*A@Fpb_J#c4*jsiouD5qsOv#?Q2TrrtDGS76 zBK3lX0aLUl<ihEu&=s(Z(k=NlkR5hyy-dJm_0oz4j3SX;@U#Ir{|LXEGkI19hbu4q z*d}38y?2v3JswV%HZ;*oc=*8kz}n_(gYv3KW(;K34HP{*ID}*vWs)x#B(a*g01O(Y z&8jcB<n$e2uM4H%x$;L=EUihQu-p#bg9T-Tn0;;dh?6p9t8d`^G-21hXshni#b<5` zuYna3fMVZ4w#!}TsR^g>_nlifZYjuF(r}GDWOwfqSt9$|fI3Zbv3hhEmn}Sz7qpOq z6+>jNZ8D<`g&(7O?^qm37QwuHav-_SzOVolsWBYLxt|!$h<kdpeoE>7@P)7tluLAG zRm7=Fo|iv=ZFIlO^s8FZGP_MwOf_>#BlzPUyw!{{CYG}{`QvvRsgs1SQDoU+3sPti z!EJxWLED6o2-9U9=HlLR*OJS{!YrW}s_B;*%ntM+PFLma@PX~@t~Pe|V{6?BzEOC* zN(CaVL(*oj!kH9#+dn|uF(K95X`;U9orcse^?FC<J#5}1DL{rKa$C?S)Q9|8#5H-I zDt6Eq?ZmSI`qn*(bouadys9Z;_P`)kk{`;kv4t=jq#~>H>?S54ce0EUB|~l6@}{|w zzvB|E@BWdWP&DJ}_(&O9PiEkeT>u6FuP!3MeT82mKk%lObFf9$@<PyO?-MGb-Mb&L zjfP$J+A#1^s7Fk|5vS{WlF)(7^VW1^p(z#?*+{E*N>)(1BH(Fe-sM+-i!ByA=!OWT zLz<*jyAa+k;mo`m)Wkr{d2^#?gho*089`0UzyZ)%cH(hLxZHt*(4!Hd$U_LpqDi70 zer)@-=T&D9+9fhDBsDc9TH^Vonakq&$A2n|iOh+;OswJT3jcs~9df(PQH(0Zw6|?G z<h3Xj=b@)!Z%|ybRGZC|?#5bl7A1yHcB!5Hj!_z#F5eg2P7NU)_*3_mgJR}!>;p#; zXCI@8x5e)X8Ho{{@?QPsHWIZ8y^*tz1#5WWUO1cNDY*WH1Zl%4M%(%R5S4*A0KptM zN$Elbv^hJ3o+~s&rTb5tp5a^6h$1CEk>a|xrpj|?KTLR5hg7_vnlz2InNs_7$N94N zu!Er1M!H;6(-u^BBIoErv1!Cb+BI^=3=W6`M=1!IzJokRl-<I<PBvv^-oi1PU6aXi zAC$A+HxbgkxwiJcx3{OO=X6_VXO%*$QDBPnE9~GYayV{o2mz@l<zT~T>4=^DK%N<9 zVqTLylio>_MH{mFSNKj)3EPJGEyv1t7shB(-w>D9@dUv<(!n-qdrN=#YV9-HI#K-1 z$)b;Ec^9yz^lsb;4$?vnuU|v9bL)6N@h=O2;6DPU1_g0j1b9<Y5nge?8fa0;oS18T zSj6<zX74hx6p%8;)BR9-j!<U9O1YRp$tS<y7j5AcjdKBm7sga)em;Bd$F}JR=*)9B zqTR8vBQc^nr<~lVtb=<uGSuPG0Tr2vu-3*e4el)`CkidwGf|Vv#H=(Xsnw$QYsj_d zZAXV~51gC2?_=%_b9S39RGey(eUimB4Gl0&?)fx09=Qe5Q`#aB%(i3p3P<tE8iL9J z@%d%zJr5ycmVwJCy<<HKkmirz8EVI&?Wr-%;cUMMAe>IEFP<GONZc7r`>EV%vRTdO z#Mr*k#7OL*Ha3h7S|rx!V9c9%2f=M@@Ck$j8a|1*h{&3y;@1KjG1E0lz$6uyXTAcs zGRgr|3f1~Bba7rb-?t4+FcRa-^6df3<w<hh@`mMc=Nz6Jr0Rg1EC`BUZ~+4t+X*{& zg1JtKfi0^UafrEKU8mp5oY^lYX4Dyt`8>2Iv84Xf^p$1Wz2%bGrOlQOX+8T`<57vg z>PdJGmVqO)J&j)D7GRI*uNS!5>?G>XAOsOW<^LjJ2OE|@AiYc$$LM|fHUQ_y57M$` zuiRKb8S(w^p>z?jJ++mTZj!Xq`B_(uSP^-Nm(3yO#F|)Jxz=NeD^s;4Q?(8lT1Cfk zsG22naF6Y|KNA8J8{FRG4Z_A$t=tOb9XMqT#UZFXRM<q_-q(Lkw(P_BMce3Jj;kVQ zYvl@95lI271I-QQ=7upsaVMTTIs$TJ<AIIWz!A`a>3lJooe+wBy}bmNOvJ0A1E$GQ z2obb?*JP(2Ne1QTWJVP#gj>pf%x}3nF>vTynAcM5#a6AkkN5qJEN0eX25}0gNDl#i zKn5k8_;p;UhC;Xk^%qjX9~C#hZ2k%@$!sF;TDm%EsUd%7{GGDngh`K_Na@KPx0=(x zABEk(*5Dxvig2$c?_e0bZGFv`w`-X$tJNuJ<l8f`8CQ$)*?*zLSyt?M-?!*95O(2V zy|sBay9C0}t>wnoKVKPZR8JTE!+TBQhWQ-!{>Qa^9?+@|+5(svF#Ev7>VZ7<0gIfY zwp~zcrOYm2o%5dG1?4mSV|GH7qX;om<j(Lcf4Tz7-#HzbY1%LDzi%t}TWsb;_xJ0X zneSPH3r%*03q6Y^MM<}QKd<!KhZIT+Nci+2V78!4&1A}l*<=UHh${_o18|vT&G_B7 z-$odRRxY293f8>IXgfCgiu$CXHzC{}T$Pg?#6WztPLQ0sqO0}`lUV1QOKzRJ>NY#c ze7QPq6Oi%Za*&<!tN(yuC>-J_;<X*KmB`8=LW_-Hut!p8Ib><HF~h&9(nSSuK;%KN z+LUy)j99ls|9vvJNOF^g?yBEE?R_I)_D!lg&j~viV6Dhk2RCEzCpg<ZCf#}$ej$lM zlj){R+<~+_I38G-Daxgw4;jOF9^*KZ80jP$;G{QfdH8;!a>|^~+UKB@7G0m9o{cw# zu=k>?r@<ZLD*!tzj2a}{1p)o;GK;4mWvsUI5`?r_J1RM|!h7{4W<T0~CXrNN{nJdJ zJL|dQWA6bGj&(ZIv3lPoWp=Vg19<m5s<wS+DAG4$=9nC5P}Lov$@9iM!yiBd;~>)@ z3VaXdXt;Te3UmVq*JdbmRj6{PqV%Gw%a9x2R&bkE&_S1sm|jiti!`e+BM~)MC9}I8 zZLYl)&B?18Gp-TcS?N6%yJGs7fo%)T9(j7CnB*>7#PzAN>MsZ%cj!|7)=p3umA)pF z^vg>Ow}{CabSK>k>1gW#az{OBq%qNZ7twso29_OPi>!#dX&>k5dbi2eA<MXWk|3fK zXSZSwva&vY&46dNj2Fio!Mp@&LcB6xOOP?3i&_{85Nn5cE$AdDu^H{gE+iK1q%rY| z=wE=TeZJnCKJ}2;5S0?^%EU(f=~u%SdiwJ{izN>sZy*@`0cwKkK9D&?085f)0i|$T zdf86htBWnbLyygOe+1>S+8DT9T3Jqbiao1jSWQ1V$VE8ZW?6dmiY6~)AUGHhN!Ame z5$sqaer?Gqpg-K?f?Pj7X89icYOR<ZY$GZR(v#;)jXnkoCh|-~PR%Zk^#k6o;X+Ne zcTwN*I;eXn;NX2Fc}mlbkXBSY^1;IgQ!OIHQud4IY!1a_!Y?iEx+;6`X?=9&7~Rld z(dzg1S+`;BCti2jpONK7atkens(x%sDQXeha7(+XXIK7f@|TqnMZIS;X0jiuP81AJ zd1$bfrfLI@Bpw+rDP`)om+BMU(49IB^*wiSP2?ct8+G9x^&rl-Az)pcO<Vnwg`0&l z7rR;(V0&@t<PHLTn5GzJCLSzQo$iGAMj@O~_x(!^p{xVZRugZY><abxg?fI$bQ#`F z<jY251J`HMxfnsEx}Xwlqtm?wwHQ@k`i#i<2|C|+jt=qMIJ7K{6xd$0^~*j{Smq`5 z$-1e;i`ULC&seq;cHx4{<tJ?7{z#7b<~@XaJ;D72*gNK0#ezLHcR?i>SEkSFuG^GT zfiaHtZuAi-k(F!W_`T(lQAe`XNdTd}c{gy=ifiU#+H;Gd{$}_^!K+$$5xfh}Zw8Iv zE({kk;Q1~92Tk0tEB{`Dgv-oBcOA+FIi%e&>OxQs_ei?4$wt(K+&fK+r$-tW(<l~` zrH5}iT=oo~zD=;Lb#iU8LyS(RQ(X)PQz8P!-9BJO`o&M(y)%)pxTbe<#K3fK@#UMA zT@CP;KeoM&;Z_0c<HTvs6`&>=6=cJ?hbF8P&?k}^I-SVE041n#-gNw{1>eRzr(z>@ z=jo38UoEXHol7TN^t5|~@01Og{z;+G4g{o6T=V4MPD3|23tT{9q=liHxTRpql!+WT zJRXBNzKn<W$+Ia`y3A^yv_>0hZWyu0T4uRMwU5l%S9rnT()%tQ&Ec)vIsSGcge_XQ z=x5}J(XMxVN63x?PC>u}b`cALl5K6q_AF~JJY-#x#3D&XT$Sy*+amw0UW_z;j`W<@ zdYp>ggKU=);1I$1^lo@NPzR=Pfd^$8Nl3wV(1yqdppQc{$Q>9NG_dIGz^5an6V4;4 zejn;?LpVlKHdsD4ola=5LNwJA7VY(C<hWb)b^2Qw$DYSr4R+S2s1^`#qzqz!f;O{# z%YEqS-8*#!)ZTWY?<Ds|X=ATcrs5<ekYL+_Y;|a{_6dAneo0(BvngZi{U<A%mcjx( zccM};D$3}7Y02G%6E<@mZmuoJIrqCu4=v_f%;77JR6O8~-{2@nwzCD6QGaCuD)EK1 z8c=4G^DoinF_D4tMV$!wWzfL$4_?7b)IV@sijG2i%NGsLdaelS{T;pAdU`e&JM*7m zUl`^713`k%;ka>-5yPd34!HgOq33~!9$tVs!iHsAD@1yM6GPI_cDpOAQscBHRxx@@ zmPyy>K5V)6c{TfW{oz@Ad2cbnpDA)Tt{mr(XIHpV6NyHU`Is$)<jJ571$72r5IZ4| z@jK{Q)?#Svt4TC!%0%1x;BtU22cOmF!gM;|lJoXmYh`Q1Y)ehGih&7tE49~Fqk6&b zd!%sh8c1xh0vt95@j8@2qX4AJ4=QBgasbNQfn8l-A#>+d@l4U-=i+vdQEa$jbGMS{ zFiaJdKg7MNv^*ZM=@Jy8_UFBb!yA!`6-IAuH%QUvf~OM$vaRlaU8sD2%MT?Mhg?>8 z`dWKIZw0JgPDL(UF6z+Q^U@a`>L!h<PHqTnXl~8)w7;ND6Rq;9F^x~TRiLUfw$t$) zW(ePzMhN(=N@JyWgXn<liN>CRgY`U)(dv(FyeP~d$fR3nJskChaJHTQdmb{!VBz9R zI+M)TB}ps|9J0j?)eR*DQR$Z~uV;L8ov?ns7^8++yG{q>TTD1c3Oi_n4Whk1iIE*; z?Ezk>B4~dO8ZYBN(n7Fj*svcB3Dr4DOG}JFrsH*%D>`AjSm|;X{2~we=OXPvkPr|j zgP~R@<yYta>1+!sdJH+I*E@%{I>WQ%{{p>YtQQK+*z4Jexcx**j?fTtp<ETs?6V>H zgn5<A+nkBaF;%~HyVxG-%+nXwTzpOdR1^ZL<PLxpbx>~xK}KK{BjpgG%mth@UooGF zSo5tzAz9tI7&X)kla@7baJdbw1LeoF_VkGp(Z9az==b)W=pZA`W-arVI6hC}?<uD^ z=I0u;_7O{2kwHhld+qi7xijpe;*?5==@?ByRsttq@YwQ(y+gHmj?dN76-{|x;o6J4 z$qc6`4IM@-H04s3{?|R${jbXC_L0U-Le66JtmlPV!>jw2r4}1jC$$vEzN~Wrsh?KK z510VmwTm2vO1_U=%}wTofMV;)OjQPHSncc2-mH(A1AF=Upp{@n`k2DRKlzdJ`|ODC z49N?6Jmnk>B{VYR$mo?~jdCXuz88kTlLsL<6OD)<Z4<tlo;QZHSTS$DaVrEjOeO7p z5!50=Ve(Mr8II=1%*2kDi7ry8OT1q=FS0NGfx-{=Qfo@k`klYFXPkKb;?>F6FoR3$ zh$`%?7BiHBcn#ce@w^z+I*^OpCdZK87;&!%n!n=3`0kJ*&k^$kE;XQ!LNI4?zxw;D zo?XF8VDu3~Sg}a?+44M-xoYPPx@}jAccgf<HH%rS_p{f0{8kAl9DhL}JjHV;EiP9; zzy>1~Fkm{8ZywNv4$lqMXbEb`QR4Xh%a%!_VSVZhBdK@cYMKy|1Z@uWdXZU?s-~un zg}U62Zrz1<%;dNnyZ3ja&{k&Qw_LIxNDL7Yzy1|s@zx1NbWS|RPc?t;P~Ki>(VOte z$x4Np$1QGsXBB<i(rpn>$QxsOY@RjSUw!zK)ur7x#Wp5^5=W_7CndS(MNCbi%whlx zwmPL|ANWx%<alc&xLf4t+zAlRU?&83SWpQrx>C#`4w%=F5=%A5rw5E8EI+Ox7zpv# zz9&J$>^q}A^M#&aRz4u9v`%%8W)>cs-Qa#V(bG&5I#$a&w@(A_o<A@+KkT3fSbBx7 z%!8K{v6yzrF~O#No8Rst)1772q)bzS%VUR58eOL|3o}kFpYZ>&O~!oqMDmxgVB^za z8~1vOeGIyLF=5z2u^-zOk%0MZbL5VKejU&9M8nw&+%Twx=Ybt;K!{n1u)Qhe113;9 zGdcxaZ8BeIyOw_If6>iaI{_deP!`QSIlFUj`;Le&TQXnbT_1LK6b3{-X_gzGKW=z9 z&+ID))puIy_?JN6@gle+MS*;YxfZamEYS)buhd4|)!(8bDWN<3l3?oA++SJ?)|=li zX2HZVfVN)CtC*<NQy+Tmrp5CS*@evf8aYGvlAD&DTB5dz>K22IZ{ObF#&$1DAlN>% zSqc-**FolyyXatt8SGr!t(I`sN7SU)SJF8odqJ(0BFANzE^%q#pvaGH8S-w>BR0*E z{k(8JyQA{;`=UtV%{8qFsh>N_W-R~68;a>cgyA4<&5x@Syg1=s057NEgwj;_kkQ<Z znc*5Z8?mM=U&L>UhAr$)mmTiHOME5muxB2-HR(jlWVMmcM+YSH&!{`@?|R^flnO_( zk>ocM3;U|sNzBmIaCSNiyXMLB0o&EXYWvuudDx35aC?9b$T6PnHqAI4vq0L9_JK-R z@Jzq5{JUWTO4T7n8_^campg7O61%!#ad<E6U<f(58u<AOviM0b%9j)bj=tUK3wajU z!pxW~W~^9xQ;jl)Vx;`Tw@lyjB+M;q^3~evM;s3U5_mS$E$}z<k)XiqLu1Co^9ykW z#3Q9mlQ{i%QC%|kQX{0BoNUVu*DKxhd)=$htZ~q}6XiDa)KpdfSKZ6M-`^ZzX)}&0 zge~^eN2a-E9zQ-}5tV8`<#V+pqiUJC^rBhkWlQ#O?@e`^*_rhG28Z83NeM?l+6Es7 z$#YolFM>>KCBAw$1U5yuW!Y4KKaA9x<AT|l_(52P+JR@~WNFiUJ>>3)byDY~A2NUc z&~y6PJk|<jXp(OG?aY%cSLAa#U+QaLJTHdtDY%PZ-2bsniEq&>nuD?Mg_5`l0zCS} z;kkUQ7+M8#4J8aNR0`ylvYhHWIl51=N4DSk%tv7E*zb>;`H~w@q*QjONL2Fc2SO3m z@?oxjgh%9an&?U2(*ZS-zcqMvd}#A*lS<TdmMOL1$ZC5~J$-yqtjv^5?nQk5m_rZ` z(zWkei}G(dC0$s%dihIz-SguwLS$UMF8(_W=Km0NDB~YIDNWbN{Kdw?W><2+CF&-+ zf3A;gtoY_pRjA?zJa~Fg=9KL%JB6F2>JHMs7Uadr;uUpaC#&2){V#!cVQuDn`x@;E zpX<_3Up3h){BE-=?%1`o`1BVM4%*x50d=sW%}&L^x2iepo_O2X(Y?EUYKRTQSHFmy zR+0L3!(Qg7oyKSaNhJ)Y7hFAl?4DuV^CvWSH}E_-MMJvds7bEzX=O)+K$Yga;DRFu z(~c|tcJlb(M^nxJbPS)X)seqAPc)9&YiQi6Ua&>wj+~96XglwYElMNY!M)r;HUUY! z0e_#EOJ>|gJSfk$-h(n45h_EmjbU^Zj1u3!cc#yV5TR6E_>Op8Ys&Ca0<~-W&tUTz zmz2HD%~wXG|4f&X5th07de%BWqV(SQfbxUTXH6rf?mx^!RqUTgRGJKbLY+V6SY}V{ z_(JeYNF}LvcM<#w`j%!0mZsk2;FP7jh!^I4e&NS7bJ3Y;r+ww0`c(VcsJA{YQ1n4R zwMG40__L?0r?jWEjP9Xpjg57C8*c}C0Jf!GXVR;m|Ex}S&xklP&mP^OUZ!1dqw;E0 zKf3VM)BHnbdjd-@pQLQFdq}({Bl4T9$F@d5(j3{g^ETB$M&ci0_`-u(=EY}Nxf-SE z`ngKLK>EblqTS~TszNQEV=iL{?GT}~xghRcw%kp&5f=$9v5DMN$d{vc8sOOEaF_~c zt^*Ep?5H={F*)kv`ga~cKZg=J<mAdX!{O|bS-N9NO8CKiubf$z82x}~%#By)9`>oh zRZlb9eoH(uf$bw+sr8bJ@MzS$6yfePa@VkQ0oAYULKM#@rIA#el^hiElHOMq1TT#d z&gf{#x+xDinTu&^&${LZnDVU9d)e}<qik~~a#g?{qJH_u9<MsvE1QAeM?LD@*VFlG zZNK8z^A4Wd&P9Cdu3z5RI0rLZ-tyfu=H344bdpz-TfblD@>oqvxI^3(hmDd;&QCO2 z=dRcr{>36V$b`GsImJy7WEGmaLu7+n=VDGjiXLcX0^6*M*Z+-emS~IcUz{HE6!|Z- z3*T+c1)HFYLq=eMdn7_c_%Q=mKY0LAKq*K(@u~;^&0B2$|M<V?pL{pSok`2Dur9m< z6@MGHOdZH%op6y^`WB!xg+D#%?%qDpdH$H{Oxw_^!Sk-Rq#o}`xji<L53OARKUdB` zY^how0a*Q$X#|wFRg^a^+vMit-lQA8685A8bM{3-*kt%5L8&l2`LuaVg*Cu&vK-lP z!9KhKmi>MO&OZ%jKf&w-j>;1YFri9y(2s37q<f#5wx;!jX%SdP+>dR?teXJPe)}pG z+J*h=d%RH5=rQ(-B^n^3l#*sFM+QsLgkZvKmjVP7mxvP{8mPO2Yq>!FbeJd<HQIzV zGRdFR@WSozjsMp-uUqjUZHYAiU*<7BW%*&umM9Te9ozA7<p^dmUliC#6G!0u<CdVi zlD6X?Ljv;fQvyT*v%Z!O!1wi*Ew%sC$3L>wsw}*|1Tu>Bk8SFUG$@$79T+q(ak2pw z=G+F_g?}AL>&`sztt;@53i;XJ5%@a-e@Ec&2>cy^|E>sFO)z7INc+ICA)ToQC`ki9 zO&WZqNzbG12v>%CAFTN0FDFeaU97e?3eHrR6X5MTXIpyG<u!{|VUBn52D|2>-}U>a mbVU{0Yc!YT8WjH@CFg(d+Q9$T=l}ip@3{U?zXo7G-uwr+>BrXq literal 17469 zcmeHubzD_%v+vqLcZbB5?vh4Yx;qTIr9n#E2m%s{v>+wYsnUobNC_wk2oees($Wok z-?jPm`+3iM&VBFw<J@z9i_hB3nkQzSd7d>h-`Rlvf}R0LHB~iK0SE*FSc5+RdX~^! zLs{8M-%w9gLt6!G002UDZ5IzuC@BEAd-(YnswpANEi4f@9{?zT2Veuj0AOS1>!o9) zY7BsvhKe%67wm+g|G4%?0dOV&80J^kM<D*-|CcB-J1-wU0D$O&y)W52_}YQ=J&?W` z;OB+mZ-O+Xts91h!ZEZDs31twVrZvdXuk74zt9R8+TPv69_)kB*~{MD9z%D6^!;1@ z4j>KF2I;6<E)D@8Jp$5fH~rmRK>84*Dcv1xd;tJXh~fJ=*g1o=5J(gH7#k{s^ko3R zC3N}|ZTlzM&mj=h69AMwy@GsPoSgj-Tz1?DVJRs|gt|k(O$R?eetjD|Hya;>Di* zmyJgd0Q|CM%qV~yV_O7hWN~3BadCbT0nq%vU;nc4_g?>+!0hcWJ$4Mv_Y6Wk@CWaY zv48M9^8r9|546prKX|qo0MHN#0JIZ-@YwPIfaD$k)DQk?J=B=<;_T<=B`qiz5D*~X z;$SC$G3f8tf7;>qp8qlNCwT%GdB58ZamB&W#{Z@t0%KG=&zqk9J_uhg8#@OC|34n$ ze|zAc)cTVie1;B=4n7VZpeR%Dl(~2~f!pn2@8aj;>49+Z_>Xq@-yHTQ9WdlC=Ncp^ z3Qz#0l>k6GND08!2LLz)0RX#}1GYeZ-#2YMQvmbknK3W^a_&JIY(M|}%Lg<8e1!VC zI3X}>WkX|xoxjg53=O^$m=7!fA0PoJ0Xl#QU<J4Uen13}0Av6KKn2hQ^Z;YP0=NO# z0nUIs-~$8zw}HDr6c7hI0#boYAQvbAN`WfiCC~u00PR3G&<}hBJ_D1$46p#K0-L}- zZ~{6q90(DF0zwC2hOk3;AtDe-h#W)(q75;GSU{{HP7n`>Kjbzf0ul#Ff@DITLP{Yo zAWe{V$UDd<$T;LXWErvzIflZZgiuN-Ba{Ow2$h5?LbagA(Cbhqs5dkidJp;#`WTuI zt$@};+oAo?&(Im@Ds&G9zzAT}u!}H$m?TUYrU$cxIl_EkcVMxwG*~{Y3f2tkfepiE zU~8}=I5wODj)3#SW#H;?6Sy7R8-52K56^^`!0X|i@L~9O_$C~MMTEtG#fv44rGaIR z<%AW06@`_CRfJWC)rIvLYXNH?8ylM%n-g0CTOHd1+Z8(mJ03e5yBhlq_9yH)>^&SD z96B6c99bMa92*=zoG6?OoC=&aoFSYAoI_j!TxMJmTvc34To2rE+%()W+*aHn+(q0I zJW@P%JZU^VJbSz#yac=gyk@*Xyal`yd@_7ad|7;Bd{_Lt_>b|c@VoG*@OKCZ2v`WD z2n+~Z2*L;+6FevACHPKoL`Y7^L#Rx6jnJR)Az?A$8^UqI9U>wk4k862OCmp_heV}B zZ;8GU9THOz3lM7%+Y=*+pAgp*4-x+)!6RWOQ6#xR5=@dtQbRIGvPOzW%0a43YD0R5 z^a*Jb=@{uA83maznI4%tSv*-8**mf&avX9FausrW@_XdZ$lsFBQNSozDU>PfC?Y5d zD7q*XD6uIyDK#ivDdQ+BDL+tdQjt@MQkhZ(Qaz!1O*KOeqvoL2puS0+K>d<>l=_&4 zkw%fmfhLBgie`xBfc65d0<8mWENwOIXWC;r1f2?<8(kt@1Kn48I6V)&0et{{4t*E> z+6BrBG8gPF#9nxLVS)k5z{6n35W?_`VSr(ek%>`_(Tg#I@h#&T6E%|plPgmSQ!CRl zGX=98vkP+ya~tytf(oICxQTd-=tOK@WVon)(f{JpiytnYuyC@Nu!OTzvwUU6XO(1i zWKCi1VBKV6X47H2%~sAf!H&l+$?nXa&fdd*z`@C3#u3fYz_G+h%c;Q`%vsJk#YMy= z&*jaP&o#o0%`M68#+}Xmkq62n#^cQMglCW!$}7(6!kf+ei4TiUiqC^DpKpwxkYADi z7JoVacL8bvZGmus27#Y~Y=V}8iGn?XC?PSSn?eOblfvY}n!@42&BEIvJR)`?Peev8 z5nWQfguK*vX-kw>)Il^y^otmUn2uPqSclk|xP-Wmc$N6F1e=7lM3%&uB&DRjWV~dr z6qb~d6jG{1>PT8#+E4nW^o9(-jGIiE%+h7f%MO<dF3-ub$lAz0m7S48$XUzf$<4@L zl(&)3m!DN&Rj^kmQdm^vR&-UYQ2ePRq~xPir*xn!tsJWS<_heJ+LhQVA5_RyOjI&e zzNxaPI;obcZmNl?g{Zwz$5Pi)PgEb%VAQbFDAm~1l+X;-?9?LEGSbS_n$_mf_R((E zf$C`LB<oD+vg>;2Hs}F*8hXikU-dclz4c!iU>oQgJTdrTC~SD!u*ZnP$jYe9Xx~`L zIKg<*gv-R=q{EcN^qOg@>4Di5vm~<_b3yY^^L`6@3m1z<OB_p6%OcCYt14GhuP$5@ zyB2kA^g8GD!0YdB(BHUu<Fyrum5o)6HI}uRb(!_4jh;=u&7Q4>ZI<n(ow8lJ-KxEU zeTx0EgRDc6!=j_CW0K>NldMy+)3UR?bDHy-i?U0m%a*IUYp&~oo1R;-8~Uc%&1!dC zcWd`14{{GTk4{e}&p^*lUc6pWUNhd(-l^W3K3YCSz7XH*zKwpAU@|q}&*dNK|NWNS zttYn*155*I0?7hB0tbS4f?|RegH?kIL*OBHAsx3b-oAVLd#GaQ(>stmws$&^tjK%F zg)p_SlDqhK-R`~*7Ya`b-;FSfXuijIFYMl2q*`Qo6mgVK)L67kbnboleV6+mVlKsG z#GqpBWBcNS;?m+y<L%=69tb~p`~dyX@!^LA@r0a5*pECOjV8(`mL!oS1t-lV>m=8w zAW~vec2liV-=&GB<)-7O`=`%5)_&ZW!ItqT<22Jb^Yas>C)HUDSut4$+4k8(ISM&d zxs17SxyN}fd0(EYJ+05@$WMEQ{p{AW9|fibJ%y5mrA72baYbjv9>p^yh9#Y)5~ZbO z3}p%B&~pFs<%(++A1kj^HdgUhJ*}p$j(ZL~_kX_f!urMNORbk3HIg+|wd}Q7brf~6 z^^p3Y`ppLChM7i-#!pR}O&!fL&2_H?UKO`oY<cpU>UBaZe(SwfbX#!S-W#tsYwb?$ za~)P4lW)!6j&vGye(2Kb>g`tTe%qti)7C57+wxBOT~nWAUqiomfBk^iK;3(>_jMn{ zKhzIO3^sn0`q=#G@~74z`JwjVE5qF*8YBImbw7U^H5vUfc5Q6ti~X0S@tfmY6SpQ# zCc~z%reeR6eogzv@GXCud%Ai?eCGA{E8hoZjb<n3?B-VIedkXX?)@P8k+#UZSh^&# z)UvF){Bh;#%EGGG>d9KvI@x;GPp+S}8wwi(o93GfTRvOp?f4zKo#I{5-Htu|y>I*O z`zHsnhjfP}N0LW9$7aWiCjqB8r;pFL&zezMs428N8jb!=4^atpaRd>91`mJ>rUT#& zfuAQ5mS8pk`IW8$Tj(KJ=lltfzVItO3({N=*f}2x(!vlZm_h)+3V8cQ;{fVlHck%C z;}ZSLoE{)Q9=xFdH17{7uwQA66wHMUz<!+*8+`grt|ieQa&1BS&vGFk#K0s!t{@-! zE1iHr0^ns0-k7%Ee(T$~c=#DPdwL=CeLNAu5&}XPnDEc8zcChQmA^68KOBwUb8z5q zjP)<U*`Mt8H^%xKWBrY>{>E5;W30b1*54TGZ;bW-6UI^p^B@fV`0Y3P3wjbz2C(5+ z*jR9EY%FXX9Bf>CQha<oJbX%G5<*fMN?KYPN@{9)Ms{ZU3#<&()CfKVD+ebx4>uh% zzYsr{5IYw)7iJO&SWU#o#izi>r{JQersw+KzR+&~GF-5boCkwk1fXOP7#RfJ2{3~S zV}V6m@cJ!>gFs=RzBsse_yk~wI#K`%fx)0~7#0>BtRh1~!FB*nhDCl+SP`4Tzy^oK zn^NR%;!|8!r5Ei~hMzXsF4_8o<Ka`&(9+Sfb8vET^N5OxOGrvdD_>DjRa4i{G%_|Z zH3L&?J9`I5CubK|UqAm_0f9lm5%(gaqVLDVCMBn&rlmj5$jpCMP*_x4Qd;)1rnauW zp|PpC<85bGcTexTzM<if&!b~s#wWhd&do3USX^3O+1lRO-P=DnJUYg#3jzYm->zR) z_7CeK1J?zG!(nh7%(@`Z0Pu#9!Lcq1W0NZy;MjOmu!!8nrBq6M`l20=^^)Nxm95Vw zd}=n)@9bNcrJb+rzqYXOe`{sGFYJ$XO#p;22)KDLGC&?Uj<VW|DvCQZ&%<TjXc(>F zoU$qe%=B8@ET5wR@75Fu+-em|NAS2tfvo*ta3~G=Jr7DUTadevPzXqqG~TIWXj>%$ zbaTD$q+UmzggQgtn_>n7IaYg)Pvg$4^YH8t2zXfNiWEdZ9iHF{q?%&^iCM;c1uqZ` zj$s^*Sn6;mD*RGHJRG__JbT1bAUw28vYi+YP7(~<F7^p)$=WZ7Tlnx(t`^YRvGbJI zG=KQ{Vx!cC0#S;GwOQiBk;7hG2!c5nh)GFfgT;^Zz2Sn#&xRo;roYOk>JH#P!u`Mn z<@#m9Bqe-OT+WFw)<@*?Ve0#(tmmpN!K6?08)|dyf9MZL5IuSo?=%9H(QjT4GuXN2 zhX!(8kcXMKD|cx`ekwG7ePr{#$LeT59Sy{kPF{b`bI0vr?#{X(8lcciyB<04Dz&oi zOj@~B%F&;&Pjj+Y0r5!f>k2O7BsNFKr}HVKN`8~x3dIqPClFRlQ)YK@bWNF|?#&!3 zv=9x<x;T#U=?abEpuS?BoEOrqe_Zci^yzfs_mW?q0Ebny2Vrg5-5i({q7@NIJR~KS zJ$t}O-Jq-^8|TefrtIbWEd#;+4oF;Aw=X+4!&3+L7+_8mBDa9rJYB&uq$MZ5q(I)Q zs04wZ7<-2^kmEA#0=w3+R1a~Bn$vCjM2G<Ez^H95K~^=3ciTP3jf_1;Ull8gKJ)jO zZy*Vz+72^UqC0%vhYZY#F_SFoQ7Y@*t@!GJXbfWgR76fe@oIsA>&u$GwH47@$@Z^_ zi5y|a5AW3<XPHtTV87!hs$gQs2{8}L9q`I94vnW4qG7eURyrt_Su{NDUb{24IsCCs ztS{N}V6|ay$xWxoMTepB;l^O*+SGf_eg2OkB*GcSsgRdXzC80dVkX8`B??vI+`@Bv zZ`db8qyt1DM9Z_4E)g>a%V)stE1qoj0^f1oCy9S!mmZxz@Nb>Y8Jij;d>tR;uz}S2 z`6Y03Klv=3fvbVz=;rsDkS0-!e%(Yg0B_?Gq<Hjd?rRi`Zjtpx?a2Dm`Y1tx!y{)J zd7H*X7~@FpK=OtzK^nKL6>@cRz1sWzjz(=o8_S!|{DY@8Eok5k#!8<zUr!lXO|O$z zu|%KQ>t@fRHYv;6{L0+ENlej5-B8o?5mLeLftv|`6naLk&cNiVumZrhCG&7w>UT@E zO5w0~vthKc1`$GUC>O0)OmsE!uu}vA2(6EYBko<$^^xU*J=t=2J3(6*&&|)q9kH|> zDgTgmR4-Z#=d<5^Pd3u1!~*7~(B-hTXGGIsQf~(H$|IfDQw6dbcD{9A3SaQq5iO~z z0rJD}xnRj#oN-Uo#=Z|z4}O@YG`jTE=@GAbpL&cLyMht{?Cm@9b(zMc<g3HGZk@@- zP3Bm;_4VTOypl;l^a`=!5%D>Tu#wpAx?%5Sd(;UUNUUSePY*%i*!;j%9VOJeL75q| zU@8-qjDyN}flB@0A-Y1E*RB=+P_n`IjQ73{tMPErr@2r5*_!;a+HDgCUkl&Qzf8Xm z4=l{MR;s426gtEh!(jdDq|zVNb`@!f|6z$RxRb_AkGQR(FX>4Afow7b{m<k9f0Ho5 z$5e0h6%7pnVDCh(PPxU|m8<!eO;x_{wjBB{8kA)z96)Qlo$hmQyO+@Xr(lvQmhrV9 zzR|W#t5cF~4q=zCuh4)NiX=l%cVw?$R>yL7(I8i+(Je6PonL>9YA(H!c7;=5OMg@l zC4xU3_LKGbE&*NA>{kIh@(TAXJlnwBDk(D{q0`ucKX>mesK!EOwRrE$?}fFbV(lX9 zP^Ny<lyJnVB8Td8<dA+ag1NMeYK$b3q2}<hfvJLl0v07;@*dK$LX)?6W^DCAv#+|t zMQc|4OHefLaIxsq4x(Zyrqlty2f}GmwCz8#*zkpX6T1M3K2HN(5o}sI@UYO6KArNu zyR0j~;7<2$FAqNc<Mw!Wh(TNbk#|-=9<g4}CrhnjowLHgdQ(M1iz<3MTonr#a~7#t zkpvX?gIiXDUeeFmZ>NC~w<Qv>qSrq3Gc4LROD71GB4VyC03y$c9%aZ3$NG%D9CjKj zJ3=}CD7}+|EIy)_W=s*zt($N)qe<JK@4Q=O>D(s(&3bY3`CEsYw68lJ9djmZ8U*+J zZKK@)-`W`lX60~_xvMvL-qDr6?%Q+~m!LTi7rJp(E955Sphr@<2B}TzRNjJ*6y2(6 zX&7jjs~?SSJq`9v_^NU`Ur~0-k<!ZQD`(a}_WHu*!6%Y~u0N!e2PU`YB<CubZft-I zSuS9wZT!r6d;fI&gVeGDHX69J(0%FKgB+{(S#x#3MW1NrZk$R<gm>d2=TMkbF0FY{ zF>FHH0k^NEi}%Vj?@wC)qm-=?>xZ38ZrM$n^E86RN8f+Mc01(I{&@2KC`}~RB3nxs zhorH7{SCwMu92=_j;u+W{EkrOz1rl}NfiDi&P~RY3G?B8$%wW0H+v5FeiqbR6!=7q z#n@0a1bBFbOrn8%651A!*lyf>@|Gyvz6$NCCpSc13BTwyb@<s=u;^SUeQkxivU;q! zzAeZr>*8j^0kVDh<@$<*{-%56!b`=M&d(BTV#t?cl?-WO#UF}eOZG)iiE|yLn$O4T z9V%C*TV7YCRL`&rtj|;FpdA%cPzvqB@8teK_#hmIszJ0OdHLvI^4Lry%Vn_qq-_Qb zSRvxt)9~OzEvokM^|!}ibSA?$V=j*G@ZH0C#N2wCwOhF^&}pHD_}Y}+K(2J`dtjo* z-;}^;e#XQ7HXS+m{&ns3JFB-C&FI`{wqK$F@$2Qm6saSg3O&aqZCNR0r!0kA7h*5B zr>zdhDlHIyiYKxBA+3h{mN096M@NK8lJ?DP7~_IqX;BOT1B3OJ0x=h<2VL)8QMACV zEbC)^SUApwhd!a)g?)_AXqjpus%SuiW~id=W^=?s_m_KtdxHZ72W9+ENjs4)(_Wz$ z(ZHO4e#y`a;!KbE%I|C3@)PN&B<je+uUDGM_;B{K4>*UIQ4)F+y*K*U62706secCJ zHy7A0EJ{8A8QEyV&`hI>Yx&Z8B<t567@~pd0{K&qB!1W}-_q%=brLji;n_s5Y>Lu# zn1^ub^v*e_JX1F%?SW|2Y|uA_GVND^Fv;%E%-_UBO6DtiXt`eHpD^=%5BC3&^2^c7 zdr_oR45MW*x@Qxm>l2iReN?%`-7WNtRf^#wOgUXfS16Rb3rh4>im4VDvyC^=1&Uds z^#??B0HIe<h0DY|3b6`o;5gj3FAx;%vjtpv<T$LEfUWQ?{2s(`wi34kElU{vy{5N5 z(nIe~T5isX0%|7%8;g_=H5mk9m4FoBoi@K)^or89%nJ*+%<>42A;(o@4G-9q%)`4~ zw_6f%K5rP)3YivHJLQP&8r4I^JQDQye>X5v1+j!dSpEhhK;8fQU?d6oy`;<n4MgSV z;aQ`2(()q?%PfJ||H)jE*-}?7`Pjld++q}cY+>%9D-V$NKacsZx3EUOZg=_^7u@;j z1vEep3}g0vAi`|;j_2WH40jLYrk~hpqk+5!xo2R&+R7uiga(LIOo~76PN$<dsF4R} z$nJd9*f4VGq7xcGrk^^)bU{11i4a-o9JS5bV3=d4u2$<0E}_E3&%1~54W5condX9W ztp1P_cVeXj%8AT9LxEz5B*D$8=oJf<_yJ-*kckmqimVqMJjNaWRWqPQesjdcG87bq z{9TaXdZ+{{9MtOw*&QZ21D@s@1gPti-*hcReHoTtV*N!eXXpSLcxzzdvKJ}=THrT1 zC2jw%U39MYyS1F9&C=DMZScRflWQ#2hd)PBX&ES<n(zZE7=8IL68|rAakh->D4Dwq zKf;YLyJLv4FJSt)Z62o-IaZr{j8`SEKFmj)Y_F4A$^l0MX<BEW)~`oZq5*`g7V32Y z8u+N=8!Ues&WBtLM=r6{KO5<;RzMAa#faPsRwvtS-_`aVqE8PeK+C710sICP4_hA; z%Omsa6Bf(C#JM5<z6>u(q&jiKWizHaBrgA)19l$vunOvoj_!{a%F(^JKWLp9nRL0A zpxcWiOA|u<wNIB|E6YbHgdkgvH|`{)$BA5-GLZ{z9X*+~H8xLs>PC(R+Ji$E9@Lc& z-U^=>l~K>uB+r*nGrgEnYW=BN-*_j{$f<ho6RMu^bNb%rJLy8ANQy}}AyLa4ae3cd zmDVOo8d@#*Y5BCcy(Qx$A>lI@Wg3^07|R0F2n0zK%UUDzEqOIA`w;1(WQ@%oj9+n` zyx?0uqyS@Jep2MI(n{W1UmsM<+yCUjjjM%CN!u%X2!9*W`-yQuqzlH-DB0KDxRmuw zuq)4BSk&<@2|SB(_MjjC)H0u`Cp;&xbd4~}Ln+>i*f*YYU^I6$ck#@R+vls5=jt)4 zN^sIC^F)*S2E(VK7bDbC2ANBh9al`Wn&K7A2(m&^8ZbnbX~=36mj_X9G~b5dD4oYo zwjBGIdv2m7CeW^^n`)1UKdp?@$z&q$S?RTE#mSSdlI0f&)8bpcto7zHQPO7b$kBjA z^L5&Z`3Mq>t9XXu_j}iGe_bzNPrqm}@1w@(%X<g7Bf);!irmDF*7fM$ZCmjfKI~VB z?(ETa(;;HHdcTrZBY^&EhW%uC+KEpuK?#2&bBhAPe%~^+GBPdO-~TMv6?H{VV47cb zJul0N>9Y}v>;+T4N2R}xE-u}u3!hyoHJ!aaV0(*>M#k)-Id>0Yp@#Uj)}3mXQWyHm z)JEkhqV>nMFV$n;gq!HR3=jILqP@{Oaz}7=kG^g=gs$E)<>lLe{c3{c&1?MmH?HP2 zDOEjG@2u!;wYcW+oB{Ij#cWM&HRYu1a~_uorkIexv2|<up*_bswyPNtM;&}~Fqb0{ zl1hK+p2&(ity{4!RdxfyN>!0~Ry>L<;&ke4LJXveJuJyOo0YFqXI_O&u9iRZ-ET;{ zrQMRFmz+eU0Q0m4dgmCq3w~~#xfOqJ+p`gK4?qKVQBG3svu@7ot>YDkK0|%I<5Smp zRaf~k!tS!9e#wJs@Jv2bc!#gcGaX*H&e_>OBi?5Jk^Tqmklk5!5aU`1ZDN#lnMpfC zyj@~=Q1@h@m0l9BbI~O&N})mQZkuR}`+`|Oh<Cin0}}Og?H$77`odBfE)}<)gn5IH z5PB}o+!jT<si6e1y~$(L>&x8}3&TERe|azEYxv|ByT?$c)rTg{JTW{-M|0%hgX*&* ztIcO8B5zT*-H{u8Xdv=xei(0{>jZ%(?G!<v?R~NEc^0g}4U)oJb-m8=+ftw(z5zo6 zs`RU?X~mr%GLiT!m;eb@Axp{#*fBvO7Ig8>P;N}hQ0fE*(6Ijt+}}i$K+QqlUy=v9 z?yyUjH9#Mwg$9lfz}lap*jtxY=3$QEN{6H`8BA=Ba(3}a*db%SvwBBRD>N|H(gwOZ z%?r6N0&mDvJ~@pO7MPm5HS*!op=s?ztWO9lPs}T}ODM9}yr|HrP>y6Y5c>>#5<~@R z9bN+kb(s*L0XlPwiOF;e`W5@g0k;c}=R{KRtGNtgrzIfw`M>yH7xQzu6;ILJ1QxAu ze`KJ6%f(>rsT6rwCjXsb7}?|to)TZ=mPxe=wW2R8itN42(8lKLCukrJbh7t+_4Zza z-Ivk8Ve^;#I7CwdILkuxK{XoKwDATN_yj($C!Fq%v~4<o&po`hv3%h_<WE`at#&4j zL6?|4hz1xyrB~&THp-AQ9~RNTL(?uw3#@U>RC~pzrz^RKB8^}na|x7pXAE_x&gzsF z9Mm=1MQJWzkL-~5L<91{#VA4!uu}=@O#}+*uXjoenm4vGnCtnqUQ~ND3R$OjN|b!+ zjA~CnA@}r736WdBHg@P8<ANfnN02sxXK=6^p6Gk6E_&4`C1+9Fvnovxy3hRciMD2J zyuE#e1{y{?!mx*R>}gG&-csQz!XL&&CVS|^7w$QwOEUB9&Trzuq*%E{zozeb9(s*= z=P(VS-p~1<0r%E)_lLLo#b<oxt+x)(++J0gf%}u#XLY>Ed@M9(a_zGmhu`p_0UsFj zM)bh|t8YPxiL1snnE0na-VTOa6)-B+LI&Nw32u%)2<F>Ek}mDj*eD0ln7z1iC`&wQ z?d4|95dSc>>)W_nLo3%7Ww2^#k;Oz)B>UBDsh`)9ao?FzusoEZ@(2w!qngcDm1XUu z4t7iq4Fqw1cdoNxsH3{oyJ=j0#w=Gu=e{N)w#V(gI17L_vsm(mXD9tyh~bnnL3gqi zgXg6qZ(bp6)pqyt6|V{}8EA+hk?xy3?@+hrC0^fL=Um+L_LlVM_GVj3%qQyxqxN*H zj7A9BVm$0bG)natG;t<I*$7$?i?-GzPh@lzyVMp7z&|hDaW!)(*DF+FAwqJA-1U!U zk^OQLn@&=4@HKaTYT-9D5Z+{HY88M=wHsRm-_VwNS#60JV<N}B7DIon`&E8zbxr1L z?V<7R4U|{5JGI&I@oeSn8Y4o!pGr`tgMVEu#;>`x$4{{?fpmM7<Pc>1HrmAJ3Uc+r zMIjP89N=@$H&jdYap-J%^ZJ>m=W6dRysajdciiDsgZdg+30Rm=fYa*t1P%6YKbqP{ zd@gsHZEA=Oak0z9GrsJj+MYTRAC5Z{E;IIMW%Qtb7D?use&_mh<nFpD_eSl)gZL!d zp{b)D=G!Wfo`N_5oN2~XOfRaZuLW*>8#WytyFFT2$m^EewI63{%=c-$hoZ-aHbeC^ zN!G|&bZGL!bo$=*^<7<q1yo!0>efCT#!XdesWiQ6+`4jX|MDwst-~YD4_7HNp7_ZN zpPbminObrUuDi3BY3J1c$&q33!7py;clTEUt^Z5X<U?@tAcunb6BW6R!$PS?*yGJh zM`Kpy$X*gm^}<Ddm*Uzt@Wn*LF(1$59^+QJbm>U87MU<+I?pq|m44q6K%J&jEL*v1 z`TlZ^`uEwuG&TnlylBD~gyj9<dN_pv#s=3JJ#2>BRVa-}4OA$Be{w_Kx;5+8lpIFf zsH=$@acNGyC!Tyw_d^1$oU>yB{*`+;Ck0i+Qa@^Uk;H?AtCyzRS*V*EQ>IF9U7Y0< zo@0$N%1yDk6CkVWR+OJ)8i1;6I3oTz3L0*kXL;2N#qs(oH*4(D^5d8VrKReYnU~>s zbnA(G6?#~pf1z~0tyQ5lizIcS5jRw$t>bnbq(#&##DldZR<^W3hOO(=5omxlWqNN$ ze%Gd|waN9i_Hyy&v)r7qPAdU_cizomd+@C6(TrbK2EBWz#O81=8b~lx#N^3X8A29^ z5t!`1J4Y@P^r;i+YnEu>4Etsq(PcESH8FaQQWqCrjNQ4mYT6bVjRtO(w7v!R=c6l0 zM<;U2|Dzrz54LYFUUdpM)njGWVh3Gbnbq358!F=2W{uufr85Qs65k1xFhBCGX6_i0 zjysVs15wCsZe#`W`*ko?26J_6ZTcG(0mv>z$z#hdG|)NHHf~vcMkrwwSu204;&z_e z`|d1D=l184-9Q5~mzhshRL+u1ksrH|hl-%P?Ox402bha~NPL}Qu{q1Y2xGJ6$Agz% zzFGUJDx#&T3w}5I5&Wi;&`hQHs*W3sy=$*)HEoJ5+s+gQOGq@f+Z+_bxVpdO_geoH zO*jT7qZB}AzPBNW3U_+~29$X+J2Y^O0yHqy&|ey3!Qy)&z2^yz1S+<YRZhL{dSa@| zz>ox2HQ&qZT^Tfh=S(vs6-yA+^t>_U=MH~Y!I4sMTKuEd75(|FS=v*bDwVSWmJz{) z9nl~$0*mVP%4bULDQt;q%p25~8c%C(PZ9gjH%}Z?w1Io_?y*=u6(<NF9<)^(jXGD$ zwl<IXy()iq;d*0JcpvJ+MTSQTmlv|<%*=QqB8fFqC)Bp6ivsmJ>DTug-1z4Miu(3v zXUW;!*c;*;&+<tV7Hb|nww~E({4RW#EGg|9GHlAyOqM#9tBj2+KNk3BcQqJ+dlv-H z0!hwj-OBzkprp!i_#6FK0?L07XZ|<Yzr`&DGRzspY^5>eswo<H#8e0-Uqi?xjwUw} z&J*b5x|~(wzRAwz0r$!0PEvEURmFel)mYwkL{@|bp8Fz~%IXBnHLoXTeV$PEd+*k9 zyf-iG5}X!SdVK=Mr{R9t=rZdCEiOaNc=4%(YXPO!A1})mDs>QXYP^g!w6jk8)LmuI zgP#S-?$WnwB3R<h%GDUN|CqLq28`<(lRk8q)g<V>2nm8ez|+^FyOss@q7;;rePLla zeV<P(HF0gvPA)5^=+b+4ZU<EfCvQR#m^I;C;@vFo>pByrt7w4OI(6<&?&kE5VD`0O zSL_oXi|VWn<wdLu6T*!WvYL?}Lfi*k9jX!w-j-^WO}LB-uh0<02(sLTzbx%BgRyzZ z>{Bg<Z!r-a>+E%X%fR~l@p(%~TGn%FZS#TwElF}uAjwM+m?3m?Zut0ab|^EaJcOx; z&~~Vc!>8uz6Mx%v(q7g9kvqF#%3RmK#+M2>wj`fypcJZxk5JU$cNNWznY_+yIkQ({ zX-8i5u2C5%ykpA&V6VyEQzoe=_Qr3`NS8y+SXo&c7{(%}R*(~$t9VWU0sIfX#US}g zCI%WPg34GOENlTaPsG&ps|U^GxPWbQ`I*uuhgg{hu0Kc!>7JUk=qRb^=Q@66@=*S1 z;?(oL(L)lR{e@F5<^T;upj0%ePmu)3jODt7^1$M^$FgJN{_X*qFPbKp`<h!Ga!v#@ z2;(rAkrls{YfLlHD_fbi*5V!KFNoSUX9^tkH)nTJB}4<Sq~OVo3|T*FZ;MZ?qYB1u zWj@R$*Kl%R@0&8Ot!AZt$8O=E(1Zs3(12!TfYm7I9vmkO7f{W}jBn3)OYASE#D^0r zQr7**;S5=_-V!^NXh8IoDBqmiC=93N*Y={5((jO~(0&@g<%p=}C^cV+<4AQmtE*JQ zi6+{;FUYqSH2**$Mw8Ww?1IR~^p1Ueq*CXD+NRpJ;yKTq(?0JPtCVJG7J+H9@@lt6 z{GY^6xs1u%CH+uhYfR)wxg7x$ap&GH8PvwF3@KZQ=;~)k-+%rUr|yK(Xl18kyTgEB zz1BR#X{uZQT9R7XI(Cs5Wp-QT#F^j{zO>cxGRmpS;D)Dj`u?C>$)wc^4b6esL}Xhk ze}4;K+<pW-Zp;@UI|W=|D}H9S!Z|ZC{tivL!O&i|vcU;Km;Y`1#w7Q6%Aj{2ye73M zNWWiY=J`>cIo77Py17CuJ&v^6JR?!ae|ek7!~Wy?PeDr~+0oUcmpOLi!VU?Ou8-Ox z-Avq@<nF3a>cd2tj~KvrvKoR0o)P)){UfGI0x?zDe;-UWu&hx3z#c{=cjb1%fr{TZ zHqZ-!f$}Wm9T*5R=XM_$zHt#(n6?Z@=w4nxEge*r?1_|r9NGrmY0k!x;Gw|+n3~H@ zZ(D7a$k&1A>Ie<gqJg%4lqk{_L^?ZYz~#I!=;p>M7lhQHI5PA-Ig)3e+kMuTj0$O+ zZ~LzQ*N)Dp&%<b75jBiL1A>R2>>0LEiSntOrw7P)$c_XaUHLm`AZh2=n0cZZd5Q)m zkmuFQe~|5yx0t>gb6P5Sb^`tv4P}raaxmaD{gj7f{kl8~xh}u44Thq>&cGKjp>?dD z+n1a>jnPUil~cYg`9EEj^GjlIRaj{`Y2qlIiUfwM8T*@e9Bl`J7E5JUklcso@~Lq4 ztH9Joem@semrVGx4%CW+g3Bz(UzY`fbK}o`kNvOG)U+m9@E5QKV}D_QWmh)P8GWo) zl9Zww$o<!=-cLd$x>J_T)UO3}bGzpLhhjwtQcUFktjAw%0VIQ)`X{_~=dLf-V7#OF za2MBANb-fK9!#}>s^4hVNk=~NcequDmH0<E`yHG8ImZ24OiYIL3!3}KC~I}XsSBRc z&|*x9qns23f(mps%hCb;sJq~yyJL3BBe+&6gSyQxe_r=+IZp`9A~c3uJ$DK8FojJp zPq$y5-X*~tSzZ*CEvB#m6A$IoJwkr`b(G3kYB6#!@?1tG_xaZ(_?s@$0q=hoVzrhn zgSrcD?gFx-;yfR894uI#7Cgo}A5_xzvUcNW^H&jt=fVC45h$YVHxXKA$;HTza>M79 zoBNoglyH1ETV??bDEvB#X5Ta$4aiZS%c#YCX%WWT4$6@DRYocDrQgOe4)`))3bZfI zvF{VP<l@_HOTUcCNA(T2Eq0uXFikm6Y@sguY5T2C<CU|~w<Lm`X=H<EE6>sN|M8Cf RXQ$wwV|B{^FWBhu{{>4UePRFr diff --git a/doc/modules/cassandra/assets/images/Figure_1_guarantees.jpg b/doc/modules/cassandra/assets/images/Figure_1_guarantees.jpg index 859342da5e6555ee08d15a14f439f29112fd4f55..85db9b7403aadb48f82a79c9244ce6c6171783c3 100644 GIT binary patch literal 31418 zcmeEuX;c$kw`LRtK@gdhDJsf5sDLO!@;U$lFGOYqAtEA#2vHFU5E7Lk$RMu+pa?`} z86%K@Kp;XGR77SF5JG|zlO!UcBm-SvcYoctd)>S4?OxsYem`!(p^{MRr1q&(`|Rg= z_O6YO8#9od4z~8TkWHI5LC%0bkc~;mslUDZSGUcZH*ErcH*Nm=_CH^3{#z&iq=WzE zu)iPvukJTqL*%w?3fdeYwrL+^v)m>zxlJ1#5GVw)Y3qOM8}i?`O`FBGY!#Q-CMmTY zJb=0rvU!u3*yb%_Teoi60v;U;{vWbMZmay>Lnp))T>K^WT~|DO=Ry89t&=rvO0EN( z{ic_0+?AA4R@t>%^?<gHuAaV``4Nkw$1G2swz0K4WAEVR?&0Zm?!0%v<-jXJ!Kje% zn-RAnu~E@+@d=5<q<hH^A3e^<%zE-PyP&YBxTN$&S$S<8g<9Y6s<Ek^(b3t}{pRhv z!J*-i(Xo%?6D;=Z=PzHseV?1>{#^OB%3A||um8=iO%SpFF=YSEu>Z|2Inb`nTegU8 zk@%Zkn>I&+O-ydf*1d<s<xjXs_+MApclgdW#gh;6YuY5WOkFujmu?J5DepI99pL`W zwEt?^zs<0_|5uj%55xYaT@w&#u}$E`6O)6$AtI4(K{Dh&+rRG^7|bqDZs)BCGYS5( z+p%t)a0<PnaApm-LvBdGt7G8|kusOifxHKo6WFWkS*_dM!K(?Z`>EFGj)x4z`lSdn zWhc}XW0pnMPRRljo($obpF#F=7lCuLw0%mfOZobIt0rK*0on49OGt;wpL$W68_FW9 zVVScl*i&7t+WnUvXVi~Y(~KF;uU@4}YW7DRsIT5!>Rhj*ku?+i!K}5OdpPHFP7aE7 zSa8MP^<dg;bVfkj&Ar;cQ4Zx*x*lm34#iYetQA8=*D_r~%Bsex=vz{k|26kdp9_u8 zx#j7bU#ZOXKz!`FS?p=<zdp&=5Habs=}TW&%e&F^I*X3k+3kIa6N|82z|$G>PQa}T zDI?OxS#Y<$ckM-&U}stM4qq|HQHrd!(GO8ixKY-}W!3gCp~gXIR0rX**SChQ<*oSl z^jdfl;c7adr+jDTxDRQ<(O&pyRT#<cAHW5%)nZSvEf~z(t-D4Afh+@=px5?fjRO(7 zOh>d@Yy^cmEI)lcc&%$>i-CYU57XIz+<k@(5=pug6WYSl1&T!W4=VR3lBlUBEHh6Z zH(qAIW%de*#;p=)kwM*v+Dt?`W0zoud90})k@dAgrjhHgkY3p`Qn=^U%9Oc#dp%sj zHutKnKJsjP;N#$t?y+YX4o3sMOZ0Q}JKHrUs&o(%NG0!sg?cQ#p)wnq)0zG}<FXm< z4c@_$OzrwWKdPaOIlpm?r}&bp-(L8PXBOmKayU^_KKC3$PM?B)(O+zM`Wj)}NJ%h` zAA(ZsmQL|c5ZdhNGf&~0TE#?z^qug)zBqcl*P;eDa@L{)OM?~wb{*4tL;F(fIcq7D z-cxNovqt)S`+?T|Z6))vB@-cQ6St~R{u;v0L6nzD&AGl%CZWw-v=2WF7vF%)<U=_w z>45eIq*Hdnkquvpnt|U&N{hsBH-seE0UY7kA9SFz_yjl7W~6wGwtbeo8;dzTRLFJC z%U<-nGI5KNI@u`}Ww7N&N^qF@{^G?N#h}Cvr?36R_QkFdA)jBp$1A8gNIlANugYXE zxO{yaTd8+++}XX*jVPPS2nZhTyjan#kdQCTo_DA3M={Lvv&Pz;_|ATpr%`bU&OWbV z{35X@3yHs=7cd=2O>7J+h_L}t7dUaz^DclFiy0?yoF&P>E&RStN{YDIZ>3Y{w=ls# zMT_4<TYqhMkp^p@X+xo;$1DzSP285}<pJ+R#=}F96F$z3L#v3%0FhKPKX3!mc7#4t z=1uQ(!Z4bG+AtF_GsYSDgf@M1C+$AA&2V;5g4NDZ!j>zXu~dOvBxl|r#u1^^!`(4? z?p5RCDWjg53-&?{up3AGcEL>`fp3AP@51lKk8eOU34sU|ffxYghO>=J{rQ%k0BOLD zO-%Yl7PDcaw<8rTI-pxZ))a!3vq_27u5T-+-_1;-8#K{%Pg1d8Ger3hFV#hQ$`uk5 zf-D~|C6zk4G5C?5@}&n5@@1vFhKrpGUX=S8={n?k=hbhHKeT%qs`uE}e)vVf^G`3F z$;&OtY2|Jay5&Dzc}k@YXRogn73zM-!LI`cxH0V{Xsevy7%(>DgHb#WSg?ulH!-Tf zmC3GF#W5>c_6thrt-<4yo2s#H9htY*nvCY|p4Qf5yLx1v>FJo$icZ5{0d3ZT84Emt zInXiH3DfYVxIej08;}l6DtzY#Bxq`TCtjP2yjPNh#Fln8ynS&76M?GoPt%hBRHQHR z-d7;K@5Nb(6)%?TL>4b|sQRYRjrS3}ZL3A&1dbCK(Dx*B-F!Gdo^pJ6dBFDKU?j&O z&CR|Jco3oel!e%0PHNjtGoOWT@fBDmPY(`t-VmIsA2l`(v~auusK`eH@2!Hry?S4; z{n1Cy4D+b)cV1-uLutEoyQ=liqO)rY9rF~I;8(s`-Yorm(ZI^t=5Fsd7xScIhcNCn z7uV+t?~1SF%(3QvmOLtR>Fj+`cGR}`3?YcAj6WpU4{V<abz?_%%ii07$ckif76Zm+ z%}znxOa+1ebF4yJlgDUd*+g)YQGFpBsW7ixe=AZYx>J}@Rcn8kvZivKu?wZ)zJA#9 z%)R|#wRZa|3Jb;UY&T^LJZ3#E+alK{PS3yqmJKuH6q*Y-wvF5kNVi(wgams7l2}FC z#jNWl;_Nvpud|u)Uc4wEtqRT5#BC1~8FP<hRROBs{8~pS{N9U*VyB?0x&HLU6m1%{ zaohI|$V;Rwem9!Lao&K8&+^g%E*HvsC9K?lpqEtI=;9HW<dr!Bg_I;xS_qvHnzB^x z(tOxU-i+5ltjIQ|bj!1l`cADD^vG~7y{5ADLyh8-3*qiJuQe*8as#e#e*L8R9f_~Z zwK?j4@T0opwX?3z%Ji`f9=^Iw_z#l&Fs$+SfvS7mpQG@bpLJd`eRnYTDDE&X<B-(d zyUkxHmwz9Kj5Pan?D;Th&PhqI6Lj8ElKf!7N#P?y!c4xVjIfbDlaJRGwp#u90O!o{ zT0Av_0CbkSm*og;))8<qKkQp$L)hkzpU#VV+~)H3XCom$?%^V2(|X%-S2Xht-<eAY zbA@HC$MK^Z5LLoV35nx`9H+B%mL$0rywgB9-&@3h?{3w{T^ulzmB9vhamNzPHB6oR zacANf*@Senl&Rwkq*CR@=ElLj!&GZikA|VGQ^Vj`nljr0&8yu!@{y8pH`Tf?8F(;0 zWTV>C9HPSfv$eUBO-lKTH{Sc|(Tp5^mU?H?4%pT!*Xtx!AD3{DC+lxO79XV@8!YQF zo9`-tZ_3hH4BRzZ@Wtm)`-g0@a~7r8qB3N#!!=*H*9F;2TtxCu;|J(9%S-4D$X0)0 zn&=3=pRP#1%1nUQlrAoC#&~sr&Mf)9ilzn)V2P)&>6-ykhsX#BXERm5t7_&-+{+I| z*x$KYeVKA1>;Xw{Z!)r$@da7SY=Z&@UP}pl4Z&{kK75KZ0sm%WnWu!+uw&<gp}S~0 zd}E-!yX^>+gV-*cV7Z)vA4hS#{5Zx5oIa1SO6mcC(6pQ8r8G7DDvRE9nSV0(gr_o9 z+NaPpE$pgdng6^;RgcLhVSqwvTbF!M!Gg0pcQ@W&b^4t`s()PBQOo<)%De724dhDp z$$IJ^{dTN9!#H@NJW)*|HQeKwJF1hlqAr$RpL;Ck&kK8xwwu*>9Nd7698dy6`8Ndw z;dA;W`s4;gB)Pzy?10Nf^Z}O$6K<Z3!ZbL_Bry$}oz5U9!ZmQQa>F7q!Nq~Fu$zyY zhio<=$yL=ijZS_=g)WYEeK_VfVNg+~@k7KwYPMS9$LSnb5}+&FIc33)Oez#6&;vug zY=kMI?Sf-NL3f#LbkNrYn@HG+H}g;8+=i75w@gsKrd5W>Ve#(}RO{52Q<Czs6z|Hy z3x<Bi4rUWV=9sX#2`#%S>5{@r!}(3k-e|BJ51&4kr|R}4^^kGur30(j@Zl7P%4>%U zs)Bq2eBB=_j2?c@d756I=4^i;>(YU(x_vtq@Y60fN(Z(oy8Q7U((p*p6q4f{8^g;1 z$Q(CbnJ||SyaAEJcO&bGi^4)Ixh-!qFjt7P=Ja(gOD}9dq-l{~;3=@(I2XW)J;^@~ zSI1uQdD1#`kr_rI{n?V|_{!T}s$BHk<@|@oL6+ufkM&u_ea_b&=O49icmL9J^ypi? z!7|s3GtA5?)eJp%*HYK(8G2MH%7k07pH*Fa;qG=9*VNhG-8#xF*PUff)aVF*=SuaE zlk&)i8<3hcfjk=#*LqM`EIK9{V^XxK{F|bo4G6X24uu{MS83G+NGx(ft1O`1H*@Qr z{|nRvrfr#sl*DPg@bdwVw?8~lexbF5;nJAp)|jZ)xJBV!Hip<O*cmjNJJxOl=j{T$ z%1kLzo=N}?lsskk_`D!oW^g24^9Z}bL<*>B-IOlXpw}{vJZfAQa$*cFn?L&e9uD}W zOBEmLM-`lII5Q3}N;W`5PPmmVJ}-1uEaMxWiS;|1?-?<t$7+mCuydw*j}+=Z{$heo z>U3%_^xQj=(W_*k`p6^5H6pe=r-c^cUM!`MVPzw1F&HSe%_wuf>&^j!iwYAN<%FD8 zQ}Aw%LBaNcyW65S%zEA(uKZ-Xs><RsK1|>Rh!qHS)^T(ujjT=;3C>XJYK9_?Bh?A* z{t^PMBNH~cj}H}R@ys_M6B&Q-J?f7kwh-E2hBO0ytmq@M{%bMUbKyKVoM_B#rX+qV zQ=nVJDIQB)rpX0y-Z9P&RntxdaSwNQ)gsEL=68nd2)M7I<CNu$9hs%Qy0)8O`@K?A zGqo@=l|65$^<Mqh7ro}MkGkEv8;>2Cw>rIIh|Z$i_{>{!{p>!c8|08-1s{uzSK9i- zQ9{35a~@-IU;HS{Lrp&<{>CTy<f7-riz=2`l!`vVGVTOutr30<swfg$fWKVsKmhvR z?$Zc-acrC?rGC^(fh)$l%+X1sqm53ninEeKY3DJVc|MOiCff5ZJ7F<8qi8c!?gG}> z*L9cw{ZD)Eo0A&|7{o^+uSQq|lLr)LA{U8#{9C@Xz;sZg0WerzoxYoq;{mMnC2Nz; zJ~1qj5v>zX@!jVez_z>hYm&LyzBKqqw7@95_|x`^A-1XEBO9Zl9RwHMQojUsXU%#Q zxvhTp^&LEZ3+FCfKsls(>SLIlC){#R=O<;H3-_piAyJ-1#RDfRIUlo+Rvvdp`+J-j z_~LSw!9Ra8LQ^x_uTNIbAy=o**6bTJz{b|zr6?a50i%dF;7FkH;H#G;#&Od4@+?81 z=(hka|4%8<IF4W+aDo%v&2t5!_(z#IU5Zc1d1?hb885}1CbQv6zV&I=OGW4x)3le? z@1~0TNb&*S;7M?f4S8agO?n+0#M1!4U})<F?pXQ;<YM2>kkQPE?z{Ls`1g5RxD88z zFK}kHT&IQ<BasFYKiUL_iM!p-wxqjKZl7Wsq;KshKP7!=GX78(Rk(A&SLw@Yd3C+M zT&HtE@{975N_&wHDGQx!)Cc2TPxUkOy|GEdq`ERotLZM+lHZ)+!nnJMay`=du;?&_ z$AKsKsyB@i9^yWpiLrE_EKR&w*7R3eB=l;=DVE73ktYUt@UMv&q@8eojPio&+Fij` zHeDRBWB93YZ4>v7m@jXcC+&!^mhc5)+1s%ZcH>p%_ZwJg%8p6e50A9mJ(kb6p*>7V z0+v7Wd2Yg5(Qd348+wNjmOe?h6}DI@v6sc>{5JLDteCY~x8Xq<17=B!=GM?oFTB=> z6?E3gd$!xFY5GA-sLZ=h=Co_EX`hqKvg8j6H492gm-8;wK$qNC!Z4L?75oX0z@rX1 z^9~u#A%3Gbzr3|9mnveZUeC!cb~-tep?@q^6~4%MN1w`tPkl)KT5NpAooHl($WA*J z1H%Mi8+w-eo^ixjK>~R(-k>lSXANDP;~xgYcE465frc=F!rVE10c$bSEb1jm;$paK zEW&N`*h~C#qcCNGsjwX9TkRB)|NXA7?N@7K(F(&mLzKR`Qz`#|bk!QM1O8DTp$T?& z8Xkz4oJOXs9jO()CvC<b#3?;u<u2b@qHga?lE--^PgrmxnCXTBM4&9u`0B(bTITg& z6_+VW>E+Ro&FTH?dMozBuSavf_{e2BQ|3I%bX;q5aZ0$>y4;Za9_Vn^A@ezUsO3F} zq6!;*-iw;51O|JrcZP$5nL(4WJB#0B&*I*^K72vX?Mo>lONm(M@zd7oU3M`n#G+&a zauf%KGFyHQ(5(45{6~5{7~Agk2KpFG>e_2~{03xG#PVI0RxxEyyv9JQ+T6tbta9WO z@Cj%q01igf6e9=h=Q>f>OTPX?*id-0*2+p7lXIFp5zhK{#|$&B^%}|XMBEoif%n!8 zAiPVmG8kI79531lz!`K(SPDmEqCnc!$q6_kr2HZ%GpfIfw)x)7P*KiSSIy!D)c>WU ze7|D@a@x%EoVPm%>2)sn#wqFSRA`H&N!3`l3ECOgU$w|RS83zzOu6!%AN{h3<y<tg z9`Yfg*t6*KNJwpbOOa2)8HZgCN0szF_V!rHu&x2o;Wq;j6P9M>!(S_$O_6;Y5P`Z9 zm(C0CC;0a%;~65|AuAU!@GlTn3Jhcu2$%Zqz{Cawh-whN5xU?~*ot<^8G<ShiL4t8 zd(N5RG^CKNEGnm&%xNE|10`AE>atd~mZX7YKznU5wgle}twFxVjAw6WRb#W-G<T-B z0Qu}S)Rh+QSQk{~Qy-h2+-f##VjPr>UPwFDt=dX8yn!$qO!431WUy^`Y_e*LiD5Y9 zklBE^+@AyanFnVQ<=)ty-FfDy*0Ha}RWE<u44xfs5qJi9n;W=~A9eZW?!PN!c%lsF zao!nuKEu$Y(fMe|^}|)$M)Ee<>gTB$l^z{TAR9-QXP4<Yo}S(6)V+OM`;K@t`%$h} z(xZDeitd?;E;f*Vh~{aLB(Ql7oMKF@rQkd&QLP%B?u1{VZXJGFT=BXnaZ*um^p1N0 zu;NOzYL&33@|I<2Jt_3M)ykQ74{XGazx2?iXTo=gM&OigBK{DH$?;C#4SZvplNQn3 zQyoVLQUZIvVaQ!JR2ruXPSI=g+3~88?a+yJj8yDjpU&X|>bafmf-~I9rAvp~I4f); zQR<e%!GFAMQ8&MOviy5n-^b1mo^wjg8<0D#!c_Dn-Ts>y=U-S3o=sbSy8-bbjZ`UZ zts0>&QCpTWI#g~&UbBfnMWPKOmB&2C<SJiY@aa=f%9gk{@L)J2@ZyO*ac-mHZ8nN$ zH~qskr5OShx}?Tg;n@jEcmw<ZQ}Kt176;i*lJW%-c?`#RU-Y#0NKv+oNqegVr|OOt z->u6ML=LMm3N`~vo5Yvh#;T0k#4p^IeNP7TvhG(CP=pk?41DGZOtL8xDUF@9A8fS} zO=q{HKC1MVT`bMV-u1yq;$&KARBUf`BNxrqOfi07-E?99(oDKI&{>pF&|Dw1e%ymI zMx-)#g!svh-n7<n<CN4C_+il=oT=4>sxIOSY|Po0j~rY3m(3dY9qcRWE$KC)WiBRa zC!X5vN>uuZSajjUuyyVVcF$tcNEG#3H$<awYI^EtV2*Foi@s*Egwv~<&kilgn1SAz z))@8)@0lSGnLDoHj<bBtPp2#X{GPR5PCD-XDu{1OKlAp+>YcVuXE>dVLi9LkZ9u|G z$!pKP<709HM9nE-IpmYXW?9(J4M;=w&|9r<2;e|g#}bBROy(MhBp-3z&e4W(=lO=; zCqRk_q}~+JUJs{3NJ?~nf<j2$9CZ8FhwJmFw5!<ZMDJlYob}kJa{Bl$6Y2R!S+P<3 z3FDT69m0B=7#He50g!xuQJ?Dm^|d7v{w{2a2k>a7Px9lNeqY>Gz`p9-sS<oYp)Rl| zKH>q+mm?Y=*&eppL*WwQ1*R{I`)W<S7QemjOGEA;1TV*tq+6{7`^H#YI|e)@Pl~ev zNyn*FwOchb_tWf3885uMV?)l>KdWf_X5F%s^D6Jd&(Rp4czf0ZmD%tdiQz*JQ<YwL zuH}Dj>K08!N1tDXHRyI5hRGWv)_%2&i$L(vkTo08zv``9KzlSIrCV_OAF2t|g#|cw zco=Dteg=S-^a%6$hTyV(%nF&0U1mS#T0ASjc?+8{zjRmA6G^h~xq9iY129EE%a6i^ zRqT8g0`zqIsV4{9^d)|CStAP+9eLFt6uJ!3BJ;DJ0IC=zG87i!4*{h%fDB(BD|Xf} z)XJB;%-$baFWS{KT6p3y$AWmX!YHycPx}c4EkVR>FMFAB;9%nKINo_7$qX?bH$#YP zm7B}{kk4KUa#n4)2iu3U8G{iNK=P3~8|t^#Nmmr<1GQgrv%hut?csi?shamv%k9|w z+i0&rC`L-B|C$K7TIn^aZ~N<D6`YH}l5M!~(|O<GNCBd%M4sYQ#U(et5GYC@Qd)_X z%?~a}7D9lt<W%^jWhK(bXT0~^$VH#Z9v*bOo7XWcAn;G&yA|x8P)J0&#&eXwhwqAi zKM`HH>zB~fOT~g9FVG7v!))Ut?_Ztqa)OOZjIaJ&Ub@4#p2iPCx30=c0bU&euaUcO zN^Iy3oS!gPq-Q798^NjI-3CyE4Tz*HLstECqK@g@gM~wC8kdIkRPV=C&o^DZ&4}3p zyApc)oIri9m_LtSwGe(88gxYXz=*{evw0FqQfZ>E_{Fg`B=TGfesx$_UeJ0>7)M8| znM|7S!Z@VPik2-{HTIQ9FBa(1@L<^l*ALpmcv(?DNrpVhKq|MYA>#}h8)~^ml%cus zH>W}@HLHr}Usyc@?(tRWrl^u3Pqb~`9I2or+v#{|9XOHyYERaEzWE=b8T|ATNs+jP zU8?Csyr!rPx|wDl%UVuH);9?D!=PMfDn&ARfv|~wd0D0^9!?4B-k+`ean6z->^+^! zf^O^8`lFZKcTY89ijvyo^eyXV*$3(w*HX=p3|0F`%xN2kzlxnBLL4&{heCO0en+y1 z$=yp#{zdFJ-fYufA;>!BYyVl&cp*==;ZhyS@4$VH^JavJAYR3g`%^8FNS?F{i-&H* z`iXRWMG8Q8cr1huPd;!Mf7fd-hyNNT3wX>R#07_e*ykUyKCel#G?RR+Y1_2gkk%p8 zx=JF>yu>GXT{|N<wd40r)MzHR?9zv6zi(GA*)J@ts*CT~rEX+sWq#f3kq8C9C*<F< zHzt04wRnO1D*MU3x!xkLpPg6fea)e(;5BRB@kc?juGkC;caCt8G+D<WKxth=^g#M# zvkwRQI(_2Sp++-!B9yY)#9*0>XYq)w0ZrCAXSmjt^&}aZ+btU!Gz7DGda9@P1KUA# z8~u|si)>>;MEwLw*gk<AP*`Z(0=iLldOVdRj;(O|wD(o4sW;`}IN5BnHgzJ5iZ(qJ z;UBe2;r#S?TZEWlf1wP>ntWSQn`I`^Wvs&ahw=Rr3upqBwCF)bLAT)#qSw%$rvylj zIq%7O`fI|(r9lOQ<yvkZWAaS3`95;}SZAIF7nZH0+<5|U*6I<Flg_=nPBnB~tKtQX zZ9q0NM;zbq!=T+U(Xx`RUKK>Y<(wl(VEfQOhwwx}vccE%f`1gJd~`X9u+wTc=h<u6 z5ulK(*5Qqbhd~9$IHV+8_$)yQE&s%fECV3gvz?|J*&5+48SanksDqVZU;iqv3yoj$ zyebCx@^_<PW<U>HHW|)2o1PZEqmTs#D|G6|{y+ddgV}jn#S4(*@4-5)g77Ga9{2Oa z{)ZY`9Su!asAWJ;C#NewhMk_&ka?rK&YmLq9m+XNQr>_tkfH(67BG;!j=gn)z=>o{ z(eDyah)EuAbM40vyb^Vk=^@M^1m>O$UL#Lf1<hh)`q%{dk8iXd(4d^J$B*5kaR0o7 zsJ;!kD81>m*mKQ{Re^cdRL{W*4YFz^zZhhfipcz!H@}_tb&uO^a#}{Y3ud9h#tq11 zzKx>SN7DYkN9{_#zws|jFA!=;oyY_x#|=uHxK;SV<h8H%0;_xJ4$EoBC>o@K?-6Pw zlFhLZaP|5AxlFF<e5(-<#6BdEPO))QPxmhd3hnPBfZSSA|GTQFAk>QqU(|X+6?HXX zmBNdx_xj5w92JEMS2z)ljtI4^z2IZNy~)xtBjQHow*6j5Ot&L;Csq*4IKw`?m@*Ty z#SMte{Zwsj?4LG<Ht^(UBayytKhcN|{=y$)zu-#WEp?KuOe^N5_i&i^%+f5798pW* z^obX9s?)^E^NbgBmaSPYyx$BUQDa9`j;x$8yE@RL`0jh>IR!HE9-Z?iLd_4~3srz= z5!y_w!U?xTngFH%i3=6Rn@@SOV~AXf#RruO1Lx38@r(4PDAf%J60o10@?)MjQ<Pf` zXGwE|;sWKiB(uhN@90Kk+Y(i(gOZn#gH!3xd0O7ws`IzGn+B)k9CAZ2*-G+g0V#W& z=WnCvX$ubhe{95#JSZB6Qlati0L*wd+ippWEwnhx)xz2#YKCYkGa9mov4!l?P7~$% zOg}}yhf~<D{8p`HEZNF#wm0C9!O+cJ3)|<fe7#uT@3b%08hMu_C-8)aEl*CfzfL2U zOvc?rl2o?Phmog_$PATbGnZCh1iPVNN4W6Eiv#n~Chdq#3u}4Mn!mc5t<YSF4(W6r zNqX|tLYu!%TE`$`Xef7n1CnXLCLCyp3>^N_#m<jocyu8AYzmfjYJbxkeRG2rgE$6V zP;r>@d^EGm$gii9K<VyI-?aeOgol`@oy&UI;|C1PzN#d7^wZ$`A0A724O8hi#KBo6 z(mrM9mIUyb*kh+rT8_;N(YoZ};K2kb_wR%0Mwa$ACI{BnV*N_LQbRsa?MGhxE_JDA zJ`+(uZ$>fwr!EbqFRaJQ;cU2=IGPN1j2GQN1=YkF;RC^C*fCKbJUHjvBrxPMMCoZU zV8dTyc7{k&XF13Yc;Z;zUQWyU?+2Y*N697g?@u?mOOq<;b+SuJoNqiTAjTo{(f~wM zM8pg-S)^FU)qWaU_J@&qpX~hR^S_w#fJLV&{;Q=|s+ozxN3`XUnkyaDywd!s$wNkR z<}0x{8~#!3OHn%*CEurVyxg<+aNc>)#iRgrNZ=kP@vG;d?{rBJX{5-=`v$S;3>UMW zFLT*CohIVJkubvnwyC~Fs?~0G(w^I&fV)#ZpMflsukqJ^6)v9~C=09MAN~!G_-#G_ zbz!{vVc#R>!mp#{6TU7$vqz#zM@Te^iw*8P?>`ohD#>(qw&c>*BEuPIG+Mer#lk|q zNSVjuXgQj#^crkUR7^Yn^htt;t3Bkb#5a;av6NT9B`yBuxSax5>qbLkIvO*P)!8o) z6J}AGzk9Bao#s~4OOPRcMnEsy+FWiL0})sQQKi76&WYw-dAs|FhQElHNuKpnpP_Z+ z*QBkY_aJo()5OVeB<UmuUG|sakaJDtC-dbb+Ih0yr%xdwBW_>vs>*RHLS<pvU^pT- zz_TeVEJ7pf>m_l4>=Qx|S?&UU5=sf`Kmxj>w7Nb{Bc&yhhZo+V#Ed-lIfwrcjY!&) z#a3C`8WeQBVR3@lNl1c0sr0(WgWcC~GBfFtdtw_RLpC7KSJyrr&l38`td>8ZUUhA1 zozLSUd21bl#u%s!X;xI1v`)2%K6+(nC9fA;7wA$eU%!JPD7nB20>(7|OC#rQKtf34 z3wFX%kp$4pG8Z)vW|r_=O1h%J40uprY~{qs0yXtJt7y2gKp&`LogW#P8T%!xj#4!) z+<AWIg#$VD;fLxQpQ?RN)(_e?96$HLsy`c5I%0mMm7RagEhpsUg}OK=^Cr|3oc0Nv zw2}Bx)$Sz}|M>U<UG7(k>^4xS-9|%H3APE#MVjYKW3$=_6bVK~T^%<(_`BfkOd0{v zJYeO)ddfJ{psB|FkYSSjbN4ObII@q}K~^?d+|3WbYVoEU{z8%Jx|eoLiwU2Cx5CF$ z+isG!`#F|tZ0B7RmQxS?v<qp~;9hB(-p;iV4EkG{aK+ns*DiA22g0AP3+RX15SvoU zh?zCsZ5t3X&Aze1l|O|#CS7(K38lU$ouVw1(O9X+=YCL7Y`>N8AEqc;658Bg`xgkx zCU|uafttaT(UB8gDd0tHfqHQ5mQ2{W;;W0Rgm~slW}>-4tPY#HXu<n&c|hRBl{nW( z1!7oZhw66GY}mfupDKV`wum8e8htljL-b(-vJFgWiU1neY9J|Q1~|^pcVgGNCRSe` zj*a_+Go1@7j^+8dA79mQcE5cp2(=BJ(o>NX8ainIHBF8CjAzWT=<p3;vI(F%e&Tzi zs>G5OKM=i+=s+j~j%Nu+R%S@;GCR6rEqFNLv;1r251^8O6sNZ;>UzNxw!2GHhkm?s z@Zf6gRf#d~gVc=gE|k*oia97hYHlvF|K|!Wa`@WA#~KwQYyJzPNZ$O|=u}@V&6>3& z+wLt+WFe(ds`xuBvh?@$AR^8B`JD1!A5q@g#Deh?C3BXxPXoeIH1bHH{-GG@L;K3@ z)b*1$-!{0EZ1Ah-p{K)JXzubW7aK*$KhrjK32cCv8D@&#CN8;sIVpQ<dNOS*SD}4k z?Jg~l)#9Y;C=<?rX%89Sj72*(ZEMTaqwpL1YY#W6>`1LHXWR$pyeIA|H-nrA+be1# zX<FIGuuUem^9~SXu>ohe_gjuc(9_|y<fK_TVw^rZ^oC33RX6Go&3rZ>Qb7H=iCYL2 z+_4eku*)99OICdWWRR_%JmttGwZRo|24L?=0_A?H27ygnOwIl&@Z{Lu*%peNZCJ#x z=gbz;CnQ)_x43V2j|iNK$G<lr(r8+W_gP?W%=~uUOOutp5}Y~#ec_3nXATA_6J`7Z z9s-LtAe#p^E?vfMK*}$*<l1{D?i+}G-1?qw_xT48{yf<r9f+CTfQW<N1-&cM5m<7S zpRt!AI2%qWQE+{Rq!hdoTWHUP#tS@(y%+G%F|#?2bf%ZL$7<!k!%dS1KW%!Wk$}g> za{ig&{^u^i|Krba3H)%;$SrtH-{LIj+EOytB?LNfIWqYsJP}koT9w+ay$nI_0-=A> zdMaK8u;&aRIx6-t2<(bUdvk*)YZ?}(hp0~89()+d)WE*}p%><K=F&aNqtJ1q@!zxj zV1O)Av*R&1S&Ke}_PphBCeL<Q1QqpfK<cfy3fdX7%V`g-V8+7C4agPW6T9Q1?>{Sf ziwfkLc6z2h>TYo480D}ojToO562Y+U4XJK*d2x;t($g+7<n9Cbvnpu<8@5RbodWMv zCcb$ElydoOr!^iZ7xt-{UD=P0J+K-j&#<NFi$8lFZ0Ve}^W4<{mtU*+r;jfxi%Cd@ z;@3-Cz?`Y5U%!%vMwiQee?Z;GE?$_|;XjFaiJbC{y}ZiL0>Q7rkk!opzFhqMebJts zCm?sX?Mb~WyJcr|s_VcXP$=k*j)P!10F=Ki*b)K-OvfWNlQ_|FDmQD%2o#uvtxeZ5 zEeUrq=5JPICk%M!hDsmrqmJF7g|m-i<bWIR;Fc~k%zGzC_ZiU+8TWlk791B+U|X?0 zvpVfUtqy{?xes(ZeuxRN3M|4!3A3z%aDRGw+jW_2IT3A0GiC37=zbUn=LQi?1SeP~ z3O>zdZr}Zmv>2(qRXjj&AOw&nh@fk)w_E%IrvJoo!bk+xFgG(RG6<Xlo6!WFdm<H_ z6Q{m3iiH7HS}7dz>VWNNt3HRBZWc?hYw=9m@^DTY8|Txj(D3d2SLCXpPwvr4*(6%w zIGhT76zS(=O#Hsv7{2htL$Y~{F>(0E*mzcNuwW?G<=2#=i{IYGfA<^|yf{3?hqokb z#MyB>9468i8~Kt8troyu{t=tZN2dT1-?dj}ES`=+PDs7KTQlF}m0;-2XOR<)>-*%H zP^j<3yMX5UhpjZw$>iY&@KykZ)7J*Ai(LF&t)T|0`*ng76}UaXvFgbedEZ#jxT|1_ zF_;fDxRxop6oBC9FczK~S6XOf`5AI$6DTE~st>v(+Awh<F(FB$fU^ShI1D>gmD{jg zScDxXGU3;(4(UcySbA_~`-&S)U(IdwN9p-$^74WAV_6M#Q_j6+AG+tT;U+wQnEHYq zk%(W;bs7^vsH@jkV3A3OMD2Im-7?DADlJo|zzZE6trw-43u59alKkH>@xQcJ;*c38 z$^?eEh}mEDp6DceOqeOoGZ~`zUbwDI<fMqC#7cv&qgs{6aFoj_jsq|gti%cGK=o(2 zH<!6|M$4||6q15N-<k)OxU`o8<_S`k+n`AVF)QqPLSDk`l=S!sa&R4$lqAq#`&#~( z^x<IAX?pnhBhT1EvCfIGdzC0J{c!R80-v5YuBCnNIWFBS{j<^Te!rlos#e4~y7q^~ z=G}%n^m79%q0Q#gCWP#%sKV9hWC4h%?}p%qrw<DYTDL90B!RD24dH0acuePzV6(7{ z5~J(cHRi!xCf=M>1Hm`$B4;{hpo@!x19xV?#bXE1Y#VX!;ys*wM6s6Dy2l6RfTf_z z<?#VsryK8NO7iBaRPm|8eTP$LcXzJX+MA8`TD$!Q!ynhP|6ab^)33QYczV}>dAdB1 z?lJVsL>3$4Fl^o%b75q8&(N<bHQT{ZZTg70ud`@w)Ib9^-8CIml5KmAeP229V!2a@ z7yFw)aja{MK#A<Y5aFCZkz1G-szEB?bR96p4@Z0Zc5vW{gv(B{$SlNwNVYEqz27OY z8h3#!dT^aJr23~eMjVUimj8{LJdQq@_NP<GO+Dj~3~$G8BfpI%zh}I!7){j7a2SCa z>Xk5mN4I3%!ZH_2U!dVsuTBC;O1Kg<K*^jW(IwOzW0fx6Vfo$|jR6yB`W9VSX_rFX z+SV7>1K*#SK&IezusMCRy%&$nfoc^eGu`w@|7|jTz9l!egoHx-`=c=d0Rfn;xfvNb zaz5VYCwxx7ob;ZUJXxN}<F+RyCFQ%E)xQl<x<q$v_eafehF|^?siN`Akvd<en5>%L zB?byse*3ET6|R+AHyb_lOMU~Ez|0*3fx*AZP0RL~OsI7{5_Lc+L<X4YHy!_Q5~ufS zNC4+siJ0q7IX7bSmU%IwR;Nq^28apmpQE2h6f{z*buVx?rh=bv7acc)R0ql&x4fjt zZ9q=$L~I@2Ipy7})JuKu(ucXe<(`>B+hNIl*}DTS)piWj*494Ub#TvL=oMl|t6INR zAggaD_EcvyjKqd+$C6Kv;4Ij_VqVl_zQdTA%#HR~lk)?s$MuGbr%B<Zz8BieoTc8X zFQ$S7R1%@h0H%wkZ#Tj32iGT4m?64hoqivFc}mNktFy>tSK)s+Rt117z@jNT_T;Ps zU{~2gxY4#sRs^=Y*Eaz%Y<tgxCpaJOlwUA~(bX(Eu4^L~%eyrkU|&LY!XC|;@S`wa zo8Rz_HM(OQZnU{)(H+l4b(e!k-KXhi>;wJ*(Es#6|NNKy&j0Jrv5`v#{1f29`53HG za1$254E#Qg11c=L_+=WRf>C;c*NeH(Y@6K85Xo`6a&{ddvxS(B$R^tdg&*@*OsGy_ zgM)@|4<`$9g{gQ2!D$XX!773~#Jk%-g;J4AfwNG+mku4Bm<dsCm27DlGP+43LO0Q@ zU%WK3h=W0KN9+2nOaqD7YK&P6;+#a^$dBD@VWGeq91n@5zCxV*;AfQ$NG(&K(ASCA z08+S#^C~N5gk)xe(sYV=3nz;e=P188Oo1neWX39K{*@gEE^C@_iSew=z2U5dA|%CN z=_uOdb9~LJSIhCP>?J7G<k>)!&HxG8aZZHl7IXGnG}6zzF@5N_5%QTY=%kcZi9P>j zxQ5f=Uj?Nf6XGo3!w<suMJx3s!BMgw8Ga+8JwSNdI{LQ-T^%Nc=6SA1F3IsN-hc7c ze$=TcR;%Kz@>H>4tP3X1>7C#5bAP~Epwj()3059bucFI;%~!w&$YzUU*+&>-Wr<bJ zEjp;dNWa*jpCNO<O&3UaNxNYh_<rWLRyEvBz>aNzRK}SCbu)dkz*?K;UdlV|Le{c8 zsz1b2_IlU`#5%E1J^Z)Vt-7swgY%MAVe-e=4^)O0Kxs^y>n}`(Uqz~jIxL3^7`zB! zRfKF3l;XAMV^!KS2*Q@+V}abDB?SK<YQ8#P=-0wz$|U_BOrh?~Owx%8VDueiV9xV9 zJOcgmlc#Elf}EGEN6fF>7SB1db~gGgshCxx$aUBADp^)d7NwWtK}`R*Kr!dj|L<Zt z0~Q(3^;$Y#A$rTK>-|CJET4JFB`nEtgF4oAUX#I8!XM8+7+HmaRRC=LX>aTw=<S-) zc&YwM!&SN>kR3aaf|0?hc|LKdtvu2Ch5T)n)ty3THPf%c)kPC4J;GwX6~518mOlQB zx9fJTQxUd$UB=@kd?JolHuQ(A#som@oRGpAw2yIms5BGK%GuV2&CE_y%#WUzZr$uT z)@+eqUF`D)e2dC6J7xH8^&QH^QRo9jUQyOr^MXg#%RFx(i9#CBn#qc9RUAiW6H}>t zyHP7Qt_j=rRXNUy8^B(>ACbaT5qJO<?6Nge1bIPpY}CBP)*~pQ-tkT4j6rDVW}W`W zF31-rcNTARi;wbm+W+wT!&>C^-;{6qj~;(BM)lTD&QDi?limHy|H0zC>F3xIk#9W@ zr(oJ*m@?So9dWPWsyGR@$qs?PFsD_w2oLJi+v&lwi48<Q06n|;uAa<?<cQN*PNPdl z4$p(4;f{)urYFn0z(DIIX$e^h)~3{9;_wH1o?F~$-8<ojChdS(;q)7GoXWwJy5}(U z5Ldv5z0A-!YnHax_jqT$MOtZ}uIKPgw9vvM4VMDguOtK7Cd#Cb2D~4f<*v~T%JMjp z2aE|6IF7vgDMp$JoI2jiyW25Rc#z+MhIM&c8-PGMJCm@($|<6LTA|DbkpJ-`wB~|6 z@=(W9LVZCErI7zlYN+8{B<P8dB8Mp+>=Cg>^scULn9y9k4AleCQ$rRzD|~Go<jE3Y z{r|rwv|tI0+e*IEscWh900I<eD!fPA9mS@n(3Mx=vVucg@!}#Zzm2qmhM0lI<ry?( zPR>fC31rFG6&K3O4M@Tr=pFWm+0>S2NuO&+Xih;D#=nLxr-`J12#cj}oUGR{DZrPr zERs}X!=-@jEM@`)x)p1|h{h}t_<DB!`9KiEGO#|qVD47FAmn@j5ka|Qaon`=<d64@ z?+MSN4VopMT)L(?z$$t^@4eJ%QSxpnm^vr2Q#RQ|?7m@m@`n5J<)XpSvFNa0_02aw z3G~5JdH@on2S9ns6Wa~ut;q5~jQRCLI&dD!aYe4%1f>$V2R9&rP|?+eHP7T98<2B# zh~Gk}U^D#lb;89!`pP^KP^Z8DKko`YYB(e!*lj={>Axj-;vj7TD+U?M9zkC_`Jegr z@Bh-N(j$Tkg_nBa-+mD|+cqG7{pU&j=ScnMocjBXOsktsj01@?kj2`D*TTsH`5>p0 z$-PgVfhNP`N3As2XhmTS{W4%Y*zH-5b#ocS$foY5^hUp1SX|4F{T%S2w)JI0Q+#EA zq3rj9<fqrggLGfLs|?(2V8+^RwuW#nyniaorpz|yalsFgRdx14BNr}Es6|YCg@0U` z=}O^;1Ab6x(GXs)Kg>#rE6z4i0s^|+fI$<<5Q}J)1kwJG_3n6c$DS=*7@g&41hNN4 zOFuM^n#b@z#~W(8rq_eHWp3mDRPX!0dtC`v03O+JFo}XwAa0NkQC>$7aR&MG(^_Lx z$1-IDUkjT7Lt}GmSJ79yJ$74;9I1P=cxF?;{_J-xAxA^ZYj!0kr%77{9{b=tkh&+Q zSkI!sB{{V9o{Dp>!-(Esv4m^YdEKr<UXP1)0u?i>>Idz^dfyqWtcfIR{^nnWB*IJ? zsjj7a=>Y%OI52C+xu44w>x9af_sIzChs<Q4U0^mT%|!EQ%zp3cToze=e)O+N-|x|1 zi7&hlPd8Jmt5kB;O(pjt`t|3lY^jpAN^=g6zez>NKSaIqC{?9=392e}zMd1)W&fnT z*pl`m<IG6fvEh<W-jAc^+%xp=*&iVpopCTK8A98Rg0i=#O@h)2SRdV&rfC9%a@A&J z?^v00wzC^LdrfEIX{+f{KwbB=SPCPq9lF`fSBo*G`jOk+xxJU*Ly3tMk9kr^X>hd3 z`x@ogjZqv!UZFIx;a8Va;Z#JMNrEaSg?<_OaaN=j$4(eaGs~5A0YGX;W;>{=qC`EM z*smGYQ}cPpHX!j};D>)2S?VWDcY8Tt1h3Od^1VR72n#IaCf=(Zb5BJzRoS<BxKrYb zjUyLVza-z_p(ol`Ll+A&?DUEq$l8zR-c@1zRg~PuEvJ96w*9o5^bP;uLZsl^3(_<( zp-1Lk424mZ+!mq@R(wQ~p!&KM!j%9^=6NApiEfOtKpa$x!?<4Ytrh||K#h$_pmQi! zTmQD;VdGeyh5khg7I4n9xjgCqDsscTh1a304RsH<@->0#9{~@};UA9<rsWe~l&U_i zaSy(v=d6%$#=Y=fp|p+EUpWVK-<j(UC!C>7KlG|=x<EOkSffuVJ0c=`w`zufRE_G# zkkx}DbMH)3FXvhgm%Bxue>H8?SprU)N#186jlfF9?-4OzU{Xqg2N=m|iPTQ<C>-Kq zqp$+Zqw5pw-GFea=q!Ul^H4950tt*R+MytVZ4clP7U?E@(+`6*COncCA9JhWyOXYt zYucZut>?`uh8gxtA-`VuUpGz2BWK<>m4RJugi^k^h9{iq`f{|ZiD5rcSBQ}7A#*}T zI?l!~H?(+d_p@sF>eu(Q)AsBQT{1~zj0@%x7?>m`1SiQN-$p968ly<N1(-^#S7#oK zqwc4!xf75pnA6mJHQIe2rt61ZpDLXEZu+THqqxc2aLx3c-*Ie76W_deOpYIf^B0!m zC2_h#IA8uAQ8$$H394vJLlu3GPH%?-!AK&!4#7jlEt^%{T6X953BczcDE#Vav0Ph> z&P}|B5<XQwIX}w>QN^qKqZh{=iUOQQLY$?_i;E8Eh0}w`hHhDPexoWCm_WtyV@~=W zQ5L1n9_W~Dugde@j;@t%vwH}eSjfAPYF)@vQVd>+K{YtpK8tu!PVyN*W-~bseSoe+ z5`3lzUgwnrJC1Fbu@IO>CesdZ>0ogXQvnBZ<Ozj=Tq56~11d#TlhHZf2+nKfpPVZR zp|NzArpA=OmYly#KCX2}hN~wt=1D0eRVyTCl5O!Cx*3a}QR&2_WCwK-)D|?AxIyVO z2^FG5JZwvFlO3_zyk<5%StVQ45X@VbUOQxm3d>Bfe*ZJ;{<L0oYcDFIF0Hj>R(B-P zU)spQG=A1K*Wui|o8_$X^1p7BF3!7FIhRsyoQ*%w3(|g%E<CO=Ll1V2lrW9OvE>nN z@v}#V+uSQ06%ga^^l#)^QuK$;rd}v_&~tUWLA2;s^vV->1En*_JM`_X`vgG%smg)~ zQ^)CY5cfp;uqz#sB_R_%-}82IZly<&ffNs4trInkEyEKs`t9>B7!=0(eXHLa|A)37 z<{}vZSa1YlJeCE6a33nUItr#ELsRfl9A*a`f=&BX6>usxyj|01uqvWO%p+8-!!ov5 zlsPolkoi+BrUxDKgx34$lv?81f}D|{EuUYCdfm|;5<V0B14mj_ygqvJA{*c}6+MI? zgPZn8%UvH{Pa_0BpQwFSyeKSB;X9M+3qNH1wyuJq!@EQoHj1u0|IIMV56X8ET*i!# z@i4;Y@Zc?+d7wokfl~sCxlvZS>oxse9lac8o9eezFUC@}%n{D_1?)}#u&{*Ufd;<a zGhS4ER48z)h|qQ&eibe$>Oo3?g_O_#rn$^ydEym?ajjYax=r&qm)M1fL)Kdm@rQ9| z02|IXw#<h|fzPoE9dxl(!d<^zEn{Y{eo!km#qP*0bF|Je+5a=GCiKzKkz##naMOcQ zeN9H5TS)HF*ef1|r}8R)qfSjY%AVKJD{55B(!Y+0V%5koST(RN553s~NeQ}bF%DFn zg!`Hiv(9O-kfCe$4P*m%!6H3u)d!tS=A4s%F3aOEz_A%JaUIc#QOCIfu``Io>@SJw zWpjSVxy+Pb(5-;$&mLr5@&?4_sd=9Q)7vM%u`W+<uk8b02B+~&^}-)q@i=ysniChA zu-c;4k*7IYwU0iqx+LaWH)<B@-5xQu^~bYv)R6sFVIf&>zpLfJ&xfkMIA6;R36G2C z{QO*|oP8~?_4Cl^;Q@Ya-N0OakJMjf&Kcg}ZpBII*2lDSf@G|}n->+9uVsu@_&P-! zbmQ#!r$v3!vIB-zPV8lny2Tl>TD&?;v5t;hi6oIWU~qcC+(c>e&C5R4GZ`<pQTE-H z4n2eNmd7{D-_>mmzO%=x7v5keP>ke-gD4LVAYOx&v%O_Zv;_<X76Ny(Y{698Eg`L2 zUvV>^Ugvt9YZ@>^sx4STlWBoPR5D}1vBI8tc33OQdspF=WXW*{>71EXa+C?L9OQKQ z&eSNk1Xa{h?=jC}j9!=XK&09A>uD(SVAh;R75v9w{_b~`6Vt=PI=!J76~!D13@CVo zloq%EL9<S#UZXS(uEOh9!+~F0gT|;BX`uO!F-otT^0Z@psNV*}Vfa9blM7n0_L1px zxsJXy_XjJ4+wfqfD$Y(=Y30W0YxmrM)Ga5&uT(*2N*=J6)YuL2FgOs|Nm7nTPf^h{ zc;qx@rDf8#w0V$4VQx7GL@Wk9^KNX7kF9g1)HftAesVT6OTDmg@~&su(GT9=AN$)D zqu;jdE0%IiJF>K-f7C7E+tE^^VuuJa_CzPD`9_75XY#c3;W8C-wO1#cZ&3WTTi=0F zhp>r0Q%I^K3lx^yTgAB+U1<&*5U*zLw@y+bQnJztI#cELdI%VDYywIvr-Gv`(x@%q z!!`~LdJnx;j=U9KU-&}nUB8W**;(gzXJcbnq-40X096vQJ{H%y53ACd4ZB^r&Z}aP zRDxA`PYd=<zP;63!JPAsuR)a?UCMO5a&z<1r=t0Xrlm(cO8uzE^adGVbo9{YWRc!L zh3?y5UrLh-wRu9eSIb~osTAc!?~RcvXLEne+W2ptIs3ExQ2pf(nV-T&QLXdIz}Raj z1(OET5{+u=0YNMS!uVPzw9Ye~<93F7D~;>Z4uuMiv-?skyE{|TmEM*t<jG~t@a~|l zO<?XC;w*Wly;T>OCt>4d-abB^x59Lkp^m`X-<|}N;HOICk~(EIv8F6~+^@Cd;~Nkq zV69VCCr)5~X{F|z>xwT|X0@fNu6eVfRr$G8=@Dv=LmG;@&C4}pNxRX&J%8C|K@&9; zrv5g3D8uprP5%VtQG~v0^0y3QpGo)MWhblCd|~H$^(JSsa}J<R+TXhAWg0q*knqDg zS;Yjg`=qC^s=j+^C!ncI)Foqr<lE<!GB+nCu*n96SJwZi_x$VAJ=z1+EGFejV{q#c z!7frKUX`N`Rl)CankDW2ZqH7a01lf#v91UufXp~-`^b)MB7==E4~VM%?j-%7X=;LD z)OcPg{>h$!o&)g9?2*$;J(y_TBWgoSzD4Mgq;<h!V*zVP!SBP^SN;*2$1;_C)L<#f zCbpP>C`jgzClFoW6LUlwI8`nNOryE7d?!0-NsIf)!Ez5LtF1SHU2>1(<;=ZE8~fFx z5&TJD&s9!);f=+f54Nz(UH@JGNG)r~b6_)Vej(>;S$DQQ^=8Y*cL>jigL=)r^Gm@M zZ^yMq!Pv5L*4bF!_11;KRIQcApDtxRLuoG95KLQlf(`;afi_QC1m9VJ6s8L-Kv?C+ z+4`+J{SID&0|lRm1>2P+lJepV4ml49^tgIg2-}$6K<2{C;kk9%P^CQP?kM~G#qQ+? zrpSIN!G%GZ9EZ>jhBoxz1yI-)^;vjns&Gd0xl-L-Bzdbt(0Du)X!aaj3Nt55zU8X9 zW_LS}&eo3Zr%{rBTRmB}d7h^`azS@c53ybSYRCyzsqwj<JWalbflJoDuU~Wgy%W4W z4k*q#SH|Xi`Y+9WXH=7E*RG>7s0aZ8k(yCNq#2be#N<;35fBg%P+A5NX<|f*K|-Dx zDWRjHq98B`h}6&mN=XPMyo!kQCZws15G1jLXH1!M&-eHIIA@);&RXAJeud}0@4c_R z_jO%6>|k}l{pO>6Hp8pRS@$Yl8onyA^{#j`{voAA>s#$aPSJg5FVSt!TlWf^#sWf@ z=&{_gO_6%~uww(keBfptl@slF7LKO`u6RbuHce5Ja;-QL^W<ldrJ6;K06!OHwps6* z?RquMw7<%9rNZ`N<$nzo&P9#F#^f9<`!P;%7K|`uy~9=oV$}fBbP+O(eo?9~BCFUf zN`4cJ_M53XOvR7p>7fknIap^M=xit3d+s^!q*UoRZ}#S)vt`Vq)GzaKi|AhuPS-!4 zG2Pz;U-3EArf=Be7k4;5?4U)*t)34RH3uj9>cftf=tag5X<zfHb*FNVwY8_#`c|u( z0J^v=HN4S&2&YVx6VUs8W#;fB9<+^UCm`KZUMV#d&Sb{nx4}`ayLRbY|M*N#!be@+ zlahbb`;AQlP6;mPr=i|{8&!o}g2sp*oSw)jR2yCyzHzR3`rC28R{<FmixK}brT$4` zALE5fS&sH8x1tPc3;dp0RL(f=f9J)YcdguTqPA*JJopj&bm~Xtc=QdYwvzT}Tx?{P zkfkCU!?tdud~-pl61sTR43+``?=9T9v};^)f{V?h?HIFgbs0?fVrmXw?)RSj{`}8w z;ST2&N~lim^?HWiFQ?Tfh$rD{kg|~JEGEF``UC_gKUKd-Ap-3!R2Y;Y1-NwH^Ul$^ z23COZKt*(9fm2HJ&aN`#L5mkp{;WM_;7Zkt`t*A8(tS}^4tspDX0Ty@<KLmL%U}Jy zxY*F}<lUt;dM9^+KjD7v&6li?(ZneB0&9D9{*`WjPBDFpQzeFu%r%Q&Gv@VO5BKPs zK{ZroaJ#JVm1dpj^!KjsQ#}Vox1t_o^YDyJi7$up6GM1|le%<FSejaTPynUu6K<sK z7vi!c=KCWoNc(_L(96fV=yU4Zm)Z@B^+y-(-#PDeOVKOmzN6D%ZOxZ23rjtfYtifB zuJbc(`$gGpi`SD}C}B}w<EUIpazo-EF6x!Zn#qr3btVX~sQ&e|Ak7|O^&mnQwgv&V zZ8|d^xtp*7h7u_f^Xj(s7P+38Zu?|>ROB1UFL%A)5i~8EIjmiT8-qXe9&w@VQS+!Q zv~`1Dl_(d;oK_y#E|KdlxZF|y_^2+TU>FZY4*FlYbNa7q*IW(Ojh?U6v{gkX1{B1& zw?<eMOjXX$;@!@Di*>F0?o>Jdxgf%-J^DbjM^JRNBoqe3j1!ZxU7x_2X-7ipasDU3 zNP6ct5bNNJ8(gJ=-|U-uCMBBDwN_QZ;XcO@kIY&QC`;aHeJirgIY7;IAkgmm#&jlG z#!bzfoBotC?wLJ)HX*7u!EGciA!2pxzjaOjyGT#6`=3oIcc92^*f1<?Kp)KCK1cYS z5X(eT$r(f=u^}Ad>5BE4zS<@=;g0Z|h@R7bjuElZnDUxpRLYOA{Lu6K-%8)V5q;X} z>HJr&cmlVF^cL3y`ndD{qHO7HC<2)QL~IW&M`tO6OE$udJ{H+eua%SSo>uycke=IZ zQL2Gclh}{NBjf`k{iZ1ubJZnJdh#ew4XR$}?|&JWCoB!{(O%6~Oe@an^P$+UqE|d~ zn(owX=hoOTnAd*{Zkz0X))!Yb;*E`VF!a$qA8@8Awjuk;yO@WOmdoieF|Hx%MpDlk zkBujsOAW^BP!VpG$MzZ+dwYql%VO#5OcvFV9}0$5#^kjN;sd0?(nHghl(_=YAJT`U zpCp%IT-7R8LAGD=%S5-G@P;T}kdP}tafi~|0Oj(WAc33=CD#%5O)!3ZANkNz+2j^C z{Kx)vi=%Smlk~{^EJ$#Ml>!O;D6{R53Oouh7e){MYX7gR#MaO5%29rWnZ!UYUJhc| z<164Kpv@(z&?eXScTP1(kix#egx<Z-yX4=WNHTzh1d$+3C#jO&BHM=e*<wEec>q%B z))zveqthh)NTedZTdtmfcNr&oJpn|&wd!<qG&?`j=jL)MRg=0d9~d!xwuKQATeawL zBqF3+E`wq#t%TTx03OP(d5D8Z!^n2Zpf+B!EF0Gnu;B%pJHNGz{&}dk^tf`Q{QyQW zh5IJ^86oKN`rvNa9$|`m1DqIbsc6mjt9sY!c9je5kN3?nH!K_<dcn-I$}=$;iHtS! zdEoD>p%Z8eA3ON0-qrZrVZu(^8aaOW1VuB+!Dgm#w-$Ep&cG-t%@Dm)keG*aKKFOH zDM?#NP|(lyyc%1P={a~&Xc@A9mD?OsbI9prG%1`k3l2m}HU#|Cu(Ng|BY0FeDw><_ z#&Op(klSb~qin>S{-CHyHU>CF8-ooenc@MYGD$tMOcjmenkeLNmsm~{9g-RXS37UP zg(s5&4=E(=pYk!+Z5VnD4opM1%pV5rqFN!DAHl}m!nSSjW5kDHFq`>gJw%n>p@&oE zefwE$pG&HUE^jf`5hO$hs1jBOv^0jwGG4N~Tpa=5CCYB$zznyu^Lgt7|6vvU?t5c0 zv1}1^Zrn#xfPM+PS%MLC41lyRxd4ZPzGO<-E{ptZx*W-hC?{-#5TGE`hchcq#6;k^ zC^<su^65#l^295Rm(#flT?)_l2i`SyjFsyaNWO|KAch_WjMi|}r@3OhgZLC_js?iZ z;j2(e=>d2zcT|fs25EiE&pJZ17eJY4hfwZ$I_H;(MxxvmwC#_;&&45n1)-cw3q`#p zlj~nU75aL;TXdgF8S+Vd6Y+=_!9>p6)H7pqE$-Xii@9J@p6<3|{CfGL7nA2!Ud0{k zvx$6u^DJep?qChM+T|eQ!}EmIe!2ACiOOZkdEbg6X8VD<0yiD%HI<xWpH7wBq#9&Y z`&B+mGDdwhk?lZ<O$Z92K!B>iMPwWVh*D2T85+?TETj5L)NYAbJh$vR;03m)+f}eU zU`whikCYvKF>V36cYYf=VQ;(bo-}stCDik;{MM0Wa9%(6qJxLj!<X&&FNN`EvBXGP zi?LZGCc<QJq@eu1)fo}AuL*ADqH<UY5)%$$VM~7kX0x=67|hik9tPefBZvSP^=wp7 z;oj8Rouq3mwiRA&xUw&AVa=`9NsJu=4r?sf?Vn8{qv_Zdd5Lw0M6H5B7t~8!5t$I@ zS+~5{6Y8@Q2TM+T2^GfCXOXHjE2vMwT4F%bgkYD1o@!Aj?Sp`M8_f`*mj0fV^H(hJ z>7SggdyQ$fS1Riw_s#T%5-z+=rERM}ens)7<c_l?ruDPIp@i=PZ@r%vs^$?WQE8N0 zx!Y%A3Xcbz($*-t80>cq|7ci4Cwd2Z{OUPr?uxa#8V%6LQm)e9zs5$O#7>55pgC2i z_l6EG#+6)Ej{DF$=zua@=uz}P^frO-Qb(HkXOkM{CqjWJLr)3(A@viBT9|?jb5B9G zSm)7H36}$%#l2;%h5dHGfsE0CISku#w8iRLsnexfXDrR!tuMb>_UtOvegq`~x14-< zFRQN%HiQPv?6}r_hy{R&aXh-xLgbR2@IbQw$+KPqHI5o-6k{$V4*-(g@Ia}xumd3B zokWANUxXStOJ6?4%S~224z#KWlY95YZH)azW&-{x2pdXQiNqn}Ts1@m(jJngY!uhl zkts>yQL@Tf<ooFJr(qM(l__+L(D>Cy#BM_RX#Z}^&ghFv)5{*uT~9{ubG;s|w6emS zUlO}^jR(6Pj_KCZM-*PNIb#;ZJ=&<#|6!~_VWPwoT;t)xakVFJo=Y=sYa$e7J=Keg zQ7$=i+|W<0)Uf!>Q;&Y%*CeW|yF%`0jLnbRg*{rcI+x9lFy9oKJ=XP$KSH?arq1z2 z*mzOA=A57Of06mfmTnH^VXxq4hh&PkXGq4x)57Tj;MYW!KEqyeb>wOcx|44Xkha5F zrF(@ly)J)udvI9_aEdc^D6QN3_{bVx7&JGiJ@+o)Z_zDH{fz2qKz+q1MgTD#1wMq} zv&4r<uY)k~5SBk8zD@#xOivOk^kQQNa5*?-nZ;kFig2?a;qy%31j+KRqWnRW^2(3) z%-Sd?i_Uod2<c^#X#x4&zV<BEIzEdfaKO&_@*Mee0OVhiFmQ_a^diwl`bYN(4q_Lm zz~srY1K~kC;Ntqopv{8D3LrRk;koM)%zLLb4!Z}3B;#Z7@f@mP?VE4W@yeG8o1F_1 zSpV9VC9y~O_l4`sGOAUBa9GqRZ2<9VbT>hd#kpE2%oZ`>u3+H}CRT~I`$Vl36;#G# z&rd<hW6|Aq9C3sH@QbdI?u}iq^Ilq<{3yJNc|gdqXyq^Ko+@Inn=iD-=X#`ip3ga8 zr{5n6o0_C1p7TNKI@{2ks@!&@MxIDIKank5(oi*}$dT_mqP<PK4)zE6+^xu9EUPy^ zudTi6+TknjNg#z+8`(A%UoCNN#NML|JfJ!MUR=w%M2#-n?jY=7Z{(6N;WSfeEuCGd z(GKhv;{yFfY49KyPsZ$>_#UX>E4=^aPPOFdOm_$-jkBm8Kc_Gl77`-Y?P1K&iP~kH zzmi4YjE%s<)+~<unun0N${QWP!Uuu5jrxSMP@Ozjra}Y|goE2gn$^>87HIIrJcRO6 zP{2=OO!&=ZM)D~rO)ELAgg5(%J~xvc9~G>Yz>19bbf1%<!3ze|qrMwrJ>n7RZ7ipU z&aTo^1TE_@ZaC2e%w!zc*!cpP3TpwuMHHa2t7N@!iLh0sd;v@r1mNEE`BT@ht!5jj zbc;VeBAQ7X%wFT!D@E;kPpLt{-?R3_N#9|1)3%uOy-76?I0Uc$7B%xe%cJIdS&UI^ z$I8WU<99BZLug-W-wfka>gx01(nfv7?2#jf8*BgzD?86BO)tXQn8f_9ot1?Hzl+g< z`*IoCRw%735WNEqi1}eS5ab!a$uJcE6KNRmGXL2G8)z_DDNB~{-%sFH5g2*gx*fD2 zi5QE=&uj2BGHE(6*?lU`49E1fJyj+c4ie`4&>+Wo&s;*Om*nQ`7XDhdHKzV_Ek-E4 zg=LeW9S|F+snpumGg#ZOS8=crlqa!^t|P1u6b<!4IjpwPbcu%;4UaOBDsih_s9gjW z6C{lzg2G=@27=6!hN~tuJRiiw&F4PWh<$12n6iv*>yT)#3#vuWU|zg%nTwRDL9!j6 zst8&A+|dl0IT0nGq++ii6lY-tQL>Pnfkz$M1neM=$FgY^*gjzI`$ou*WARr9JE=`s zc6h37&4I&{#=k`fb{_tY_fNkjIqst3W1VvEXhdk7kU)sPk7lNCuHI0KJXLaz|Hg=8 zeDP?W_3*O)98|CV^~Kw~mV7I`>3BO*#Uw(LZ5&dLT)y{W*{L%5Cy)*MrNtOkNiz6~ zc-a(SeijCES4Vt-_ZtT=65ZfLRGQ3?cm}=^9l>2Q;b1eCTMGw@pnjG{M1l<cR<d9c za-g7b@YyG?M2OaM&5h_N+lCYWOd5f<Ll;rA{i6GVpiJR@qBl$tYY-1)Gh5+eAVL|U z>k)RXq2t-~Z2-ZJ{l(gj#tTh61l6enomKaOPLH?wqniI~eEx#8bUGp!nuOBIS~^m@ zw}?#_WG_KrBLn49nZRuH4`PU5nMtyiJ|iXr;bb;T4Q<4$p1CMgEF8$!vws>f%GJa< z3FVebbTaW*$IkmN^mY&END{>g5W_&CE#3szId=ayYmh}(2HDM}|K@rs4M}=krX@Qb zj<J@O5`X%0)fX4p-YQtW#i!{+Y(&O2KWuwY=2gs(u}fFA!|Of|AM3klLq@ndaK3t{ z-aDca(CDt{VqPAkZnRJ)ZOJp)?%MV0Xtn*^bX%~S6waDDaxB4d#Ci5Htr3u4jSR!N zp@nWa<#sm1EB6mSFM2?1aBjJ;7vWZo;x5wH`h^JDb~!d4HkIeQ^72&)^dY3OO!=L? zP%PVC63bomKPN2AlI<lRM6YLjdvOXRy}!Oj`WCPV=?FRE&8dbqacWpZk>(*{Q`4Xx zwj2*1!1KGf`LnB{Ux32ZoU4(8b`Z*QX_@SWXo!c*Ltmgo(hO(1orEDwCE0%A>lv+& z)I^YT(w??H8+rPP^M0QhU}2|iu8gesV$C(|eFVlCHFUuRXpX`|fSM+Rp3B;f*#t36 z5rdDY`M{h9*>5tM?jk-diN_qBh^DClp}wrRG2o@)04inv=VnfJ{+<hjtZfe6t$~&M z{mtGZqht@mhd&&=L~x#ZzZCp%yqui$&!+!$89&fhc;J2aezlI9?Y&Y%o#oHmK=ked zkApQ$n06bFXa{nQpyuCJrU#r&Wa*kY2YrT4P(PomN~_VV*H0;&nl{%+i*{4Z0xYpx zxBhSV(;0O3Q|y;K?83V2b(aV*C#DQ|H)XNdU>0l$=;{6<7(5#%8R!JI&I|&1mpdER zkD5hr@%#;eGftU+V9)Ty+`Zz{ghKY`!iXnPolxIm2$zzL**8)Zp>d-rLUeY;uO~gX z`|mT^VV3o+3og<tk~44&do%~9goXqKTud$)_G<LAvM>r@YBs7%fW(g8<Jb;hw|9J( zxWCi@)Hl>PlU4sU^EF$mswBj`>F4Or<zKsp6S?<2W;ED9mziGHTHS{tV>e?I;V7B% zdKz{+aqklt!P!WI+JQ3x=w|j|IqphxL%|iFa0{izQ+TSG@5{C<i?8c=AYfbQyik%E zlBT6LMSSyg-@PLi&CSD;5tCUp&bV`>A+}jPn$^ygmxOVN!8Vhr+`5zXiTin8T`JkJ z!(Rdp>BaWX3`bp#5A{LZtt5518>MNStVgz`as`&}K4`d|EhxA3iglfkCDE@!Kg-^* z)WL14ENjK_?SS5AC#)~NBpZXa{Ils|vI|#(jJ6YpNkW9k6mZQDe?YCHk^{SuN*1fb zxHJ#qLE&2206uHuM`Ky&rH&(TAjLwgt)N+a7T7PgQ9vQUom-`C2T+OKfZY1I&7B;y z2~0+i@WHUGHj1J|8#dwvLPOZiG?m3w%&u_?+Lwb*nXk@7TRwT$&mN=9kGAMnL>TEb zolbFYKll>Yh-{aa?D}VuNK@KP^kCz3Nt4(u<w91TOgmyeFPr_(CS5`jSAGzcre`_P zeY_6c5*f6oOBmLN(EiIdy@Qik%3aGVNsft)(8j$(A7*Z-nv0+oyhQb@2rW%(#2j=N z8xU3A%H$cs0uMnWMTj^hc-qA?k{qdzEL9l0ERg*HRl_a;LW8A3j(_FEp+j=RpLQq6 zdz6NCO_HV(n6YDj+^ccX#;G4eyO|*JtE{Tt)cFi9+gxF)SKh7AB0j&VR1y?^(Rgy; zReUnFb`t$M+UC`<1mqTHwWf#0uyH0fc5bZigH8R`niGC?W$Sg!xSk&tpE%r_+8e26 z5@2c0KtSvVxTf{BXn~`s80Lw+h#FMspWU%|F(jMSQxZf<+=SNI@synsGXXPAXc@2a zRDu%RvwaUFWsNBGiDalI(t7&Ar)NP+0d=l%V+_0w4&V~TYkF%LpH(Lt1v2g-yrx7M zwyY)EQ>6f5h2YaMhA@K*kxLbOgvoSJzGQTEMS=38JF)hYS)xY&1*mPEHdP%EsA$P7 zN0q4TqOG&&b6p}|VAHe@Mh(yrP*~Y!sE;OqvSs^W7I!0~EgyE0gmOpM5E*4E-}g^R z!nnvagbso1)!8P#T(0FLI&GN6WNDk3{duM#5?skvsCRvSfxVtX+XST0N*2g0);2U0 zh7aXZ<Y4mJ+j*SX{WV%ub&y}At?LKs=gfM)NS_BCRyRZ%Rx|B4c8`rl$EZDPMDsCP zjT#5ouW9#w`_MP}Jn_xlt7^vFqqP^V4;{3WAJ!o@#n-&*s$8~>@2QIPyc$<B=2dca ze@#yP;u%pQfA57uH`|*2z8?_F2z2T%BG{hKLHFE!R<3nVEn=!I_}n1uAl^@U#o8p% z;33KMpXgzzKUi{-Ey6tk*u3t&f&vb7OQuT*W4b0)VKQT|8Q=2d!>1E|ydDp4v8;Iz zcf)G!!_RYn;6@$!FU1z5QC3TtM280wcskS?#uz1+ZJ{B9Acl?^`oax1D?LbK0&T!q z$ltY6M#YoANgV@=UB-XQGqQBz$0ZvZT{h|AO>jN0W#`YA&4tLe-w~_H2I&ZyiR7@f z1fv?Irzi+xN{=dPbJL3(0BK-KsJA-M>DqXwA>67n&TnXdo^i8hnH;@u9Bp5JGPBD4 zqQ+VS>Ib_8wfqWVG?ocYK3m-HJ{tuTywx(TFj1dP=-zv5*;^a*AJaYEO8aaM{ZjSt zdgt@g!BJ=LyYAjsJ^kffKdQ#<-1qE?TZI42^apkQ=+F7|s#IyETuG_%g70=u;}XC7 z<(7NIlN^5Y!vN2^-GAe#1E%PjsetUcuSQAEKL!K{_*k{*hV(IYbZ%WlfH!#1E!@xF z@fmMr3-sn8V5?5N_;2X`nMLSB-kM>ld?$mQ6BPKX_qg9@RC-H^r)2TdjE21{w5Hkp zSH}Gf(%+JO(!1DD{O=?MVLsRk!fGw-UB4y!3C=l$912nf%2ql9QtCG69q1V^`}KL* zz)i=LwkW&g%MlhAv?hmb?TQn=q+IE3K;+X~k-fP20yY_><;y_fxrOsIMQG2i&ao)L zI?PBExEhmbRU=RD9Isl+kBO-LH1jr63)38y6cT#8v^?wv9*T;I(h+SOd0jsAyKERK zIFmnHEi!{4A&x@^;@0Bx0(=U_`qS-jmYT&Pp1aPZWNZW(j7P`iMxCHeJekHANbIN9 z(RLBL?{IZ5s<|2`Dbu4Ppc%P4#M-#Po$@Nd`ffJ&^C<qR<W%Xacxk&TsvoUqvRK0p zPdcN1=1gthp1M(0gYQ?vUsbu*I(s)ivO4NM=dMVuAU6!J+a7wLuZvr?N$imKp&U4B z6TjWm?rgdIT7!Wbc#W-w{4p_>^a>Kh=I6^aWg}(szz0GJP6mC6t=HOwp{x{=CINRw zP)NRtcBv&z(U240brYgdPPc3-rwp6NzCEjY*tn7Jp6S$g$j&_<Qov{cLuf&aBWNOp zxi2{O1!=-B1SaeaiE1Xzob;N$nPwaz3i<_jfa9`)OyNe^F1UpEQX)^eKr80ach3_d zr(J?|CvRkjMpX?FnoQe2S<*YIC0d`r4>Haj|5cja^w+Bui1QS>qMz=m>4sR_lJXzj zsBe&n3UJ1C5);>PAXBjHAbm<xn1e;QRO{SiLD!(Id$t4uoDt;jp+)oa<DY>grVD5$ zHXsMIu9n5h-M`({7U;t7z=v$@P#(5Weu=E}QqlC_xP>QQrYM;*?^SrdyKlt%P~uh+ z{_^YdDaNREPL1caNKb~><EXKC&+^BdHG{`}<^;dQwrlv(lE>v}?P2Ywyw5eBOvj%* zOA>3C=Qx>nMT9iXQ`KLOv5#w?gh8PI+m4DHV7Wcx^bpi7tQ<<>0qE^TrW?L21WlcI z9&Q$!{OfdwhE;oyqDS$6MhFSjXQ<Y;>gDGx9yLsFO`k=YN?78Hq#=4Mih+WyT1BU% zouoZPO@VJ3_6oF#hy@?zCh<4H#+r3#wI=3RzUC^%6n3b16OYWYK1%DTfv>Y1r<O+D z#t+b<&7E5(u9#U2`@2MA-W-Gg8b~V>xFcpq*=77@9&VPx!S#;FGk^qQpbP;o^h3(s za+OiEqpprayOAnwz)4y=2xvUiWcrY5RH^=?MZ`#3Q4h1}NBuR%)`X2aX4L1IEEPJt zl+_|XNCBC#pVj0zfK0^+Zj)Zp)CH`5{I<WaJFpxKTvnBw($CsP`DW(D`B(<v-E=0; zAHQo6tl%I-{R&w{Uv~N|8*KmKfA4N+!Idr*GUMWxS5pB(6I2~TGL?8Pa;mET`w{b@ z#n|EdTy;MegzHpM(bI~IIt@9`WJ_bufxfep^jur@m6)x|Coxh3X&t%uZ28qtb)#43 zBfmFaf1m0d@ay6>o8i02;D}bInCkQHCRZOsnOvy6@>8eNFS(0NKmU9o-?_`9e`kVV zaH~tp!I0Az&$LG8S#HaIk!hB8_NbGgPSM@x2PPgr_I|Puk>LB5+c#0?qDZdtBaKsS zujPD5_`CASy^_V^uC6X8TJ_guu?h5QkR|X2v_xpO`kzfs-~GNI{y)8_b;mR)6#9K> zyYz|7ST;@CBQO+&t!)vuGH5$M<gyzA-S-0c)rM>rew)Phf)``nQL?X-oDn)^fp4z= z0+bLDKccIqnCstas(XhDWwVgL9Xb0|Bx)9zb*#Ayu8wl`Dg^yLozpTMnC>hL%adtI z>;=a~f8HLd$=^wI1S*+ajll6ZF}1zTIw59d^P3NR_FcmM2TSt#`9C=mOtCxrs%$QX z-5s`w-^Y4^THEnxrlrR9xzkrpMUlsVgUu;8$1XKYZa6UhVq%OXe9O5_@OGr)LDI1k zhBi`rL_uGJ?VXQV=Y;0NQNizdR?|r3r#$OXGISR!IF<bG>jPyy0z8f>(tqfiDztq0 zp#n#qA^$rtt)p!n3+tr8$$~oW&|rRg5%Yy`xwrJEJ0Fvhx435y+J@G=c-ydpQ8{w8 zX}?wP-(fEH|7^NnON<76xwB)?_eN6_(H2p%l#C6<&iS*I>cl9?CivK&5=+>CtAJN! zaB({#v<fp(mPoZxo}K)-#G?+bW6r;!#!?7{12>AI>UTxnn1B4^RzH5LN5GJ7w*vCR z-T!0i&QUtTF+Wq(aBh|TC8yoS&`(4OstPF7ESRY?J|s+yF#Yw2n%BU&62joyI%7}V z`BKY^bz>u}ef{S~nw{w_X&;<(jCj``{)cM)YS~xC6#XHCYI`>A=0uIFv!Z8yFSrn% zBPcF@4b&cT-(huN+e8<|z^dXvi2>N1vTfy`O?$wis@H(s%v=DSXXF-Qv=AQzlj{Mz zM-DtUQ)Msb@=2zIb!S2J@2&YKCM_=Po@b6HUo<Kgt7TjNg_o!di5#S$fV@P>VRd{i z=DBYry<u$va4`$1j8T{T3S-6lNTWF2c=y~TJ?Z1_q-O-HFNiErGp0(=-)GOTwaysF zZj17{dCAs++UbZH4)<BwQ(64Y|G!fMLB?ctSf`>@*_8Qgc6z4J+zk#(0=Jaq=bFR} z+96#39Tk%I=0GunrHVnq`OZQO@?GzVHeS4XdyNjzarsl<$36@(=@ILbK<B-M%r|5& z-sokubTCE~IF}2!Bfk;I%$t=rt22o>9;CCHy_^w`(~+DKe(b0vq%eK2HB5~!=(>2m z@IU?eQr1*(<{aWs;6ZlN)L#~lK_~c>xkt9>oA6P!C@cPAPf@jP)SRbh!QiO$L7~Cv zi5h;>y54eb&8dnkkHX37gVv3DPgS=yjf>QEE-mk9@isVgJ~)w+GjOUh^iaf%EDU0k z=>iwj><v*nC=XpFF<|qJ!o2Kl(6J6F9n3Wl@&rpD$}S%4oJl9`hC@!mMV%QFz!Ah* z_qr~Y;j2;l3ynTW)n0Xpn?&Icc|Yw8XLsbG+9385D`+b{9MDw4z@3=G_T!pZ4BSGj zs1d9V*az3~PM#;tv-U_FKxH?(z+Ps!CULjw?JV@m=A6TBmn4M>wYeW_xTi0+i_egD zM~z_KjK2yYg>-C>5T}T(Nw0uQ3H=gI4ld-Z^S^=(KfX%{usg3zTM{JPeFx+LY)<zD zX}M69i>WN4=Sb|O<-C05l^Z@~S0?qHFfX?X&oA&otJ=2Tr)-lpVgH@kHjS?FxcQsm zRZ%&ti^!AQ5G=S&#^L1Q)qa*T#t2lzKufT8;X{F^V33uKQ6{iXOj^=+p4rg261TdV zU3_|8#^Iz}KYvvAh7nw<YRZ1sd5&j{w)>qCyy}&bYlC*(nHxH_`u6HQ;5e9*)o|>( zM?+@$s8&IhZHv=*ybheLl3BBM<U%7#*C%H_HVT2WCcHbYMf-*&r@VjiMA~3$S+V`3 zrM}xy=rx(|5Y`AA%C>_ulDCnlc8+0m30wNH+lWbG+@~OI*jnJl?EGg_9!ZX<-!%<| zxnBG_2+r?#Euo;_&+IiuQA=7c-e+rd_s;Ov3=8qaoclavyCFYGeDvQUJS7Ty3BORy zAv4P42wtKgfqXw`6&qGF5eU*mf?rGl@1d}?l{=d1hhFx8D`HJnZ<dkGz9k3pzpd?y z`Z!@xcigIP;taK|eS*&=|9A;I0aR8e{hhQ~a!guDQ<i8+OMr7IVB&*SAZ1E}9QXMx zy(Bd-wCX|i>sh_!2@)#U8XXhg9(G53ApE?7SBU&^&)S-#9k}oF=cFN8XK9oqP>IL2 zmZ&a>{tKpxk#Le|8=L}MDs~~}tovx%a5P6(p5rJnWT}3mG=Z&f>aw(2A{wk7;n<F$ zqoTR~6ux*}F+Bg{Nd3#>tl4&!%k{fwT9Oh2uVA*QKk%tv&iIk*wllUUx7Mu;kp!*n zX^bo=6uXpIlXGfcddEQLOAOVP9t$Za<Bpv3s`6DCv<!7ysnjul5xlhPkf2)A`Ci0i zZbA*i{!x;;<$(%#SbU0T0prCuh{1riLICrc_W^IlRxw)gFZK$H!s<d{K(YtA=N2xy zH#o=~5_T2ZQ`U!5AqFOLTq7?}0GF5^JI*_m`h>y+-@1q>?Vx@B)E9SgE;BSW$)<k+ zoDARsKO`JKI#}Yf0mfkhJQ#;(6_NQYHcNwm8`M*f=FyJ13T}9JMPbuQcaF7Lbiy%a zA9_-4C7rr!iz|+?PX-<+&-oksBPLn%U_SprtRq`2vi{>61cG7vBjGM6-TG@J=%;yC z!^NOihrsNEl~z96g;QqwaH=FtxY$vsL=K62;Df!=mg4E+G7+*RkYRp8#ce_43pOlW zM$MLS+$Bctb4lRg1M-FBSg{wdvhY7Wv44u%<36Y*tJAr*60}!H$wvD=W39++gj+># z7y1oE6euY!dM&u)>#xd#T8Wl_p-(wI3cY>ry5yAp_D8|No;j#B8U2wva>w)H=hr)x z@7pRJ@eZ?2?SG|S<N5JP_4Z@IUw4*i2SVYF^8SY?KI&RQZ_ew`Y>aRJ`FH5r@ot`H z$MMq)hl|b1HO}>GKDa>~0}mVGnd`YAK>i!SxYhh-u?|=wMpGnSQz0tCQqV{?4qaax z;~t}jN4xX1;UW8(?m$2)vrQY~m&-{%Qz>xc#(=F}IY-x?d857PRCzrZz6{LxY7t!w zHwK?d>0&!~+0+-MYLN+U4cQvTXW_r-i^JwNV&5lMyq+sf;L;7+DDW@Kl3o4|a7&qH z9BX!gahs5b((D*GPc$&hp_v2{RXT+IheL%1bpF~6-jG1OhkLZbUW*VDT}~K3|C^IX z0IBnLHMz5!fB$!x?UqfOY<K!LXEN<oo#_VZX@u8-R|1{^IZ6c>$bNfaG?`VUp}s@8 zde~hdy?WUBF<@<8Pqq0JyzJJXVOw_oM}yo6pQyVHakY~*BSDIj1$XaNc^)DZeaTs! i$-4SEem4*gqgVgmWM%)uEYJU^S+M^P{5{xz=KdG61NnXc literal 17993 zcmeHu2UL_xwr+Kxo18(C(2{eKjATKAO3o@-XmU<UR1pzDo1|ok0s?|G83`geM+HQJ zBmo5!5fHeIo;jX3^X`4~X3gBN?kU#)_t#ast7_M-+V$<d7Dum-W&pXSnuZ#HKp?;p z_Xm#Vh)pz9R4jD#wAD1utKu2}AXYz*c6Ec21K{H7?WuQ88D(N>hC=oMC?Eg`AOe7u zjhFiceKiAsD;laQC@-AQG5%G)jsaXx01OML>!47->i;f8VdL)U4FE(3Cw<!1&dUae zZ{o1CpSSyQ{RR$GTRR<NDEt_E;wFg0bjR5KC+7bl^Ajr`V_O$jTb#`CoZW3*Y>#m( z4oCX>*x@kjJPt?rqV4=}cm#*poqb%;IQ$KVsa@=>ya0d`AJ==^**M^^Fb)%Y8tAFu z@EHK`iS7S@t^a_%?fh}`1VF{jJ-`!f@8FH%w&6jENJ~qh)b0G7?YzAObgXQgtUPT| zDsC?BR;~d6{M7mQSAg<(ZBe*ImJpGakPr|R#4Z2t<v(ovUFtsu$F}_`@mcqW%^<Wr zzv_Pd_E()-764M)xV5?etIqln09CgDpquzr$DRoQ*-Zc{`~RR1?eTtb@b-3>5fbwA z^Akkd*$5sl=<ntKDe$}GzYqQ}p5XC#zpowYtld>BA7^jW@uJ$eIlK9IqP*O#Z0t}1 zf4zzS;UE5CT7Q@azn<MyJ5M`T+)zfiU50kG$C=yJ7VV98b48(D|2+%;gFpMj9FF17 zeT@TDIfsDSLJ*MmQv=v~4}en<0obK<TnprPyPYR60>>xMm}TkbzQ<u)`;Yhk@q)(U zzCyjw_Ne1(6+Hu#jgP1AF~*&V;};$v1Z033&;w?`2DpI$5CxJz7AOK$pb4~r0Wbwu zfDLc}F2ED`f$Ja)M1Z^CK1cw`AOqxp0#E|VK^3S6&7ci*g8?uKCcz9?0IOgFd<FYB zM}~xuK&T+}5EcjrgbyMLk%GuWR3YafdJt2HCBz=$3h{wlhunbNg<v4bkjIb$NExIC z(hTW@3_{*OW+5w(Eyx}e1|^14Lr+3Ep+Zn8s1j5QY5=_qwTF5@1EDveG0=z5Ea+2c zCA1mZ4IPEfKv$vLFaRTh(ZWu_1YlAy6__^640aXf2@8fr!4hFvuo74;tQ|HCn}Myt zcHsy(6&wW@fXl+w;f8P<xCcBK9t}^1KY>@mTj9g-S@;J05RU|p36Bp?22TUe1kWDN z4=(~Q5ibu9i}wm|6mJ3VD*}O_MQ|Y`5$Xt2gd^e_A{vo~C`B|O1`+d!Z6p#&kK{wj zA+?cKNN;2W@)7bWvJp9iTtI%qC&FjJ7sXe@H^X<u564f$FT{U=KZL)8zfV9;z(F8G zpiN*)5I_)1kV8;Q&`+>Huun)q$VDheXh7&l7)JPzu!Qgx;S}L#A|fJIB55LBA~aDb z(L<tVL>)x4M7zY4#Jt2R#FvPDh+~NJiJOSu5Pv2iA>kxZBrzlLCW#>_AbCkLP4bPD zid2wPgVdHZgfxY;l5~jlBN+i12bmJt6|z9GM6%~({bXz81mv9LD&$t=!Q?6AHRR*u z+Z0q3A{5#bE)>xeg%q6>%alk;PD)iuTgsc1*_1CS=c!;+Y*Z>#HdHsLa;RQWEl?w< zxu`X$9jWh97gP69Z_rTEh|w6)_|v4&G|<e@!e}{ZHE5k_V`<B2$7uIXoIIg)!tTVK z6D22xPVCSz(kar}(M8df(v8yX(WB^9>7D2w&{xsFWq>pAGUzh+F{CrRVpwCOW|U>L zVT@udXPjVyGVwC$F<oQIX6j+uKFNIY+)4M7k50ZkxyDS(tjO%h9MAlMd4+|FMV<xC z63^1e@&QGIQbIYS9->-N8>g5~sh{#W_4riZseM*1Rzucs)>78DY=mr5Y**Rh*;?2( z*jd;wuwQ2{VxQn3;E>{Q;7H<V=h)%o;xy*G%~{2{%tgng!4=3=#5KiD!mYsV!JWlD z!h_(E;&I|h;~C(E@{031@TTzg^FjF}_|SZ5e1rUW{L=ic{8{|t0>lDJ0=@!80<(g& zg69Rp1*-%<3b6~B2|W;M7djLc7j_oT5uOyG6wwq37pWE566F=O5ls;tIZbj}?R3cL z>eHKId}4ND>0+<Nsl+dc-xhBXKah}=@RTT#SdnCxw3JMh9G9Y&(vgam>X62hR+bKt zu9x1Gk&yA0DVO;qD<JD6TPVAHhU<*onVd88a;$Pza*yR^<WcgL@|p583a1pT6tWcN z6xkGQ74sCAlz5aJm7XenR2EkDRK_arsK}@UsWhF1ojrFp>TI7XrK+K7vg))NtD3!9 zk=n*N@pISCHL2sNYpFj_AJ;gkVWUx?v7src8Kl{&MXaT-m8>;)p7*@x`PvK63z`?= zE=*nIxafMZN*icvXvb;4)#1|d(5cf!=<4XE=q~Ds=v~)q*Qe6A&@a^gYM^WoYcOfZ zZRlgzVnk+i$*90+$M~!<#(2g=$Rx<5+myi+ZCY)HG&3^GGut*-HBT^KxFmil;?me< zuFL+HJFhTYalX=EL1tlP@!S&6(%7=l^1GF`RhHGZwT5-7^@fd#O_I&3t)gwb?TVe8 z9ma0ys@zq~)n$7*`#Ad*2L*>jhc&bcIvKs`sP35IxZ|Yll<#!pZ0ua>g70GKQsYYL z>g3w$#_Z<rHt5dh9^pRYA>)zYvEixZndb%Zy6jc$O^qW{JwDt%w|r)O<$Y6pzxf&Y zJ@=>ZclGZH;0?GFuoS2kn0pO=&E{Im^;6fwuFnQ31w9Ui1X~BUgs_F&3|R<07y2ZO zFw7~eH(WRz6aM9f@r~M>CvS${oWFJMR#60LglEL~ZQ0uyk?=@#WZ#|BcOKn2jIxdD zx+{D)@$UC%o9M24BKIENJBqm)(-$idn|>d0-}V031BC}qFcg?T%zWI1xXO4`d{q3G z1dD{uM6tw-B*G-0q?w23A67qNe{}!R_hg6U(G=yB(p09@JE=QqwrNA@is>a8Co}G5 z>}8@eUq3$gxH5|~D={09?VG)rW0cdLE0tT2$B=h7?;zhbf98qalhy*sf`US(!q_5c zkx$Xe(@ReWiq95Tmk5+RE~PEK`wTqudG?{qvTUqetGwm8)bo-Gj*3()6*j68QW;RW zQRPrIQ*By3Sfg3fQY%}FtrM)vuRm3v(m>M?`-1Sr%@;?FfsNZu?oDgW_RaGx7A=!6 zO<s<)>bCa1(t6d=rq=edU8%jXL$0H~Q>L?~ORB4?TcW$NN4y8yE8dIkljy7Lm+Y?| zkRGTVJTv%WNMWdX`0Q}oh{j0w=*7{&F~hOf<Cn%~UfaH2e&hURbHaCGe=>9mZz}37 z`P;;4rs=GAJnu?pBxV|B&(8MD>Ca8h+sv=N_j><*;pQUAV&W3ZQo*w5a{Y?h%D@Nn z4-2dAtNUva>lEv$AGtqPd{X?>vthEau<5yZv=#lC{&W5pu`ey#I@{A<UB2$`M17<G z_GDLTw|&oeZ)xA}JM#O(1D=E0L#@N9BbTG2qgkdKs{ZJ!xClXm7vSUQ0Pcane-H^) z9GigrhMRCL3=q5@^%FSE_#2+XVQvWQM?DmWMIcZdg#h3K?(w>f1nM|8PKoOWefwvh zuDJSW+ye#B%wMCxe#1?;mg9l|u;2Pb;J*E3Tr-ki<67hJAIF8@LX2bb<A|$={)Q>Y zj{5?~0o0GSzkKRgp<TW89o*beI-YJQ5lKPe<1pd>R{o5!aI5lXjP+Mr<97}Y{*1By z!{F=>>-J}i^=FLrXN>h{jP+-X^=FLrXN>h{jP<`M#!|=epySx&FP}%Rk0yZ%K)~@3 zcyI&)4}nA?@CnHY2?+=YsY%I*$xl$z(Vd{CrDZtD!NS1E#zaet;zzM@a`Eu;(6a~# z3vdf_aPx2<cLKp>6AAGNsR#+FxEW{}xc|S)Q4^rR2WXH9gPa0T3J8n>a?}b~a1+ME zC2eu#FB}d6h2iFl#3vvm!U<r>0Tcp*LE$hwJUA|k3<<)u12_d9<tY&*1eLB8lGTG+ zH0;4+d^Y8>W*WW04ffO4p5X+9v?u8388|q(xOsTR#3dx9q-9jjs;Zq+*U;2AFf=m8 zQEMApyQ}sNXh$z^A74NJfWR9!Z$;dWyc30qi%&>QdiW?gD?2AQFaJqFVfph4Y-Lq- zO>N7|)>m!q9i3f6!y}_(<FDUL%+AffUszmPUiq-O_4&*8*PU;>d&hb~aDnAt%1_Py zN-qkWUQjq321g$21%dkE9vB52@019FQb`wS<w3<N8ir4;{NQm}GXdLay$u>`&p|?3 zcClHG&12PmX!iFh7XA-u_Pb)g^qK&~FbK|gFbbdm_Tt;9ZSC2T9qyMkw~<kUPdq^v zBzxVI6iF!}=%qUslP^7<5D8mVB+TxicWsb*+~~6t&|eIQ_TM$M$#GjPM3-Jk9e^SQ zeN!#>^rNlR)I6m#5xd39Jkxr6$-pgnIy{w)Skx=77l_rURM^1A63#soeH-YZ2n$~Q z_&LLjdiOTTa49tiS8hkro%!1y(Dw_P+K|*UVQurN8Ja*%-n8Y*YOhiHSL#RBNxh;c zt0|PXEKlEruOnex1?`YnRcpYfRqF6{^ATxLe&nH!X^B4U6yfLeIg`!RMWLMt-FlUr z6N%E|S|>LV_Nr-G_evKslB8$*`gf{VUy*6;lJFIku4)~c?Kx@(6rjflKly|>A)2%T zp7Vx+tJ0loPnMRWuQ5;=l_&aDnRn~+HYP4}T<&_u-Aq4oxG8l6T!K`k^2Pl>aOEw$ z`)w)JkpWsC(0R)FFghk3SU8d~#xsbAZr7e@xKs=XeG1;8`||j9qG&^)mm)H$YO&9w zU-L}wJ3*0jnE}ZABY@r0T?%==Hvn1qxcfZ%^U%FRi|*+820-^Y*fp)_wEmE4?|NEC zLqnDDGy9kc#k=khV#<pIymYW;mR3^3ht#7j3q%%#DhJ79elm##0go^?a;-DnTMxUQ z@D_!h$zcN8?3h-p&714u1M8UP+1fgctPTwgu?C|}&ypj`Q2N1(nM=f(EvJiU-}UcJ zx;DNi(h3MkXyn+dd#*O#oP6D{wxKCeU%v2(Ad@OogpcvDbtl7fs)m#$e#m~^xo%_2 zjoGS^Q?-Y~FA{8O6W7|=A9dPtml0c?)g+J=>E;!YMso4%*J{}|@PEpmCR}klFnjm7 z=2+vwAIj^4&y8;m9JY3@ziA6tQA8X8p@lY&>F9t)$%`W(q;RK~{&Vn^v}MToun2^w ze_mu)YO1A|BI#%2f#K_U+tgqdrF$bah9{RO?IoRj1t`2;7O^@W#kkYx7acf&aK-uv z<nZA==bf6eMNC2Nott*YM?H8DswS%DU&uf}t1oaf_`I*D?PTkdCu~-ju1r#;-INjX zsVy+5w&C2{HKe`_rIw8mmg8wtd<m02V|4Y#@u;7~gGhj6mz(ZIN<2F0j=4|<oti0; z8>^c6-E^)E{+W=*Ydik!#e`>`C$0=f0S7fWr4-Z1t&woZ`%OiP0GAd0fV-=T6x7xl zup7N{bgZ{*!nq+OM5(20x#9~kL-bc`AF$q{uB&@SScuTzdPWWD73=r&;Fv04cM*iP zEoj_Bs?LAV*6*gkfV`dv^{(5tmMVNDqEAUxh{Hwkk@r=7)g<o4oa5_SQp)>4@MMCP zkS)#J_Sq+58><`P=boUVXJ&4MB|oDGgU2AnJ4P#H;$Gyuu-+SyHxJaKmwfmFOQFVo zU+T;NY%rYLNQyOirlxC4MhDuyU;k-dtVAVRh^rXz_@F`q&KOwHk*m*mDN^p<UDerj zSy80Ctpm6oyb>9yqt(6d0O?RBmP8g?t?t~B>+vP8=6`ITL62vwfSQT6i{FdpgG$rr z7-KkH2km6|p1o(Wa~}(g_H?n2a8^apb6gBIS`9_0OTklbqzZVy`muVKu!cF_!+v*8 zBu*zBPxR4aG92Re0nh;4={u=aW=D@my@`719ykEi*za*(qK+hFs)FSLaf1l>jp&h{ zrUDQ>7mAvA_f{dnWU6tRB@HNcX;DJj98Eg#At;tWP*iZK9qqb9nejGrP0b`OG<ymI zPIl7tDKVn1#usU`xh0I)c(L@%0MID~h&Ro3&7U)rKIWhqu#`BfVdL=L(A&~5AwmiH zgcU2sn3#CljZe1MEwryg(N228(<?44JV;3jM75O<EmovgO85!w++H<D7CSaA`@& z#`6gH2m8Qy_|=j0nFKZ{6g)I&1#_>5a7$z&lvXH7wt8=<%a5C=v)>V1J80W8^xj|H z&Lk2y3gyqAvdC2u+Iy+U^fgzwzR>_;Mx^W;*JsBDk1l02Na#>#n+c1u^U*iD)2%zL zr0Srk&!FPuLG#{$ezvO;*)v!v8Fq{7nM-zoDm{&;u@SFe-;!EPIcvQ$J)NEDMJ48} z5@Y%?!y=xY6Dx}3T8Ga@?gzcx&V*-tL!<VWRuqY~Uo@IMI|3dr;vr_mI(YW`>)VC3 z4XdUmsK~iFzbz{c4M-$=TxcL~*%G*qP*W8vS>s$Qv7DHE<>bah&e@TzQ&`i79B<IN zoR88~437devjvz5<6-4n$_#S=GULkFEY<W<m@VDUl32jWm}xYpuUfVhYnbt=YAh$q zzi~7wCgg>DJa`sD!+!+agB&0|nLH@sxyMEvq3rWw5c$!=<?n?r77sVSFGGEF3@8yE zmnxs<uX?SD_Pa==x)>@eVzLp@<V0ICn3v;phV#;#S-Y2CarjtM57jHbly{BW)lgXp zK6O5LM??B6rs8XVSyI{MX0b-{7a`YjqOai^ubk4&kkt5|lRYWlyk_iCl;?lzA=zoo z?>YHtY^^A&#g#MHG|p89-FRKH5Kxgpu8`O*7+>DnqW*3t+llmcIH^qd=?->gf|7TR zN5DJSFm<8lBbsijh-OD}Cw57{Fg)}A=_pKO4!bnCFITWh(xv6;H62Q6k;Le~r+3y* zgruWA5IC0cL%8S*9x3W6(!T<h{?D(-vkXpex(Es736{Nbb0Wsnm2FcL3G0Yz5V}K} zxWH)rX5Sq*D}gCB^NL%#i!pEDn6u?-w)Y;Lu14slr^0F~pI9?drPhR$eshyc(lv-j z4=)8C0SmuwFT$O<^du*@#Rlxhm3(yp8^gY>0sf*z;a!{A$oHx19DbHzWd;7zef<~5 zKC)slvtQ~=1uy&MF+IhoQ40&iCUf=$FnOegiCiBw6n|g(Vd&sOw&utFy?5X5t-7em z78O;WGBU!32wJ3(j;~hSvQ~PnyP>#it1=blE0iKzvo|bC5PZpHeno>~s#TKh<T7Ln zklzRLqaEGzc-Uc%kxx%cPFx7r>R7}u^>kULwt{<Vi_<jkLX~exOD)E~<7e788B3E7 zZX9cT@9-_Q)pt)YjbnJCKHquT3<|Es@Ytf#n<7o@kr=5wuE<%;$GK^#`~XxM%Ch|I zMH6MOd%J*OBsJ9k3{&2RsKxM%kk4A%!tpic+WEVehBI^{7+J6n`{TY|t&!%<cMSEw zD94cLDKcZ*J~UE!H*o1!KYvj<Vr<u1xmhnP78ZT=t%h?@li7kI*Mii3QvxFU0+#E0 z<M{Ts*H;EFtNIwr5H(Uwn5_z<wVkdBOqD;ie5%ORlUwCxO&v;84&fK+Sn$%Vcgkms z%u*M6v{1b-Y{E*l?k3h=^F3?YY(ZmcUk&Gxvdr|qrb!$&mNrR`%jE4iF6o}=O`AGE z+c6r@%q(H?qI4D6NzaMX`LaX(&*;F@^(f`9B;0%AAS0g@ogDEX?=72pc|!V_G{om} zVLRIsJw>^&R|H#Fj#@HSqm8j!+$x;+@<j|=7T=H}JDcWXw>^%)JECRo3yfNoRZiwX zqHVG^<e{yJ%j9jM$~JTp1Y9Sj#Z3IJC9d(i6yIEj^WGDSFs1YHZKCIT0SuYL6qx{T zfP&w>)T?3ekovwu56u}=*G8ZYO!w|HBWXlYe=d5<OuRl(qTki_bI%3Jcu}{{lIv`Y zRo~yN)}>c?KeN!H`Q+UobCB({F5kD1t-+=uhQFYvbcfp3ozDGbV_=&1Wf31l<7N2) z3o$36OZdh;-1(jb&h}snd43?R*fk{}#28ul{3N@s79`xn+TWB|O+wM&F&$sbLpH2& z?8^z>mu?IzGJ);jVCb$lRN?Ke1$R43-LBLjvy6wvE7`YhjSq*>?6r^~vDe$uLU9+% z&BrZ2xudj29&+nw*|6V&c%`R$=f#pO4_oC9dGFZOnh#tk_i)c9lGNS~b!RM*9sR7i zQ^U*SEtYdIE#Z-TmZOjH!B~zZJHNZfjaO1X+5Hb9p0*%VkcoIim`ZHvaS@-%Rl4&_ zCfsHrRp?^<Kx4!Hl+92m;$Zykl6(cvXHv%eE8GxGIAesT96IX<H`jsH=WdI=6K}u+ zvRxMX7er~K=$ht2naHK4yy8~W+q<Q%qy=O`Layypx91X{DN9%#P6qen_YNFi;;tbb zJV&e|&9NuWzk6VkiNCZzzoLcQ)PX2$3@PpO4~4QP9hg3$`cyg=%0Frbemd<?E}ZO> zV^KvS<fdN#4stA1;pokOw{W-#!U}`1{uvGf?En05IIi;uIDN^MH}1yq44wX+_My-d zeMi84JDb_9cNnXxue3up7E1M})IYEiS-DxEtUsc;_LR@w>$(O5KltHR$ixA6f5O)G zC%uu=2b*6q@%{xe##~-GIBK>T{McsBobiGh(KQph=h%ijcG1vF8m!s2w0)xj(P3d3 zDe=v9Cu8Fd$5+&@mnM}4ezsVrXTP&9Mn-+<4O+%Ee&XhdICjsJ$BUZPz6k^lI%?UT z_j^k6Lr{qtPn~u!7m7$OnoGGXJyXhyWD|Mz$V!wi@$&b8+phHPf$u%E{SE?-z<qIh zx7GaJ?BVO5zJ{2IEPj70YaFXxy!yb*3H|g^zi{>Q+T_<_5>>RK_*1~ro3V+oNwhB~ zZs2>q&k9z~WBan%q|Qj`d|AEIq?U#Y<<K>eN(no{rhvho$Bcf-^pbp#QTLP`>H3uY zh=hgim2XGjC3F7IL_!AN!s>Lr_v$qJuuarE?Kp$F8|uRXTpZ>XV|c0xtX<8F?j;D< zDMboOlyStu9`E~Bi{~^3T`{h3G1G$N2T%lD(J^kGIuX3l<W)yUH=!-IIAYiJ{RoIP zN<>DN(g>|OrQi4wy2nZ`u1wT_QXUnTJ|pN8us!JKA%J~Rz>yS3?&Q!-ixF>$;5IZh zb{B|ud-&y~{SCK;(TW0b+J_o0KC+$YDmro)7N6~jW!;w7&H>NFgu?3H0ll{N>fT|p z#F>n*jhd$yt~Xp8!4e;V>p?u=h2OqH@sE5+A#XBSb#Mhp*WS0trQdpSrQw5F#1GQ{ zPnDsep@s<k+9ZjOGd{)U0-?0S{#CUz?^5y|qw;QiWGMz*|4bRv+QnPht2*t7x}250 z9No%qLtAe-4Y;C1n`XuuR21H^XbQNW#>%m%kE%Wy`I163=+L}=IOvKA;1}s~MWSas z<#>l*n@t5g{W`!RKKj1L@{?WLn0$`QCFa-6@q5Xva9a#u!Z^S<$%q^hcTNm9=b+29 zgEFukiIK$~adC^}qTewAz48J?=dhz6#ic6J<b1Slz7e0B%uH6Daj|cklh<vQyPa$G z`die83s+=2b68&cVS>`(6Oh={b3AhHOKc}^v^VV*6(28_MDl<}enIs0)3d_|X&GcI z3Y&t`<wrmz_TWU?yV~90i;4<@(#5B2oH*&s6|3WW6nquNm`e|LxY8^3ewaAIrgJt0 zUOr+u%yxli8zJQ3VMXlG@iDN5L!Cc>9+e*Cq}DZkQlgJY)}+O}nW3FQ2x34C*(I#{ zZ|BtiE8XHBTrgkird@M<dzNCmB;Vp?;Y0k>;qoPF@A9^!X$C%=aDQ#m@8>;<x<ype zb`hERF{mhZc*y?r^mNH&cjagHOH18!Ep2iM?s9$e(-ry5S5-AUvz+calnB^XK7y&R zQroRG(-jL5O?L-4e61>XZ7gTyG1*QC3Agx0zIr%dHBn$FTaa?T{<cvh<uhm!Z?^2r znX-1z90ZFR?32+bTTe8)t7q-|ucIINVYwulp3YWS8Xh>JkGnNW^m8m^4>HDnBnC%m zE+D5A%z!r6LFrKRu7%K*ZGj&r1pKGVL`6k`r2ksFV9Lxs67rm%*;a|;rL`0*7qjiX zme{-iu4nMCnZ!YXpZ4{QwBSb|R*TcLM2^{&LF@>GZY)jZH^>*E#~N+0CU1JmF;`60 zn;*PhczB?3#&^ZYsf1#|NZMY|-aRXdx8vlj)^i*FWd24^n$OOCfiF|3Sw2!C_n)S3 z56$=P*7}y5){nRvBYT%WsCKmewWvGYJRZRpOU;b)OIV*k@im#4G?KSvoIGl?)t0e3 z=d8)#zN|Hl=MvJy7Q9XVnjJjt{a4U~Kb(WVi;5IDWa8KVetjDlim|vKI6(g9YT}0c zy5s*TLh~O5Ln_6q>W)B?tlWS(X)l)s`w@799NZUo@Xy*)cgZe)?wK@RbJk~-lB|*I zJDei+>!TpY+zJY_$yacEO1gMf?qxs+t;}Vtdt{hU!JZYxm$^j9ofS$Lt6F>5Id)~V zalT*2dNc5xylD3qcS$lI*EL?v(kBxF>#6roIJ}<4sO}F|OB*xKgq|D9^H?<E6<zR? zIRd|q-h<hLvmea-0}^cN?nFyc#N0u(d8{cB%RMw|#VwLk^y+^j@|=~G70U6~JM5p! zKhfo)0WSP8Wm!<uha2EM2A$CNEq!Sn)1KAp8FOZ}MOHPHKV(Cc>^q)`AN5e(vy}20 zwdP5kniQjWVM&xg?XzZeBwxGr8~$rskbP`m6ki!1AJe_JQ`}im)<5iqvmGx%V>vdm z^v)#OqjGw6Frd{l7{1OmSnhAO_WYPAjq{z+?I*t96E^oiQzktczZp^NZ@s5+D(+&P z=Z>$*<+XrBwzSyoCf<+<BRTw%z_QAGRUU5{vR_91=}%in{?i?5RALmk`>zeacpa0- zS8A{CB=KLbPGaZov>D(&*~t#%)Jx3Ihnvh{uJtfZCVkFFy+_DVv#0{@uk;@r`>MPS z3W|%%ncl4Q)*P@`vxR-6xEL~R^%lK7s3~DAT*S9bt*h#6jeCK=JT&KYM9s7j`OZ&= zCYE|s=|x*S8iM*Ew`;?uMnCX~`&i}Mdu3N`eMH4LaG0__P{}Sm6(1eWd;Z7u)%(q8 zX-yKdyv3wES;F7BR<k>g^U`q-4!bRbCYFy=B$r%FU$0*LFt__|?Xbe;*zG<kbdaCj z8>0K4Mutt(PmQE#3?G5PQQP{on-U#w8C23=G(p?nTyQpGi4>C!GWs|XkXIXF?qjDG zu9qrt`oR%^Fb!X^4px)9RrqFtov`P<-J{7*vKTFq9exe@#ix2lpoI^40LNN<CpZGD z1`qr--zIVCj@BN5{{ACiR`s=`{f#IuZ~J#viFaQpMjJP{jzF=l)~uF~Gl`br$`Mdd zIB=9JKT&SOoqGgs8j<Z0I5vKxIs#(_tQ!e+f)}%ke2zdy#vzvZ4YJ(Uz%cgPS(nCT zwxW<3ZCw9ZNiB&VUNa;X4Ic)F?2TNrW$uj5N;;%4QCQ|EJOV#ta_+Vmt<}iH6p#Nd z^FY@~VV0uu2xP_-AAvC36rwB$&5JA@%}TX*cX5j7uHV1LsW)PoXoAr)GN`Ze5U)!Q zYhpvb4ma2rDM?aqh&360TpvmIGMqtTzoTh3em@*vey%KJmgdG`xLN5T?%LljGJiR7 zmO-we#k|vlD}yIt0!@UORQxVO*q0NJRl9#TX7To4hcQ<j{>8D5s+KBH{S`@O6=d$; zUR5OcLlm3)#QsNK@R!iD>lRNgp<l4nC22=!Y$KU3Jk<>~T?k3=zpC3TEse5dK4&&} zU8v~ow`&(hzU(@ee7WnwxybiEe0SE*iGY-Y<H1|SbDzy4Iv`%59?Ii;hYdlH^@9+z z203kmUsuExi#Y{K$8racX_F7hIeOi09tG@XF4#Q15y_7^81sy`Pp#ZF55DrNOX;6~ zKE_@@QqiEin;CgfDE;0ilAE!jeC?vNGi4ygBkZ+KrOTHiaO3d2%;fk+)o%94_umPH zain(Xhv%HR_CW=Xl;#@wFGHyJavPp%Hm=~q%i1X@+0Rn?_TFanNR24TSRa4p)SW%$ zC{UP-9seG5=_Ofm-P)2DJ$F>*y+Y5-ljXpZOmx!tOXJ-5G6lLjX>VdFCqQ96INrW% zH*lvEHOy9BU$^0?8YG_l($D?~FmI|P9f3Qx(SEeb+R6?7#2z=oWywfZPbBA+f5l8C z9d=dj%N>D^?A}^IMK0LFDVuj;VRz#9jP99Y7jfrMb($i>J7e1C@UMb$EZaZj;`i^n zGb@eNO6jh1BlW*Za1rZ=8aGpR8{nA7C`X>_y~V3=E!r|QHnC#8{z74CwDTB=ihj+o z&{F*^;<2$Zv)~VsQny0pXUfa*_;=NIBWH@qRDBH+a^~MmZkBxr{ylB5+FqJH+Lw{< z#1PTo7vRGcA^0&jT+2xxcAB5P<Q*S}XwwWPSQn$FRR75@=Ub9tt$9Id)heZ^c`r3P z=Ub_2d?Vqej07)?d#0busMHqcyR+}6*3MeUT$BsxS}OSEqf0rGE{!{{6~E}8qbSfS zMBfdL!`7P%q;?a=^0P*_wp?xU?A;2Q@AYFQl^ne6B;%V~aG_A(Y3G8r2Q~VX@g0+* z0LMaiuW)L2cdzJ??zg@nW5vPIR4?*X|8Q37vyBJDSV&gQ4j*SNGn~A)P#+jL`Ia+- ze4jqq{@x<p6~ZD{`MQ9)yb*B{<@Q#O*HaeV#S%&0c2NP@_w4w`^c&+3^$Ib<_$@cE z<EU*OEcN8F)cz!M_OHWN@1?ORd1q2Xv{&Fm!+N}*`_wceG-}Z#X@>wF-Ah2pd`X1P z;$fWecl{E_34t{&tKwD3yEb0-XzvSHjClm+Y)AD~!nKd(#Q0!x>%CWZpjHCTZDRKb zWQ(##fAQ+^w9dlSuSrNlnWt6468QnPwdn9u6fxot^XRx9HHw6#wc6aPSuAo)9xKiN zrbbY_y&`^Xf7kiY-v+DUl6c*N&icX37Tag9&i9QV=hL@fS<)-3&p&^cl#~1e$MAds zYaDSCTdm%X{HQlK^uCE?a#C!q&M++A{JC23XB!w3a6f(d5{}I}CO?a-{+sJH;)ioo zRaFIAe=x$|I;`Kk8ike}aX(Qnljj+s#j`!3@y;||drYT^V_Nq{{Ai6Z(vlL-F0NB4 zo2fT*O0>OyA2b~HU|_~@;zwEwk`j`&JQ1C!{lsA}X!v#NM8((T#i`96+t{Scm6#ht zc{enM2h?2FTuxC?JY(mTxSxU$mTI})cu~eM>PH#{a?fg(g=}R|%q;j$u36g~{`;nF zq3;W%fgktnL#P}c?!On&GKNo$5*aSv!d6@yYNw`MVPF$J^^ltm9<C&}<z)6V_ob7h zIg}eCwJ`Mbid4EZg&F)5vq<D?Vs`_`jY+KJ*}=JC%sK|0pZ>wOIz6B==|KmuchC(j zEEJaE?z^Z__cN`&^-u8-zh$ObtI8wZ5Sq6$sb9|FjijcvHjTy6y11!0)5=4pfv2U5 z^;Lu39iQy)_X@O_wUx`>Z+b3F&qX@Vl`66={xeGyHq~D{B$iU<C|Hp~@G48p_s|x9 zT2pl0?tE-K%}m&chxP@=3o_BQQRuJekW=~`O<~u^d2<bt+B67Xoy%nRD-u7B3(;@y zc660*^xd`j+E|7*^QoGc((-sekrmKKsv?i<@#@nK98k}yOgtCEPN}KnX9y}02<>1z z(;pX{IzC7En7+s2gz^z^P@<l^&mxX+FK%aBJj0^DYFN=rq%pjnc|&WM-`Qoxi(S`z zGRKMWzLFgwz*&cFWVGw2k>l)UgD9IzCfe19U6MCjlQ<a-RAiy*_D#Ey?5XLCiJxBX z@$V%~e*6+7Gq3Yz;h@mm7VR727eL=$>2&p{ffWVTPh6c;M|7U=zRALWTiT<2S~i8N zlxwV@dG;-h*nS<i*gY9W#!CB|N-YD5zTTZjBg7(SZ0~zTHU2a%FFH^)G0x^>VuF*4 yojtR!k_%xwgYElSV_=I3VZi=KP5(S2YBT>1yZA>~!QVw4{&x%M>MF7vz4<qw+h+Ix diff --git a/doc/modules/cassandra/assets/images/Figure_1_read_repair.jpg b/doc/modules/cassandra/assets/images/Figure_1_read_repair.jpg index d771550a4f0f91a9cad392f4c7b77779e8b5b5b8..117f5fbcdf3608114bc33881cac207f86854762b 100644 GIT binary patch literal 38223 zcmeFZ2{@F0_%A%A5|U&KF|F2^7F%eh5|T6_d(2Ogkme_3WXu%VY06r}RJLT9WGiGa zmXKu6h%qz2lC60rj3=`=Pv^Y<bKY~_bDiZq@44RhocBL`uNgDn>zU{NZujT@+@IU} zm-T7bj&oKvR<I2lHo(q8Kd|))n8lyJ{;Q9T8#io#em89Vm&bp2weinI{!bV9kGlO! z_y6kidJ9Z;>xPG$bj3F8gKd=EASSzEy&VRJ!8UCEch`pfACC<i#Wro;BED5ZQVQx& zy92gygP7RHO=6ojZ`uU)j)FdiZIa!*bMKK;TjcD$#rFlvAHDbJ`PTiXt6CKt27m*{ zuH1^2kW^IKrL1yLLsLsz$H36Y*yOmW#TiR0>$5iJ94|RJySTc!`&{+C=6BscAS5*G zb~rBLPR#w-xcG#`q_p&m%&hFkPjX%q6c!biyexh7rn;uKuKw-2hBjJzM`zcE?w-M+ z;gQj?FXMD3i~Y~H?=wGU=eWO?mRESI;P16Rc5Q%({re&NcZU5ZcF96^ZQQg;Y?JsO zyEbeLhnkq|rp<egY}t9rUferaZr{;+TjftbdS2BkvHzF@pm62ZfTZF91Li^QAJhJ= zW&dr4MgQNj?B5&qAMB#Twux<kCQnQjhJuMiS}&4d|Jwd_4E}Wu{I9eIs#k(M&V4qh zKzojboj_R*7<eLAoktB*-FXui25a(kFVv+iT^wFHar)PpI2t;7P`N0_-*bPG$&<;< zOAj|Zw<MtE;ehjA?uB108gG;EF>;^i%bH;mNe&+|inl8!d_ui(`ll*8aj1=|Vjl5g zPNl0O(Eiq|5_`Zp#@z8m=61(R{&GDxt*{S>fNPMwXbAg!9oA}ymsY*H4!dF(%)cmb z!qt-xYYGl-#jEI>KNDTxEZ~~fVfz=&^aPAIG={~YF4n#hYpLU>7U<v5eaGGB`z|vp zp8WOqufNL@Q?Awco7K)`E-s-0Us5_IP!o<KNe|#{e<YX4(*TwDr@(86v*o-*U5py5 z=gf6jqLEP@O=(uVjY4;6&DoO9(#=jW7@d_N=@lKcdlu~`iPro|ebc|#$2TQ>kp*+@ zKr}1Q?>zWudc!rcJaP*j4UiKEiul{0TL}J{M;DtEn{QZEGhANNt+}+Xc4X0~>2s}8 zv_;We_CRS`Rr_CDSLdvK_e(L?Mo9w;qVx1PgmfbMi%Y7<&GeBw6=PqcA}w*8PB<Mt zEwixlg0vHl2bn<fbap)5IMa}tOx$U%2F#qx-2Wj&dL5R$l50}DC;D(odp@jR;rHA+ z4Di5y(QQ{Uc7}`9uEQQMYJ-HiM?bE^-qbHt@y`eXffmUL#WP?ovp<gAh8DBQ+$kN| zE3P_jp?zb&MBlGA{aaBxgM=UGj`!s<{&}u2Qj^Uh)Y)T5*%;swDMh5=KouuWXOZjb z6W$vVQwW;E45A-HW=BiJtqAo4!smvF9SOgGcM10AUm^A%b4}&@6I&f>2QS2|!!`+p zX?VF!@jGm}x*|zw_T*`ifeCF+#gEqiCDJ~6>Qwr>J;VBOTh0_d-TuM%Nvxeu^vj{S z=|MjI>_7QFcjMAJ9nn2DA{Ue&G_l6Q7sTn;F6>p})^*r)j!5B}7)RhId}&VDcOXT` zxHqN3KPiS<>P1fK$b--{Z%(d52F{WJJ2hXA??yH<i`L9^xbUcG*fg$UhB$_<>KaqW zD4QM1HV9aUIi{@yufvWhJ@qY}m=ETYf5Yo8xv)MGYn22FW9Mm8gpU~DLch|PCtq{W zi(&k;IE|k-LVrCUUWfTaUDNt?Y_-JXK-AB(RB?g4m`GBA$?JbWoX$2CPn;$_IUppX zZj$2&M(>|k9SvIMJfKR7l+V0*^mIRC^9SV&zkLTYzS}c4+fTfqe!Ioa`8GJAym@Z% z_ZMOI3+lc_$kxg!wSiClNsQMRfqXjicr0<~CuxEC93L{I=O)hi7B|mx1oPOl*c-#~ zy8Arhu3qdspm}|n97_gViJ(S!M_-uv{uiPza4B;+_7k<e=y;;~!LfDNHA=#ur`hA? zuApf+UFi(aSrK*npo?b~FD?CBwf;Y<We<uZmpxw%WE!o*a`Xt-7fReyF-)gaG|-Le zCKrV|*`Y4J9x1N~JE6bU?=Xa&elXW{ap<Ra)0?K+Cl<7j!0?-kYVF(aD}-Jt-+pRe z#i-RiOZkf%{{PT0{W3|0iNxu0vTWEoOddBl%OOtZP$im(uW*6WoVK&l?Oy7_dUGPM z4s#auj)_bKS0|2AlGkB=L7JUw*UEgR$r2w(1z;AN#$gGA!c-*;R85FyiPNu83g#HV zMI_P9_Y+uwk*}nsy6C6{Ay>>6Flst8!x!sBA_lJlesPB1BHa|vs`q+uH>VEv<a@_f zY-&EwH%~P0Wopu7B=B;e3otP~bN2@7jX(w{Ysr@b!<f?Mb*nW1BgQ~iX8VuVw%W=$ z!6mb!qO0-WAFZ10{Ad@Yw0&Cp3RgkbIVsQlXzB~zt<1%+5nbb(Jo9bqFjW_+)0sL) z|IR<y^ULF9=a_9ZzSnEaEZ5A;&UA1LJ+v59e&bNkv+6fh)zP{hwq~}jSyF)>9@iBd zrsyX2va<3zI-4)THkJ=qMsa~wj67kSQ6+wt7|7Ua{s_$Opl%W&LBEq@p%mzBso}Kg zg<l4&I}x#*$_bk-rG;)_eVa<>lTv6qZ(li^w^+-OK|Yr*F7XRNWFO3WmUSn!gud(? z=I=<k5c;b1Qi<WU{Qk8ct7xav(7E9P2Sb0mwnDWb56b+j@rC|QGN?&fnBec8H}y+p zbFm8{fMzSPj@W|U{Z(*8Sa5JGREm<Uek`VgcbglKuK;FFa}pUUs?J=mBA~G|e_O)< z-j+GhSaI5o&{u9#Joyz}I&1t5b(Q_V#OLgbGf7&Hd>)vLIGC_4B@+^!Z94f325S-9 zNIl+ku7pVyDEKiAMY+b~p$vK%PYdj4_AC33&h*4Pal;ofxq-|rRAA^bHTjnHiDh*K zVKMTMTlw3Z?`OSst;52D{QG}=^Nqdl-*%}$LCNZJWw+IursCk6#*`OR^r>#uD@8i# zsWx`qPDyEvH^+i(bSKAc%mS8cC%i4?U2Hze+grl^GzA)N9mb5Op_A4Q<ST<kz)}li zXOpo$7ZEQYGWEnd-48@tO6*BL$5kYUi2Xb%cJ0{1!??=tNp>1J0kf+rK?XzB)-&aB zz?L(v+vYOnKGt$Vi50&P%QAa1H9ZfnVsyzQ6RV@5d-p44T*iqrDaqvm?XrpLhwjoA zCmZkGrC-r;Lx&a2G5|-?U8OBVXwqxD?ybWD$(v7Jv<4`=Z@`vxCV98@wYz5SKj1a1 z3**dG4G-;)Ngt|l@sI~d`aWUI(U&;cwJ53EPs#Z9<-V|3QM<Y7GKrH)S$a2VVbH~- z^KefL8f=yj4KDRPT@P_Yk!0<2oU%hNZ&dh%aDwnbX~#=`cspqU4PNgbe|q&H)`LxM zDYH9=*>v3}kkibai8q(q7HR!h4j)OiL&EXaU?exNbxpG!5lb)y!rQ4Ea5)25!ErVg z2Ig9F<L}qbD;?0AN@XErroOt(-R`)j%6u}mC*>a%@Kv)JSA4+*m_c(1Ra_Syj*BKy zefm>PC&xz&DGBSaYOjhC9O?qMviw=bd+;3)=`t85vr%yUQQnoEmvLm8dsbnUl~48X zjW54rX1I9$!CS2^)Kwiqn(b23%83QU&f?+VP(9%fLKU%K+4`ri0yqN=(}W|hMwqLj zi<uB(TIKOi;}6tsG$YYNc$NS*(fR@Yh_9@rDYX+ZzMbtbYM8jZT)+8eEhavvD`_Un z`v4V)`!TBHg&HEqU{pmiq5*Z-fJj@g4cZX>N$W5fLx!ONLlIQ+dd=PPaP@YF!#d2i zF{ksI>9u$M$0A(nlx(FJGk%P&{%vv8JaB1OivE5HlwU&OtZl#FkI~dSMo}7Yh4>B& zy7%0xs$Y{41}o_9!aI5L_Lsji|9AByUK{vlMZ=hl)T^ZUV|aV;H+#{nMWhmCHFWax zp}zN`cdT`o8rbMK)quE%RK=;DtbI3%+=u%mIy-rzCQ7Ae_2S!KlY=rk+E>v}PywWg zI%2%Y^quet%9kqngQ$)Q<frOh7^x+1n{%)Do(GV4VL)d?Io+k5NXK^GF_5!mrXnMK z)1u_svZN5n=B`)Z<D*&bCRDCs_Af>ob=;j{4W=+^ToyO87csnCVHx6(e$5Ej#ZEms zbpEqQ<$6xiW-6e$@C|4ljacr#hmv_>)%{{9f5vlEUILG1N#9wJx;Ovr9a8gffKB8f z(>r|?{Fjs(2(pbZEP6g3yMf$ri#jzHcviFiX?{}~H%fL1zW4~f_#5hBCd^)kU5AUc zUZK9`w32t@k8<&-t)gvniby4{-{L85Wc%#~yU}W3l*5bWdd_2hpkU^j!K_6T?U*Od zk>+0tKHAGrEMN(bqHFyPd1DJy?!-x9G30swcop@g<2%R4@fQH2WR{+VrQY5~fSjZv zQ-vFAX=rlP>Go%-iFF#?`I%hm(NktvD~buh9xbmjCE+N&L^aTuMO@g+b?l<nh_}jY z!4<Gp7fQLg9e1)x>o9-PIHQGr{!V{8@jhnfN~Eh&JE|spfzRZJ>~20X5cXu@FLIaH zw)@@0;FoAbQj=-*oRUgsE|K^4hV-^q+Dz8uaotOi-L*hpWzR1Mdv}ez3Md58RP{KD zo#@ti6keS5$VPC{>t=WHL{Ks1HfcCXx2JiVAQ(Kww*Y0%fw_PM%QzOPiBodD><-4a zxz{!qVK>veRKL!uyniDb{0tZO<wMn!e?8X&^W~$8RPS{S>(nUc_a;3@&)KJC_`dR= zH2kOE)IP%_B|Rl;_Q%*ONiE7J^aElYp_bJzhs)*p3h!4KU{V>in;Wg4gI%2$f4ld$ zpvDi+IaE~RXuKG1YSF^&K`?D_cxuvrYSZBNMy^$r(bLjkpA!Exo6;@?owf|DaoNDZ zvh+oTMk2tmeKR|h=jN_!*?32*@>k@+b=bCAomt}oHcFsi<~~xBuCNZfkGg^$w{QXd zD+11b90deeAw>`l@Vn^bxQg$Ir<*AScnE)+D?|{b%zj_QUmkKRNS#!s+62x}$4R^p zpk$#Hcr(WAj}J=?=~VetLILBu7;;Dr@Ll-~x2vN+Nn|vv>dIxb_p45R@vu<nq?QJd zes#(F2YLJW`4ziaOQ)um9}!A8p5B?V`<3pcmtPC(tM43~>6I@a)KTq6pZXr$Wj0it z;bJ@Cyf~C&*E5Ou>qU~T&z=`aTCo4LQPR!B`3Jz8^IdD%U8uYH#$4h;3Ew*4in1XQ z!iJkaW&8VcWfv7XisHM-v3Eu?yJAJ=*J*!~hzSPcwPjky`tcP%#x~z{oHy0g8P;NH zH#p6{s`)lhCJ>OX)mmkgIXyRYNVUEY`0DmLEZz%kg@d;-KyBj`qztah3OoXR#Q`Au zK4Dux^S+Tq{BfX>_Z}qBYcOw=I)j{fuFnQTW#qiTq+j1xd7fZd#DNzk<|tMpzfZ{_ z-zbJALX7I?-Eib>+tG6LVNZ*~U#B+7K0KvnrMnc^fm8sx9+)4m0L?n{)jl_QG7)nA z**S;=0fH5zLGuK7)PlJkweLll<v-|!J16GugpU54JtSwv{gC1`9FqhtwKfnJWuTA} zh{Y29b_qQAHiRzv*(A?!o)LF@t$rhyI6V{3{40BmT2<eUkqc12{qFg=D}e`(Q`c(B z>N&d*n;ip%$J=A;2M`3~W6be^?-lpnyW6_h&1<d0VueNDxWt4tK_e>GMa5*o;pMU; zMrs|lldrc5659yN175~}9Pb{8;-m40KO>LerMb;d*!?i@{^Z1|(l14mS<edtO)oF{ zQgDtP`RXG<)5GSfY)^#4gUAyemLqyWhIU7P(LHSET{Tk+)Vnc<qV)Nj<?T=YZ`BC; zgwKq6@eT|f!3s}Zhh3$BnxEeJyg2nI%y|pMFWT@!`b%8F^j6fyl@FbLKGXdR!XiwE zKA9WBdmcwW875J=xhqz%Y3MY4`A%I;FLUIRumU<e;fcgrY6pBbfku3znX*EOXKZg0 zBj6GIzeQsr*|&k7n3mOJ7Tf6A3uS!c`47d%QP%?(HNJ@_;jLwIWoW6!4rf_Q=H3zh zThbTd63udWV^9H@X@eZc4$$ljAVVW<^xtuvpa78L7n}QvXK(MQ?)fMCTGr~t{5g;0 z+}_^AgN^SS8}onujQHu7<+5g;djQv2Wm3qF_0Y?+qH3*1zRz}UCK`WIlr={xmtOJW zgvl&r`woQ<AU&p)7FGcoFA&%RDAA-f)^;IQQf)g3N4eNU!$>GN6r2IcY+=M8Q67H| zEMiw&a|@GE6Pz5mIpEtoOes!tmQZWjCuJ5Hcs;5vN%tpn>rjgK)|aPsx({AW`|*f- zG^0fO!mW^}c@sAmn-^cdNYY6}9t7*5odODT2s+@r1K3#dgKz|dbGuwqeMSk#34QP# zO&CCoMM_jq&Nt+a)&h)6Z|Xtv#;i6PEBQpg=8HSd$bGBaLFi=EA|E`Oo!YSui!Kq3 zW;MtkNsf%f?8a*d6Oeo6>*_wayA_f;!l`d2izY=cG)$tS&M;n`J|AN7H|w~MP~pXE zT!8+CbNV~8D%u=k8RY$`qHrVEq*ASS+8sBFc72+9+CA|PA$XG5YI>-7Z-hVpJYkGc z7r%I{$maE2{uVC0gCv9B4tiGH|JD6w48MVY5GTi6sZTO6!d>w<5C(WU3_3e#B%Qft z!Q;^2v6yYmk~m!}5DUaWur{W77bs499YsC^gx@naE5c!DIXAA#9I2?uQy<;s9HAU; ztL=E`)A_W<M#U3Cg^LB}a&M)aSZ&JtN-od)M|9)KXsNwc4`Gh3dzNyx%o6|5-cq6T zvAv}{?7z@90OU@13`4egGj6E^Qy0JB%j(~XM}SoIr-l>~LrSz~u7b%Jujjo1!ef_z z4OwuraQ5j;YR3s<i|R+A@Wk5}Yem`eQYB!VG?w<Hs;&{37fF8OpCpVEw>PP=h+iCe zI%5X!&D!ZlFV90Tkaz>3{9|eZxS5%H>r0Jkb1E=3#PsAOm|O5|=^ZaUye1|B??^X4 zVyaG;o~TdSofuT`o6~_*$rVYW_-AnQf5N0>G=Gbz8zpP74wC?Bt&2g8XN440076nH zU`iVxKDg5Or~nE~_yFf(!{-WYXoMc2mj&PL?&X28IYElcp>tMljgHbg=~F#DQ&P5O zXZP0D)YQg7OV%S0H>#z#IGuR<)W$OjFOxRDdg%m%>9$bP|61DS*xs`}(}0ezoEo*6 z*DOx9s;teYSo+(3Ey!g7Ccg10Bj-{z-en4(x;H#nYcyC`j-xle6x>~d3lvZt{u<9& z3{Z<mlZkha{rY0I2?faTv;jLx-0~Htj9KS9kONIb8V(yKZ{7qqy;lD=&2pRh%kH>5 ztg%3W%$%BZX(QHQ7AE-?hSj^sEsPrDg`!|!N-v=A=l+Za{6%mB_vk>=v5EtyT?hXG zZC$FbhD{bnyt;<1(yEMq`0c`OvQs_TfU~+pY&q;Ub*-pxCHMEOHbyw&Tj_U<I7Hf& z<lMmQFh>9=-UMj;Qq;jvY6>eWiAV0jkzBc*sa##&6>yTJeZV6UyB)71d{!egw)KFk zPTFO?0s2#FP!hP9iw`cTSUz|;drNN}<`lRj#zNE3@r+$f(kyHoM!{T_*`I)tY4R<) z{39QlreuRMyTDVU%e^%ddbK=qr0RjkW?sfH8~I0tRi@_>=D)!KG$$Fe1Gz!qI|AL{ zQ{wI>?=o=d>}$ZNjO)Qu0b5!8K5Kg}sr^q_F?wT}tR|2F<5_lIJb!aft_w4hk!Y?Q zq*(093FC-^xmLrqu(hJBsu1H|Kjr-`*7NcQ=Dc&OsU2H*jqWD(FK#>EzOdT3cDdxZ z>ujVX_Yzma(}S7gF@D0)*28nsv!a~P3vH>TnWleX9R62*Rsi`BauOF-5On6@I}A9W zX8nR^1u~g`PH+auBPWTD<0I;8?JQ0UlTdzrgr7_2P&<BFh-NN<Le>$VvT?jQuw=9p z>=(icx@b}0AIMvWm9E3E*a^jco*}5Mlw<&xvf)~=sqO19x!HBtR_Zl)w4(9G2}iJ% z+uz|l_6FxiTbjh$KL>Tu)m6i_RQbT2Z%!59LTGm~m#S@d@37-qFFfKOpMzJg-mPuy zPf*vxCSht^I??i;0(W5v($MqdU+=*JZak)@rhY`fW<K0fL<6G={Il-dj$xiqJosS3 z3gE=wIK)mkr++Yay!%c4W0&tQbJ_+*(Z9)OHD^XDvab!WzyD5avHw=gNvwbzepp!g z^6ccQO@1x_pT^Qi<Mm9<=w?a0#=zCj0;y@cqqb`LQf2Q}PIx%HZ+_n}vI3zkFx{=u z2T5i1d9<s<m6p!YQ)z}BqJpx^lb`wyeGDAC{(@?i;<L3=_S}&qn}cs`&t#`rx14jx zDAG38wyB#KsawVN#md{CKlR^34O|MdT^_8Nw06vrYJ9mouiE%quY8ZO%b0{8_8_g* z??$0&a=mL$=iTU^t*5?I>`%h;;egHi{R=4I0%lwZ^X_f*I?M|!1(Gen6}G3dDy6Qw zTyR?W>?eX?1o?5!&Br&>?cZ`qq_9$b+1{p|)gIN<%(tTWvF~JooT@`SlhcM)7Imo1 zSCHCezJ##E6Or30Nw;q98#ct=GtT*5#X^k}S%&Z5)3=Z<N0CxfxH}`2-f58`6*yau zYB%RV?dh21H}3%Q&tWCkwj+VnN#D{|><k10%{uEa*Yc^R!64+^h1oTOAuniLTu$9} zp}}j*Q<<m-RKO0Ee8<AZpGA=%FyJx-HOna9xM2idmVa7s9DEL7yS#Q8RT>4T7=9!i z_*|>{;VWglD~37D(hDxJ*i8-@YtPqvvkr@U00D#qLKnFj3M<?Lh2lJ?QA5f^0q_2B z4LEZc=M`k!N-%=3^HEUz(#XIzQ^GqIMjhmLH1CaTucwy`62#}W&8k%l-?IHS<9Pq* zM~~8)0^b>p!PNKaRNdthhw|m}ePVf$oh}4>8wST6zd+y)SStK}vUd2t;EVrDe)|6t zpE*S~Hrh&g<nsBsk-Dn75tST-{`Uv;(ZNBwQ4Z_-ch<E_(NiY&lan8uE*)xz*|}1> zu%NEygIZGJ8nz7qYKqg|d#%IlAl{=OvJ%Z@dwiy7)Q%9;2{eXu$-qz4^(bFT`IEwd zTKF#83Y;UeIsHJwrZoJeVg2bS`k4-cqH|$$Lj@PiM|2G3n<nZ7sH=X?<yu2|?xZaF zp`hQ0dGxXG<h!E%?}YDB{v>%QYiotKRA?t}#%q?~DAQ<xTqA7^=*j8@B34}W2zsIs zTOXV;9ZY5s6PHz84QvFt{!BnTX8Go>!1~5@nC1L_+S-<Qrk(g1pgztZ)r=-m1x6ng z|4>)|92#k2Ak;kbc{$J|bW!u{;DVn0m8zkNa=i*>^sW&Ht>>34C7(IJCluUkb8Pi4 z^`4#oP)qF7<l3ygZ`9R~|Ni#lCGOjt$4NQI-z)v<xY2A&_{n(*`IpgK(1GyZ9cIC| z6NVtZ71IviiDS1>Yh@OwEKj8tMlFUGw7E%Bjhd`6EXVZg;@tlgK&lA9h!KuW_&@AD z(VP~hO+nzzSgIzz!vei>&+c%n+7vmm%>PadVT)*rA<?XXtDKmocGyb0@Vtek%!u+Z z;&>z5_Y?U!uFP@x#y|D-4b7kht0Ut&8ID&vRPX%DbzWV*iRz|D*=x*e@MY$7%DfBg zMIOdWg9!d9+-WQf9|od<dA3rtU*64$HijGo4WMMq^<az@X}3ydWgbfp$uiS(KKSUP z(#7)d9Y+}3UQQpL$<*$=ReJiIzIzXXna(M$%C<fjy4KNX*ifo$dAjCuMrcZQhr)#C zLAd6%1{>XyxiI;q*1%0;E{>K#K|%8AUoWHgS!~*II`u%c-Jujktt`W9*RE`gJ@9Zd zA$%Q1qfq^%;j6@Lf*`&ZUS=r!dpwQ21r@XoqxZLnBx^EpvCHc)*eNa{Na}pF;h~+Q zj0;I0u~$YvezcJjNx1P<1BNV69VltMm$1NHTegn`1&<d*gX9`goc@`k9fq%_Kyfy_ zBgls~Hf7tM?`T`tlytQ5!vUrK#B~_?Fja_$IF6xk!H!!@>df*YbVNg-XZagdPPZKn zT=bM-`*n1qt`b4btJLkp04zNpy5~Eq35F-G;#h4%yj!3QGfK{3pvkNpZw1o0aH}*H zM!{|N4x}FlHSq<gG2Wwii?FaTg-q5D`^)-8O7oZnH@+GU%R321K?9CenkOm|jPP#U zqk^--0@QUGr3nvrkjh51|E@_zoe6SUo-hg|isR&L2Mpe$Y@iKsXqSJp+CXS;q!|(U zjYHuO4?)QeO6)3Ti@6kl<vkZZL8{L-5Zg&##M)h`@=dZpFO!#8j3Z2RkvDK^#tn#c zM_%OTP&@r8T@ReV1}I&zg<>6(ZGkM#R$RcM&yrro*iQxxo`{qHy8(6!Mp2{!M)!6Q zWguSAj7C5LsKlBvgbxKochFDdzh?@)hF@>n`*x7C=quUkiSMg?)pCz(Zha`AeW)ZM zCsgHobW(sZ-K8<8`-J{fpFt)a*%i-@!9@%|-%?i0Hr9;rz%8WN2fpwq&b|Ha>mRfi z|M`Dq#$)T*i*5;$8xEwwVB6AcejLHvg}>QSL#-jVQ|Z<sN!eF#hatM+dcWFbyn*m( z<+r0Yiivz>QGdgy_tTX7Z?4<ilKSF5b&Okvw*^zRQ`;sxxM;hbdlZdkmBy)nGa?zO z4Y({6g#&@i+Z9L>w~N=$t&gYraPF&QCZgy_<z|o?^r{5Hfr)kbju8*3F-Wl=R0X_y zx{iqlX^Tf^FaKLH9tCK|5|nT<t(a{B>B81$pe~bfzv=iicI)-q&ys^w+eC5$3(IO^ zw)Oo;yXH*qz@0Ns&m8NELT~dvQFP~RcPIDRJ0H>Un`4Ec>#%y#6V`dmK^51tji&uM z7-sSS{u@Qu5H@UY8TF3){J#fi{pb2(Odzj<sC^FXXOS0eA=YBKIfD*jn9lD@2t$}R zMJXT4)Tj%+e9L)L;Y@=0h8HCtF&6;qMd=rUi$j0ie<$jPpk}f_`}$53{9VEW3WUW1 zJ#mCj@LfbmT{JN@keWpT;H*>7QF+F*^NjsvLN~Qmv6M-H;W$pC=UvlgY=P{KWO%g5 zBs1|*5M*%tT6H-t-<cwZYKscoaJ<vFEN*oyG791j@6ON5M;=$yo%{0R^o3!rI(vmE z#a~{&ArwH2?ot7#ElKOy#{XAZqF-zgPVssUuC3-_d4g}M>&9|!OvWbJXzSw?c`}ru z$qgcCAzG!#bh+s$YG>B4-f|`g9j~Mf7Ti;{9(_nX_o4F`m<=V>b#CXZBQ0b%>gtmE zl<$*EovjJ)XFH#Um7?Nw!(X}UO;zZb)->GDNjd0%&U%W}KCyG)y;tv<srPbGGJn5# z*3eK}Rr~gR^Sk=$>gs5Z+{VVzg8JIpx_c^aKlSxB{(5-wQQ5J^9j{*5?2&x@hbF#s z_td5xr<6Cz4qv&uIJ1WRO=)u1dU03sKWde>ZC*$KK`Iz41~gOg2idN8J(ee?m7tm3 z=Cyx7U=5Eq_jzWs+nsZNWoCe*p<<nQqbTQK<LkFaCOr&ZX(UR)&g2Fi3|IIlc=7bg zKZ1;+U5vmU-P`wmq~FmPcAWh<Tv#y2RyL@wxym~~fBnBv<tnfezNkIMrHCXufQdGH z*>ENbVrT5QGVM{WZCxK?*rdfLtXaI&^cSpDmAK<zvy?y*%$Q4fUQh?VQ*Fw<b_wM= z){1$}D`Y4dGA8!(-Zk#~PNc{1IzXD{ea>#tZ=^J#FDI6|kO?FWnncIey?!@9*gA*Y z^Wd<=F`Vf8weX0?V^g(j&erj3%^va2HLobw*I|o77m;mGv-M0w9})YQr4pX1`GLAP z2^ZP@N$ub1%K9G-vlYG(cZ=k3k!<MZlgXRR-Jdrd;S!l850J|9AFsI4)pWbmRfe04 zBSYkGSl!$g;}CIWPuZbIy1KeB+K5(>R;gv@Ic{3D*ZJV|l#CaPT8$;&Q{mu2PA6lB z=rjB^VxgPO*?o2pe_>FhZVeQ$b-HzJ%`vLYo95O^T2U#z!QZuTc@T^e>o}@&);Qt& zY1^x2Q5?(zIAAAzMqnf?MokxD-b@^^)97V!Xy^$Pb9v{(R4fxsUx&4+6hC2?sanDg zy_SLY0=hHG*~+P>o1M9{%Flu5Gye;3rxf4r@@%!>xb<jXp(xOCWw)U(M#c(?a<Z#Z znpbW&qfbJ+T?|U%{9p4o|1;mCq!Y2|WTZYQ#or+sW2oXc1Y`Z!l>5|b>rPHW^R}kO zd-@gSO-|F$38uauQm2eBUj2RZO1x85?V|#0)+xo#>72Emw~Pd;j9@?42Lgx>2#Vl3 z%&MNNbdPWVXVIR&iJN~fym6s{5sUXIn7VE%%KZCc+>WpQF01xO9Ck_AY}&0d-S_tj z*Uif}wyif^E|E4`Vz*@CUO3xx7|&gO?r-qBpgi2YfT}}@Yy1A>rmix2!u#--Btsoq zrftqs9V<uDp$g>?OT%7GzsM8Hpafr)*t!?FF%VM?g%Vx73J_H>kJ?NPI$?GeGV{FP zlCV-Q+l^b_iK444vBYC*sJjF>K)kar#EpA-C%D>{e+<e1lALPaksNY<sU4JV%JD3> z8kJXt1CHdo@C^bu#LGasq-Zli7?9nHx(``A`A3SVxQ1@WrVw_6*y%|*7KOB2VaD?T zDO<?JxCxfxlLE^!unZloCTDflo7$9bl6|)Jl9M8L(=sWZ44kDt$nm5si~){pNbM)p zfl?w~!HmnXAaY}mKpB({1T<+=Fp$Dh0IjvMr%BPxri1|%bh}hBm_AM3w)I8^{R@aW zneAyS6*e-HaawE{Q^yd<h4a1&%Zb;iF+>&fKesId`T!xV<Z72rTPUYmJw`iW%TtPB z-7bPASP@#t?}m}^fM8r#R~PU3XHvVgDX!@hEAlbh^g`}G(igU(p9XDOio`A0sf;(v zsog6&q_{CZ!s2j|#Uxt#UQue#{HG-Q^jPIf1@UF)Ob#7-^%wgPs+QM-Yly#REtM$x z&^aAkJ8FEiEW@eQsl8|MN>%3H<FN<b>fRTfeKo>%cYS)^pG6+st){6WYfPMetF9v~ zBB+bn$m+-gf&;+nLIhW4no5I)o*{>K;adv!2+Ik(1nyv}cn4bQ$EdsWbBK^{lL3ZU zoLKMAw635EsnOxswU58Xe{cM1c_iD12Z<b_Q~e>*avcV0qT4X^Qvw9=T}Byyz82g7 z6fjY-c*_bDjT)T~2UEF3i`S-hV<};St8$<%Q#?L2GAP7p_`F8o%VqMm6HN|>9_af_ z3L8=0WICVsS(tW%>YEIoNN29Ly)};jlG{;I!~m2^A76a}$8A2=$FkdRHAKN%Ory6$ z=Y$oMAk=yh8ANcZ_E?AQU>d%?!groWn9k&Il>~D8S(MIo*cS>~n7IxM;Lv#pNUF1g zpn^XMdU6rG0O1Rfl&F6V(Sq3_kl<ol<}osTK<?$_mbI>SlJYoi#;Iy7G}X%Pv~6!0 z4pL}i$QKkxZ|%d3Wz0IonzM3B?}QyW?ve6b5bz~n*T;a51*g$zgQYR;j`>>*=e&O( zD1HizsD59`cy4ZG=NxR~K)V*4;hCYu(j92C-Jmmjy2L=6b=}TKZL3ShtZ8scpjL<D zZ#eG^G`u}vFN@qkqRa5YxtLDu7c*WdmwFG<v_e7%(I`f$*?>T+7nx#Xt89T^9#QZP zvIIdNC)dG{H9z(8N6t#+y9)63JepcJ_9EGBP`?;;ZHVF>e3?YUf;wb@y!8bT8m>*H z!b+qWD91BM=lMflnjp9+%p+{TB{QjQ6~=6*`c5c|%H}11HnakW?S1ghDP3-6XM8Zz zdCcHRB^GaJL<$Zsa+|a+>0)^X{JoYVdM3~orme$n7=A|T|A0hch{b_F$5|B~LU|N{ z8qgU7Oz`Xg#DZ`dD}5Ox*)|LrqLPrBL;JX^iM9PIIG$75*WZfG`#;xg^$vWzcOc8= zN<%$Y4N_HV3rqOBiLFOaQ0nr!%zlv?jmzPk|05@sSceI8D91!fnu4Q?#T9`N4cjHy zM+M#}i4<mdGrVF>9h9gHB#-A?A-iAR?BPwTJcx+NI^G$9ssG^Z9w4Ad3n@<zeLt6W z-&ex1yUcf(v+Gllze7nvkv4k-<Cn7cLd9;+<DZH)n;Qjd8TNjv3e;(7Dzzf;gG+NA zJM2oehD!P!Qbx?hii>`s7L`~eQ0EW|@E}H0p@b%32woi?+oZ(ed|Bd=xEvY-1~x(` z#i9%g9{UhP(HQa~#o6Gmcps3+q|j-WH6V#wIk|GyXHbB?5^C!qrJBq?RdDX+{VJlB zx*Dln7Dqgt+);nW5Xuh~4AguO*hMiXk*hX<kdv^pXY=UIWj$TTVb0CaJ{7w&m)}>s z@A&o=Xqnwzf9ykrt|*A`h5Fj16P`+}VYI*}RR1s?29$9Uo6{;2kC7utdwGHr(@vyC zDPXCKEZM9b;H5ei7Nq0|T&2niw8IJmv-0->&AH)$FWm#?T({KcY&pG)kb={-<64m* zvm|Gc#$3aaRJ?C7F5K!h7n5?x`~>`dIo^q7W`~?&xN{ZAvCoGoj<nnL4JqMDvR^8` z4bt^m?p{8>e@Qi7tH?zvZJ_PS)#mDWN($9*=J-(AlWQg^25g5<b0+7$;KpV${B+jd z4u1=iO(BdKwqJRcTi_*sIRbNN4?CvSzCvPX)y1sczC!21k*D64;}>>|J_=A!$}y1% z5<~dXc+GmSnwiW{5$KV5E5gLRQ0>5?F)e2-#SpFl?nI|H=-}#US?Y4YWDPw&k%I`8 zBL*pweYVwSlWx`V2AT%<k?*g={z9k{ddbr3us^IxI4>63JQ0G!U<Ka@ikVDMzR;mF z(-p&Gf{`rpcH9zE{C+K(?sw+xfJg(UG2h0hF6oH;5r&ZwoE$=KoAuh4?f_9BRm&cM z$)4_mTi(6%@3j+d5=~&<pan+8cbj+O;37qFfi4Af`a^il@@<F##5Qec>4)<;0xfu= zyLCGXv_N)1?`UKTp9m|2nLN}Y+?2hbPNXASBkMn4^~Q+Pc_NizYLHASymtE&cFmwj zl`w*--Yd+b1`@|RvEotHG=}Pmv;rcO1QWPol-+T!1~HQ0(0M?=suFi=-t=anq29iq zQwOs;Jq$8`zYYz4drnz+-k>1<<B`W-S<Q+r-(u&1I+tvwj?eMqQ~RY3d2VykG4WG4 zgf!iCBg(Z`TCrrvjd?`j4aj;nKmB_7&4<yX>X&TW6O_ab3khwy3I5FAwb~<j5h0y* zsuSLOG9x-KLKF=|08KOdci~TQ`xgq>*xh(XXuLzgbY}Qs2)DWgGTKM@NF;?@?QEbf zzULdwVw4SO&+yt@Y<m#aft&X!p1a?%dYBV>w#mQHU$pD9hqY)#+AsWekiLab5Ax_r z(Df*yL8vf82I9;8C#`8jMdI`mFIk{#VT5Ui=oG059}{%Jp)P`%u!4G>DD935Gq+<Y zXR(=M?RiH1yxSuyXZJVP0C^86Ov=t~;(;BE$!>+()@XS7I?NYREF%_!`FfN#c+8rN zK>p@Btesdj){^IA2c-?A!eqwL%p1g-&8sxH^qIYYUn@$DFn9-Ae&x0OoBVmBJfk7Z ztZ37qxpjo^O+BB{L+8pz*eF%%bShO9A89O*hqsU=P}p6f?V?Xm)rq|@uoGznj!mmM z^1Oh0h?7V%MLaMM1jf){c*O+Hnypd9HM2X5RQ0qdn3YD7LYTj6r6PA#mWKv~pKWLo zjmA(_zWRF{O8Nb=OVv?Yr|ChVZ+Eu!hWL(C?*=o{MFVAyvZhb&t648-T7{0v$XV<@ z<fAj>y7uVXE8rnz;rEC(!ao_;@Wo%Nt;R)jn5lBHBs#k~CGd24pt+}{=pHvxBq_na zfOi!>gQO^!aSNU-FxG<I1|@g-{C%W}dKUHpF~Cel9qex-ZbNRqJm)13jKv8M1vp(M zHpxJ>UjbZl;*ydGJ!5v|^l?{@>*;;G$l@pF55z^%qTJ4O!5(Nzqx-n~x@hix?aF`j z{&h5vfd;M>l+KBERCB$@@cZzm#A8?N0xwiIQGvBNa)9EkN%x7b@2OeW5)XVWpcD2I zTMvu0ty$<VLu?GBICuMk7Z<*S<`XeqN8_F6YPP?}PH)fSdN516u+f@!KtZ6oIv3lf z%SmjoW5idccY4Xb9y3T>Xma3`*aZg%9v|MHc`bSBPl}iNy0f0YMPLF>PE+OZw}+uc zB@0R9UJ?!yhVu7-_sa3e0fOord4j0)eL#Q3Q+*v~0Zi3~mrQ=`h$`$q<~eRrMDM}! zP8GRz?JT=9Jhd^-7@4K^cAz4z@9*Jn`u>qv|90GNXU_$M`khc`+xrD7Rpq6&^Otmb z0-SVSW#T(l=EkkJ4|bHXUyk+!n>_AlwaPo^w#7I0W06D8kv!|%D>FJ?z4Q+6z(M=8 z)Z=a0gV50<$VDy6K~sicF+|VQ_fVKr-kt%}Rm4u5rfBObz!yn$08%VYD+<(66F@Az zOFJW#p;F$gE^rf)o8WBp_-ac7F%Chzae~;Hf0VoApQE%W0n9iJQrxG^qSVLQk4#O? zE8cP|4DM0EaCBLuXe#rn;24+)%=6KFC6OFwBiPuAnSlF<21MOZh*5HK*Z{SNT7bYt zEALNXnh2XdL;L>Q!~DxC7M^WP1kN4v1(ZY*kEYRL0z7uJ0OE<GuS^QG>%kDd#VMff z#3?*+nz{|A(Z%fQM8`EB03+97Qd=Yp7-HaRYh9r9O1ambw^2slKea6$Yz{=|6P0v| z4jB<2eAM;Il&l4P+y1_3sM2z!n*O##yY-#WA-Q9^h}2f3{C$^c=HFmMq-!(A;xKfq z^f2!iXUtj4<drB_UJqgYJ&r%jwZEOz?i8v^lSl}ni6>m7xF0Qk=lA<P-yZTKc)SY} zLt@lv-V;d(AXF3mH+~PKinL`_STp-(Xov~bE~Me0$eb`r-YHNh?5M!od?xI)W1@D? z-iAw<8)8EYrr-lBLFn^bivZ-Fp`v4M?rvQ=h%chO(8Y2@pUE}-9f&X3X={NTn?@9C zLa;IP0^YHDAyu>o$Ke$ZHbt;e+XdR76cJ)jWN_ha^zggLU0^+}eoBk`y@ZXC&)pxc zQSIyFw<R(7DDCzTVqB=(P6p01=yJ@NE-YP>MO`rF|NVvV5xHGhVg46Go?wjE0q^nk zK{MW?Ap=s%7#*3;<j4v<!Tps`ioVHA^RBtws~>CIA7jgFbFuP21{nnyHS-r;SKo8~ zt;&!K*0M<{<Z8b~YyNS<m<xFm1nseRG2((#d?YTN72e(-jg$kwF|j1Lc#{kAUithT zRC|A%NR5lQ_v|%S1=ml@GX0jvi-{`qn3(JffxR|*)0wrBt+`FzX?Mqkrm@$!B4w>h z5BBrTj-*Dkb{INJn6uo12-fa9<HWYl4$`b}DbBlobebORyoMF$H#o%Y_4ZhOR@!uJ z%cZ?eCK^>2dkY@CD3R3P+2tHkmJs@q8Hl*Mj0Sbb79(XAp0g>*&BkCj-*^^^76qFj z??VDg1ad^T0du`#f!%1c8advWsBFF&ptVnF4BcG8sC)@<&ra>2CagRu4^wvTX)(V| z34V68@OAGhq7wmXkrPE*q2+Q8@5et3_S*n<^$Q~aTO{GnmE#Gx*~!lb4QB~D?qD)= ztTP;NrWB3OQCRJR9<A(Fqx$-dBg^;hjQw~+3^>>vG{1!6=|YR+7XG}j5ao>mwPY5? zxY;atJ2n=hG$E4YaM^5X8<lS60or&M1+1cEMap={tFRgIaP?(IY(ttq9g~Pxc!kTR zrF|MT3g6*BmSsGqSm%B}(%~EGbTb6u^#}(8Akkm~R6R5y#(NEA^ZXfdI6hC6%UL+a z6=QbB^O4t~iXyekpbQkg?KGgO*%Fz%YDfw6&Vm>2$x=}PpVr14o5u7xl}#sC>*$r) z{#b0`L5HHGSe=l}JGUy5dZ#op{wZ4UyzH{k+kcv^rFFccQp2(1(*4<k=bYFMgt;=6 zJKnE+m$?xm=1!8ehpRfr+S;C8j(We#+oY|09xA~Z3~<}HR-yB-)P+3$mi{U4966g> z%@FK9&X!3)9v5cdgV>}bB5;Ac6R$6<BpBh~%tORCs8*~xIOZa(r(T=XWox_*m5D{_ zfvFX9UdP^j799w<eJAf{qEf0sDvG8ebsfTv&&z-Q>?`^}gjDVUP-sS7w40Wh5a*GE z`A|h5`im;>uk`7t?0bcv5-`T&WwSKzHu>_2mvQNB5DXti5H<?}o4^t#LLAh!Aj&d` zZMuJT2ePGJ8a7I3j_!R^AAFtaLg@XYhoG~09+>+m@ntxuHMtH;fMQ9?J=4w{vP`q$ z9Q+Nr?F*D1k>bQ5RT3<w`=be_!on39Ifh$c{??(ziiMzJ>X1O95cl9$&sX=Za@S0^ zQd9r~)Zi>0<DZ-&d~V(fP!~svphaD6y@-4RR}*Z9Qft?wWq9Qcpa-&q`9omHa^xH0 zhT0j51|~@5*>LBF6^mJUWk2#Ys|Lan3x`90q-hu_RPG(R?;mweu2kh#hv~WUf(wQI zJ^A8u58G|i<;0zzvQsn4?5<t<rZZIHkTE;hIvTI9>{aWVTDohgT&GkgIO9jpxjM~x z$LFfdxtABxX_k&Lz2BdoQ_vnsz4N%i3HcYax`*-5@fv;aI=mXQ5Xs#?O>M`<L-)Oj zU?ePBhk38Vqy^^y`8tLy-kyuH1dF(;5aSVdV+M%ohY<FC`0h|?DB#{y@e?~{$JI|& zd~$?q8C%;!pEt!D+dp21{Y}_73sp2{PaD<>O9b8krVAz4S^e?@ozsGe6CDSgri}o8 zMLo~%1?b39>cA)mNR)TBaTbl?6MCCk*}p2D(I-TD!XoPQ6Ol4F!QUv*62>=e=Eihl zBr8#VHNvMW=zAy?Lr+~hFqehi1`hGOhVh6}fdSV#MR5Gp*DoHpwd0z=``06SDv_HS zl+qp-)$Kb$y)5c$mJq%Wh)+x9!Fl=M(lnJ0XBn)mObU=jfKm$=5U25w@U$+w`ZvHb zK@zpI$$8b~;*>|gv2McJvtxN@t?KHyZAng&2Ednmosx5fUUITe#@f9PYV|W(hiSeU zrEQY8DjYJ;j?q{}z0L)H(|$QRy)t@s%h;>tsObpvpnKMqSRHN(8xo23tLmh^nl5!L zlW?S1YF^O0(2!|;PH_f#529*KPyhQ|jV?f8krr*ay*xfQKAEtK(1nt%CEul1jYY4# zw{6A9p>8lF+$T^S7}Au^AQoXjQWl_u55lCcj7B0PaWXbd_3wsOFiAVzr@BL4b)6tb zduv|EZ5Jz$j-@15sO-Xp3}=7McAo0iUx(=}cqg_Gv^iu3Ed6tzz~dhr`Hk=-4Z-PB z5Ciy-=xbb4CoCZPqhsLWVX5b~R)%6EaT8r4^+7=x%cSx^nko^r#EbDpN1L1urFH0Z zspolz?XNknVB<(iYJwd2<|g6YWz1Hv+6@BbGRa=rBg>?1f%L3ch3tiU{HtW?dG-CT z2Y>#oAxU<#ZIBy^Q%mN<!>ioSWYwGAms@<F<m{Am=HU`~Bk?MHoXWdlnz`C06YB;4 zjCY}YTFuu3v&*|naT9Fu2Vu%SvglVhK3hDMQwx{FG3-M(88j4^g|-~OM2`=O(!`)` zV?C0>s<LutcX=Xm{i*NZhK`A2hA)|j?Y}Z}e;*j+kN>uWD56Iomv1r$1*csL9e`3R zMxCGx(%9j530pz+G1d&tgQ(;W1-<e$@FXR5)f%p5hXGfH)@KzM6?h>Hr4m5c7om~g zX2S6KP}y<>URJxb4(q~>yYNDU#Wjx&NDmm|cv<c)*Cxjw4O5iD3QCGVH|nNmg&}q` zj&BVyg)M9U$Q($;5(@(qU9x_DW9AzjrzCKiyt!nN#QV}O{oHVtQVR7t_KTAJFtRn6 zOX}(mW?@O>s(<Jybpx{@f~#D-SU=d!KQ3&rO(^F2E{nG}@APPlFt-jb|50^$-V{4@ zs$c5DQ=JY|gF%w{&GIyx<CaqcoP*jP5n3Ha&9?(~<)r6njJT&Z+XUX8TvR?#$=8&h z>aH6^xoExAyIDc0sNmeF+|yXy7_C}n+vryCwyvS>ZK`iHYYO+c5HP=x@}RIxcZ<WR zEhUxJ8UIiGgO1;t%t?k|4+?Na#g0{zH7^JYsej3&h&Jw|2;PI$JYR4IDCo*peoa<W z70F`h;;c~MpBcq))eY+2n29+7T==X6D<K*u1LqKliX}?6{J)@r<zp{P&3wo-xH8cP zLHFnE&5Sp5>BtuYPPA#6_?Pedu*Ze7nGj4K;zbnit$3wn%Sl3W&kM!(l`bBoOK|LZ zP@%b)m_IVMlsnA-Tt4}K%6kgGx)#;=`$0|K(?%e3Cb?OMFaqBWZyjCht4+aFlNP=C zr$l{5Id1}Q`M>#U!<s-x!*|q}2r$7CLwgl(1?b-5*@oPztJwMV(*f98ev$xU6XGul zcCEt-$dHeK8fzGZrHF_%MA}QjHsU}(gtQ>un;QVHA6ti^=rAGb1@$nBIgHwZ`Y{7V zb|nxgUMn*(4@q4X;lLa`2EH5u#pI3<N$DQHxUvrG=iNo2e^5TkfMGJNq_yKvf9-Xc zNY8?|who)&TMJC%r@Mr^q5hDp3&M!c`BB6r;WL?`HQgmKk(veIBYFVU!V;n6`bA+g z^|xpMs-1cMr$opa1)VWa>v&0EV4(2py4bSlUoHCAIr<-4bxWLNq8uIpI5N@gjFe`R zv^Dr<JQO6e(TUB7?DI9oYZ+cjfKt36LID?Thio|;$NL?cr?b$p)%WOG1!L)PIJAoY z^P302(hEcqo0$j&(MPD{GFGI7kAQaF1Kb$PFA*t*$9zUYrJJ27z_EYA_%c9B#)m<? z2zPrP6TS^BvE*{Ps-Fj{@1~CZ4CQ;wdtO)5V2%ECt=(~BDgd@N#4~}eRZ46{Qn9&a z$N0LUVd^&Ieq7p5x@`#<81|}fnwmTs>~Z?dRBsqS?oy<OCrGDk`$URVB<sbPBmS8@ zSJps1r7M^vdNaVIGwh%5BA5&{!dFa<C<ZVcq|ewvq%G5(@ad(<3@q4?m=)q5#YMTt z4V?xw6Be;AlFr^Vm_R7et5VgjT5m$E7V-56efhhD>CMtOA4qi6ycZt~J}<;wq3hxx zRDT-(7dN|;oIoe1`v>{%XyT%hIsd2xMSHlly!LPKY|bmBZ{GK*?W<WF>^Djz*~>lF z*(@)7Y=DSiNC|AqKm>%7NL(rtyO2E1QU0NHrOOMrna(v@br=&J!_iJPo%F5dDrWLc zK{#8np!l*!%!$f^V)~@6)$5w7dgd67u?4RI$V7{j1g3z9%tUX1)bGvl)BRh(z;nsn zL(2$h9M-OWFf>SiKti)8u5EPF*N(iN?Aw0Vd|HXEu6u)2y`Ta27o2a@KuhCKK(CW= z$I#p}#vfHrRwuKK7sepUZv#MhN8BQ((vFfNXn~5o=ktnF_rBvQZ!t?e@IaBb#wRG2 z*m-je9fS+TFZjP+7B}-46-}g6RzNx~#ZXgVD1LC_Nq5QWf#!ACVJO`{hkhdYadg=e z-kxJbVKR2i-XXku7P#gsO0K-#P+O=HmtB5+t)jfnG(Wr4)pNgPWXV@Iw7H|jzM5Os z$M%;x@0lI2;>S(EpOS$Kjd?!I?<D?dLYIoDq;zhEcNm-u2gaCrqN-AK#hfEFdncee zG^9S^Q$x_t`djC7Qa{B^Ba|ukr7j=ye1kW%sZ{<re~4TV_Pd|2dlZc4eT4?$3COVV zCDV-ZPY9nh%jC`<h1)i9`)Sy_DlSR*>&ZtSYK~Uu5p;1FbhA$7K2vi=iWwSfSX=U= zBZOJ=;zb@?;)1T=$kPINgLdFBL$1O_zdGdrbjWL2IS4!;HBQ5}jm>`s`z~RC0A=4$ zK5)7&F9W!c2&Kk$LRw%3Ezi31j(RMsa2P|Q7cHiaWa#>r=|3U{-U@OzOFZ^Zzg+Z& z$-Afcl2m=_bS?E)5@w=A^l#dG6KJUaxbM4NNRvIpREP+bt&nN8B!m!R{v}x^Nyx;Q zDP`Y^P{jOY%Q7Z=*#=|zOR|#@vqjlw!uVyD|NZ@6&vnjoU(a&Rb6xj+Klk;V=k6Tm zbaKWx)A#rNZ14B$^Ot!nVvS@)%w`_Dqoz&YtMU1`V=i3oNpIljyAh`*jiL6@oYQ4R zM7d5&cm0r3@3I$H3*)~0x~q@Ys5!B}+(b>bCeLUf@$J%J(eQDccYqwO%?lG$0R#@X zF1RQNq9=p<pI~(o$V#g0G>kqwX36dm`%ZtGD<>zQ^sn6g!x%f-tC~k9_I34F-zCp) zEJmK4cs;UhI*uK~;Xy71*pYv{JYI+>z#G9Y))r!JrZHt==mE*d-KIyUpxGac!tSRb z{lec+Mh~M{X_r~Vbbr~7?8Yhc_SLo{mCC*^=IR|j_o%#8xvTRP&w;TJZG2f8uFG1P zNnOfeD&EnFYLWYBx)&<ux?;SbbXJ8E=pXrORou#?7svC=0}EbhMRA)WMeiTGT^~i@ zJi4s8>MRgpg9r@c$2CcF9R+PjF`gxxOwK}dASMDhTquyO#(6R@!=fgskf2c71s1A1 zRSb8nsg~79E~Pxe9%ohlTKi2r(p3UqZm~2W<$YE>R<GClUb?Isc4_wIQOBgqaU)4` zcTU|nW4@IA%=-e&Y#kM3H}le4-*9(N5A@a7xywr~*(*BV#62Y4vCL{>*iV9RVDI5t z{YCpXOgC|-k-1-^5A2f+tuOm^uW2#xNeB_1BwK+_Y)8mI%nPiIj+8{t&h7MoXX}=Q zwIfX-YLPX073c~oxt#Uec_Tyqgsk(v^4te!FJ_CZ+WS)Fu(F^wkTJ1j>0V)1B;nWr zZ36K@%p(SVn`@6Ag&M<CW*ATAAkL2Kds$p@0_~=Z(73x*7Ao0`sWm=53RMD7kGGv) zNA492kScB4ag$We%~3c*shFz?Ewtk6#@qbE$(z_h?;s{=3#7!Dg(YBRt7-t-h-goC z2lg%W?8s!#o(I-?P9GD~FaN?#z(58kf%73wC<sI(x%V)p^^L5JZak#Ej_*?R=7cwo zHZ7^S1B)qaCt3&znRAur(r$eIw0XhW^ACZtlY7aYX_J#xl^5{3;whas=YCkzrY`@M zKIZ;t=*G9vxvpgRydKl*v>hS7)3M7T(MzNv_*dfX{QaK@`c_}bwD%t}U+O0&iOBU` zF;$W-8jdph(rlhsqQ6WPjcz-pWp016ikM7p>*>-Z{iHra(DLx_$5RA)R$ODWCWi<~ zGg6XX%!eh%;`at4W3lj-@fH24gO@|xw_mZi_t~|D<p16M5yzmaFI2qhjmg2b<F=Ix zPznDvm7dv|GSSUZ=EpY$asw%tBTyg9nGw^<<pzA^l0pAxMcRsc1#=i;vq+t;PQ(gZ zf?vZC(vkdPU;na-!cZdmi<PF8=91dA#-^5!wx(!Jt`pG~rsEr(W9z@?a<NCD`d9(u z=XkNm^{^L&y_RFoPS3ddIwT(^Ez@o4YihlAvlw}cff}j#M#4kSri@B27oPq^Kwp0I zk$(BXOwuXCqgM<4QcEn48rj(X$`pwF1B}?k?&e9NIFF#4?EY3NEEo}21>reKPzbBI zV_J5UG3M2RZxb9D;Q3ads?m1A?q*XY`te^ptvyakIK2O?ySP@<<HWFCN!$lLO#CE* zgM<`W)6{l=2vFc!V7tJqt~59xR=pt~N(~+~v1V=9unOd_yqHJb$x@w-VFsBjXqA0C zR`;_^Lu>)5g=-6>MgIV&hh-CvdX@j7{sXvFs{~!gZFl@tMb<k>MbDnTm0`HUQbGB+ z*TKmBktdLQ1fNKvSXH2DW!)gCaW0eIUpIwe4)IeEzMv!)^yI4e-YY!8lY}OL3a>vY zcqQgnVTbAt)4l8wtoERm!CybPCqH${tB;+cAal{5F(=k;`Z|&O4Z4TpQ#6{7sfiAE zHYqgR3nC<R4tCyL`YUh!inGc0VjIz^Zrbt6PbYcSJ4zTzZZT!98)J@8g5<O7#N|@G z`wOB@ny>qA`FuLz@udL2JX;Gs?7jSGlhf>vh&xN6ocEA)A?88EgUR+j7G;AW^F%0C zi{}{Dt5_Tr;8?-ShD%u)cS?l?51WS#f4j7;^-hLfz&U;%(C-FJWOJ?z2E+FOx&Hy| z0gTKL#^GImA=(e81f}+x;X7nG?+KlYM*7bx?&+)6M_ahgTHHb}b#{1!ch7b={?;27 z9N|f_h&%~N<~UDMmJu4QpW`MFdBh;blJ+<0L~u>1=<OYue{b1us*W9;HiDQHPzoLJ zh{Wc_=pU5X@aV})&8*o<6r=V7SG6Vk?JLUqn93^2!qgbeG~*{BXhn!hp(ROf5EOkz znU)SZ>%H!LJO;rgb{LsfmP7DXWbG$3iEOlyw4&dKvKp?Sv7$dR5Zfm_+$N?!v$Yl< z*2~Nt`Fh)P;==I|EWze_lZmfmu9D7ybn~a#giO!Cs=`Z_T^>Wy?_VDG9W4_+edm?% zNa5x3T$$^)9()++oRvCsnczapF!~zaJ^AU>jfaxLHnmJSl2>eD93d$oN$#9oZ)Lpn z{JHQj+@B<tEp^g^%c(?8dUj@w$G6BYz(q}WVMhh0fXEEg@FE}UQpiT6wq{Wf%|Y)s zA1ZcYA04Z>yB>|^S--PBQYbn}sjV`bcHQ9}@Bo~9EVdI-)14fW5{=)2^Mgl{WOxyI z6I1hs(-JK6<QjLBYuc6fZ`_dc4UTVQ<;H_w9YH@EcS88o$KJTuf4F>5R&c+`kdBiV z3?W$7cu42Y3+@3ls$;tYl+C@zvth#?W0VUaH6+VCuE~uh*dXJ!nm9B1lrON36c%3h zGuVuY#VVOI;8oXqy`;YsH2k{oeWiI<b2!J7?F^nRz{~=j=g5d+AfllCsiXwOFsTzU zjxB7VNWnIqGWZr2%qf5|Qx@$9{KKTDyZ{!NX&bMH+&68`a%&^otm?ha8_oFgDZ)ck zd`GCw`}dEW3<$XOaA8-xN}1OUgnj9o%rP~Shi*1SnzCAjc30ncITake`evceIk7Mb z+vVt~+d7zUZrE-2uJoN}b?N<U4E`JCCq!#{$r(kX`<<oB>GJ6(#FM10a?JPT+g;eq zCW4vzWD4MIyk=bh{om^PMJCq(QfNc$Bl!_2-5rACgIL)K%+=BLC0V-d6O1N{NjC39 z>=UT-tUj^|-)!cr#MDg61%JIYyl9d@IRC@ZBJX2d+FkrxX$Yh*JCBl{axnal2wFC- z9^YZ1?}nVHr=hSz$lZcdXy)ZXegB8BC^)N`2m5mlC_#TD5Nh0v!BBa($vOJmv0AaS z9f#hdxR2ECS83!nxjoGlNyj*}{jo5fh;mGPDW9S;a~W@JJJBr}n{+^BUfboo^o;to zLVfx7^!D}#$tSNJr{3rsB#-6D+j{tR83xS&Ic;*3DdwWj!qSYHp46Rrb*lyVgdsB% zXr>XdI2pj>!A&aDh><)wwjq7xS(6qMDI^#`Y^@>1xn?dlvxCmFgByzQs+lp`*uyU7 zq2S~y{R}I7rdj$^R-Uj#ZVOlZ?qKdWr;>YtdlD1MC$6nP>du`jU&)+Q2+s;&2x4d} z6XxJ$OTzROIm`*@Pz6t_ImHP)!z<!wd3RC!p^%gWfwpYjdl$&h*Xi2FXN~h)_K$!1 zoZHOR!L^+?QQ-c8afC)uEaP@b&=7Nk&d_aPRzgdUc9jv_&^ipmPSM{Xns{lR5fEKP zShX4a=*S9<S<^IjSyz16f9+$vY*^w$VeiJfD<F<i1tL_?37sDy(W;P=0==(-;oSWn zAx}1pa}ZKv8MoygOKHbTnD$oi9N$9=%%nRjZDSlQT4m_YU%m~?X*xOjO%Dq^-nsM& zIUcVq>D8rE9(06TGH)g&G7%3POZesnn>pq&ekRF9M>6D3g;f^A2t9U$b6+wJr)D1U zlxOwoeyW)>LhW?c$<(_=$;V(F*S~W8ALZ{+y>f-UF9;<n>{RS{9v$`uKeYS;M4H@Y zwqX};S1?Y_XEUZ#CkD7;yW^YuX$g<K{Da6?MfU3v+4hF6!mY8#9y%`-%-@=NTDN*r zy;Z&S{w93MnR^PIK<jGcUh>7uW8t{=1XhzkxfUQF{H;1<vfz<=#woiObfM6W=YCns zbz`gvD{msD-%I^@3HKWI{PN2v#ETYHetrv}RDc#`3MH+B3XTTRR2D*Up7UQK?ij<L zGzjwODbfqB0Uay*cqiCG9ZR}hN_$vO(ViEZ({8Xko5$mVZz`n(9LS3Li<IzJq*>O^ zOnSgXqd8;;<aGxL-w=v=wiGohC_-o{hK=f=^v9d-f+VfkwjH=9o0-w1y9g;X{vuS( zw2i@PCPnAdboQVcypqP_%Wn_sTfGW~w5tpM8q|A^K6Q6PHukA~p3TeA17i>T%YfH7 z&gc7Ate4SCI`^}*4^vg7_T71h`wu#9vQ{}SI5X+PN#-)M-zw!oi~>V#2qqp@$dd&1 zU&AE^d2w&w&j*({2PULmLycX2L&_&j{Jn)U%1^`jl@a&(QmgBicCtu}SpwZkQ_;2~ z4E~g7>lMxrQ?MLiM!Orjw6J&t#YH34_I_Ae`1gnm2aVDzp97J((@NlC1nP!1;}9g6 z*x=ieJefb*se2%`Ru4&*QawE-$xv|W5h$Y+I<Oia#+G=Nv>XKsB}L0)`Vl+V8g=T7 zr>>|9CSp|rUj}+8a(4iA5wQygsf(cfY-902LC=#iAd`{Rdn?t!J4IrIA=OY;aW_un z@CSBsx@vK2O8jPSHp4%F9QJw~Gd8kxEEv^nG<&)xJ0T(I@b0s})|=e8*Lb`6u?Siv zsn$6j;kTkV$&+9OED3Q{1fQ9-jie@I!9oumuIPsl2>~uCBV4kcn5@TeO=Ij4w&ELs z`jBwtF>UvI*9!b+FRHARBoo%BVOj6;9`{wA+C24)LCb2Vt(II`9ZmGw{iS8w%cJkQ z`o)bNMBR0g;~sN1IX)7RjI*6z2es#p<BmrkSf4i2UznLEF!zr*_+@C@2l<)2Jmf&J zD|A@}7t1ZIDRzWZjbE&$YZ-R`<G07h;C3|~B_NB^-}(tm+1mI&iyBLsj{M*vF4Q)> zv@M?|yz?!HA3N%yZddsHnBVmPa4m6x(A>6t0hff^El`Iv8P4tD*@_NV6S`YO#>+A0 zbgJ?Nmu%N$w?1tblY|A`0HY~43z^wzi>%XcEw{Szc7cyNbHU>zWC?<R0Y44oS30sh z-ywwe%0#lx1hR`NhS|;q7Dt-aN;fGQFWhn(jqf&~Oy|qStuf2KcCxy!u@R382JZsD zrns$%sh`h=yze&UK#hn);J#XB#gKb(kMZ>YU>t?7@uDul>2QirH}(t^7r|)O;oN7# z00{0QF9_bAU@hAvPi?o>b^e$!+yzvE1y#dhvz3LGT5mS%(lOSZ9epb4(m^*anKZ@* zCMSA2A0^M~J09V0>fb+A7*l8^zEjPv+**gz^2l6tk!~M#%B%@>zHyRk5KnwDkQC}6 zmwsiBm#Z*W<9R2&(4*kpC%db&<dGTjsq;l@xdob^Z19yYVa-h-{_m9LM>j<<snJML zj3n0%bBK-Xz$J!R=Qaa{*rrhY!BG=CMh@JrvS~D!e}3{(jalxe=?FF3rlx081-sl% zSyOh^YaZgBC&?gaktCrODYUmh*c;?9#b<$~AW5@Bpu#dMWNP6$>T>0VBs+|{<B2?H zMz|YspP$i0jiGZ`UbO|yCD(p%vLl22GD_kE*A{8+`ixZ@xZiKsQS&On$z;&x#P@{D zC#oOZv-9(F)RDG(&|_b)vi-;V5AQtRf4$<h+bH<NwWHTA`=4I5(YCRfUcB~}Hmd(+ zt_5_Y>>a$M>|7ML{_8YEsQvFPbuaOMut_{g5^|0Y=FmGe_^*|z!`{3jPI#bI4_Rsy zNJiue?ir7-L}7Obru^=?aSjbJ7z_5`{Jg@pi``|e{5fE1<YTHcVt{KdKnK~G|GlN? zK{J-?+!}b&>O_KDLm=;a#Q0w0nUzlkBz54rMw*1T{STSP{Ewpu3w^=Y)-O&1uh0sx z0GEL_X79C;rZa^+ivg{rntmCd#|Diwq&yVrGHh~_^@DNbslul?gJwE4))da_`sjHe z@o?(O*{CC*uh0ZH1ikPY*+h)nBZdyBn$rihS`G*()NR2@N8m|HJ9gJmWIMzpd{u^O zbSQ?j^qzLmMhl3qI5zvCCYKt#xhr3*Kf3nAQi#<q7g8|lJDk$XIRwGEH|MZ2c!n4; zwh7L*q7z0r#Vl{mGswAI?Ww5|Z~s_`IwQfZi3LlR_=NE_tel)YQkLhJLCK5>dAD>6 z$n<rF_Fu#nIlmt1MP0sZI+Wygv>P>f2Okppeb{$0WOnw^U0rz%?HlPOQ|<1=2CPkQ zpD{vhw#K=;0Z)cO04cG^SD`#WBRCP@WJEzCy~2f{(!m-ea4zu+E3~%yZbO&R$K!*~ zIMb;K>z!B3#wh`DJZY~7-;I7|g+&rN(bXIRzuW`>UnNjjFc6Qtg8h~i(I3@>C`Xq$ zMi_SE-h0P3X`=r0G4lUXTQb$lXohimvm-u-Stl(<j)@v4ygrg_>~kx_Hy@BotG`v0 zUu&r<X(_cVBJINWR5MF&e@s7pitj$+e!$gNV2IR9X<qoQLmB71d&&tPG|{EQwn4X$ z3Gf<fS9qU!JFXhv+N=?rV$beRSX!G~xd#eaDUa7LOyI1PbJcO{I#TG!c?&|#l0n~) z4Bq}jW%bA3y={MQktfmekN`-FQDEvs;zdn`1b=~CS`F<?o<6}^0V#Z9E&@@b*5zU% z%RHme?LH9FFNzU@Jc}kEd$M!dxu$5J+oOVm)V2WqwIfXPretBl%%%h90Y4dg0NdZt zABR=(K>Z^J=3c^_ud@n>1NluSH%ILTkfte|TAtpdlI0`1(h4SgNqYtIF5h}UM~W%} zp=^h06r>^zW$dSZ*FV?@cNw$1n($186sL;PS?;>L>@-pHw!Vq*3F5z0uUoLZY)26f zZ4|6!He?28XIg2lYNbqU=b+QTcvmqGgBWYF0OfI71<K<nl*cA9OrD$jt97I#Bc(zX zsPdR=>^Bc9Cp(VmCE%s!#|M3#!i6~4l$*tKv?@w8!q)Cmz;3fQ%Dc<VgjIhgsvWhl zf0HwFWb!w-8?IUW*W`|VZN%~3N|T{T-6Nx`BHOMX!rcDzmW^)Wo8h1Tn-_GyXX1yK zZ{-{>AB%EEn;onY>N=nL=ZiNIJ2ksfk6MbWoIj^NuV`s6XX&(G<@k;;@vIQ3Ee<lL z26INmJ?x9j9j^{aUoCZi-242`t+|hH;Q`_J*|yyW_x%f?kQNEEi+5Gu`Yuks)T}f% zUK((yOP@S0Ej?Z;boS_o-C42QX6i1|iT=%^>Z)+mk?$%PLglO5LD9;2#ekE1^3>Z{ z>G)F&`_D6;m(IE;yTo2S*|C1c8YX4GxT>6Nt2K6g<1H!k@Q=H3A21as@0%;@ZK^MQ z-|jAP>2`>-xqU@)UV3`8oQnnd=#KAj*Q?temu@S4EPZy%mR(!61n)2WKbm3RJ~lnb zCoqgs0HXScT1@K{|0B504?+GXY#PVgaMeGSPOx=VLU&IKxtwYGGu(OQ40b;>^ch+a zh`57*Jr1E0I3G+ET?JsWT$@bv`S8O378K43u;OReXmY=HZ@vS;0*At{J-oXscjRTS zSs*tDnHwlIQhoUa^4Icl<Zm5A!MBJFSX;MN5e1qtXx0fB0D=QniqC%!*}?zYKSP7C zo>Wxq^{8lGX3*ohpG>`sVO{2KmB{&_W!cDCiqB+B9R0=U6I%3Pg56noe+*Azh!GC^ z8cr7?E1E4W>r*GX!1&&K5zkNvk5RiitZzHv$si_~ZY_*1l;m+XJgp#H(WiGbNwIGy zw^k0|2g0BrdUP^&7q0EnKQ~DA<hv@?kIv=`bfKkHR$b4_ml?y%(CB*t7$&g3@|8rv zId(qHAg_F?^Vj}DsBBNR+tMy`ZZhfWr-gDe<g<4<)WZquF%5rjkq%nD@g*TIrPHA{ z(>Qf4vxy`GOsbj#r~Y@D(|_Ea4$H)OSsaG`NgO=+MLd^#z4|GO!(2%$P$~$?cxG@B zKpsO)fJ+IELBm#2k2O-eX}}Ld-KAY@e_?4%Yk!G-G|8{*m!x3#U;zv$QlTlXyZgKB zt9;P^^dlwyz2(4~=<fl29)2eeUh-AYkEpKE34BVrf~S$%Qg)*ZD)XCwenI$U#WTGs z=Y=V(^DF1;Z|-SSg(m*c#6=mKvn=?ATQAXBXg@92tC6HKXI#Qfpc0WA`)_qS|Id0o zRv%PimTOLcnq-A-C)K^N%SMX$PHO=qtQC9G{@D&Uoh^bf*_OT8v>$Q$JzgEC0$*wk zm5h(tJo(p`=t+U>&d;qiSikYX^fkjJcdiM9<h<u+V-2uhl#YMITMuH7^Gw*XPh~x- zo&0UvU1#{&y>#ROpD3>jvaxzPQf>Jt`QRrb-4Soy{@SgHe2uR}bfrjPg7|MRfIk<@ zx$g^?$EX6v9t=BxReX92CWakA?&O)5Vva*S1%&sP4C=&~{h|woVd&cq+OJEmfBVy? zIB9YbLdxVAL8lR6uw2vRPgdU-1$Te%Rc)sP2L`%YKd>4_!JFMNFV_7*`sj*z|6d1* z|H$8+yGZrp9RX56t(PG<?28k#$YgqV;TDk-WDfSdOO_lFYlIzeZ7y!dN7rloB`T_@ z>%lg0oz6eZT-<by)>+O;J*7ClSPK2UrBz-;FpOt85QTk)4v`=iaF@<PpvDvp?fA>> z^KbwttdJC|vuxOt7I9x^nN{uH94nAqP<T#bcdp723ZW@_V<)=cdcvcL3avj_`U`5l zwY?yPyR?DQxfmwrA-jO4=w&q>{#_{8KQG`buOi%9w(^u%{fnzCUD|l7?amV9Zm_k% z1pqJTh-h7j_LV<*Caj`#`WX%1TTXveH)o`Y_N<=JQI_~@sW$fRAHk5tzvU?WpF_!i z>hJTkAtJp8;n@$V7mm(rx?ub9Ec+hh^{VI5DdsG-rEykqYsOduCygY$D;w6)|JZa- z^@x9dk1y{6OZocq<;L-Ah~*?q3ikoF_dWJ2yxKVhD>i{zpy6Y8*4QfX#GzO3<(36f zm@vq{d;&O`Ryq<cCG8;s+ZyO=ghPil#y78xZ=&kc!ZqqWgk()*G?I_V|2pH0ZdPeQ z-TU%+ri<>GNOmOm+JxjUsg@&qR%U`9MM$rzfgKHk4NeSX4vRH$qd#SBFm)dDEQvg2 z{##S2zEd9>AxAcozTOdL+o2;Q*VJX$aHtvOZ+pJwn}Z{3CAKEXRpMq#;`ngLX1dXH zf{nCqZ&uF5tcc-Jc-FOIBZnNL7c*}ZtWA@A+T+#B4*s`<%wIYwc=;v;?rGkBD2?6S z{mVT?1f$C;DrNxOy!ErG1=p$0f?Hb3?E=O34vp8?W(0zDv2T`p)*Y)pbEmr6SUJ_J z5!nLaYVIi@5RTx#Npphw#Qyw8;_O3|TOidGgC$OH%7ReE7<cG3st^V_H@H(8MTvXS zYOP_0kr-IvCqf;AOdpZGEqu>QjoxQW4#G~7>J6U>BwSkqNO$mG)jB=)f_QOm-b*FT z42t18=NRco3i?P}nQ7{oEXd#s_aB9_#b}Y6{}s%c`g9@6JtlHm12R?aE;l=J%Jo6> zF|FT(u>c%zr|<mt2oj9e9IePGH5)yFfQ9VELz3ft|3Jc!1`)geBX`VyqStJN;<#Zz zMa0pAxWEC3Y<KnQ1<;Ws;I*fR@}zVx4f*%AvbtjEi@wNRq`Q?xt=Y3mu;}FgF=!dx z)Prx6ufUh}N=`yY&uvywW+@!X^sg@r(MIDBhd<Uf)McDrs_-xVSJ7$tz`u%4w+Zi% z=We(R)})C0$`fOF=L+X2vrjAL>L~{GZ~Zt<l#H};U-_K!I@gTMQs40xO@^BxoiA4} zm`LPzA9K*VS6+UOCk6%O&Er|NDM_2oi>H~r%^p8EflxwF3FwKpoxvVO)jPWakXio3 z#54+fm>-AnWN6MQU1U?At}PgI`?py>((GZtwz+9NbLcNc+d3G+UC1Bb7OOrcYMu|+ z@Lx-thI3ZOOv6IVdw={6Fhb9t*kF**nb&IU*W(9&yR3zBf5od4YfMf4{m&BVOv@qc zWbPrhc}IXWW;Z_$D}IS<$8%w6=LP=ByT%6PWa3+3#W{)D$(Fh`ecHqrpgdK=w99m2 zO%J=BS(Nw&A=+J?8i}smI}*w(o+%xjG)li|sb+Z3U&6AEdOLk`<?W!~nzqyryIScw zLS3pAs%|m0C98Su8(;V>=@f(i8o)wfvAOOWUBF6{Tge){N4tQd5x<Ue(9mfX_x$^y zIBNniiR8mqU}Q9wSRUw=PxveQvSe4eQoHz}OBhn6+7j$-ULy-EyHev&D#t!-7swb! zA#PFQm1J{IUR>=CrE^mz5LZ4Z`7hqDZZ0bHJ$BV(Y%BnEAogbe<h7}r-T|(me{TWS zR%7{ajflYGq(Z9*9zMN)FzdHB7zD2}5Km1-S=4BOB;?o6Py-dpaOXoR;Fuz#>K$yQ zi3x(EC@0x^vK#wi-5<RLJS`R}CYbNpF|9o!xcGj+<V0xQxkqQ-IAEwHm}{V85Q*&- zw<^WlgMw)I$xvELroZY~E!&vm!m20+tnDOL5q>Kys04+tmW}7R6MU)CT6s|uvuW*0 zN9)E^lP5Y;o-P*&AD~{swH-Av^x`^LzQeWIVImkhd(rv8IudLhL<go^){pb!Ye-37 zYE|YU-=Em2T}kVE=r6%xh$1NFonf<a`SUo5?H2cHd~2+g+Y)9v=?kxQnwrK9uAAt+ zR93%L8eVCiLY{UW)6#xZVc8hm+<c6viT-^4M{0(AZdD*o)@Rc4$mZ0MbgJjt1&%&H z6JyMU3;L(Upc4hC&|F=%VogJLqUkY=-Ix~RX$NV)Ko44R_`o*r3`j6l{{<EpwA3>d z&Fw#0>`65*o=`ZWF`DtIouapy3V2jqlm7J&H@vATsg|0mRUptm2f^Zh5})e7)B85T zIT`$HfvjK*u?sw47Z|hIxDYx{QZ;ENy7td`2_LL3OD$1P^rt>?C(jak#6WrJg5`Zm zPZ(}m#@7aG)pb<~%Q=!Kd&M*4&M@L7eW#4|#Qxb<gpWdQZ#jx_fI``z!Nt+x6SePx zEGr)a&!YlMg&0=TWp3c7hX}v2WslFe(<TNfXCmq;x?c|wO9`8rdq~}9qAXdH3m+vi zMvloK#pX9J^>7WEv%zAWv=&9|I9|-3C(hJd3U3n}V-<b$XX2mY_wz)90t$Jy&>{8( zD<I|{5FFy28LQBmv5GL$44>?^;wrqW&}$B`99gMxzw>3os->Ye?bi=Fu(GmcTM@Dj zoRj?8#TwWS?1%xypHspjbyLI&za#-t;3wAJ#K%)S3+Rw`p$Y!0;B-~(aZ&<rXJC*i zf!;G~`0#;Wi&ECOBO)}_+peQ)|CMWmxa1dUl(#>#1vtKDQE|r)Ig|H88JeOB^y#X2 zE}UO1NUI*;MuI!C&@}kpb4T)i@hh7`*lG;WHHt#KA8#effZa?#l1vSJS`I@A>|Q1; zN@;AkgR}$ZMcjq-ZGb1}BX{{_4)iExvb{T}kBtW-MbREJhjJrMXopZGF6_@FvzNJ& zbfI=wG-<1;JY-IXwZnJMdF%pWH-C*Y%4Dp<kn(dkid&q@#|H4{W2)u4|D9z)cv&Nt z`_a33L?^(zxj8qyWF{ThpK|_)xdAQ|AQxye$GE&RtC#PNs?quSgSH$Oa%y#U7^6_g z(`F+(!7ec2!k)@%d?#Mk<U*i{xPP+Ctz=)rzhu3CP6Z9#_*LTJ+V|-yaw<`U=_OE8 z@}8>68F6gTG_@GcS3I0Ahd6Dc;f#&HQC4MLcG~!*-KF3IbYvD)>TrrcxSPF<oCL@1 z*3{3xw`>K%qG~Wc1;11xPi1(ylNh-U6GLbGy~UPQ_kxS^kr(NhD+H(CsW>RVhcR)N zA;!S%GA-7cHzgE&okGl6qrpg9l!k}#tRNwFBYiyKU|j=?lwvB*vtWhB5>^gzFO^f& z;xLL})yCT!XAeH_6|ZW_eh~ZZ#o&ILk_lEK!%)mA%A$yJPpABLMiORYxuVY7LrB^| zD6Y^o+xN)J%+*WH4t!%{-SxNeP_aOm$czYLAbFBRjtD@%dN*WuU`_amxI3`@sBzi) z!@uTNud~(K5Ia!vNo(UCsO;$6<zFuKjgQLJb~##Fz2y^->G&d$fc?8EtvmlSz6P_z zy$TdtJ8*Yxqe7{>vBqdCfrJjspztOz(Jf?o<b~7wl`sbEvDhJNIzA2|+7y&;B~wU) z#fCY4OVWxiE4JOkUP%?kELbIEKWd44`-~E5)3=L@?>ENSKveD)h<X-`0nq3~V6BUZ zU(ePN{TCj?)u5sYpWFj%MNS@KA{h1zS-YN`y~h>Tj!52_D6<(5mv04JVP6~gc&@C+ zYRx0k#jQJTA9(tga-LEx=SMQ|>*3>-#Mp*pQk;@Fi<*Mj$+VW|l3TP{-idndhz<`~ zw!zbq)-g(SO2yAGLQ@^3C`WH^uZphO&(>d>Y}qb+C3WcnsEYXj<IT}b>4ewf7UPO{ zWM2pAS|Ed3&qG6+e@@m-#vuS-C_$h<Qq5f1<LcX?2jAP|i<>P7bEQDuod!kM^&W11 zD~tUob^rI%2k}Gtp(q|IRG=C<b*4Y#^RK<38jw`JT3K-wj`ow(zSyW=$yt}C;j5`j zx-1kYj}^6&AlQd~2Na`U<!k$+F(EX5fYk_3jjcuxP14&)RYbPpq)ipksbiDZp$Mjx z@vA$QYkZrbX?tbKwjXYmQok^-*|v=St`tf!aAYPUyoeKsE<>a+W;-h%gmYvqZYi*6 zj^b2V0a<l4L??Nb5f{1K(=I6SKCf{0`=41|*&#$Il|xy%!m_T&57_fCD<*c?vwj#` z=Q#K7{C(``HR!Y}W-2(emQdzjklN_oYZhgGx*auPgoB?%rTdpoZw#tbOvk}5@*eR) z`Url3{A<HNSeM3>jcGc=bib$eVW36+r<3U+D2|Qts7n%`UO5k$vnI_(dDaA%C&!e( z)DC$iM0EI)FfYo`H;^p5aDVHa{ZXV_Fi4#kugB?C1=IMG(|I&ImA$qT+78xGJ%hNH z`f4^-){(20EkeJSU;iRQcP(^rovVh~!q3!0@pWna<YrAWr3HxMQS9yxMH@Gg6vi2P zU2gn)-1IP*57>58Pob#$YJXRVT5K2*_`Qb?pSpOpAW&^EMB~(Jl*Wq!yhZnGqKhRE zEYQRiGL($W-jqK2<YX(DiU%Fc*4MuWK^tYK1wW(4jq7j7bg%EWop3?{Ja&Yk1HTmE zostOO!IQ>n-?s3~F+B=pw&PgwE*Foykhs)1t~HbRrpfQ|d;Ye8^(o1YP1Dy59Wj)= zb1Viy_&VtqX3&^7FOyZ5QZ`rr;ut3@`gO@yYZY0}$@ZV@Udg6=<#N{}#oSsdD@Tt7 z==Irr8Mhk_QE*F8ykOf>AHCeQ*|jWN-&oyL^T8~6e(bu+3zekN7lWVPYgiCJ6BPyB zrh5EWwP%n_98WTS@-oY&sNR|Zi^El3*~n-R0#uLlZlWIpLS|SVuT}1hb3&k*Ab<UC zPC#tU#oB>d(XX_pomrQbnu_f!6*}y14A-Kye&1s1H5NWPx~c8#|1#(KP@WydFyF9V z*FFJzucv!-ReMRZb`y6Aow>Ne=C<N$#@lf)P-24v<6E#GYcv~2Kznz(-pCS4YTC{O z9n?^P4EqNeuLL~|^#&cJq#7NWIr)~?C9c&LaY4nplYN0G6<G<ce}J4>U?SspzH0FJ zHO}#uUc#48-l$5bcFzJC<SUQ>{9k5}us*ot{BuDc0#ISpy08bS#Ta#d3((@I(FWOP z2JQ^$=}?Mk(kn^p#OUe$q><w1=J}RP<TJ2+CZs4^QaXOY39<_Cz84J^U5Fy)La?ra zF`T#_d{SE{mh>J0GVuqKoCb)jJjX$b*M8*z*Qqj1Z83)7rg(nx8ml-(a6&MUs}S#o z-;TL9u-<EI?RkOq<5|lTbGC6H|3h^D<$5?JaLqLoaC)l+!gF*g3<E4|k@0gBViz@9 zUN&ZJA;MT7qQ%moQ{zL3+rE)+`qXX4@boj=iF+yYuJx_3edmlWkUH&fzL2J4{JQq? zI#}SCkaGqT^_q4cYItSIxb<-|6?f<4y<g7?tL{0cj$73%`A|n6-8<PLv;PBqW?5B+ zQ2sVO|5W$G)EbX$mC@y;O}k5vR>;T%oJfl<`rc$0VmDabBg80vMpTY=LrP+d*fB9Y z)StGBpRsqvWQg?U^RrX+ovS)4C(ri#t;}SoA5QfxNZx-lX(q$P-7{W2-~N}SO7dBY z(SIQnrv`dd`B`4px~y}j_R24c?}wU^O!E($kT(Dovt>JsgaFQ@1hXIV0EDR}ajrYE zmwQa`B^O>_A3oWb5=)g{^LM`7!p`X{4HSyPz-iCe&I#+iJ)a+DG$11@_WUYIRegcM zaWR;~)$D7e-6ZcM;|_S8_mYB9w{#3pVNT>6<AsOheVejALn!>wr4#os4KIS(!fOB4 zb_jnJthg+!Pm@aVM7_IG7N9l?%)gn7mzc!em<ug@BA~kg5gB|R0!0=7Dcd5=iiyD} z1eW;0cj0dLN6R--ZulWZT6Qb?jZH^{c7(m*I6Z(!T^kt>vx;M9t0TK8<5mNA2{p5< zAPgf~Ch4-CpT>uhVyU;wc@YX}u16ZQR{XU}mS$Y6G1h(|rNhCj0pGgOkL8^B@Z!-W zRM|J~DjiNn!T`_Ff%VTM<5}Mzi`bqL(;^C?==l-4H$3n#NR}KH5Vhv{=>GTD%cSa> z#()z&s`z7;N18=I`Ndq>_J8xR{D1Y|VV(Vu=8wfbu*Qm1O<V#Q>dqa5cX?i*;Hcy0 zwrqo#T^>6@qnx$<MjcD76D~d9vWK+<{$rm27NSKoN`aujM3T8Hl5Rc?;XggJhC+IM zXrvdx?+7|dr!SH!Hx_^Ua<v700Pof!%Kw0BQDsqM4v&CG;)~0@d$q7w>VAxa$+SAu z$kuUbX}*&Swp##ApFgrAtNhoMPO@L5{^S9lgf*}oaJV)12$TYHkx%>&Cc3mf?H}&! z+sNwf6rKxxYFpox%-n;c0;Rd)ZjRB;az9b4pa?{?#HjeL4b=Ne-V92p)$|l=yklh> zT9aWLxh^%zQ)G*_dmLbQKU?>Ca(&%a3|(zrJzk?_rmr?*xz@;aIlHJzB+{L-J6A0~ zo-<E3d}}{D>-*?K*Gdmutz*MV*k{7J|Mg_3lE*b_rd~Nd)RO>4bBnfV|McGx1}2g> z16VMDh&NlZRbHCJ33!a~Uzy<A0mpS!hHdqn4A2oQPBDlvRAKC~g{(7?ccafZcf$9y zXz?WR_Tpy0`hr*u5$9)Zjo1|1p)lTe_DePd_D0pRBrgs1S|Ge_zou{!*!ZLhE#@O< z+a`XYKu6FInlU#E{B(CZ)}gqFi4DfoH+4l2nCa4YQl*tH)@~Z!fFjN`d9uv%XMPtR zjTnC5;TCL2BvuA6(@#;qQ&OlqWx#MOc>ll6!kter;R<M7IyELlXFujFJ2cuvl&&QY zukjP8vy(}MXisv(HMwql2`X&ro6OkpA5dYs^K_c;k;$sFY5pn|6T^eZ91e+PhueX1 z{qS)B$mnz;NlK&u<i7F^`Cy}!R@n9&UE0o!^5g-sY%Ic$7#|u2BXj*tJ}2K!vl`dW zE=X2GFM4Z;H+zzi?=zPMxi-&6z<olcoFC_SK;98`4&38{cGu-%f2Lcnut>kmYqkUD zGutT!Rq^fH?+0g^XGo8Ob%eHf9u^B*HjprUk<fpVcNj(ChT(f{hPX2<VvE4MEUG6E z%D`7~q^J~pV=<5vXJo8iL@JW}`y&zCG0sI8^>~~V#v6bzEVpL2k*YRY6}Q>OBjFgE z>|40{UK&B3rWT3OuDnuZON6o(E)HnDH4jg+IsQSv=_prRFhQ*|Pa|L_ar>4@w~3E8 zx<A{3e%=lCdIw1cgDkXA7AwLI!D~q|owZ-^{(6V!HT(Bsblp$FG*&)kcV#t&=M#rl zz@D*l{MWeKFp()hLP9~Bw18&~PGv^ZP%4*9@w8=7<>%vvPK=u#gf^~cE!+Uho@9J) zPrX~nO3n-8ep!groD=zNf{jS*@{ppAzC%>YuITgUe5h|NL$7SUkuiVP?iwOHhSIL` zn8!^eQw>8W@;_lkFnb_l?kS8R3veSIgXRfFk(EuyN8L<Sgtg@g4IW$5k{Pm!(e<GA z?JT%iD|Tyst{|7QIaAcs{4vWa`{Qh``HN=Pcun$|{M8?fP@V*+kUPOz*C43B3sFgG zoeWTzDlZFX61So+>pYt_9+7u;*+WT5!b<hOozF@dx*W^4KJzWL^{0}o*_QE~bKn;E zr)7BSQEUi)0(ptuS!yYT{|YodEh-pqsGi>KQ{Zw!C)!kvXFj?%$9$0IWnO#xwNU2C zCY%<waPV58%A0Fh33CxUm@>nG))jRltE6c7Zmc9moK;JD(y|9^cMD}Dv>Bwz5rAZf z+~y(Af<+O$PF&iWK_v^MhPAW~ru9TheS813PWlG@tWw-1Y32XaW4C{UUjKhSGE~mm zUvW4?n30TDOQL7Vy|^9kQ(&KZ>Day9sRm|;UPkU#5sw#y2CFLwgoD`5SNJbmG;fge zv7I<co=b({qAHv0ND>E2YHl}lj>tC($UxDs-Uy{4L{ah{g(WCBtBkmEui$Bil5A}P z+4s~~lktond{93SCyqUU-p<%~(gZ>^qd1bM!<MCwC#{7OVGy}hpf|9#B7t`6ps3DA z0S;_4%&9(uYFCXpI&tjo_QMVIbc@ASX4xMn#EIq}sR<OjdHgKiocN+Usm;FoYOu!K z*h#m}4P=_T9ot=c)oX(%>CK7f;|qTQJZC?pMNF3-(q!tC&eg$AU=`TNZUA;Lm8eU( z?Ubd{Zkv>aND3yx!6Ma}=9kCM`QIc=LkqT5Nti}aO$+IRHhvZe8X;`&62g%mi@S|G z;BJ1{<s3r?s4VKc6N7Uk1dKmcWC-DJp;eAvbu-A?W0(pO3q6_6Bgzk~Yt!d3<fV6k zN^VZVqi^1?&KJrdK6dJj6Pm@@qaSc5iVaHI^_hcRcF3?aYpiMUxb!M<NtJ8OGv>d> zC>NnjHQn7h1gF>)r6$TO6s<eb->i|1V&Xf(Yqpz1k64ZE`$%}=LUq5&F2-He<aD1Q zKiAsP4E&Cb6RR22?Ic<STqb2g-`$*}#E<78!iD3R&MZMCJAgxlif_(*9_-Uj^CX~+ z>G0^_ggN_@oK<&R`-K3J8;3qrW?e!%FWOGHfmEF3#`%NoQOm|IxUpB3wsNmP)fU!5 zo%ouS?kP7bc4+4#o~gTb1xAjRJ{0VZFoQ0z7}i%UgQu0i=sSNd_0z5Jzba<x>N8C5 z21wBJgw1#4gb{u9m^*)X`y#AVXM<@_sQBrw>s{}L+&cA-A~ef7!=^~ezieqKiwPiw z350|yc<%iC5PTG=lDH%Uh|O=ufyN!iC?*#{b8Kb0P^V5*m`4Kr>l%J5w%=5Y-8@c4 zNTM%v=gP3mUWb-aT&Ja3-@_{M%ls(i;}TXS`kmnz66g{L(cWQ*NmSMq{A6!ySk%Xx z5y0o&j@(W1(-A!x85mfA#C{>k*7CjW>(vc+W|BFXh~rLm&Yxh@^O`^2d4Ejb`Ke6f zMdQPl4x^3r!D*v))6KSFEtZ+im@jF?nyTm?f<)EeluxP~y#LG(p0q7TnKjABGcr4H zirD^f_J?Q=ne&Q|$N2<IbgoS%wJA++ZU8BHpTrkK=YzUhy<IQ`NPo^%hbLt#5au)V z;@04knH<E}j!)%r#n>Uzu8P9n(GQ#UL7$r*z!+v%Kb_mNb%*n^XK6!~QCWV8v-#Ha z^e21D3vRxQJSX%t>$fcfv8ccm<9Rk2#dK1?_HwR5C0H-r>=n>mt*&o}AMlV%wtV#s z!nbYMc&WUpTU;G$-W_Oj6elTW<MC#Y_bpkN^+lQ#ByO+SpMgw`V`I~}_Snx2g8oGv ziVn@DcGlCIW7|oM+05WmL%qN9&U33LrH83t4s@dEY`tq{Cn)72s*7J`%1)cI3JdnW zRu1bkEidhP{z&M6b2*M&jH6{B*OCxlw*x>ol@kI;C_#XP5-Je(`g@DTF#j=v4Qe58 zq##tExH)c&igf;a3lXG|&rgE=wTEyU-2p5efyHl}6TgP!nX?_o0ef+{dOnT1B(56( z)=e4SF<Rkz$EO7_JuWbikOt_9;@bQ67*gxui_K-oZ-3pa8^lPnGuz<e81taxJD(W{ z<XO9jCG(P>&l=llB^d0!YN5ZmQ<f)@AQ1N7QP~IMNg_GhqkY^}<FAuE@e|XhZ@t)m z((*tY>!;NH)_>%SFOV)6V@RKcql=|*yxRk!5MnK|b1n8{KAyGG+9JzxOi3+}j#<o= zA6m-)^~AA#sg-hy@%DzJJMWZ*`ItQ}Jt1Y!0QY(c$)Zr;%}|ZuNdX>YVkAD2v>kCz zhazJ5+rbWU;KH!On)Ev_s%PHv&O!;SALMDpKWGxk`a$Abav-N}^8hxWMd-2(n2V&H z0BS2xWgW>0meI5Yif6#wXv0S~DAuIjAFo#uhU6mHww;o$9TuK@Ht*o|%6>hSTo&b$ zFFkN%T=`TuqAG%*M}bvM^~x1^nAbpcY7PpgNBDb7!8?3HnDe3?cxI17Suz2MxO$y7 z+le1nUSDxmlH{l0?#!?Q;{S*|#~p@)-W{%Eb1A*=>z<#f-K^!$+N00S;D?)o=tQ3Q zB<CP2Gq#2l3zun9WWp!a7++qfe|Qf(_;=9@&iwpZ!pS+CpA}77H2nUpwEmc&?xUfx zkazp$@i^*<C`vZYUY)4XVkjH`_ZD=AP3hwI-#C_H=C`om`{)}8)^S!5_gxvED^LwL z1}y5KvG(?{>8`F%(+XC`nx_uefz7D(NjK~3YU`pLwU(H1;&$Ta|79zxMyHhX(V_oY Za<l*SzCaTU)V%-e>;Iph0q*aK{{p0jQFZ_T literal 36919 zcmeFZ1z23mmOr{1cXw+NAh>IALa-3rJp>DMaCZw3+yW#df#B{gf#47z1W#~xYqW#D zCg;vQnK}2)y))mu|GV@4XY=ju)wOHaUTanDs`XoI6*u2*7680QvI?>Q2m}J2!2f`o zB`jS9X=xKxbro5K$1?B>0DvX`*xJq>0S^Fd?OdGH<sQ)L>gm&>egO~wbN~h51^`nt zX9pz>Sxo?bD9A|DI>Rg7(!U?469Bv=02pPLSEZ%>z5Jh|@XZ{YTmS&13a`y;ZsBYO zr(eM7=k6{Jx8(<Ln&>IymPSCjrJdjm!fCQw+Uhr&?Ps0eXvtgJ+}6$<Ugwr)2XkBV zTe=fYzjSl8fYXSN;dH2*wS_yJ9)r{L&s}Y;;WP|R6WLmrIs*U_)@`|qh1oMW%>}2i zoHW&?;j}0KpkZ14LO=b5cCqk)^9cab_70v-)>hA4Xqn8IX}Ry;7owH7aDQ&$;=-<K zY6dZNGN+Zcw{<YJ^8|q3WWK!%Ah^{nEnLY0-1h|p*m*eM>i_HUpI-cHt^b_3?c3jK z9I5^6GZ4wZ?`6N=`+J#vCIASXz;zS%d)d=B08keK0Ay3Ym(hO!0NfV<P&@RCJS4Z{ z<(Z3%gD@wjySqDwwS^hStwR5L{KE==t@-zfU!KQt`@FyEj`pF2rK#(47us8;n%O_M zcXgt5b}%)wpk@D$gZRIF<1cLe!UvnWg{6g)g&q7++VD|kZD$4VZaZ^p7i)VvT5G$% z*TetK+kWBUmi%p8!wJIdYk<gv1Hc<10uXlx03<?80MRfFeg*W`etV3r4cxwYI^gx+ z#yy;dU;p{}?>-P>;Xe_at*vNpi>1{yY0X@n+-_<3J8}Dg3}67b01-eA&;WD*6TlAe z0D^!BAPL9-j{p@w6VL;U0W;tkU<)__?tl*v2!sNWKpc<+qyp(cHc$YR0+m1=&<wNz z-9SI^6&MGmfdya{*ai-OQ{VzVk)eXHL4+W35E#S&Vg>Plgh1jT8PH>pI!F)n1Y`xW z1G$2HK*69$Py#3wlmRLLRe%~mZJ<8TFz7pI5wr<91f3%wB48m9Ay6aSL*PUZLXbjG zM9@SqLa;(`MDRj*fe?f68X*&*7@-!S4WS=l9AN=r8{q^IK*U5OLA-;=jwpmEji`dC zk7$YLgy@GDftZY#iCBu*h}eTTinxHdgZKjp1&I)e7Kt561W6uA3&{-05y=lJ3MmyS zAE_3p6KND_5$OQw8W|gz3YisI7+C>X7ugEg9XS*^895iZ2DuA)9C;P_6a@u^1cebr z5JetE55)$>8zl<mJxUqMCzN57Wt0<CR8(?QR#Y)m6;xAH7t~PHH>ky^t*9fYtEezE zOf)bW51K5RKAIg`5Lz-?AzBOC2--T@1v(x&1G+G}3c5MECweS;HhLra5c(?m1qMC_ zBZe4;CWZ}0AjWHqQj9K)8H^)LOw7BO_c7HltuX^IUt^YI_F^t#{=g!@V!@KeGQ@Jl ziowdm`h@iz>j)bg`yRF=wm!BCb_{j_b_ezxHVlUlhXY3e#~jBW=Pgbx&IryPE;=p) zt`x2@t`}}HZWZnj?hYP0-aR~NJX1VBytjA_c$0W1_=Nb}_$v6e_)++U_<i^r1gHe} z2xJJ%30@Fn5p)nN6Cx7Q5lR!95e5@x6Lt}<5}^<=5-AYb5JeJ|5Pcy!ASNK@Bi1JN zAbv~yk$8awk>nnU0?BidSdvPT36gVCYEmgu3(|1XQqmF9GcrmtNiqwv2(mJ=ak6uA zT5=h32zfkt9r-K;5(Nu|8ihMW8buey4kZz#2&EZi1Z5@V6cqv$3za&RH&qtZ0M!XK z4YeG#1N9r~4(c5m5*kSw8=6-%Ei{{8La;d48vF{}3f`h6rj??7PWzg+llI^a)gAde zu6Hu-e7SRRm+`LF-JrW=cW3D^=!EDj>0Z&b(;d)*>6Pex=!@v57|<Dn7@jetF!V5- z-DA9`b1&>(-MtM)GDZbPFUBIq876Ed2_{FTOr|kr6lNi22=ja9uPg{G{4CE{-m(m_ zBCrauTC=`q9cDvjyU%9FmdQ5Bj>Rs;?#5okzQ{qs@t7lsqmE;blb%zbGoG`D^O}pF z>p538*EBZ)_ap8g?ndrI9u^)mp0_+>yx6?5y#BoPyih(?J`27yzHj`5{7U>`{O$Z# z0)hfg0;K|*g7kt<1m6ix3K0pZ3PlO^-bcRw;J*L;=KDW{1%zFMD~0z(*hL^Bg(4fG zjG`8z*`mv0cg0M_GQ<|dX~mz2e-K}gxFcaIktwkxNhfJ8nJc+2#VlnbRV=mlfa`(N zgPI3t(!$cd(w`n8K9qYH@$icbfsB?+s?411U0EyHBH05uemQTsPx8p}it_RDlM2)d zW(oxg2ag0F`9A7Y#8T8yOjTTZ%<|ajaibD~(j%orr5R-gWjp0M6+lHnB~fKol~L7E zwMh*{O;zo!+L}7Ix{rE~2BC(DMxn;3<^#=G&1o$rEmy5}ZCq_b?E>vHorgLJIt#j- zy1u&odK7xrdiDCK`r7)r`X>f521y31hWv)1h7(4NMjl3e#uUcSjX#>;nwXkYJwbk= z^Q7>}rKyT(rs>I3g{SYH9+*j+rI>A-OPaqj-?R|3NU&J96thgQ+^`a}O0?R1Ch;u! z*^afeb*eSgM&2gf<_w|&$%EWH*LhxMi)Q=8w!x0T4r147Ph;<4KkUHj5bChtDD0Tz zc;KYyl<N#~Hgc|aA%YuI1FlT2A+C#V;%;x<VD8%PRUY^rb{+$sES}+>>t3>6Io?R# zX5Q^ScYFeU7Ja3BGyFh)PyO2c>HJ^#uLj5k<OgB|LIMYaxPlUbj)QfA8(&br2zap^ zA{SB=iWBM-IvFMsmi`jyrS;1%;k@B*!mlIDBR)rRMJ7jHMwvx@j^>Vj9eopH8S^Dp zAT})yCC)BxB3>dsKLJ0%D`7cNDY5ny?W>4a$4MqheaU>u=_wc~t|<$zAHS}DL;oi3 z&1LGd)bY0u-j=<idKdog?7jK>ku=G)(sb(d$n^6M)*rrQ$Ys=K-pfqRLdkN=TFchX z?#U6#DafVBjm*8uv&&n^SI_S(5G*Jtq$-RpLMU=A+AKCK{#x>|q`s8BG^329EV3LZ zcP-zlcv3M@saV-wB~(>f%~1WWhOj207F6q5dr<eRZlPYUez@ULLwlo0V@(rBQ(p6( z=C>b-KgPCTw7h7!Y4vJ7`Q-3vr_HKux!t6Fx<j{PtW&M?OP69-Z?|lBM~_rbYp+;u zbDwZu!)Kw-b^QYUwFCSEHG}+vHD3h2)D8&_)qlPJwQ*Q<xMf6Qq;2%!X!n@HSpT^4 z`0#|*#J5Sq$%Sv`-!{HK{|=pUo4S||m_eS2n8lk-o}-$}oM)adTM$_IxcG2!U`b<X zdf9Awd&PO>a`nX;_FD2fc)ehQXQO#jcJu3&!Pe@w!}i5a=q~>5yFI48>V3)mfdk!x zRj3p6<}m7r{3!33@3{R$^<?hU_VnT`0!9wY{~`3F=UnG}{lfhc_44%<^Ht-u;`Pjp z?aj^2B9)4ahqWafAt<l_XmE1?eu9vG8WNreaPlv-72L9*03rV@pMuw-{0qGVr<p*A zKg;3O0d5ci+=KvtE%@mih6>2Tt#JbQz1CsBwP^=0kAj~F0K$jgpMv-o`t~We2MU1r zmo`!0zy9sH`q;le_bHtI^|>H8#JIiL240Tv7kU=wwk_ZmK>fV-Z{Mn>)^;u$&+Hv& zRh{f<xdl16ZehaTI{w61aIO4_vHru*_^S;L{E4ytQ*ic+-u}c`e`2gZG1i|L>ragJ zC&u~{WBrM-{%^uq@^Cxo7MuLr_szGPX+Ro4K|)4BMnXYBMnOeILBqhqz(7aGAi}}L z!XqUjBO@guA)%mV08>!XQIU|)veDArV`OGwCI_=~u`_WoFflXTHUffs6EV;*2r)1S znJ7ponEuPh%_jgK4bTTZAcF1y2>2jGe9%oN0ERP+40qbXkAJh_KnRF%eo@iTF)-m3 zYVZIA5F#Q15+X7(65NXn@`YaqknoWS?r=+?5UQD?-gPA635?G`qkB-%Myx)3K+pTs zDF_{dgp`b&g5e$`6Eh1Rzks07ePQW`GO}{=3Xe22wX}8MrnQ;5g{9RqYa3@5S2uSL zPp{w?A)#R}!y^(BUnM1{ynd6KnU$TBo0nfuSXotFQ(ITx(AeJ5+11_C+xK~7bZmTL z^4s^R#iiwy)wT7F%`NER(ecUY8SKaTty~~Du>7~<H_85kT=;Og5Ri}%kx*~t0wK7= zPegnq<U8Cb1X60KrjCSnc>>Xh9>ix<w4u}Ssvi(PbsEMXq32s<fZmGsQ?h?fu%Q2z zWPcUxce$njEJP5z^APa?3E({O29O=T0kWO}H^6YUf9u(WD(LzKnBSGVxB>iY0Pz0R z_1Hfy))PK0;dsB&VQaxKd$~>b5S_dIt<L=fL%a_aS^$b!(;azjmXW4PLN^@?t;te? zA9O9Fuc!Ly)#UiALoH>Wuk^l^(!{QC=h*)U_hJtS<DPHBG+r4Rg^)Y5hF!T{GTi{B ztsx3hH-M2phQxN+q9p2dbQYZTub^Ar;hgTDen7ms0a~YG&*1Fa021>TSNVUZyogtG zqQvC~7H-SB>J+)H?I&w4=_K}Ew7h-?^3Mfy<b4RKI&Kf_rm83uKUzY^*BT?*v>&?0 z3_px|P-Ul+Ak)E${f>o>1BYy#{|n|b7f!PItvYv7*?HXvG^{$p=86s;1!nYynIAT5 z4)~f#$nuNtrCnrGV{!@JCcn7md&YSj*@}z)&cx<)M8CT{-;+*GJ(^ox7GKajK~|Z_ z;*I9E4DxPLNwctBo=c=XRS}hYl--1!ed9<~I{#^!BmpOt6<OCK41+{+p;Ya(yXb;R zrJvXZ<>^8fsroQiTD}ejm_4WcvOvmFSh_O#o66SUx_mQC&dmDZry{{8lLrM%634dS zbAbTdE6RViuTiuM3h4#GrlD|+SG%zlzn(FBSpj-U&7&2xUZ71&ovr;?RgS2RAd@M1 zNkzb_XOM1zV@>SKxT+XJfDoV~!?fv)Q9051O1t}x3_z?sq&|Adkk@AGU}+@i88d~J z={d(EWlAL<&>)G37lG471w<aMtBdxIyel5ul`t$H<3}$}PzV{2r=MnTjz`g$#bm;b zirQtglXZ-LF99DiI=o8HzP#8kycOv>I-BdC8`4^?ir<cRg_rggX=KAO*`CR6Qo_`n zBxia61X%?lQ2%D_ku+w%7gRCdu<pGpn`0qrT!7Fga*V?KoXTl6Df+0lu2V@#F#>yg zL>hQ%lE@Z4*cH%+M_ljb*xx-1mAM+paA#kA=ezV^A2`;nnu_b6<d&jO4qdcEQ_R=H zu6Np+{seZ8YhG-6&fc^-Yh(`}@2nWpZ%4kB<s0>wS+S4p<li;=ve%|X+Qco2jF{jJ zK%&Qe%p4c{7g1VQXP$e@KTfrMm(a&LxQ=-7VW2OV;xcPM=w$YA(bv}Q1_-i0hkw|d zUbK>=Y+vbjz4fQrJez75eO|IJvk}J(d5uMfxMNXSEL|||OM}RCU*`)KFGoe`{6^}s z_y_f61?QT(HKTdnFUt$Q9CI){-Qy;c5pl0oNp{X-&l<SA0TB5``ea<|mYu}tmS`f2 zY)A#%w!P|9AD1t!fvfBoGK`Tzl%<seGC)0G&LNuf3Xe#GCYb-?4L~S;?=4l;=7F)1 zfdYh%t%T(cMgm)Uz2ty1U2&k!nke&t-jUOENyM6VYf@hZEH+D}R3<y8rI6D++!AH2 zZ)c2Eg$Q%tLeS?3L16ukE>`XM{avVYM}Nk9bpu~bxwj4hQ;W`g0*ASq&MV#Au4MP* zq~6cOf!JBUyVot18Yr>CXOoz$4)Hs&oixwCPS3Ovl<iGyk%StC=f2#N{v7nQ7Yp&p zA!tDh>N2d7Z%tuXRX9lZ@yOs~eY{l!br!kk9X*S{HmX&jeeHxb?L%HCpVI56YDRKQ z@G(FHXbK7p&lIA&U&d52%b7pX3ZyDPrE(I+k8?sN!YXlG(Sr!vWjQ*;b)CLkNcn<G zE7H+L(cRE~KiM4lQ=c(<H}|yvq)B^2VVTE9&Oz93v)zPj;NO|<%!lcFNgR{0inGMp zkX+6X(tpn3k3OGKA6Kj%`8;ZFiq)^a0bW3i$`dBlDGa?bbSGx(=GY4d3F}3^k#N@a zTOd0q<3Mb%DV2kIb|6-SpNuRVnU1g$5YXbkR~j7_x>vBt_uXG>WXdVxDm<0uWB=OW zY(D6E_q)<cb{(5u_JhLhBN|;MhqvMMujmt;Y+9rE)RwUEtoNy#E5yc&D|;LS>3XO; zorL3q-xYGTk<lM@xkzPC3Uo8z?$}y>UfXmLi5Bkeig^;Yvt5B;z~-1Cqa`$dMA*Fe zR0a{tZ76%nBUzyGdVH*S?fYqZ^;?fxbo6@JV4Ju{foQIDj`vOb!Kt}9Q)l3lYw2Kx zi{63eua}DX*5wAq^9CY0gGt@%aXi6T$q5pe7zBzk4sx_U9RX<gb2TIU{k>7*^sS?% z3l|1_ann=85o_Zm%~hCLRc2>VL*tc1IJ_ZBCjQ$d#_h`UlX1G%!Zn#5Y0p-IL`sw` z(!R+Fp68Lob@f-<eu2fH%aXORc<SeHBrWq-G$%~zj>wkKD^U#cS5=#bGn!9jgrywz z;q@JHSFii=e0jf=n)2i1hYPWc=e^n+Ai!1MHPKMdy7v5Q#MTloQX&_aRf&?-o(d-> zBMk9FY0q`zHSwStUy8uGl0Z#u+iyc*xaQ3V|Bb1i>m7UzhcAR#Ki9zk+@A~K&vFFH z>1X?F>$MF$$f*KRuUDQ;BbkM&W2AWnq3u9o8!9J`w<8UBMRgp6tOO6+iR0BW5CmnO zP$+<2@2eK($n|<rGxz103CA}!2)uvB_I$Zc&*F8dq9QZ$%2NePA;3IA833#RP@py> zdmV%MCDR9l2%spRf8&B;Gfiho|8NHX4kEW_p=G!C%NH-)mzu!0b2Y#1mk5KuEaMZh z8GmPB|6v_3Z#K2G8l`KJM>?Bw$vi~<_N51-pH@a)P~_oZVQ}{vQVesRl_S6F(|1nv z6~*y9&31^a%)PY7w4+7U%NR^RW&lRe)<WCri%^AZH!XA;9?%m^syMC>D@X5eKtf1l zwefNi;m@OI$V0#D%Ts5}zWHwCR7}ZSE5S$IAG}sdNgJ{44>v#zYy9Xk)7#+-XzUfG zUh1tJe^C41>kh;O2xj!Xb4AxciOLm*-C63_ot+88^-mod`X010f$xodYg#L9w1eKa zETPRT4uEsqBPcL2$HJ}jJkHQ~S~hN$<OYaMBu|PG??NO6-j%!4no^=ilkORlG*s3l ze>Lu7=u~GTQ<qf0d*f#ZMD6;V^s4f7N$uQPPKGCGxyQA?t&qjh(zoW{&$&Vp^{2u2 z)7}Ga4L-F$7WbJcm@zv<{MaBf_x7vln=`P{e|%J3!GHQD0oqLav6Hptb<u}4NsPru z7GF+~aVFxc^Tf3uE)PT`Zb7SS<2G~WP9foX4s{qAbkRWr>jfg{m?22}7O2k{k^}1} zUA*<|PQr^_Ma=JHlaQzz^kF@r{zBBxm@4<#EjkZW(NLSQ(Q1~~BO{~6VBnqmv0lEq zH+m}kp|f%xUMHe(1LyD?eF62zWGqcCRd;LyFpfU<T?o6U>3HhKUV|W4ksR}v2QR-o z$Xu{Bc8p3G(2Et*n}6t{5mIAuLCvaM-cVm(TV7pT-+;!DOr}Ml+JHfp-`B0oJIYAx ze*5Ny=ik0&$F{(2pZ{Yc^KTiS>!LE2PsU8g6M(^$YEp5T4+&O=p;3D#uM5+;o`Wha zh5(A*bPZDcew?t%>r16m)wpzS+7#3)N7NOqHb#0HF1R%<GMcK!^^P)rL?Fs|N_Vv> znVbqQ;Zq^G*rz8QbTjz7TD#4h=Il?ybd%S$Sc&4AjJc+TCFTeo20pdpLeU63xgVzf zcvvGsuoXTQ<Zew3o({NlM7N7g0PYX_YhR`tAo&JZANk9##4DR?B0%A~#vg<J2GDN} z0SNp{OxnXGCg4^$Qu?;eL8i({5lj`tcs_o~^vBwSHvsnS3H*`pzxn<z25muB2%d<b zyMIDkpyq!+w0-}Ym=^z?CZ)ke-eHFL`IXHeprdsskan40eD(U^a>KuV{!xh}33;G8 zh`dXxrnS+Zg>b)YqWCMat2!A04zI?Z$v*Cvu#4#9!ZDEPpN;+hL1*PYM$5RkFxde^ zk3{Y2(c!jl4F2zLfLW<z`%e@{%g_$#^tIt*Tv7-=xPc)iVW3n@cBxqvwTu@b8-X_> z7~=18t+je;LgjPkG(TfPw&X)|7@DgA`W@{XAP_5XzJQu_jBolvOud_OR$vRWNg>6> zS&G)haSLtXD?(x1r#Q#dbf+7@!_P%x#y&^;118p|0w8MX!_@q&L=yVh<hJ%RgvJ6a zbX4+!GtRTHb?K|_b1fU7d;lX3?F$Qb$<^o&)pTFK8p{CIf0fpp{&%taudK@doxB|l zg<p%cjMgZsV0dsaaf<5ryzQ5FCP~Nj%(;%7pAVQF3P17S!^N>l{kVSV{cT#oX<4XO zh7oaQnRq1~rTWdJR7&Sm_jmq%{MOTuvDX**GhJlITIxc(i8^RUJd7NgdI#wp(VX&) z!lmVhkU9F4wRDZvt7W*2VygTAG))!Q4XMwJ*ivA}ADH%zH=OQpMeK0Ja(=LkO%*|| zGDdL}c>^GCiDP|CG=Ul302Z$a8RkOw{QA6_35dO9FCFBl>&b_5)gGy-&)6u75Wo<- z_1$Ac`ALd$_oBZ%t*=Q8XU_$D3DSJHHbbeatvgd$HD+bJRJ{QxT;4yLJoA!^G}s{> z4h%7<ArF<>n<-oBNi<$JOfW3%(KH(Wyp_#GPi2`m(>e!!k9$VBsa1gWzpR@n{)H!> z|7EKD*O;_*7?wFHVeW0U?Iw6O4$~sBv!BF0b57;Eb&t9oBY^GS3?lXB`q0l)ijjR; zb#+k|t4TpDvFa)a&GDCd%QpaNtq*&4+oCW1)%2BxXn!7fP97B#?h5SXCo;i&&X4)3 z8CqLZ=uvp0ym<N}Vj2$apAxs{pWY|7dG`h<Tdk_V=lMZM&9eR`fe9=Wip#Yz5$)uU zvZ%6P^DZ6F?+!%N!~Ij7bv(GqxpXe@M+bO7LU_CSf>KFfoW1%&m)$mTuM5c(-!H&X zn^jB3mDkloy4{c8;8I!w`^0;Qz@u?sN-&8k)h_)>0VZk^=~_~Y1B$<j*BPsa`rI7T z9ixiw3?iIQSQslUTpNdwd8}p`Q;fZ&lE?WUzh~W*L~UR~wt;~SVmtkChfh3S3r_7O z$hCa|leu1!;ls4LSvY#b75a?czEYFX1IZAnqQ1T!W?Ej{GBWsV+E6uSYZa;m=8<?! z#C1&P`mE;$z_fa}Hrh#n8-ABsL%mX<5`P0rj{J;YdYRYXdd@?l1CMHEOTXH=o+>E_ zH%<21q{Pd@av^(mkm#Lu7)d&bsB}(&3CX4C#1mC6JH9srKP2X8pX8sf)roRLP1@4v z>Mz)nYp_VaPuX&6PAjHMQepROrhY3nA%N*Tfb!8Of6UQ0;ozds?LN5yTzdvu1Br2B zE!B%=_LJnF$O{dsH26152p_z@25(=JQ?^jUdQ?XjjZ^Ki{U#x?d!>7--chMF7Eouk zgP8(rsG?+wLLXY=#{lb{Z1FsD^P!Bsb8YqDn6*M11Dc6nW(qBKLhaF$CMNsp$Z7BC z`{Fwjho~C6@$*wPg^&I<UIVyY^{4IfFKSf&{dv7$iP~S`U@e6sr*w<tR6xyl)6xg) z95j|L$)n`tkJm!4@8Pt|r$jM+(n>s2w#V1+b`fv`Pz{doQ9F4`D9bTq!BC78HDf!$ zczcP4lb~7R7Tj+;VP{XH4jix6Ru*WQ$7OL02$}Qa=U_eEVEngz2akC?ARS5_5S<Fk zwN?Mk7jk>SSIaOwSyq$wblhq%j$zF)mx6Dx1kaZHo>hbM%)VXv0hNw{j7xWF_z-gg zGLf3Lq?!y7pc8LhCvi~dV<eE0ZyVW`I^LGtzH-%18M>O`zgHj{{MFCgYgc3ZxS!oh zDP3ipJ+ra$y;~nqFiXefQQBB)gTlAkY#zAVHa<?yOaHJ}lz&+T(Qm)4?(6mxf$t!r zW}^95+`TTryeE_aVm7_7lp_7;#@3Ej;x4AekSKJw4;MV~@%xHz>zz);SgL$vN;c=H zhxho!p02VNFBn!eY<UABiC;kz^_I|~$ePD8i~C{cQOaI}#gCKwz%4Tg^C1#)Fkjz7 zn~ACUP9r7$w=i^XUvZ*oi)THJ1*^S1bM}+LGW+HF(@`+n-1+VEtMynPX}gJ0<96DS z;`!v7>h1b--UF39@p_sR8KRN$xAkzPbQ1Mt#(1PLXDQru(2DPtM*J2$kuC#x*5)0w zunS+E#D=9t&5=$x*B68Nvu}W%KJ;a<RNy2^O|uBpONmo8>>(^Gk2CB<P~^@SUUGS( zcoE|P*m83wCh7aQn2BP=gv*cTt_JcPy9{I=xssp##jj6Z6);w0+weC`u~qb=d1TAU z85S}3KJmieR<pI=OaI)ABgWf#1EAZb9<?JZ8z1=Xo>5Sh>U*n=)ilw_gJ&<>E;_zl zVlT53cAvA@_|Y`$EiNcf`c!_JpK;yFI~EJ4niS1k&JreiRU<p{2W%wG_6w5KWSloC z(ffe$2OBXTp@WM>l1?aJ;MUxH;CpgDo(hx}N+U>h+xYfhE&lj=130!M=?DCc)=;W` zhpPXtJm7zp-T&)aYtHI1mKdv<wev^y;g6?Oja?l($rO~%<8(D1;xrcVm15QDN#ht7 zwO(RPNA|$kEwN>9QL;CPOsAzOf8gEgi4>Er3duRh3q5Ydcvfz$&rsP0A!TfmpPh|$ z6K@?Q&SpNc8CVxL!q;f?!x_0`u%dUvq@PT|UNf3E6!@`c(j=0ma(_8bLVsf;vr)a` z%WQiDQ!Z_01UxeG56&`Cs8wKTNloxlYuM4fq@~x_)^wZp`J5V5{iBQ*b00n1Xu@Rf zC2E@~<+-TCAW)p1?%EWAk@j<|m&GlLS1iuqckJGn?BChx7_)5p>C4bbgq^j4r&9!` z=7QZ^(n)lECQ@IR?fZ}3mxJEf{Nk9lOnYpC;p2178g%q{(pKj&)H3;j$+yHd>fG1~ zflJ3dG{$5HM`2Tby1G9&_Eh!$CtY^H>Ypybe+4(Hu?AG7>pZQbG!}qSEJzm)k|k=6 zO~&ULq;-G~#{(B!f#7bZkEmVyR1z<$y}iX~@I9BJ^`dLvSl;hHi~oqvn+U@}k?)2% zE}~u1Wr{iv*z-qD3JOOb9ZnJo(ytr^CZMj5rH<gOnJfF`j~v8?9?zq12%Ia&;-=W* z^_lhet`{}z^-LaYAU_|Lae3D!bDQ!3U)%rLQ-^YSAJfQ&S08ds{^ohJUe0O>2kz_} z{|IznwFa8b^@%2QbNQ#9maZ{;>H|9tL^#=L?alF4<1OrSUy^lrH~m0C8supR>b{y~ zSVT7fmfrIG(KJt$HjCiQ7GG=ZHNjh%kQ(i#n#sejGuHS93Vg$#o4F`u(S~SO%)M9C zW23#?oP6k(15s5(H`XnE#ZR4K+UudsyqU14eYSkm!gHVTxO{4UyuDfztvI8f=rYeg zvn;KPPh6CUn8Sv$U%F{ibUJJ?ochK<DdueF0u8a!b0J^dZ+(@<j!R|e8-6U65L<7h zpk1^dMkxf0jrBiXU12+(D>oEfvW(mE4G|dk7kb>=Q^PJTaOfP&yMIwJFy9&W!H#<e zg@r+>M?FhTQF`{xFD`lc*W9w|81L}pq_vB9AN+I9(LY@l{~!68^G|%$`{hRVPN-!U zX{&0&b5mtSS@Qe%ZLgPoSvy-;zRU?gyWcNhTG{UuSr{oAnG4MX1QQ{_v&kGfUAxgX zl~|k5n^@y~1Ztb0%1!=IQsa=UhB2MK5!>u->!-N=$5t1e7b_0m$`o?D!-*hNHu(_< zh54TV7eJ$^lzUbCzWI;$@b#>ao+&c|_^+HtCQ<IB6vd6WLC^1T^)Hcsx69EuV#t>E zsXwDboR6RTh^tanH(G4FQ*0{FZo90T8IFR$L<aZ~uZNLJRO^Rb)TU9)Qq-Y38!l9y zM>#-73bl7%oyx;m$b3)$@iNM-%Qo6fef0n$l0TrHv4@+NK`pf-W0V>AF8!03`q2fY z#zAcv^IO$}uK91GqZ0;ByqM{aOXY@TyjpH!RomT{L^5@|$V>HR-P3~vQ)`d?J*hqn zwcxHA*BPp7yTOy!N_8!f<&mt2Q6{c8FY{GeTJ@{sE&dqD`g0-sKN$}szvhNwr-U?f z7HoS(2HIL6KKevsc&tUUBab*U`NrqYusu5;>w9NI^2tjn?lDdUst@)jj<}MAa@65> z6PF47+f|m~UUh0eh^%w5O@v4re&@6Kou&i7pr+4rQC_ddQzSDZSe^f&<@Pr$JH&xz z?fGvel%mYVgNIW#6Y3kWcQz92j-G<G?U21}o&E8eIl+;aoX$DbgSL(hmNgB=<&VRd zmZ1w5gnIti&6ejo?iAw9;Z{q7Tt9Rs!lGfDY<~G2s4;JjY2WCJ5I2fMFlFY)WkxJq z--VTK%`Hv0w-A1JT$<H5bNC>t*DW_FnieQ4e<J=7!<NB~uXHC|93OYLEZkTE%I0~2 zAu23Zg}{9$MB&xNW#^NN$&%e$Y}T+J)<RN!U=qSNi_sMoZqq6{z%|mVyzYELe}#f- zvsDz`)b!q3Pxo&?rMUh-hT1^I|2X=llVswtZ9I80s3;G!&;B-d16avFX&SG&pDH51 zMs21gssBJEKFaTIOf4?&_^+Yl!<U!dOR?w>VWafULDokx9cr%)x{5Xqdncfe{I=v1 zu=~^Zt-z(iMh4xar~YKsVDv8Rg3+?$AEm4DNzGq3lo8&4>!EJTs)L6oksSps+*e^u zepZ&hfWQgy*sIplS(6Lr3(g&J_e;f=o;A-jZHd*|JwYq5dqk_`-;nv=g%=IoyEw_y zkv1BAr}QElIBU>cj@2(7kn<j;-3JhR{MC$PD7SD`w$FEdom+|q#F;!gx`ZX7MHv_8 zQKd7*=Bhc+eYGr@Lb;0XM+Z_7emmV%_G}TD&U$>>id#AkGDB{f%}(%UwtodKBXX%1 z+wuC<@n)Fbp+erZoQ`n=SeIlc3py$?l&oe)H+*<6WIp8AQ-{(hY^$p<^GBCZ+#iA~ z#-7mNuE}f6gTdUaoDK-vkqw@H<xCOJ3nwW$2x7YOv#k_w&Q?jgzLu%0o{ay9#681e zo6VLL{Ue0kUx*UphAGZ$hF;~2PdrDUeidM5FDc@6^yoZW<k`IaQQekT&@b5DSkcoL znx|BqdbqB9a4!do$*APHd1H(<u!Yy!pI+id13;DXqD2|<wrkO#GA>~*D*dusk1EqM zP}5Gd>uZIRpJ)ciI;#HkKfJiuJ}Kf0_!~p`Z=wBv1fTWhXvaw(esyKnTH?uxu1>kf z<zRR>`IA^xMA+gD@Dki*j5Wgloqgha*#hAYmgyk#cUWiV!>uP*Zm2oUH-HMv*;F$W zDL88EBITmvC<DHXv(x~89Sz671+5KJx=U=E#SW@E-M=PqU>DQ{+s_>_+~w=3=fGW* zq+;wqM-3O1UTUShKTDJ4_PiL^)Ec#y+<(rt^CIF{zx{wCw1wipEjBA3RKlJhNSEPb zF|^c~-dzW&Wcl1f|Na)C{?}LZ+tr^n0ooe)TvYY+ESzu3vf-J#{+3%h+b~%AYV|~8 zf0Tb>SfMCdgvN}d3JUI!?w#PHv?rR%L%2|S^W=4`c1S(r)f|foOrJDt1&rGaSwCqj z2XS9t?^sQJWQoxB?Q#8fgzgFsr(vd%eV?rllkNIKae8Xe(11CUt@ue|Uqc+%T5+K) zQvB}0$;#pUjLn3ajrLS@FN$YvLS>eq|LZ-(a&WX)l4nm>xJ3EMK9Wa`xXR9;9($>w zW2AL=ANslDk1@^ICZ|RA=id6i47cC10@j58$?Sn&U$(y_6j)aAc4eC=*dHm!$qxs3 zvJowxl<{7r-ih|Zex}});vLL4Dxp3?ZbF}K%nj4YXF?7T4TAK#yny1iiww%C)Ew+( zEODZ!IGnlJ=CJfRn;WiXa1&s1I7xvgfygTNZTpVPeHfz7qgjcvjuSC>a)eU#ScWb? zD-*{8PTA_rLweOuD^&PVvHC|~0^%DW^zA_?TF)u_<dIF*l0g}fk3r0+O!>iS)uKPv zn-SXw7uss}&Udp=@VoHngXB@&l_kX3?G;)is&PZ?7b+Tk1?J`JLltwvqtq%GF@~ZL ze9;Hdy@&0aZh#Tv?4d<#PybE%wM*u*8=#$AqKDd=*gWC$Rc<t{tSO}}>a<WPq0EFI z-d3YhY1VQXv5osRN2-K?_s%?>{^t&cr)W)p#;1iCG1d_u#)XTLr$<YBDTPx;divWs zIqd7nBwF#0xNu%XaiLO0R4l4cz|z~&z!e9R;XbHKN^%<IFZDazvkgkRO4@1L?<YEZ zk3(@1I}q0Fu<&1vZrSx$;em4*VUm_!u<q3wX$95U=~-H-Z%Nb;Hagdn^S_$}dHrzs z^~w`Tk%;myp37;F5@=of?!Au1ko5?P8$dI$TbsgtT1WT7B~yi)&~03Gnpo%)fQ2y* z%dubP#J>b4HH?OrjNG>5=colAN3VG-5#GW0#{TgWQbIk=jE(^m<IC3mB1)45#FlR| z!J6P<oE*6|`G774k(07V7eQmxyphD+EPGd%$s@T0AM1(=sC_%U!{+EF*gL0cR=Yx~ zN5aXSse1KgrqHP3a2(^mO}4_ErOFTHA|cruwYk$Owa8B+y~V#OFecNdDQrQyn@tu6 zE!P(W1DC5u_8G_cJozsAb_n4AC5gWyUplznD$kg8>XNhg2C(07n@Tx#%gObeZXI?d z6MeJXfq!&QnKvrN3tzvFel4qx_ex^n`kpWC{9)PPb@C-HIqbW8!R9=fABEAgRE6;q zvI$t1#c4?0k)I6eS#<k?`O&6^N9@uK5Pb}p#uibBdD><f*{G~c&CYGd_dTcIp^oRm zy$}f$EAn2tmy&Pf^Be~8hnTtZM`zCaW-bZoWrev&a-1S!CXR~1J7PVg*vD$o;0jkm zZDl5x;IVn<@?PUb@X)4O4gaLC`188rYx#wiLC*vGg|L@3tdaTlK2m&@^D~blFRENc zSb{#BXfb##3yJYY9B#AAEeuY)!JhlLjBUK$RYtPu8fQQ8Y>kfo17<nkg++txg&WG} z>)-8<!M_i{y1yKb>QtCiboh&M+-cOhAFAcV1!DoVF_MXGYe>T)m);Y~)eP6%0AXqO znxM$lPeJ@IFxlU&$`TJumj^5}(k_J+`LlW%<ef~HC-;xfPb%F2U^<aDak^G$&!?uy z$aLaplBFq~s)otO9{2c6FWQogihDKfzDO}fO}_OQz9#Bg98|sm7BAE8HbS+sEoH`w zQ%}BAdvdbHJU|v&iEaOAu;doJY(nBgKl-I8PgkjQdcMowHCwL&e-MLPT*~q(dAXG6 zLY!k7E4at+4wNyvg=VV}{NfV5bklqHjg_xDa*dV@Id^+Na{OCLS%7M6u+_8V23TOX zpSr51BI(b&q%PT<N$i}uE?OFtShNa>$#S%uW2_=g%^586bP}~jXp+r#nN#>8CMHzs zzU<ziB-l1bIZ}Fg&z{ZYblf)iu&dNOnOVU;`SE=r2an!vsnSq;?Ct(V+)#0*VZH{7 z$!7j%@k2RZuy_UuA69%0b|7;M4)9em1`WZQN@00D=Qt_$nI3eV$rP?FdG_p!t<~Ro z<7SY;6$yg(Nrk8|rrb``_K*Ci`VZ=sEI1cMcKc!9uO%{$@BW;~%P?TE1=|!RyLU1T zVfGIwP7V$(%D#(#OKxROI-H)t@5%9AhUb3i#BX^SU0e^Ji{~Z_D&_Wk=);_SsoPme zuKR*B3Z>`yTIGe}`VU^nMZypG6xT7?!V24U^|r_ATYyO5LMSqMzRzw^v@4?Qy(4<e zixvAyz{_!vhK&@A+=8gTIUyW6E31$^HvhU0zn_;RLGq)YS}Rz}I1sARrhP4X5HgX- zjBgaBWZ^i6KDk#e7G}1i<dF9mD%jZ?4E~-v7&4KxEJm{?7`^}G*@)+&1Y1O>X>e3f z2OWLd>#VFFCW9vDjm9Pn<8G7twds=u>pH2iZIQk_1|KQAg`34W<m+Tg2>q-Qp73A> zHOVvP_ejHc<1D||=<&mZS!GYp59ZE#lv>5AHP(n5yj<vgb?2I((_cAh+`s)RRC=@f zlMg?X2`Z6N9vUK>zIHq`C1_^pdBr=I*fU27mCecaQ0a7#GIfS$`wgBJu(S8YP=+{$ zKYh>{-_514<7@N2ao2Uz*52YrxTn3<4{QIg3;xe=LsLzH{m6}sV8W0hOo23_F}Bn# zh0I12nW+^K>QX9BIw}Iz3Rq~~HECUY4DL)Te1F-4djBcZ?$r5kKguEG))zG&DU~9Y z2z?!kI?D;%qYDW37`k_;XRtUwddYQNG3o_(wxC0k!*Yg-u~8I~w?0q5No7`LHV1dg z;?gY76>=sqvNxn(C#0NQKS*e-oKL#8;EeqgZL*jq1%_%&ot#fHLP@(@h$h5YA^NzT zwoZE;9TX4MWN_ZwF{1nOmacnZ(bkgE0`e!+TQ@+bl62?X{Yf8?x5dJhv&t}A&9tY* zg=;MbZ@tA7ANy+e2dXOkgt_sW;_r|&0QR=nqe$WFKL0GDQN$g8<N#b(dp<LpN?G|i zv?0U>F`<E+XBV^ctCW7NB19+Cm6!7;XYW!1MJ*PwNtcoXvwV0a_VzACin43Zyv*Ad zdaPMWH&$K_@CZ@`fh0axxsr=A3wGk-Py0Afyc1lziiu|%F_g1OZbmwcG6%d9JcfMM zJ_fhXu|ee`M3rnNT1XqeUh-0KH@r0n``q~E1_*f?ZL&(fQe}?60fDc4oyK%uIcrMj zdi?OmFHN%zjrdYsC*I_Bxa6X25>;SlJUX87x~@)F{BaF|?yfLQ%o+IH065qFJhfSi z-k3uHQ@crfA8+G)Z|so=S3}lcI|bsW@>>&sIeg(tkN7o))!@|jU?Mu>l6%Azn`cbQ zvv8_QY$pv;m%1z@^z^}r7*OmVCXO_=>q7`d5=iC~x|FpNppbI#`y5SpcMmayP5yK6 zC&1AF4C36E3t~H_){vTTc{@viw!7+mVb5`YXmCI&{T-H*Zx?@a^O(QE`EmXG2D6pb zbAywLt*HM9OsSLb-+Y66(&!gJ^5z&nbIO|Ymf;c6DL>i+)Q~KZ<O{6}m#zB)+WpN1 zr}VL(%eK=gF4-Zz0`D_QbC!%A@m72oF41?$n}t304uTAxm=Jg*s=?lxE%h9rwD9$_ z^Y1JL{P1&#pLa~O^RMTQ)z2@gG~bDZWHz5E!fF;b_m|5jmdAWV^5v(sv5midBD;`a zjdhao!l-NveLybXOS^(^WSv*>dDI~C-}%IKb*{M&zP$Hj&&^n3r<mBH3%`?gPVXY- zpjj)!kP>yt>FZ8h5*yBE@b-<tbB7LJ!lrb|j7lZ?(Qmc}&R!8k*7@I!4QCCej15#A zZQ3~QMrc`v#7Ztu6`YS3KE+^y?~VEOWGP9@vhWrg5O7bPB-C$B5td<TdN@Dpfcx1u zS|7eE>2_z*z<9G%*0V3+MO){-!4k8iI?VX?xl6n{yLq8oNU2l0Ya!j*^51yMPmo-5 z>id`qYlV|yK1^bodfK{79Fke$eJ)#-7tVTe(DC(1yC4AW_(x;Pt;(!q<q35TK|1Uq zUp^q(`ssUQO@x(Y!c9)8gXK;Nr_xQD@$8pgz3IZZ7n?G!uUxkkiZ=Lm6b`11Y3{+i zKiI5x^t)S7K61VR*bx?cu8t|{PJbpF>Rkv5#xEFrq{}-MsPHt6-P?WDA^|-=`Mf0Z zkI>&=BFB*bb};#$e8B$@SpBaBy8lVumc*H1Oz<${Bn5hlut%!y%7t9w+U~c-iaMv0 zo)WiwEj9SIH?o|cn*n5YN{+5u7QB*wb>#eWdjd-RJB;}b=3nk**A2VA6)AfCK0(nn z@p++UoHKcco9Hs}*jHrM@XV^Z<h3oC&^clr1^O_X%)o@sv;a`*!A1B9ZZxTrl4u<X z7$@UZG0AWuRbzw3+0219TMTJIk6G`ApM<ee0emzGE<l)58KKmpm~~N?{Z?qWOsBJ- z&Q`o8b*yZ$ld=VB<JklbKEw!peR4N20JWUBM9A4@pip`0ZcA*A?z2W8ZDFN%ByRDs z$xNy&vHoQrjWQ!PxtJlsTA?+0eP%^^ZJySQa}nlPdGl#u^Ch0Por3eIr3q^zdiyQl zK_<PDnKuk0ir|*8gkkeTTspD7zLWEa!zs#Xg%Z0Zy^=L)s`&a|S2`r{Qx^UD(em2S zE>~OPuIg|$f)5Gz@m7uCGsPK_X3|<Z%IaH%x1GiqO>c5+5jHU8oR9dkvwY0vsbb;& zM<c$y7vryg=7*@s03NMD)!1M|SOb*v0-`W_0>0<1!ny0#Cos_cy?ep&9Zp)(rnWKG z!N=$s9c^*SVcXKDw!?$iQSuwU(#XC1D_-EhW|3uy$aH-FMTXIq=~KO3pY-`Yn%Uv^ z^Nyj;)vm{7j}KxyVlh0V)e>%ijz*ntdn+&7hfc%IKNFynxzQEIe$GriGjkpGlNp0$ zo60OR+1`VNIA+71PVY8Sy&Qd=gAZRyKI>L-#nOhLlF!0)AlZ-!uauBo*Nqv5R}-20 z^jBTe_3N8MtA?l@Xe0J7O^EI$Nz0ynEn?n~imqf0Ow?=~ZbmP?6bMp*z4eDg-y5J& zsI3?2@D<pn`A+<fff3nu#?0P`bnWX8t=Uth`O$0S^f|BhDV_Z$Fr8CBrg4|eU76!t zR#q~DFR3fZEP=A9>`}|*`&zAwW?PM#U;CFiQH?kBA0M(Q_@CC7YV-79SZJ<sPcw>6 zazp(e)sLK*gu3a=X7fQr^yWrbCju>X=?}-v7$`9mI&gdkpLtY$x@K!W*ga_tZ(*L! zjb*W|7o=csmM5hv+NbpFxle4TqTMLsXYk_&&_*WC^wqinNW1~s0I6Ovv@&t1RcG)A zkFEI*e7FlHdVDZt%JzDYcAh@uCH^iQ23K|pdjRUAT|N>2%8gnQxEqkvPpjGS>I0T% zy{h1enAeFIWN_~lcuIdD4xNVjsaUIzO<wjZjqP(D(_#Zi-d40dwvS{}6I9cW3Mv=I z;UiHv^KewZ$S|kVx$vYTk20XIM@)DujShM_F?Z*!<Uwt)>k_-=EK&-07mpNny%^%> zOBA{|R?;QZOL$g!XzSj*|3-!~Othag@|sS6Ch2<Q`a3U|ghzY-Mb&Y)^}X4YvW4CA zS0mB$dCUHXm&%E4eu9ym6w!O}`I~jZf-l)#iJp?Ok-ZY5rgGSxzFUzEOU$>SNZRqk zKS(<LnEUPVlg1Xs*9%75XD=Sj^v6E;%ep^DKRDSl%fikTTPu}e=$l=3P((n7eFGrS zUh~_ZI(opNmZo(%0`g$<B)(xDls*kYKsk@39_g$zeNA1~q~xorz;v|@)q%6mbBzp7 z(Mqd<VroVDXjj=N3hqyu(M~y$(Y$a@&w+G78Fk0C%6ym?-plEH)q3sQJS%5sf#IOf z^mfg0bCkbp6WPAnK6?I}zfgi_kf(pAPu*;kYu~KC1ooxGvY5)JRO*RHC^LI8#RxZb z<1@EIPcky@J@y36ARf=9T<rQHmneU@O9x;}n%?IeI59zkC3JM@QwirLk9cxC5_Ky5 ze%DpV+;!*EghW>>!3<32Q{8!JT1E{jvppm}jv+oM7-pW{UT{f&aOmMTZ>+WivB#S+ zvcVM_D}NY|mPc!^t!W8rBoOM!k%n$Rt{)SpGFG%APhM6@Wj&D^e_;#n!5(*Tdui60 z-DTP4<Oz|^M~l+aEA&g=gVqi&x775hbEDj<!U3cd_o*mWNBWWYBdU5K@(pCc4sqnS z?S6s{O$R#%(;mjB1a^Gw13pbr2T&|iDYD+7;~tX_TeYccvDO8fqx2JX+~N3R>-n1} z)M&+ut>gwhP?Vnb<k}%uSN5ivnSRPBeWr`1o(q#PuW`OtDROX?yqb>O@`TSX@6E39 zuPB#FD^`57c7(@QdlFkMuL^MlpDYV)2ODR6*lwL_!Ks||8(Q_v?X$a#zNlRlw4Du1 zf4qE9M)7s5HNcOrFe)*Bi^_jm_*ezmND8%0>E4o_+LKrrS$D<|1P&+P3O#21MBglM z-whC7o#mBQ>7PnF+QRr<;h=G{+bcS%c6f$6nr+el{_JIUw3ad{{qB%dL@4LWsnYa( zUAbj2s_`>z!81%-dk@aL4Qh10!?em8648$0*fj@N@HD*T#S1X8@t_&R*=bnHnV{lX zo*=>V`@nMi*n)S&*yR2-?Iri9azXw!g@1&o7-HrC$DKotalN3+91q6$J~rPXy%zmw zm)8EveYjc%Ww-i${ngW>`O63le0)Zf!a}i+lTUdWvL&XtyE(EXybR-CwDnGT3YAGi zO=2h?P2Bx@EJm|!1V!vve=9_<o4|AwJ@RPgo{Dm7{*Xv<FT>S`OUAoFh#RBjsAcq% z{MNZ15f{tuF}BD4#JIRd!qblnr+nOH*YIgTMk`O47nUc}l3sCIfT4cyH?9LNA7hqN zO5RfZ#vCzpAT_zxg%fWhMHr>=>Ctj+s#fa#uddvmo4Kibz+WmTm;7?vOSe-6!<y-w zi)<*gMx|MDmL8_5cGGh#0G%sH<Pywlg*^+i#uC&ph$>X4Q_=Rp>EJ3vgmE;ZCgDk6 zIKr+^9oY~nF=son2~6plp%!Jih>39;qIGXt;nY7^`?7PFc;0^UGFC0*{i1A=fV?Kj z<8|vj-nn!c;tW*6gXdSVOQfA288dtG4YmNct4lq`8-TjZ^yiKfkNuA2RUdcgivQ8P zaz)DZ53HJP{vP)4lhpBl%#Ko!`9Bjb)}7;rQq56}WH#3f^NZQenWEQ?n&I9JDt3>P z{~<f$tE(gVm-I%_-b=l4UA$IoQLCYnLVu_h_>!F(QrE#r?kBK$19a&W{FXt9OAjXE z|0D&+h!#uNVy048HYPCr#&!EpW423miEEaRTE7J@JmX<`W#xDL$Y4e3&u)Nj_<s-z zNm){!#}ZY$0pj2V9~FMvC+bNHOIms&1;)Ho>M4?yC{cuG@k?~hwvfUip{bnEGOgeC zj?$R>-Sfx^L|k-#dH)8;d)fN4#mJ?-c}!P$YfCIA^gp9Hpq1by#wZYy58I^BNz)6> z)`LnPv*|_4D%iY~apCX6%0jacP~on#^z)ZeaE{vjmt*EJYV3Q|f_`TVdKMFi<ik7$ zI~iL5kL1852wQHM$FPTYy7Bxa>$u6D;7E}w>KpJ&8u2HRFg5R?D_L83TX0!ltS^z; zzc@>esj|N&-ctCv-4L-4HOB8Q0vb%kxu>+jP9g^`D*Ae--8_2ErH_;?yk6PuuBCj$ zf!LT9`f^$y20#C;r?of0l8prH@%6p5baYCZYr0zS1-n<mZ;7gxA1Bu9Y3RlnT_6T! zHS#+VKe)vY<*|ep&WN|>F~!+^G=la-u#rnI6(|D4`&<zXcftEn!m&k9biN9c?yN3s zi>YsGOkz*exbqE3r0{w=Fk7;>b264exlu<x>O2}{<BR009w{y<F45TQvM1;cK0qV_ zExG9@*A-tn59dIQqiMo@aYJ+BZId_cK=+od?*sr;&M_YQ#wCW}F2iRML_WAndN|V} z>4Gwi&cj&2E{3N++O{{1R$WG5%3e(&L~8nP;91*}G!UzcvMEMY`5y|G_o*uJhmGt6 z0(_x^URs_h9cL70U0@1WguuMj^QXfl?_BKJ>{8NhfcxP0Ubc@P#i_{<KZLg=Np&5v zs7OuP>2+EIVeg-Xa8l0DN97QGpnVP?2p=UtK4eA+>)Pu$BX(SqRDM1A=&o6vyt>C9 zp6nfx^s2Bj2;~z%>pYV>^bzKrIp_1B8v3f$`g*)JvR@OcMT3eMOLit1^wArKwJb9s zLc30>m9}YepdTt2c05FAO=z7un`xxi-zVEk8xw`Lg1^NYQv;LBEa_y_9QTnf(fic4 zSTS~jk~+sDY|6zOD;;kfDMO-DZ#4A@Km?v(eDcTm)*3?xRZCvKkYbB|8SC`JSkLx* zH*b?T?6l;ZsZLdSit4%oE}ub{Fioo?`{y}p1T_TN&vf?OZnXzej4`k8R$Uq#oXJCJ za#QoB_}LkCH4ow0K72)q_PmT|x&cyPwbUJ^jO_8Sfqmz<2kt?~vFMR+_*T*&)0Gr! zr+a36NAaW1=__2Ubh<E;%=v9n;pmdRs%!<f>9V_{oceRDRr3RClcMWnTac#AFrqU8 zTdKMsAM7j`ca2mgTOUpa%jbh8g!qLgEKd-l;3IS!XMK?-^0Z-f#8m}v!|e>mJatHM zi~`e>?1X{KF<lBCkYpChtG_qVZ4&B&&D$aTP~S4rrgkDe;X#msrI=p$O5zqoH9!i< zoaV!@jP{b~;r^mP<Y__fFuw|(ho_Z4kFJZa)Q9lthVMbBi~mo1=N=8!{`c`I*GeWy z3S*84g_O%dW(XmKQ0{a=a=$Ye<kGlvqA2(KCCs>wyCMdK7`Mo6#&vQT2N}`$4V`sP zzo*vo^!(O2&-wk<(;s`SS+n-~uC@1G`}6&LKJVA(y~F0<povb_rt@85-X5&1LJc^A zCqtRWSmA8!+I%~%=Ij}s%;FITmn7nN;MypSTMOE1iF@GOR9&@NOkr)(?MHyjX^dHF zkxQH`Jp_8!{DH7|wOEWAZ>-Rkc#^G#Gxu^}_v<U2UgF29xrgB0)$Q}XfHw<p+LR*+ zQV!*Ay4vTSuu3O2c<>t^(WKrNaj>nJq@JnZ;lP{z3>K(YQRx<Q2&8t+st9MXmBmZz zH0NR(tSC`1LlCIdpQ^K(P?v7;7+}Sk5)$k)%};XGV(sCf@)9??fVA-~WBYryHN}hi zN5+P&D+@{wtG2ov?f|OeQNC&GGC4IJXPgW4;ji)AXNX%W#HF0!^s2rJ6XV=R!s|wv zP9nJ**OLG;f#R8yjfV_ie%A+u7pAK{E4))HG)`NbEsD5(o|X=}YP6<P%G^g$a_>kw z8~~-3D>W4KY69gu*O_nmK7OCLJJ^rnCZ1oVv~F1`nGQs(R<dz9!>~TCYIa2w2|oCZ zL8E;BuR((QKX}1EjTe-P<{;@}TV;+nzFGfCn?0+kgH`BIl&;(oJ>42gibxlRoM`l; zCH2INwCGY<OYz;!q~9vLtf_rz%B@Xi!rN?D^x(pyXzq=PV>2rTZaWDco-s0cK4$!( z`_tTb$%HotLs*_lODuo>DOG{)#*zkp*mX}s_^@R}`IKxetF})JDXPPmynx@r#2V;< zkHSqHa4f{_amI7Z#x@QWMJ4^|!*@o}j<M`cDNvt+^y}g0U8kaTQPQMH+*Yi60~66k zs5=aW7_i{UNoCr?UQw3cH{sS!_l7OT2!3@%byF{9z5$3$+Q#*h`@N%=g0}(*QY?co z;U+fCAVD5<?;RJx&W0I+`#7|D2jDqgJ+<Wwd)6PD!ArauEJPt@5<zUw<HFva?`1+f z?iQSV7o%?1qX-Q;rlojcchC7(4-Vy~#}YSL8+dQ>zvlnw^9)@h-?9VH78pYBvAJqn z(x_oq&ExO+QWs^9n5wh;Va3me$uh95v*BWEEzMmO5nnMVh;O)MFn-0^|1i+_Q++~A z)mCo#S-|y4KFIyltV;MR$@Np$dVpJVY#&%DP9!MdPieM1H6%-KKkHhQt5te`+J}3_ zqTnsOkJ!QzFS5|J{DzauY2c|(QWz{a(#9`v(fYH(YB!lKk}|67hSu(~_6`Y#;@ zDg6C93dFJeXRjhFb$c}f$E{=Na7qFib$EPc{p^D^io!Bd!ph0jw^%-*_blD9Yy1)` z-FqEA3+D{vg1hWK*SH=TAk4bgY-`ibw{P=(&0Ud+t;h*qmUnx6-&4Dzavay1(oaQD z-v9wP_UQ>ky#GcF|CVjW_pp5b=1v;l*qFZa;{OTi`wj8=OIAZ<njO!~9W^j%H65Gb zMJTT8QZsF`6@RuQfv8Pn#SX(v-#2>&F7-jAJ<la_L5?fhwvyKPo58AUY?E-^RALz1 zxwvsF!695orv1W#@J}ZrV#&RrG!OAny&Dp+qzaZSN+`UyH!%6VLlyt^JiQbnr7;20 zrp?iSkA6Y6kuR&}Kh*j^&{>nK@(wf^$oHX*&oLPvlZ+Z3;7!nlK9=Jl%NI9V`kaGp zGs}il@Qmb>;+E94jo0;uaHqiwL4fFu0=BM>*77(s^vqW{oJ%sz*RgY-$yetAq`v;^ zcarXRX8RIL{}*G~;Jsq*JJ<iQ*xo4n=X*wXdL_822oKypW?=Ahr{UYU4UI<}VjdT^ z+XF`+NWWo<c$;!Mx~1U=cHQFUIZqvnm-;WlEH+K9EF$PmE{3g{Qwnz!Yn(7iZ=7{$ zVT$u+Fx+&8KygYAU?<9Yf-A`<ypr~fm1he`pX`h5?^dL6*m*Pmq(y$IOa6DiCm|O2 zf#Uwb<_4A$w(c+<K%(QwoeFKy21_zin<T-fRMY_%9mY$<AkbDJyaUVAMnV@EEaA(~ zCD%AoH2}*xa0t<-F-YugNSpn-P0s52q1oxByO0=7leAd>A@3M#quSClRs~Xd4IrBY z;_9{=Zc)|4CH8h!o1LA`rxe;*&RN<VFm)XR<4UecD8>1$xG~iiD%zOsrGCsgZN@T^ zR6EIu;tO7)M0W-l`nruuNuzo{pQysA0LB-*5<f|TMBjO!R7n;q2eAfE>m&;1@y%r3 z;s^mN`)0G&%zYE~QBOcM-cdHzRTScsd{l-TJ6YR1^Rp=r)@YH<j=&-244mMrk&p*+ z^l2ooD>h}Lp;lkGZ6TE>HN%x7AzYUEYT$=?O05Ir@e}9Y<uX<+4*AbD)q2fb{7xzG z|FsVKcWSiN-JuOfD_1Z<!eXHoN8Tl=TRH<f9USYj?nfyB=|G{#{NMU>4$VO3Zk!P9 z6}Nla(MPOe5`{FVA`07EC&{nOeu}#2-&=cdf-G<kopO9o566d?@Vt8E)S#^ycxByI zb~+Ed+IykPhb~QO!8KA9tKYlm2Kw18rEBJbpti7*&UQ4~nvg4F?xF!HL@OTCMcn>h zra^QGHU*!GR#uxUMLC+Yte6QQ$>HsMv9a%?GJEUK)%7|H`rQlHb18+YHFqAgBuz*8 zV1CtTsy~+lRo$J_I2*bf9wZ$QvMXmuukiJ!&IvhiSN1==5Q%)A6+F&ITOYe;4d#YV zp$6NNHt_;^WOM9Vq7mQiH=?n;&@LLUey47nrLp_hxw`o2Z)_V>UmAS;*h&6|-M7** z*f^g+k7t?mC`P~NW`WoqAbA%W+au)KdhLy#fv-GSTaevvG`{<AN-H8oUK#W*xh`rC zOBb*|y6}o2kn7P7Af;YzCdnwghFPURl<TC40vAJ;`4MOOY}DCa<Pw?LrWj`S5-p-R zDUBY<w8nx-8?vj5qV5<X`l`2M!!s!kJ!A7MWR!to%h291+8zS4i^2TQ2M|KS<d{av zUR}`$XNuP)PR%Udp?aFe&8#F-&^_TjXfZizjd^)dD5oI%DsNeh9FOhA=xfS%80A1A zVm0KlT#}zkRo{Ae-lA=qNjG<D+rD=x_Zih;m;l>tn`~q@UiQ#E9+;7KttbjHTKP^) zTKVJUOT<D9-z|dVNp^M1GyG*&TG~066`fMpNlzzzTsH;sPb1QDZR6OYgk~da^K}Wn zL4$_&i!WuYQn-#gHNr+xiwvz66x5k@Wy7}>%_eUGO|3s>`QJjz%g-L>-T@pG&gTnz zvBFBRcVL+N^^m@>U&Xqf-9xKk{jICN%5FH;QLUd!BccM`8jruHXzHZkgTr!n$@cMS zXBdkmtnSLT@Ht9*yXGY;g2E%%F9Q7=IO}^Md@j%4XWG}}4k7xM@U0BenD*ncZ;hRX zVx;xb{ut;G?{IzmU5OoF;Hx_PTe;+qT<?oq?=R1fBT;jPej(+Fk-|o0QA<seyRIw& zP&ooUfqsf@#oy@}oU@OscUu869PfYE34}n7i^Wl|A>e+%TjS*HjSWiSUbraVF{o9N z(nTH<NK|=)5fsU@aD&Rx6-l<Yj2_g|*1gLif)P^@JXd;|a1z9*Kt<tXjuBR&3eB8c zBqM<_*hT3`j~83v-5F)9Ri$(>0}GxOJAhlOn{lA{pgPSgQkjanvO=v|2#~R!lXnSO zf1RTIwzw%-D~qM!>?mS{`Cey<rIm!m(I!q7w5_$-Emhe0T8uJ2y^}c2U1HzXFpHKH z4|}+MeSqRnxjM&r;(kP-D&kj)k_AhR9f|fzxl^rqHKVSN<bt!P8mEgE+rn&(nRG=x zMEcb}lEO3^KB)7?X&xwz4ru^iE{1?^DhWYrzy?RQ=b$?PRjJxwVY5E|XCK|xk-26x zOQ}wv&MX3_l$l+}#0G8Q4SGs=qNbF7fG-c0a0;x?Y~_2jhqqt6!2fWK<1;DGo292V z%ys~Sl7NJk6;!m{n)F<`31_c~LyH8&Xg&@jC_DVP<`iQ!w7&Fm${~`S3oi;7Cc`zV zTmTDYd~ah;@y=G4esejv0Jxko;lUo|l9!nw_CP|&AaN2*W9<v=E85ms;T&%(eo!|m z&|K(T;-%&&8`{00se#W!Sq%Md)oJszme3Vclp&SM1e0iP;Kgk7uz?<Ip5Qbm?k7M~ zmx0d*1!btWMVU7ni4k(OCosdgBRc^2GG^X1>9gkd%j<3+Os9E_?Lj7y9GyGAd4hM0 zMc1LgO*XVh{m(J`{KoLlU(y-)xAZ3c&%DxA2lnkVm3`t?aK(vHm2C+eVtV#I6;03k z(2TjalvnBKVXwf?zOXB5z!1F#a9>7`FW>yD^zrJqw3#P`ZhI`1`tAE*pS;pr&jm79 zgbj(D?<h?QfGXM$YABeW*=YpI(!tw80?052G9t=$)eWppY7|W7+C(GRR)=0P;xh;q zNWsLn?8_Juvyl?Y<)rq4`{Ne4#)|q(u+{DCIi}N?Q1Peg6+sRgI#j{}ex>)PDBzKG z09@rFs6&5Su?Gbag=zFxJKimOvI8g)nF1PfCU;&V3RaQ@NhqA(QH-F3?nT>c^Ionc zRj$(>9UaOXF|^8NhO2xfsRBmH%9PeQjR8!)Ngjj}4=;p%$o$#RC{#K|lgs6epCMr` zBVXX?Xj9@To|1Csvo+_>b~OJK*vF)i@g$n7ugGYUvyn4DCjfFd?{&$|Vf)jr^9IFb zO43|>)If(@P}q@6KLr5g1!P}dRIjW}AGgwLL)m#8lAnmcbEEn=m8+1-6Ykkgd2p%X zz-v6?Rw&+N84cfiPx2^Sk$YbeD9o!MdHtBy^(()3o%DTqSM2>KL*S!+y5%L*<~-4> z#T1`WT7B~VYHtT77^XGX^8Olku2m%F!&qgG8213b@9Gvars!%;X0@u&g1Z;FItS*h z9(!#7b-T1Ix-h0v#0NmoEN&NnK1?tbAtkuOl=z6<2ryy*MG1UYs&0O~&3c>MvrW&< zGC=a1AU|~QN<fF(FTwSYd;(5+`1X(;0G6MwEwd5s-r{xQ#0ikVfg28H_poXdjg{`c zE0{{ZlQWw6mABKX$<23-Po12{kUId=W~vc!>=!x9!~Ab_vHt4J<M3BD20vP2Urq-9 zXJvyjODE1PW(Ukj<{}i&_Ozux4o;^lg897&m-Z3WBKhZzykMz$fm4fl*y_)6ip!l} z>kU8>WxwlQu~L7-Z}Kqdy(2atK`-K#0z+yOx)|9TjCKCSr@h5Q!iq`Dl#e38vkW-z zViq)>sOOD|sw>yqUoyj@K3-A2mG}q%bo#{;nTQh(%^=1H39swD(g>d(xtu8^xY)!G z+LZc8WgH#QKtFCZ#)`$Iwx;l7QJfcqIxCWULt=ENpR@a@%t8iq62g^!35>DK*D}`L z=6F1GK!%GnHM^A%h9bA?nvuX+?Y|Nl&aC7Vu)<K{?q@d7qE@pP<L}hRF4+?dyPjqv z7d5p@%S@}Qk}0q5n-HX?7O8r|n~y?4pcj{NAcF4qHfXxjhUh=pWMO3(;E>)I_aH)M zqU%ELh7{BQ-qhc3l_2vpo3~R~qpZ~Q3EJNbOqydkS2h=RJzl57Y^5=@0tq=y+yNL= zDfHrRm={NXnqWU2j7=zG4QSwk#gdhxkB-b8S1wBNKPghYk-Vuxo@?<U?4df8PeEkU zX9f`Co0bod?o8Vkx(20Q8?8nyt5hYSH_#1^exofOiGNIf%UOJ1r~9vt5kHz8|CiOz z->?b%_JD7}6^MvreDq=UEP`Oq&yzwGuPu)nU2Sw%l*wGD4x3N?%({iaJX!~RIs43r z=$B>3QwuZv9rDWV3~FgTcXPQ9?$?m|h^yyu7P+V!RiAEU%8eb@7_=2yUdV#9T%-kj zjGFk!)P{n}(lnN~a~@tv4RMu}HZ0FrqNdp8f^%oQb$bfzUT3Rrfg<m~%32I1g4Xcf zsGBV|#~ELm#XpLkR3u>L^=BvjB64WlfMJFazpS-X#viU-TwLAnxl!bB08ZJugWXvA zd7_r}2bAb3d0ph|4j(<pnGGtLwNZ;VkJYNXV0_I9xl!9HmU!V4Z|f0{Ew(zDl_{>p zpOCz!omlG++fpB(f7Amd_Dc(I1@47?<s11Q=Pmx7`sTj9?j@x*Ji8aR_|vFAkLw%* u<nKX@8TXm?8py}4?a$q{)TCHHDDQo?L+jGPJ#8|K|Hl%4+zYCmxBmc@nv#kD diff --git a/doc/modules/cassandra/assets/images/Figure_2_data_model.jpg b/doc/modules/cassandra/assets/images/Figure_2_data_model.jpg index 7acdeac02abcd20125bc036339f2376e57ab615c..4abeefb40f3eeb0a6abe65749305e3e562a816d3 100644 GIT binary patch literal 53211 zcmeFYcT`hrw=Nz<1wn}PrbI=h2@wHNM6wkD0U^=_gls^h2?&TpiOH5;WlL3BM0zs< z0i;F<0R@o`0#cHQNKZt=k`(UR-~FBYoqN7B#y$6ramM)lu>)%ml9e~_obOumo%4B~ zx$|jf9&*sk*wh%ZYu7HwRqzM0GYv8P>%afwWB2Y|yTIRFyZ`O+&u6>;8sy&%@V}k* zZ>Rqs|L(knhzjhw%6Eis*D=U$(OrC^yLP%DFbHJV-v2o^<o|o@+Re9T?>>Hk{enW^ z3H1jdyLa*N?cT$;ckiA(;MtGB_aS>k_lh0WytGf;%8mb6ki?nCPYVPTFV}QPS`RUf zYq<wU>=%@hmXVb^aZ*WHMO9nptnRt<dWKhwj7_eZn%UUe**n~Dbn>|6dE3j|$2a6& z=>0JCgNKn((J`@c@d>GE=^2?>&z@($EG#N6DJ?6nsI4Q`H#EL!YVIU=QM!BHz3&|! z866v+_%u01rP04Ed|mvuw9Ndz`Qs;R3;4DDmtDIceE-X3|I@JlV3#Op*X})g`1bJs zW!J9VVc@|hx@YfE&3$5*toYr6#E+eMEFf|DX+cfLenl;7hNOG&kf79YZR!c;U#9)T zvj3W45&ut?{V&7*Y1b6w0N*Zf@%Tg`2ndg-{4xRZ_wlz6{=NqO@2&ynhCBxYSTGj5 z3_G!Bm^MoWNA!#`o3(#1b|48lhTM!Ymu0E5VYe4eh60XW-*U3s-gD+b<m#8&y`Jj5 zI}q{>&aNFu!7`4c*!o}x@_~XbrI3D|hl=7RWdWZZNI9O8-A^d#dxMo(J$%ZBq8Ctl zHbG@N)AGWm`r4zEyq?9n4)<khrmU^)uh6f)TM3z;32iBlb|5h|0N)P8aC8R(n4mXz zAkPyJ)G91w2Qv5*!4sX;-+_E}Zxu{f+kx1rfG^%5G4}32-XB7(fbV=0#?1%rK%QRQ z;>q>QL%C3!9SHR>eSmjc`9B+&p$fkI??>U^9zbmKp#b?E$QR^)eJ+SGGu#h(IA<5` z`z2o80Pl|p-o72kFY!6tzsx)Z@Pqw<_;C@r1BnVtLHv3~*!?o)r!}6tuWtteL2>@X zRe?6Y(Aj}JCh@Lp&HcyI3;&~GYH!O5xwfS{kV~~Ykhju1kmInwJ@L0E{=O#uaohiW zcl=ub{M!@%&z?{{Px8Y}vSwwya(dEC4v4l6znc|dejR<H16OR&hYzZzVl(E)deNJ_ zD>&I5$ft4|oU83>I}lyoDV{YUzhJ=;dP&-p6giQ*D8RMSxSiF#-GWouHY!}6!nPGH zJzP%<VzsF`ja1e9pIPb6h7y;ylZg(QVG0%>e2!mOD@ks6m$1?zQt|e2;Z0pK^w&u| z6tQZ|$l^WI-d^{kr&J3nzx?CB<1p9{H10r@j?&<4MT9WorxsA>SYONx<XIYap;*RP zIoz@r^LCxz4=fDabS<UFOL@7^f3d=MAd1(JMTj4tFDEDyZhhaKp#0*WhZ-$qI+nE# zSh4xgaH`i6o;u(_%St5T6LE4J7(<QBg6-E=@uvrhO`|KjGo5C5&%+Thr1~{NsQ-EQ z!GJ%fmrgJ1Rc#^<NI8X$*6xuVtF}ytsu*}`o2O^BS?*NiW1Z0GIGTH;^-Hq%_3HTC z^1+g2%hTmn#lsTiz15}<FXx`nU66W2Fv4BbE9D$Qu`hb-IZ*sI4r04^AdKr4P-`xp z6I6=!W5o>Wz#q#U*WH2a*4KG8;O|c*9L89+p<wc`12=rb#QP<alNxaW`4(puG8KK* z*I(fbcOZTUK*i)Od{V2Ev?tWU<O)#CZ0sQsyBe6-0mPnm2%z2?&h(;vKC{$(yC$>( z9d_x>=&9=JAI=w*97i+)H3Bw{OiB)Z7o*vE$7+Whk~9+%w`?vqwjp<%D^*gV(TZb| zeoxEI3+-QRavxo=tWtb1R4L<sN<>2@Iyz5Ua9wBNHZApu)ZC5qqPbHZMrPvWD%ypX z#j4e8J>mj<@S)n~1=h74NJIffja!O6kM3bPF-<xMQ{t=}+!Bs7RfoB{1KEf3f%7fp zi;uV3%m+45N(X3QcCuoiz?3R?qgC&dZ^^MuwaexYQERITJ3alm9|~hz?LazQ3=Z-> z?m&olP_<rvwD0DPBI>t#xLV}1`DcMxFt21;1AcByIq*(IMZ;g#H`TN<KQzuBf9-;> zX?orM`b|Uqr6ZcYQY4@Jp^ZDKi;gtLC^a&NAZvK~D@y~OXZywOu%)Y|ze>1V`&f&r zWrmSWMyIKf{>fuiOk1ky=I`mA_N<gT8y4n?rN!}za9@J~)NHsDCigRwv?46|tBEa) z>#!nuk*0Sb6mXhekvtK=gAR>)wF9{=PvTpe6J2lDM5_)^sTMZfnX{^?k4{csX`3h+ zyVq}HJc^Med|ElU+#nYeu|B|+nVEhXggW7eXSi-Ww)!@!2UIdPS=YGD2HLz{lCZu> z84%bFs~;f4BC^jGFsv+7E=;ZDA7}jRZlNa%BV8HxIU$mrzE1d0N1hC<F65teRR6Ht zptV0XLtvuoa@M%Fp}5qSbc1Ruo4bVb9EopzQzP|esY&p2m2;QrkGm~z#F+@|+6Y@~ zd&xVQ5$BhU6B4(I?XO#!rQ6_7-TLJ(9XOMqMkmDM8nPnvmDsB2a8@gTWc1Oq);t;P zZatF-eWQ8OeqJ9=B!G8<W6cbU)eqB2^_lvbIPX_EV{pHY*x2s)^F$sszgz7=>$j*H zeUI-0-FU{`jn07>>`?$mMJB))mPkOgr*j8#kYmS`P!I$Bi#Vp-tXBBE`Nm-&leXO5 zuZ<L#MwilNDuEOmX}SFT>b=?hsyUw;l{dxDG9^5oINn)&|0J{^mKAIXpB@dmUi{#A z)0ah=;zu)$7N?fnDjmo@(#EPKa<&V07H0bGA61%c1GX<~CZ?s;THPG5Kz=h$0Mb~4 z+?w0m)K<;;etX0ulJ$XGIzrn!-dKbz=Jo7A_?3Wq8gfqwzI5tlo_Y6mCm<FXDD3zV zo_*m$^wY`3x^#^Zh2>w3Jb^-{JSzyuVITCF+E^vxiJhF&bwC{xdF?rPYa_*d+EoDx zA_?Xk>&Uz|m03`TPO&CNn(1!8m12-jDFs{TbnRRBOH7O>yZg{;p2vu!S!cSKOxe~2 zSY_20SPmze4c4O(ezF!Jz1qJ*gOePJRb{U2JL8v`@8M`&GW#_3UXac`-;F6;?P$G9 znXbTSS@C*s-_M2w4w|{ek;2W#MjKqj4kK#kA`xD=X+AoBZ7*BbcUE%vCn{!ckAcV} z=Mst8))-Hens}veL#;Nv%5=U|09-Il@INMOz15}uk~CG%zTmGhZJTc>$5mwS!F620 z3W7K9C5|MMuL~xOxxj=|OOvrs^!N&Ot{V}Fki%&5M`Betn#|BLyp+PpLI3&-@8%=C zYDcZ5#eY?5JP+{?+gm)-%3?md<0>x;^@*vL8=VoL*`^KKo}ACP(denHVjFdmj`KRN zY`c&j^DI|Gkv}xPWa-=(OJhw}?v+?s!SS4h^n(3nhsGDJ(vnvP+R6xIoB}v2^bW*} z%io0%KM6RnHF?u8VowJ=!iiNt9Y}%^!9*V(A;r@G#HqsOMtnRmV<OYCfR>}8f61)~ z{#>IxRJis2pwo-C#_aXstGCmBZq5&M!(--%(w(H;7(W0(%Yn;rw15n1A5;>p+r<){ zaqg(_-j0xp|58VaP)pWhowvUc;2+?g@O7Lv>l_|en5|@lzG-nF6k0`9t@z<@d9&}A zpRrBhg$oETr-Qj=qm3tSbX%9IC^wXf$5uyHl)3vf>*SO*xpGExJ?urMZHrYXb|q_i z0ai9iH#4|r64W*WS#$r0#|O9(JSEO<z?5lD-RL2~021qIEf5bUyXejblKW8cPWgpZ z97O$)9AbZp@Dw^>&Yr~7sH;FHULE-nS9M1Nyl>mKqr36msV*n%2xf`Ja%l@04}vc_ zHd$h7Q=`NzdV{U9PFV-1nQ2;DoGiayiJA&m4pFrVNozJcZK0%ccG$|IvSGFHA*?9f zVFAAyy93#a0u0%<7&H^PT1P`gkchFXh3u>7u+CxhmL+hBZH3Wm0B|&f99rZ;bVQ&y zpl2=|%sXr?fO$9NokzWYNxK1eCt~=!9}WX(vV%El0G)k<H-@iSjBeFs@>u{W?4yQ^ zkGQzz9Y~z^oQN}8$rYLKS3VJya#=x$G}3Anio*;#a<Xs%BSc0jbAO(}LluWYryJ*n z%PP!ImpWC;UtpV|v7WBr21KwFPrkCIY<4Em%*sbKy<+G_xm{#L$Hxr4M%12_?`(-3 z$jWl=A-U#sPg$kj5re@pTBgdN>4B&H@?LkI>b;BI6B%>)QpX-x5NZ!co7;qy<PGB_ z5CL<Ogbo9F;BgUpnx*NY*ohPSnIf??dx*$rj0=Ur(0-JO?Q6V!EyEbA@GzX0gsW`3 zv^vhSq?tG*P?#>Q!WjdvWEtFiC2t=$c?U9|g*ec9z6#rmYbfo;32@*{2TGeW-K!fF zf#dtaRO$e&&y?HfN;!y**fYWLuU?t*Z!a!$Mu*vq=m;BMfa?kC4Om6LdgYPV{_&j8 z?Kh_5zM&rGdu=qxx3&EmKC2E<Mx%Vru4bqP7wVv<ZBrAH3r|R`hmQ6IT`%1$7*L|_ zz8F<*bYft?33P3}J|HA4N>Z8fe)D7wc8Od3{?y$<Q|)*OzvSVo%W+Ldgg<ocde67) zs~oVVA*9~c-@00O>UIC2V&kApfgV1c@4Uid-Ul#qvcga5%}&@x%+aB%6mUtPMq|nK z?bS5Wet1HwQrXQLEBCMWm!ZR~M|6^Rv(ztl&IBBNLK|P64OdYw73@0|3LM^n>}lj? z8|=rv!!d4d04hi@kDBX1F$h>OK$qnM><^)m_U}MCO=xpcxZC|S7cFXIe0?KthS}X# zz8sd1ZL0JYr-i@2g7f}fAlex(_p{<z+~Oxpj_4(;7}ZP-tMZRCSI0W2K{DCIWvjah zNlt|_#zB?m&s$pB(#i=weahCErce4AKOYglZ$+jr5exTU@4kH=>tcCtx!kn3$k&5g zRo<I^Y-xD7GJOrUd^-WavqrhEOSn&4&(Cws5R(q9_uR5V=GuB&p+zaS2ge^^pn;C; zc)HXeRhMT&8FEHPB5}f>KC7>s$mo5yx;bSgp^_4c0k4leRzNcQ9LItir@a=&azakI zVPzaSjzDP%n61^9#uMixG4A+oXf?SCDDgMHKGbSb?sTbGp8LBH+AUMO>0~yIFHDC* z>FlKW(3sC-#QxfOjO@z&HjSEzDpOXzXQOKlV#m*9FQ)r^pJ}?%HIy$e6hX1Q`MPJq z)@P2IvE=USbE2O;z;engJt5+4Tx_KDt`I&xb@{hFsSrr2>03y!snQDy_IKVeieXEL zMey+qT(Nzy0|pS@@Lc`Yn$QchIg&Y%VgSQ_;PDM{tO6O0WVz;3+d7L~cyS=DYuhC& z77kxiP`OiWx9;^k05s*~@^t?sAk4;LM7b?I2zEG&bbv^Rhe0{vjPNVWfyeDy7*Swi zev1L^hDGsI0}uFB`<0-3$h&##GS&;H0+U<EC$coP9FcpHCCAz*tvAtC(@=Abeof$Z zO`tlN7IY~74f`t9b9MQRDP{Ar<S%dA^1bIqDo^YWpIw?TSAWI4ame5pT=-C^lSFzp zGK1z`Vf{71hca5ktv=UJ8I5g|r+nrfOSn;b>G&BFZU4O5D=WtiiHXSW=ihVi5yceM zt;7)}^8_>IH$FA;_FrV+Et<I*xH~(LDSl%hoCZyHRRVgLeDnMrLMh^yGl0p984i+Z zS&K|)eMGbHGaCuedDe;u@+-eNHHi$8!n*}$=~d6>6J&79ipJbGxZ5rstjjgr=9I2V zVKtUlD}cs}Eik_l{2usD-H-e#fBpx1O-}h%vw~8hv%`qIMa=nEcpGHRZ)w-w5vEVM zipiKgo1}RhFp!E-XgFq#Nr)Xk&6`o#bpCv^-a{_)I=(i?-iH!$>ZA%yg=)9ovf*Ug zLv4vii;ja;W$X&wt%ZIw*8zG$rDTN<sp=bnOaPJCj<EJ!KqLhv<N*}0Ph+QXbzU7Q zPx2LqIVuc4Ru1r!p$Fg3Il#=XC)6N?C?Pa|U5fPZH>rmmn(vE0_Xn^xotoO4Q+q$E zEGGUCVl8kRu#(ufh~3590)sQ&yffn*QT}9%^t@jVj?W3Wzu{0%VzyUk=OJvGQ4;7Z zGh(DsWBT*J6S~-2cMH(8^kia%$7$kS2F1QwPD05wUp^bVsBx^=OuNANWk#Or6LiA< z&X<V^m6q0KXRhJh?8fgYiae}Ht4jCR@U^l#zI=)D=A%@xUC5>P=WTKw+N82R-E;Wj zu*wFvzi7aMeVsSVlLLep9xQ(@fn&sAQ@#T)VZyxe?39jj{flfZzaap1WxW1<Xf>>R zs=NL$Wg~h++VBl=WJ5ST*k$(fB?<cyS$i7E#qXg_vadvt_WC!P_+5YPsia)zg;Z<T zA5LRxQ!AH!NAd(m3haW%A}lIPJ=Ladr`n$^Z&^4NbtWiR9QpVBhFUsvkrT($+JV&A zbfLDrTVK6uLMq_irO**<NqwUoNJ(Pv$3{j42zsN^v<o>-C|b(bl=gNR+<R>RMC;vk zqewqGN+&}86FlL&*3aKwCMZMx8xFpCu*m-&tZ-}xQme%n*@4&~5_NK%P*Nb={^hS} z-}0iC@=9=0erHK0R1iUzpg!`&ExpI%y-44+ySBmV+M)~e0}J`TdxL@J3E_7jM+m@4 zD7Uzfb!7)K#5+TnNucV7G8(B_ls$}AoVdYZv^eb;T9MLhlF}-bug6G<<=o6a$efG) z(DNIU5Iun!S>N}VHZ2}<_s&l><jL>T1Gtm7K2LtU9!$uzZ+~{+J}S}gS7Aht8#6d! zO{F_A%6Z7lv`r(_O;eQI>XCl4y)LT9Eu-07I+75bgik-=Xl8yt&D;j~z%56N<2ojA zKYLheJCKq4g{Uyz1gQbhg^TD11Aj0#M;n<uyPKP2Ad4Nu@78{Vq<O6o=%{@FUsrZ4 zxev<6InK!FCp_lBsyX&VJh@$<ee9LS@5H7t9f9L#Q~jnwKln>rzgax+L$Ttoy|^ys z4OT5Lkc9;lt6(XLMzBLqDjB73vmXN=^|h<=55XjP6E8@dQ&y#gZ)$m;*j|%{Px&+V zSAu06m~NB)z)S1zE#|0#6|Ff{sf!S!C%5GD608dr$#?;D%iC?$Z}%%bUkbxIn^`GA z9>v@z{r2+6V*<PZu{i3dNJ;u|pQiTFOEv`AiW{!-<c~R_N9FpCcYHnP)N4~RcJ*=< zlejWs<!?I3(xL0=_+EBFSA`z1_q37Mv((ID^(7LORBb+d`$Al@O*)x(zBjRVIV7a| zB2meryIAK0ryTo{cLwmH;em5$toyOl(zOm+;Pm*!C2pd@ac}G~fI{#?2<V%ZF3lcQ zqtCbp?g3C;q&@n(UU)mrCI*(y-I&wO;Fe#k|5z8~Z}4(pS3|=VX;PYA+7b9Eizjp* zyz9dmi)$%N5AYIv+aJdIV-&lOV+*Lz2}$jWg9e8IWH;i#k2NI<&U@qs;t)`=K4N)h z3QS83U)-oJbUEIrg?44So8B5({0=f?NC93is&<3S^?Hj{WWZKu7~&nS#{kE|<#rh0 zNrUd^z<_Y-ObB&6@)ei3k_~?x`bd#H_!(_(+NPKJGGJ}>cw@qsW%1`vKhy?r#=%M< zf+NSR)qli{A$KWAawAFe`G4V7fsZp4<c8J^6wrpO*v27!9cJYGMv5HGgGM?C72{lC zd?gh5&5Dp<lc^UQ^6vx+47;cF&WUC-SfSsWzC24%p{Y^JZ@<yYEL7<j`cRV*B7LqR zaLTG&F#Yup`uX#h&8LQ{t&Cd;g2`qNqTWPg+%LcWfmFNFv-M{b)#em={&i4_j&t|j zTxHwxbC*W2+ZoE(uiCy?EUrVkT@$!KgRa$so&|szMgnVq))N_uOTa_*J?6s+BE+;C zXx_(8#FSjPz`gSejj%gJQhr}=!*i>bc^YCjlAh{mVJwQV)8nvuC`TO5`V-h-n}OSq zBN6_12O`!Q%2;7N8PrAqO1f`nq*-pjqd@AfS*Qfsp<_-c6dpUP2m}%^zcD}NygD=4 zA67vL@eev28*i#tUFowWUCspw#13WL@3;h9?FsyvCKJy}8s*8l@P_dXN|d;EC5*|S z!GTY+jvRF$@P+q~@1JNGwKTGL^iK6+d(Fs^iTb=@?aSJ1YiG)dxj!ei@PR3x8d(<< z5KMkbAm9m`Y~e}a5|57ib8MIgIZ)s>CXjXxW8$fiLsR}aCxJh}ZM1hWw0x64%O1!_ z)^})8g1#D4EM;=DB%M|IzArePPBwt`eSd#BB%McyD~KpIBhpv13tc{%fBI;5^xOzb z^`3aN(~spX@<sB<K*)hvnJ|^s#g)O-1lOr#iE?A%kE-=DmbT-^6#6@h=*r!dNd^4S zUKpShw&uZzWq~Yegdp|<jF{5B=1-jC!ytg8+9l(I6oW?EBkW>#yGjjXD^g#}=Xe4f zV^p?Wy|=p6Gxb~_aYD?{f6t}~6I>Gl>oGTtcb*eEN}8|0F|0__ku2?DoHt5xX-f~e zN?_}*Ac&2f9wj^wW95fadZ!&NUM{z*#)B1nVs=LIXVR3I`JS+b#0S9>mxb6XiHEhe zZ<4r~h}-H8RE0b?Vi{kT_4bkZyg^FH;q|8sOAtcjC$9`dJ#_ivlssM|Ih)6rrIY&$ zZ+w152oJwkYjo>Xqi*m=flLz*I)brE0#0sl6<1hCcOYHY0Vv2<ONPr7CwF=Min-gC zKEHq{H^yv{Ro`>nD^0)miCbCP9(AxYfey%-X%QSr_&$16#}}DHnU;i$$|Er+iw-%A zA2l+vD%uE>Q@w7tY=hjYPnR^clz54+<#=*SNw;?(f5<^&f9^n@;9*!{jQk+SH!BEz z!4W8A!aHX9hFk4UwB<_I^-yOFyS6D+P9w#63q@7xvzHZ{Ob}`+h=DMUycXLW`*9pr zlic_g+%=xe37l#_50)EGfwd|iaDHK~D^Oq>bA94R;^R`k)4tj8KfaD-*^X&14r})t zFWin8qkC)rS`e;P0kIv!IVr?#{A3S(JdN50Vn(ZAz(CRsqUR^FkpkW%DFaDxYt%@i zI=pFUtbKzMUTTLA1|A)pSzS|i5Al5}l`wT~S~-EA4+J<RxN%rfG}#!$bMY))ZU$Bc zL}<u_Fr`>d2>A9xz+|2v<86)lG~*LC>^*yuF&CvyxYv_&t*x~&y+cp-NK2;lhhV)l zA+xpvE2n!bI&bK^%H23;gqo^=i^fn?-jhP6*f$Z1=AT)?q%MD_dbuVAn8rdX1uUL= zdY!^`;|iiaFs_$mQC2;Xn(1HW+Dk1?tT$lK;ySbu3+G^rn<?8|?z0`pPtmc;ByL2j zE)6A#ox?GLVNo2R`_0ItRwRL=I*2(@rda&uHPf<%F+zXzBp&xQSHMvxvBk$cKiWKP zbmE5<6(5Ngz(~?l4sk3Qft{ono*0nEyhv;SvY0|-1Aaz#Bt0e0|K(-YwibGkl5L+Q z30}-j!?UJhqjR5cKFT{(<t*{k8jYm#cLwf4*Y&htWaxrOX)N{tbAsYLqiYQm(2@HA zC9>;brdL<8iE+aEVG#1_X;&Qj@h(u`E!;<#yX|aV)*p0SDEv&<yW@9f&&tP+2Iuax zNqDtj91Feec-YZq*g6vddrS#`h?y|8DUHrmKR%~VV_6)oue;5B(0(TRc!kI2p%$10 zayAM3vH)#s_WASrVLAKzUQA&y_h@3CYym5m-pzUV6U2PutOytkLV#GRbV9p22LaTu zVVGM0A6v<1_823R5JDReo%KsB^+U-n)K6fKmlS$ohx?E3TX%M@jK3Gxsa{R=+KvW? z(8q`ab_{;r{tbrOTNoc!cNs@C>68`k&Q}L;Ob0uLeKF(vqU#{XhYo@q8C0ae@|N{y zAA8nzeqq^_d^a22OfP{w&k@!bb8Zrt7s8SiwCaImW+X^b<Qb@9eycS<w}uI1O3xF> zDRMxA3~|d*8_4Q%Rb<TcPbSP73GaSx)=Ack)<LYn*?K)}V~pe|R&WU?i`C=xH%v5Q z59JTglvZ2lV}J$Ikj%=L8&wbdeTZ{`mgBFwbgx?OtGq3-rP18IcJduV_>YK-cl5g5 z%4sz^Dm|jl+{>wDwqME}GK*BqzUQE`Gdv~0YEjP2V$iZ)W~{8DA(&P{W!kwJi)X_4 z<bE<WZIaI}jaHO;`bsRRCL0Hr>eV__u8-y_o@?MG6TXJ=1er_}mGl;ryr?yDM3rc< zlsVaNFesR6c-e3(9J=zgnHC%;hgf}tLK79S7ad~RI{wo+;@B{8q{??26Fg)2J}{u} z`KuENds2<I@!;MVq2}x4T3WJ<V{AczQ@lcM9?lEL#aCdbC!`;G?m#9r6yLBF3PGWc z;4kc$8$7GldrJeD$x=N3ip-*Oqp?cZVLanH7&jYxX0*up{7i?p_{mqWDK!gVr3e5? z9^{)G+aca5w2~R)V_K;bfV8g3hDlB$)vJe5h8mx`&bVb>r>N=Gbd7d1+K)+O=zY35 z0kt->=#^ON6!)!m?4g$SYdHQ)aA03O!uY{x;_kN#RdEkqy&|5{a{gn0+N4cVV~!c~ z+id0#UVvi+OfbS&;MYd%KyV!wf5ImfX|M?1*(F>Z;w>x%tg+e53siXgJ%Sus+LY0g z)Ow!q%|M(nHt|GXEi9yTc`ImTd*)gAjdjnfCz!V0Y%g&WSvG?65Kv?v#dX|A%vW1s zM^VxShp+?V2zkyf_9=i)^@=k%4GfUyMD*3@7lp{KklLZp`$HHcUHJ0H?(e*dmg_e% zr_N9Lpsuyu7|Z@RZ^9J-bF&;UhsIA1_c)Cpyg(fff*;H8)2`S8&b;d^OrTv3txKhp zw1oT18~TM-IRl2qbrxpmg0QREzkk&ZF0(V0G@&i7HnFuhxG{__0Tas2#|mJ)CiLZ7 z(dLd|PG0$zbTAZk;8f$mSqrJhW$3!gEjgyq>uYNZ==pmlseA5QzbiUru&+Vzf#fxv ztCl5Bn-0$T@J!s)X4hEt>Zr@a>vE^;qGC1M<nz6lx~gAlqDh5Gj(PeP=Z&AJX1Z8r zDc&69_2<djgdR$>X7%cfu4dKQ`IK#{B3gME+<YoZOkbXfV{L$o!RD9);gy6q43rL) z#T*>NsMBPsU#;EYTzc6Ui9PH5X=UnqTZC@Q;D*~DH$S;l{L*c3xbEuQnz>0)#7u(m z#~&Odf=31!RuC4AojnSKGD}%80D_8&Q#dfV1MzwGb0ZR~$_$JpZp14nr%;+Fl0pNa zOS7Utc=ZhB5&h@rs!pP^o1TB$LW_lV=H`2OvR)7OHrivJ)G@aQi0sK%XLxnuV)8{n zkX2`Bf)*AJl0--$Z-pa3QoF*)x^~_!%!8^WCtC*txbtC+_2sD#zG62YWzT-S`un7O zEncFX`^OH%&BBu0@zB12EsgD!&PI{%HK$P5p9it^F^XB13~h(0{)xB^H%4+K+OU_g zDvnNIv3Rz`tJdbzO#z=}zxY58UwaCYj5trXl`^+w29NJZZmGh9?o6p<y--b2^UO0j z_g+vf%`{Wfn{w}@Eob@|m$-2Y?4z2>zouwK9UHbZ7ZMaLRJl+uML)SH|6WP{<T^GK z*CDbS<nI{9h{?Jx5-}_OUSL$d+%V@N9Uu2FCC=caUp6l49&|rin#NB-NdSo*SyQt$ zG+B^swlLT*W$t?#vj5FfX}QtgbU#)Y@Nu7!0^py><T-|GAfN*4b_r+Ia&xs&QLYBu zJd#&G6bw)jp{Qh(2$;3Swrh?P;ewpg4dcX=1L)q3F4I!K=DcD`G$-t|(`WNjaL+UE zD%|gqMoCY=nuL!Q7+IeZixa~L&x1%m0?IM+l**w&cLQ{?9z0S1@T;Xki@hJt%JszP ztf(cbTfWy&uP(Y#`gq87Mn?ITwE>9k$AWRILl?IML)Ace$70{&c)wRpbhid5#(8^b z+bLh~wCU>1IN#J8xN^hk(|NXOWym?}^8Lo9A(k)G@{=}S50gz!_McgGAm%Hc%h+$9 zrz6v8K-PZy_5Da<%S|9jzJz{q)VtrYsWQKhnZ?TI=I|ii2Cxa99_A6C!w58FR=7~+ z*Su&cT_k=q)QoAH$-3)00E-QUimyo3_*rI#Wv0Gvd?4^-;Si^)Xa{m}j&WyTO`Uz= z2Dba`9E8BTfH_hNEYfgOCiI3nDk(AxC)%p$&oo30YwXLb0(qx)z8rfO8`}~5)e#-} zjBFu?<QF50>1W>+sd1s08{BN17c9jI8S?}B1Xl<80+F;W9qH<$fAl5i24FXjjB7?l zyV9m+d#N1V5`c86HP$c;6phFCS6p#Y^QPZ<seA}Z9{6O#Itm!kh2?3mPTZ$CmL1?m z-+_p<ea?q52gn1HD^yrSt6PC7&LgyV&a@DnL&hF#Xd6-RKg0SyYqRIut%&s-D9v}v z9{;rz&dzt#_X`@+l4&l9ls18>_^i$zixJoAxSWlWuF25vwn@Wo=V@m(PXZ6v*QR+N zvPm2X0xN11WC7;zqPY2VTw_TxV~g4r+chW6v14p?^c%7sfKmmCdvUiIS#hlw=Tnjl zj>nK`mgU7uXB*Cj+%ybxXI0*wzH{PD?Df1y;k^B&Y+sHWw@hCgtUzIa3p3CGILd6M zv%JDdd%w*!1g@@8zv&|wBdpYh362nL&DcENzLVxBv;&b`^79aK=_~p$6_`a6u%9{L zuEsTCh0t|Z$?UV(p{L)Nyg@|WRtM^d8g=9X8po4b?A+MHG4dCf@n<`*w1Ac?E&lNS zsdleFJvPrK|DxQ-?gK@)1Zv9T>UJ5-S-@`|8apRkwWhJEZU93ob<8R-2oe5O^?O4} z#K_KOg}l8<_9Iob@qhj%h0<(&JN*PxoHp^m)jzX&Fj+r9NH)4upuE|vLAAF0kXh6M zqB2;?&bBZnsMx94XDl_j(Bl5+hq6A&2ugQ)-c19=y-qe+$I8m7K9_UL&Ub5W_m-7+ zYFD_6=PRTiH4}bQ5xdy|24&SNX*^+e>Ul{QSSPIEa<g8Hi9J)d>qT!b`9P7Ffc~M0 z1z(I7P|ZZ~gq48@o{Zier-`|@b7HL)Orh>6k>r5w4DKQs#m^fiiNZN1r=0G-Z+bnV z5pYVUGM(?veI6{I8uKZ4g4H*VjF{s$km4YAAkh|#KB_Jm5sMP!<lUa*N#Y;t@23U| za1h`1jnqHzjzsbf&Ow*#x5LtKI&1V<_+-P3&gbfLA&O}Q3_2A>hDNp@7{MOm90P*p z5z%<)z5678#kY77{iD<kK0wRN-=4K-GbV4rAjRBUah8{A8V%G~*Enz;o_BFH^?kF+ zZ9V##>}O41LvG;RnP?FO^01F-nR~G6&l=529ofs)#r8$>O>TOj=Pwr7T{SZ^yX*L} zgKNH}Fy^*-iY%`^b#pZ6hwo5^a-*#BSdwa^+!L*&+v!cmv($L{2AX7s7psq(-U^$C zCC@Q}z_bAKfDuN^=;@3aY2OQ`VYOSUO{E;C*UUZnN5FlLDloVXULU7g8=0gSox>Np z)Jl6o<(!-;{=&qVwf46kQ9r_YA`ZiIf8}~6bKZ@*slEdx92ai>y_V$5U(cj2BfSXf zMU7da`cLdllkTh+e~vu5H>aGN#>iTAU>`>F)6_U91{`E_b<w)CmRXl&B^oLkOf<AX zoleTW!ZOWAhS!l&ORlmBybDV+&r6?siCz16EvYN@kp~UeiHL^DW7Ob}v05BW-~ltN zLwxR&Zyovq4Hv=DrEduKmoJ6LCu><{k!RIL0@jX`8gjEDJocwbUnukp@T+(wG(G&H zRHkD<w@*v|%ApGTD0&wCE;<XDUzfgYTW$`uPOQ{0I@hdOzOSrYpMn17ruj^@+N}OS zi*Ouyc*|!v^+tkqRCcM^>0*J+@bke&7j$%`B@JvU2TxTI`}wbo5Le7y5Xg}q$!B{z zJ9}r<%+=J?%+LK^PbAhqmQ=6yh&VTVZpcpbpz0qGNrHu)t=$#7D@KBGFH|eaIxEU# z?7(iBrG(Ij87G%*bEp0L8#XG(<{UmV?-E^M!m%%=(h4jOHcx)17_an!V7aO=h|vF2 z;`T34{PN0w#Ote8aDoKD^Ou$xf>O*==6SPMsIi<hWGtSgh&JDWT$6hgrbEV@-hrh0 zFgAHzf%|bwk;qqHNs0*D{|=?!Ut|8KYQHq*4&*9Rs*-K8!gq;zk0*VaBM;o0P~VqW zwEUHMiy2GN6gxe3%ng!Um>%&=eeuuFs`Mz3zBU9kkdmOxigS!}Vh8drV_V9|I;fZN z-fdAk_ornZDIWec8yAZ-oZoD|o9P)B(0XL<WHN~A?7?LUW0lH=TwQh`0<j>sp#~}t z0V4qP$x~3Q|1aaNuzm4Aglsm7cOZRGwND_Y@S*}2isIQnOqN`jW5i*Mb|5cXaNs9V zQKVmDDZmwjV<0==LIUAhcqfVzZ_-e{1Np7qU>n7GeK8raEm*o`w*y)5`Y%r*J^Y_D z;QM<5e|z9>5B%+czdi8(FAoIggWX|IVcj^EDmoElk)<<0mbCLIQ=0czBN<N=D+W@~ zW?wm8RAihe8bJyEjInEPiGT~Q)FORnp$wv0h@elH#fcUR$ue~03(r$UXV!ajA3Kg! z+f3T-Ks?LZCdRV@w*eos41^z&TLPVRQ4It2XkN=o|039r0PJ7k4;UOeo+?{}+=ae) z>sduX#sqol?gPOiol?JB^7EZL<PeU^fR#i`SpBy5ABV%c{EJ|s_n6m#6XYFR#t9fG zW2f=8N@QH3lR7V_*bgDp8qPN5$T4tTE=soJcUD6W!z@NQ;xw3?^YVS^t%un1k^%n% z4durf%csr7nl0otVGoN`OlB)>s0riNW9m{HjwfEu6p!%@pUD|MIMbGyyv%Ys`STmz zW2ISDK0GnJ53RAL6tg_OEb~hJmH&e2+PeCpQKsyC_DMBuxV)O0ifZrMh)d<=#1WrL zokHZju52Ie_5AYcY(eFcQ#CE`<*p_%CB<Gg*L6D@mOZ`HD^?_$d+X4TBF!HTxxfyD z2CfGwS{J+fHSnA%-GSZJ29(mP)u<_CKcqB}X2#$Sq!=9Y%WRxz)&=#<BBi17ezVHW z#1WnSM`|0ykgAjKtVBj!V~^K##+cr^`lsaHFQHfVemWNM;kP|?$_hn)oG(cW3Nd3? z-z8aw%9wVtLzG8Ll@5=3?HenW8u2lI+G0VmoOy7lR8lRlcfm5qda_sCbXH^FWx^&t zW(Ok8ljOJ)nP6(rnG)A71f+P*2OdV}P*alDK_^?Nbv4XN0rB&FWvi`@^T`kWPcGXH zbF4~c`Xs_KZ6xw%imi@mP@gI(DN{nsL_>UhM<ojQb*EHARVeSj6`wQSO1<W9Wc2=f z`TI+{#b4<vWv{;9jr#JkJ}9c#}sFl&tXbA^VCJ3XDaZyte3MxP;lfMHdtgjXF z%3DFEfeGt^N`5mC9(<*5PHS_TxWs6TeKE!{!TASjG?n{hjoUjtdigE@uCZ~fvGIv% z(G{f|uj<l0B*#EWXp+MNGYexC-!H=s{(M7b$g3Uab%%^qdaVOrCfWG6iCMIXP7k?; zS8;9vR@AvTEEF)JLF4D3_uz@JgT($YX9kok?A4twKg>G{(2W_bLH#s;R0mJ6Z8XiA zdG>j!p*9j1<ms<(pQu_-?v@e%W~=TYTkU4xXONO<9aY7`)Fl<D`78IrZ3kba7w_W& zPAe5(6Vd}3Zw~pIu_+})Z&38HPr}Lw@?SkL;{V#EVosam<i^I4R+Bi|JCM-<TL0~d zZ+}&u$8zD-NbY&9&DAZQ@iY8Y+|nD+*(!9Blp`SYmUo6ZUf>@R?lw;nL?=?R#_!)g z`XJ1U+J9{GMpHw*R`ZLdRC(R2TeYti<)0nM{t<+(GT;}A@1`wR$Xq*oeymbA{Bqv8 zrjM5ITZ~m2Z_Q-JZ7L6y$5op{jTMIL9~d7@n6*x`N^lslA0A!)Mp(qZB{A;eqFRF( z6$7j&u&FcRF0R959mJGX)^r|OZqZ|-u_9>0?tK2?A4ur3!!{zW)i4xSg!xT0Psn2Q zUYmuEd~}xo%cZ)y2oF_U0FMuIAFyLz;~Zru^#n$^s@F1m$k2GXLwaD%t#9HX+5ftk z_~1liQ+?a(_LvLFYWDS{#^(Cgx~2rR9P`Ta{)@>clGFVygDTg4X)HwFO};^-s|J)J zj^!RtFBn)|4&FRoMAd#pTeUB?5m~Aj{K9(W3rD{n@v%)vZM|90uYfM?m#-`inecu| z(ree@%?@*pfJVjedho9!IBF?fqS&!^L7<WL%Rsp?+1V`ZZ7ek`nqxxK85!w_?9Afp zdI@~nEv_<l$Dp4@((1$C?#Z$j5`WyWi6eg1Dz{9!!~HcxcGt|1<jH)ezOG+(wN&{Y zY}A|@b;~%zbciyM6ll(8I$p8fHWifazTm&$W2ri1=`^Eu5}Pn&gax%n9j2uDa#TYW zM|_N@f-%bi@gg6lA?qp-8a5v;kDRhZPV<9gAgoK;KpQ3EquD*jKhJQjp;^G+GuvCY zEiH(BN9x!?nog!|517w5XLp~@R}C(;4EWqDc|u}CrdZ|jnWRI-$6pprT)3Y+8v|UE zQar3e7MA&VZ$ekC+)~VU$>8cos}Mh0YD8L+ZOL$wGpgCvUOKv-0GR4)QRhVE18Z=B z-<*u=S%?d3uv@&bk+qHXepZSKxT(Phrb+GrlP$iBd~XdDIpNHW2%PuEA9%i21X~wl z55&$7Bu;sy=zt=AEMscvDOj!_Z$CKV(K1}c2?`B65;m&iH|fz>|A)R9>G@Zu6PrdM zk|ck0pKyGP2=Qi+c%`a-`S}yps==x0h3IZg+b=VwrZ)CKy#u}AL>#$K+-TD(8SeiG zUsoGubw54F+CIuAr!C#tAopd0$^m6Ue_#@caXr2r++RCPK_YJsQ0nEJOhiSp@IW9_ zyk~>?JFxJh2_rVkH;4(ZPFBH)=qujUH*PfPFqh5`4TmYRuaYu!Ga8V?*$FyF<H**O zAq?>FJ3*k&?txp4&g+b^yrL#e3DuDnQ}eWZk?ZNVes*22jH*6#5&hjhLse(E8Q0up zn4kVC$CVShK+?XSiu|i${}A!-EB61Z=W3fhJV6}$uQGiPotqJgoN|Rl4{?4^vtTYh z)|b$ig(u6uH9gIIRW;=}N}?V`3a*^gx+9HCW1ZZrYwS<lu($IYe#sar*KkPea(hku z5t!YsC2B4d8xuc0<xp*TXqb4kzyG|kptUoVF;-(|CwFkMjDFr}DV#`EO$+l^KPr2{ zb$#5yU-5@tLd;Yk(b@U4|Kwta#I#mRV2)h-%qRJp7TLES^=!xLEY9X9dk2q{3lf*c zN`94(C8WDnESoJ&PF9wV-2{WGQS`rNMq?8y^nqWZ9FV(mFO4Jp7<%A1&XYtj`o7;z z$0eF`tT;nk6aSLc!~kw!@A(LIksCJ)F80Viyh}gU7+dyl5nGZI3xwQM;t5^=F<2Ty zox~C6?}0I{b4_{>QY5d#QWWhOE!Is`tSby}#r+c_UD88^#pSDVUtpELHDYCkb!0WJ z)PDAP=t?+X&oI~Stu6B2flNeW${7gOZ-5^|h)!4wWM2X<++=<%rX!}cdSD{0kC=wn zm>Q)oT+tqv#tmdR=SB=<ywA?=t{vuRd4&}(rFqNQwNAvBxh|)^M{gY0a_+RBao1L{ zD}UA_HaEt69-t^N`XEqcyc_JJ2%vqIt|2>Q>E9VG6lm%8*AEh@&9tzA?p1wrpf^*c zQN<)xf+3qH;{Cl>f0*?tTXP}oW^L|RvA}y!BN1|mTk;Q~R*eHkT!<yibXcpQAtFe! zCm3N}YJm#h5Oof#BUQX9Qav(>NDzJxk0TOdNqnsraSRfrSt%APiB>`~?K6s2+Epf6 zFX$uOhVFgS?(0*zDWBiSas?x5B6bWRMw(B@GwetSJPqb};Uo5Mybgu)+(zC3&RJlZ zhKhU$?Hqss4amnhp(;!mV~z%=Kp*Q7Vq+5jVr3Wuv<p{5T_nCSmfG(ph&|Fcv)PFQ zQyxDgptSLr<V%Vt?Zcj*!uWCvNVf-4V7u?(52Kyml4M9;P$>ZZ8`BvrPjmNPp{MYn zV@-i9#wNu{ghZX5Bnm4~G2BBlPBTld2mC`s?d>;Hq7V!)AWUw}_pA=Ey|KMK1t6Y% z37z;B)##PL(?rkFb>mxcbS_^vO8gGShB-}LPho7(6zZr5Ip21HGZ%mMXC_J&9LiqR zTa7QD8Bil;R%He)C3Vq44#8#YyL!@eKHhz^6`OwRm(N$p8o=zC6S2a~y3$PaZI#`5 z18?Q=+>uhN3iR`BNB2I)x%R`+X08)^GwG+rDn7e6y}noGK7J;{NrG+}#Vt8mEo)O! z{!ukq&-YB>p4Y@y0&oAwJONOFQs8?zlH4kUFLZkA5)&86)8X~Ci%z70A~Y{H9CL~R z?&PwxIRyaTnLQATJ!Zs=yzZBW>zai(B$%_SFHf$n>R$}Sy?QTP67Nwu*%7F;*uxRO zz`DyVatGHm57%+?02n}1QnC!GVQZJz$8Tb+9kEbePy7x<7|w_NXmEfDsR8~3c~q}k zAj)-e1_KrfVh3mW!GBx>U++MQiOH3?(up5=3@@rzEN>s@2;j(Y=nTvLSV#?wLW#6W zGy2G3ykl=_TbO%Uk@MgV*Df=x@w2a57fEbr8vF8kcy0EpdAPZ^tJnR6SqU{cj%GC& z%{BD}N%bI9jJt!6#t|FW{MZ)(68&r?gX9db89noJDeVfA7&&0ElB4*#W=LPH482HJ zxB%D?P(BmEfQM!MaH#CP!w$xon@Hm%(*wf;`t8Mcj773UzsghzBfK(bD(<I0g;Pwe zN>B9(Jt@%pQHt|7wIPp|T8r70o<|DHb{6X$yobI01S)T%_<qU5ck}CmM`kh=OYL5+ zRl%R__top!;TAr8W@=Uo)_gIwAxO}eIyZS1pj&f*LKXqwXWN5z=(nE)U@XEof@;dS z55D&p`;k=lv5kG&54H<*D)b;4X*i^BM@^A4rx7GTML#}{Co>C-m@<vN1K971K|a;f z7pwGsFdZ*8KW^zKfy%5{lqko7%~zVJugU%my-6L2YQ4zpVci&N1=Di!>_Nb=8tu`O zJrR%d4UBBY#k9H*{MxmF@N&Ox`%3D}C3^eF8Zp_~a%4j(%BX6}*FL&>U~;aZlp_{S zMl?bxfqO7k%z?FP_I=K&SZZvFz9iFu8cPP-vkn^U^~MQf4|A*;QcoWgdgBgq)Qf=? zRwz@dIFRNMShjrT+Q@RFRKE^~aF}?BH9^7zfL&)YAW>6uzkn@<!2+>N0<AGh#|28Z z*URdL9Rx_1M|cPq0N>HBG=lM=BH~eu=V<3nz5CDo=rah4)7iA}KyA50PFU{6V_BA| zlSPj?trGOuPR}tl>s+R+{R5=tbsO&5wL^%yTi=Wu&$Ny*&`ngj{mW=sT3K)Il3C8Z zM>mIKCrY1?tt#bYFcXTIE_zS0^YqY7t-S^_)TxDByF<6dB%9pxkGiQ{uE_OOatH(! z19cl~M+-Q+K!Cb)dTx4z4ofnCqf2d5*_|-rM52KRwoepTp=Kn-zcAKz5u$EIv<rS~ zKfn~u7KS^Z7siP~Y=TzT$aAI43*qNpaJIpvx8Mz-<i2t<cOZczG4Q!4p57Br@`O)^ zu!aGDfBHECdCqabs0uyMiQ+39-l$%$M~Pq*7_QMTQ^b~aO)GECxYnnAwg{=H$*0*~ z?c|QkBR`E$c*0yf2FHL~ahrA^{saZTS-mD=eh({#nU!oHG{D@Tez3pp-N3lPxiJ0% z`S`4paCCu%%yY|f>Wou(yiWL}hF8W{8(0LE0%Mqh5tuxy#oUU=9A&$KtrxQTT1*=X zLahA=#}qJOZuG$T^eq^5pv+RN{Zx)Uv$P8->doWl=3_3DxdJa2swF@;-r`P(+OJG& zOwO6Es%vux4Ta*t=8y|Y<<XrJ_X<_-1<g+EbryECYHVII6n0I1(5-5>;9^!(LHQUo zb*A{|oq}<4b86>pB2%)w{7o<VL~QpU!E_8XyIxbqceMO{z1g`3-&I3`K;@RYHfkEi z6EtTGur6``#LZXZz~sOeFNnDcxUkJI*4%s;Eo@b(1akoJDZ!j&Ks#Vl1R69(Uud4I zJ3l}{h%9;aWcN@8ggR(z3g2mT$MH}<byu&p#w^pv>PLi^^G^XFWw}?D`@+Bgyqmo^ zP(}`GC5IKr@LH9nkz%$>-_Bh44dn9oV2sB#ZjF;X5m1ah*vn1hhJfFr5mU@q4-}tX z4m(p;Z*WXkZsnB0vBnW~_u*o;ICTRQV}WgFI2tlq%7YGHL(vhFJ@YV7)$4-WjdKq? z4D!)mTiEItWgyn8e0BvZMH(cY!zxSzm9B%Bi`8?;2h=rrSLSU((NfqMt>WCodhr$f zK5QpUpjDVYFkMMgNTVU*w&W%_5>lJwtU1~0(8DOQxDQq~iwqO^9Z>(Bbl~>}2IkDS zAyRbpFwL!Epjv8lf5w6PpNO(weT%;5A8|B8-|2iMt@*%K)zpHjI_h|=uZ*VJ_uO6e zdcL$y`pN`4L)pEs-`=un$*McG%jE34Wp`DwXhd;5=r`}))7Nh#*i!Sj0mZ6jLxq9^ z;^|Y#GS|OkDwt_56bq)cfk`ftTcGa<c5Qaf)gsr_=>3SR!<;}SX&y?3$MN7bgGckH z4I{fisb@RB%rDlYzA4{PU3+4*s0r=%z|-5~smGaj4&m4!aCLfI726nL9Pwa;v`@SI zIL50CaA3XRCSXk5!2Cnxw*ZVGtz`zv$^c{G6cBi+LC~VR?$O1n`weBg<}Qx<MdmuJ zpg?KV1wf5u3JL**upjUuI!K8X>~27uu}Fp;z$jJ%Q(LB^tuAGt{A<K*_{R!Kbncte z$sEtH%C=Skuhj9*k5a+c?lWIW1mnQ{Yc-iM#~KHa#hk-HX*DqS8{^|sFq%qJ*nS(a z+nJHYegv@Z6>N;S2ai{fs>xsJ!pe>LPaD3O(7)&@RUkH1-PMrVcet&;>O)#-fbSBm z&Q~V)P<v%eanA2SL3fkfm#&k)qA|9`mN|*BI(ph)mPUetf~}&n?`wA0RGKSQM^k!M zZd5b{s0p7?d0$aq`Mc_%Ro<1+HN85sP;qP1Cf9EqV=y*dLw`p~VL(<aBAque*9eXH zi35eb9aimf%eoKY6hZ`6G0U9P^q2VT*2N2*B*yFZ-C(T;4K49)9B(cqyLg#2w2{Vi zrp4t~+Lanw^rBq-i=evQq`EAQq!&+6i5Y10*N6Qq#BXq48K_z?N9ZPxZ!z(!7pZf= ziAKr|geOt}+m8@y4fmnOk^q0Uw<AU}@%n^*AT@B0uPE?@PSNG5w=zD~sAtS%`#g0% zQHNg&>p{Jq>x9c=KcR#X{@--Kgb_~HeQY3wCePy%)-dvqsLr@OoG`$UHkoA=N!{vE z=SMNHg@}ZE7O*bW?UKZk_y<L^j<d)ksmrus<9ftt4oKidTmn0Zp)`cZ4bCxc1ug&& z!G6Sw0BJA^*lyA7Dh+VGUSdDC%TBx;pX*|xL#{TD=tHGb)Z3+@;ulb)C=ylIK;cck zqkHqn`l;KQ{iS>c*IaFEL<NRBfA^j*Qx#pzQw{2%RdD^^+vai`Sq-voOi9mkNO5y1 z=ASpR1I2A*9q!yw9V>X=o#AEQezxQ!q5gid<0B6t0XM2t`lBqCr>A|4T&|KFbnl@9 zxt^!A?-F_5Kq(lhjspbtJ`O0Fj>zZeb1ON{@GiIzT6|su&kn$LIOr-!GfHC!7!$_E z4XHUH%q8~O$v!Fy>hqq~kGsNsvEB;v@VrQ0mRso0>S8Okx8xk8|1@FA4uM)Z7ikc* z*w4Qf&)n615<3Dir_g)2gBV}tImTinMkP@CiT<@(bfgWVEhcszF1)1Y*^pO|Y^E-^ z|7fjgfXATy*RQ7Qw==ysBiveqr#g-&_?KWtiJMHElg0L7l^JuZTreJNGuvC3J>4i` zX_u)!-17w{h-pwx-GS_1fr%LKbL0kzDY7%Z2C9r(DA9Wx(3N)8)`@%J^4@R*@ea3J z$DQl*H@E%=dv6{NW&igNw+m^KeaT#f?6eR`=4wHR$u8R@$rchaWz3X)SJ_IK?Aek% z>%<r;Nm(<}n3<3*=cJ}Hv-Etf<2mj>p8NjZzvK6Pp8NOT({U(jIyvY3Ebq^2dk2+l z|K_b*=1QYnK3Sl8+@D_^U3gIXnh9}jKSl~IDr+!&t>D`Eh9#Mpd}gX`q}R!nH`hzP zc`mBB_+6_vXm^fq+MV~y#}c7_lwxB~tS{nK*ljAZzIZzpFxhwiZ_NBZu=D@^PvkbP z>z_@T_gGTA`^0$S#v|^<8tTZu$h_G=0n5Ov3gKnDKbx%2b2*^IzF)2~g4_t&?{S!8 z2J+`RallsBaF_)?zYuk^tFRLw+QV^rNClh*{Af1nK!8S|Yd--22nH){rtJf4*jGKI zC^pn~M6;ZfH$nBg+~E57Ri0zI%oXSHWYd)``38-`iTzOB-r46_1vp0?o&J%som{(0 z5cshtdOa)8a_HfgpMLL$8rpxBSAHl-DmreyD&1e>_V~N`vEeWJd_iT-E&iy-Rxr$L zxpKS$ln)I)=(B3WIRk&_M6pW#hV&_f6_Oy<i|xfaSj(U#gMvY4V$Bv#sasC2_q`oB z(Wv#TME?EX4Ku#Kj$LMK_q@@ivg!aK+3Db#Hel)LRM-Mzynq6Lh`^ux`e&0oPMCp| z#^%lrr|SWeha1-A7bCZ?3*9G6yVHaahtW38g-p|yrmpCBt533AJU*QX(e$7AZbqcl z!O8+FX)HxxE{HbX&L6}`>5;Z!Qy2)cC~Hpub%0!8=`qgT2M86x2ntFL%e3OZ8Z)w@ z*B)RquVykVr5DzlI8m*l7WaSGt6Lqc;EMlZ??=O`J}W)|$MbGsd6VuMp&7?<`V7Km zxO?NBEAQx87Ad-UT4&E!of?<v7>%#3I({e9rh65zfj^I{#2+tB^30rI2-%h8d4=M% zF3x|ftDDQN8cUA#!EDXF3gh})6w(YhhW^eEM+%raj3L|dIU(E|*Ewv}(SAAvKO_t0 zj`T6BC?{U<yu{F=@fAtcDnr`Nz11N$7P?Hw(l#IE-RM}tEQuh=*T5a%Wf8o=OjUs& zvxMpKU3L5!WMf@i_)IW;A<f8qwie?8bvGB|{?2uVKG#)ZgKb+L+rHPHYitU5e7s<F zA;9@<eVd8mJjh~c?Trd$=5i&c`GO8mH7Jv%x|O(s9juHVG&KU<1rwo2o6#mr5_LzM zRE&{NBtrwU9h+c3`iu#^YkTT@vpv2*;c3qIPm5%SfWbqp7kZaJfD8o4dKlIWULA#8 z!9EQxN+mKeD-y*XpltgIMKXdDa3>&FE0|v4N8Gc*6_!{{NHn_0QW+>b41FsJl}gDx z-_<R9HM97uVbG({(3($f7?0`FV=W(eejEb+C9(xEK$xy$oAH_m-iQ?B7Gs%ZTEhTY zC&ln&EJ{X~D2pwp^%ZKf%F2tpJn5@uz2Jn(xQG*Zx+q&%;#uW?XxZ_0`NLdV;AqX7 z&)oyT2byIpkGx?S<SSn@6-tlF=(9fDo^?GO>oB`$pKm{!EqM70-EMb&fwg~@ifL&6 z*FjB?))1GoS0Y>9-XIilm!11dIfFm-R>NY{{PTqp*WGh2=E^(!ai8Cb-e1>Q;RsLw zj)~?<3e!m|D#4ud`1hzL0#F3H^51<v0vFLwBG~s>yaftoA9$n%vuZy%tEZzB`}NtP z=E(S<A2i`5H8wU!`b(ypNVt6KvHLS_k@{SDP>rZY@=8c6{;=?$P1VD6UO1}3XGM(T z3&*e*K|n@y0+@E^@CW$sHr6P^+i*%RU=;{K2mY!67J@Hfc87bs=9BvjmPpB+q=~N- zk0X-4k(!zxZd3o||6zIWYvtK<tJi8f0em^lks|cC5Q2w=xR-ekyCvWfzTh&Q^zqlg z?F=y6$Y84N(bJ)7nDUBO%OUTDIGj5+?{gV_SRlYyK3>fzycnZ-TKIU%;*GQEAKazF z296e`*OCo(wTJm64!jBXE-PPrpC0Tg7FXd?HnV`%$x#ptwW6HXy)st$!mMMfov^|I z3e(F&{JMSRg^q~iN({$19x0}7(^7>c1?U&s;zt^O`H17^xW^J!k-|n>1s-V{6Tr9= zKC|#Q-;H&vQ2sgkh|D2eo6}8)=hy!>sGHXqAZ|8R=7X>ZCjzYOL`H7M&-`PH%t%Kf zP=5O&*~wjS_AHT#VLzGVE5p`{xXlntTUS^Z4ferf8D7Tu+MQ>NMob97xNX#^(t8hG z_F~wRY`OP@*;2eTG?{9I;$T1XDNfbJzJ)>`_(FpmRYLE#t7>H*18#b#WHLgzE)921 z97^al?OXJH!n|;_>gp#aX2yWdM>Jn}lC{7t=jD(BP~uoJUr@`FCBQ~?z?rDtT9tpI zP6Ikng8nQndPGZC>pFJ*d^e;u^Z6-jk$r4_dPO!qUopF&^k}8f$xrh&J`@>ipVM&` zO<w6G$+fEGljHj=vb{}1=FP`avTiL-H7f^Qz5iv--p4Xw-L2u-!7OznsqxwT+^(cB zaZlU5b0j4lgSHuEmBtTslWVa0yABQp8_eDfv#RmE8`kwT_z%~8sb}%B3BBw?ONIJP z(CR><?kl74N-k=|$5FN-U8{5d!Fuh2ZGKv#R?-2|SG!$(iI580jg_%7&Y-2DHt7ye zsqlprIX*ZQUfmBJDXb$R9oaa*6}O`xC5>z<Z~^Ia2(B_xBz!HNCQ;KADATR{+3jtP z3$7439;~fN&!+ayG$RiDM&a=4xUIYvlyBX~n}hVaG=gv4F8*6$OPl29zuI6j=b#&Q zLnkgTI3-MH5ACy`%gOWtrf{`YfD5#{h~CoBI38H%W>9!b`*ONx@HSg~5DpwdqH$mn z*^*|T+|GE3PN20qB>`L(zYR8nd>9q6c=ctJ80u=E9FvfU^2LbL-~$V9Ti<h)!&w^U z4fSwH#S*KzXDmodN7nMzP*eS>aFLFSg9FxSw~ur3%9nZ=kkiv?3+dZ8EV?YD6Ng6G zqN!Eyx_2}S4$2sq>K7Q+rx{l|l{hJewT-Lw#~=T$ln-Wz?$gm)(+l%iUzpOKEc@y+ zOX=b2dNQXh@cN+qIs!-3aW8_On?vY1!k32w7!~P!A%71o-lJ}m-x2P4_!3ndDQcuw zIz&gN)jlId|J2#*yAf9&!#GhOqf@IDuF}w!YxMxexSFva?)P5eZ<@k6fa^tp_aO+c zz$fSFteSDG@b77jb*m;EbvTpprBIGn&R6l^x=(frvgF9^t?BdbuH-_c@%?vK0_R&G ziKX*@MXR*Q{+8-zt!mE4{6H{iD`G4H0~{u**#~%?ohKRScs5}t>MALgCe9UL5_bo< z3@2ZLv#b62`p|G+t(18iRG#TG%PBZ|VYH;3m7X)HbF1d#oBE#XAIL{F1Jqa;wh-LI ztb;XCD?{Lo*fC;c4pFCPhkG)n>L@e;BO7{wr8Da+O015Ht{Y<6TAo<lYT5#YQ5`0t ztC?vkLpe;B<`;IsAr0eflLZ-5QMw{IG$h-Q<ib!+=`0-q;UGq*Tz8g9sYs7j+CCF% zR;4Iajm6n#n_Hvn^!w<ntkA@&`bzbSrstlw0Ab-LV~x%7XD_cj&ObCLCX}?e@_Xbf zUW6OM>*gc)<9{}_l2$Wkk<&;v#H&YL(V3cLV^}{2dXRU3l#%4srO)(8tH%DK7)y@n zMTrfN#P}*OnJ(2!6RqVOBo*uCIiwd~4ol`E4j9-@tgV6wWfF91jrxlrFc77|ZUe_1 zQVMM^$q%swOaTd_lU~qPb}HOKM~h=I470xBXa(aYmUy8=%z#S2p=3d2K)2fHvrgqo z<#B#dkWPhRa#FpJtI!M!h%&{P>>t4G5rD~{HKVzEX^hsD*E`ZUuKq3&$#`P`fHw0p zSX$ltTyf^3lyx*q<0Wvcn-y4`blh058>?t*dPp5uSemr83L%Krw*b*4;KICyckri- zB_UK&GjckDDbSk%AZm$DPeu-FJ28PTfR(!dW%Lqb+mUQXVnxLru2dB^{Y))9dY<E* zJ+wF$-kknpu=4fp(Y^bRU;m60jek<8_s-zq?8)?lN8Fm0n1zbM#~HWEY#aL=g9;DW zYuKjAUhkc(HVq*+TMF!Y5#1ss>exEghOaep^Y?V~ma$IXt6ODqAxlCX|0XXc&G3Ex zl6|4NI7>Zz@}%FNO~)~f2v8CM@sH3^&IK?PoVrD^kNHCTIB2d2fEI-CpD=rV04F|a zCoa6UhjnD*`=3qjEUWam*;Tm#D!(EeJXJ(b7xK*egk1X2L8`_HPOW$0oP_4;9sUy( zZ~#Yt#}6a7^7r8+a~bHq;FL9pb2dtxYXPWkLEr?Jqugy4p8hgs<WT3fj~cg8M(wTH zQhX*rKBV}7IwB6(p$Z7kCn>}!Ic5S^Jd}X}aZvvNggW47X4gyrv$b9mWLnE`w{!x_ zQzZs2UlKVna46uj=&0wY(OJbwt*vCwkjm*D>3?f~+^(}x>)vL%@6z|ws+5%YgXIER z8<V<a#|OnMw_8g{n47;|x2sfdS|O^Nn`oFG`|_?HgBd>h_H+o=dH!PExcz}5D^Gu~ zWy2$}mmkbkn~M9-RJR`dG5?P)<C_um??i>vDZ#v_OR{D!GtzFiWJcX8TWZJjqLXCP zA5|_++D-Y(-YoK-UA;R0W}}}yHDPJ_D(G_hxw5ouCU*Qo%%*^e<$e_4{eJ6bWT4ol z#y}Fhgm^YN6q3Jyj)9+*9d8d;n|txw)F#m4h_g$r?nGrpRe=3Zv&#GZN8U9gzE(4J zt}*mm$_w`lo3tPPlF7E^Wt03!)Li~f7JUs?-BI+<iVL3Iokd#H+Fiah0J?sWg-;MN zHnNaHO|STZ*inW<H<4Z^f%RQni)7x?!**&M8Ve7!vwzWjKRB@V%8|t!K~Mcqa2*sd zsn~Jcz}RZRY(@DY6U~DP^}+N@#&4d1=Na!tgf*<rP&}kTm=7Jjcll;s)DzJuSDJq} zva{>l0f%!YP_nLriOoC_IuHEhNXSWtD#qXW3yPS<tT2I1P8&k65Lwe?f(%!Yfso*y zf+{E+6g7bE6ZiAarc|1!(FvB+DJW_<UR%dVpD7pc(R1=ypmts0k7aluS^2r^V|0Uy zza?igCy>yighR8=@CEgtLB3!s=MY@|vOWA)@A7a={UTojCPh7iBTrAR;<i`4uEut_ zi~}}6d{U)l;-|aC@`cVlX5LC$0@!KgXD=*#cn4v}AI{$&e7?5XG$(?rcYQ2Ga4byy zQs|?}xAhW=vJWyVgi!W52aBt^UwpMFRd>1`?&PsmETJUr&CKohSy0`{xQhel1`v+o zi<b_4Ggq1uB#X&jH|KdA`Qoo$YWv#@>@QU~40wafc}-kbmIfJx#05ZS*hhHD;rsT| zUIJ<WO&(4(J`DS@yg(toqcvawj2TJ($pLzlV?S`BeK)@}<XQBnyg9$GMK)IP?HGQB zcXE<3yc!JPjTyKb1a6^LSR5>&#~4iWSb`t`SCH{H4SmQU&S@8%R*CKCA?*P0De{hF z`;Chgu|`%Z<l&vmVpGf1FvpuWsaMG7?n{c-KblMXvG159?@4PweCyq3#qo0G@dw-D zKS#8+CeIc>DvqAF#JVsf?mc(%E%vz}-5<NU@8M34QMb6VOmDYj_W_AAn@USjTRS`F zaXr=<F$;TMUx~ct;;?|VXwm7|3c3UTpu+VAvIE(3M%{mK=|h~JHP)OG)vjkxYr(VC zB@~k$vm2|0LTyYW-k$@CrvE3c@9zFz$^U|2y2GDMyGRH9%-|*x1v!MY!d5dQVhKLz zDS`z|Vb&rUrd{;l9RRB@FR5B23Ibu=8F-L+YgTr5VOi7KR@@<IzOu^QwL&K}Ebu{p z@3seP7uuT|lLHdL$VW`4@DaA-n%cNiv%w!h5bcW*J>la)KYCb}?B{vT&2n<f0fPmG zd;?<vtiVNdgFEF*Q_eaKu4P*#t~Ft12EoAyoK1G2zU1MIK>CqfFJ|M?=rU7hT7xg> z^ba=0>s|>S1+{jru?+4sOK-K<#f)V*XinJ9z4Sc&vp_sW6AWkdK$J(oiJMm*oKRa^ z7=1)X8^60Ehxp>Nl+a6hp23yGSb_vlS^Qht9=-u?*FVbvyaoh{X?PK=Tpw+(ktYNQ zU?S%-wvOJK*omp}ZJBJJsGP;@<?6%8&?n2?5)Z!niDXY?hK6|SZdQ<)vzynSKtaID ziu`d62>}#|>p)lqlk}ZX_c$*fzXgk6yTBq)2Kxq#;WYzO_VO(84(>Qa>p9bRJN0&V zyOAE<-Eg0dxx1@<`owlpx8yxjrK~dwY`Ns<0P54N>s$E>U62<)Ee1$a!24Xs>n8o~ z0VoZ^13vs`lj%Pc2~fIGocJ^hceeTmKY0@u%#(yVob;bfC6~Y*5&>XU4T!Hg{&jl- zd9r&K;lp!65Bkrhe>9J+lYcfDsPj<8cTk5=^kv!()Yp%wS%7B3Rev^3FaOz;FM<4* zcSk~heS&z4fN%fm$=EoCfYrc$t4?AAvTb3z8TVx5>;O+rfD0%oP%2<JQH1sAy3q~Q z|NnOW7f;9GhtJ9)=Q1qCel*c(R)n>r>#X`gOABOK_h+>cksK5LD@;@H%5?G41^1c9 zy3Nph8>^N6E-8TFZlfeO*HoVqnRxktpQ7`-rHyz0Uds?z<-bZOCvyKq+m8=0fp21f zaUpKp@L8Bz<E9-pz6UfxAwsG(Ljjf<t{<8MB?GFLrJJL`UTc~2>->vjx3lwa$JMlC zgjdYO1vk)48*-DaDy}5f#-uvcIyxv?lvdiYo4q8NlE3XsGn;PJ{0>^F^9lI;_WP3( zwcew-N^<6JCL6k+O*%D9Ogd5L3naLKTvOg1FQ|?yX~vG?#WW312k?b-Y1=@w!E290 z@D#(cqYl(k1AE~)Bk!UB(w?Si)uy3aniHQVO^)%UM{%zd`Y+0)mYy$qiCIixWKh=b z(jcqhj~u?(a4KmJ*9X;$p8E9?F9VGaq#F5Ex?$HC38`fbRw%N9d{Y<6m!8{?+M^d` z>neTz*?uXhh_sMU<xXMU58v(Hs%kcr`=w@Gcg9-O*kwJ-D@y9~)Ht?M!^G@nTC16l z+ohxt+c_GS{LPMBbd-fVZJlq9c?E`GjkFD;)26$=UY?!*aUsiFDy%J|&O1+fOVg-{ zP{%eq%_M1`zwRZwrM>;zU~8ee3;!@`wu%PX>LB=2$lZT7T?!T+;%M+c(i)8`DhWNO z2v<o-#+y2M<;Fsvu)<`NxY71tousDa0<RupO%T&*+PiMs8b~0oyDhE{Zy%Uyo_6g( zwNM*}k?aUp&j8+4>DS*Q_2MY#UCk4258V{@zb?;=&8oL8QNQ8#A^YH$(ahBA6dUJT zm!4FqXGq1xD^L5bGeRC>dvvLJA-(-g{s*&4E-KY3+{&^s36uWch3)^Buk&8c)6QmC zNjXE`I=pw66$fm16csPahWD>EF3CDp`{|_@c^mhOd>QmJ`p~lT#1(_T)?KniD1DOm z(Ba{_juS2KUx$Q-gd8}ty>kzUK^?W+FDz~@+j#Ww=F<<(ZE2_z(o>zNNJ{ld$iGr$ z>Sth}JC-%pH7O-mspdPrRund8tG8d>s=vl1B&7Yp1N?G|OL;+UU~v6BX$M}2s|KPR z<gt--M32svuSgNR42<U3@kL47;mcDiKTgVV0-Q;#9o1|(pZjl5M+MlA1zq`_?%i>d zX;61Q$M*^Ph@V&dp}%!BYJ3F^wuat2Vujy}&EN}jIEK#psodaM!c+{K2~#-QTm#mz zB6_eqc<jRbZ9UHbVrai%T%_>0E?|R@6l~-a7+wRTtRD?xFauXp6Ys9BetQsbl}G`n z4kjWI59GLDFAZ&ifwQ*Hg^<#@l_kK)SYa{|J7A(kM5gs5fYL?6^-e0<H6JN%H`B^~ zYt?Yp+)HUdT`Bp^aqm34KC{h9MW|Zxp@BJr<K``8+aG&9hb<fyWVG>4Lf`vr=55js z>XuiInOj_6BYS77drb(4@&|20DDT*|ozVLN3)Y9y1=*z24fFG5hDl>-RP=^fG07OO z4$`W-MZg<bju+xnjQ2rxeZ+J^Qv$_GeP0lWYs2hl0xUtmR=U7l&z6>);6#$_Pog+H zc&-`JiRBNf6&||MHl}^xEJrVP#C?(V86V>mczm?h+TrPtLvfWuX?eGdb!waXQ&RKm z6Qh}Fj4IQxwPEX!j9dYm5wdkH%F|BNG#hWarrO(de6gj;&%*eJc4n6R)m+WuDHCxA zaf$!j2Exh7ts(>U#f(MdB^Ltba@1tb#}#%Ukab??+3>Y=SZ3>HG|Yc4Xxg61{bxU@ zPG4{mO$3mfXB^=;RwzU2E?*KW@ORSHNx;9!QSt<$K@LFgYV*PIPD@7M>4`z>#MaH_ z1)GhwV(t^G_E9AIKyh+^7L}^71<Ii4#PKyg;KHG`0qSldBV#)}NX|`q>Z_mv&DU<3 z@w+{dMv!_PS^mk@(0o^A)_r-y2c=@u7dD%S|BrtN^q?gFYyM+Xw-#X4USU)CuZGEJ z-g&;}L?Au`Ai&?%dN%-II+}?x9wvP8OZt((f1K#xIz4^z*qb$*v7p+OFrM)QgT2B2 z=DWJ8xw3ls*0-Q%&y+9Sa~;?g^ljyIRngVIw)l@bb~ZMSUHkTfcY1=#@8we?Pb|y5 zI?&$O+1Q>m+t%FCQXr$PRT?*=lr~mfoqg%lfi9Qs$tiobeD`$r_fT4H-f(Un_50_^ zmX@w&qo+4Q$yChLOz4g2$2l9+4UcJz<u`&VN{aALpav-SVq$^rByA&j)th7BkKBPf z&e;h(FFl3(;YM#(HGO=N93@Pd{Cp6u5A~r(dG)`?CN<6I_wAitGK?JX(R<@&Nb7Yx zqMQ*ihJvmSl38Ya!C@$pIY916(MiE<HJ0U%5*jC0x3gq1TbB-DGfuIFUp};Mtr)IZ zyj`AP^gIAB|6qaQfy_sX*ZK!8aDwypZV%bG4>T8pp!Ne3XT+Fv8Ob$4+ySL&kJE%9 z`i_yn6Ifw@o}MsE-1(Ix(ljhGFgv49{{|Ao<g6iMB7J8VP)|I>sd>>)oKSk+1X2C| zcGXI~x$NJ<d@Z_@@3LMJ-8Z076qn{}CX>tDpGqFiv3^`BKhY9A=wkGNsM1<si0R1k zUdf_n`QK<>vfLM+6PPs6ZkK~ROg0ltOgLg^Uy@00xcc?Ykop7ihiv=#GU<t|sHsy1 zq`6SOupj5<KVfoB17s55Vq$<)ua2rum+IG%SR%HrtxU7_z#vm8wFx7+TqqC4W!;`h z8=6f0F(fp9%SaHL`&s$J6G_?jDU8vGrTopefkB<b<<s34AaGA|oZT?`e4tb>IAD-g z>gcQcI^Bz~+(j4t6%L5^9W_hu)YOsc<sY_`LklX|z{K<Jr@4NUOU3hjnDbiRDMOhN z<3Fckiy>%WU3Ve9A1%HbX<{iDO*R$ma!u}ae?9&9@#7r*FB!YlYXU4cOJk;H4_)uA zF3oQ{jQwQ!*!;z_;MysZi)OjGpIdd4Rg2qqU;lRymjCXxb(?wBK#RJTNNZSF-4FeG zxz`Gj&JBl<jI|i9j8^Arpb-e#(Ya%EsstPL<b|2tHnL+^6P2D-ed(L@P~=Yu%A-qu z^S5q#=8QkCusj69X-Xdi9j>;vpSCckG?h}V^rA-1u42+EbG@MBank+a*Ci?)Q?JxX zsoSM8O4}(fN+<0JS^&(bgH?gXwUrGR^Jqh?86C5#!a7QKiZ^nFbm(goS^^SDxH2q1 zgFQXWG)wL@3t-gA5PDr_UD0>(hd@jhF5*@lrTVvmO>3mg+<1A**fXup6jmUhSspPE zo3MlYof@+!ao|%wQ;i=cigemB5mN&ma0C?7ryvYXlDk~#Yw^vbJ(f>>4NZ$?g9VoK z+k@juC0h@<MWv50?Bjx>pI-#ZnD>|@f){E^00@waK)DFS%3eT7iS9ShS<!O_eIQam z_*hy1<A8tO1(T7Vr<f<r$T$2|Y+FQ4*pV^Uy<=XZ)g~`9$GTOxim(l<wYLG5Vxq{% z1fmp9^dv97T0f>+1=b<sjiDN9Vea7<qk1y9lZ>uo-$pDsey_0y4?Wr++J4M&zTxfj zhxzmB<(?O9`^E?6q}wxgFPN*&!5i35W(@^1zGHQRs`RxJm#B^fPPU=0-XBvH>@Yi? zhb;{YM8_XF>`-1|!@HX5XJ>oUCg?#jyZKsKa&TTqLSrm?Zh);a$^{3;ZC81uW)v{) zxBN=Rp;#$o5@g#1ttEEKaH9NOwkaSdzQDYSsj~oqw`52*SNXT@UBrv$I(EnUe$f5k zQkuJQ4RWRI8~P!O`dQ?GVY$u{dV<sWGEI=v03sIDAV=V1WQV57YdB?=>q`YOIEnsP zE2#JL(z}6+P497-($6l_{?Dlx4~j(Z*+#Q{U!JB{E5<B3vf1n$$c5S?(p=kg*mc^~ zweQ~a^mK2E{Gn63{3TUw^&A{7NI3kxX|wwaXN&l~g?n+&q}ARP?&~$GGC218t4+RR zJ{;utK9yN!v8Q;xtI7Xe9nq$8EPlVAtEk1}zCW8Z>n|i%*ftwiNhkJ8C|gb0)-0(L z+OyPv+eEe2TxiY+4AKKAAY@#z2WHVc9gXT|b_2xkitsR>f)*t#7JqN7<*JkDxnG6= z7Actvm;5eStdjs>J;<D$inM@yz|lKh7X1GO7V;t-KzBCMV#Sd0LJ+dg%B*K2>3tY4 zx1xu*H_UZa!=r`jY;<HmIJ0RWL{uQ+_Y&A5oKdr%>R|2M6eG_XuEzwxoJ4?HE2C(! z(LbWyu^j)`Qag%*-o1E~!tT1@N8E~64Dhms#$!HJTDlAO7`8N>Q@`h2@u}`d@_%E{ zAe_K}DfzLHFQh{shE)bu0-v~p?|a%<3;%BNw$92rG;G-pi!sn2ne1D+eaM~PuHp23 z0z<$`c`<W<zgu>C>9*U4t{NKk3FcPLD*K+&3?`I7RwVSC`Lk(J>ob5I`gC~@0A_pd zKhE|ta(p4=pH1dL01V~tuUN?id7N=b=oJkDK>UVfL<T?|L{L{q{3{wi_>DiKoUGr6 z)uP=dwCSTi3gpmT<KdQ^c#UB13crbw&lvZ2FZVqk2;B8-KtL$IxYi}iZ>;v8E>LAs zYQWZbIMrcJ&3sC=^l@N^(z7?QDz?`vtaq1Lq?sNzZ#_8kz}$cOwo>yo?t74x{g2KL zQ0ybFonT5N`f-&1Y+B!M#yz6X6-V(O%rL>;cm_SC1A0;l{XTIDym%1jI|lOHwz6N1 zqb?P=p8IDn<yyl;I&C}Fl^vhL{QS6*d%GEqWCqKUp>=?`G$E2u-igLis-t+RC1aJ& z(NbS6uGg(+E{n2RE}eN^Chf9-`I&t}Y$0;R<=Dq~Wh8IEABe5|{%%!X63H+4AN2nc z@4;&GK36ArN`KY77{D)b;cB%;c6FA;7hPSjK+iQIB8Ley4j&6D|HQt`L}$BPE-=8I z#F?sS^!8>|P7KvIHEj47hs~j7T$h##cAZrjv_t4?dMNDHD-w_yzmS*8G@zJ-)&w_X z1@O!So#b?Ulx>faFF2Gr*2?s>hnGluE{}Doy*3QX9}T&F<5wMgB^XvE@RU0S03;Mo zY{2~4bVU5W83XU5f*BEHj0kqR2L-w$u@EmmBjF5|I%2Oo?l?@Sg&Y4K=?r9YFgvDP z{XKNrAatD0&z);rhaK4!9xmw0ld)p8sroj9VVBJuk8ecuqhk#z1AT*5ii`?!l;hfL z?cyl@h|A(<1Bmi-pw}IoD~)U?kYf8=#$`GMZW|pA4}AKt*#_XjmtLf=2T#phF*t5| zI5g`)bZYtZ?Hi~4^B1DpJdYKr1<Eu_j^5n%IO}ru+;LO2@P7<&;_Yv2Qp?1f{f`8k z9garP?~`T$Velho4UA@nEdDit`wC!gyV3yKhWN9ov=ylQfRJL8Z3&u|p6-4DYjNE1 zZ%|FK3M%cqLV{o26o=W>8hx6S8f3g1<ZMQ;jv$0gmlX%}6XzhG#5yOFyARYZDzzOh zsk!#1Z;Kg)ww*BBOaLH6I;}~CLYf)`i%YZ9%g1O5+%Xoqm$U;u(?=Z`$hh-*wvMXN zhmzD)4G&bY;yrFYsl$jGAEvktidfH2n~v{qZcLAi@~7*<#f7G}VHMV8E`tc`-Qk}k zOty6`2pHI<7Psc=E=W@}+K&CUv&k;DO;vkWQ(9u3pRauIN9u!)lqh+}1~h$y-F}Sl zZ@vv=c!1-GAEY(5RMV}5*>`xQ##?bB6P*z!rm-sL;2u^a9cYZ)(UkSl*Fh}oRdT<_ zUk$l#A)%T!$VSZ<IagA--;yK$Y?9?Vz!+$P*6TzN#VcWjUmoi=X5CDfMQr&5PxdP) zK((YaqgxE@w{nOUjZ;3>@$_Z4BI{<6MB1|RV$0Ql!RE&7TKYZHFK1n{>>0m{)Hq5L zmE-Lhjc6ZZKa(3crw5K5CQW1cvi>&ZOHIG*i_6sP0|-}3WXy-@7Doc7j;{aW{~IgW z5=n$VMQjjA#UWVwjEGp0f8Epo*nvdwLNqr&qICEEuSmIWS%3Zx*zPIyK-YL~A3^dX zHqQ#xqkp($%VMRQtHi?fZSpsW^?x?4+LZ9e3q|2N;E$q^8qL5(>42KhhY)wU7>+J} zq)?JqhSPcmi;>xel4bO@J@9&8q1cB&O*|5JwsSN*wzC_0Nwf_-^6p!|pPo$j#FjnH zX6`0+Y&WA22RT1ga00id+q8@{n>(abOq5EP_ZZk!sy?tjocejR=BskqjB|f!#coUr zNa=OhS8E-#=W%M)Da!>g{r`=npn`jn6O5+-uUf(#V+9z^Zs$GZ+AxQcH*6;HQdk`_ zNugUZ0IvbxFHWipjIW(VAr=eObJi#uA)r{&z1RdlQ@lE=eY*bG`n4tFgZMF8BeDl0 zgFQnZ?nhB|W}SM26H$8{h%fQmL1u68ONH&Mj5HgSfUB9kco~*iA}A%#NVnTbE}uK7 z5d3YuGp^Xm<kai$RS&WhqmEYyC>v;hk4eocZca(fw0LXjq||&p*56`gzi$3P&S}32 znUhL+ejP;lf4$C?{OhrSPr@Pw_}k!$*|pu=v!MALMi6HP3<kU-A_2X4LkO9q8!QE% zpvazlx7O(Yc^^wZVJ%3rx$B5kZ9}BrToSwBSqMf6?9XTaY`V4vUkaYea0HSuI?%E9 zPVxtV=gy2y<L)Ox4+*`(;BoYRfl0H&aj0n|bPs%`Rr4D&MS^m9q!o~98Qenf;pGlD zuj5B0`U}i2AA(JG)Js?Wxc)xoL;KI=({buSY?~X^wtIrC52vl^OP5bb)a$BVVmWU# z@9n+x)Uj_YuQSW{LYd)De`UEg)sfipEb%f87a?0el;Xd)ovz|eaw3LVC)jKjY6Sxw zrSGDNaTTExJqfP7T7usWRxg(DlD4HYvW`0a8D{|RFZ%i(YX+)?xFVL1-Dp|nh4HW- zV`13N6XiQA+F!JFJapFi;D~9?SZRLB^=s!n2B;7fJ-hGMD&lgc`7P+cFcZXw6H{+i z2K!RuUcZ375s9uYiIj@U7g>7u4!=D6PV(5U%b5<9boCMc(vT08hUU6+2DbT&@BEQS z<;6EPnaODuZq1c#HkH<;t(BZghTqsP50Oy^%v5aEX|Y&uRGh{5fik)E(}AOTY3JSu ze?k1Ki6QY1mJear4`KIOEYQ_5TUQGhXez;cT(`sZq`;|3;~iWX*h(tyr*zQUNyARr z46?}Hi6|HI+Qe9L*s##%E^(CJx6ixbN{Gx*s9+XIfu*yLuo=5>5%y%<>J$T_B`6}* z=p-W$O<zb|U0H1gTru;ph?GqsMi5(|%NacWxtFK)Z9?wpoNu&YJ?J4P4oGirsGuSP zzXxXu0k6we49$PTJ0s0l4of2Qk3$9XWptkH+L_5F4q6n8v1xC7(Oo}Lqf@hXz&tf) z($UWKZKY;Va`D(m7E;%H=a`j$!EulfFahQH^<#ErTIq)!%bRa+)tu4CXG`hLa;$Dx zv*28%XI+y)-;-6#|N8IFum9=q|3{jMXbkf8adsodJ8&{^Gpnvo_GVf2SJY0tEOvo$ zv$gaq&6__t_Jg<?_OfG{mQ*Yf0t;0MO;?*n=F<^|&Wv>>%;66=$YUDJVOTYo`%q`K z1BATZ=a>?fp0MHq3K-RwZn)9p68sHP6@WZNq;#af&@SH<ZJPC<{r1TO@(XpDj6GK_ z-c;9izJ@<4Y+7tZX%7$R6*I533(0JBsi<UtwVf-WIIFX2#}UAt<#ij&eZ?P5c}WxM zJj@F9U`>(>+;Jz7eeS{+U{n!U11F61!nen_8W*<L{z?c8o1QtD_BGxz3O`<sb$)Z7 zcY?DYG_#T5)f27`#}b<+qjiD55z|YMHqzy*y#Ila5YfdZoNu&pvKcum`*cJnC@}7I zuGRheUEiNvl$ux3<67Zg-}r=oNqRn1%bS_|Nm`i#wGKEhMEm1Ew1^EyD)&*%qEUev zu{4CShDq7qLGup>%3ZN!dV6hGr!y_Qa?w&A8$Y(4SHOG3MKcgP2|aC?#_m@78kN>d zc)oIvEBidgiC5a`K>v-iQHj<SCMFghNO5e0a#8}LL@1N#%?8qb?KX{KOKKsN38Pvx z8&}(Qr?(P1t4r_lY~D!jh+TYBs#aw%Tyyn>n}31D0}F9~OM!%b&);+E)jD+vwS(%} zZOQ{@M0VL5IDH(Vgb2S4=*49|E)=nOwLF{eH)@+zWrO8j-A!t=KHBfOY-ii7;LX#t z!SvN7p?s}0*dU1-oR|49kt4<joUVOcih?|-XsmJCunenvfIagb<tInFLy+$jWgS@3 z2w0PWy<9-oi)qu5T*kJX1|xQiW$tQHV2H$ObIl&A$75m>A-GG5DFwhRSaofc3`L@O zwIqLHG;&wBB%wz=&=}?0EdooiLhYbqDNjKIqJd;1lhASxf627Ku=c2|Ny(crc7_*5 zgPScQ?Hk&G$n=Qd@sPH0Bgx&^$_rVa6<Rzkn#3BD2OILHhFqQp>iAigOJ-M=7dcUZ z>c@Jw!Y`9u4Gt{buaUmJ28ov{tgZA3bysy8-dxXf`jUTGLdV$25I<yUQ>pDU$UFZ5 zLup+(49-GS%N`PvFdK)G?L5Qr<O5JvN)HLE3Xs7+Z6y<tTme=9qiZb%FX`ro6zr4; z@tbu9(aV`$yJS5|g7)^$GcUY|?6UY})0k<~))`s7YVvIv_kT!C$6<sThN+t&q$vz9 z%%LJ%J@wOBeuLTtI~|+mfBRctTzl0oRJ@&2A0cU}OH`k}J{86?us9xjaPBU}^SN{0 z(YWJdSrf+24{cM*b9Ik<$57tfAZj|O+hp^aoi7lm*%DkHp+|reKubjK`%y<-m_;S? zb#NBEQX^r`5u82rGvg(KdX%vi2Skg6B&3+}PVB~LpDj?v$`hr(y4);#23J_OLpi0n zP8)GAy7QOR5B8~D)7aQvXRT#l6w`OgLbrZQ_41{vQ=^r#*yKZXJI5-tJRJ`|EwPot zWs4V=DP<804Blkp9A$5^Ul=y4zI(^IQE{-XNJHB{T+u&k&U?r@Bs+Trpo#z6?wJ22 z&GsM3RR2{$@;@IRAu&&r=~R5m{ToKHFfVVPy=PL5ou=2NcPf?*-k1(78N->=qwGu@ z9?%nJ?znyKb7Nl!81a3kIk@?Z-eCQ25A{Er9Ns=1FS5&~dc12=w#L=D8ecqd;>1bo zRPBT12#I*ght}ravAA<H?xCkGY}&yuCnuM1RIbiPMZQoO@RPdn?vlJPA7iW_bc*po zP<@^27Mtd(^D;oOz1<|B4r|9mC2%9?ftd49M7mKRBT26Jc9w!1e6k#CqRj;g3PJn& z?u3Yv(gSjIBaRmuHbAn!^6uk=nIKxBq6aO_je?`-v=rpFfBL69A08m37;hU3*GLSf zZNbUES78M;*5XVi|2l?E>19)|R#yxu_cnS4*#~}-G5aYsXR35`=jf2C;*k~1q5c|Y zQ;dwNOwv|^*9VCn?fnb|@9g>Y3puE*d7g~`_przYQ3&vPH&xq&6WQwE_6tlGY`cPe zJnXxw!xqP!FNq&nGU(LaXHM7f{#etV@2#}3^D+eI%P-sKB|E5_2`64iPW2sSH+eZw zDy<*$gGfO{Y6UxTgcL}E4*>Jc)iuUHky-SF{Vlu}Bf(1i==I0H`Ql$a{({?}4l-4W z?9D<XdNpViE#`H%(Gj9lm(QPSZg$_y{p*4A=x+Qm=yN3`#;$~q0vnOy5?(!peQy%w z6P$tpy<evb%-nfIWBKJ(R+-T*+9tffr;FTkz*bL`{<t{FSA&espqnPXF*r)kmp_Y2 zNjYAp`Axomt<RQH3VBneeD=Pet*wcZtwW5O`pf7us*3OL`2D09UJt9+EzV0OdmVZh z(1!n(e)MSX)kgOB3iZ%hM}t%&x@?w8jDNQC_ooh}Rqratt7U&1YfufbsomXDVvb>} zQnoEpBI&)oVdA_R-^horUYizEOsu1C*rl195H$Z$4Wt=whXGWuT>*}Kf^~A@0WnnY zcBpkFifOfDn9bsWgdE&y9x;3^x?8lFB^5iA+e4t{()mQN(28Oe9XeQ6g9G7DE>dSc z`pbGx#_7q3{axQ0toswS6@QHXidH|#l?3vJlvxy&;06amC;CuLI!P$cU~&ALcApHK z-tLTGmT~{|>=JgXT?cG_i<dk>&HjQCJw4?{@&4;Y`gcKvvkGU2kw2*S&GvYT`$<Rm z$sbVPasS=I{ph=GufJrzKItc`TENU^rj;fTvK0?BQs%WMzkf<CpPOaM$jh$8?+22# z+O{m!%h!Ldy{WL3&L^-_vQ?~g7vERQeyET_c{$i;v{u^g?Ype(VpJ9Dl`vqF_GGzJ z_p8e{aqoEYd$Ni3k<0>r+X_=c2fOLh@x|VJrfIC2CH$0E^OhzAP_ar3WG4g8Jt(RV z+m4ruQ{<e5<>&{qAp`~4idZ?xsH3I;j7<u4H4Q;GDNJI}z%blwtT+vb&h7$wlo=Y1 zxy6v2#7PAP8`o7W-7agX4Ho%u=>R)pI&u~f*F=JqBe;8f*ukv2betu}1b5;EC~W|C z2~r~DF+jMhL&nH+#aW0xbiCmTac|SAz7<P~tRuT*8^GV03M9@Aqs9YKVUy9+p?zh% zSv_9FYjN9^b+2@^wn+}^umrcw?u3%n%hh{VX46fvddglo?as2dOk~HrQp$3?j<}tj z9hsxpzOKK~XIF+PaLBAoOkFaqbx=l6JM}RXWbqT?l@aO_177umkH-2hHKr=r<+ zX<Oy@S!Dgb<+<`T($G3v?X;aeNW<^#GO!CJzBU{tlRl=fL5pP82}}n_0y}|!M{I1# zE$d;RQeyswFU|&U#Tf*un8y!VwvKcCim^WAd(@@tuC8WYLYASyhvTKaE;#DS5jK%# zqcJTJD_#tgG@<80_g2sjYaD7{>&Hl=X3J<TKK*NVi6S@+b?PvKc30n67>+!c!XyeT zO5POhLB?36$5if=IeV`1(yOP&7YNmc8UnHpZY@>X6&6NhI}R#j6cSq~o`>;ML$9my zvkw_y`*Sr35t?VCinE<9QquR2yre(3SL-J}e|zZTuY=Es<6jwH4Q-OYd|@}Snm;Xt zTK>9Uq-`b}8i%1fy(9?m#l8}tARWLnG3HaC!I!dViYszDg?*V7NbKu6(ZfeJuz0P> z0L%@BxoVM&ojA|nif2#qI=iLfMGP(PhRmN#>%5uTzjH*7<HR6jpuC88P_1jA1-BU( z66tV6uL$TAo(&VA;&rx3C)@~iT!0@EWe7fjM`yPwh`)f{A=w_>(dzEj8Xz~9t_?4# z#u6+tR)0B7I!><o9B{n_JOvZA_%Qr9@p&!)f~a;B1+`9S7~sF0i}+#GZewNm2n`To zs2G~?3dx5T+UqhIZ)ZIbR$#I4QkF(RMpM6aKoOMooX5}H&s6&7NmdHaC`qpTm5x<> zXnMVBUf3vZr+Ucn(tL1^ZAiRqj%W1X^(FBo(<5O%rsWRiQCFpJuPuGdx88S&2SOQg zw)e-=#jx%pC4{y};}wVm)oCaYl~#b2+)=IYVh=asgp_(qVkU%1~|^zIxHIt4KO zGl0n(;1AXH5H=)!Vx_q9B|y+k<L-@M<fIe@Jb=Ai$N7f*f!iKwjB{EO-w6o+_Jq}o zQ2eO%<o>2x7u+rVA~f}dWx{p`_U2r;f%5JK4OHMr&Iz0X>;);(kHs4Svj`A#bm~EC zFWZsSAe}3txWn|5>eFT;mtJ&t+|1#xue96;y~r9Zjoy8KMVU$ZDf@HX0>)j9v^$*Y zZ6_@MCP&Yt)!ISr!@wEI$~KdA+id4iHjWiIdSk)D`WRket3V?8`pS_C16zuvW@Yrk z^)z3*N~i5cI!$3!+7|CC)YjYWEcEPeDUD<X{C>(y;Hoibuu|RX8I}%TFp04y7hrrG zl3FEJ;Qj_^y(f5(xeTlB7-Sk>kw0=28#G1o&4>YTrIB6fVonH_$cPMR$-!%U2zs_> zFr)-ySy8iovbn1X1tk9sX5>{3FcdZud}vc*Ow#HAB*nhTYc$%*xdbBr@JC@$0DNwH z54VB3LCyF+$M!4m-AeUuBxHy&zTM^g{kn>4IvU;)IO4(myBYe@jm+&e?7Qz_<jDTs zgWM1s!`piV6XymNZsDB@>TxuUn;<~$Z9lGy%QP4^7z&}Q4<nLaGM2tiR!67r7P2@T zn{Cv(ciDR>%Qj5SA`a<&y0kzjx%2r$&9jWE54ArFx!yw9nqw$IDRteDVX@<8g*R*~ z)5sR_$HKkeR@;X!7&bG;eXUa{%u2hwvcy<lf+k3(GVeA9Ii}5%eTXDX113(d%LULt z!k|ryDKE9=GBlTpnW6!)TJX%4Z<8QsrCT2x9eT)BLy*4{+d5DiW!SS{1d12@=G-Tf zxgo`SK)OoXf*+tYn)M*3GU#Ou)381uW=XqYIjhH)%zsAKY7gyhV_2Qx-}a}LcWGBV z{d|1hvewqgs2NL3BzR6E6EQ8V(WGnW6pU=QBG>FWtXv+T20+wdsf~!}FfjYJcDgeW zJ1=674iLq({%m^Qd17}XOL*Yd##xt6N2fe~*)4@6f#h&$-B<fBUiLL!dD=^8oV!kL z<!r3~T4S9NIsHt%TtGy>#@k4j_3@W=7)|MVHuMmE>r#=*GcEPsu_1ygLb{_??rN7c zKJ<=K?2|`6WNYr;k)KuamSrFEFzZuDgtxfOInV!1_09i;qQ)tjKaK?H10c4@i&xw& zjI~}j8wIYF(Ai*0M!bg{i6uc<Ex+qshh2M__bcv4O4p5LtI==Yw3-svep*JfJrH(% z^gtZsMfnKJMUG2+1^szLKAv|J*z>q3wZhQOeCoS>N^H%BvE$0d(TvS$bvexE7t<xm zl3qGicszd4ZdYb77B}fIBoRLDix<K+@^=D)1)WTU&aQ1C^c=&h1kgn0;6-TQrDP-M zZnHb>DK_~R{o7r@Tze=@8l9r2c}%*F86kI6BjdGeeg>pz*gXofx!I0yi~8bX`B%zK zfj$JRNTbk_h>#g#ipB>D|4N(_BoIsRMNUzF^?Wvj*3xu|@Hs9Tio6KN%m(j)8y!Z& z(n7*@5YGOF2{AzcQWP7KZx}2!%9*q=xuLsExpPMUhlGiy_3m`_Zoi~lJ=xzaih?QC zhBvMUNZhS7e0c4w7BpUQ++=3P+@?gZ^0a*2E(!BXe%|70%C<R{2^3G&{$e|{NNR;? z&Ve{fZdldnw6?Ih@BoWr*`~wuq3mR^^Zeu5^Fal6kfxkGcolds9D9s@2nPe<Ul|Y? zbPv#jU-C{cC*q!9N6$kJpK=X{lR8ha1o{hQC+b!^gC(l0yvlvqhVIYObUpTWbaZtk z2LzlGm_dn?F6pG=^?5G{0W>Pcf~7+~LD+)BgYOvT!GBe_dn8=MhjJL(Y6WM8RxL-o zu7&e@r+XTvreOIqm4Ax>IB1ps<+^U~CasO1HccNilY}j02mkVtKfk1OPgOa2|M5$k zj=y9|3?{xg<eVn~{q$1Vb-w<=<9Xaj4oagk>#}QR$FJw%SD&<$89P~-?uk_#M<435 zt3F+8?A)n)y*}FJmzTVP=dL%=tvLrpLxWxJTUoVP*b;^@J;AWjz^V*1&*ASRhCaiN z&X$b<ifmPp`Hjfx^Yc(-jjb^%yVZR&%Hdnx;whwrJt8+$eJ%xCKMPZo4c+nynbUy) z{9TVRDi}SL!fu9n9DDGk7GDGZ_MR`TxuvWETftmdS>|BCYP)xuKHl0_V1&^HpLcF& z=)~R{D;W=d7Jkwq^09eZ4IW<`t~4pdl9LNW{VJ{kU37$-LOrM^2bf_u+)k&rZI1uq zv(5V)>WsVIp|ClZX9pxy68aqb7_1|e)@}_q1|ku)cDHrEj$?1{W!qYrhE<NGRgG2{ z%q36&4$$k(y<Zp8<9Yq#IlD>)Uyji}i=WrmBT<3Y*u06Sd6}_u)=m}2doil~fkF{p z15Rs}Am)#1S?fo{A;tI_STaS6A&f$BcSArOii|@EhM`t!QE2pLSjV0vpWw<Wf5aZt zcs%-aIbh#nuHA!u<w~dVZ6$RU8ud+M2TBHILt^)ibDk&12CM0-y?$Du_Rcnc?iTnh zpLp!sji7HgvI<{&*xv9{dXQ>&Gxc(omz=4Zi4ur_{7VQlnD`(2_<yC!SgVUiez!7K zAbd=t>+lZw{rog30+IRV735)#XeXfh%SDVk^0=wI3&7O2>HoyX|9AQPcO-T9_jPMn zkVU#M`t8{@ofZ5Zu0>~kcVnTe>)?q&%#HxNX?K@TFPGTA4!=!gy-Jml*{Zf}t4G(r zj462mzYDGC`E=31y!ZOx;Z<F#UyYVyUV#1RxL(zjD}I)SubIl&PRD(^KdlU(e;~dg zaK!@HTLAp&kAjtl$-~oTfas6T07{Q;1Q$2KRb~<UhL~wO?0S&u>O}{o3OIj>8t;cA zO~=A%g3BT2CJXnvEB4-w^k=VC_6F_eI8LX8-@C>cWSf8sy>Y|p@OMrE7Yv`*ghMei z_ANW|kG+6(7`1jFy0BA+*Tv*{3=Fil-ql4RU41v$9YuccDG`#7cjeuW4!}~r=vxK& zK6Tv57Xw#2Lg@L6;De$v`GO@JL+%Bb4$NR}EJT747Oz6+b%3^-^GZ5B>6obnyJqND ze+KgFu?;L7LH`(7+R#{_8xUDt6ig5siV8XT?&qy@$Y4TG4`875p-KKu3Ihr5Y@0Fi zBkW4g?Lo-m1iR{?jM%0{Co8Dp&JTo~^J3^Ha-dL0SLe~CiIEw%^FxY{$~)R0eeqKY z4$u$3aenl0wXOK~l~&P*bJDlySzTqC$^wr^DjqUs{8Xzfq=m-)I7w&jTZ&#>zfhb5 zpYvS5c2#W4?|O;K%SUA_nzK|4svPKNw(q30+Gw6t8?yHJ9!9rLb*XctRg+p4R;Qo} zVCnaRrWr&^h6pf18ON~BFm>+mcYbxmAarRWOD<ix$;SIHF6XuzmsOJ)XmOBJ^~6Ln z=BD<Q_*>sZ%Ti4O4{6%5^!Y-ytd27vKxVdu1X++$fc_-bSQ&S!2^#1@QEv*PHggpq z#}XET5s~_nwg*1ikC%hi;<ysCJgponFSoAA@Id3whrag&?tJu2M?0|gavqjT;tQ7m z=Jz$Ki3rlXWU*#!Jg>-D0_Vp%Ij~yI(Zl|-dk+ZU;nOUTIf2;a>gS<3`lXY_BgN|k zW!%wDuV84YPn>Y~T5+;-d#$;r_q(fGVpl6)$di>qC%i<%+I8H$Xhv>7Mh+&>N#wMx zbd^<Q&pMb%$^GDGEDinWR);E5xQfMY%haLQdwkokb^kRPRvppNhOG_ce^h#rr|R$p zbFl2zuo3-t)D3BrXP->Y?t+V|uhj`l=?8K5JG1urO;9GwENmAnYAP%3s)|-B(#3L) z2OTjUm>#D<U7GH<J-%o5O$@P`ly{V@^;pD9?HZFPyE-SCKZ+?w)f2fA`gG)K1S<wq zt~aZp7<LRVm*9s^z{@A}Q!8H5r~xncV1@f#hVOjSuGLax<;KOOH%9KRE77E;=QXTh z4}vZR7cZqV#$D(HuzSsZ0qlVaVI{(gqre{ljxBH``?E=d-Pp+zc-f&r1f>J?+*p<L zC@_RcOdyF|&EW63Xmkyy`v|ht%5gH?b(nq6gtoRkS`2qV9c+N%$0PtMi?)ydM!^UI zMivk?N~llhbH$|7B)Y|yT~MibWWSN;Ev73{5H=<#z*i%y_XYjbLyLV~uWfK^H&I&O zIsLAsGhGV<AGc%Y^YXP(v-yOk+!dz404XLD|Na7q3Oc)vL!>aAV!ptwyAFdM>rsrh zgSm7@fPiDVGDOxccBXF9eCRIrrQj#IV7x4aer>mLvw<1Xw$}HwxyBm<+<eRSiIT`S zhx$ArsM6-NZAxfi)#FsJ#koVZu?!;*VZ?Esl}(XhlKz!?nebh%B`y=TY=h*=GX;MB zPQ0aTQV}73gh!<N-0}HdMtlthiy!Wx*xSNj9dZt0?u>%zZ`UC4D_8Dm{N_CeOG-5U z9Z+xqVOAqsdW5gee?x*C)}{#mS9{+X)nwMT>(~$h5orQSoS}mdktW4tz!3xllqw)( z6cGU-A|*jWqVRTLs6^bnM$QX+&hzyL}W5JG~5Do+GMG-bZ+I_vv!&ikFU&RO5P z&iOO^;RkDlWIsE5-}iN2WwEINcO9a>I;yrvyt@XB2KTUP^92W9wzA5p%n=bUo<-J1 z-!FYu_2;e>^v>TX$@gsZvYE#Y6}Ry7`8hyZIx<aL+@__gir*tD3J&0Whz#)0+C+ph z7C0!DR;R%9QD<m?hh#ZXJf#HKtKFW5<ze20-#avyl$L1oe7|sl9d*QtCndzD^V4{s z0o?;m-WVgCEtJ2G;u>R#jH#}mGwe#3R2$|A&Yy#+#UdG<EZMtT+DLOnW0oqBsd1dA z!Jb`GOPE?8Gyl{tq8#<`9L%L?BJLDb;}b<FvFCcdYgRhLzHzl8rQ%|1kBMVQ{MpYW z9qa(xZ}|Oq0HH>%%vC;6GGWm3<ws8LzNW-NQ+KZAqK?~!fr#>$-&PgZ3jYSRk8lmV zGl41KzJmzlym=AO<gZ!_CRlqL^!2>n{keJhj8D10JmOIR*^oj(1z-&G@?8I@u?-2u z90}N=lK4@ri^$R5=tRr$9606>=m|h-8aEK%k5J^7gtsEac)K|5Tsxdh0|f5^(x#s$ z4+%k2iNCXC3YmlKH(!U`b$*XNts1{h%DC~N;Pbxs>BJwD75JPWpL>STsf&{awuaZ6 z%6PXqDiD{Q!7&DAs0YYUjY~bCDkkte0iE+DHMtGBsI{s!a=`1d2R8Q1(@6sbV~YE= ztodnu|4+|vj80lBeTOZA7(>w33SVUDod>KIsVTZB5)D7dyUH1a=0_H$08SbE9S~G9 z5l9G`K_TmuL7%#*k{qEPMU^IWT&a0WdtoqoL(o&RObmK>tjR}uAu(0VyXpL(q&@it zZDl$^a(8_Od}h4DyGTCx*&Sz>5NTcMTW>8(mMoh0*VL>#)$BhV4s9`Fx`uP*TrNF6 z0GB;(gQ>753s1#7nR+#G_(DZej~CbaV()9`NIw{11vTp=*sleFh?Gfz6wYjtC(cS; z%<s$0n3`$I2Zzb@COaB(1R^(L+By#@1yZ8DJw=8bdq=FGVWMc2%G}(<`<wMWX0Of7 zTruhBAweG%aze!sn<zb*0Pb!e1O5p;gWlk_t_xLIG_w3s@jC@?1j6-@UwSZ3Au`m@ z7nY7wL8MaxT)owx2W{hFe+nj!b_dgEE}Z|vpz-N2pucrG5^hqGU}8u#`t2&)E|HxG zGVBx}0+UXoDDuo%+v_Ce$Uff5iSVUzXnCsk3CJTS%JHr~djW0J`aj9!m2X~bYw>pk z$#zU(msAfr?kx8pu;Ca<B`h$tZ5$F%&B)}xcLaOEepDopfgOqyf>WGqCyr+w4Vko# zP+Ibc-;U!MF_E#}JHZA_<XJHW6}&9EE`8QMM!oSl+_crY$@qQK3C~d9Z*i-O&>y*H zlW0CqfiJjov7Gtul~*78*`LU@`NNrY3gzp4DJK^@G?1fiSuPW9)VJXOy3-}Ae2IDD zIZn(^YvAIe9;UkGSWdD0ZAThYs#yNeZIa`WrU3g8iVg84=uys67bvJkYA1a55r~>R z11|kgAc>cQmJvwY1<0z0DmDVa7!l>jn;T--;Ek>bc^Bf^o2u3CAJ#XLj@f-L91k@Q zWmJ7=W@)GPE1_Ltx6y1bez^g129UIsSo2jN<8lnva801dI{GPb>=nJSg~o@y@;04f zHon;!a>to7)?vS~yvfpFB|b!-UH|1_olivTSlv}!&5~Au4tR@bo*QE$j@V4)Nx&Cd zx}zOuyI?{cM+6hdgSEIQmaKb7AOkHl$P9!f1eFdlwE=$;LvrsADqan3+mudF2vS^O zx}AG+KXF5-3pN7>g~F4@LJW!{#u{XZ_YUpq+J56-q&x-2C+C2ZdXTuqczJ`D#c8DL z1P@m0?Rg2MAmiRv<?Fut=8;NAv7qeX(t!rwq`vC60Jc&%`{GOJn~OBx@SnYHv7-UC z?S*n4agiTR>G=1*7%sK;Icf6%Cp!VXt!)r@dw2VNu2sRO#t-K06(0J_7jjkxIEmJ; zD-4&NO7a=fz@tNwjp3JptC&Qr@6BhqvBB6chJptIo1^4Mi$?K+9m~igJRv^_&^gkt zQK9e&db$DNjAK1Em{R$8V;PgvF=s?44L8q9y__pqaoy|<5~F^0Sl8p)^5pn`fQ*bO z_C3xw=8~B#H1aCq!sS)@(%H)^vfKGB+ZUE+3}cJ^4KKERAEriEU!L{!@U$S6<=wj6 zcL>xhhJaZOn<cRIScyF84Z*%S7RbJ9BCLI25)x6uP2w|8aE`3`d#nn^{L=;SD22Mc zcfD@p@5X-WE+8b8H9!@!=qE)As1`dRqVn<kpM8%-9|hOUmnnThIw&k0h<Cxgi0&+O zv*K)U5j<63AS%oY<CNC&bS96R5g3^gJXt8Bm`N4KQd}E}w6S+@Im9y!z~#oL4dF98 zyU({E;<uLE>`d|0oKv@5mZwMw1;d&QO}7u3&q^&j;6gUol|LNQq9;%iDLJ1`b=hP) zKK6GCK7H!3vS!Hc8_u+GHevcc#XeDw-K6=ZG@X1z%y>ZO@>s6Pr5h@8_IlN%evKhZ zlqF9YJd%0bAy!2XERG<C(8vEpXii)U4dk8)s9z%N=upOwQ}=3{?E-~!#0y-IWjbRP zdB}fWV;uPmcZR@TRq~f&DY$vQmA8G{(?fl<L2m*J3SSUki{$>sJIQ~DQ09OlAVr{z z`yCiH!IOyMK?n(+2`c#{r|8p}_VKM*Wa4*ZV&myRF4~Fpv7*vHtLKWS{N=n|Lz1W! z8N)XtW~fXm<Vl$M#CGSuL|&&%KLP+;Mu#Czn_o=5LC-)n!C$RAG4{qnQ9T|)nQiXS zELWRC`RZ^K+hPl?zuP8=jrLiM7l@RNmp*MvFrB3sN+*n52qawQQ{Dg#OKc`#AzODq zaP7Weh}gK_TGe`fc|km*dLz4kaA)MFf>qjN&!R3?R5T#k%VlWuNc+Ssr`*d3*+jKR zmSwIv6WFVsw1E)sk%E+uwAmSp3e&DDi*db+mm@#es<{)N*^%sP^;8nxW7_x$s%gp$ zIqS*z7285J3#%pZHGqQeK>^9D3;s1im!AWeEO2%{KNr}0iD1ommh~Jr?ggGnJ#?TQ zjMKp(Y%2^*9u3EEXDd~<ZVl@1z487_?x<Ci9J;irJqq1Rh(R?3O|1j|`yN;DT%QAx z0$|4abNzTm5c(A|nJzNMs7S!?!;c|!I6hC7>#<YkSQ?Xn`S|pKw+E?&npmrNc>PMt zrTNH*Ee{c>`VIrlOpb9j+vN8rdW23>gmUojJG5JRh;lpLMJ<F;z}vqL6cTQ0AzT8X z(;Ix#8`#t)YUc*W(-2tG2{ds(KZlMJpnALpN1S^AEpuZY#M_8d8N~Fh>*IpgMX=Gm zaIsgZYY*>QUdtzBTTlL_H?NQbZLwTbazo&aX)E@*MrJbM_d1;@(uZC6pBhR?{l1C; zxyHWxnj}kJR^&6D=cQ1b#BIfs7|Ry7;`|)7V0XKKwq>-?w9`ROQ^)@De1E8cMik}y z34R*&3U#NTH<cyKlVmNji#any2?d%+C35{|R-Ys1ua2YW{pn$|lvw(Ou^=$5S2Q}) z?W6$33&z+Ar)P2TBVE*k$OS6wS$=GrKS*CtOAv^GQI%|m2~L<Zvgi;iPwYg+q4sYP z6|tK=dYrxQ-(s_w-pPJXuwB@#k@$lvc6*+!m^3VJZ;l}IgGlGEJpV$lL0%)gtzlp1 z#|e(C!h{LFs01QZgMbAJbg3*h1=PKq^Bu>q^mV-UgnqiXdB+5`39p2$achso1L2_z z{1>g9X4KQj<o$!n6!1;sNyyrcxv~4e<NvT~TQQw}6$la3lEYqvn+NS1R~`!jC4uT^ zvC|a8$b=!Cx;o`cEjPvW42=#2(z|UXJauWmpLk2XVkzhAu^F7>t3FPf8mM0>G#%=j z2IjwGP6fdvsq$sV>d}fmSk>sDu6v%E_;FXwX1};D8vRyk{7so4Z^!B^^h4TD)|H*y zrbC%#Pu_p{-V(&c@L;bAS6h*@#KnZ{MapywBY<<d&5H>a1+4<>4MGKbq4}xdy(^4% zRF{d@(0dBAX5CBsN_~BN*^B4V_S2#vf3771O8*E)2i6HY%(LWJFVIs_&%g#JbJek? z?6}D-x)^RZXBMQs9g6Kl3WK93(fdng{MKQdHHkyK5G3n7F^csa!*M(&JtJk9%#}$Q zD>OhxVjp%W@a$N0o@Bl~ONhpezP)YEG3Q#~;GCzd+hepDeS~sHECYYfm)Wl6!rI-T z(7Ko2Z4!}9UVXW@G&B&qCn=`$GezDs%I!YE3}JNRhdoY^cpP2WOqi7^IPe>U>_rKI z5})bNbsAj$U1(AK5UfE8CH*@mGm`8vgF|xN?ja=Tvz)%pe0D)mrBW-`x2do_#f&<G zxU4xbXr++U{V8{;OQJNc^ua=XQLkf6hr*%y%^HuBq1cC0Ary~1VT-uR5sA25W0TD) z$HPIscWG}1Zn<c>exjswGFJzv+V+>+iZyc>_7~%gFQ%*CcA})*u3C2yJ%2>W&O(pE z7st;LAxBgUq233l4UMFksjz^-dv9ij$I$0Q2+byNgc$pH7?xjOi^=aTg7I;E{()%w zVXbJH8P7rGJh#UqN|)Cgnp$sENJOfw&Hm`QI0jrx{P;_uH9xgiuo>zCdVN<V?)Vf! zlamRm=SHC}vL1tmh1<QR43+xddHRwSHYMNGvtBxb@^2|{A+z&C{Rps!(dfvdLxt2= z^cGB7n+Ep>zK@|U5Mhn<br@3{-UVx@NG4o|rNDOK3|LmZvKN*CxjmY7N0^6>j>ItS zB6z6|Vj}DCo|)aSpce)D!uT(C$`O-}LoibYY({OtVf|;os!_-3vD99PrxL{ldK^?2 z5%{~xV3D3IKN?)hA7KFHD5|9PnQSMijRu)HF}M%6_^uzdwNRtpKK`L5UEAL0YqR^5 z{fkiPWRzrn^@Aig_XIt|ViVnzwEA&KN>9i_uvs<awvwZ3aP%BW6FZ<;kZvG1d8zG) z`T}dr*-I$4)Av%6OU{ejJI~kbQKG;7GWB`g{AvxuF*Yanyo&#bYpE@p22$Q%;H3ql zghu2#hHJo4qY)13XDZ-(5$X^!PN1@?s4>=N?9V_STv0?kWl0ONk@xDhh*I8j^ywbi z1a$mhfQ(0AjorZ%WAod82nMMw=q}U@4ICCt0QB8?0{|P7Roo*u-wA{;4umC!`!L9Q zzfGE1mpPC-L6eMFwPe}x^1bC@Z;jGa$A2EH+i4=xe7WAC2Z*}ai5xNRF+wK_D01*H zeidF12j>^n2{f={>@ZF&w6eKqlmVvJW0tIn1Z|wkqG?SHMJ&);Z@iY=^ymST6_NTV zDpL7jTGpE7!ShZ7)P(}wKvW|ETrvxbN%)^uFrll|BY0z8h(m0_NH*cgy8sMP!c3BN z11sXxBzPU8kbcs1tj7QL>|d-t<&b$Rle@pxc^D7>;;yp`W-rmZ=q3V?1viq|yq~r2 z3G&%W4Vy=o9$Om5RaU;R3<<e-o8(g=k$1)C;m9r%`o&Vuq?5dkT=%v5!;f1|^tjZ% zc<gdkKIhiNY0Alz*rw5{&<!x|0DBRS^fOfee!Q)WjkQ588h4BT5V%+Xh71jM5R6dc zh&$c7Q4PJZi2ET&o2lftoPv#R_LK)bKCC%ZwKQ6~U|+FI!Hm}PPtE><5q=eNp`6fQ zwMOP@;Eef=X6BL1%w*V4cnxr;i)|5vLBL`Ju`D2T&Yvo!twRC)WCuTHI>wrdh!@K( z%knNJw9GC54QrvIiogZfcp`LC0Nl&H0-^d)kQZ@}fF_T*7e+`y?MLX~dw=9ip{DiD zvhwe#<^wA~YC6uyU$*;2kl97nj(`gRZc}36=%Yj5UQCdl60|O~L}B^BD)$}~`wF>} zy6_G?9mt?|QPSioeUt<bOfMns`}SRgR-pI64oIeyN!iDtcM~OXLB(wy2gg6FJbzu< z^pqv)dXc6lEPCfoYuU{Yvdz*@AOBH$#6@$s{0i-Pz@=2Ds&$Qz4>*jcPE`(K9;Cx> za&>GTsEOKCsSme#+0>;4?oU9)_be`5nKNb_y>Y#oq&4nX20ie1E><6uu*|7&&2?U8 zGMB$>zC4+J$|5v||Cqqh5qcFQxrpwfCZG-xu5ZjbEKq6ij1K)Vydv)~ze1pk+uO#8 zhTu%u<ge4Bcm<P)V#?~pKpA<RO8L1pwU?t;p9e-YwzoZN7>jGU&&P0s2|x`hEa;j+ z(`p<Pmr#;;4V(*Sh$%u8;T`7eg0#9|pytuzU6?$6H6iFc$FGYzW0(}ydg^nK*xdNG z?k~~Rrx=p&On#;3BcYHlRs0O#7RlBWf&WlA`plf$T?Zt`$BtGsTY2<@i~Jt8p$otT zRy<irf$$Xf68`n~c1kSpWVKol>Y?1b<JN<ezD{S*X_@Krf3POj+d5QPV)XB@Skm|w zTGHQeIr0`%95c%FCwZD`y&(4_)y;`Yv20Q!dv5M#Ii^Lnr<+m49*o*+Yida(k=<6L z-jDlpZMJ;nT{Kl}D7!VBsw@wK1$RR!(LVms)zd#uzB#6l57G?Vt*peXL^t;j@II%@ zzfLPHn_8l;g@B{~SB^E;ljp*su5H%fRQOMjz9TbkuMqN$tPV1yz%GP#!^GP{$k+p4 zq5{vd`eNhr8h{8AE=v0Pq4tpV$Ie@n6XDHKxBMvQ`PqQ?fQpATQM(B<goW++HVqbx zG1UW~R%4=hTF8_lAr6Uz#aK;}vX)kqELfBTC)&LFgrp0kTB%=nc5AiYd&Zx6Pc`ro zvuJO2H$ql0OMo0UW*pdpCQc|MOH(|=FGb#f?c(ijooXV?Y%^eqcp01}ClaDDCfMy5 zQPlo$_>A~D*729|btFO!sy{?AnbZ??b)?$#ovqX|wjO_#(CN~KsRgHjDl&Kk95bCx zI1xr5AH%3v*MN}Vd~NVs-@`~(@%21?W)T%{CP!dqPLK86DGMmq4IxxweXK@Rp5vkm zXtb8J7g_IXQzZ+`_8g(WQd7xyyPbM3)|lw}+e+@wExuP{ZPzmB(JXHqe$M1^rfeGf zbpgp<YG}!qqJYwUbw#q`fg{#q{EF*Efh5VY{A8{rS&CuVem+Qphwl@UKNMHvk_%$X z{D4W>6vJ&@8#r~3QyY*pm@w{K+c@Ex9m)cvtBI#zzGLExIS$c_UJ=Hgf`e4yfJwYI z(P;u_?q`tEBKF;(Y=#!3lW5Y|_gAwXzY~>BhX6b@t>^s%zP&QC%c#538<hzzdDWU) z{3x|i^=Z`p@|I|p7V~`-Q*F$kD$<WSx+UBgjbLD;Wb@o^t9}2d{hj7LH34Ymr!K1+ z_qrF!r>kFmhY(7=;2>4C&aDoAx9k|PU1f0ylT${&Z}Jv*7U!z8hp@IX5n!e*P)l(9 zP;89yzB4`^XlHuLrZm2!1n6T*{`fCR+y5^}+W(pJ_wW4u-%fFL9B-U6bswW_rF(Y` zx{QW<YLShcN55)af;X<)+IPg*dd`Y)wJK+{1~tf}JmahPJ&cv^-yg3!HdoSqCx_uG zNpjs!GD+^em}P%5)iopwW2e71_J%C?zSl9WRK36VqJMeek9q0Rlw}C)0A3Df#5ujN zt_1LQWa9ijkh8`GgOp%FCj_Sv_uoM5gCdgo6_s-<-}Bj431)g%0)5N*rLNSDm&E>i zX0xf&e%WEqOpUC`O9e*BVbS25tu(4XMEG?|pF5=H`TA?Ch@G3&q(0d~SR1`H<aE9x z<(z>SBFf!2<HmKXj7#sYh5ITWHPNglxjVb$H@gIrhBiXgUgmB@eQn<u|5?K+>#;K9 zRL8Zk2FLbq*&Y?q==QJgzWM#iPas^)+>iC%d?f%l6VjKztMg7~ioSyaonvLi5r!xZ z(9lL2@1!q6f#o|HAINZNB6b&Gh{(6_0;N#DPsr)A)R2)1&f<d6D|+$_KOf*9;9|JQ z66rBPCzzdCC^YTvIK>HPh|2`B6-8M$#RHjCVfR22c4i1b|I+Oz-Id0VE8P!5``pj^ zsNsCyn{w7qQ~!F7ug%Gn)`paT&-wMiL*Y-_Q#Wk4343Ed6+Egeo3KLP`7BtQyR=Ge zb$OZEI9ssYWQJat0yGVzV7S1QT#O(2d&jO037q(waPCsm=_ngu@x2Ht@dIIev}24U zdT(*92iD4}!OoFw>hq!RIMeFf-#a2|5B}sG37DO-{ABzIPM;gdLqoGo`FRKpAm&DK z<)(0<;IBB2?IPjus+E0x^XS%Q4d+q!yu{1)@wW7n1H&^g6U>(<jVV&IM{bm^+|d}$ z)od#oeH;*_pA(sdt5T6F4KJ})&Km94&eD_)7z;mYl2yLKyG*VN>_FccK}F3^eqVjT zpbG)C0X|J(-^uL0kJ|~1!}jtHr1n}@STQ+Z+O7(Y@^d`r0+SVyGPPvM&O3Ylm^w5O z9^~vPQmlSA0JBrOBkWNd08<wp(jj|culL!YF5*h`E2oVpXM$>L9|m*VeQvsKPu|t; zEC=Ps1>D;yP9&SMQq7!2#|tSjw^M>FO)NE<%C0tE^w3FC@@O<3*BsC+b<8cB($9L9 z-7rBaK|42|%G}L6$w5P7o${b{NCelM1OUYu9jU&UazK_5Z<A!$33V0i8x32xh<a4R z71Qe~kdAMUkI&gQYRR&ba$P=><u&lRcGMW*1X0*#{H%^WoQ8|6jZO!^Xo}*TSxons zUm0;?WzyiWr~^C^X6mZxa>tt}zfcO+ht}NU=stu{yp+~)Ijy+xQl-+;jOL6zr6u6& zOdxT6#?A3ENL0nRHMDlT*RatqJltQcUGXQeHN4@&%xY<Kp|~oi6OsgtnlOXwAk4eN zF@hQv-Qu_S<_1AX8daM269bjxp+l@LL+=lWkoQZ~uX%p6=_k>wV+G3Y5$F0_0}qCn z9p%FDe}P>E1_40Ef#eqo_Hbl-=66Fm^E)79O50H=`xQctr51}54hfHn4q2e=UoC(q zF1&=c<I38{o8*Xgjd==2c>;g=Ltp;%oKvq#Hwnq9IIKP#WMilNC@Z$SY-Xa#-Z2L4 zZMg_Fr{vmX#Z+j#a7uZW8q}lXYpU|>^0GbsubdX^$3oroc$i3s0rt!V4grk9`|csM zu#zrW^Ii^}ghOU3Pq6f^mQ9ays*OdPG=WO}i^k_ga(A(XC2)niOD=Z}hH*dhix5BI zM_wMNVxR%0yBmK9Cj(8s!0?O7OR)X;qd;2C&LpG0X$Ql@qfNUC&X#!;zHRn7oELV# z+iy@*GKH<F?p#&jYV+8+rY|peU^Am=jXsl+Y876xre^$nEcQ`I*|(BM5orbf16ns) z4eM&x=%Gv_Q1$r{I!~!!vVJ6uu30H(>6v9A*GSOMb94QEnCp;?d+H%;v#i^JEAYKM z9gZLL=~V%tYa@{gw#0%%f`?uF=Ufuc1VYD!iHmR~?MA<9)YXL_HFAJ5k}EnZrr%x$ zi6i~`y0f*n+uxCYX*l<O+u1Q)^J0a`#frnDRle$FIoGoMHMHEP4Ra@l>S}?+!!e}B zv8>uTHOR&>*t*qlEO@y&m+q<4H;md(@Bs-23sx!N;zxKAeBfT<M*+c~6G6eX;c!{r z3637L`U<bw{<QsJF#LG10bPW1dve8A&hNL#pHF_XU#k7{-0nN74N%|$a2XLOa~u}o zNq8xq;S?Uu`H|oa`1mf9prVR)Vmictt7~2`<vQA?w)kjW`>VqN{}$JmbrnauWiS_d z%<iNuYd#w~hPE!L(@m3>K3rLQdcssI%0T5*zXe@v(YWW!%u1+(wdr7$vB|L0ZESo% zdh@r*0*7z;tw&X&^RGqDZFGsp%>det!lG*_L}knuv$AO&M_)jMM1i(9Xt~`8lKOCA z@1dxWNJg|mU-oi(^)miwUbNMP$eX{n+WIRBu47YeJ^&BhSmM4mp+#{;+!+W7nh${} z7@78%k`Y`cCIhFjKoG@ZdSMcju$&i}Hf^j=$&Kkc3Ut{ykG{x=<}U$~@63K#F_*ir zPP8h?xw3CSFHOpC&#Ce(JIk^cD`!uSR_RyhH$KkSulp)&<MhcfJFDu?4-s^|3hDir z4#BW)+7&{Ku8pDfF^VZ<1wIm8^ef@Wk1kRm%QNLf*YO-!QXMBaxn!(TCqf=7r?A9( zTEs(?1?nIz_{nnKCbp&9LFL2j>95>(mm9p!oiA(kd!t_AJ2Zs8Gi<av@b21T^xB7g z&CS8OXy@8T!Di|6M+RJ%hosf3JcE7ZhBZFtyw7edTr4<T_e<V9Q~gs6c3!Udq0FW7 z6c-I;$KHixrEAwk%MO|!OLH5-@8G8re5mn?X8BBF*T%Fh`xYmJb<(dR%OS~o4@U`_ z>cShI3`iTQHo=x!K;yDcn+8o4Vj42dntkU3ms2#<&0W&Nch{Jo>W#SEZyQE>=}>Jy zQu*!E+9#_*u4cNi<wl;}&36$&UNft<MIkJwg^ey`M$sYSE2IKK7H0-50Rn4Uy(|_x zsggWd#)u-;JIJE@#nW+^Im`Z0cmR6AU-`t)hsdY0s)ml%l>Rvp5O)hGPXx#f9ZPRA z>Lezhgh6O%Kk)$dDsV#G>mb6}IYsLbmPckZ5sq*wx*1`aiA9Q|<aOq#a?3<ef#-vm z1_wQv^!ib^bvx;7n&bCj1)0)*Pch>m1K$x%phvS>KTa`_Kb8DtZdpYLBzIg5sMS&1 z`(fyewEQjq)qyHz<<%*bt<>;a<KGHA`W^;<Zj%L$lw__h?iR}`ffvFm1+a@83*B3E z;G7RcOzlwNsX?%eHakWoc>~!P#CY9Ams~P(^Kh0}IraRv80WK(Bdd244KUtf90#a@ zG5K^6eUOI$oGIWUSpZb@<Y1&E&T`63VS#=C>vIWg<|Ib^I+G=o-gH{hE`}f-7`Qv@ zmpOXdX7#xN<^AU<zm_z{n}ltwEmYA9fz{B8&7sOUYLPQT<L@2%#X?%oYkZA|De%VZ zs{X;jzh+yG`yQU*-UfwaP@}JM?IY@6^QZZ#NFQoS(dkrXMfZ<gf##&L4CQY6Win+m z@cVcMEWh;MW<0f)vyGC!(#0I2lqZLiCfCMf?%h9scHizhnC^`jfbHQ8vG%Z^c)s9c zjR7GJ`KuVxwFUC)-q?xt?CX$>bs2TMv_-au9pKd1v^Jz(Xtw_78JQ=c{L6KTX9>nz zFV(@PV919p_ew1qxSUnas*=;FPS%zhFO}2a*;wm;&eoW${aF3-S-6toSv9r4ii5qK zH^LQBnEXKhK*dm&(!zEZGVP9GCXh4IMes*>&OgTFZD&ZFmS10YC}Wjqh*NJGk<v@| z-}RJmh*o1}4&`3v<=BK_t=XQY_V_~|=Yl_6BAF}g3T0XWARvKwnD=uPcrsvu-wBh# z36Ya5pDs|7ilkq+`7`XC)<$dDLzUR5?g%5p{PZJBS(ew5(}M54uGwJ~e)dp>eaK_A zJ*S*;qjst!t+S!lrxhBH)qQQe+MLqZ6$<oyaqrYDH4pkXE=qk+Iowoxm1Nwna_#Z^ z^+?lia8NAXCG0}_QN`!6P7Cl(N<49onQkS$xu%j6#jv_-rU7kRjvH8}Uk$PWbS$VS z*I(>&@%w7ELQS(r5>64Xcku4>YpA{(vsQxrLR>H0363W#pD_ZOf;ccXwWCh=EjlC$ z^yvY9OzFfF!Ow!p@1J6~9t86go~$5*-(+2b5@X11E}>Q0UyG_=w-1%;IJZyT7i;-^ zR$8)H{=wtDs%4KSy&MZY!sgnuYb?T6u2kC_wiU^f)a`SQwCA6+A(`C2XnW4B(B#y> z1)Gz7PE|JH()^MN56KGe6PQM?fN(Y8k=$_`yRpYA8~b+rkDtRk4(!+wc9>L>^0r4g zH|v@C|4w%BFUF?f;s2}qscSej-i>#X_mBi*0#=$7K|lKIzmjel9ft}1`W#@jcGW13 zy46`R75c-o^uO$7|5JGU-}nvF$Jz3U25?231+nkDLwCDVXTu!U)&eRVe&ov_i@0ap z2s`T8T|jOJ<8|(Uo;A}p{%z9pFVDQT_t@JC<S;$J^T``$4xS<`hVboBM|a7~Dxar0 zzgNBK0Xd&WjXuW8b)VO=n<-8O<s!l!!KBLuCfQLhEVNs1)Xl7smv9(h64eTZmEYlr zV}KPS-ihQT1D~;t)W3HuwQ&Fhybrk>Se>dwC!h7_lOFuN<8?TSCkNlu6AZs1yq1RZ zb|T;Z-@FKH#yyiS1o#M@0I5Ck&x`z6*nigKUpL!7Yx2*U{GS|3|Ll{0_R0UWPn^RY zrh#c$-NVXd3{ug6-8^0!yrv_?5%0XKFPrzpb3iy{_wS;&jpoA9mQFE<(UR=H8uk`o zXUNF!k?>9VK}fi~J}evcF7w-Agv=WxLk{XoG&5qij3XJa*J0_B>tAR`+w!T?CeiJQ zGw)-5aTUs6;|^$OlGw3#weF;7C9VD2#6|y3q0&FE_3smX{j-+;U)K`(_w;`N!Cl$= literal 20925 zcmeIZ2UHcywl3Ujk+bBSbIw@>L~_mo0+Msi5|)A>l0kA*kemd`L6D5(tOO+^AW1;- zTCZ{MbI!i!+<V`BcZ~P`_r`BIdhvC2RnMAL)m7h|HAQ?vECIKa<(1?C2m}H=0{;Ps z73>E}va)8H+8Xjos&e210KisMwRdra-U0w;7cUQOg?n@l42|f}MgS;)0iXi>0AOzE z>87S5uM2=lNluo|6D)Lt|B-&q0N|SdV3J!=laB5m`Tr$^Z|UaY1pp9Du(Y6+wWlSB zpMluP*URlD{{X~97LGR<ihP4Tzy^Yt>;~KZfw_K_`2$PeU@K=AE3nK>o87FOt!{8P zh$DQwtwD^W3gU1dduv}1Pl1@p$=lf;#AhHTa<(@21OR00n|v>8OFIzrff(CES6ddu zw*deh+x9Qm;xE|C+7E0e0LZ$!`Fq&g+Ii8jS+djdi;IiVDO&qFS$lbLYnod+ntNE$ z$+|kbnY;J{z#lr_)B*@@`j!stWD$OG5fN?y9<clWp8nH^e=qf)gPXDaqr{QcuQ7v= z4E-bTA9eqc=b8rqVkcnV#Q!7DA{_u4!T^A5?jLzfuK)n=835Fe{zV^>oB3kr<>e;9 z%j@gw%VTeC$#c`8e^39@0)H?0AA`Th<GGRd_r9aMZ*61l?c_yw)2WuOPOjb_be?YJ zmezFK|2&ER+ZF%PtiQB_OWWGU+QZrf6r~T&GJ6+WaJXHp?7i$=UFhsx{-Y88Z&v$D z8*bnq^BM$%Z{Prt84qx4ln6lD9RiREu>hopIp7n>-^WcALm#;L@(gG<|Csk62A}`> z{Wm|*IPezgX>Ut+lP#;QOK0ir;d6t*Z{p?$1;7OG03v`KpaB>GHh>!t07L;vKpKz( zlmQJu7cc}&0ZYIRa0Wa8Umy^83WNhuKs=BPWB|Fq8=wTJ1Zsc=pcVK4^a6vx7%&aY z153a<umc<bKY>edAwz@UKnNk^5LyT`gcBkF5rf=?$U#&g+7Ls?BZw`;1>y|}goHw( zAc>F+$ZJRmq#Du$`2ZP!j6*&{mLXe^ACL<u5)>Os1f_<uKzX5JP#LHSR2OOjwS~Gv zA48u(W1(r#JZL$z9{K?~2%Uy5L3f}hNB|NR5(yGL5;u|<k}Q%2k`a;(k_S>SQY2Ce zQXWzzQZrH?(j?Ln(s!hDWK?8AWIAMSWJzR2WIbd{WOw9X<Y?p!<YMG{<Zk3i<YnXo zWH<^A3Ka?`iUf)h$^#Tz6kn8ZloXUglsc3ilxdW8l%J@ms3fSYsG_KfsD`Kxr~#<a zsM)Ahs2!-|sH><aXlQ8UXq;$w&@|A@(Y(;Y(bCb%(b~}_(ALq;(6P{I(FM@u(T&hu z&_mEu&`Z(Zqfel3qF-X%!eGXbz|g?3!tlq4!+3+yj4_I_j&X^JkI9O82U8c*0rM$l z8fGPC59R{q5f&B}1C}_J7M4BM6Rb3>w^;pH%UI{w1lSzdve*x?y|H7li?BPeKVu)^ z;NY;}NaGmcc;Uq2l;CvXe8D-xCB)^yRl>Exh2du6*5gj#?%`qJG2_YLnc_XhOTl}G zH;VWD7RD`>Te7#zZw24Vyw!AT_SOkLAwEC82EH?XG=3@m0RA@uGy)a^IRYz!X9W2K zT?DIyNQ8`pvV@j|p@eS;dkEKwP>EQHl!zRNqKGPpMu-lG35bP=^@;t6Gl|=Xmq?IE zSV)veoJitGYDi{CE=Z|KWk{__UyxRkPLQ6GQIbiMS(8PQRgq1TU69j}%aJ>hCy+Og zFH#^=a8PJb_)_Fh^iX`KB%+k0w4{urtf8Evf>Lo%X;TGI<x>q&olw(ID^R;pr&D)P zf2Sd#k*0B=NuqgAvqeitdzaRpHi@>KcAJivPKM5jE{(36?tq?(UXk9L{x$sw{Urk{ zgC0W&Llwg!BPOF5qYYycV<+PQ6D^Y(Qy^0r(;PDfvlz1-b1HKm^C=4}ivi1XmIjt@ ztYoZ8tdCjCSQprE*reFp+49(?*iqTV*d5uk*~d7b9KsxS9GM)WoKQ{?PJ7O5&T%dj zE^#gwt~{<;ZftHDZXfP4?qwbl9#x(Yo(7&hUM5~6-UQx0UO1mHpA+93zIlEEer5g; z{$~Ck0vrOC0+|9+f;fWmf-u2G!9yWVA#0%=p-;ku!fL|Lg*%0>MMOnBL@GtLM43b% ziDrq;iV=xvibad{i=&9&6Nibnil0k}NO(!qNbE~;OFBxHN`AY|dfWQ;o7<~*819(g zd3|T;F5TTncVFFIlA@O~m&%h`k!F;(k}j0qlwp@~kSUkhyT^CW<6hmpQ&|bwAlZ)l zNcR=)N8TTiBaqXR%aHpb&meCrUnYN`AgmCe(4mN;sG^vlIIBdhWT{l5bf7G%9HiW> zf~}&XlA*Gq%Ax9^+N=guQ&xMawxG_e?xNnH0ca>`ywq6KWYu)nY|%p1($vb-+R*0L z4%F_`A=EL`Db@L@drvn`cV3T8&s(okA5Z_Geu@66!F_{7gQW+&4}u;H8d4bA8#WrD z8R;7p8l4!+87CXBKNNl#{&2>G)x^(az?8z&$+XQ3&&=HH-6NDo29HV~U72f`=b4{a zC|P7#99YU)rdsY;Nn0gZZCT&3PPE>%xnq-P^Ud~-?MvG&J1M&qyYKe0_8Inv4vG%B z4yTS9jzx|LCj+M{XLRRB&P^@^E{-nUt~9QGuH$Z;ZsBfA?h@|F?gt(!9)+F|PZQ5Z zFCvgk4SBP9hj}mi-1W)yIrG)`edmYo=i)cy&*A^VfAg{Y<AMO>0Ly^RK>EO^fy+TM zL9c@$!4|=tFh<xj*!mNNC&f=OpE^Dr4&e()3^@)p2yK2w{p`uJ)i8yyvT)pRkMP;& zlFxG^kR$9PMqUWMNPhv3w2J%~#TS(lbro$H{V|3=CM^aLYZE&XClZ$vj~ee1Ka(Jp zP@IUL_&9O(rP|B-B)X)?q~m0><bf2Sl-yLzRPWTKG}W}mbf)z9^s5ZJjOomKnN?X- zSue6qv#qixa-?%AbE$KqaxY%lzxwoA;dOl;OI}JoYQ9hY#vA=NeFb6#C504)QH9q< zE=5bl+Qr=^q9rAzRHboc&@%6`t@4NEV-@!+8Y{UgUssV-MZE>ydcWPSepEeEqf*oP zPV8M}Epu&F9bsK$J*3{h{-D9GVX4uualA>nsk2$Kxvqt$rKpv@HM5PlE$%($`)BVF z?T_0}I@~(Gf3W?q+G*A~-}Rtts#~jjq(`NvzgNDut52q{z5h;s>wv^S(?_w74TB<s z^+Uo#b;H8Lbt57p^`oMrjbq|t&EvPn-%m(Qe3-mH**m2)H8`z4JwBs1^J(_s?9wNz zPv1T}eLkG?nY)~SvVgJ>xp-?a<qOr9yszwEtCmET+LrGx53T5|%&%Im?yPyPU9CUc zz}ZOIq}?p}Ch)CwOMYu?+jx6@$8G2Gd-yK?Zq^>#UhTg0{?NgLgY`p?L&T5hBl4r7 zW1-{D6U~z^Kb?PGo<^RLpB0~ro%dZBTx?$YUZGv3U9(>|!&Tr52xkNWu}q;Z=VxyN zS_n!U06Itqzzc%>i%7(RYy$Ept^}V@Ku~_=&w-foPrL$RHVD$Md?<+dAyAM)0KhhQ zc|J!26hSsl0KUim`5*6e0rR853kpDA{X+`UpZG?~O+p2b{(L7Yc=tECMmYbFYXRcF z%7uU?#tr##0P~@L;)NSa08Ed->*m?t{%V@ryLjo?xw_G5dbrZ@i}LW@m<j)0`fX!@ zz4F_}`lqS!cMcBxwz2+G<Loc}_S?q#ZDakmv3}cFziq7FHr8(&>$i>d|0f$u5#&KP z_Q&7;B0eGJ0a*YQ83h#u85I=;6%7p)9rG3@CI$v35iTC~Em9&fGEyQE5(;W&S_(== zDiRVpE;>dQR(1|{a$0UaZZ<w<Hg>j~mq0*oA|^T}Atojv8wCjk+y8PxbO8A1priK{ z5`-Rr;zN+|A&72(7HlvI=(GjX-#8ouiUhV74IKj$3oKA~3xGn9kf6v&C@9FF7a0-+ zJ_nHTQ3&YyWl#yV%+VO!i3FY|yhdldSN(xld;EY&(841G1CxZ5jGTg*g_Vt+Lr7Re zR7_k#_P(6Ff})bLj;@}*0Z6Sat*mWq?d%;qy}W&V{rn$?J_`$f9`PbF@nuqSN@`kq zMqd7#g2JNWlG2)YwRQCkjZMv+UEMvsef<L;Cnl$+XJ$Wro?Bj7U0dJS{I<1y_~YpK z<mc(x`NfT15YVvvTlzz@f6@yd)C-D?jD(DKqZb6~3tmY0$SCyus01=vXy)#O3<6Kl ziS8x5uKs|*D5!lvY~eAENx~$w%zSvG+Aq!iYl?;Zw>0~^V*k)<4!}l&fP;sG4@dzQ z$^GtOYbUSKWx|(EUZFOpT6`41x!oKueLIH@5`jt8fK<Sh+uA$gE{z$D4~`8{M>IN< zoA@iVCS1POoZ1BVzU0|3z*yqLl}|U>fRpKYWJ&f4^%a}P0&Y@G@DZhW55U1EhuMd$ z<CA?^{bLiXDT$_C>Y>I2YZ-wHN~3R^M6vY??RMELTz%?ggAn=Ka!(c!@g3nB6sg-m zWS>#0d)#5q6DVROqZ)%&AEA;~ncA5x2JLxnNuz5vE%1l#s0*Nykq@jq!B<G}Sd?&= z4L;pFj57i>mr*CP;*?#wvac*F08Uyy^T#I}Vb`-?Ht!|@VOZH+WOr+KS1H9ul>xhZ z&)s{`3ON7WnjE=&%gx>^;dc!?k9UTFM-C0A17){Ks+<zI@CPpY4F_y&(c)L5*>-nS zB~r>|Xo8+g9e+mvS042-LeoRmT1A$)eLWf7`vrL4cOulTJS6>Rkg-f&QK?<-CuJ=` zvW~X2t0KLXmfj^9E8=v@(#uc2(SN2W4LM>(@#jKSemWu`Xxii938z*VIj?wq+mvxy zE-shLUHZM>Q*GmW<nQRDKM~R<C8Cdt?arlJS8dN~$_lxao6l3)*7$vv|Ix%@HR^QD zKar=Y^zn|t73~j#(#2ePhO;=MOs&@1OAh4v2mNWE-x?Z*5)h>WbU>5%wtKJmSg7`N zwE44o$3~>^y1=-r7kc08-jUYX)W5hrTm4Y@bMydJiy%QVva`<E%bgvA)2#2S)SM*c z-<`6jd(79Pt4y=T+S@J{EGOc-c{F#)I!=n5$i9v|9z&8%l~m^oVmCfD^Ub;#gFeF* zrCPht@x7_f2APN{#XRtwiwIw5ZW7;@rm~H)uB)4_tgNC~HxHko;lAqZ<D~<bV1fkT z+NdF%TeM9fw^d`T^4X+{-Z6q`n9W9xgKQ})1gn^CDKsG#SHk|I6wV2}y{e^Ie3RUC z%l=wGt7MC9g3|##=fR7t0k@B&Lrn4*tmGDA)Q=xWwmTiDhwW|<yz+^zQEEih=NOzV zek4O3!BDI3+n{mgI-X$bhyY9w0E<Sf7)zQ~oDx+KfB-Np$jm(_Hnfhh;KaJMHoLp> zrJY;&)^@UTZCWoNk-H-p+K>~piGtcXCz8U^aN?ENYa@SVDf^Ml4yo<QF^CB&T51;d zs(7Vgpz3yPzm<aLt%Ydcay)#;qi?B+19w8+$DuS23J(d|SXm75ukBK%VqI}POax?S zit>ZP7c)$H{X%<E=C~`Kjk?U@_+46rQTf-<G|It6I2*&zvWvbF#l>=Oj#mkD3w&Jp z$r*Hyl5Ctgiu9fmgI*#Fwr+`Ga;p2o(>d`Xo1%erSWngDW{`-rW}Ty>V@t3RT<WQ# z{9#IV>5kyHgR;yWyVs5{Ug-yDaV4CGFr|fk@d-Q!pl^KpsfEl$d_7MJCRTjjhgva- z1MzoCRq=Sm$a_7dv86TF8>!jc9<KXSvmr$uR&=~;Y;AO;jM``TGH3Q~p530Rd?Z*? zA|ZP9J)c8e{fkH5C{uE+<0ecsH3;C1Qp1Lr>q|+W`msy*n6IK=B+Clpnio=jsvKRl z>*Q3<Z~0yX`(9zMoxzs9-%fvTuSuxzwpX34ZmkhZ{-M72V3#gYe&Nwl^dkcWdq<o% zESy#t<i0QMSoZ}<T&G?di;GxIe--=+r@TrKCVIk20w4KY=kLNC^3x!LtRgL+FUBAs zzRUHH)5Ewl1f7wUy+v-}VS{6TyLA9}Wd(iQ{TbeRNs9pEh!hkSp^r7U2U9xwnH9;G ze#kjhmIWHSAMBN)dJd(uUp;POJ`}8xWY8%!QRZu=$-^@>{fYo$rxMH-3VpXo2XhUc zUnps79(lSmx=?tJp-tx@fR%0QW{(!NH_O$r%CJLSpeGDx!B_k9ir;v?(gS*HeGWsJ z5ou=Kgs*nTWg3?qB1f^cK=B76PAYu0Ip4j$!K{rZ#BphSZI6%c<#L%fpR7C)YbPIi zYr43g6o3=0CF97IN;t5d$1t?sK$Av?KZ+(+>8ud3PqMNyYCI{^Qg2)#fu3#^P0Pgz zy%bf#-B3b$Ldk`;*daL)owXmt)h;z{ws%>b%4@cxF?n$`Gr`<8S-Vrxs#5wujl9V= zWSz8Mi!h;@l9e302QoSYeR)J^WR!fJlWNVpQRM4tYfmFO>dw#`l?V(H00VSEYR>g> z!G-$oOY9!V8)){+vXdiIg|OQv5^16!06cj^*NDEWK-jKOMfR_Cdo{Itr1ASRt@W%1 z^(=krFE3#M)Z3NC=Kgt!Y+slXR0E=0j8X3iyq=6FN=)CX)ApLUrhs|QZ@yj2RO2bB zYQKuC52nmOcY57A!@GCSSP5qp>&2>LIxXCYodCW=`KCjwcYV%a&?tu%PlGmMbsD$C z=Vu<B(bzE0B%jH9+pqHtHZpN8i=SX^1137wcg-FT!aAz1!^(nUzX%`Vbf?x2ebN0P zFYU#FJD~M|b9u_Vy5-S)_RtSe_4ek=h0Te=ENimjfdvoyk6RD!g^esn(;E34#$;_4 z7&7qv=ryQPx6|6dZZOT#MB}WQXC7(dQh&hF<xIy|sc%RX{7Rp?A*_Pi+c99re~<A} zJoDqz`@u$US`71PS(aL+q|n4yCEe=AmGL#^jP_=(VS*ULS9heA9*nddpyv9G*t~DY zL;yPk{MZ)vm=x|^T2g-WH`lFK)b6_d(nw`FfH{FlV4*hZaYIiOkKq)tX0WNWHtAFt za-xTN!iHW@*<%FIg#gmC<%?84dm(^u1i)nT)erNdzv~`%)d^)lW?I1u#Z)gX|C-iB ze0)mvAD2oyb@G@E!#biNP6bT?nJw4y+sns&6H$W<0KE-UeB^7{AE@r*246*e>i1zg zgZ;`2BWe!Q?NvHv_a;acLcbr3Oc<Aj#b9d{E)ZV#X!>!qNidv<v#RHD^iyY!wJ&dB zRveHafVjBp!sK^vEn(t24{OfSr>Wy31G~bvW+N$oZ8=j(Y|^XlBN_#w{v#v&6Hll# z=@tYNb`<~Et)AR!lC8Bwn#ulEUm7Q3&w?@`_E_2HEmlXL15j}{)!v^j$IFvlGc3hG z^@$R&6sL1OEjL~sg_jcCessi&0EjixEA!ShUW4bl5n5qco(Mon0Q_|0zojUM8;)ML z4j_OYosCp?kIF8qai`FD64({SO9T*EVk}#Y8LV^x8y(8M(xaN#qbjSWlHg9TOwKy| z^BwQ(Qe?>t`~G-`)-ccV#}?9ZZo2vg49rIaN=zwC7m5B-aRrjBL%r&P(Yi@@wd!O{ zaRapFj%<CUi~y%+&*=!z>Xe?YZz73}*g?excpyx1kG<M?L=KN%p~*v}6_GEe{O{Tb zxMx{Bp_N*oHKMzGz9Wr42!A`Ze@eL}jW2ULI*|ffeudTpFW1F5o!yZp7(GRsNyJ-u zg{lIU`ubxKjwFr$;jhJ{e$2vAqzP5v<&$)94#28?2S*J7aJ4T9K@gw+Z?kMFA+bmh zhToPAQ1}17WmEV%cR#K`r3uz5B@wCB^wd;dQWM!*h}P}efxF^*@?5u?x>VN9^UD*a zV%y8rE$uzCeY}ypB$r1939(ZUYBjV4DWfPym=|1+OTi(2f$H=X1_wF(>_WFC^|I@z zxPUT^c+Ij7P|%S%#hb~S`?@HVtb#xHMilMNH>h$r?ET$jo^|-a)i%5Q28ut~t~CDC zSWwqVaec$7#=3U5PYKS4(Z@jI??(Ra$=`GG|F#dO^l|bOiO<BMiE_1ZcT0}^ZMIiv zzdqkeS2)2g9vn6czivO%9Xo5khv`|&wh8xdKht?fJQDfqbGC$aTE0R6V=_|v^9UeJ zRy69Ep^J3e4*}%3!ifyShMP0+8iVVG5kRNOYXmTQfdIZXfQDoSE$KHIb#D|ZFi#Y` z=jz;)(;PkRo2YJI<{yK7qsu+r1?#-d?76@<k2<*x`?J)>;;+SO0|6}aj3I!{YuGQj zYIsq{bVU*&v9Qfduo)!go_W7kx(8nm(!NO@-_p4J3D(dYUb#kp+jEg4^|w+kg`s=2 z6`rtzR6E!ay#1HV!sk5~w{)^)6r_%BnjOX0G_EUneRy2%dZy|{dx6e+eF-}#wmA%b z_Q2v=gZ6KQ8h$N~P3>nYwNmE@!0eYq9a&H!`^c;O-1D0z%cVX1R(i9r>Wf~nNZ?%~ z6W;W#UYnF&^>lY!;R17K_r1Gg9!2D9G><9t$sK;RPOh%K8OD*qSf}WM8!(2PjEw*S zgE&kEub0oKv}1-afAG&eVJvwC@)s><*S*}21rfX2l%YAt5N27aue8dk9tglM*qbQd z6WSW>^9p_2?(Xu%l(F$;(T@@fN!{?J%{D;kSJxLTw11(^mSbD!sA8lphD!VzIg%Op z)>Y|)*lGGk&`&||t0zi)c~75jYAK8IarT2;G4}u4X7(O7T)nu68;2bWS5(~WcKW<A zX9wYVcxYH35I~T9#fRkL(omhZcX*v)``ZX0hHc6zPG!*b@}hm!<T<%RBLX1Z$(YIZ zEQVPXJPq&s;JRiU@j1gaL)t={47p#OD!lJh-EXVsI(^IA6#>A|&ZZ>?of(c7DN{pa zy@C47(@%@ELp}$7m&5&=88qJ<24Op9Kg0*yZ*9-0lo*+s+G~C3;<~HaIbw^#r0*OB zZSsFZNf~C{zlnrQalsquJ?!!kVtJiQJaunF<#|(WhH#P`e~g}`4Gz8&fiDUBq@!)@ zbDeQ7Zgqo{wH040m;}}UeS$Ko9>=#2sn5+nkRyj?bk7D-+xxmYI_E8ZvOl;#DY$RA z=6m!&FlOL3jFZNYOrpA1ffz@{7-nRD9dt@%RP!DI%$}@iPPe+37~5w!9}8@q2XF?I zMJaxF;dz+Or)V3!+wjoV@$Qlvtl2Seai^$TRI}h`s(YbvS?v~eQo;bP0e|1dc{k}A z;WJv^+b%{-2d>XqL&$8Em|`XUcB3tR>M0SU-fHFyv!PCl4uK#)vKY<{_li>4;@MrF zLhApac$lrf{+jE2Auyj_U4`(}Z&y#)XLl0;0BJ2oaI*vT;I3|?oq0L$Ta)3rE>3xg z>;wiWX5tJB_ru`Yg{oH{8oZdi1^r+>7qJB^=*t00F)`go(#08i6G7QJK1=~AG&<p* zquI!KP>qpuGN=hvBF8K8p5DhE=QrbzRh7BtnRS34+_-srT4y;5otL$Jl1R3g-=AJk ziEHLixg(S=bO@bnZLM;`S5%6BSSs>bm`Lx{%Z0j?ihG;NV=h%(mf<RCKO-h=IG&|o zg(%33ntnM+aISEuZ!R}7Eu0Z0W!V<8-(B`g$nz^Oe`aezCUie&1-(ywFzD4zb6e6+ zioLS@1^c5XHFBuQOsXAxVuKIGKJ?;IZ=YT<JXZMW_#mw^LnPT&Q26`Cs^1b{R(l#c z6<%f$wOH|KQ`Mx#sCI2#8;lWGP37T~_cz9gJ*Qn0?Q6VT!a;m|4GMPtvPxs}O}AbR zG_m62{aX(mC}IjG_I~V5^mH@Wd<vWJn0J#FpOc8D!j451SwAohV{X-qGR%L_{o0hW zDKFl?oJJ%2A^$=L;i&MdW}1weJ_aKOAq&2z7<yV{kDjoJ0chsYz86{_&$W-tvOg8r z?6o3*B`UVVyA8gDrm{inMh>Mv_fm4LbH-O#mhoQ(kQh$JV<*Jp2Ja7Xd?{L~a8$)B zTF5l?h6MbC)f;W4lHx~PCGhOOlOLwPy?c2E=RyEDUOuJ-<K@XSSpJbB9VOgT6I?Q6 z66NM;5mUrw$$WP;`JZs~7A!9cY7qt%k|PZYj97>JGbC8)p_7}C2~g*|ew3G?I}tKG z=Ggw;J$`a+soyq1ugFlNWS--g?`_K=8By?{w|_A?Ud*E7EfY32#dk+Z^>HVmRmi(g zOIgb9Xp^+~A@V2!B^xe*2Wu}AHCSpLE?}TXv_H0-{R<ufh#zT#UrAl?WU4GfQR}L# zo*sc-g1jA$c8m(%i0$ovrO%$!zbW}&?S>6F9jw192)+&bX3>1-W;>e|Uxf#92P-f- zcu0LEnpqlZ|6Xw$0Stx}U7wpBypfR4c)ax!cDfGhEIn({eyMn{yM9bCFUut3&fU+# z_M?O35xF0*S-dsV)P~o~&AhfHQk}o^Gcw_&XZ}6CN509Hx`D%V%neo^=;#YHoE1u* z9-zm(e|2V7qATD2d1baW0thC(V#j?^K+|j;2S+M8)7P(Cv8|b+t9w`7*b;9Vl=2{7 z@5|bJZI?M0k(X8qo%c^+<o8@W2e<i!Znfh6U>bKxVb}LtVUu<f<LX#S=%7oOYKFJ* z;2s*lrWa%0y|zcIOf^RUPqyGh&!xvlO;|M$K!+8`e|j=|v4ktDnZtAaENf!dnAaj> zT$$dk2~by>xOK#(8;C9C2%NRr0`=LIwZqC-X)`2|i}`A&_w+f*<8Y-HI-<q3+a9oO zJe!=qXq<Dr0J#&cBdqfd=vW<NRR2?k{lP}W<8O1Wa44u)fL}8{bq$W7w&QuF+Usg4 zuZTEuusL?%aN6f1D}H4BuaR5!p9}<G%M@dAS0B7auTbKy*zv-JkY?{sAb<k`se_8g zY;Pa-A%JIQ(g@%cOFF1lHORea52Ed)j!`Gu56GoD>)G=9c`QK|#s3;MM$-kq;yu$s z0O-BX5rA|i@8yrWYB>@aPewR?HX;0}PbmCzq|`F`IP4pxGe~k~+K3EMhT_iTYh5qU zKAclXEmMqxl&@E62;9USXT`p7HJF$>GMX3r^O<lb&9-RDP*s*{shhcv9Ft>8vnI0D zw1s?c0>#ZFAb`6nAQuoln|hBiQC=P>Q#s&>m$Evby2l!len+F2Sf|VA^k?fW-Xv33 z&Y<oD$ojoLWcq=T6<jyLry=giBeP*L5?@-LXbhvlpBn1ge-NIU$(|}7N%>dmGb`To zzd*bm7J$0c8VAW68yah&k_W!9r7#(9P#D48>JEMY<S04et{Le4vg?m_%rH$HJRQEe z%m}A>?L?hNZ578e3G}rWu99ZfwJi5~_M$LdA2%Dq9j`uIi5i1m%j}-}7s(ZMQ$C6P zU+q)ac?g7K(rV;2#mUg(O~kf6JA~H+>}P$Y@W1cx2Hthrh|<pH*`Uzf@QwPIjz4!2 zp%ByVb6=5+MVEj-GG(??U&6X5+?s}_=hBYU{q{~#N|K2X-=qEy9+i65Cd08iSy@D1 z8#g}($G~%6xJP=dJDoJvNgBE^g@_~<#Eetl=S|{j3HMvFY+CZVJ#_>U>MDWS@s)-{ znUr}&JJ<EbnhceK8ta9S#iYF?9$YeyH*0Q+YX%ZRBAK#{4U<B5)Mt(1{mr2dn@AGx zyX=wkN%`qwr3$!MdoMoCkzc|fnNR#a2XEIVsc%nR2{5r~f8mvBBD$NnNXuNAF$}%8 z&-9Wr$x3mG!~(Nn5iN}M)v%SHnM}i2t3p{%!PcN<k&3G`-2;oygv05w^=K&l7>y@V zJOKv0`CAgH&%H$t<%WD`4{+{XEFK>@nW`B3^JKl)9m5$?`VeT<j33vCWK!2~at-T> z;q`vsen5M2M$_<%RJX#E#(8P`j)br5e-7~O(fId<@+CSd;MIt}7l_E=4(DT>Q{-r8 zF>u=nMCak<;oaLyeo-U!>7eORr`*#Be^dz<%l@4o*vAFbu&SMbpz9E$J5p6!CfAE2 z@Bek~jWPV6^2S*D&BO0sD|~CepqQDtF%im0d$0Utwa`MtmJq=ET5$2sw&}W5(FiaT z{J79RU%%Hoi!*umQ=wm|y0d{&#soWjl#|$R{c|g0KB~3>BO`}^fPiFdoks8Ep!0HO zlk>V?fesBdSqXW9%$h#u_FNr2-`bO?e%#({Zq#~>qh|Q>9fP1?I7dfK&~QEy8Y@|4 z%B*ljTRW-i)S5wqsOuSQEEkR+q&xg`vI4YB?XTbMX2QEJv#xUoKCho=)`O(~>!{OY z{(&bt1AzmJ(odyLMT-?B8n5&;oUa_~idZk7D6mqU)arePnYm>)zk`>%iw2_wv^l`x z0WG&qZ;cWR<K{3ln==q&36-8D6VZ^`NpV;uVu%JRES-7xZyK&YZTNs}_i~<VeDB5Q z4_=nfby`sTR$rrmBtUyzb^|s$c$M(cFQ5~D3p$Wc`^^+9oyYW!l!l7pgIf999PCVP zGR)~O#MiJP)8lelBA9G>PyWrd`sm<G(cO0?ZHwRCza2U7e~Gfh-J<HPCg!9I<k*k4 zR(|<m4ap?g)2BJSEFhp%ImRkOtwqJDI?UXS8aO}Qn-B5DX_4v0-J9l7=DBu0H$JIW zG94Vf8fcYV?muR{Rhxc4SSi+T=1v*sJ{ME38ztBI?Uepe86%3In32LSZ0~BODuo7L zsA8u2Y?mKqXG=d)Xi&d;boAgoWZK0$h-S{v7{AXMlQ^An)R(`Eoa|+X_>$$Ff$t(t z!WgxMhjQzcxt=8&r169HPP$6G17j|90Vlh?8pOmF`IHRBgXu?i3t?SV&*#7c7mMka zx`*$}itkL%=Db9St~QCh>ncnY+GiPU(Ogn%yx4V@X?m1)0UXj*;g5lmXD=%Ka<~mo zcu^cO0_IHoF>ud9A=Mhg8_iTlpDf8LE7dGDZ9bkk>h(%(R#YJXno7$zahC-9?Gbgb zkKj3FjPBSsUpb8{U5R$H3vjdUE#=)c(SV0lXdI@1Gbwhe+TkLx67M<hhf)9CKKRrt z45_!3k~<fD*C!w?q%5`>D_jt)+}?~k-=akT_}VZh2eV^d(_m2%FjT@sfv2SOops<H zFVIDjn+7{81>!DI)i1tO!gfnxhr~h>NW|hZ-rxz#BJ3<9>Qcb~0la>^de{_ob8uog zb>jVy8Y(jy(D?iWl)FrO=OhN)UZJpyY3whwl;Pec4-dg36oo4eja@Y}_!EW0*4X@1 z&6)I?=3Af-Wd#-nD1aN{58tU1*cG-z1}$#4?xyxM=#?2g0yUTWx#$UhqIC9!7Xc7z z_Y^xuHcBud<(ziFz9&IguNdQfr-b9K@KUcF$Xmr-8*MHPZo|gpUEv&W;A68=-*D{D zwrQ`!6u39u4=0a1jnW>$VP}iL{Xk(m864lT_yCwa$ai|i3(h2!t7~Fs2I@E+?ZQ7T zKi6uEx%C*u0KoK}Yn7dstGV(<1qV})%OHnd@56-9_~@a7y424Yz{$|4AiGqCQ1x5- zhYbDX5p{nXrku86&|CmLBK7HauHM`J%&sV?=YQAVEa<Y6wN6Ui2E%;!f83+t3A-h= zW&qmQ-crZ>CNwv`@tvEqt-B=cj0tt(wEK~cZZ`V0DFpT}D=QC~<51;OG`Qu_=~DXp z=zPmh$Ao066#T|lwQNPT`hKQjwMpwITP!yCkU>9q9%1L$zn2W*tim!iUXZVm;gq9_ z=%P>Re--*&&RrBd?59|@Z;twsc3eSnp%Ty%NN!cn>-shvmO7+XpyYB_&+@c!OU?6h zWor8u1fVgr5$zwRm)Gifuf4GqKK`zDYRy=rc~aHYzCx`Sr$%*<W$*>y#HaR#o$i*K zXO;umer-v4(cAW^N*>%2Z*w98(nRHOdhC%1y|i%parK{rKNn-!n~X1ZU61Y;FV0`= zz*UbgIEUaQ+XlXmqvYqpDkdnUPwg^lPbc1-o0`xtk#t|3N8tA_%TLCsmQ~Zs5otpH z`7)I4nzQ}c+MN4&e%GS}J8<)J;%gQ3qqpLks0X5!(JOU!1&+~0)`f<jZ##peZUAbG z%`erW(TsO{l@<AD;G~R(1hm$^w7`#PFHn?J4~kihXWcN)Z@iYA5??XA^m3l7{OiHS zPTJOg`LG0&V()E+s#{oX^ZVB9$Pe}orB+E#Uk=vfJrQS4SCGky$2Dn)+~p}~2zN9d z<m_v7G<2?nqBXb^svIR-ydioRu5aFd+SRIpGDYz|z0&@thl7A&pKxnR`~ZdG;YUm6 z_<lVV)E9nS`6x~Oz{cW-O2<?R=joX&uDsYic}1H0_T~}uicdc3rK^y-M5sfkPC92m z&!ySy$(WB3<<a(>cUyI1u4ZlIg0xHEY`R`xLi9kQ`&+BPMR8(fTJkf$1-k>;vQHY< zl|PuX;um@c_%+&U_s*MEqmO#rDqcHgq{KP87W=vUq<w48X<IM(O<RdoiSE(CTMN|N zZ3FEt%FCUhFJBjK(a<>>+qBlT3aI&d+#cw8<E!~5-icsz?QKkin|~a4gDrBX9yg); z?8(g6SonRp`npLtxzwm^OYI@a3xDdhl#JeN3B&eek8u-5bXu{wHQJ=jXo>>SNG}cM z%1ph<Nez6aGQU}Of1Yxx5K3QFMyf2N&=5EG&1T+bbw!gg6({ejq8!P}NjN_rBLMm& z6>?7Wh8V+m^m$b6XHdR$%ZAdKTFSw%6taQb_v@AJylENaK}$&$7aQ@jXS}+OVJZLK z+AbMvga8b%+DCu1U*T={PQ7OwTdt{T%`}{}tZjRg@~&<IxzTb={&N@MH$ej)VPj)8 z+DirBTnA-hJH<!KAH(Q2kpnw6?{E$ZUuMEkP08y50#HFJzKc#@m+^BBwfo)v7ZL@N z{*IVsU%vC;dm7YEM#8(Joz9X9Kh_)GUlNzAMlRDZyf^($I2KLYt?ldPaYN)0K;q@c z$116_$D*^{DcmJy^E^kcMZKm!u9A!$eUMm0Ji0cpj;NqSK`R1++&S+Gw&Uj)>@$*+ z*GM6)>D-%!Bx*%CwIy^*Cw<CGC$1RVJ`_0M#_A=uXFaZ>0J9cY=$+`?k{D!e5_I|U zAf$hU$~y8rfIQLMFm)V6vZ=#z?Y%QBFI6p9Ht4vi!6{EqSCQv*neGJb!w>0^UeeIm zxqvq>_<-SOJ7Xv4ERiepOax5TS<B<0NTH$qr)8SpX<#XZeolr>y1gZ|A=#OryJ`Dg zwAJ+4AAV)sf&2fSTRXo0e}l8Sx)ro78G`eV&8~64z^|J_Yl+w{euA36_UUyZ<( z*`@^MTJTBD-*4pYeTYRuZOlLiM3;n>$qgc`p>T*3Uggf*RyEI<BaCa=Qj;?sxk;|Y z!ET;|LEbC>I5Mi&XVT=%V;f6{FMW=$0yEtEdgZF)mb`+g43!5brSZYl&&SR^R}B}H z4gREw1|y15Z&U;nmkG$m_~`}5s^@^?%1Yaz$$2={THQ=$wsX${$&s<F;BMmjkUKVE zQ%DNqd>Cu4vUTrkSCol)#SvS(q1OZ=9=ZZ0w~-c~AAQ5|-+1$OJ0sNe>$m3Yk=Tn@ z39G3ZNvToanSF(reQWm4eUe84bRJE*26s8G+s|sw`wIr;AQ9S-sjaCI5cl=px0J-= zK;;uT@RZsaZ{OKq=j5#7?a9g2+pnT~`_!i8<zUR4SUNY2p*?&Bn(~fd;m)lw<|H^> z(l?2)(FhUK+YRVdn2Yjee7&qo;_PpE7JF~=oojqqb3R62d#=yBe5M}Xr<U!!drV%D z(ArXX1oODeQNYh4nZx2eSjn>XNe!Uz;M||RxPq_FTO?a!>x#|wsIPJMANQ{X#Nym- z?b4aty}K~s_focH%FB{_Ga33x$)k)@BH5K)H(uEV^Q`lSfT$-?4r$J+dTYy!(3VSG ziwVn+u^Pj9je=W_Ig2RL^&?4-!GVsvOplP4JC^K5qUYHyXJu+8>MAH|>V~UH_jYkp zy_uA++^ipb^d*;kD><|4(M*#N*gR6;cTryi{?ESbp15QnTy8F3+%7up?&w?-j$>CG zO%(R3dc>RNilOEwxyk|i;<rDy<_aKN`)jE{^Ea^KKXvpgz1RJkx@u8gi2y1$%g0Ss z=REof?Y|x*Y&FbfXUsk1&KVYw6PZX?QoT?91?iogf(;6EIg*B>=gm9Wd@Sb46Y_$# zhRK4b%XPjHrRC-W@i?Uh_DuX})R5N$Zx%rY911t{Y|MD4a<K)sEgm=3Dzuk)7A%@! zKS_!JoO6^|)kw6M>c3N32=8Yw+I1#Dpoy4OgUQ1Z^Idp_b|Q`i_Ag~T-Cq;G=ckWV zc|p|mqu*~RQ}vZ?jynIn<Y@#@k$KI%t(*6<=8k^k(dB@%S1z{kb}-radW}jzc*xv@ z9ZuyDKmL>I_n7QS`H!BcPe_ZC?|MZMO$wE%kfo<{uRI9lA8^99y3DD&pYkm)(gM>r z6alOWR>9iw;8BGWXVOnA96!?q9w2~k%^#~e9aUB&n*TMx;9;N4@1TXhL{>DlN6`M@ z{0eK%y&$WM`{TfG<m`k-18bW``*xa~1-=RuD)Padl2_IW(o4_>mSYL3mlB6ZO5kx8 z-q|$X?ye*}0K5ri0s}GaB;Xkmpo{Hv9RZwWYzLNas5{Dm@BBO)mtl5!2J4M`S9J=y zt=ePY!np_l12q!?-1!Q>Y6oowW#MbF;|z_<oA59~xxBlNvp>VW?~D%fUSCPADtw;J z%`k;+<-=k$;a?`(H*0fGS2W-(p;cS?<)ff<GX(G=e-ZblQoq?g*q)$_yCbcE6YdB= z4ex9QcEv>DdUXoB`~ezLYHYK|s}uH8`(O;28cBrb)ZQQhur<AF5873QF>s-vY*^bb zsk%}PwfpD0tk-8??AeP~I~R64g5V)^>W!wgvN=UQrjAzD9IDS~UN!ZIaEh6~V0b3v z&JcEAo?aL71fWJ0TPcBc$`pbYk}s$;Xc%=*#K6?=u)Ks{4S+Y}dr{y!%zltwf^N~T zU2q|Ptm`e<{xSlfczxYg1RDjL*bO!j&1_P^F!zsM`lDkkf&?a3KtB>a!GH_gifhw7 z=B@lX>+27Ucic28jJ4{d2gFXw+s_n*zMk*>Z8Ws1w));oz`mwSon<&CJE?f4Y!ncu z{zq~&(g|%Gbc6pwe{KIsmFazyex4I*hYghYj(vi%w9m4)vQ`<htn~Ga+=bp`*y{W{ z#5n8Qf0-Enjw1RUkM;k0(3tSAD2hLW9Atle4}XTy>HG?Y_*+DU@J(ow-JjubzbpOQ IjzE0=A06=;dH?_b diff --git a/doc/modules/cassandra/assets/images/Figure_2_read_repair.jpg b/doc/modules/cassandra/assets/images/Figure_2_read_repair.jpg index 29a912b49693db7b87a243266e8a66df4ef8b327..6dcfd0109dc78ab577ff41ddeb43543477bb490a 100644 GIT binary patch literal 60618 zcmeFY2~<;A7$z9p5D=p*vI(g|0Rb^CAj*=IQh-1KvTs64kzI_4NR*hA3;U811WL9P zo9sj&fU+b)2$ZNGI|LylQFi1-B|gc*yqccwp6N4vdd{3VbNY05;2j`59`C>J-ur*w z_kaJ^<klQ)pS_Kp4NOW(3U&edfNf2~tp5G!zxmp>O-c&-l-l+mU;p{LZU0{6|8#-> z%W41d^#A79tyi%9yQTizVYppN8@6q~)b{;STRkuY3?{YnKU^F3f4ro&ZQrqTm-KEK zSvly0=6$ejQrow0+p&G;&K*0Tv+qH_hwa$EQ{m?m=XNQ&T$a|3QaXA6aq;d$=Nr0| zU59|fr>@+5AR~J~MO95*M_2ENzJamHY11=j&8&X4wz0inXYc0j;pye=;~RK2C^#fE zEIj&F%<Wj*ox2Yo#U~^tC8uO&J;~0={p)F7NoiU6vx>^9>c%E&b4%-smu=mQ9%gUf zKd<|TM@GlSCnl#@94_zUr}>4&rDgt?uirKV-@zZ7|HdT++y0-1>^}tdU&OT^ifh}B z9ou(E{~MRowpi%czJJHgpHJ*kIOigLIZ9Fc<o(@B=N}h0bjch#<q9ZYxj7_z;IJ`A zhyQPA|9fQr&jNe!|0=To6xe@`iv`=WT?%@5+xNrJFp1<yNeb+L9sg?#{(GDQqSxYb z`dlLs_-%diD^Isrh?Weqn~4u;3Zj7F+75J8cmSL)9WOaN^CqYQd^VtIb-wfE_$~BK z|D_?{!JsIm_;>S@PskR%9)AdVoLD9%fvnLX#UZ88rSbPTh!li8KCI$c9S5xmD)y*{ zq*xeoH7(v4wO?BDoh3yZb+XDw6&VIKISCPWe$ICFo>BSX4>k#X@z=!ZTd=u?Eg0ZU zNkhw^uduZUQ|jn=dLUwQoyk@(9^8Sy2>JnwIBABKEP(DY3itQ&rSd<Qiz~P_hV6(o zydM~GVIoGgm!!L9DO$AR`<nJ5(%8UnY*0Jo6<tjv$M1{Ba4SSgg96_BW_n`LVQ}YM z>J}`Owo9CM3prido7dNq@Se6)WXa!~8?gnGnXlo{w=J3GsGoj(Cood8FL}czxi%<6 z*Y4%Xg5R3<w5jGjp#;=jaC!LXN#I7UQ*2WI4OLqw!bZbI#+4IDjkN<>somqHt_C+B z`m3&**q<#ts-+e{Zj4!YTad@N{Ka0a1+7%Ev!7OFVJ(Yr|Cpm)7Udq{b~Ej2xj1DA zGeB8Y<~s}i0my5EC3x+8pDoyfzwmM541$hm55K6DJD9`<E)u8mZ9xnd`GAfPX;y}L z6(Lf)$@@F>fu`=~;YD(M(*(T<qk+e9L8g<)Z>cuT9hnX<v*{-~l=&oy?1{K7*p7Jc z%@%B@?{nz#S(5j9T>9z|KaTggibK4=xdlr=$%NCJ*3(dG_@Bh3F@w_KC@JvqvkxL9 zQ2}iC9ql+9b`Kj_O;+$DhV+cz9&d?ya~)gO;dDy+VfZuTzUQDl&H&upD3{G)aBfU4 z{(eF^{M@;*%nH}C2&U_!PhPTPzEMv;gjp=Q)TIcXGe!h2;!_!OVIxk4Eba+!gBQyu z(x%;qKb9@JKPfA9R=j)t(&KCo@dd&pyS1VVKIz90ppJv<oGlnB8g~o}2WyzV6X_Bq z!T?5{W&$E#6=?!j@8UNF4g6C~q8whi+8b{=QrbM~8ah8-f3&nn!@TNpNX9Uybd&rl zzy#QW^-#9skX+M8_0Kv13Q45LVXK3|0`54sH;xWn-nMDtmS5Fs{o|mfJjc_LDVDzS z)%1eW%HORCL0hmc4+~HH9cWO!#V_$Uua%9!)1YZhrmI?b-kNd>8W~KY?FrB8n=C&& ze}OjjskI}RmLV>(P`k%hk;p6n(y#7@WQZaFx(lDUN(6|E7W96a#>`@i77DPOt%2US zeOsar#4IFzvvlg7?z!#S-^8-~To@Owdmk_K;mv5=h0FWKqkbw+e)cKjb<OWfl}x5C zA9>nCzw?sc!&k0p!DS^sp0S!P(d_%WX{HI|>R~}Y=2Z~>{VrLVo3)KW_Tl5ykb}Lo z71Gkk+j*{@xu#8cV{r*#A07if<o9tx7=ubP5wmB+1r~mP<5YOM*?RH#z!DbDjTxW^ zG$*DTN9s;vCPe6n3S70Fg-LbT_mg&|{>%HO?^S8HWvWfu225cR5rEygBL3zuI-CL; zEI{)uM34=R2^~R<;1W0^v;ff@=TwRLGHUObMfBxj+}F9i+5AJ8W?DO-_t03=EchM~ zQ?M)1@sA$|ZZ8XG)NOxk!Co~=J`e#nbSfEeWvk$k;!eT|NpH>;tYKo!l)qhICC;WP zij07TuBQaeOW*<r{;<QchAr|Emu2;B!Q?FLYDB*Q3rTo?uA^nHBlX4b-;G1xmJYS3 zC=8yCdUA(Qr>d9aY`_R^4%yv$&C=c{*7AZIVGCwlGw&Vh#?W((5){<ec{q8LFaBQ8 z@FeQ5CplW#dq(zWQ$mOBJC8b{(gM#~zrJa3=eBm)3hTs^u&)O03$98k-$>Sk!-QFc zD$P`6F3zW4*@7jaRq5edumhr<;&RkUFp(iq0&_V;P><53kFpfSh|5Kh9Cl*vT{cN1 zFVyiDT>ykEkv{OxSY-J)rNe`Zly}BH3p*X<^WjZ}Myc`aWoSl*i7Sbr^dtlo+5uw) zcHlI3on_lAQ4nXFkf+E!2<Z*%v|cw^*5ZWFZAl9O{s4r`0Nn|^5#I*;KyZ6gC8^G6 zgS1DkPV0xZ-_xTePkbKPH~5{aN5GFd;+NC)d!t@=mc}S{8t}_4l+X63<xpn(T*H-> z&8m+VEbO+y#rnVVNKlkX&6cUW66R8Ord|JiML-8Y{p4x%Y-0SU%6|9Gva`$9Hf{nO zxn>LY%NPfvxdjVbpPCS0fHiUoJ(Qfz-h+}`yj%Y9?)BstzDDm|mG}j|R^6!`4{ZQi zVb!e8M2g8v%|BA!I48_6pN~3tG-4*=$Ez*a!_+NU5MruvL2wWh3Vks6{a|#pWcc|C zrZMEfH+DK}BB9oVl0?%2>I;Pz@h7Mc&jYzh8=>?0T>^LnWyR&gvz%*^DXof={!hpL zsSUFp?F%{+JQ^0*@ZvWUJM0mLVp7Gq{{2Jt;U0(h#w8T=J*Hb><!sT<b>Zi;?YbgW z&YXK4ZGO7+>U3iywe&MP4?VQp*j7{3-<II^;AE|o^p2=5rR4_`63e<R*vcdHrcwCa z!*#m3HgCfmN6VeCwr7~_k+-vCkCZ(A+r?T5_CLuX^+n}A@e2Ks1SYPdhaso+9LGf1 zYS9I86-o<#2n^zDbZ<(EIqmGk1E_<bNnribtOjvUXp2Pofn;3rdAi4x;cCiyR=h)a z=m6kb8AB|<n?eiWzBrTWJW20GKR?|N#rNyof;APj@~xN%WeH-Dc9{Q+Tme=H?oGrD zs=wx|Fj&}ESaqBR!`B3wNVl+i!11`6^}QyeZ8^oE*x=?4I=%669ckHLIqmqjpRJbP zmzCOPGp)m}@|`tqeNW@1K55ujnOA6*nN@M*tNo^Zo!yv6g{i^g0=?LAcYyAiK0b_b z7x$Emq%D;_E5ACiu)g|IC{?jK!OP!*C2JiPLzw6~nbpc90*Qlj=)K4_YySKMlW+ha zB^<kcYvs-2H+te0Y%gAo6S>`wuQR%ddUD-lCfaeVxk>QTbIhwur$z|^%1mdWM3iVs zRv=!LL+p@DWA>vX$q5prabg%L1HB)mCt)P>AO1Fk4slLZ528amq7vB0X({JnccJ6g zHzLu$j8ZR%42gH1g$DdQL=~TpJA%d5(y6S^T<nT$%JBY=Z}o$ZIe9z>?_6H0xPBoz z@c85wEaFXseM!&dQkS#-adrbuPL!iO`PN5O4@obb>a~ls9Z#3v^}gh!a6FBIvzodR zq+ylad;F0$Z*M4l06s00$TV<};1M)5><P0-aTWR+X=;H}NO_14ff6+niW(h6ObL(y zW9CSyC<T6JeRc(Vk**{{bJyc(hZ1|nPqTubn$$gUN%iKoj#C03>nc=a!&eueSuoow z&L@UoQqfI>heE7`y#?EE?3^3`Zv<?iKgD~2?76}HB5yI-*dN(FK~Ngnf^mxQwk0Cn zk)s{kV?siru$B!kMof<I+J@^6_6}KnY+W)^cKII4J7V4JaNM)Do=yG!$tBvfZgj!) zKwi{BUaI{~=d*4V29pM#<yfECuflXz*LRPE>-*g7-}m5&Z&cPJox`7Yn}DT4B(X~Y zzh7uT7`jVdJHbJNM{Gq(-?`Fh_{&_ebT3=Q;soDM5Dx}&;Zpe6aXgX}ILF=#Pup}( zlt?Xk(TtlLfiJA->4eeWGmS^)&LMN+HE7^hDC!tcpI^!y?8T-Kl*A=OXgu>q1o?^f zse*FQUwBFs_+GRVXcr`se;YmojIvrYj2+`%slQuK?69uH*{D*}y<7lVDM$G3{Hx<d zfwy}<(=Q#Vs#5hxSM{h+e_kx(niizme=LpTRIun|Wa0)t;!>)TSXpje?Y;$z{cJPl zWRO<llzOWo!2V{`btKCm+n~B3tU|WG+&4?`KC0%OE0#PPD;g3Pqvx{GP1rtoBC(Nj zAFo+2@)PGYlO`Wr;EA6|ju76`j*sAf<{z4yPm?IL0H-OOdm66%ibRWZfwdsk9ca!V zhgF}0z{l`xU8Y4=vQ|hgMsN~BxM6rbaWd^V;XPZkLyeyb!45kAl%Q-FrRXgR<|EeN zd_x;Bm&mmYoD&psEe#p7M_Wd(00HDX<;&j>p!Db|^8*^$@7MlrX>J~(rt1#Rnn!IS zt+DDGrCusNp7}MTY&*}aCuPg_t%s`TKhB72t6ZyGBF(5(o|!RW-BIdJ2?)8X&nJCv zEsi`GK3;CqM6s7Oc$1Uu<8*5aw!o%V7l)2jmU+EFv?Q*k0@H#;2p(P*8sq)OMNu5~ zHr#|au(!2(nZ38!NcsXlvXjlAAixmr>0~W3k&BfN7ex8JsRO;7QWotl8(Nl)97gVV zo7J~rsa)WfZf|mB%?<8a=#6TqvX82AiWpG-Sq*8N;#e~^{qzjoPVWFMtof^&`flEZ z37tUeUz|>!|D*Jb<_-OCE_UrLv#WnGZ46F$rg>z3IcKeO@pwtf5!nA!2km!p9XbLV zk7(%YUI%^9idQF03T-b<7(**dbPV*Hi#%&##rNWcFy(RfG*!?asOiRpC5x=N!XKBP zJ|8|8bU7p<raHX!wwZtSr&UDO-HL7mt5h)Zf~ZQEK`7z905T_|O8ZhM`nmJ!7~jwZ z=wqa7ocq%%YFDn%7JnPKmy^pSvE+J#z2_vu4I?I+@K=wi@W8#Kx@qIK?C-awk1>N4 zqZo`C5>u}%aCu|mwXP`RXfRs-{)6m0ELWDH588VA-P9d&JyB^3c8xd%p9?OJEyYKQ ztBJu-erxNDr?b|nloaAYf~p8LT3dHhWzF<9%7|}W&>|2%wfTPW<?pB7d4L#UxTKeT z&?M6Q7tl!PD|t&-*_a7XyKY>l!XfsN<BQN^%?>zSzdH*f)O8uK`g6pjD`!Zv#Kz;c z?1d{k33$;NF@(ylq8}YNz=~zSrykBB?$@n_m?k3=35qxk=TTky?_C?_GrpFJYrYxP z!u+>wFY^47<rKa&=sp9}g&LCAXcg+pj3GC1wGWW%8^8M~*>V`H;UMmtq&|#Mx^e_r zOzU``7w2ey*kj)ZhPLxw=7|gO=d`|W)_6gz;1UUa-8N!#b_;f&ORW9&vTnof=J$Co zVI8(doL538&zzbISwjjBcAyN!_X)?q#2%XT(B~qdw>X{{hDp-;iBEY&QC<w<<z9S` zk`a0Fjan$&!Kqy4Z*`S#%73Wj4L%qad_F_L8||zy8C50kfzodkyOq60!t(ObQ&@qW zxR4l<I+&nkGKM-RawJY8d6ZPVKi~Wn%5-=~@VeCmJp_GUva2O$>S{|KYYX;UTRXp( z98tIAC1B;`AI*u~=dAY_qx1I2L^?chEy(=Kixri~joZy_6(L_NebsumHLH0FW?!HG z(C=%9iBw)+u5Y_ew4!>=d4u{8;ol3AqD|KOh@gQNQ&YBg&7v1TuHFIU=1f<=bMB$; z>|JPi$iyh%Pw^4ViI$|Yl7B4DMHoI(kdn(ZM6F0<Mu1YLy%$fV4?dm6TUNL<2_?%m zdiWiY@m2htdc4dWHbotg_|Fa4Ei<ntUB67lUy5<#boAf8d9jo@pAE(H%AI~Hr()kH zAkuoE6%53R^?StAeA5iP;@pc8axdyQ(E5n;)TcPVXZEMzTdL*m9G1R`!;42La95eW zyTDAWU{aiJam*j@AoP>GEz&&k@q!7PgpvK0`6y<u30%3aRcqsi<=@0EBjQ{^fM%Y< zKg27DsjPs}PeFHL2DR`%0kJ*w{cp674IQN&1{2GdbH6yW{`B(2(dgGEtLo_0ARz)5 z8zMTi1#2X&P88q4eSx^7WTyx@ayszYU<1zQG_S?5(ZUe3#+~cbW-kP0UUIZuy?j-C zHTP5W%31ncC8V+m<Cr5AP~J%<biWUW7v?Au#<yUwD;@XBc1=SRZ{PM785k!n<(sCW zU|F0b`t2ePE9ezxi+%~3{*<mLcmKQ;Zme(*Vt=J;brpk_4Y2m^4Qd)JvC20$1% zQ2t?HOd4t%P<uO)y05)1%Wn4#nZ7q4Cgf`g{}3BvlUlc6-RFeI2?Lw!Psyi*fpZm% zNN2*`bs;u^9P2ht5xW;sd!IoMV?y%b^(E)MFHYxz6I(EC(im0_?+ixpk}b$P@m7Fo zn(hf|H&PDQ`U`(Yf{t3)?J%=PeQ&nc1a2R1&lBzABSeaDiOg$mIhw_OUa^|1IsWaj ziNDI63In+(0(h-vbTDUD8yHMjL$3MzB)e&L|Jiq=YrY{8J(tIBV#mKCKGHIQV1CG( zc&R}2Hew{5HTX*XyiG?L0*Id$UCw8G@zaMTk?leU-mBUFu}hJ2?Fx88X<_9p-6nFc zPPeJBU-{>yH^?!>dq-tZ8M(S3yasBlen&_V=c?JwyGT0i3KC6m8R4L0U<=lact8m} zC(Ms)9i^#(SZ2)u;<yFscuFjc@WY8a^DJHH1|eoYiNcz55=0Lut9OK#7cX+R*NI>t zg-4%?;9hF5CVl%3hVYv%@Ue*|a#M#ec{=?0r||rr4TgT*HbM5&e!Do+fu^Rc>?#M5 zae(<!$eqYYRKqGetOC*dfu1aUXiW$J(GuA%;K1xmptu61AtH(sHjb40MF_WGtT;}; zH~w(r3+b!F9|*&2*=C9o-emaO*Pl?o=2fH`zpfyCPrbIyzCn@!qI(H&FrL$C*o%_l zha_}FaWrpFVU3-YacKWxkprhQcqrMUZMgZ}f#o>=^OKE#YN`3#N#}i8cB`_#lpWSz zkZY$){;)b|udF|IX>{494C76mn9ayZIZ}D_zef@K|J!@&*e-Z1uA+yqrxZCK=ulb= zWv2wFgZ^_Pm41tD{=LmcWAm+Cx<WurOCblj4QFN3{Blf0<QWz#J)5{NbPd~V_*jm5 zy1w$0qd>kDeF(2v<t+3i%wnjCYp1xW@nZyp<R98ue(S1(P%qS}M)zrP1VQ~d@VA&k zL-y^c(S~a*(^cx$J9g#%`TAf>8?vh=^~AwIkI@}luv-)JM3F*t-`&_A=cz!ZMCYL+ z{}`!$8Qp?j+o;9$+4Fnj?{Ub8$q2~_F_m@}lnk)@W)b5qCYlH)uRGuv(ACj0%FEc? z_C~3y;ZU&nIA!EZ-tpqx%ZGMe$JyNc{ONE$J+G7Ii8g%~KG1SG<w@a>U@o1Z5`@*w zmc&AeL#N`Ptmv({`dgFmv}6R+w6I3uoc7mau=iO+&LJR*CwMu+Q(%<gYB+2rd7mSP zIJLrcs()UUY=PmxS<*cOh2co#ykj4Ou3|E+_Dn`IaSF<1D_`>^vaO$-g-T$nEyk!B ztB4O3|Jh;43DpS8A*zCj0vf1X4!W%liN6;a-RAc4PF(TMd^c}nrSew?Sa-xdZ`eK8 z0uyHI@trl;78zs8hzKY^Fgz+vKIw&gzus<4w)z&Fa-{y@e~D0~>XWsNZyL0jHJX_v z*xKy$@$tU=<iR=V2l9ApE*8{duWrGR$tLI|#4ZgkVl|%c<@Lg}t)gdr@_S!i>nq2I z0GdVfdWOZ%oOtYRk$YjmukWBr0XAV{G<w_h^D4>$v2u2}Qq=yB6nm3EmL0lHh))mm ztUG_{TX_GIc4qUB>-Z>0txlr@?v$xV$N5qqB+l(L0{wz#1KWIYAp&=C!>yH5RI};V zSJ50V{FRQBBMyh1vB1#BxCHs=Q#+;CA<17ND<d=^jOO*sZUZWkP1uPD&2OE@jQp6G zCcQq4JX>@~Tw<XgKBT7C@e;&xA7p|xj3H%%m@Sz6U_I`7^uw^)52c1hs17e)?CKI= zE68|%)WV!!!RbpB#YB{_k3odW^^19(DRXxBMBoW3>M(eZS0_6^;S1j5Cb5pZ^({&z zUM3#s`t;!lrLgzLfO_@Ku6LXTQT>;PTd?WUl{<H}1d2&~C02%kYzO+O@2M@=N6d## z=LrKtfWeR)5_b>-gC8vXI}$PL;1QKoVa1;&h=&OE81Z%+alJ^R)V4KFq%AJN#%E;n z>o4qIxH^P7WE)u<kmTaW)&A#z8qp3C^`QTS7Tgn4jPKoCnS)9>pV6G-fV+Sd%Y;9q zogIon!|*pibp55i&RfV7jI4#*PFe%U@L9~$$olfud#Btks4rZgj~?y6gnrFCDGJzi z=|zf@{+doLrYwi77cztJc+_{f;rT+bi(MV}+mm~qTQEzq%{@qe^V&bI<YEo`AMWrB zcv&A>q&gHsRKVqW4~r~0Z1_^{=4>ZrB`^M2R>@-C>Ffw2K}F5-W}RrL>)BGy_(f;q z?}`^5TPYzAXH=GvAiXsiLmliTg8Cb4-~XTok(Iy+0TIGLc&71YED0T9316F404Ie0 z%!4FsC1@yc$@#o{9i@)n^Z3`t%NT{(GtK_7QB>f!+X4q<Qu!j|as<!NCPC}q*W3@e zp6p-SB^4+=MOztrkD9?uxpi!#KW3xh8+}a!T1X~^9rw7Tsa(M}Kr2<~Nf;?oZ_ly< zQ@Qyw!RvP7C-k5h$5*Lu#l<M2(1zb)Flv6U_3~&^z;im^|DVP@b3O+=%_hl$CPIyK z#drq{D@2e7l4>R6=$4)DvS-rJ>K1Y3tT2Ka=u*1uQX5;eOZ-%#=<9eo8_;_-dogtW z^e!G|tYhif#Kw%PT|Ragy_>KX;)c?|Is+j?+XJdF$nuhdpt&>25m<lJVN}sqv6OQh z`Of&u+S|O0s;IM>ac`!pbUQj<7Wvh@UU->fhZ&AozxjbAfB#brn$7iM#aVB`qKIpQ z0pp#dq_@!pHYWWA8l{O6j}Cgz7R<V$e&JJn6$QC!B)keX^$?7*jR&X91p<Cz!iFJ5 z-SH!95wV>ZDF>GYv7DJ_g*=Uy*F7yYx)6#Y6%JC4QL%iiL=LMC{xa$Rj~+khKBCIw z9~=Y_wRVFwa}WkG1+zH*a(!Cc`1PsxIP3$UZO>+83*~R{;jg2`5pgr7?jw{#p`Rm* zbS?I0|E?+9Rs$cHG36lE>V)cu)xjyc`!MmE^Hiy`p*UV>DZ0-8^c`YUi1^J5-hZbu z5W9(?>k5u{%0&)9d|r;(^;F*EwPp=JoWdK4Voqzu?vii0r#1WE#~iV1{an*8-`@PD zqhvbr=B!=&%`a;UG~cCwaOINBi3{%xb?W==Lf5YjcdpZy<&}T*!9_|WhWf4*Y#Ol< zEmGP(N2>~<uAIO~{y{6DOu$(C_>)MJ?Yu46ly6^O+s^rHMMKl42TDS4{GVFX3>sTS zN?BcIo5twf(7r>NUgv3*->79BR>!P{@yVP6@%6<!ue@yKe_uQGH>4J~90<OqANS64 zX7qZLqCuAPiZ}7cCS^JRE0X5^@Fv%Iv8PZ0n>Qg+h=C6X(AFzoMm#5q#V_gCNKzoN z%haJ-@`nE0xkqk23A0}m6$EGDEg{YCpN97v>$`Cp9F+u<KX&-ivVawPJ~9b!#eKSV zHP4W!JcGm7ANF6$qk4YdKqR7DP1h2%hb5+%NeP48RQC;{Fx^%_DzsJh?BeIHUFIWf zfBlk}Xc58x-hGNuj8}M%JbY3^h~07eWt~I#0Xb*YC5rJh`WHwHI04=RFwlPeNTMt< z01^`C(20ayLu*uoEY5+E6Bbw&%JOfS_x?Tp=J4R*FODY6o;n@o*t@r_rYGM-88^1! zCli(b;hZ-zeVg}$6;@W-mcD@>zy6`4qpqReSB4g3_RjdF|LFBv_Q|3C35T4e0MUe} zdE1eaHm4i^HIa$(RAMISC(5JFhD$X0Uq3)avXDM9jG-+u5|`2TEmzGArnC2|+gbPI zn0cj?H0PidaHqLZTQEh0+=`&BHGbn(Fv)%B^MnTBuac5FgP3B2nBS}2a+Yc;#DtwA zRjyBXu1{vp_5~N_q3(hn(Dt$$OcYu~+*RpX2lXwq)PS*~s(T?&DGt#Z`+)iyZ!R>& zJC7c#FMU2lMW=6Cj;(cxvkAunEL#3)ST>DC?H;e6$^DhDT)5VwrSVW;q<Ppn^R$<( z$E{nx|Fih&WMgB&$v@BSxN(j$fY=Ad{ldQ<UyH*q?`n~_P~9QXv2&;y(!9Zkzcr5E zGK#hxt@}tFHLeT10Li>;w~>(M9PjxDp|kpdj~|ac`IGPykn@sHmdGUDL=sdNi_ZQl zsuuOnhNPfF#3!1a)kQbOq}tqklCtETU92%g`jk=Irn}WP;0a$GPB(DC%R<{fKE7;B z!#U9mO*lU;y7m0-R%I8?=_sMMkK5e_GuLZSjH834K7H^sszIlcAdMVGG`uHK`BXxr zfAe~Z*8vBCQXZWFmlcJ9XZh0I^!+p?aM~KP55)8p9T=-?M(kbA^9*uGj$b|_)L}OF zs9X&28_<5)(AHEm{zdiJ?|Fy8_C7Y?MFMsE7!*Y|Xy_uYr>WxldezSbL8VCmqG%YF z3N)CHuZH4DN}*dYDJopy67I^!f%pTK=gq^^`b(;UFA^^6vh4h6eZ!|S+i%-0ZYclo zBm=(dpzhd<uNIE{KBxvHe{}5ri!qupwCtKGuinY_e)yBl@+BvsPT=L<uTtxjE=;R$ z&T|b-{#0e%Sav7e$7_Q;z)$c8P!?bT?&rsHNWJ78j;2$kd}23A3+Km0B=|18PML$0 zj_WuF_x2W*^D@QJ`=6NAR@?t!eb&b1_LEDPyyrG9nH^WlItm9}OHvGM{zt$Q%J;bx zWnADZnkpZjj*sAKs&ogyft=JTFpG)O=j&DkmEhtSMOs5gpN+<(M_(E@Q7dg7{+92L zn&rl&CV#w8oN^%3#>3U-!-zFjsZ~XxZ^u5Hua3q84c2>H^zZ28*#2azUeXYJ7$$X? zOE27lY2ez3pkZQ9e&_iwpCJ|VK>%KpPnoii=9q^B4MSV(-^2rWx!Ojy$cg{<52-rI zYm^#4_^;mqT2lF{MRZ+g6OQC=dRP8ga#+mR!>;i@uMDk}j`A98XkRTkQ{<mHRxsJ& ze^6P*t5!U@m^tgXnwu|Do*q)}aN!~$zS!m8;aGud>dkZNoLdKV*5gf5{X;mm-H3fY z;p)=KXzabX3+)b_Dm>fXwQYhk^|we<S6G7GNDGa*_DNno^NTf>OH`UZ_Lp(GCf&9! zGV)qvWNmHT-%X884>bH79J~Wto12?k(=WHSw6w;bKez4Iq{@_S)s-GFn85|!g~!^- z3TH~KGtTfXCaeD>o0RnT&R_n9!Cr0OM#hxQ9alQJX^^da2c6=~&H5~AH_H6lPAm#+ z7;D%l^sOJl?C~7(&n3sU6Fe`Jq!?0p5as*h|MoB};1kpEhX5hNcQD-|j!W5#JJdC+ z*_h?Zmrk|7)Q1(R1LN@(zj_4G$75!Kmo@y4z4jS3Zt72xgLz-|8erwFF@-wbAlwZk z2lD?#{;5#CmA+4+fj1bp@CSGhxQ;I@4ue)xJCS2)&Y4fF3P;0BMoXSM+F%ZX_SWrB zGy7+!{U3b(JTDh~GD+~{#|%0Q5n5>SCMIsIvdY!7a|?FI)4mYBP&s<&j&aIIG7^30 z<M)lG)|Xm$R$5=~SU`Uwb*wg{YD;Lxz+7RpNC$N0QCQf2@8ugtV=A28II;S=oG>Oq z()>+rbM5$8hNgq$fRY>XzLtE{=Vqhuey9vMULF~zJOfn~9jkqI&*z-={S^!pprLtJ zA^KHZN-)8rz$hXIF-7mC9R#O<@63RcKwlC%$T>aWFep@6-QcE6GjdP{!!#Ais6=~2 zBpq42cz3qE`~$m7tN0lv>8G)8cDE9mvOgOgD_=-3hr5<74rNhXl@?szr^dQ}N_{pO zFq}($b!F+uGj?;yQoGZwQdIIBU#Uvy`&{7uqg3N{SyuK@qlorSrBL3D=Bu-#D>g*g ze9{{84&@=?1pYX9zXtsJ3lN)r%Q4yq4CGDGRKTx}t%aO+V1cyuP4f^_0rdT1_@eYw zcpzg_kac?pD%{xXkDAPS6~5jad`&~>D?)?qz_G7_RX%Zz1ibM9n|Kc7M<`1GMF*S( z_I$lnt4h2pe{{3=DZuVVY)2i#?;5M`h?%oQY2MX&=rm^GQ5D|mTb)!gp4sc#$e5?? z2EBy)q19yeuTqZD+90N3WO=R5=@Kbudvz9E0O>qzB1V}WMoLNNn>tYW6%1sCmJE20 z@8CIXf#FWmwVYJC+OnBtUhk?LBe=G|2T_0i*&B~(rTj5`qMP^0^TGG5nms{hTzkX; zE?yODS~8=RRc4P1%Dv=+UvgwOJ-e#in^ddp>kYTddT>isR+ld?dlo6OnAGZ<wGJfu z>JMcD?;KXa!9cN&%Y|AskAz=-SKZ}l14~Q)XHMDT%kCYw4jDI3-q6i+G_Poa7Sa89 z9DPlS|DDsfIt0)!flYkW7R=@0x#5rz<3gl0Ajpp8BGqxcUag;LiMBu@V?9x#G?HE4 z;lt~U2xx0+7%^UnL0U40b3^Nf!?=PJ(a4JtZE+cKu8fsUPI}yoNWcKjg9)fT6L@pZ z^lP~#_1rZVp{*}2VHt7Ixx(h1rl1bOdiU?jH`B6p-p_=X_L)!Td-w2}(OM=IM{Eob zJ(Z(@_p)LmJ87yFhW-KLZ7=JN*tlkqfu;H|zuWbh4$U8j=?=34;}&vr7?I)?Y-%1P z^Q5txpJmsIP6Eg^1UCYawE116NaEP`kkv!cO-_aNa(8|wG3nIYMEogk(l=5DTXVz6 zc*&7VZe&g3%;tqoF41LR%I9JEyO@c=9k}hU$a^}@UY=V|kjMhLjP>117aGC{%7AUc zH=+y}?U){zp`5|6dVj5M?b>_mJ@z}Ij>suga(>Z_R5m_NJ^or~5aty5(*8}J<MMQv z5!+E=>M+%yMYogS;aYm*f&Gd0Pbt2Kv#=Fjw>Mk5X+MM6!d-an3A|S<KXuMpKo=qi zBZf*hm)KO()zNVjjDMOE7o^AfHjPZvl5c__*Q02`Bfz`!0%5EKH|Q|77W{{uGBxL9 z+lM~EhH^}$OV7EHI!BL(OFBd_WWHHDJDjRp?NYh)F2;!sth`=~LO0~Dz2s3;@V|}{ z10l&m!v>)L0huG{@{4y#DIFSIXK9~hU#~6-Yl0K-*s{5Efr)+Kvf}kUdJ&~JqAuu0 zsDj#`H@~_8pO_y^$6I}$-H(f?q%3U^b`iU*@NjP2`qa7?xCH!tFHw*z{_}c>xRM?s zzB{`{UHQn*?Vizd8SAi~v%`Ao`E;1@kcng<2ekESd}(*>`fA6O`T3)aJ<lZ{Aex%U zSx=?!k!Xs%fwU$^j&rhPCqU{!?{U15s@XHE#it~G+kz>CHV&#c?VgV`b!$0VoV4e{ zjt$B{Q^Cui$$I<9Hf1kkWZQ&y>-Zogm{K=L?>{9?Vw`pNth4*lKW%xetR2&OYxCfG zABlvJAqJx#5lq2#{zR`Q5e|8f&y4&$j*sLSooD!ru6~n0dGGw7L&_I!A7e9S=2K6w zL^5n~4r>TEkCNJc{0CeBsq_zCKx_#e$<}xS3<?6o^$<I80n>7Fy#$hN3HRc<Q76Z0 zYmMH`m&S6~d%*9%4zJ@Kv5}z^qc=<MXN;&W)U3aTd=dxPz>O`Ks5n+2g33MC&xp(2 zKoH^D5^+DuOv0uEE<{k*c6A*x4EUr9_Ci?s3R-57+*<UQ%T6XLi=_B^+=Vr1;4`B; z79(>QknXxe*mJt7Buuie9Wtm@T*9?YT;yvnJ6w(oI^1Rv{8JOoCnzLsfbSD@ff8iz zy}Lr?^^uVtx3@LC6USWV(>?BYe;SLzcsxkI_M84#yna>L-KjjBMP5#z^;yr%K6CWl zlU0d#B9H27+MLcdO=kx0>lOz)x!?F&WfK$FKeGj^y=w?=@+a16pwid`nTixCxnj2A zBcS2>3$Fq}qN{Y$EK2+r#AMfDgl>8xIv&Eg2IM~UZew%B8gN>W%||9nlq0@tnRiZD zMBcg1V>sOVn^CbCbuRf5+wR1s3Xi>JD7+`~qC(f)4Ia7#X7jNEI5;E>0ZH7}hv<kp zjqV~jptq1GZ|DP%UrpCV^yG{aWWg#<p(n#?G~e%iv*9}h2Qf-ud7`;JVz4Qo%y7?` z6NTCxG+C~qk~h|0HT0;yASXKZ0okM=^19EJ%fzrmk1%~^ztf2cMH}zm2<B4$ta?8O z<0_xJJ6g^=kNTS~=tsLT-pth*c=%>ks16WEm^1xTtPe9Cu?ppN9TQ09x{lUZ*V_Hm zI?d+V-NTnY9CuYP_KdxveD1I;XeIa;JpL&ez%+;SqSYjWTQK;yK@w^Y$e2SW-t}Ta znwT_?y~^gd3hwX^BH$Z?`|LQ8hkC~EVtSo-ERHPS4H{_QIn!~{uTR^2(n*<=rOQr( zNEb!{9Xc3K2QHGQ=@J<>pDaiL<Aq*$G<X7Xc*KE_>)n(~@0CCGijT(t3yHMj3~oA- ziP($3$3;RZ;XSnI4p7b36HD4h>JxG!wfzQ-ao(aAmz@*YUmA#OP>C`$M_feTsE`<X zIbl{${8igLQrn<<a5&?4F1J8Oc5;tam@8H1EzRJ}Dh+2}Wt-`gZ8ucqTCg-m@Qg^Q zx!)DIARDgwwp}@UI4`?G#`SH~4-eJXH=9F_9=Ot=G>v~>z>%en7Mq1Evms{~OOGSH z+6nw1=;Lqj9VZ@H?3EY+w)X?au5}pl5Nb2;xRj(>{b7;H+>EZg^sBsG6_p?C?r{A> z99tB0O0A?>7AsGGzOQ9182q{<T7kT~(6BH2;EA9j5R9I7p2Msa3!O!XF_8}ckiZM{ zN4}!(j3CSZ48pmQrd;@%3I{F)QZW2oq#FG3cK&u;&I*HYwk^J#%Ga!WZf2P!chQrG zUK#|CI<KAMQv{LXLfYO%bW>k)Ewr^=64yX`@8A?$a0a0FYFaTiom18%vU+{aiZe^B zLFSPTQ|_Ln&4H?l#XOs0%|~9#0?3o=(vrzN8)&6rt?FJE&v37RN2K%x*BF*MnjBd1 z@VR>YePk<QcI<7~>+$HcK-m<<a_RFihPQ3PpM|8n;eBT8%&|RHbGklT>FZgq%%=J4 z*%mEzO9t1SF3!i>7qN_&2#qe7r|2fV)jr5r@eV^JRD^MzrvsVnX8LM0m-5Kbaz0CV z@!ecLnH9)c=vzD0<5{z}W)8Veqzj(lbn5women}>xEJfIF=5v8Pb<U5NtYMO->tNM ztrCV3*ktNNH}=sMj0#_y5uTB-pt%TVom`lkSd$WH!1sa9f&fqlSdhq==slvYXZSiZ zF#AMsKWDAb?^-5&s_*sOJC%!WxCQHp4P1;SVMjZ)NIFobj<Zx75R@H6)f==Js{G2b zyO`fQ{L03pWVlLope}SQ-n-kq+$#HP^;qcpsv&21a-HmK@535TlhC1x9o)jZ&A`{5 zkqW1q;bilR*=pzA(z0=bwu$9_`<;cC>(N|z4`MrB53o(4>4+Tpv5ENYfbbPdtC)*b z1qYdJ%{lQLihQWLNBc~;8Lg>i`IN1EeA>lBrz|q0e>vH+eZ+fU!qYSMw_@M<@!8uR zuRX#_>}0Df?+22~ae8aEklRgKT#u4pVedqTVPy5vY4GqiO1#Jga#3BvmG;c;ABmA8 z!N62U{<C=9<|@$b(#U7Mp|y<}xbw>w&3SWEm&(c6clL~b;$1V0K4&thZqaD(mO1av zd7EutQ1D)mQE;|G{d&2Mr;GbcxPepciSE(zq;&P6wCdqH<ukpg)rxmop*EjSeq6Yn zntS+z;!QzxL}mb7co@RKuIS&f)1}O!Lx9R^lkfu0iyNC^VN?dGch*%nv&ijz0o(Z2 z?0DLkCP}2LrT(6v1A5uC@yQQp6DA$RX)_BiR)Fw@G^9Nc4{X8c5kJJ>&QruLd-NP+ zMt~Es>yO&>f%R1%p)5oOriGS}Bf2+-KGh_dBq~YtJ}jBRd+uRgJ@0I^tCS3&6q?^3 zJ4$_bkvrnU-_d$%DbljMW7t+K<@H;s^rJU5!n5z|QYu`dD_km$lqdJN6`LPncAqIY zd>nk?mc6i>s}!U=l&0sxZBkc=s&ba`H0GSaWYcSg3h%~7R=tr=F1&VhL0PrGTqf>u z*xwKdRs63Ce*ZNg@c;k!MbNJGAVo5vr6YbyPy(@C?47s7dQ4*T!kQu1hqhPJU$k?~ zF5i);IkS|t43*Cscn)U{Wd?zEb4VG;OG)k-sf|0_IvjcHR;Kw{@GDE3lLe)_t^1?H z4!1_OAMTjebxk{@|L#k#e#`7@qbWXRt&{K9gK0(dI;%kQ!<+zLW{oF85AMu8ZO9)= z>c#F2H#Pf}_04QHH-syXfSm48TzQmxA>Nsbgom5?LyB*tO~1Pt;!%t@Rmv=UR_Ahs z_%df59*R<UZ|p1!`q>PN&T`1|Vd}T<Lq(s^+XvbLQj0_HS{9fQvPzfKgPz#!EqvzZ z?bo$u->=CBUCuX@R99~AMF%6N))|;4co%(-=m;RGsRnCYsQF%e(;h_9tj?b^xIr$P ziB$`aMd^UK{E$Agv`nb6!w3=$HZ<KXw%>j{j%e-+!8Q)FW{-#^bo>6BlR=X~2eNwH zo2roLPtHFA8$W;EoD{7aztto4oDCS&pE-}hycK<FUn^OJ^s>8&dH+c@^U4uj2Vc)2 zA(ya94<-Yx6k}M2;IO;tG7(y*`9)QM&)5;BIa*rp>jJ*JwzUjXC$GMsXB+XZXveEf zt6q#i&n>LU0fvJ3VH2b1xsYCX0#U_yX0QM0i5@r68UA3r=$gf;%4OCyv(AjiTq_94 zEQb0rmPVMSQ`disesqI*`p@t~u8LKjX}aZ}k1n5O8h_gD9-V&H?t1B2w}LCP>CEzK z1D7X1+n1H_WQPlWpB>INQFGGYh)SDd%?>*UPF7O>LPw?|lJgQ*?+YEffIz1D!~>cU z6F9}aUJOhyQJOCz{INECS{BF1%^esU$iMOUWy;+0`0a)V${D&p?RkueCqKuo$#9&L z*v-zXI*8R~{)nKBAC$;MOto@tdj_FBfq`i@?Saxbx)%Y*2VKU_5#=39{6&zIl~6eP zNK2nDAn~*O8d_d<f)Uu$$=H(o#SOJ9R@=}=qeegdb>+B`52wndN|$V6Q{(zJyy|?K zTS3)GhM(8;sIRMsKFUYmA-4n<vPa$`qOjD{W4Mi~_%{&N?STv(qZ|!7b2+S8xcRtB z`N)?RqLaJV*K<%MdkRWO&Ka!MZW1emF^mtTK}Bmg3*!$U`q&BdeUPxOjz1zM5%!A^ zEg+c-k2eia1jrd3KHM2|?!Zg_^!biN>w+0Q0W+b!{>SE-=S|d?O<xEXp+$OsWp+0e z{3N^tk(+1b#ZwX%y9v3rA9%whcY}Ch8U9z0A;gKoTbiE(Dn0GwU4%Dz2!@fZ)xU+F zs}X2v|8TP%WWFB1C_gr&Vr7DG_<)Mrv1fO9c_R_X8rOWrdnPTsXsB|$+y?FPWaWJ3 zy!T(#-jf4)rLGS94NNX4X8T2JX7(G}(9CyN+tlmH=s8)Wo86=gCYP^PyJwM}hqq{+ zFRf8t4GhS(uin4#QQQRG^;2k(+tCn&UW^<*CUR~tnZ7q)kOEldGRR4>P`A>?w0uNQ zYz<KTo$HlgOi}v~zVtUTYi`X~*yRTPg}dwT8yC|tY7}@NXX2Ck0rAD>wC2&(Yt4pg zp4>MVsjW0>;ck7Cx)1u69hJ5op=IYS^XM{?lOjd%yGjpv`l|y#TBUO-ES0(MBIQGK z6b5LFR0~BNg<LTq;0azk_I@sLR}674{zf7fnL>RRXa^khwICqIYuBlc<~^n@P0m$5 z%+l9{z!c&bd+HE_s3cMo9wQ8*<UtNLFp~eUi@l4ciF4rEhvk*soh2jFI!<u>PgzY| zU)>1(EB*Xz+=@(yPjrro8aCf#BDdpL&ePhfIyQAOoDc4$M+jd^dtI6D(}F*gAJpi+ zOvl$b?Bf%?OR~)x98%Rbg7!4x&Kmqlk!`H}k%cdMv)xM`efxEsk5{kF#R!(`EoFoB z{=MuJ3~<p89p9!WQhG4qC@z=42_xv{kZx=$y8%_fb$*22j@q+KRKfe~jpv8VRrJWU zBa_t!k^x>f0X2Lp@_^*~t0FXjC)HzqY#Uy&ZvBEZqSRjzdL^!G!4MKtFhnAg1v!Ho z+vy;Ki&n)OfuHyZyl$d8%^;LWO{f8|J<!Y^yGGEDz-)s$?0yy3Uio90Eg$sI^ErTw z@4#?(tIPXFtjo-XIo!%x-lHQ)L>(f$qsfC={I7!B;*uE4h@MR})HoYFDLLwk^PI4P zn%0Iyj%A|DKpiPw<oGw(Yq+M%jeDn!UaixXOkG=Od<u1e#@?A_54IFac8IglA?yT} zDU&V-I&gAQp^26eaB94R$UnuA)J4_c$Q;^BE?Lp~dXX-z0gU0^JOA}pfZ0+Jb^Tbg zp`$@}|A0gghOM8=`|P8OJ)Ps@t7<>#9Hv?|;hm_XD%ltrFr3%wr5+4FPmSP2xpuW} zxEHCb;)XNSJ1wm3Y+Ih0rPYk|+-%i%anGChZu%wwv!?IFuX4IoJzSA4Fb;BpFk+#P zXGIdjHB&<T+_96->!mk&c9zV7Sqwx=ZX)_B2|N<}%9$nSDlSCr7sZ09!bn`-c~HLG z&u_WN5Hio7gBl*3|G=9^34g)y7~jo$Xhy@g%q!QMy*Z7;-{Nv#J~n=h)Ekx@TqHJ6 z^bLXriK|V(`YVVY7j~FH*!e9TMuh50_Z|899F-)(UPl$?;3F2T3Rv+5^CN&an8ucI zs=PKFa^Z7+YmFq9EwvT)w47W%E$}*!dq8D^K1_ZflA9wXNHiNDiH|Og^LiDaIAQ_u zNI4M6VA;lZm{*8PJoAXp7xOd=4vgo|$Hso)`oCpn*y&b&z#9WQ**%d+?%Kp7V+~R= z3;D<d16$HE%Xdz2R#+;Mn}9sn{^R&lfmMx9ji{+mIdHiC`p@SN_}p4l`*P*CTU)S@ zfLlurilz_9%`pX6dCA#!AHUsPbTXMP`1B^}RnT8!gL*Dr#hx829#v<K1Rm_Zn6~0p zt{NB?r)*6vw?AH<|K`-ETal`|yyU*iKZ;c=Y&wqItTu*hQYS2=gt~+gwkka=9Vy5Z z*ISqhQMgm*$Nu$oCqZV}K_=z_LS7WXBd5IPD|QDM^5KaV&eEyP*n=yxy8vd##lh-x z+{r^jiWg#sS7RzJ*N30NyxU1?Aq=9Q2h;)5?qKv~s5F-jab*Y43UD3ZIfAADvKRFe z7+3-tc5eoTnHC|2Z>_81NSr=~JRHhuYQ|zitIoK{lOxQ$&vk(^8HVlo4(-AS^BU~? zyI(O$7&QqTe~2n1GO?)$d4lR>(%ateT$KzFa<2W{zJVa2bFzuw)i1s9ckKsb>L=Yw z6F)xDt}-ADZ#Jtz(y(&+lLa}U=@D+jWZUSyv4h5ZWluGiv;rp|k2+7(>sFJ{ZnFTF zN1+$HD{bzMy5wlcCl-6n(zFX@Bla!7m;n*@iBLaq!vX`F3?Xyn4mH5){4n1C8JOkE zCnx@te~F*bFh|}FzUjHicILuW@uxxmdR)c=1F@&WEACiDRr7c`M~ranELir^y1KUT zhQ1mOI$=S5;;P+^X$v)e3o`Aq(`4c*8)}i_R;-<AJ?iw{C`eo+f^)CF+?gcx_~|`w z*oA>y!H-n8GvGZ){G3MYx4QqaO{524y1wP!KJHBzK`3?X5tJ5?&mQ_+>PPe+cW9kC zoG>f+K@WA~&>EY9AZ67W9Vk$1ADyc?$wrUg19vhEYHJ+ep3T0k*KelfXMg-SUC`Q5 zu5aG?adxd~on`p7-<g#wpp8$$cS{)Ro(~QieroN5vr1n{WcOeHDfk;+4a&X`_gy4l z4_~@B(n?f&^8$oA;C5<^pF2R4hI)1m=5PXdTzDTQU3)vZBap<Q|3MhLUu0@k5O`zd z(#T+ArT@K%r}gjiMRy{>!hQdIJ-f1T-DBYhl930k-c2iOLg-E?u|5u{jAOH~f-^%7 zl+$_eRV)4j{@TF8&o(u6lG&R^E7$bDiW5F4z(2?Prc6s@RjxKPx31X<jlvrfW-JJY z#3=-A2kgnuI>oIhO>v9yL{mS$`t$YM2h9)vtvZd^$T}(sqYI-YGOe4Pki2#j4VXx5 zmy8I}gm))(?+c-R;VuV@Lsh0(wP2HrI8$P}=z`g1BDcORf(=`h$ah7T+5O~(j{h6# zE*Mm^Ih17P7an^zEz^Ovapkwccy`NRx-pscy&KbjSW^rE9$=Em3X=Ue22Yv+SH~kJ zHdf5<#r_6pRh2ExYQMJ#1M-<~Vu`?}BqGx4V#g5nXa26vkeYLx$Y4{mTbnn_XV=#f z=bV!fdr>-~W8yZ{PMp;xFq!B4(05lSFwJZ2hRdD9-r>^I>5c4E6GQkuT-I*@iX0cM zbT^TS-HoHzjfk8%HNR%6HAAH&M#F`W2_r$Qt+K6(O%76C&of(pW^n|(6jbWf;r_BV ztIaoHk(;C5a-jLrF>Ta9xwlVv9pA=fEG+EI7k24g)>|>Wq+<lCSo9(ihzNoP&Y8R3 zwFOfR(=rF|@FGuIdNF1X4VB-1z&#KQHDU?IxyCPsO=|r9S!kh;eC9q7bblF+Fn@zP zb~)$px3+ahJALaa8~dd?H|Jn-V0^^^-eI0jsk03^4=T?5McN^*?udqJe+if-l{H1a z3iIX?0AgQl!M4*3z?w^u?$<}yj_*6<>%&bi`u8=b@g0=gZ9f^b_Xq7SF>kCAnhLf9 zh$Motq#NDf%z&%dR#|Ze6A_KhtDHHsBBS=RWFoZ(wGT9(W5Xj{mfQa#uj!0ZS01D6 zP@&hd#_4(@w`Y(MDT~LXtme<jg>6u#4CgRC6gVzXzy~RNPM{x%Lb#+v-<HQbw2Y*O zAUCf5uQJg_N;yiEV<%&;H2N)SX^gNLv8~5fP>1&U5YLYRm+aEYlbCL$c3+PgggZvQ z)2+{qJ4{)Kc`wY+>UL*vsy%A7Y?H&vzJIctnlAsm_9<la2TAZ&`~;;5HPAPyFxRK7 zxC2c*XE@#*9O3+9ae`YI!kH~CUP2i^f8IKN>s^#CASvZy_J#{iw<fIHx4KT4#jx&N zpR-Y|p={?91$#hiej?XvHIknq?4c{*75Qj^7GGzU0_676q{cc7D<V)o0pl+@hXJp2 zlq%jfGOjD{i7&)U(ur0N-iEj2M1{2`M5sP{PH(uuq_Ik9;ZJ_H>1Z%6DGqQec6n`d z2^gR}$I_bw8`&*S!Uq&x)ACRWr7@M_fYSNW*GUy`MzWOEFZS3v664g3v*Ie86jEL7 z$WMq6bDqmY$XUek`?#@vY{;ofU0ACSs^RE7#!@VXsznkW5{0~{A#kR=facQ4t}-7S zgQ=y(k7cayqa<43jUBwNkuL$Gd0OR%bv|asc08`H`3PBo^hAp5eRNgHSz?zu>L6a5 z$U5Z&o)exCIfHa8Uz$f|Kow@KBl$d)F1mal@Dt>`+k$17z*8m2u^3H15B{E_qtyS6 zy7vldvJJy^v4MgZ=?F?xdbdynG5IZkfPi!aAwMcYK#U4VgqSG3N2$LcM5Kfs(Lj`v zNLLY%E+8dAK&gpJ{4~Y&I%{Ul+GqP<&6>RrU<M8{Nxru{<-V_Hu;|Xl(W>r>oV-a7 zYRk~oW?JTN$*0dfgMtb-#~Rwfa#_!?<)<V&L;nq~{GlCc7-{t6SJC#9oz8JKm5APV z3+dEPoN#~iXqR6d7Hw1YHMPA5=3Y!}*Qq_?9GaK;xCn;s%O>6<T%Lh61KJoL*S^n@ zQ>lccKO!oKV>kd6FqTyMIfq<T{~IV-aT2;n^4?gNvrS+J6nQVv_p%GqN}!1L$X&U2 zBUCc|$${K~lZy*Zh&K&5WhN2Y?-2u70lmmT9<UT?^5y=BxImqz@qFXiyAfTHj8#i^ z&fS1c(k{L>%bO_w6}b!NSm`C~eKxg(->x*jR&>-eC2@G4)$O#oh-oBFtF|#bz*;Jy z(>bUBr=~qr!pd;{Iv88gwmbUb!)UVE&`l@X^I1h|tO+yS52Z=%V{IM@^+m~9WG9=3 zMQvN@g)B<n=Te8qMFzn`S^H;MbIN!-LhC8y>_hw(a0{d_4}9Xg37!)$ggBK=N{;te z*65-*5vZ5VYlCmy(2rs+rqo(sfU-VFMS??YrR;8=wn*sEw9Q}b!6w7vsSC;5OK51d zAFGbS9r^71ZbDsI4Nnet7aHO@;fJQ>hMUh?1o(4+S){YcCzPAXW~7Go@1B6FT77lc z#7^XPtjie{_?U$$#q)w=QO_<?;<i`OtC*05m&urcM%`@=Qbit;azV!ARj;N&$_ex2 zYE-9qU4hb(G|ywhF{O?<Hm6L+c^{LpWtkM)!_nE<zf_eTxPERZKzNO~kkz%2Xs4Gn zGt*z)Jbgl|5nUPI%TTOmOdRcG>?5GpCl<MJ!+1C`HWJ=tIMKs0O~8e-Fd{guyADI% z47S)kBjOgULc5n?0<}qStL@!SmnLCPz8m8eP4w=`A`Nkf*hOGpUtFo0GmJyYA_FW1 zWbjRH2U<KSoKr1!@~j}cS%I_ImS+u4(>S{B7$2=bT&ss~3N$5&U#C#0Z!BChr!RaF zstw~#KK1@I{q~NaK>f$<Tz3XT*1hgA(<?6WR#+wT(wu$kw_Co9(N|B4uaGyYSlMS< zo_5-LW~=+&DqjOFjMO>X>pG=g;qz4?BgHl!N;RftUL|Bb<D~t<S5L!pKhjU-<ogQ> z$ew6`l6cj^+m>CEJ85VU6A{7nRE=3RwHn?2ly|Psckvr5{Y2kwtCTpn34VO$IXRXQ zGBcjfqOH`;t;6}AP{1rP)k~cc-|Wp|MEhzC@Krfd78ZGQQpHlE#-o~Lq0*?{++e-k z@jYQv&Jt<n3&(p6-I}TXP9___qmz4N910DL{gR@q#QVOb_^#A9+3V_>HiulL22Dup zFq_p(DBR@`%YIy&pU%EPQ%#8&HF1xsy2y_dfCO@unCGz)HwTZ+TZ|^mJ_XGpSC|TR z%WQ9nxknr$fRc=&YPRn~adFM;jogGa{_zs5w-<DWiM*Gmo!Nfdgm%b20FAzL^}#1} zPVaYX<uiXoo>85A<aFJtBHW8Bv~69kd%f+uy^vKRSeSp8-xN8uHpPk430qy6eUf!~ zYfh4t!hCY_fcpE@E7YMY)lQaj|DUH&Y?AElt<SFSoS*NI&&hdU7aATOs&~WN+xw)l z-HFbQZ_hmcJ^S#{RHB=k+JjBT{O`hd6H~yM$yuC(^JOWW5~1O<>nf7gvyLnJrAyjJ zESZ>bvDd%9TN60ewskMlHJ*(K_4sE&E7{Fx@RWf+n@*hVxqS3=$a8uHI9w?Fe>e*1 z7KUt^a0iFo4Mb3wL^w_IXOfX)GD7{sn%9g6<j>m!iJnkd<UZ!Kw0zQ8FAY{2&1M-& z@DT%=S~V(Fob=<+UXxsHlrwuQ{s&AH3T+qeW|tAkP*j^xcEE@t31R*j3@Ls-NIkLT z^s^Ydfgg8|^vUf&lPA%m4IkR2Me4dYKfNbHsbp4*QL8M%E!Uiy{T5B?3EZXIzL-Wt zKw{l=-{V8B^XqOu$@8*5P~7iczYN|_g8KU^F-@F5`~2@G5q8oF)$*+~I|=eeL<ujx z!!TX~g+p_!SH^gjSZnT`A%s2}q5YEWP&~WOd#*bfuP_dw&W=9i+@((3%vf|uelr!9 zbp26*DV{j?ZJZOzGHpY~(!oU>{fW?OVR17;g~EZ)kvj|(`PPCgyeQhl8Hn`_J1J85 zG1O)%eD~KJlw`9)>1qA7Vo?%j(SCFdW0)cLVjz!seQj;^sbwJlF&<GX-V`KWqvdy4 zj%lf}eEG0zoucC4Yrl!iAi5^Zdyn}=YH&|stpCcQ;=2Mz7sJ-`U6o;E)Y4E@a{FL~ z7pk?58PqOaITwtsu{eQcsv2t##*p37Be7E^cqYzEKn0bB!-6vUR$=cLNHFo6z9LMZ z{K{!G$Sm4X(ojE$UE;=cmnZZm`G-}+*p3Nd${)`ukrMUR23hOAO@kqpHTier)(zl2 zL<R0E0c}?BA1<8jJv)<t-^O=_RzX8KiwdO$vo%|%caYl5#p3vObFc(?bJ4mg?By;M zB*(X9tBME>NKqoZ=p*yP0o2C&7&~@Zg_ewOcYt3M5Mhn5dECNL<dy!9%gluKnej0e zs(XEc#_Ufe_`}jdAxdlHSQ|<L5(BY_6f%IY8`^T7!$<D6AaLL;_xP}`QR)thn||c_ zwA$)1$-Db=v{rxC-$l*M+@INlMDr*gW9D*GE%ZU)GTuEWwlc{(vxJmuK0SM!A@9Za zbce!gu&OL%J4_bR8R|kPKC5fU_Zi_EvI?YE$5o?rb*nsR-!Y@x*RHNq*fOYHOnJjX z&x_NZD2%O+(PFL*Q@h&7Fj%QBtK6aXh|3MP;$zGM>RYRB8JyGi+Lf-b&s$=Rb>!70 zd6kmM$Dts<@G4ZHJkQdo7|A-6wW@xyd(t7lW{0{v;wbNuaGIou7r{nxE!gr!Ohy7} z6XJw$#6!hUMT^eC$8nUorEFyDS-cWtGSumNY@iv*L`FaO;x*Gv+;h=)DbJyxXyPLx zJ$JKrfh}{ZiL;m8x1Sw^NmK3<(6!g%Ycrs?0UQOc_h9`Dqy@W2xJpQ$p^C*J6mS;> zk7{wJdB#u<>jxAyhiOOc;@^3Z(n;EkL$Xl2XRz@0$T(q<ZjjQpzTyh|vL%D~E-nMb zKoiDYn8a1bBJ7d&?g+XWo1Bc;gKdK+!o}&@I2}$pc@~xo3IGA}`GluEwOH#;^Q%my zPhTR-O8xI%j>1Hh1<BJWk{j7=Tg7n(f-Jfm-xMOuF%t5IuvK$+b35<aF1tgtTKos$ z{@l4;O`KC}>$cB604di1R5(Rl_@MgXM+>5F(yDG~h3ah=<+rc4VEY}3@6`wmY-)(@ zl3ZX^q;bWE%*-P9g1rt~c0b#YS$wQLQhIt%bT!5NR&jE+*Kl1%g*euLSnYQp;q_4I zj;_XHuWYZLY_~XDYTdB=!A#r3;o4C+m4if+x(+{u7g%hRbrRIm4u6rbLgtLUCQ<uW zY(OoUt}3YivA#7(-d8WhUQni<&rKnWPj;YW_?sw!zOcb&B!|)+DfdITpA(+Z$vSaq zY_)N;hK1SH&iXIH-v9EQGPMklWb4qiu#N3NTVcx6!&wRP@kstbsD_8csSR0d;h;K% zMv#65v}IOU!CH_1hMwNWi-@Kl2$Guk3fuuR9VQKzy`DF-e=psx30UYU_wjOITi*PA zl-<M>Ya`qj$~3SQS_$z;7+wle>04lN$69B0Ht+e%=kT58vWuEqQvLx%s^Fm9Ort>m zEXxJk^9om0jlX1%ol1Dt;Z9L{Jae%qzSP%*MlIYWR}^3I{!?kBQ-@pXyrwow-`4Hw zD>IaS&{~V4bCz+tYlXRF6Hm+IM$SQTZ7Y)+^Bd(^u9vMY4kTyX$|`zAi~csv8eCAz zn^@WtvUcF%>25ulVoN=T>O*e^UTlVK-RrTa*$#&^NNqD)aa#pBbV+PXSE1pUg*|UC zG~>iccuh=Gk^6?*!GX1MX?Ts%4Ouo$hB{FaQMl_R`#J>{zfoYazG`AV`iR_hUB5V6 zzn3<JI05CeCz-yfLRE;wkpNNNE{hY2p&Bi7GxC85*uaA`MV}2TvGKy`ZQ>>JJ?veM za_rhVP|gVy#JO{qt)LK&x38cCXNfV&8Tf*Fp3p{0o}o_qOr7ey5)VWTitOukPbcn2 zpQCw|HjC_hbkoTlYcA)PSmP{PIh3v4<(!d5K1HrCK5mkG0u=Kf*YqlAYf;H+OQ|=* z(S2#+p%o5KA73e!Uss|oT2(9C&~398wOr$VY1H)Pr+RI?1W^?URhNp~a~+Sg<eh+| zk;xwP!wIYt3w3)Vt0N_6I+{WoVM#QEi2XV2Z9-C#;krVHed)d{wm2qE{nzq}qmJPr z`}yw0t=~GD6y8hae7Y=s`M7Cl>YT;c{7CoeBF`TCF<>@;FB35r#&yO$z~*ubp&E8D z_xgt`+4pn}x$+4VXs+{t*EWB>>lTI>!<|p{+v*Gq!5gb2NNusM;$IK|bl_F{BNtx* zI0Bo@N?6sLJG<qEeoZr)W7vh47+{9Go8>r)O>O>iSbos*4(0f}Lt(`UMNT?KQzPc* zgLb^^GPTiaHB~%i>KND=B8NG7WxQ6WnBtYs&9pHa$-9L%84P)4Uox+e>D@3|L#hsc zhwbhz6on#6Y*?^Fm7{F({V>IdI}<ahILi`jz}b{bYlNfDqwD+QEetgc4w+ea%<Ipr zT;M2oH<dyh9$xqfCF`}QTOR1U6UqlR;Dpx9Ng802WbzClcw6Lu&jORu&%7F~o%Vby z{bv4gLqZ&3HV^qMB6cH|b5Qsmxs#7({nzjxAH_-Tc7oRDj!k*Kd}QIrsd66MFn;yv zJ#HX;0k@j-D;@W{+07F5(N7&|_STwxapuvCUA#_H)HPQ>q2epNe)qQ%xgAKhI}Fn7 zN~`$v#Y=2en~6zPIap6n=bQ141tLrZWaIWQE9o8{FpsjklYDibLiW$EZSR(rZI|kv zBD7jtM6l9EK_Lh{jzUrcqr+@wfi{Xz1$jGWrH5&#m9E~DKS%3W)$EWLxc)*jAC+|x zBAT}%?Z4#7CopSFCau^DX{CWmCo=Bbj&x5Vd!fBTo)T_Nbag4R?w`mOsqy+=$uU7h zy2P35zB6!K&kSat5Wi@4KY^oZPU$V7f2KerMSD(@P@Fu+n1(R3nkz26;~$Z4m4Ui( z1SstiIcI!x74QmB2z#zIGH^DvT_LHoee=^xhZfCIF$~&z+FKBl<boZyXSjLa<=aOI zPw_n@(50V=*OkS@N1Z@+FqCIlw@a^A;@w`Rdj{;k{_BW4eU?&4*Ab4+Jnw(qZbaCF z44jFE?=+ZIS3bw-?u2iHLN8>TzxmlaFt6~%zNz`X(8Qv+&&Q7WO5Pa>#r09TQ}Y<# z=#GK|XMSkUn}zG%V(y<12KW3CvAy<9O>kehXPJcT1d6!&neOpj5|cO&K{fsuq3y8l z?lWn~c0vNMtN)CvHI0e)_%MFRq_UFZTLg;OTvJddmFqLa?-TaIH#Z-se01)6@Puhw z|F%gehx>WXG>?>&o*FLc{v+;uz^hj|&gE4M@h8|}INRB9yt^YNpv_*Dn<98p`yWvU zd5`fN=;b(?wmC;%KgJH(L6_(%@gM7)7J90>;eESS$;K4eDx7x#J0!+5<jbJA$vFdw zZew%=?ko_IZ3;|XF;xYsNmw#Mm1A+6clOgD7QN5*`G3z9@Z#~uWrtiVS<1`k9?tOi zXschQLye9^{T#gg^eTJvy~@p%_vHQ&5fK#;0W=dz+7mX7FNf*csp%+*e?7D5TJ50$ z?`4D$AQYZk&clW9&C?418uj8$JQQq0vuML!NAI2CKX@h75#%v~;V^83Gl$qsgmmJQ z*qLogSKvXMT}^=9brpgEPN@05;+xu;pdCUrOr#$udn6(j`0{1$ZT!t=530s?5{>c} ze$!qMJ-~gk;o!mFbz$+swSDv#{hH5H{dEQJW|FwQa?n6A4#^JaMo-!aauJ8YrRvL= zE&g5o-6n9_7;@x19|AqdJZ7VfNiU+5=Ztk}BS`S&Ld%}+P_bmk7qd>-Ul!N@@pc|N zARt-&W#{2;@=jg!MhS;W%6tzV6035@ukbt5@ilyc$bAXIYrUo0`bVY+{t$(mE2tD6 z=r8hoQ&(d0Qwkf(^ojX?D8}oN?YU3&>=anIg}rv~-T!QQ1svkRAGgkZElv(!Va8W& zN7f%JH+<ETzm~+c?m|vz&JsbN%CB^W|1Xryq-^bBNl=HH^`A^%>@COkbmdiN=s5+_ zx3D#mw)NG|wS<2T-*LJ4qwmx!q?$%1bG?0LGM7tfq~*Vc?;?ZBzt}~^2|_A?N{sVx z6BOY*$Zz8VpbI>M+S1(+-bdb*=mtH!>vz{*S1p5&9MSLZ{Vm2*BD5M1{{18JQGT-T zui!QSvkfxsijYr3SM$Bcq<?mklyGM<PaSAc>hB~e`=FMI0XHIyat`@=e=WaQEq}g4 z-kA6MW#5ZmJi$v)x{D>%AzR_&L)=s-IE1o9KLo)*x1paQjW`apm<#2FI#4$USdWwz zM+ui1NlNP-82Oq4#w69=v<*eI$jhcbBu{({RN9ssmosy6(5MD66_UID?pLNzJe;%G zN}v*2*NG^$1AKf<8J0$Z3ymRNU+5%9!kN7>42}~?(A(R0yfX*o*9=iOlif<7nsAj( zPB+pGjvci)I{R^vAQ$|*Hs5a^-7sOujep_yBRfmlYkb1nKHT`nDEGrj{na;d*A3O; z)za-MMyHL^u0G6a+Ir@nzT1_xB0RjjTMk^BET&}ObTcGy(%yDzHE_9(^Voi+P<7Xw z(NcN68qMZ-myni2LXckrUF{{wfRj)Ri;)i6jRBqG1b<ZOv}tp$7cePzvkPb)@bTig z+U6daeG#mmtzsi#L+H6j_RTkYSTo{5)B8BGUym1fEqkRMpq={+WVqnClw@zN1*}=& z4)B1lwj`Z#!ZNtF0<uuyQ()u^5%}tFV$2_rAOht6lfKK{{@rNK@7$!F&iAUKZp!Q# z`{A4XTQk9iW?NDo8Z;ad0IYv+ZMEsO+9qpL<FmZmgKpYgG=rpfKkaBn7q1L@sZH7h zR^!7=S0D1+pg2Vwu!lr@;Yap2<_BVghk;TxMJR`s|D8uP9&@#q;AU{FYUES65iue2 zImGhg_NpEFH)S(m|1pAzn+UBgLS5WuC}FMu(+L;H%LV%G8ZrWBys#VoeBl*GtP?3M zJOS+I6Qhq<uq2Ww?j-NurAAEL25Nio@0s={m{j0zZ!EkmZ{JE}U&SVnYNki&m5>^; z+`|vm)|d@dBvokj8W*SPhX<3lcYYmjAi2IHC%KY$w8QL=8ErUcudD|<ID7rHLA>X# ze|#hCJt}-Rvnd?)alO3|#wM-wvQY7bdpHrTQ?S<)#)bEvOhDG14N_40sP*8gXvoH| z>&C|))nC5VPtd43@i{Lw16AKc8v;8DNGN`el?bv3LCDmZXY!R@{Y+SAAU~LQ3E!y) z<e4CKiRd)Z=XnVGz0MvyR&$9B^k#~I!ovgCLIxVXFqE&?usZ3{EaYS)zgs4uB97t# zbMp$zcLb7i`wJ8ed}ab=r@gq_SH`VM`#TZ76;ntfpeXzI+W(T3{r?X1fZ7dCYimZJ zp%nf-sSedmOjg<KgzRi0?i|M*fna4ZjUgy*G&Dl_3h)G?=(>L@`@_;Wc3snQQL$-u z`lT%$<RGiHn!l#mmfiU_CQ}(mb7a82a0VJP5zK87yj=f;!eUUI`Sq1F-qCaOS7|Q< zg#>?!G!)ZD*XL*u553+EF}T_JbGxbsD-9=&?n<<)7Y43|jN?}+3WCRUagd#3pdIYc zS<SdBrb-HJ@OYsVds|_V8s2BL-Z1`$x4k(~ESe|0@3QxH8un{qG(UV)=v`xl_IWh* zAI}nBalc(pMFy5&+xOVR^l{;Hf|&{4Kdr;USBHL}MaO3Ho$gsmmb17P<tWHcs%95k z^%aEDj;S;9cqN<~n>((|i(mz$Hyh5{@2&o4kL{AL)`-4e$dgoN-;oo+74@>8@{)ag z6BlgWmdf4!t>1sHa`)Fr{}#b?m1u#VXhpd-Mp9TRS<oEuv(Br+&a-x1`mp-@n-vF* zZ~uGm!4q-*S`*mGWy1Fp(_ipgzYyTaV2NoQAdcd<@K13_F=hR`{}Ud8h;{?bqVg^z zJw~$Y?$WnX{EhcM_#p!D2_%+uv)j3*kP}<eF6+)ikUdlho`f`?WF;QJAM9^KjkmW? zAAH{Us$IAF7WUg4E%#Kq(7FI++7pPwzO;4v3covpc@c%kn|%L6YyLu7_?n-;Vo!BT zXNYj5wl;H6XuWj8W!P9kQsdpS^N3;fk16x6JqDO?$x{o;$90cm<G)@vsIA?1nHJ|N z-M1pUDZ>))JcZjUc!^hA1Na0Qw`I21Wc_FE^lrfeyeRG%beMdQ_TC6;#>n)u2Zq zS^P3qkIUfNl`T3SWbS8+@nr~HhY`b&dSZ(2PENq8F-!T*T%-BoDvIOCLal1~uBrY< zj%g<-@ZBoEM4l=W%iTpPy)fBLy>p{h_iat*G0(53VdfNG?@0i@SU>yY3wu<<J-n(d zL%0-5;05t7{J^9EZ8Q8?Ky-K&QG&L5l+Ej=AFkhNfZM@7woIa06Kl%cugKGUMIl%* zM0BGz2VB%UGNG3C;M8o4FD4Y6{5zoBIjMQvX2CsVFnnKv@>7szd87UUNTWwhUMJgC zfgIuxvFm(>ZO&P~oKbm6j{WN=zqhpcinB5Ii~=o=&pPhF2DJLZfn$sqBGKlxE1TJ? zG&Aoi=z3H<ae6CKr~KFX8&AUIhCH&plj^=VmTo`qGkA-DD`0$c87^)APchg%v|8Zm z@hSU{$hXULDI`U(i=BmP*PZV2<c06@PJGexI5#NN7moQNhW29~9Y}E<xh_=r;{N-I zqDRD#v7M`0w$1B3i?xI+u;fDIdj|ZE2nI6=XI<*eYG#kMQ<NYVt~>kaL_@^KRy3$m zoo=Teh_ZGMBQ#(Oe|w}wTadSsUTNg%VoM9}T#nwZdZ#Qez>^iPHQb!hqxwKK&B^9} znyvv@AM_=*NedDXGC`OB0U!xWHUV&0v!6I_f$TbOGo*z3d(0Pq9tKX>TaC3ey^gAF zme2I&x1T?||4w+QMY~k#IUg(1nO`oyta{idbS~W998YCGvkwn`MBznx5R%|A$k_ta zv!D0tAF|SpbNwN2pr0e^5%=*O2mGNP&Q_yyYw4hHd3*WdD6aRLnLc;*{ly%$Jf6n8 zcm4KM+Owe={cCz52hZy3RoAfUsrAu^8;*snKCwJ|$xURNo5-eny0ePN>m+5|$qwWT z!iqBQIzPnp4zTio)$3M<e~*&f*nY@%hR}8a1h%<t9%<y~-!i<u{Pe$f;5=P+eH=|- zyEBqr=5;ekD^5J^z%jE#cp^gzuZ<rj?ACL-CmEp`kKYGHz5XrzDK6x#5|p!M{P1>@ zo#LmZ93NuG@u9Zig5NKxBteU=VMloAPDx3fOGWaH+8SemCS}HJ-qVUDrF2rsg^Np( zHG;xNW#eatrh}B?hUMMeA81bJEv(It9y_>aPr$#TcZ|QCfddohgmt$vfr%ru+7SXn zZ2NOF+X*kM+ey&?Uk6Y49Kkv}>4^dKzRsjzD)Wr5cCxEh^$XEyRM`MXaUXz=kh$-~ zG8i@hTZU63lJpn6A2nO@+;OfDg?G-!cZ$Mq{scsDdzu5P(T5K2m2DV$5*tm3c-v35 zi+`b%6TVl=&<se-KUdmuES8!r@5TL+T}En+6F&DMU(ez~z+yQlh{bKGBD5W^5)S_n z*&fc2TgL(~wFDH(a)p}aD8}0*feWhbzwnxipRS}@bM+T*v^`GldLf|U+~!6D5u5Ps zNSa}n#>;N5Z%Yk-&vQW@A9h!{97`c45rRS0`Cw=%^~#~VF~_FcZzLp4{t=vkq5w`6 zWPfUbARbZzZY%$yZE2z@?4PZmneCe)z5k4@(YoZ<h7WP)&pN%>VsWWubBp;!U7HJI zo$q|lClgtp=sX<JoFX$E;d-c!`1tcYqz-@2&!;5_#fvyH(t(7ne?%_Nj8~l_JiJaf z;9mh)(v#4;L!Xk?EOxN-UkeonRdsglbb>~4&u^Lxy$o{)L8|HMeHL4RHO7}s>Gm?t zel2;o7!NY!7nD}Rm>av>L%k+~|A;7DqiZaUKB$`A;nxy>+pX`1CUIDYjZ8<@Td$1s zjL;8*7;vFEUl%%!30J&pGS-bZ`4F}*;NsO#+1tXs?2Y*^437Hpo9c>farN)N9eMDu zPP9ew$^{1%iw*72@^zie+rO3-g<>R&bt8(pQn=a0hSdF(Zv#CX#Q7NE4m+OVvS;R; zVG<HensB%o?FXtmF8nYKGJY><<EdkOp$exqlWAhJU3rKn=C@_jsn76o#BSjmIN)Ro z_4{A1k5|oFx7nq*xe?Dpci2GUB{Nd+Kt$u!Xqe9~<*J@c>h&CSp}VTy=kLCIN<h0+ z9$^xa!aX90Wdva+q`Q`bKjNH00FOmpp1Egg61!raO(P=Fa=`n^3OU-VBae~99`BIQ zscJ@CBKLl6bX);TZ<{gnc`Hc;8BB>K)PdmjI-7=sAunMPY&b5g+>Zt~h9-RQgCr_F zK_mBag&MN)JCzZ~T}xlpQ@C?>s;<!g;lIcjMTcM^u1z0oiAVrhnzee?^SwH}0PS$- zZcuOOACY^@>fjgY6g1gQ7lqJF@Z1z}RvdCHe-l$*-=f#C6Jf%UCo)mH&~Hv3iV|Ra znxiLr4^@Y%RURUL80+vt@ns!f6BO1K29TGCF(^3Ux*+6gPG|V~IPY~Z87E#hyftm7 zO^t9EDZc@PlZfOGZ!#3jF^R_B_2~-OmD2sHck9^QLLf}%iD6+KfC>oP%hwq}205m9 zT_67!Y68fo$xdWFJgtUuzUyr{Hgb$LU0nW!1Y2`?muf}dGw{RnfJB4h<%`LJBY-JU z*Fbyzydk=;uWx%zh`zpG`t-!;#I$j;L=^Vm`3L_zJg9T<US;Lxdm?i4m#|Np*Y_Hz z80CJq`Hu9~RTJDHI>OprF9#%1bSP90ogX1n^<UrNwE`J09QksZb06)9i6Gn(5x$<@ z-aLMoO=*?fg_>Y+6Xl6K+6y{V!*Mw?V6kJi#-qnzMIR?Mr>J^2MUMaED2>542GNtH zu#>)<qms?p!Xn2uqj>-FFAS7<*O;<Mt3`7mF5JY?A5p`{i|A}ZbhBqnD{1Fi@$HAj z7;y-7`rCxNj+mn0@XMDpDy?vB4I2^?;+Ml(SYWxvPft(FqBRQdl>A$F>S5K<qq{3B zZT5@vhae(YLpKG<7OE_LkJs>>78n+0Vv*HH)NU9?$O^mgJ4b-avR?}AQF;6AW!AbJ zqy`$9doHSGUw|d^ZHlawkHHRR`pR9qete&?Qa$-XPhi!A@O*mi4{MsWSafT5Ubdo> zlh@mIjM)i17oXF*)#O8aNXSliK<2XE*3v9ha=vxvS=D64OR3wE)h*rJ&Tozr5!rSH z%Iug{W?zp1wwg}i{s9YZW}0}PkM%_k9LQ89B~A`dA<+Q(NO+nvOZTpM@1||@d1oy9 z^>jH+7yg0*$Vj;9Bp(6!u}^civDFv8fHPMM>aOhMx#0)x1H0cV$isIcl!GH3?;+qz z3r8N7(C~*i!%}7sONZ3O6F<o@R`vjwWTY@3fHJDO2(pChhKYGTE0Oeq8vO9I>JT3_ z3zJ;7<F%JVV%%_cUs?&y7>c@;yS2WhFPEtpt`&PzY*+)EW4qqQ7WyT=C@_vM``vF1 z_EFV4VMXL)&&NMEB{arP9}F%rMnN*vkS{Yu>vS`)0BE>l9KB2^imLz}<ydoh91IC? zuu>56*1><Pf*wN#t5^K<e4@CoptNnYxbUj2t`8a^HSDCbvy@fNxw<vB1O*VV99I!E zM;=-@XztODjL*x6&F8uc;%jl<AR;o4OS(E$%6AczZzO@{9!GTA`@#aPxt{$Nx?Ae2 zak-_APP?5<pQ6!dFKE$&gWLZuq~z&|iD|WHu#mQT7DaErbpBb}!^+A^uu()r#PjI6 zV(l;Y9#36H$vF{B9x$>L<~7xD3%bZImQ0+kUj<`+6nxA{;gnN=7<-5r(N2u<l^g)J zR81iHN9@Ha|C97|&_a8*7v{j`CipKL&R_-p5FFV)u&(hRl_v8NU90hMq0mPCcIdQ| zWz<`(QlP|r?ycGp(l8l;srsvHj84{i0s{fmliP@cY>&1-BBH<ozUxO%jU9`!qRNp0 z)eQcL%VpbqygQMS%}4#pzaXi(?Yeh1!EVBnkO4jfmRqd+>+DQ-dkupAe0>6DZ7jCg zgSoyBa%o2#W2aXIQ?QO*zA6Ja4`$k}&POZtBp5cuzG38tk9+U<$tdqMlW!A&Z$+~H zlir5g7v&#Nalz;7L|RP?msIZc=>p1g`t-iu()og6$894uBhBxeqwef~I52SE-9zVf zwB;QFUuv#{mZtzFV8F+fJ3Ph|eDwtP5k~<;(Bt0H(Wa#chG<x#5h;=Me43_q^v$4z z8R04^5iU`ErnJz^5NC{k_~x9!Lq1B@&o$_LmnA{!mfc9O!qUiCiPHw(IB6jcr&jn3 z$N3Ry6R8=ntbwlsik5Ey!*Q}s|7`oe{^J1zVsJ<sxhhkQ^B0iuO85>!ok{`E9ebH; zm}~z58R(d(0%&Gb!o8cmxbuzdQWCKq6*IFnOqUWjt&Wsx27{JNT?R|&B$p=kLiVoW ziq0}YeaY03ITtCb)W<=N*<A&;gST&!t%wJ!ys9&9mRd<y*h*J9b*YTjDA_vH1f4IW zei^R1ekMPQ>7Y%0yfn;E*FjU4+%i%ZDq3yveE<`rQ=pTXuwp3(W>dHe0W!UQx(ZZf zgltp3tetSj<m~!5@gii0W{a`ls|o-(!D!F}!6ILqBV*6bjcryBVFhEvv0F|;K4txw zn{$)FX{uen@f9J}D9>?;CglUcxPR}EQ6C&*;nc-AW6Z}PU`$4HzIQStg*zck7hw}! zmhTItu>eYvWY0#n8*aS_g2z_jzA;SaSQ0;=T1`nKMuk#aTHl;IR?4^|i#hu4HO<G? zSgU$u%nqiPQ?GOEzdk1inVOAK2al%Iinp_MJM2y2Q|pyxprsx7c|^-@S@!#qag|@` z))|XHJ4vO7L7xW7bB;=?JJ{H?QPpy|zblh}nQ~7E%C0ej7@M%mEQ1sla(t|FT5b@q z6Hc;Cqb(eHM)(00B|)i$G7Gj9d+XZ=$i;_an8MdAp16SZ=>9ojyjn9|%<PC-7|h^~ z`~D6!k$FA{iuvOKK2U!JzRw)Ks?COZ3W|)bG-i4L04}IoN0sCJ1_!$~Mc~3=@i~PD z?p}T08t8nXBgfV+*)rpw@};he8f2^d6u02PxWM*Rp4nhp@JpOc*JIX_Wy#m3igM~; z(2cBM*VN3?Qzc(*34RW8>Ncf@apYq*r)*yKUTCEb5&m&P9mZJ;8W@+6F*L7vz;2nC z1X-8QT_#cCD+AVS=g)!7G1_KCeX$YUd0qD}vwC$~y;doS%`PDt4_Qq_@u0}t6}w0` z>Zab7&9vj|lx?Ihvu0>u)YKvu)c&Kx_=i8>)r23A(m3->h~dVL%Z*0t4OB6O{6jd% z?nq=Z&YwvVLqS8Wcv%i5oqCmH8Xb0km9OuD8V*s<(KoeeDCD%So}D>XZ+1TQN`A)( z*{ZN|u<C-9>yyH<$C`Vpnxaf9kDs32XVaB!z7Z7ltlg!`E7D}nAq6Oe->sJ0O2<~` zbCZT>$VPmOlP{9(0S2gS<JmyIKyCgv{_ci@!Y&-AP`aF}qBzV)u{~I*XrVTQa^R>J zq?0P9pvXTWD7FcWfDWj&EL)mz>&VGCN82Gv#^_q9=9yq@IJ>F@F;9Z?7RQAng0$kL zNXj^FMs*cAKI$@km(>4l=IzYq)tNf(THk2bP8NBuoK8K*W`KN(RG)b>dj#Q#?thn= z=^&qVjb!9?vt^)0{8p!1sa3R_j(Z+iHgEL$#mBS|wO=h`w=*JIs(T}Ke+@$yK+r-L zJf#posd>DgJ0y6@n0=gs6#EM(nNanD{GNLAL!c1eC2a6qLaI=i&^FJx{b<%PVR}Dj zrnSanmJmaM#g%g0DdfqS-qIznw2h1@wW_jD(XRMWzOSH);om?06U8RRAuNQ0$W8bi zP;vLSL<<RKe+ofC2<MxP;rw+kmS&LSs6<6<eBYd}#_zLF;Qr|s`;VL!){Qv2ojVa^ zZ&h+9$iniRp{CKrQH?BgU)Wxm&@|7sdhKtvIu6FFYIzZ6=u3efqiL4T25}Z{Bfor= zsFiI6po7>LTA)<t;xHDQ`SItH?i>0M{0HPFMgU5zlOw^s2aU0FXCv;d8Lm>R2ue6# zD1`}&pO$?!g1M9vtZy*cp>lXsZ`B}N`)Tym_$hUCsnp>#Wl<{Mj6<6JnTk+6&1q^w z$s?4o<k`-y`vg`MA%(t|GZVMLh?jv-c6N`9k<zE<R*$;n+UEY#klwdSZuWkG*<SzP z?ZMb<p=Oq+>OIr|&!o~;xN0)jmo+_bU1g&*pl3Y<Xl1g>TQwFFNAtXUQMsSIdwfnz zE|)EqI-xQdFyw66v^cZ~o}DsyHoEwJ6$G?}GscbLP*y@Y$GI^rP&d#NM+S7{S78zX z80RqdG_xHt>F*FqaD1kdjRwlFkI0QQtkQ3@LDhv4$7jMix2_{O$`UE_H;Yrb!oII% z!kW~M*&v!iCCamH)QQo#ZBJE~P5tpxEjZ+n>~qNH3oq;qLlTOUq8kSo{=TFmyrT;- zqk3AD4h-m6+5Zs<k(il)afcx%o++XAn1w#bY`1XX99m4XGDoc2ML01b)<#l+8UeUl zg&oip+`x8hV@UHiH9l_~Ui|@=Khe@s-=Dn7Y<@YEWxPruHNamXq`;Yd66sG$hS$;B zFq1TIh|DyMM!p#9>=z3}=nwL}IqRLiGSG#~%<=H<!hWF?NhVlDb?oxcHNK_ji-2-Z z>iOw@*`M+T*XCa)6+ax-w>a2Up~d`k<3)2`YS!W5MUNe3r|ex*Z_0k{b1l^>HLy|p zm3ijf&2;sD*L#z)>YSX2Eq4Invos|+3nS0o&>2cQlRqCyv<_YM(tXc2<&b9=Q|XG9 zU|GOHr5IZ^+woiSXR;sE#O&V(d3!^b4dabou4xSOt>Hv!ue*=-58c2TgS@D^+aXME z^v(<|E7d-{9DW@2T%p@L-_9@^1v2(IPBizA$Y~I<uT;|&fI}TUH#ylai8be)8?x|b zDF-`BVDpx2e&`!IjsEa7R<3u!_?u9I_q~7E(sAHZCJ<1SybAi@;F2<UKGpT}ywOIo z6WOU@(B@Rg*F9dD&T-}sal=bJInrJgHZN;ETS|4`>AYL}mb^sykuAr1CLAQZDD7^? zh~oTd5RdInh=ogSkYG?KbFtGCuMH`o;-@V*uoUBBCPVxSsLs9%CfjkWUQ|Gd%94oD zuBAKc80+zn3>z@P=>kE45wt!Y3Z|p+Rhf=NB$fCY6^+;8@SnmqgG$a!Cm3D`v_RoA zdu+$FuT%gy4q%(TcsZ6t&(IPZznpvYJ*YKRhdVRomx}>sJsWlTEw_cH=fCnQg4_H& zwWZ+ID}RMx<2$Lv=0Q|7vdxg2Yi6^L#)O+$PGK)~u;7=sJ<2L-L+&%kyb&jGl7hl! zMZ78w2lcQiv$^f-4HM|%Zn(T~Kjg?9i$yszY}B3&6>}6)HiD1agzp&Kb8fd@*M-=S z`Qm@aU@X`qjO)zqe|?7#B<aG<fM5zMYx`~VgeHKq!GvzY+wOWrr4*CVzwwzIjm3_E zFA)=I)78@NM&*W09q;IPm-&a(7uZt2I(K+H9!C3lxv4kCmIf%k8>5zn_b9B}E@vMv z_H-$*`9R)2jV+lnrkN8vwPvtg{dsn0$ET`}eihgKKFQq)?#Civ^kzF@5(*{2;Li<C z?rRTUGfqG>2C*LuU!~V^G?{KcSC!c_?Kw(I*xPICEKD0|EB5<^W_F%+`4sxV=ZA$` zl{OO5Id<hAgt238xD(fmtsPoS9nf!sU(7!)(v+uWxWpFs%8wa&(Nh=NSz4^}A<C(s zQk}Zg%#q^`IyW29w)$O8*(I4aRVASDb?N`n{gQ(w!C@r_3U^F@;W+$nd$?u?eZb-b zi;zxe_}O8vW1`|TL<pd4{R)gqvIYO@it-Hk28I)-+Z$Ul_7>0J%xun11(h+bRI$6d z&m-lKvz2g~VTW?G|1;1HlKEQ!t6Pj~2*m_fFHI|t5rWo}bDdtP*@WH+;PlUqR8NYp zf|`!Wu}=!9+@ZJSJY3HX(b{5zh_BmvDZT37Ej~@tp7nrxqR(j_#pOBiWiX%brlGd{ zJ#T=wB?3~NCEh2`8j#sN?Mh=^unBJ_19Xo;$CpKpwy@?XpMt`Rap#9DJj$?y4z9{h z_DlzCC%vi0o})7Pdg4U0hZ5DKQm8B4vsdVuB!nnHx>SaQ)%PdPJR_~DUS?`Z_NO<? z%vu86<ASXF(O!-&7dK`^OeerXR(%kGwO2XB_FkoJ{;F`a*>M}nz~SVowo7h%j=6u% zO!5A%9{#fhcE8O$ST2D(;8L<Er^cNR(S(9<%cd&@z`hJ$Cj^ovC%F|+%1&}fe~b{e z?5iNiz^l~12sO@9omEL7W83V|re`VKex>O*mv5>Fk3=~pAfM4z?z5{dAhg(omGA5! zp^Q8Bb1us;#Yi1W;hdUhsTW7}Cjm9E`UvhMw)O3`o(eO)2zY+pLOli2qO3SF9k*lQ zo%F+kQoIy4$T87Zjy;pYKRo-3A6yKE*GL)L8+Mu-r@tz#&1ZjaA2W@duDWrpUgBa+ zO>Boz<;xx|izyla-@y}SEqO<U9jFFmw`-#^m7;)6U>4-Abg~=&F8(*Cg)*>-%!};o z)#+U7;R|6?3NC1qhqBHlvO<5{Rg>E-=Uo4YtbSz-3-(#>h)Ex2fA}Lp`<HQg{eg($ z7LgkwU&Kn~IQq2vpkh~rl6!U)<PThPwty>oHJ=r*8pCn4ySG;%I5-Tt$xPXIes=fM zIq!y4#E$<`M>i;Z8C_>it2YriLT3ff5$3->8<3f>9k|nj41Z!AEobV-9}&okaMeem zJ@P$w=ME3Mb?KSrWCdJY>+SWT2Uo+Zr?y!cR)5Z&r5N(Fm^Fp|!(na(R~t{7LQU43 z6Pc{sZW4H7TYjvsVp74OB~-v^4DHl6_vN4E{ETas!5G<k%F{AC@@k&FQqQVP{CU*$ z-4mMJH&-~@#c-V0rPj(jfi>km5Y!s9suEWEdB<^%fJ1WU_@xM_ruIvub<BZiYo>Z; zN3g=FzRGuw<ER~A+7aU!>9ybt;0pq&l3{cnD!HTTIQIKRwtM~@DM`lR$)#XY62ijX z(YG-x+u6(Bvgb(rCC@D=v`%HY+a|Yv+}uQT$`wU6ec0q%uOb&|VP!BitO~w!a0RTA zbG?&BE92#bp}E3IIq*+7@dNH8IKuLA&K-oENWcDg)Fx^ZWqK0(f=d^agk3NW4Dl8< z&1E{Sj5F62n{z|<OmU-#GGA9x>8>?<)Q4`5arFevLbdGNxm>>NIh@%b<ypAlR$&+6 z1?@g!Z@#C4>4lFtCupObe|qH8=PMUm{mFJ8_wVUn)|XovV6HsReq3SOgzDG)Wu1NZ z%eUo#@ig!9;WXJlA{n~hbC2>K{Yvu#*=mv?wchLi9U3pep#)VM<puld4JRmbGo6ey z^DFs^XX_~Ky%C*grtYz=PoAYuwzI7ycFP*b-tA>GW>KxkXeGY^yaZnb%pgy(K3uI| ziaQJun8;1&Ug_QLCePUonP}zx|Lzih_^_~kDa@nFP9naDzt=*W5U-7F(Jl19=x~t1 zsp_LZB@?{-XHF<B99qx_1U=((Zp3zy;$J|VuQaC=D7LQFK{pDus9@}tkNcwVF4Zv( zA<K(M+~`ToR@OIqH4|Dn7pjE_FBOg=sUBVWb;7s6668Rcc)<#fg))X%rmLIEzM$%m zZ8j?<uN#J0FQH#IF>`5lj=8tY5&5%{|4<Fo-TbSo78LdL1#}JBiLX)M_}%1@JBvQO zCADLGpuO+u-4M|2nROW8JGSVQZV1wI{lhDQA{z2;7hJ~ZPi+V`I!tohZ$CSb5tUz6 zmEcOFogH~J^--rX<LqYo#lL((Lr5p0OFsjcC#sMEh97zD&mRne?&EpLz7mIr^7Plt zGRB3+go8W$o?P(h+%2w_z>J-He>?m^W!YWFb`sli=Ht_aZ#@?UPxXX{@SpJ7buW__ zfXQhHYGuu&6)M_X3VUQpdCes`KsQS*xv}3(E<h$z5r-8gzm26OvT*y3SD(;ksWmcg zA)}f>fAwQgJNDK1Se25*^PlrSb*=x1Xdo_e0Q#ecYf;BhU%y9J##w?2FS<Wj{@lS# zAoGlwMJj^|r%5_H1JPMI3DEfU`Gz~*7pWaV(C(I<6z}Ll7jJI(L6QNN%z1=4*yQ4l z+<k11l@Vre+Y3)HrnZ@W6m5byXtKBHWxb#Pd9{_*Ya85f`%_NiObX7>yhQEXL9_y9 zm2qDPhhR)%{ECG#d#n{91w}C>wJll<+X}_l(`ihk<kyNgpByEX`~H~VtNBi%s!3L| zyV@F;IEVMz{O8tK+7q85bc^^2(8ST!FHR>sldWF*eo0-h=>OAdCM94F0eZDtUjmZ} zGGuHO%;DH35B-``HcQNz1D`UXC=gW_(AMF7PwBheOO9c@Ve?6jp~rKHL}w?CXb&!E z(6||+t0N%tEC{XU48QepM`vL2+YYE#@>tVunO7M+RslC&cns>TI(uH#rWS{W?uS-F z6rTt8Zefg*-s<)Ke3Z0C2t-XD=bjT}hJD;XPIz$BA!D|^!Bgq4d6*7lqRL)S&`sP} zeeZnC;xO}g6{iwg#`$#UtkLrEicb~e%Ky&I-t`uWa{%{7)|j&5!IK7Szh3br>Zl!* zIfoRp=IzJ#({g72n-C%s!3fR3-@>Lc0O-=`WzL9V?_ZI6Ga$Cz^p}C>r?8Fi!{JD9 zmo$0ft;>pIT2Fm-X=`URZ58<D5|VfaE-t3v{FrbND9V)+k<NEv_HV;Fo}mS|1a}Da zM}PG1!bY8?I&bj_${Vc<m#mL_skZssyW}0qs6;P12xzwjLO)+(dEAGP7*7go{zdq` zb>_KaI~s4!M+y6z;EcLv!4m>{pT3bk(kUG_Rs9>&E*q9(ouf?2(5!X*TPg9!_jnW@ z7~+R=#_XXao<INWkO5-Me{N>djuns$7N|t8`PY-*$n5oKi-;)0Se!KH#qm={@Lg#1 z4$PF!wtwhl@PtsM(QwthM)2=V48Evo#5Wu{KasU_n{wr?oP&0NeGkdj{H!85Lu8?Z z84o@kq#)cYc%&yhf}butG_>Xq=GqcKrsGBGgm1nuU1fB*yGr<BVq}!a?zf7ol^Vz+ z?Aak})%*FGxbhog!i0=ayod8O6(J2^bsz!Q%4ge;AGWCi(Um*cQP=t>)I$>1B|Ay6 z{C&WcD1udWz2R(pG?;#@A+b7r^gB@xUL7`N>M%g%ozOfB3@9~*4h`}BjzeLSo_b+C z3A-?<*5H7EU?uC|YoO46PzTi7pKS67o^HR<x2vG=U`d^{4`REZ2h=6s5@n=1u#LH{ z+TAdPvANvoRX9Hj68$3rjCixe<nNu8_;HdFGSG06?E0NW&u<6%Sj`S8G}}P`pJO>5 z&jh)6fK*X}!npWL<!RxDl1sT#ol0F}w{Y)gf}!KBh@DUbbIP<04&$pcfkoENI$`Cv zbsT@+ldrye*lXFMp`X!>8Y$>)^{Hk;#iwrE+QQ_YfqpkW`u8LhDS_)#$A;n2^y@`O zO;X1txN+oQ<sXseNxOk-bTClWec=F@s=Dca|3QHRYS^(Agdn>9qd61-iAkIc<{CD? z%279aF$|tpCqw$jNJByK;=3^lT?#)qmWZ0?SrXfz4{JQ>)!%J&IYHfee>*Y>^-Od1 zFed*FV!QAYL#!Fj^7)5Q<(L4k?@$8BNLGsXEZ(I|GZAqNQgwasz}B8=cgJNh?c}%6 zcZf+K-|%%E1YS3q&y3J6bogym)ZXr0>w&{KV*jHHgwLZ$T=~;v{;**8|JlK8$s+%6 zUQhQ|4s?JQfCJ*qXczn-5M~fWgcHI;nM|=H{9)l}p}OEXG9cI|_!MXF2mBy>CmoCj zfo?we`rEYCu143o5w#D??e8j-2Y8Ghc7~=dwB}cFy+yRE*@UvZ;DcsXVeWyGIRC7} zY44sZoU(p5xv$Im*d=or&&u!vNmWV37Y1@v#S&i+qb=V$bzC~+Ni%n_Js%f%=?MR5 zYNbn!{SbF}-lp+kg`>awuGm1g(p5i4DQh{o_pAahqdFtmpbMe4S>=~FQa58OdR@bo zbaG;gT-droaW`<BY=K<BsnWVnsJ1*)PvfEhQh~zl6C~jgz-S17M_SknyFViPkZH!0 zIFApLICJlEKGL5{QG9LT!!)ATXAF+4tZh)9+cq&HDCY(D3D+?nHM<FHTM%p<8KWQL zt8fxp8RJdd8-haI7M|@=-e7kvbd*U+ruHZJ?s)bR;NbP^B@H~Kxieng=f7OezVyIa zS5DcNf&NlUcG6np6x$jVKX#PRal#G8@B4S27<NEiJJiFs|4@y1THoVRE$6(O)Hy`B zf);zC#@Q|C%EK<5p`Vxa&ebu_bB7Z2TyK1HG<Y)mD%i>(oEqdhVmOZ!ZujT;3Ev}* zvjh0DQQVtAG<S?|Kms07;v|U+K|Q?V{NwDAcjNX1c8`~^6S)h&|BJbQed>8nNVV5a z6kNo&9(F4|vHayKJTL6TZ)T6BAa)76QP2B3DU-po3a^n9Adj_Py~&>IR?!hW9s&-2 zP85(3L}y|{xsL#^e;K2Qo%7*nb`~ge?>CO%9E-8h1L3gMnZ#SzYtVJVj%AGg&+(8| zdQXV({&2qm@=#Bm#bESNjgjv={Zh6XrmlRl>AIPjhecPLnCf;Jpqm*InSKSIKYe<R z@MH!%-*U*-9&w80h?gqyq2yzcbvMYCA1V|ZF$&hkr6xs!(v;ph_vHEL3mujHGPb38 z6~+#30pikR_n35*(nx2Q0yi&s516YGjT9AX3c(r64!vFhY6On$4EWN`i#T=mD!q6P zDGCuf#<+QVXP%X%RP~m6P6wOg20cI0tI+;ooVGVgthVW^?;o6cJBy;i<JX!|pi@{W z>@GBV$*%cJr+_CAwh?9v@d%E5DVn)Vl!r1;Lty4#Dtue>$vMH*V>KZLhTdm}_;OE| z6ik;lKQE2X2Nl|L=RA1ZpT4@mp~QADImaR^A@jY5#a$QWzf|%~>g?Sgs~-Dqz0bnh z*wgBK!LhEJZk6Vb*<EG3a&%#-ZI=U_Yj$>#rOf-J&)2LRi>@)V%{I?SbK5cdWyrOB zB-{K!rt88ND<fcWbQfeW&@<!8Trykb#T@G691GQMm<)#*5qE14YnPGYtmqTs-JLRD zCVZ=ZhyR}R^sWpJE<t?_vA9MLJ21GX?QFd9Pf8;4+kR+9*a?&VBl0ze+xTWKUDoqy z+QBqY>0<*<vf*t{@5x>%%=-Pf2e$LJ%$``Xwo|%JYDK6`Wx7Gwuf5ds_8Omx_vtI2 zd|Ursu#?_$PDN89sl3<=8&p>jNZzKd^mxhq)%ds4J$ByjzB$=^qs+E*v5@8*CYHc< zT!#+ARuAwTg+PR*L`M)>jl&tre1#ZptRT<ARVc>pZzJ6+Jiw9TKLnCAnd^qlVoZBY zHxAI~OY$)Z?PRZ^W~YiJZr|@hQs}{V`vWIOL4fdO4*Dx++z%FX*+=2{40UFufM*y; zqTVImyC&QjJk!v>a(E6lkpgrqJe`j?y}>Yl5VkIIOnT<V;`M&~`ax2>xSUjlJ!)6? za%9bT@=f?z=WkctsKP02;@(@=jvO>g>v*{C7LuX0-~2jU_ul(Wd%p#fRpy8tbHNT0 zALn!y?1wV5GK&6d7a{L>`Hn>E7%CLYT>dZ4y@c(V{GaCbwgU02muw^d=%RaMCCcP% z`M=t{vs$fv4V!BXZj?VA9#J})uK(7r{M)HQjkLWNuPiLL(Z0EGs=9q)f`|NVAjCKi zxpAynrmglz+kBM;kMRd4zaZHzj+5<NH>j%o0wjvr1<AZ_@KpiX0T@mj(LsPw+6+^s z53zBaj&;_mulJ<4p=eJqCO915^I{er*XWUiivTYi55Bc>&oA<MW1X;AUvcmk(~<K( z*n97wrrLjB6e|K^kS-`uY0_0d6fv;@0z#w+0zyPYnix@#03lI&5qW9y3PMDhNQqLV zM!JH6)IbnI5*6u*LRgaGdEVdLJ#+4z*)#i|eP_?vcMeX3KLRURS?hVe-_Q3`^qEGH zt#F(=+q~mAP_8KX5+#r~X$eHx8^9)U?VbTO>mg3~CDw2aDS?cOWh!NHx>Jj7^~vk6 zex!+49c+v>7oC?EPL3+@h?{o0ynk)mnZ1d@{wCJGluvK$(kkLVdKsr?UjEopFmQBn zc~`T?V&;W=-v-;AisCkMO{^70HElGTUpbUk$lDh^Z1d_>8W4|3-r@f2nP0nf)tCD& zf(kqL!Qq@M8*dXYzY&vjjPYqdob|>sP2fWCB0fUCp6$UvdMWMXc*5%_obDK20Z8vP z5b8g`JS!Qoi+2(s#<FRsn1jO+60KSJm2U#*$;JbnVc(PKIZFq}=Bcu8-ww)R4f;lF zo^E?Zup0sQh{_el`NxLx4#}5yo85~6gvzwAu;KYo`d(RdWSmLCgN<3To!{M@uJ2gh zhv~s&;87oRy|&p`!|mSPPCxt3>w%Krk~+1Iev1zEl!<NgxMOAG>-Nbb)8dqlMonfW zjorZhXHR+~(8c)z<xp%hfvbccCGSN@L$r5DdAx)Txti92#clGWr=&dD2wC)IopWV7 z#p&UkUt)C^x|C$tm8CN;*K(E3n?Kj)o?Kj~9J^bkFYMWlA8DLzFYo}L^bvj`m?O#X z^!d-oGg+<M_(X*A903Nr4+l6AZ1j=`*F?~dRI24`wTJ0NGC(yuldH!d#u4aB>@RmF zvBT%8*^g^q*<_;<bb_54g7cPLe;{k;{a-a!8J885I+<_E%zU`ByzpP?qEXr3i$pFJ zJtz?&ya%Kq54_hW?d@(!eXn}U^N){|DZK8A@D#Qzvast5bZ99$<e2QodO2@qSd^G) z&A#{Xn_5OtEK)JHdpO$jQ17#djMu%7)e5h@@?@>4i&m8!x|9oNB=sP-2bHBjDgdLQ zF=<i6eKnX#eIS5N0BgZkK2dNA7xtNa6Fs31oSEiWM=?>YBJ7?vD<ZY}BD86y!nFsX z9LETGu(FC0|Kg9>KZ4VLN&7*OqvD$+OVdt95*>NFpa#%+4Ix=qF~U$1XZr|V0xD=T zBD2g_l9eP!o2^*>ZTAH-ti1e+4mOE?OG$yn@2>Wu_ldco?S@trr;Nv3&xd{=lmQ&g z#Qw|2m>X{Z&FLRm9H7ozzI0e<%k#~xW+P-z^5%JT-Fg(^b}U;xp`sZW=1q{gXw^!& zGkqQfC6TvHRGF~&Fj%KQKWtvVIA@R{UyFuO+vKu>o=l(riFx={bKyw74fd76E6c#X zBC9>tKR;*;ah|Cr&^InsrRZEVxO#`^lrMi48#+pIslb_j4X`PHIoRn~qQdSE@>50l zwigz39>+26ja?mKOST?hXr1VlkGIb^%T?H$N@h=hT-)#&67NKbKqQwdUXq9@;sZW} zx8CF|N0$ZQ7a%+o!FHo$agr1h8Lq#gwXHzw;d6ctos$_De$N9_G&NsMZ7aJvf5~Nz zbnnh5$uQc|8A%=(J4Ax<NSi&t94F6lp8jM0;`K?9lU&7;$!e>($C>2E@I1Mfd<cq2 zuK*<dcKVByt%81{l(yg$_WFSJ+aVLrdzHncnNPai%qv8<rZ|TfJvA>^)KZIQvh$R! z%s0&&!I#Akn2HvS?`#ki+Vf-kHrT(np!@lMzEXZO;P2}JPEtfl&-EjzHKQV^!|h&I z{CZ<#>{f?D9ZsC8y=Zj>XQLU_T%y4G-gc=V%i}I@yKr7COfCOm2aw73*7$yd9tF6F zws?zO#=WI{oU{L@<?>)b(P_~y2Y#RC=i?OtrY6Nb4&Jxy8YvNYa6U#TVtY7#kOxBx zR5*1ZOXs-z{T2QQEhd+>Hn7MYctzkeM4@d*s6f9iP!J+mt~2YXFGnEtwsowg!ls>R zIYM`{^nbvq;KcDHrLM5ZJ*WvXXQB}dC}<BFb>aZ86`7D;-vzQL&7kBGeD5EjI!q#l zw}tMXltPu7AwEFpO;6XyXwQz^j4~T>y4W-l<hw=`b-Ow~PbqxQ7C^=-@q_OZ*^L=N z!8;#6+TrEp5o;&1RcV*+<;xD54_-aj1WHebokBuCgeusj%s)b7GV!Gv<s0(jQ*VC_ zW4}1UrHs?k62GSfDP(F|#>$p|+k2ixb1qV-WYxEs?@!+PqDl|I_TD1pI@;F9&zSbk zBbSb*<vLS3JJvJu;~g{`Z3TCbGhfKmbDT?W`Ekgbs0m?9cHF~N;-vQs2~fWFLid=& zHpNfd3A=c=SgK$RZB<>6%~P*qx2C@QF}7ak$eNnZmY|*dyTq6IGuteolQ6XH$n|6; z3rPc~jMB_e8TYfVc3*t7{c4QR&RHSPB1BW*_=oTEug0mLFLg4$8P_)425D27>UQ{E zShmBQyW=BOyYkGOVAXJ4lcr7PT5a^7i`qrnvfHJ9eC8zER25i0(-^)yZ+1A@r7n5a zPWolH^_=Cg5#ToSXzAbL#!OZiJzeBj<YvS^h{T|Eqm{5|I<W(_W$x|*LYrOxs5aah z*0G^RR8+B29bNmHSP`i6@%5R2&6vFT@87ljeTv>bIX?N*>Jq;h8GxaNcF`7HfujmR zs8!4@OVGtHLI&`KyC|(1Y(cs~w-!p8r@>D(5n%*KU5)C{pNMUG-NL3Mn!pKx`?{*@ zV_ih{3?%=^>JW%pb5HxxYK1#dC$IAV5(x3mQ%v@8cj45?TOs|PX(K?<Sy0*yyg_`S zg${}Yvy;m2NjgZWLB;L%+XT|I#5&TT?B20cuANE2z8i{Y=L{B4YbaJrfu?GT9k!vQ zn5(yBkE(k6pIk+t4F|l>mb1nU8t8?^Yt{2F>KZ5NgTu>$5qd;AKm*j>?}aeXkd|CG zU-H}gzvqZM$Twl*Ic?K&vGd!xZs{&AJTpj&X?@kXxKLT@45I07{jtLp&pMmFxS@Wn z#2`IslPNREc)Tu_J5#bmro)o*)S(Hs8X@VAkhB4sI?TDls+b}JaYf(^aU1VcmVG^& zxHwsfm4>90q@W1;$>Vmkj*Z-yW)}mmp!(Eh72=v@;)bQO<L2m$AD1}#E*fU`sFm-O z!6`(^fWt@j=nJ|Fxi%^!FylI9s_%EeXC4;{<Occ~8+D?@KmtQGV)vAw7rBS$AC&FX z;dXnW>xt4SP|+DLSawg<V4}!-1Ii7+=1Vuo#uh%Iwd2)?O-?crmS$tE_8Hltj$LT| zBb2v8_jxw;yL_ci<-s(uq)?{C^E{EidbIUT_iJfeD;}+SeIQEW%=YG4CJgA`|79#$ zO_%G23D5mszLM5jort|-*Z=XA^m;<&^e#`Ie>=EVX7zri@%4!Ix5y3PE2-G06|d`& z=m4#7PO)j7L{KS8g6<34Z2Nd<;QGQQwJmKih0<Ayj6Xt&#gQvrguO^FvXq~b%l3z* z(B!4++C&``gB;cDR5FG2?2-%EYw!1R0~TML(oZQCY4!qXqc-I&g_^#5e&jCUoeR^N zssqz3kTHde-=ZFAp1rI=@;>f=Xo@;{&upn>A4zr+EqT8&W%!Sf@ITvnA1AEJnDCD9 zfGms<cQFZ&9^@T{{Aa-E>C{kz_<<H;N7$*Ftf7++`hV$H*MD3@87@mZeIV2lyD;$x z+0Qs}meb4@-*tE`h)iKT#RDcsl3h)T{Sa;+j(Tf@+zEd?eR!xZ>K<$Tozj+i5GPf7 zK+#aFZ=Y7j`@ZnX3E{@tL-a#7<x4inG9fq8>TlGHE@nC98_bP%ImR4vFLJ3+4T(U9 zOx&iKmaY}a{PyG?19p4@QOJmM8al#2gBb5j=oIT0^JYxJ0#XF8+r}gTd+-h=ZJ@#; zMFLK;a2p{}&IJ=G{>6y0QM8@{f^@}-s)iTk{|FUdpWY;K=@iO_-xy3x^t#4RCtqKg zbmio8uQM;OV{=)Kvz$=q1P)TIlmj_U{;wb@n~|DKYFlN^jj=5fTX%nIxEEc$S6bt2 z#l?#n-GvTmcdw46S%;U_SN}*Z$YQG71(nig?Hs-`s7c=%15UYE1&{U6f}Gwfg!~Je zo9#YZ0dQ%1?k?y$Z^y^Ru>vsv9fdv=zZ0U}39{Y<oQBLTM&%#A+5U#1<+Fa+6CsoG ze2&1n5Mb1^?KV*F0#I*=U*f{t;>a>fWuK*RY$3gH$&@GWu)-XhA)IQdpMrZ}<38`s z><?MjfB&=$k^8y8)<xUQr{x_0ur~i^HNZZd=LYFtf~|}zrR8$I6M}A&e_khc)2P2V zF8rJqQ|tx6cRR((o*UEmEx^(k!m%|d$@%1Cw1eVO?e)eFMrtP;jP$a^bIX5zAf_rb zYgzLf!9{&d8(C!z-kFbJL@YFB$i!E;ZI0j#i;}BI|Lz8Cx~K#8EYsrBLN&_Ny%#PE zuDuU2d%3X$2c$QUyeo6G_?h42$kmj^de%w0P4g$^DQq9OV?fg(*Qw*^91>a&6zcwc z8nrYD5FR{b$SDZax_#naI@*<zf=)bguQC6uwAB2{i3q{`KksnCmfozby)Xs_tXgOA zBrwD=r%z|Gj0B=f>;nG<1<IlqR}AaI+kSZb3>(H7hbSeJBwlr>8d996_<a7Sj0uUS zdh9O|VP1qxN0=Hg+20xr1u*UE3AF=DD6SWNT<XhpH~V&XZD=rr1)r(?)cZ&!kf+Gc zs9D)EM}3^}K)kN;D|IH|qxzFOi?D@Xjb`P`z43|t%ZoC$(vlxP+Q^*VnuZl}nyv4K zLz=TZsg4;!{0)|Q62XrcJFNm`?s*DpM@H%UBWu*V^-XkB6+<9vmbwL7E<V7^w=h|5 zVEDz^fj%J-CBs`i@WlINiO+tw_TEPcrjoN}!ba}$cFBN%tM*+;-?CaD?z3Eqm~G{4 z;-?V2kQ1&KBeX!&5fIn%cGYFW)iwyRXmOqfeV^-<t)-#C{E!Pnj8?YzWEGg&EhHrr z9uj-sDY{`yRsg(v4cZc%tx`)`ECHuv99k%&S~X6?;z41AN^ai6-3M*ob8}n3+BQ~D z0$APRDRqI7DrtLo;>Eqw&T$`OCa#9&mAxReIUtO(;BtfjGO*Nz#p=OQLuOFp{xj}R zLm6wsw5Rz5su<@vvC9teyzS|>>8zQE?N>VeRnF{v=6mkY$3WR2R@Ec4GdGDcT`RD7 zO8In`cj`C$NJiEL?#dq_)t1)#$h#)yAf5+0d42?E#aNN>$&+P$PrXeEtB*G^e>LtC zrk3Rwy}e;1*v*D<NV}p<Sx-B$z$BaZo=xJ(VmKm0hRNpb$Q=k}Fq`g#qDm~seqj|k zYLGt@Ma6&DdihS@)i2K&@-1@1BHPo<EE8;M8P<jeRenEi+Dq!;z*zuoolw6_(FV3O z!yT;%{u`6<l78I^o;TNr=LoXDjRLpzMjp(u1lIoiyT96}<$talWn`N@Ea}>HIzi&# zHg4Z(7Uv<|S_$ZPWk0VHKzz|vv4CQqu<cL13Vb$#tI`~&J1!1Cf2??Xle&n4=A%`T zKdgP!El*uaUiY&NYG@@>uOaUgUah$DHRNqH9iY0dRli>FJ=c{3K9|TW+;y7VJH7F6 zI(qQ~eKJz*HS~~s59eFQc6wwT-5}BTM`-5)SJ$fiyk1?^g|T(8g&CAe`J3*y(E3!1 zo(SQTLlX@k919H{qAkfV#nqT0RJ<&{_kOnCq7^HDsnUGoerE71p5EJ5D7k!Y0KRNF zg*)ekwTg`oEf=0%b*eqLzweN9qb_h@0FDg5HERf4dPsRU4VN=7o9PwGkbioI;?;ZM zTgV=Si7gz=$Sg3OB7(FQVzh~OsIqk_?n2g?lrFfe;nSy<H2|jVBFS64#Ac!YUcT$T zemOTGb3C`dlaDr)Sv>npqwDN5nd3Scc2>(SN)Gcjh0k7D_oK=J$Sr`%?{LA^R9fiY zdpOO4SPQcr+bn^|64wD6XZ~>6FeL`KTMYC>96SW-b#OjI+1nG?0~fIoYE~<8e;Iq` zzCY5hwYt4opY!~4J}^KW<etagWNZbI?{e=3lES1pLx-@-2S#~Y7~+U4`(V@NR<U*` zW4k=#BTLV=7}d~K*SiW_q&UGpK4C<<_@DIGO3X%A`GgBYwMz%T=>_Fttlx~T_PwwE zHZ`}AS?=`2>Bv{_8&8JMGf(B!lzl}(>a+1B?2tt{?rx~+dq`sg8@ZUkJ%&wW%$w<{ ze%zq6k+#j-4JZ>^S5*6soa(#T+C=(*8fgb&kj?)^57-gt0~_WH*|R1)TajR-+m1@0 z?K4b)Rc4G4L|etXW;9w=pqKM5-=7xrw<@MELQ-%?@~fRE{iOWl2)8NPr{4_Swr+Ua zB;kT$h44)5nMnRYeSs*LSx3^w<^Bj2%#%?+p2I%EAZ$rDVdFd)5d6!tgZx_WkC6E& zoW(`~j?HTP0Hgs%eqw}cXhF&=KJvu_<Y6S8LEHl%-rYo&JqUV~6}%(!;SPk3E`<Ic zp&{je<zhiPU_!EiC|eaI3h~|nCSxKF>|%vxgf<j_*j>4s!QYOs1^HRh(7+#|3*b<Z zM1Ts_a{wawV}mrjacIl0EdmgqLlmTfV{Jo%o))tx_D5)N&HVqooB#9K{Fhv5=nkhP za3Epbe}E*UC%6@|mR#w?9p}oYcuMuqAYoSuz*5Ji#aZZt<})lR^=HqyXvk3yBrCCt z0Ld~gSY_L%io8_jKKt{r{;L@-S5I1jOSrfQN16c}3+D{v$u40CFC74&+yMPpD)%(c zmJKrb$a`_Oa@vjWSF=%0W5HJ8{;t0Qm@W5DR#=@KDva!(u*og?LY<rGY6!TTpt6OD zhEzj%(yp9Lek)FBS`FfYx?__60_YaXlH82<%mBzfX+>PG1GDE-oq=<+axv6C+aWMI zG99_@SZQ(~yucju(<r_q{e3v8ntU6j1Vz!EfD->2?-J`-V>Ch&!f<s2pFte;N9x79 z)~`$%Fkzk=8xtSo8#JDuZIpG%&D^H$@T&`thPH{$U+`E{y#st7Tj3CaG0=gDM~m{D zrPCoNhWA6BGg!zvKcDrK{6q(urXR*}afW}t&^P;`5^&P4L9AiVU9)c<f1O4iB73fE z0?Zu(7`Eu+j9Iy@L=0e$;I=~M^p*ob^%mtfYloVJpHq{@Zx<vWzEEI_{p;HK%QqTA zzkW<Rf;6MFgNYD)50prlSh0Xa*k>&QfPu^S_hnGoF~?E}7{kY<>PZMG-zN&{F@ghP zR!I$1W=GyJKo*0sAE$;(54%i_L!&08xcA<P@FuObXWDcp@v}unVfUh4>iu;Pu7<;& zKfhR?7y>}2txKVo3LirrARGn|%smNGKZ+VA{h5HlfSlV$L5#7Do&=sP7dV5BdDXPL zp&`XiT6IVQLf>uN4+lq{P3a~I;iQntK~8<X-x{^6RmcUwzl%4XA!{%^>9W)4UGIb8 zaln><9E&iB>JsdK4Hipp6c^0r70_|3Q+o8@BSmiY)VNr@(D;JO<FWwXW>+gV(Nr$2 z%f#w-DBHOEnOv7~#$waYetkmFb=rFxELby*d$`1Ig=c#QjVsLh$d?~mYk_yT=#m)7 z-G15lIPmen=E7p@EB#$+zsM@MrxeUNrlaS9M{2~~g?&$m7%z-TwUx4_`CjeiHivBX z3ZzEr2=HX;#q&JDYOMz47trRi$vqfx(;4IhylrFc*^@A0J88Uk*fHfQYdv0fbs^q% zm9!6MPg%NU5>Qe09cyqLCzpTq*4xjwP2LmyU=M(R%ZdeKpx?nIu?1a9J3k9le>~YZ zR>05`&fR5wvtQBL)n?3*YK`ntI95;uMV)TSy7+j&X(;%lV#~=ZLztDIGH$4jo|=6( zq%IIW!S>`><y>UR@uaKU!eld9W3<IaBLF;LK%z<B-qB%BSW~RNEU`F+ilT>I^;4){ zEFtwT-Kd(6jKI);eY*P-)WxUdvPw*U@h-keFR(EvJ>lWG;t)u=T$SeXvSOunDcl)7 z1(@<NeraX4<}!NsMShg8`!0Uv)NyPmO-Sozaqb~}A8lvrek>W>8KjAU8O%e0CNQ|x zt!L`(!&$RP?W8R~&=Soa46>v<cnP38z*#ZfjLxOljn#(tEi{$hR)5=&W^tn-^;y~? zicFb>Ej{E$%o8`M^8c=ZJwR@&K>Cs71Qe&~<2BH!h+^IiR!9dXTJA8U_%4C1_Wh5L zGgNjgr_!LVULc<0G!ith7@_DK-iO*buYWK_P5SL`<Yl%IPiC08!;3Aqw2ga6(A_AV z{9QMlYmD2+hA*kIkI5HNts9_P?s-22r&wmTs=O00NAxnTexkkS*vsD+b={J?OSBDz z?L;Pe=S)v2ItA!t58s<Vav~PC{db^DhT7=Jtx5luGz`88-Y{iMX#)WK%u6&2MVwf1 z!HD8)_~ioFB(!$p$pHZ1D(xo#>lW2)?YKl()E^-co!|kfFBWfQE@fNE1|Mp;cA7Ue zik!(I@5HOU$pIBNo$HF9_-V!cu!^EeapKr7czWSkCmK@mn|Vs<?e1H!Dj&}6oL0y+ zdH$wu={aSLEP~&R6Nf~=V%dIz1tJUESE8sQkB<{CK%O&tVp!Ss;3P%yQF3fhlUGx^ za$xAWrX$)PucSpZ?$t@i%v0Vis#RkTqID*n7y}CRlJtV$v(c|wfBJz{{x%`ZT(uPJ z9G8K8?jNBbWK|YVDxFga<%{9Q1Sk0~&~3CgsP^CVEWxSUX~?9Ibv!bAs;B=Va@RL4 zMJ=JuC8h9EY?eTDjvY;eEC@cx2^7ch^9{uHAGGSMwiOA$zmf8=Kpi&f8lY<ay2xT! zO9qH`&@+Av$*SKCaN!<I<Q%#)?Pl5kBJaQ-p)f~uP$)rmK&@VaIX`6|m=9m>URlWm zTSdx%j=4bO4T}I2{VU)czTsXHOd8#Gv<MoY-eb-_s-?xc`|n#9hb@B2Z5}O-JNZ}x ze)}s%6!}3x&60%l1nk`ttfU!Y+;BU3d#fzZ804S%bRIA<Vhy(;6j{2_WN)(zFb*o# zVDgIC!ILp+CJ$zX)7?`{!=>-My>+x!RH8^AUnszY{PIqBm5T2?Ng-*H8R`CC>(Kkb ziu(NBe}ua9Z_m;Cn_U}iOo}pbE&6g)wY7+z9M&H4*F%Ml&?2|V-hR}15oM`GtrC(Z zv;vqJ?fNVvQ$2qDH-VMH(}dzGR{D<-ys+k1$h5q1F5=#jz1E=*Eook7j1Cnw`K*#5 z6|(|F88~U+gN+QB`l4J|z>(u;-kSU<@9kzZlM)&9<NJwumB*udl^CbPDL*oCrr^n+ zJ|pR%x{ka<XhSw1w(rBh@SU`(%7lZvBUOh4+aSqaH$ml%m?WtW*DXF6a+dIC@#VD^ z>;H!3|9AS1)FlV#jps1niFlPC#5(mw?^3LJ8@aY5)}TyJ40o25l8jjkkfO^ig|eb7 z;~dgD=;mFMQ@eVXQ9H2|ixI=^jV$lS2Zq<gmNq-<1~pAuo1Yn})VcPXNRI=y>~UQP z?YNoQZY?(g$DUtY;rTZ1Dxd@Vlwon(n0so)jj`oxCsSe(>a7{ra4@+Qj2ggvNmxCH zu^MiUYhP%gzw-@$EIGvggWxD`I|S#Rhs;nyfJJ?hMP`Cw=Uc7^I7Z4Q2+fd-9fHmn zR-rozu+>&g^--5D=&C!sy)fp}S@DLV)`Dv~(fRGeaN^~cMtBRf`DhnEXKk>uuC*ld zZ!h^<;j`&MWfdow6rNQ@qaQK127#&!>B>9I%6d>EZr+7I!A8ZI?4d^-6mA@CwqSh1 zTx2ajESfYc|9REVxEs)6TQWrByYif6PTN_$Gwibf7ZKpFMczGs78_0%?t;ZV5k1zG zkK4~D5pE4c3S{eJPG8XpFi~gaa2^f@^;sRdBvT)$bv|}tU)q$i(=~#&K!nhC3OMWx zhzY*X>U-J;_VC15D_tEdPlowB3>+smLI%=;E<(*sGKklM??HMKHea<U`LRLiaSQJ> zGP?|Ynn69H9GSQKq+6owrzcGrQrWG7Mnw*cA({`5u%`P9heI+}2i=wwTl6MJ;YN?% zPm4Z!BXa~X^i09=k>TiMc{v$w7Dk9i-qK4T*ABbkjQ+VJCQM7AdzYMR2IK|Pcq7*L zVq*1dN4UnvvHB47ro~|YLj?LOg!<V8tqC=-R!LiD2+n5@#NRwnlavBLJ8#Uy`a5K` zdi#<(-MfQHLQlB*DGVf-STk1S{f!W+?3GRe980pzab?$c664*~n>$+%vg>L|Q*P6= zFRvf({}r#Zzpg5hN`>DBjeX*AkH^P7Vhs`xy)b=z=tc5b^Sy!3Hz_*^{p+g>(b5}~ zonGpQA1ZTfsgSRHb)>24ep5ihDc!|1ckTHSxVH_WKX=7{s(#w0moxT3C3Dp&pz3); zU0qAv+v?`}$mRZJk$}pI+t(k5`{(#FeM;)zMqbu3fJrqr9@gyblsI#+<VD?4q<S-G zjJLi9;r0KzNK?6bxEnyq_Kn2^OuGa5f<c4^KdWYP2cXZKfZ{p>eUy%{&`GtZSf%YP zBLM@8>x?PScVr2kZ!x-$9$K*~2CW5W*|NTkemUZ%z925`XmDrsuwZz6*df#To`sH7 zW<~g|y0H)Fh)-qLwVXJ3YQf7n-7EEFmg}|a_vgX^0^}<HzwGFM1NGH5Np^Dm1LU=o zN%Jns_On#vVm*6e(IdvWQqM4FAU02BcvZ)E)IpLG;xalF(|mf@jRsrv4X1ZxmTitH zU}zCz&|5b@N#~^V-yl8p9@c9L;UL$}f1Eaml00jjO~51W@1AzT4rsCQ#^%JX$D{jy zdHQKyxp>)Z>4OdC`->$gSN?eTRd)sAuaQXZ*A(i&L#6Evz{dqSL+NlGl$^$y0H9^% z9mQY7G3GW-l`alH{p@`21)u1~@FnHQBZ+H%he*A!k^d_2=l}ELkfjMeQ%71Q2yISy zdFbPJm^hva#PeiU-WpTrG))#P4Cr`;lupvt$uD1b(&h<(tmQM1bl0ry@4pdyjxJs* zN?8iVUj)U>_0<A8aiW8swPb&zJe#<9hI^4G&o3}k%(>&)h>Bhh>zZwYQ+33L{<{7q zi4z8Ou#X9D-1WHs`M^<)BL(U=(dhZ{MV<Rg^44gc-c**ra-t_Cf9;v9T5{#AFovxc zhWvfhoMd}?|ARAIcmHL+Kk9ST7hCiz_|yO83-EuoA;I}F_y#;Lu<)b`+Th#haKFLU z?Y>~@NWM*uKm4Fiyg6?N>zv>ddix^grjXZ)i8t2_B3`bE-upqleWK7y<F@}|<L?!L zNJ|l@9R~QN<e3+QTAg;W<YRpdWCv<Ij-EW#zCG3C09O~XTkl3nRt8ao;<ab_&jltp zo1x{ib#ui~;O?gmZ3?8gPDB1<*W>Cd*Hz{9Pzi$F^&dG^3QCiQqp9mY>P8DJz|;AP z?*@)4zbDq){%0!tf2W_XqB_ZOco@{en(d*92oCs+2@ZmE%9U+EufCF8Bi78%y2{FW zw0d%77dGX>+n$4pxp2GYnE0<E34194rb2iTaMdAsZcGt`2$)-K!fVdo^P?$}J(2Nw z8bG(HJ)iaP3Af9LEMgjs78j`Zr|nyA)rZu}a7Wp9Qd6ZrFfBYU^9OsrWU^D1oQm`o z3&47G1oB|F6&QQB5uig^kp`FvGQ;M@ODT&%vOl;s?l)563EmO+oElQ!_`k#s%zRt# z)DzXJ-1cd<owReje#QL@a$<DjruCzZ@0%Y@wzycOMXV^1c$NdB)xXMVB5N0euc-P3 zpMo~5f`5j%Cv20pb<h6}{{K&T;=oVfDR3owju-zKLYwCwcLDQ{Lo8y?F~KGl&NaE` zu|N}BEa+<kT~aH2qWsn#Qt(QGG3d=|;_GtT)?$>4B6_%Df=@pMAJA1HZKwwudF8FD z?5t>>=0<%NT0W|wIh0-B(^yvkLiR#ZzIrNDA8TNHb}1bh6}sarb@|GNlltf^1;G<y zEs-Z31t^8jf}lx9$b&1}<-xs+AG&K+Y^?@Q=n78RA?PBC8(O`TU!th6qp-giJE}7@ zlJPr#P;QG(xytf?lOh=xz3aQpVZvCC%S7m#w_byDsKLNn`l9-qc9Zj&udcgMAlamq z5WMem&;iWCp8t2Mf&ZVc6>)`MOZFn|1pMJ@^)A%JY-d;nC03W`UEqqw%JooPUQ=4- za89hpC0>k`7`Tkd32i%-zZP2eK`}3|A&w7JQh(DX0s@pn^?NIRf8a{uBG~^B<Bs(M z$u&MDfDRIy>8L~#YnC!6mt|NdtG8Y787->k)IpOdFTZq1xFakBxU7kbn(AgL!P&8y zcONruQI=)!qJrsZsRn*D!B0#NI#d!T=|jGWNg0B~W@dTP%4QGGjq^;{;t%Usn0K(r zBOaRKmBeDj3Zpi6lS~!R;O<!CM3tXvj_RODQtDH0ifv2!TB>!alX`S9mp_5Dw*MKh zYI193Gy8A%u)r4Fsc%Y45ifaTWuIJ2*JY$ff-)s9Ox<{F>h$>hwI4rbjv60*b~JN; z<njGq?aNl}XZ}-LU^9psfR<)bCQ@cFU9c45?&5}&cDNW$xfEMqSNqyD$C092F&a>V z6#XIl(R{Mmy4-)=*gdP*b+dSXe!1&uvO#3a3-T$P6g?EoBnk-i;-olb@er;wknQ2s zeFUmdCjODiAb=DnJ35X~{qwd$P8gL7_V&PZ(}z?8)ZEyFi_=%9kC8tcG*sLWbhJ*k zJ@w3g(C$#VmNO*QoJUh_`gw7=@3mde^?XyDm~-o(D{s2*6A|D9ZF=OJNJYT_t$K9* zG2WO8%mO;D2uAToM*!1~n)uzw2sw_Hz+p<}#-<gaGey%E2h@27@j_5wG4@;=_0~bB zdL%3U3$0ydO(Y5b*PtohL&NfJ#`oVv$0jlNrz^;g_*D7s-H#47cuJ>Pfy7~4Wa6RH zL;u!3|9AYZmBsFf#3{42+VXUuU_cSvHHXk+>7|$$R$@<fyGv6l4vrY0Btgm87|@?W z`eEGi=v~otPDs(T+ZlsfhuQtTVH99XPB2DRfh<<SIEAST+WW1Y&_cxma9BELIcDO_ z-NtifUnLn3Yv7Cdg^G@~tbIN4H!ylG5RKKS2s(Of;^%k_X;o2Nu1my^L&?h8El76? z0eJ%52pGQ(zmxQ!R!;(;C#@P-#ZF^F=gI_)NsiU2m<8#?gydFv3tBr;0%x4H&EEdC zxMA1h`ofC~zCe^-A;=7})phhCm;fa(ii+$eNcE%$wt^-=en_wfKLPBPZs!+cQJh!^ zQ}o5!D9$itNzqrGh3V0Yu6IhT_9t(}c^A317QOh=_?sgoHYH};onLG-6fu1!yWI5n zOj#fQMcTS6H_*-Gn+!R)a*imK=Y@gUUFqH3BAPO7wP>*h2HnIjtMe%}J}(kC>}SvZ zzZ!vR=7DZv!;BbF=0_O|bI&ZAnDR5uD|tC}#;atG8j%6J7akE24nXCiXU7IDRI1+h zrWW2>ZaZVJ>29kglnXX%7(YcI1NGQJ-Y;=i<9cYaHL*VpVNt}*CI^^gsup7@ncJr= zyWw8OV$OKS;?9)$C`8vEG4N?ju6R-U`x}-azW+DrY#aj!;+H17*6|u_+B*>2dgE6& zNUNYlVb(}-WJPj7i_)I@0VYijOS5XJ8^gl{AH(7kq|Oc2#17+748F1VXs{dV;-FqY zP|g6*m_*aIw@TrJS?)Qkg_-&e%vPQj>sT832SSzwk6k70^!cEFP=!)W^*#1RY|7^P z!)4vsU%Iz%=;(A-S$>>->QLr7{USr-REY}B=-KsW5tJbtOYY0JHMDajmQONsa!s4B z583|A$bQnoc#?BMJ*d?HjFVEK!|)oTMSwSMlXX*GN%j=TyE2(kxA5P9g-j~eqqglY zr*&aVuFLZ8rIFa4e4Z+(W@3Pu>>d+z5zm0N-YtgQ!E+niLmw81@_bnXj|7`=H=A1@ zWHb%eti+HNc&DB(xQu>1s{ODlCU?Zk^JtgKUzr*FB6nwY+ze34P+@HV?cLRC1pXMh z3$=CrN_fgH2yhtjEgmwxurGctA(I~7Lx%Bi|4DbMiP8O`7kwB5yBBY9Z!=$&iTplp zNKhHcZ3*@I(e)`V%-Ak2JMI3OjWM5%@Z?vup1@sr-3oNCF<dR{fR}hGq4v;-0W#Lk z5M-UFvy|x-&p35(DUcZPM=0UR%<nMF10K8(UDFukWCGgYUvn<o9?7L${ZfdmBvh3w z4zPxS!yK`7556BQqN8`6<=tT|<Xig6Uk;988zu41y>z>}fH(p@Wa?acmFwLNT)b`< zRQMZr(Zx8q=B{ru?F_t<37Z8HSakruL>}j1fhDo&ye+nxX|x#1?(b`J7lYc^7{)F# z69#fak&9>O@U3PWgpSd;ZoK*!X@wWamG({d3rii-wmVXZHLXnVbjm{PAD$Uec_VnH zqr0TH6hK<k4OqE_37|JsH`ipJ%yDh9`g_bjJjXxYIW0W&Ku}PSs`J&OX*&#APx&~v zs~9yk;+q)NWwUkbQ(tp4e`iz9k7l?udQX0JxKqQzL1B4g!Emycb8%gg+X;K>@_N9@ zqg|l<+`aUlEkFOa_<ejJev0-QN2`gMjcP^WwzKc&vdlS5NU>;Mmbk4|f9lGVJB04S z%e6Kr<TY`T_|FQvvKRCo3pV>xQlkQerb@9Pwoj0|vG*?o%d15K4#NG!mo`DO3K?JN z$4?gLX8YfKsvqrVcIlZzzUs}q(vmMc2mVvC?|D$7GGW`Gg*F#r8=C4m(^}+7$553y zVeC^&wJQn0^{kjPu^jZt^}?u5<~25@Jvpp*PcTdr`|91X+{(F%mYs5A>zrc*+cEM| z0$0je@VSu~-D<#7XXB7t07-}zogE-}$|5l%S`(?undOj^U4tv%e}>cp6LF=vz!L!X zG}!;gi|h!)<%(Ky_@xVL1H1ebZkFy%`+|#NxpI<6OfEAC68Hh)KAyPVI~29=or|46 zLI*;=@+)xu%=4xz-__O;9Oi9e_f5ETu?aIR-{!7RZ0qamMpj)L%+eXj>gPuzsTS+M z&4N+;<Uf;CQF8D!;xj_aG}5p*iGJ@^9#+G$cI#{m!3d@idOz#KLfJ-M;vo`_)&3vd z5Xi^LGX)1%7&bB7<AR>Lkox<w$U4FjET5+br3QKhpL+C@w*?BEAuHfNqxbQUoIXcM z*A50!h^N-ViY^NLyH)wP_R&rJePF~odDM%QvSbeS=?_rJtL;hc+1^WfY%RKMQ>g^v zQ^7oKANeLZ39$!?2n>3cmK4F3GG}$i?K+)Y+3a>Z_Kfnmy&!aFrk3M%$P7)-VD2G& ztymbohcjZopmfQo!`Ws>0|84it!jQ1d2V&-Bx=ZjyHDNXP4&=fg>FB^X4v49@tBwn z(q?$^xs|_`qN3J~D@)q+y2Rs*L_>#UWd~DHp|T8HH^p1m{)s-h=^v(>!mw6!Vy(mn zwIHv4Hsl}uBV<Z3aB~?FKW8;&5aBrHx({VFQm&i&qGfnlyChE|Kh@=g^3&9jG6~$Q zKt!0~=5G!uvI{s4Y^5T?D>nhiAnHj06}#~CLUuAJToj29?_oXP5%-dBLTe|JT{!+t z?`y~p2>|J(ztmIJtdDvJ(+(GSW1LB#r;fb*obBC-gfs@an`8%KTMy!=k78l%RUJ>Y zgRKY?^&C%rE~qL8oa-9xl7cmPE-Ykc#6jFUl|;-A1GsNvq}DyTKK8}^thatCP->|c zEwifue<WeGSzm+izk3_c$Y!PfXoekcZwh7p(2Lf+TreCzB=rA>2kCEuA7Mh7uD298 z)1uWul%S9CkPO<k`8242F5Hb4n%AI{9>KTbg2qz3VaWh+>k23eP`35I|5I=A_Q9sX z+CJedOG?cY%JS{os;s$rjbg^gizU@p{$|<p(L=t)m3hghvXNqycx{|JKh02Fjy|w7 ztCYMeCLE|S{ofJlW}{Z)2LE9NCs67`j*R=bgmjB*f3XI28kv1}!&&FLzm8+5$-Z** zlsPOGXQi`7h3<ANJ~$`TdrnK}^24nXW@okr%afXM=jb#T)O2PHr^9H0nX{3xCg#Ox zpS%OC+4!J79se_wpa2`H)hVkBV?oK5A^~gGM;SxD*Yl>a(;A+cY3IGJ!$7s2jJ!@t zAyjGq#3pehuC(gmwws0462{RCRI<GpnRRiSD~?zN>n6xpXuFFKe~$hMs3DE?=0jy* zo=$?)dUtyTt~Ge?e*5zCM?)*q%V2k3TCE3cN=oDH#oxDj;zk<Et{H?k2F!t(s!=`U zo|`?WG|2$Q{*XPODa16NW2WrGrk`UGqHwAO^GXNU=84T#ubh9h!daY{B3Fyh()o>k z(o5^T3iorjmD_#L9%_7XQ1&Ju7K_J$5x1LQVzX<?_(J=%0zZ%J9p1fr4qiLEBt_Q= zN3+SDNfJY{6fkK3WYzo5H3`M6XtuU^3AUh{+L|d&y_(Ym7xk-CG94Yr@yQ0oc;s^& zVT5;*v9g(Wo~5^#&Q-uxcZ|_w5TdxNtQbm^i7gxO(_hD#v+IZ~E+ZvQx0AI1-ziY^ zk(-tsEvl|>Y8V9NCVbCaNEw3k0^~}%<GY-O+c9!0!=<;3%;hR<maMRCx1I&44f_^m z8GY&(6aB4e->Q1MuqN&XbIftH1Y5W`9P*9#Elwb^&Oq-z&;pUUH~|g5yZOxh&iN38 z95h^j?-%SnV!8Rt+=hA#;U-u_A(vB<ZOluh>Qr6B<041Tm8z6rT~k2nV{QJn7RZ8& z70}4y!3(F73#S&yN;*g%o$Uui17ayxSoKRfbeNp$0!QU*@6Di=j66(MB+|^M?23z+ zk3TK@Dh-nP-5R32U`Qs~e0lTjkI=h-zMpS?N0s7k^Q!^yTq_4Wkl`__Flrg+-f&@R z?Ftpm31!)k>9k~mc+jO36F_wA;p)zLI;C{A%DhqRFRFy(?9#qdhqhn^I}>+Qs=;3% zwy;+s@fukD4&-jUB1AgP9(Vxol?iNAd!8PvuGG5%8{TQGRv?iFl5~QtMBKIdoRx3y zk)MgIh_w@TMQ6@E@8WJ3j3b2WSON7M6o_KF4>6rs__CQCi<?StXdl;mYEDUo{XK!P zvSaQ5FGU<H#kiROE<ouk3yzMdMK$>2!$HYsk27;Rv!6X8BK8%3pTCrDH64<n=&KhV zYPNy!|KPrCbJC!|&2sQkh4ge{udm-K^=K{iTh;xl#Bjk&0Q3&si~09ZLM#&G@W49A zH2`1Vi<4(7Rg^(Du-TKAKwQei?E0irpCH(WJIBH|6=MhJC7ookFXsZKOc|(MK=5;# z^}X7Q+x6HgS>|9)kmQBreBgZ%4S}>)pD_1Ywo}*VAL&q*z1x;HC8?p+DRdWMH$nx< zGZn|s;jwV>W(|+<N}d_}Mt(+dZh`N5eYn2X`PT*Sn_)2sA^bSG?wZv=k{v|kNb&!! zh3ibxNwL9G2+X7_<9m$m4X#mgGl5>ltx)GUu%?#E&Zo}_6dT8QcRij)Lg{T`8JOE9 zUwV$TT`TRCe|oRimvpl>9o5LySPuww%X_><uiI07&!rTVYr~wB3&lk9(aK)Iee(o- zC$Wx`!>)H`uPk=3b6i=vQCtlEQ;6)e1oUeLBZ|$k7&AG+JlUj!M5W^OaOw@!8HVc* z*TTam75geQee%bel|#$Kr8;Hr59?(1=WnRH5O))1o|4~$biyNPyIQxq@Dv6G^4O4> zijj?;cPVl>g*+xbs|)?GRlBi|22IlAI%&HM!s_A}O$%wm{!aQ=-PGg1K0!kk2S$e& zbXDjlaMr&DmV{!S*k9u~U7+|i1i7xTMHN^PRWS4h%Hg>4^UbnFyOh!^aE_4HjGi>W zE2z<TdK7gamT+wL%_ryXRvmZ3z_)bPsI=rAJz6+6?Ecel<DRqnrvPe__0+_zu($pD zDvJi9GyK$>@rsD%`hdH%8&g9=EBEF%w3uYb5|C>EU+g9tQmcPBHmh{4DTlKF`DM#x z^z>cGUWUh1p~cwybq<Ofyn9_$6Jqz#W$M^1q!$L#g!TBwWOWTBqpN!sEm#cR_R%2J zgx_Q*u_k3n<QVZt;)&G&eP&!YZ7by8-nzXt+(LJq*2bE3&zrWDa<)<=QHArJ;*T~4 z-c1g1{v)LAp+)A2m_{tpOaA(PS*1WzGi^n~tPddicL{fo4}g3AA-lZw@Mokac|!XK z;wZlkadgf&3VXkM{>qx(PW&e_%(}X(5<d)#rLR^0QUOfDj!*K|5`*RH7nT8IN@UUG zRAG7Ysc)aM7Lbe8Tw|;<#|pH6^7sKbOxG~LyaO#BBytPhsdS)0E>De}l~9{&e~FEJ z4Cq*uYYsXqNzJCrTui6S_ll#h3L8`6b#I(Ey;dN`aRu<h7JLt0e3YlmvT6@8j+jjf zCO{SfRUqkctbYYfGALaudur4~c)n~;hIc~cZ9~jHZ0ZZ0y>{+bn~ynuyH)c;d(ktI ziY(D$>YMaedO6!zO$?O;@4)ykPtH$e@+IA9ly42eCXMOQ=u<9F?)@{6P5berZY>~2 z+UX}Ck_t;87Y72lX8=67KL|OA?9!m4CQiBo|0^FY<YOmF9+ExVacU3mB&_Yi@~XiY z&W$yR9yW|PI)ZK2gDv^Hj$e7-r}!q}t0U{p6=GT6rfu@|kGtrHOZ#zuMRi@*ppRPg zJ#kDnvd?qzOLsD_jw@Jdt(nYx{q~ll>dT%yyDL`>ubzwc#7<w9(Y>%xkQPQA*hqLm zenNs&^|}b4kJ^bx;Au!{q$fO{xTo0^Y$(<chzG`=>v91x<Z4g>6P?nk0QrNcO)N0~ zYCYzR|AapXt(CehYz#bHLhPM;G7Lp6t{ch1CsC4T)H4F~C$VsWSP{Dwy$jz@t`+Vg zJfKzeL=&zLP%#W*2QhAXABc9~x}Rg}O`6k<Vg&k78e9y-_*AgHXNGtt^`E~DR^DEz z(aJ@e4f4)}y!rwB-hb)hdWM%XxpVfwJ44gqM1lRjy(ceK$e0BI3k6F14Bh!euk^HW zu~P6I>k<FmF_|W)1`&&Mk(YBdkMm4#Jb`aAW(Ob8^R;==PWQRazFYX)Q-g1li?!=E zHP4RK)b9vilRoiBsA%KUXl<_YnKxPXlwda_)8T7AV*%$JTWg2QJ$wtV<U8M9QN6(Q z&Kz`*-S#x{QQ9`V2F?mB6c6yzts49mg1_E)2*;TJlyGw(9^ft2=Iz9XWSM2e0pPwm z%OuA%2CS8IC5t(az@iHVdHYyBo#_jRLyaRrPxFV-#y6j0WD0-!g|PKqRcK~{9@<{K zD4`8a_RtdtNzlYa+DsoAh9!3@sj*PqDHCN(r4%{=gk!R@x=4{^2~zVZuxZRYZgAqM zghZW16EC_CX0`9bt?TK1{#Haau^7!~Z^PbndkEe{Kf75z+oy52-=P%b%2YT?;CZEk zo%Wfc>5r~zjOFfd$dEl4RXXgByj=P6y40hbXpGD!8M``3d8)xT)#dKsJHwX?WPca! za168_d|9w@vcR7Tk1y}fr}MonD7Ur_+{V-dm<>2oOLfW=KKL?mac=fdkaknO5K}qT z<>{U+$|{oEL-${k6uu(@|C`iCgNz0`5PO0B$|tS|)~JoT3#13_fUE?fIb1yC4Fp6i zPL6kh4{*r)9>osfYMuKWbV?Z&don`Hyn0wiCt#`KdGdisv7kg1hs(C9rMyefoF{<V ziuixEDhWo()pCoK!9;mr7F9>%*nwS@1^n`Q^rS(d7t)RvJ~y*Klk{yp4uvy_!av;i z4&m$>;>XtCwa-0QSa4SK=Z5byJAS2wa>M7<+ea^w)u^BDNV7lGq(AZoX`IHs=qL7( z&HHAz&z-IC(dgfuZ~yVJhx|;n$<2eBX?@iS4Z-Pm?VjmprMDOOnG7boB<j0=&Cqlx zvAp|eZ&}Csd?K%g|C(?cAjw<#rFd~X7z4f`wUJ{9@*oE)kt}B7$p+Nx?6@9yLgPQ9 zYjDpBBep^MSF2fBJT)b~QJmroX7@tgfzjF&MR)T;-!ObuQIqzYPaz=&_gR^E(hPDb z7zC^S0;;&@JWn=?BLylhGH8ToU9AWROWssNl$`+i24o+C1lASxW&{{ZU?x_WC0!AK zSbVdAMd-Za_vB*s=&A(4v(NwK#FTk(ut(KwI<;dS`{;gUtWste)yyW?*Q1Q@wA)yE zC|HZDeVO$$S-$^vxwm}x)%3u%qO;{kqz?nv3BO+j)@maT>hUkw1~L{FFNZJRS(r4t z7Z_P`$kHyQwrF7Ej46H#zMsf)(tET)T58}T1>MLR8dyHW_#vFa&j=kitgtgb0qNg2 zK-G7I&M?tYcwk!;z0gS$2iynDPJj5WASc#x88R;2_A$kFcs8%wiof~$4*cgwL#}up z1$gmW2Hkm^pz?$1d15#%EiJ=$tSm0aMyEn^uvD6pW_jl>crjOb3sjr*NBPGO^$tZp z|8kSG$JE;NZC#NprNd4}1NEfB_^$+0JZLW3EXg-uvG7`>hdj9o!65!H^t&AY5x)5= zJF64@2wg>c*93#lqqvuzv)0F(4e0}We8|F6I5%d5zB&0<tr&8b&-ArXlAGi_^{c6* z-s}KRLSg-MGaJQ|cBgS7M!{VS9|swBHF6J|m4QwE$3H@{yvvLTbEe!oFw2ACkgmu_ zKuHkF7VZQoyaG`yywm+CR~HzW?}8p&agTU1#JrSZU6Z&dJRmHSnCEGE&!Ozv>z|!3 z%lcOf658obHTP|cjH^2>gFNM&@^beyG~R1g##UZ6)Ryt-qngV(8tmg!mb^sQLN#<E zN~(*_Q5`c{GCcNlof*jjS7d?Tp%;al|D!<C|3=S)&HfUIeB71KieeHT1<PoF#uP4~ z0RwWfB=l26EMc+P!jl`w3uSrbWoa8}Q;r2XW6!nYWvlDzo6?*s78iA9oMhQ;3K8pQ zF`P+N4(AHL5$T7VaO%2rEFGAmpn6)5@f(17iMZeVHG>t|ah@q**Hyt7LA+d7Ra1pc zjKd@6vnh$ox9mS={6OzD;J*DOQ!!r0EheLSHHXWd%uE>t?=XY)+FBkZe?kuy<1yJo zx!N7F#kJ4Asm|^>Not$?b`1>K2TfR{ej=nv+E8Eg0x|%qU<I(UyFuS8+@l0*PWno& zV*jJ-^Cvx|mc+SQy}f9Gi3lX@yzd;tcyguoL_qXl`C`Ya3t17&OkcojO5TB7+*JNw zWDjIKTmd12I~Lc8mg3299eDQ$Am6cBYsU35<i<r{pTL!X&6-XQlx|<EKuW?n0aoTc z`Y?6;j7IJ%auYW88}rcX(B|<5v*Te2HMyqmE0<PKQ|P3o#q@|>k5q23rL&XLihqMb zYMeX3a&()tD&-z9y)<lSt?xR#WEh}m`+9V>;_TLj*CVTaec^lrhkEAkWgWc$73}gg zkHNvGC{L$_l8DE5{%`2iQ<c;X;8Rd&dp=nu?LsfG!PD>#xxfG5wykM4LVFBZ#4aL` z`Yvu`IbA`Efk35F$Tpnmox9`-1+!+aGywUYu~m&Qe>Xw_cc6-5NUJ@vbN8;iJ-xZl zXPCyBzP2fbE~!OIKmD{;0_1EmEvvp2*<4noYSgc55km&idDDCNufWNMQ`J<HIAHb{ zc^`)#4VtV!+{TXLd`g?-XhBB)+qz|YuGS1TdwG<B_U!SKGrsY|xNYD1BsD9(-n{I^ zag!2=eBj>1`H6&bJ@7R5{VcVIL3ZK9NFXO6iI}tG&er?%cfKAa`wYOX8G7-EgAdw= z_S`nmXx{nLVt-KNZiPcXkC>dv@>njgWvEj{7ybxszMt}imcQWfQ@g$8S!uZm+_?$K z+vbiAb1x?Lkf^_a$>7s0t2(HU?bIPqWFzCBUS|<Hk_`>v@;o;-7)zax?V+!?!?#WN zDXFnXe|=DVnPNFqIllYXl2GbL5Ti=2r!56ye8XpeV_pXtFXH!C#~l!YdJ22bj7?d( z$FM$1SI)4SAx-q}GsD1evblp-O(U37R!*Z87xRN*aj6%I%zvma_Hnlf-V@%;jNS&| z4e22s#saX=c}0OFEn%JM!0zcL?A6~<Z}eDT2zGMYQ4=TTnMP?ZYt)s-$NEGK2}!Mm z#^24}yO~EsI(9vg%$d*q4sfYif5K!bQvb){XUOR1BMbEj**{j}1Y0QIsXsx-l0&{m znqYFgfPLrz$lr_&?@W12UH&^LMMf}$-fLnN%C=$-Bp6KW^CFs~6c=7fwIL)!?bN}p z6VlGfhQ+?{QTC5el1wz8sIf&OH7xD!W;`p>alO4nJ{CUs=z0eDS7&v;MFJ>sWB&n_ z|9Qy#CUU(5-9#`J^GTV62}IICxQLcJOEzm}V7!B=*Nu!PN(T){&*vSeVk25!;Lbv) zN^yw60NfU@Lda!f*kSbG80O%UIYY%MP3c-kY)Rgw1IS`&$9qL0qiv&EJr6oP*bcl` zbpNw0YTPb|5+xsgQSivye%n~JY$!13h<xZDAxg8g!rx(~_iyJ~sBD}>#4$-7K>I{W zG7)B%2t;jJ^KSm{)(I>d;Jtb%E_y9R-VeP~+Ysdc0rUh&q6gtpJiUk+7<K!4d*eaq zRL$SOkD^61{FKjB<=d<$F5jL$-lzONbs316t4KUaPtIF@9v#wQiFLxa5N48S5_nOb z48`O~5sW$vq*ZzeymMR^-gyW$)3<OUC<i9N)5eGbLAz7;$(DoNC&ftMWVQc~_RcjN z%Dj)`m6WtmNkU9TsTDdnPMZ7LTE`xToDXp~NsI}tLq;_Bc0dxd8`28%M5wS5Lxma4 zV6jZZOiY@CqA;0BO?T$7&%d|Nb?vL?)pK3@a^}^{J@-Ao|L^zvem~#j`$yFd%vN~P z8Cto@eZwJm<mD_r#ybx3tIW8&rTJB)mm2nQM^DUBt?Bne8K!mmc(L>28ue2@9k^v# zwyTifa8xIt{rofM-z`hOC-wC}+O`(M88IJk7F1qqCp{TWANYzVKyj@wYT2vM#eI=& zZd4W~GxV(HJLT(wN%c%V&9hnxGjT@6YZtyFeIgNAI3p?Cm5T{@dU?)cbX&V!vH#F7 z(pk>Z&9B(s3oXj55H~0ifFOEh&B~v_Uu?;F%nIZTyjjJ@=rPXz)<zUjuG6igRPid@ z4%AAZeVuNiIEg+;nOewc*`=L{^-#6ndncQSds-6`ZLWKlP&rcjT6QRUx)iTG(IlzQ zv{oVkQva+hYeTC|M42q0cp=-u`XY`8bLy-|KQ_(9h6ZI$#@QIAk?27i2}Y;FO*HN6 zX5VxOO)d|-!|TkjeEkJ;<9i<3EMmVFJo@yytsu&$5>xAx_A*=e-aX-M<&jNI-D67} z+p6!6yXP;pRd|u)A^MTu(E_N!e!1A>_WTiRVfgX42?-?(@U!M4k3sZhj?7jX&!VW4 zM;?PSlA}%h_|%kM@501ES&6i{T4LA3Xew)$Qr%IC^hFr8yHCj094E<znb~)#D?`vs zSCK`^L4M^g_BTZdTT&shwtN@TnGC|S_|#Q^rc7PV1fA;bmKR9mfc8X$@Jk0<o75^Y z6K6!FiI7alI4&Kk4xWWK?*;H2?r|RJBbP~D|Gdm^__O+pDSB(Gzui8KAjHitFztbj zuVu~Z`I$mt+2$h4TNR%PCr1hw26qc)_FA9*>_-ij_pay^j%_6ypOY0NZuRN$yFm+| zJasoOA-7gknS1r8guDlw))8ajw|rO?sJhKY3*zJQ!${{*=(I<oEIxjs2q@jz*N`)q zJVX=cFbnHALJ}8I5mQ@_xCF8GFoZhbh<{0OsD?;h&-rAP9TGdnxy{Ay{1lQ~R`KTb z#XQ%PW5-0)c}Pwo1An&%e_}xvt*mAcVKt`QRW<sYR0eAkF{>4ZyRLw(Dk6D(7kBBW zd+DnrRpm)UZ5@RZ8h^Ms>po#=y1jlo*J@r7bzfs%n773Omib(tsd3kR^yJ2-!bfAb zcI?aG<r#0O^=lXt9yu_5W0N$oG;+_LA!82BATiIl{esP5gDoY_82z9B+#cf76Zw(p z;Th6jD=CP`V)i%6V@fr;s5{<P+ISXJukovNKc^z%zC$|qL2+XMKMdSnP1J78GNf}q zZiND;%;ZkRXMSd9APpZ%r4cS;j-hwceNm$nNspE81c?%k`c{R;wERkh$3KRW64H!t ze2*r1BwJyMEgcjzHLD8VmA4W_b0WNZDIwAM%EPIu2i!VGr_gr(;S02wIdY(ncb3KO z7u8y>zBhC=y6Gou3`iGZmYj3<jFsB&RPtkY4oW@ylef-39i?_c%~U4B8wlgj*QR?b zuhI_!RR{>|(VHXh#B2b_v<XdSNDHbjKLVn?4I*h*&%fRw<zH$$1dLw7nH+1wJE`$W zp@X)Z4<l(g+#J8gV)YZGYK0C@wp;mc^1AeNB@mazLKFOMNFxr3Uf>Z!&_k=QskW-v z2hag;bibsxgSy;N|Du4Bwd9#ELarGEP-{?bcM<k{Wiqbr{wqRplJU8qT5-VMM)ITK z6KJZVISU(K>@V~weOr@(rR^T8(KtLBh28XaAu%CMS|!vhG_iO%<d2{KT8d5Z8*^n& z`WNayYbx}kTe_Dth9<ff>U)(ASN;KtQucofAO9a-<GexNd>R;Rn<3^QFLe@l^ui^9 zx#@=Wp<s?o?P087ueu|J1aNnlC1;v<6wMN>3nztb6mM^RMjK7y-J>4kCe<BTbQu%J zeUkSH<NL#4SvD${8w|bH0>`#2#lErpE`u=k%TF+`Re-tF2Fxkrlpcd!Ghq;nvV|Ad z%DrG^ew%+oEq*M5i*J*JguiN<h`M1)ON@_Ft{e_Sw+`qQ4fbDE_N9B@YCtS0gVrh+ zEjue_grFkawt%c}VM@O+nQG1bQ&9-yxId;ZN5`vz=HS&hJB2A)cW>6?G1D7OmKt^$ zAEZ4i>L*NSR7yV7pEG!i4~(KHPpypYw=D@mi=>MnoFf;I6N7=Sg(~#wYBzHSGwdds z)iwTl#ZEzuq1xWH$(oasr2N2US0iz$snaR_Vdm~_m&p_qnCVtdg714lk;F-cH!32O znLx0HWRuMSNB52S6b9&UfHaxLHVAK_%5J5mN2CdDM_IWJXU>E^8l1L2b@Y+7Y75eN z5E7Jtp$jd{iiOdH!1#Pzt;2ZNaQebLh`NcAC7^EAr2YZ+Q@eyz+`gPHcmkP|6%LNS zY;ew|ndAlnaDB+kK>PzsvTB(3;SEFeFA_4#a;X2@+sFGAUYKqXkO^SFx4OrRx8hhl zFh$wZ_4p|N2d+`wXKh~ElvReKuVCRN*iwZ!%dRi7=*#o{b38c^5wqT`=m23N{>moO z`Dw9#GOQmB*k*q%E5|f0D0H%9=YR(?iv?-nwxKSdsx&R+;Pwy)fj|u-VvE%T{57O6 z-H<WabTX(LAj^unFPv$tJV96r%o1C_D}T`j3Fo%#7*-sr2>GGv{d4u;=_m3J*Ch>K z%gp!ROny^fBI&|9rA~-jrg;}@n+~^Wg=Ka)J=`Cc@H&CxHBf%6Z|BT;vfnHuaf(OH zfW;_P1YKpUR)oqeNH=9}kwpPY3f&$o+-~h{>l=zZKsqQWVy!b|wluUX>rJd+8?TzY z<K{cS2Q}85Y)VQ?0{Ci>cT8<bx%9Sg!+U6Bs4PlZ?r7Pd<+DdmR6YL%$JoYJ@4dze zP*a|Ch*(lOCHk^N(KW|eo;TqE2bYi0?;m6`IVm+qbTMBl^uAU|%eXQNaCcBq|Ee%O z*<Z_)h4h`{&FXEi9&{4cCtMhHtx$y$8EY74CpQnJ-p&h43VN=_3humbNnVb0o=Eyg z?wtbbNVA;FGZ=y-1?<A!4dnf{fdrr<&LlvsyZxnpe98QGhn}c)4f*6|q&Uj-aY=Ki za9g`2V;6;0h2p_Ym6r<$pX<PqhkLy2Sq*svrdeF6vS%wyrey9<H_lw4h^>3}-x?BD z$9{vxi|<E>$=kt)K8L!bz>PuV7a**ob}6;Rc=IBtv}YG7Lb2S*DRgKtpzd=YBG?BP zxCPq=F{U>_f#6Wzg2}am8vq-}jx}u24w?ssTu*WE>LWrN8KdgL=qlj4s}Om&jW$Nb z<KvxsCsRgFYo<oON-re$s@9}Rr&tfGr>qpZE;1H3V%g?ll?J+2=B=b!n<awQ@dHy} z+b+^=G1=~){^Mh_l^(xzGw|<fql(Yi;P%EIOLeRKVs3cUqQjSCk4;&;j+C<dC};v@ z-c00X5l6-nAPf{EzhfJa$(L0rAAFm5-5~4yfj7xQgKHHa^kAK#tJ(pi3u<)Ge0cB5 z(sD^6Xzajx{Nh#=5omYRUtz4Yw$0v_?}oYomoHVT>1+8B&01O6hT*d#Q`6q8C#Qnq zOj~T`KLu`@ai9|hhr)QVDzg)K4a*>>Bj)99R!@PT%*XrhphEw=|M$S9f3D5{d~KHh GHU1wpf%g^w literal 45595 zcmeFZ1ymf(x-Q(q;O=e-?!g@b1PdPALvR^fgC!wAf<po%xCVlT;2s=8kl;>`;5O(0 z1Kc6sxA*?;S@)i^&sk@!f8YP4Sv~VmUDe(7)?3}rD>XM$H$MRUhYHFH00;yE9wYvM zn`LYRWjQ%>%|{vv%Bu2+0|0=nq-yW%f`ktM5NA*KM~bra28KrTXoCO}fB~QayZ~Tf z>EWuTt)K%SHf4D^dJjaUTl|mhbOJz}2>?d9lr-t-|55%=Ap%QRcTWHSX(DR#TUmQp zBJfKDcJlFby)8dLVB#l^w-^ce7P}*E5P`{WvF#t2^H-fe@cmnC1#z}Q)VaN9S1X9s zE$&3%SKeOM2n<$5;4p7{Yaaw2LtsWHFNi$?!w{GlVr}680La+4<(}4-b_jeAfwA3n z9?2oF1OT98+x`VV`3v^6_C?$$0LZzx`nlWN+IiBmTC&meii(QRD_Q$ES$leNX<Ar1 zTDV)$%eg>YEu8%T;Ez7vwgL!mWlN6`vJkJRkPsIiH$wb>-~Qc$f3NlLgWIwFqsEce zuQ3CWe)~t+KidAI%q0f^L{1R0iTg*{llK5n8wvpAGyf=K%mM(smjF;R^q2mS-s+2; zr>CnJ508(J54XLwCHJjB|Gxc)75-lH{|^3gJ?`7<{atqS53Fr0yqrAgZ-r{<;^gAx zPVeDrVQEdz^>3B<zufRIxAm9%;Cy6lW9@G3jJT8@Ld)!(Z4txmY-R6h@8V2v@BDuo z;s0W@zudzu{6nu1fGF=8AU5X)@P~*2@a{JNnFtF2n`9u4fc`#isu+5}?ab4sTmM7v z5g2j&*XRHCffS4Q7s<okmj1R_?vW0?rI)++Ek?|V+Yb}~6TkzA0SbT?U;tPFE`SdZ z2E>8;fIRRJ&;WD*L%<BM1nd9^;12iz0l;%042T5cfJ7h#$OQ6$k3bpl8K?!CfHt5T z=m&;@abOns0sI8Efdk+axI{e2&_FmKA`k_L4#WiF0P%rDKvE!ikSgdA$Pn}xWD9Zz zd4U2zA)rW50w@KP4f+VG0M&!qKz*R^plQ$&XcKe@x<CRWVIvVE(I7D+@gRvH$snm9 z=^&XR*&?|iJw<wn6od2*DF>+psRpSHsUK+^=?Bs_(g_#<V}VJ*cfed=5wILs18fAg z0lR~Pz!Bgia1OW(+yL$YkAi=IcfjY!sK`Xf^vGPu;>b$Ky2zHuZpcB%QOGICg~&C? zoyeoeOUMVv*C;qB)F>P%VkpWe1}L^DJ}6-*Nhlvss!_U7#!-HvoT8$llA^Mp3Zp8a z8lpO&`lCjnrlXdlenI_?x`KLwhK5Fg#(^e@rh#UG=7|=D_8zSStrcwq?I#)x9SfZf zoey0B-3Z+o{RMgwdJ%dH`Uv_u`XvTF1`~!Dh6aWeh95>OMjl23#t_C&j7v-cOcqQ@ zOdU)I%;%WzFv~EzFy}Cju&}W1Vu@mDVcBCn!+MAH39A=t3F{o25Stxa4%-CV3p)n8 z0Q(E}H1-h=4h}QUeH<ekPn;N>k2oDT3pg-bB3y1<Wn3%VVBA#P8r%`wJv<CNCOjED zGrXsGNqCibLwGy*82HTia`+bbLHMco_4t$cCj>+UyaXBq5P~RzB7#1G4MH?RW<q&F zE5etAxr7~rD@0%-1|m5kOQH~>Jfbe5pTwxdEX2yh4#bhfrNo282PA|f0wj7Qz9gw6 z%_Kia!KBQj%A`)Dv810#CrB^IXvk#9tjS)Jm645*osm<K-zT>wk0386A1A+{pr??h zaHNQ*sHK>vM5bh?)S~pE%%JR|+@T_-5~s4HilF*THA9U=%})J@+Mha?`Wy8L4K0l# zjVsN2nhu&BT2k8kv<|dyX<KMF>4@m0=<Mm<(zViU(UZ{2&^yt;qwl0YxI=wM>5kW( z>^p;ZF7LA3)xG=TZu#AL222JK1{;RA4DAdDjC71@i~)?rj5ACaOd?EnOvy|=OlQn2 z%=*mX%(cuLEaWW8EKgaAS>{-ASfyFrSaVp%*ihL-*c{o?*@oGX*ag|`*i+euIFLAm zIP5vnIlgnEaEfv|bLMbPa$$4HaCvhTb1iX`a;tK`;I8G~<6-16;)&<!;kmvic+crx z-o05~Lf(hGFL)bx5Bb>nEcsIT#`tmg75Ibs>-eDp90Jw?83I#+M1pF9;ezdgS3<%< z?m}fko5GC3kA>5OCq;-wG)1CBdPPx0WkrKUn?%pWgv30>K8x*(bBQ~O7m063ut-=- z<VmbZ-j%eF%$EEiMKASODog5z^c`sn=^W|h`waK3?ti$yF2g3{AX6f<CwouUUA9{G zOioNLQ0~hE@B_sM5f29C3FUR=Q{)#E?kd<S6e}Dk3M%?5eo;bEQc;RmnpCDywp9M8 zeDF~CVc^3~6>Jr4l@yg_Rd!W()dn>rwTEhN)aKNg)ScC9H2@7|jW-(enk<@bnvGhh zTAEs^T5FGZ9|b(>(I(P1*DlgN)sfYS)tS|0)%DVC*Td5@(fg=(rvE@cLH~yVk3pb8 zzagcey<wdZnvtH-2cr{XdE-RmpC*DPVI~u%ET+DueP)zqPG-&Kc;*)7m5)&#>pw1f z47bp*$gwziqWmQ7$$_PuWwPbA)qShCR-4w6)(O_@Hj*|8HXF8*wr^}V?WFCJ>~`$s z>{INa4oVK04rh)Ujs=c4PWn#e5Om06NWC+mv!ipT3$2T<%Xe1}*D%)~ZenhUZU^ov z?jJlr9;P03p2P?;_05abE7WVrTgp4t8|I_uQ|U|K>+Ji@kKOOJ-}+O9r}_TK{+9mj z0e1qP2P_531ZD?;f}RAm2Qvh}4F35{@mb+>%;%2J2VUHJk?`U;L_ehACC$rcFIPep zLyN<3!`#Cr!^OihUm?G;e>M1;|MmOV*AZ3`UnB2DCPl)dETg_g^G3gmzKOAk8H^Q* z&4@#dbB>#cmyRz?AV_$cu<}OjP0d^Sw-IlT6U`I*k_3`6lQEOMl7GBYeOLFM@qOHT zc#2)hc&cn_c^Y-v>$J0UtMrkK`x#}KG?|f^7g_dMQ`w5yH95>VNx7)G-nnaedU-wh zBKaRbP=1K~a8=-3@T2fiVdqETj~|Pui(-qBioJ?AOH4|JOCOZhm2s72my?!9egZyu zecGycTru%k<#T(bNM%_SQ&n0uQFTNOsK&46pw_PTN1b8a_xgwR?G54$)s5VZ1x<IF zQkzMdV_Ps=Ubft{K5aeu;`(K$&9-f&-MoFa!=PiVQ>$~ZOQox~TcNw7N2aH>SF*RM zPpq&0tH{^dexd%FZ-U>d2LuPI2ZaV}hJ=UehDC=PzDs;>8Ic}o8+|a^J*GU?KdwIh zeL{C)YSLu#$CTC7#<bHkbjEw;a`xFA%3Q=e{(RB`^+L`f+hX|-p&!jl50<_yYcJ2P zSgvfZdaS~KzFfmuOIoK}|G2@o(X^?sIlN`O^>f>G`*J62mtZ$-k9DtV|Nj2B1A~K~ zP<QCfVbl@DQNgjmar=qp$-*h*^ztkMMgc247dh{_(7#x}^ns(n-(9g?HC(G)&)q<7 zZf=%n-Q<1kZ4efMGCP2dpaX~<g#3$0Y$GAypEwP{EGR)JzshG2wW$8Y%LvR00{<#U zR0nuLNC*l609%ONBODD-La=c{L|gmtKhEilD33zyNB~mSKdu7)6W?Ctc0&cgf1VQ+ z@!#LBYlQQU>pnr?zg`!FFfnc$J0QxD{=_%<x90+G4X9to{`ReDVejmzZRg@juj%eW z&nwJ*@77HCKW)ElEQC~k+gSfbHU7@Qf!{XPziXWRMQ*=stlu`)ZyW2kjrH5c`fX$V zwy}QOSpSo3EF}aFy0uOI_I)#TGYiN8sK_X&D9EU&D5z*?sOXsZn3xzCn8dhv*!X0` z<m6<;q@<KIOmvh~4Ai8g^qlkz%q(o|Y!q}{_qbT^F|o3--kt=6@FrrSV-jIv60uT} zQnLQ1x0^2j0Xkp+WPw3<03-qsm;iLs3D6;K7zN?9MQnfLa3CZw;(pQ4F)*<Z6{_(8 zBoG*kgbYSOK}L9yL4k<l05Smz;T>KXR3a@4w7YJ^e9z;v(HUeb+DIOKKVam4;{E~y zla!2{f|7}ug_Vt6Ku}0nL{v=ffxLpElJY}s9bG+r1huxbvbM3cvv=_D^z!!c^?Mrf zGBhmw)$53aH*XV@lHa{g$;r*j|4>l)vFLMURdr2mU427)M`u@ePjBDXk<qd7iOH$y znWg2G)t_tY8=G6u!=vMq(=*ul#cjVp2*dJk+aEpqH~k_&^a}|Y8H|i}+b<B34`K%s zAfw#jMJ1HcLbGrqy36+*ome(LyP^$)f&b9~$rJbQn52vXOH9z)uKnuSKi9Dr|D~S& zy<`9A*9?FS1|bFyOaMp&7jH6dfNgQDA(xZ%4d5d9%f?FWMawbZws>XkAk}(+35f1r zS5{M>-T;OR&+rQ1xH00v#ryZsXl{V<)7<SF0HF*2nNwg})T}e!0A%^({g6TQYaSy& zY`tco<=B>^ksJ&Qv;zjoy!r4BF0@8z&OA@Ez~qqnKIyJa1-@C27t)av&v5H^Ys;G{ zzV3Ac*K|ezf9QNv2wW;rVO_VXCdD`3KC%n{ea)90r1UAw`*%`*O0)kA?~gy?Hd5Po zIb)p0l|K{{#c#54G_`eT>fl3w+?v5yQ&}6c=GsV$vY5#&AY0lT(wnzCwdM0jgH$;u zx7d*8p?!|Ev?Xht><?!`2DSRHRta)<#9fJfB3tdrXBGWh=3C2Q436_IWtj&mdIQVa zF6H@Ko68U4XH}EN@XhW@=_Z=QuOZ<oiL3u`6(t#7|D&Uh`u*0B8(^vRM?4wxJVLzT zH^4KB_PuVIYr2PGmMwc3<0xVGW<3&uBS)gFYu1s2$w~oo@K3Z=Ni}k`pDe31H~`Jl z8vt+qESC}98L*O-1!fHR60lyFTcELgxiAJi#(PT_cOvkZ?rmnArvTu!4VU&=sI0~@ zTNu&CM_oQgcAEI~`8<-ULW=>F0jneJ&0sfH0<RiJ;~imZZ%ecJt+Qgj{JEzN%J`ZS zg-j{Jc<Eb*NH0mg%vg$uu0kWk2<%GkR5dn*?Q6dfxO9py%Jws`&SP!$@ppFepH#i` z)nhQAOO0?vs~)29XhhKal4e2vt@g)^sU@v|gFJ?34~3AFe}0ZzVLCF!f+cdk+%KJt z6i4r~@S)CdvU`6Yy9p_+F~POBn8L=ZFkQMTGE_Zkqv7P5u5n*M`TW&+kBk8os?zzh z^C2xo^~|ob=0e~L9jo-6ExRw$Y|``_cEB%1{-wvjts?)@W8e=(jz#G4Gu&H6R{W*M zS=iCWI)}~)-4jZbM|SaJtwY!>u7nUED#-+xLc>+?#&`b_dguPfrZm8GuCWHtG<=uP z17Ig;kbHkAX3}0#UBN_B@k;AH2^Zy&^7zY%mvi>Dk3b#{hoDn$2|p;Q2)MB)4v)Jo zu%E|zDEK73I4RF*FvYzA*FU&qUvcyr%`gRxrgi)*H|7S|y3_=vmF+ynTRO=VF`ep_ z?%2GKrs^|4IurG`sRR6N?)))zfIsJs1J-4KptMF^YjfhAs@mxM#JMF)dE`pE#~=aF zx|mNCTv!0Y*~0XlcFl*cmrV2d>6`Ch?)f$ad_q3isMXbp&)Y^$NrwXvKb(>7Fxi^Y z^?JZ+SErx%6nHb!F#1r8x>RWI+q`3^u($d2XM>H}+M3$e3{`D3L4%}ZaenbTd`Mlo z_tDjr$>&H+jXU)TU{6i8jwO}9s<~!bhw+V)ulON}C}1u&q2~d3eB42r1_HArui%o) zekUsVFSg~z1@`YTJUE(d8NnuaD3@NRfWz<sk0L5+{bxxm0qpQnKExogNiYCvf2!uc z{wdz#jZ5NX^RiIKHFWSkJCfC>yE0U7e(DozJ9jlMQq0Y6c?%KvIvS-ICq{a1JF-kK z+hl%?H&SbI0%eNKd<CIlV{F4IN{oaC(63S^4zcX^=&NME_B(Z8g|eo~er|w(>Dfu! zob+(&J5nBgo8d^(Q(xDPs+s!{yL^&c<`3E4H%LWwZ>)GG9X=7Y<YP&Xn2Py;2QA;| z$?P`QLpr>sdnSvUM@Eq!T|T@nwVJxt?x8G3@=@+bib4uG+SMiXw1Yp8`qt2y<QLM< zBUtq^wRQfUEA(AY68J5BG0oIauas={x<-EH;;_^}!{(Wn=(_;^6;{tkf8Qn<!ucD( zYIDMUi1D+k-r60J47WHw6=f{=m-C#vu~kA#6^>+)BQwDwn}dhWUa`CN=%&{{;*4<_ zv=X1Ztbmk=9AOVURAy~-gn6-ZG0*A?1gO~-&83T}k@}9PsJs!4F`Qvtr`0Y``l<Ul zueRW;ck4i<W%p;;yFSuO8a5pX%&9RGcqm><#}&18VofLo=u02(aRKpYQ^t$r4!gZ4 zbM_emjhFQPmv+RGd6mz-4WEZ?33<sTK@*rl&4YHLzu~i&8H~U(-HLn*+EW&uFA+I0 zH$;~dwAN)>u_g3z+P`$)L3-;<%=QHAtcO)(vM+BbbWU2f&QF`Ud@c;i-vB<&eAzm3 zMLhn7YO~Rx#S=z~uhL>`X6*09T&o=?1dHg^R^DsiF|nJBmmQ*H+(9Gg{=y$9{5Yt( zk0p#@tXf*=2KXW?`ns>NA*SyftxQkPWhAWWxir6iXN`x9HP}4ZnUgoJFvX-+KD@%% zBP~#Y?INba%h9%A!PH?=O!r+paH(ofmYEUXnwI)pWCOF-%tN8AK49DEjf!hBp={Ed zfY)4A);86q(aBV`C8dq_V25Y?=~Hc6p;bsk=wZ^V30*=_gzCyYWqudqTs}m@KX)Xk zN)?w?_DSrXHivdN*kI_c8a%Ii&9(cCi%-^g-OgGzmzC6l{TVdWxj?1@O0_8*w=KMC z+QT_KJy-X7)$ZlDC+Ue=nxbl%tjGaHzMNm&`t@o1w$~bIyv+txb|qWZ5wGg{k`IkO zc?*gtNeO2f;5^MalU@s-JNFooB;NMD#g{tJ-E0aS{hm`-Gs0=v*kPuK#5ikkZ;lxJ zf&jGYg`RtrxPh9u_uOpe1~4WQoew#?q;#q{FZ!O)7W=3~=h`v_C_P12cRiB`+&_LV zdIR(nv5Y{hkl*vX@IgAYfM-18P+poPqp+_w4bL)tv=Pk4-jqQwYQ8_Wopv?fFu&3B zlMArfqQg1%pDo6)Zh=w`*o$WHHJohr@itg;v2k?Na#T0kqC8L9REVUD_c{%WVYE^9 z?w8Vf7>d2_W{>-m+_HXf8MIcN&mpP0rLZlV_>y35i@~V6*&ec2N+_tbiiefOG$PLs z6}Oup_M#<iOJ7!W8fST~37hk07+Y)T&sV*<TiI_j2P>;m1B2N5#!tAox2Ro{8Uo^k zf*0)Pg5Ek%5_|b?<M5l;2C8&}HICeVZdMLBQmV{G_pQ+eQkFTabua3VIy<*kN4HrL z-yJrU5S6geiR4{byBxvBtzUV`SDp06VbW%K_I|ADTL(OW5mKd;AG<D#1Q>VKx*C*Z zuJ7_?QOtJm4iwrQ7=DrdOt{ptXf;whJu|zL&|ZPAJYDG)z6$UluZRlrjc~fb;x|kn zeV6H>wD0%N0*l{n;rcwjy6%qgc^r7X_Uyk%whR~Ue7YVP;fCm8`A3Fg31GN(S@J8i zEpy&y1h&HIY18o@;#2E>h@@KNI?3<)S(E7+%|G=-iI+dFTh-c-96tL!oYTj`*e`G@ z*C38Pi9XmblJ47)Y2k`Bjq!t#>X#VBn@PeOVqD`552c|$>;<B+l)lSF(>?q&+Htky z=p1jmU4_Y6i`5)W$@Z%FqXX%T=UwA^%snP80!n4Z$N3RCtem^m2D_T-#A$9SOX7vq zfu`j@QkF$ZcfGL;eZK6j^|~L<_D1x_GN7-QZ7PWL4Hlm{*{eBveEQ$WWtny!Z`f?w z8@Sn9ORicG23WJ*zvU3yldz!uW(t-yhIJYgfP!HSye=tEBNj}yfS{&8Ko&gwV=e&C zgNI~i;VSCI;GiD!C&hGMucJR;t&ro7KTOi(T@&TR0>3AZp7u1#X!zLJxmH+8pHsz; z$qJ^S_kwy(3M0sup(06^6PX?!gGO!CGsXA}zbTP(ht}7qr4l<541z_l7bAmfYNeC{ z876T`74$Zb$h_LWOg+hd$u-7C+`L#@;RdS`5Zra?nlb8QY6(9*GcLSOTES8BNpPXP zNcC$!Pu#}O_s9<4dsQF|xWhbfOUG=38^C4L95SB1PkUhKcUKT|&_bSzJ#kKmJ%O%Z zQ0HfHgmIZ^L%LfnN6VEY&gQ_`Y0WzZW2h}Fmn%{4VK#eosJr?N@UHGMwLaQUt1+kr zbgq1kCa26?i@HD3);$+nKd03_msvlhjsMiPb^bIreM&mGA`6^8MHgJ3J4w@W)(lWX zdN0$vsDG(1AnyY~`B_p^`>D$fB1XQzrpYIEuT2l_1-c0Rq?D(BserMu;cFL&>?a<9 z%MTjwoG2a(7e1P{K~QM=!NbnT0x7)*E8n7``E@Ask8gk{@lyScuL@!e7?qQ39>lR> zli3iA#mR^U(Bz}@Uo~4!XlrCB!Kat}BfT_d-{a@>W$?BoNO`<zEcmLtrGAk%U@Utg zV=Fp_(v;$4vp1s;%yoBs)uB&sYmMEDe>vYr@ud&*k(Rz?lu3I0#{xTi`Me?9;<w|J z2Xs}{o%egVgP$q5lim}-7@|)2W|(rF_`zV~RBLKePp?Mk`)UtPi3+iOf^vHyV3qLH zr%YjQZ!oW<Br;&{%jMMoZQbr!^Cq^)Z2XycgvjihGjU9SKXE<^1edC^#cHHLg9Wkz zn=zLS{#sE&bE}!l$5~(sNPQb!*S4a`>FngfGwh!}mOwCG>EEmg;FmcuW{FcP$^vU< z1)~NMtc>q*3&3_?EPVYupKPPYd3U<W3)?lWY>pKm?USIs%tSTX*4LWf&e5r>HoW_6 z%6!iFbFP#zpTdU0y%<8B*tZ|soU03}&YW!f_DvSl=+qBO&ZfJBr``$Rh#X}I0fnyn zN%JohcGlkv31+8p?^w|-9Tz%ZrZ7Q5K8Wva?kfCg*~Fp#Y1hU4GV6f9*>!)Kb#Fm8 z)Bj@54^SG?HW+mGxAOz2{(s;3@gX~NKQ>=cTjmUJ;@O>#S(oA4k97F2pvPGlMHJVt z3mq>3I@c>ZB+3-Lx}a(%Iq5~Zhd|2jM!yB|&uT<rQI#g>mwx3H+Fq_~-Y5E{UxizC zQcD-Z$bz@}<?t{pxJBwMNbr|fV?KoJ(Uk&nzZ?A)#6PPM0=h-4z%TvM8XFGmILShH zxYe(*MC=<NAdnTw;a0&CY1Gi>q>X^EUvgE}96IY6dB*&^)$MfqZ;*xwS!@fV{WbRZ zaWnUd(xjmAl`^BhuRyVNdatVUxe9oA{h2T7XT|PRW4jbnxi1US6;-NLwZfEa2FqqU zouXghtm+q`4YZlR=KV)sALG#`#hjU~8Lko2DU&i-zWPcwGOY|6eMs-!(%Yd+RTJlB z3Do+lirk6&HTq@2owWBxkEMxW{xXeWoBM)2iCh^>+LYvD{4Pw-scG-ws5zDe)^<M4 zNW4leGQJA6ly=Hkw65-krad@6@7?ev7vS<nEphpeo>UQ^;AAUY3|+~T-^3Wp#q52g zt&?2Qk{Nc9cj(iek4m_U^sdFy3;#>1VouTb`zXKWL&C4uIiU7yKIH!50Qd2K$?+rm zEeFs<9{)9q*u`#1Kotn*cPqp*<)4-YM*`G5wuxowebm$SR|4D0Z#}S7DYX#3D3jXF z`!tYWue>AlAW^t*k|m*U2Jq#1a_xKl%+m<Vld>D2Bjq~2eEKT)!PrhUx_@<5KRulZ ztu!S74UTG|AC=&NmM<-om(5)ztJl^xAK4sEpYFxlo$C<#(YtcwNz=8^3g9MDRuATT ze>wN|o(XJAu#9cUj>FFM(7mP)?(2fzy8+4r<!*onDYLMFA0GqM^0>3J>MNl(?<B6o z4jHP?a8)0i>1~E|yY0()7f|h%U%&n8a3ZkKe<+$6KA9|y)q=LwaQ2#$_WQzGpezml zUe#TFB-Y^RK89L~RV&K68JDj55;7AhmhVf=c1=rJ!8TxE<u1R}KEGt;sMOq~;4;K$ zefb91fqqOyr@9Op%WA6774ICbv(;=4dW->Hq>;DUXdv4;@msPx@vl3+)F*IB8VodA za@Klw{^skE4^dhSXvl#nA71LBVh6tge)w&3Sg-qEEV}S)SB%0Ym!si}V1z+T;p^y? zzB8%W#VJlITe?P^ApKrx^#!Q=DP?Wh5L82Em|Mbvxia5G4nj9m79hlVUulW@b7{9v zW$6AhK0KOOekiYiDbK0vr-p-!@W8xHNa5^!o8?o3uO8g)?t4K_RvyC<+fsJ}bPh7| z4`v^kvgp+`k5#J(SJsSLKqeLJb+Qy*1$>o}SjEkJ_+IGkoR5Y}!pK7FL^g8Ta{fW} zMeFGPIdOnxi7_l>*|ts1tY6xug-o9yWDNDWI$qr%N3aakIhUua{1v6m$EQ^C-%Cqv zYF~9nu7*Wbrhh>d-R=`?SIZe%S|o;}kGP05RE4dH-WTmS$215+jyIN~{S>Ibg4WF? zB{XyP)T;z@x*O}h4)wLb&%JrY%NyVTGv=eHckFdy;Q*a3EO|u(S9yg7tUr2O%zVT+ zdaU?~D6oi$ktdZ+JTVn*@6xkuS%bj0Rtx4<;9y35u%tn-=x5LLAuAy9GfTLSK6{rw z8sP4Z%Mk_BhjMhZkik??GnXY5X4NK+(etY6N1YilT&<$gkfYLC6vV7N3|k4CRF)mf zLxXUep>j1Lq@{%qQpQrAHmN}zcDfXlIqOa9MYp>}yL1x96&;HM;P8vn%_nykVvM(8 zetw?&Yeul}BDf3;lH2;KyvaY$kqtZky@20wZ)rf64e)B(pq@cGj-`fU7UC?wn^-#X zQ8$D5E!%FLEnR8AN2tF#Og{%MF$M|1a`~|J^k=UE(Tw`!$o=PIO;P7AoR%@U;w{xD zWG@*F1Gw_~DCT1AvDi*fNMCJ}tqx049<^>ffgQDvz#7#<)F4kT)keRimJ@z^nXo27 z@Y7|hNO>KxZe~03Dx<1jl1d-;`om^i38`MgWFq$c1d<1jL2vrXt;_Gi#BTtG@Xj0H z5>NR08M|fPVlI4H)R7pW&&sroe|a#g{>$*1h-!I&<oA04!8QL|-;$yQ5i=8pdyGoi zuT`32T#K@-68r)7F5kuO+4qP^Vv4o2n^sCqw&8_iI=qxLDD*{Lx&>}K^)$uaTt*n8 zx;=ys3t#RUtMGL{-0xUGjh+!2yL--k!bwfXnt9^3E-kEmUbuKL*|CV_>n?ab@V!(5 z+Hdmhh2gt1rdgJZCO1>A^eR%lxS1K*xY_m>A}i6}MXcliZU+gMmjWh>z=u^CuJn@O zSNS!e%fPd!XV37Q^odFXxQ6$*C$$dtQ<u`6`BZ&ov<@{S<>=21HQb(O8tvN^*-Q4R zmBw&<X<>_RGPhjXfBn=sL&6+FmCj*^|Dk^h8h-dLs^<=!A1eP44a(EGTh7k$O^VT1 zSHgJdBjiaigLbWjN5MQ>hm31eF_IYeL11hdM-EJGxfYuItm;bdr$njT+KW?8%l-`h z>TYqXsOM7zCY9kFhhi4oRuKCOq4x!*^Yiu>UWxewD=zgZ=%m8w^`hQjPCtoEZN$5{ zoo#9QglAXun1<O@Noc>iISs-c=o6ngQS0!_3qtCLh~>26Y%Q=K;Ez^OX;d!+HY~XC zd{erX52)zJ7ok0wz{A@(aM&%9%iDx*xpUL*a>c1G>Vw_j+p@o`4Gg2x{opp9VamqV zlevM0ksa=R2oLuT#I~`D6SAH}Vt<G8o;NRrNle|x5`ku^nTxIKh3*NC?b4j(H4$B_ zcdYH<kNKu(_(!V#;AALNi?|!fGjGMBPh$&Aw(wWJHl&Qx8bGfm85j-(Z8dAd_E=tQ zk*~dwpMmyfCx&OEy<q#rL%)x-tv#NGS>CT0-Q(n}lH^|KUM0y`pi&wUmD!@W-+@=^ zv{^H3I4uvc#nv>I%@J9Cjg-Smj+x~O(1aTPX~OGyib0}7yS~*3=}nBxOQ*UXbQX?E zs}0^ikF8%T_*bTo^13<CFn>F<{tI3`{Y+Tna%Q-MxUc8bhg5?}jCT7JBfq0J8&gY? zXZV599<jkt@h+8EevJdY6!>lwR{w}3hKPLc@v63#i-jmt{bs{jeZC!Oro+SuhosbE zR>GQf6wwBspHzEY7;Jj!eTq_muw$ujl9vcazJYjArVpeDtwqgxGZPaoh%(hOakn0d zTNre*R>52EaRY!{-r}`4M*2Rf4ZU0Y`tEB~5(3$x*vA~Y@+W*={?xiX(Jd?vL(zC` zGwRV8mz9pf3-%0Sc_y(1f`f2wvB7MpMvwjid&bC$URA_ke_#=D6q}wqF|FH~pQK;8 zQMtgd@g>X3WCRy8a<H>~P3>^4<!T54ota1M!)g@L_uS6Pxg#4n5o5iqIS~LyC!E1! zC7q#rLHg4HX9{uBy{()>V~7?bRfZ?FH=pNbjN>fFfE0G^Y2tpg#Qk?d7fBU&jcXr4 zA0Tnq4j;v;>Zq|9@-5Fff3LQO@^@Xh^dkhFJQz~i9qB<c?2`8&j4s`N6Tx962!g%2 zoCSk-ikCW!9X>VZW*;+sF`uX{3aiZ7Oqs8pK8cnZxuDxKdGHx@+`F9p-IT(ic+2aP z8c8RcVAE?(bzzQNR6W|Pm}-Yxf9^8QL7S(&(fZP9Ymu+nkMwSIEB+bL6FZZ*U@1wq zm1Vq`0Q!z=qE_6l;!Vd6hXzu^xZtNs#+e_<!3&=gMH%D;hFpI=5lkjS;I>Qq<kNu# z9Vw;FThEiDjR_U652^XC?>6|}I$}uw-*xBx_AUJrKC_VlPFLB(p}UI24u<o+m|5=1 zELr80lT!4Kw8E{aN-&EW4)octjZjAD8u^e{t@d?A^F`3(QRQa*)S8mr7J9h+Fr0|k z&0Lvlt<<fnEZL?7@@@6F9XDj|hpXG0F29`#DPdUHf=k3cUHb+2YMwzY`HEThIK_Ek zt65PK$HkZb%Zx^7it#(-$%f`0VWaew^ID=elblcSEH=bW%8F{E8EbhHjdz~F%-d&J zLQ7iuv+Qe5wKAKu-1qms1=&bHa&(B`(5BCcV0*>!3|5h~MRsp9gXv(-%=VQhh3Y{k zuGA<7c@kaM(;&H0pW{akE9%`WApvRyzK!>5$E)58%h7jqF9wm(`B8gY+yIjsj?PWn zoo2|b6_d<Rx{mEBfo3yj=biOO5T53F#d(+0a26<YaSv2>T?vVf3Z`u|!fZylwm?^H zR9hD>E%A6{t!AbAR8lD7$O2l~hIcr-xG<+K)<K#n>gtb#Y00v!sj|FJ;ag66jdL(} zAy|2}t8c2o^W-9rf$8;*25qz5nkhIjIfxyWw8;?JZbl~M5+Ufx+fxO#$5>kTfYW0? z-yi*eXS^hdIocsbJ$ek0Ab{B7M=Njj6!V@jCIk*x!SLEh&4?oW!%|EYjhoKPy~H!D z)0@S2&E;w59ZRygOQ<$pM6eby-aY`A;Mxa0_a|Kx@}1sn%q*w(eYB5VRizqR34<#H zZiOaC!_5ztnqp~ZrC$Yz6kwH2$-mArzi`3dy&i^7rRnDm-=}}Uho|W%s{M$Y-%Th$ zw0zTyX(VT?it5t;l1*yoQ_X#;mtJUN)CBssF$6Q3WxQ}A`-Z3#goDI*5vF)Dx0f&< zyfS!n^`2pae+z9XLs9o&bUC`EHDG}jDwsVSV2*zE^W{2*^rdU90vbiP$7USA$ze#$ z`U9K%ij7}Eu0MhhiA?|M@d+(myDX&N9_+vR^NY5PCeoMY3Ac48%9blX)tDZt-1*^; zyPEL~SNVt5a&)lTK})DliK7n(#KIwU)q5+D7<Wel@L_H;R~<Rgh@fmPkC1?XjqUE$ zrsmmkjnZ{UtiayI$Q}b1wu0pK;&-i?NW7&iOW=<axxHTPc?N?}@mxNv=68Pp_4>WK ztT?`-Xx0+eOxZm+(>_S0LVmZlCRA3D_#uhrQ$S4TSYPK!Zp8qbs57yayw)9gVolJ0 zw3{qO!^v`CU)$)%v#sl1?$<QE+{u`~<P&yw>rv;(%Xy<u=`A9S3Omw0@UC=O(s=EI zWU%JKqXNaCvTUHqeRl(3XRMb8Ywu_yLb+vhokq0j%aV6jR3{*o4WH92;(Ih?@iY~m zh>~=|v^yIz+vlGQb(rxs`Y&S4x4y}D)TT?BZVkf<+1ywh6&GFCNY?R`Ef9)2E&2;_ z?j8OY3<I-|69fN|)BgX@;YBU;Eup6hiC%?>%>G$JV+Iw05o&f;3g+_37qG7<Eu64< zxcHO30Fvx)xF5x+xvsvZy&^u-mB<W+VSGrOk4ydB$nYuVXnFk=7hI*hjj1)H#A-3u z-aU%y1Kr`#y*=rMmW*kxEuE?FccU$%bs6rH=qdo<ETnW(w}^;}el}6@P+~0|B`tYk z;Gf;~f?9K@d_xo)Mot_(>ps`3B`7Rrqg9~A`4bosYw|yXc`O^>07%=X3t(t;|B$8V zY?i~}4bX4B>vCsj`wMLo-osNvM36#1!*{ZSREqmxcks<eHVpcC;{LNAae>psr--~{ zt>Lj?)Fuo6LZ<!D0KGN|<D4ZM(d9#0BomfG28v^r8dp&Vxx@`m*?`{TUqOyPGabJD zYv}m@+z@&{l(yJDds9E<Lq2QR*AdbbeaS=_maRavr@)UEVOY-f#G?8^c<9<8TxA)R zm$P_cWGc8c?c%W&UT2KF=|OC`lLW`or5c@btaWw63d_FIV&{&3f}dayoV}uHqTGv) zZW~H|BIyLO1$&&T+_^4AYjo|I3>ix~j@Znn;z&DT@_C%|2Q6d;aw7h7dl}kt#`Azp z+vOpw=nS^uN&zZG_-&|#i~h0L3;yTZalC1hf%42{CEe-i19)IqN>8^G$!PUFS0%&` zKk(&h_dEq&!zYG=Qu|x|4)^sAhUv8uXQ~9x!}Pn{-$OJWRJUHUUcF4STm(D?_goRK z$TFdlox!TTvcDQzN>=|LhVMumB7k2GtN%HPL`{|}fgQdDofl{wvwF-}If7h|KVH_v z*M1GZIs^P4Kb<Twos+kB6dd>#NsXVKpy(WtW%UMz2`vw)P$?;^TuGNlZ-$bz`RX=w z*i*0(2FbBZhUlXo@v(60JQcg!9q}q#O9AQmGqijzVtvD}2ucCm?j-#Ia!~ojP|sP6 zs$G^#qr=lvan{n75&C|SC%A{`*VG;{&5j?-@RoZnhERj2x*TtStUl$fk-d@28E&ER zme;NId@nQc1C;X~OlEK$;H)L*^~4J8&1!}neWkek_6iK4=^gOFD>Uiz&%Xg|r@7YN zd&e@?7K;_WgrK0Cl~IYTd|(SuadUKBnpY2$k9S(BiTh^0w>#sJa+rzP;f6BQR<{r4 z?Ef%tC9Em&@{4pBoBVeFCTtP@VsdiwKr$!dkS9EddUVf4aG%gzQE(ym!?xIqsj&C$ z_Zz3;3eg`a+c3syXvmoP<l37tLAhh`UTL>chUJ!-a;Tv;Ww0oj2{Eo%8O{_HM8EWI z`?&6{K2LOcdR;|#w^4t(`ixF<BKTFcod6A*=6KnNV(sVWdyEEOICT}2DClitjgCwm z2DiIfm<|fUnx0h3J6}j7Po8QHYT4GI;OFwu$|BY}EoVc6Lp}THUM}!`53UOPXkSp0 ztXDI@#%I5(*vw#qZSX9}qLG?5>~&vZP16m4W_tsGkBOh8DxU|jlp4%oHFEeTKFYD1 zxL;7C;C3;IvusY~`(mhTQj9DNe&?$0bmF^7KhJ5J_pDqJU@@85@RV}#05`H=gO*^_ z$ulyza^_W%Y3lT~E!Df9uO$-HT#dTlEaT{}pw_ieLWNJkJs*rHjCTm;{j!?s$lZlj zGSWSC^|l;18sYB+c9v(as}*IkVC%YpLH$Zlmr!mAX}91eBbC$My`03i&a8&K)t@kc zlP;g?l_0dwW9zV3rp!(n?`_S=s!+#hAv$IeNPcMdqNFUS{uz^!v=2+U(HE#%OX>1H zt#6EYEb1IAr$@C;#nCscfs>)Sox_=6mDGksy!oEE&d~-!8aziFiKaJz(G3uPOhd9V zakwSs(}7k2-7#9r*e}F#6gQ^otZ%A+75BYktzcp3Ti#Fg@hy(j;gTl@i(NbzH-H69 zU-w`jBbO6ug4HZHaA`n52v%GZB^ME8E>2xgX>O&11ugaG2DLsv=JL2kgVMp7#Hbvk ztuMK`ZEFwV1Bg{+t5$=qV!MO9U9L~jLOpK)T8Ha969Mv)I}M>p#gd81Z+sZdA1i$# z&e8*TFFVuLs;`mm#4c^!$=pS8*+)yi0VJh^ZUEtxi4T0v53h7@faBC4ylZO41oe5E zR_Q|?*kU(jOXM<+u;RfD5Y}4j!o&5mj^p&WdaCd%9(NvHllVkndF;tHmuxo=IgEWp z?sHQ*#Ih+?Gh&xiK}2&2V8>XII#F51%`DyuCWecng`W)Vs0{7&*9e<isiYMqaF+Fm ztj7cvlP;=G40W1{%faH8{YxK?`ZkX|(5o<(@!<?eeDiqlr^w^9khUQQ`dNpo#r13R zSF4Gc1)JBlHQ&wfpag>E!~sg<RmlgXJ+F^T`4(`JR7#?y1nfbU9M~auq&=yBK%}az zxg`vjGDD@9Ol7;*ALwMT9xriSm?=Y$qh7%h)s;*tQ0ggY-?Gh{EspQtlG>YJR1dl2 zG)ecgAbp23<!zi95udp#yOkK^&-8m9MvkttVYj`eYNPsGx8k~QS{vTFS`KTyZsL?a zEfwT~th=<YRwJG%r8_=4`k_OmrW-9-u;M~{T=|jh&kqt90L|LBp&_2gR)<ds3}@)V znzZw`#k{UV53bLYn=c<g{DQ8TRm!Wn;us0LdvFr_R*<O)HXqmyK!jDG`fxN>_ktrh zUbZ7$=hLf}kgIpxmPra@JWncKF^l*NVp$!7)co*b;XDBgd}All_`do&P@z-~j%hRd zhRD1LbbOrGfW@0S(joO=%r`xhw&4MmOD^I1<)_UL7t|h7fh1R>Dyvk<l|Do0UFOM= zPkKcpzEg-As~1CqCpX;=^Me&(95;Y@_NF-@{GNpIkJ&3;-2Q<aM`e!?>dmkBB<p`< z$~tDjq?apK=s=qnX)88a?`v9KMUn3@oSP8LvaI#wVg}|n>3PS^8khCnk&hAV6Vbef zsv(KuDn;$_V<<Y=$Y=gL+?>d?U0l&0|LO<be)+dRbrgBK6M3y<1^>XmWK?>j{*fK^ z-x{y{Cv$<ubd6YnKhlf!@S@<#`7jNIRK1BkWYTy*gzEj&vAkBF<<ukRFNp(Nzf$#n zszI^3`B<B(VCKbrxrvKw{lUmS;=$XCNkcUrrW#>z|9s!@FOv%l-vjz`t77z$CM<sT z6UUdW5XE@cu1`t~FS0|9k;_I*)pLINO5BWCq6HzrOAC>R7;1+(q+tw`?9=5_#no<u zuwdS|sNMWgNFJtXw~<Ho6#J?RA__LfNmJ#be<VTM6aUGo%CtMA?o~jgikGn@rPxL< zNq45c9X<p1N}Uel$&9j;=hMoJtn2>@+856$ug>;!^H-=?P9il0w@Z`2J(klkam??- zOiu^I+}lN4m+rif>{8Y~YygkZjya=!*11hzEEgVbVKK)ew!c7&hO_uBMP4boevU88 zZG2%vy>mna2#E)uRBntSjQgB<*{lCBRokrv3z`gPoH>i|T>KQ6e2IHt<d@-n14z@S zPt6D&Z@xJxi!>jX{siYQc4*PaJf$*p4k(@V*^!;~2#H#uXv>??THAEI&W&(SW?wix z-!b_$l4jCba5Y#+`NnBV*_(rcT~ac)*A~gwgdCm=1=$_+3{aBFrOel4WY-*#9cG*d z^Xp;N(@Xru@|>G)IPK*N80@Y}t1C1`CHqgDOFc%m{vv7Y|7?clW0e&ws$g0JnBv#; zYN}$zmIZ(y&%yi9BSD#l7xsOXhTG-eNoZTS0jh>xTMoBSsXTTLt(qC@FF4O5`q?@6 zbrn|$9bII7jpTJ6{0YqdgMCs;U`7k!%mncZrc06@%q&29`~q{O)#1JL68&WLMjwyk zQXuox^UQiL4-dMdSNhq3F_YJU**8Gzlj;JmibNI3-k$Eo@Pca_#7_WxF}wj_5w5WD z1K17lt%vLytabx*YPa+t$eGme4X{|DIJFzRK0|kT%3<95^*acfx^#X6yidTpg!0?~ z1dn*S>3~`d!kKH{Ug=Y#*d6cIqg}kKOX7*qoz#6J<IED3Yhp7QD`WG>Ah2@=a|PT7 zZ}SmPmj+l^gJ`og>Rn0qi09hnOknm!d+U-a1WVbaIe&_-v>`Z@%)-i<rE?Lv{tHw+ zPO6q&I^EKDROf&t>bT!gyqwx#D%JmOLxew#9=VMn@jY_Ow7`|oj6_qoaCTkC#*8DL zzmD`sRxldO{u;5azslp}bL<W9I(vRk(7NJLPX{eZl9gC_C)PgxY%v;XO6Ds<UlnlN zLnQto%XriA5nq3u$tsAUm%pW3{_7;<ur<jjM$OVXcVEB~IY_Xx^?3mMRe)rEZ2sri zRrZor95@}z!mv}%Wn=O=C6VLf;p2U*jZ;bivWm?~JTcSxe*Zf@dL6yfkME)NXJii6 z`<K7Gpq_kFMH$v?U0@=vER|?qD52+`B<j>HPWuC&Bj(ML;)}G^jiQ2aXJ+%&<B%nn zFocT4dW|90T5TsD=iUG|`Lk*vEsYuPmtDY{Vke0&QOb^|J)t#&+iLIH1V8^^)3KO( zhKkM*;hp>=noi@O>pHC1%?(nBfwDSDM8Q&LOK#&Mk<=6Ha3piEH6jf=orG(+Z2oj` z<~<*a{4t<T2RnwYM@|l_1+K-8X*-`a%#kKlPMw%zZ1BPyAB3xhU23*O5PGWw3#ZbX z^*4f_5979{L$Zx#VJVj;%FMz(Nl;^@m(kBHIdz^#t*?u&k2IBf@=$ajSWhPY3Dfb> zPEIP~N@Shyq{m@1U$?ItH+eq<@4r#|y0?p!oIj)lMGM`jZq`>E`<b*ht2wg9*CtuX zUh(MO_Zpe6_qN0?2n>T*V16IA8yX!xZBY(Je^iR3KBH7ys<j_~iuNNU*2H$9?quwm zt?i7biN)&%Fx_2KMJP0Lo;e9zE&R&+^-^m2#gYdnBCn$5qnTuQxkq^&%qMGLNoQD# zJNt#`^;j~m$-O$`s5k3=;Vi){Zwwo3+tcs%8;^RuuQm<V94}#9q#3`0rcb-dM6Fu7 zuw<bvRE#dOL?ec1<!tHVTl2!CZWIFN*?iE596*D=gcB{-Ng7e<rYnwx5${vY=_Qu9 zY@6c>1ZtZm>l&9DYb33P_~C@E$dK%z^0LP?xpng!OW_`j$G)tRAF0Y^t5UW~e2_=j zI8^}#u#sFvHcdQ(shY_1aAX4&)ufWW3CpDS69p?Qp&DP#>J9@mK5z%GSG8zdhewdH zPf@;^Tc{f@wS`J%G#sjO??UpcA0Rt%Ft9O3*-F#*NL8an!g(y0Q_;Eh^Xgux*pI9& ztWCe@M~-l(P)7Er4rwdale+WW3p+4Gf3(bFw69)qXw^}@sQ==!uThkT)E7@3O0y%p z>tD3_v_$J%+GS)fx@mycqv91ggT!&Mk@SmW2#*YcCi2bCE;<V*m_4*f+W6FFPkpEk z9QQbEY~+8fHf`UR>z1%S7{p=kP|19lYCfJHqFZ@S#>0c@tKkDC(*+~TpPFcl>ws6; zViz6W@j$Riq`vS>%lkskClt)1Jd|@o<f7=XEz#YOebZ3lw}hAbFoS?^YXTZzJA_LT zvEcP|mpQ%Y;fMx=7dw}=P{$SbUe;=YyYhgQy)>49Kh~w++F13j!o>K5iZRx%%!is< z>&txY*&5}ti-^1gdxZ7i#L;N5bg~#W<NUGM1hFW#rZrD;6s;jrrPc0kos*?U86G2m zL0+8F`pnZ*u3gWJ_LZsqsUCO2jIuSaa+W7!xW4Fly+qoH;DnxVBwV?`S3{<5yxI?Q zwyQk{yQwT_i+-;M4pm8Hat>L8%zdwZ?c{Xl^?HKti+dmJ7rEOM7i{)?`vg~7sRQhn z@<&R_ss{Wuk~@vKj)hO-I(Xh0CVu*!P!Eu>&|hGVR33b2gyK;q$FA3pR{5bj2ICn& z)O6Z$CVyN>w+D@NmiM{f39X_bM2S@XzQctdJj6;&w5QIeQJ=TZTm0Z-UVszKzX<)M z8FSV1IdILU8BW1)vZl1rF@$e#;q<#RXW5;)=jVoZUAGoxMarZfjR?#?Vu{oq-Q{5j zZmMkm3_d~DM+rRGTJK0EbXdgNUzBe}mPm1|aHRU??J4aU-@P<CTl!>Eu%f6aW4&?g zcEwj2!%}%n7Bn$&Onas!wcLL2&X~Zo5)al&p>$_>ns9)QG)0p|5sahl)0P;v5Wl%v zAWlx{R-*!V)oS-~p!fvW?BV6H;HF3YALPIs@e?;q?us4ml4!H_rFh{%!YxqJ{F03z z#*xv7Q0ACkZi@&FJ~plHM@mB`baJDyW|U*J;W_^&@BLDcfbWO1wqU$!HGh?e8RLE} z+Rl1CyTMHob=kP5y!*<H92H%n;tUOc<RYxT@eNzi8vZcaogm<M*LLjXwQXiXR>7Ot z<T8)O+J~Eu5sTb(#%r0x|4N4+QT(}%px))x0zWO*aB0Ddt`nL^*Snlfobl~`qtf?{ zKan%009(JZs3zmKX$4`gf24X&G<oFuy03HuIWYCx9jQ8Vj>iapinI8_h?mrCJD@xw z`s6XR_jzKtEFS&$ksp5~Q2lL<ie0geCdf=2Dyr}i+PKsm>n&{XfV&z@gk9WE84$iz zklEe*r0HyVGW<m4b%6Bp5cW*ufJ;X*7vGF6>K2;m=k9{bPS1S*hkX7lSJH1LS4klP zwT>5;r;?cUYL*=->aUlADHb-p?I9i?XEh{ei+PW%4pZ0KL})Zr^(CYvU|b9a&YAkP z6X-WUoN9{I4dAeN^$jwc2_w-O4v(m&!}3LhusO-}?rkNF)MVxkP7Aqms)8)woO6Vd zl?yqauja1k;KC<U4K9nZMIwFGYI4d~?*cHS5(B-g#m$TrMvPUzSIBYvRDOhu=Vlt* zWlk=zi0=fa%wVlK$bT^%SgSdf;k%sJ2Kk7sx{leiPc%tap1Cc3=(qqzq@`bKTv47( z2B>~_I_8O(XyJvDg)>YxarjuAQjJWg_?%LZe6A#>Pp?~{e|ib7x)u(s&%913g{2+4 z^ic&f1`7v&$#z_h)ByYDb_{Q{65vHvnLdI#IhXQyvAQ@zrmt6fw~ru*_0>I6aFeQ9 zC{IL?2y9(8q|2x#IcrwV;gBnDv02uV+(5<pN#C5|v%W7U#ZOV|4~{x%nMNQjG%z&f z36FdJ>B<73Rbg%V>6Z)C$-PSoeEt!QKg`cX%wM;jv?E@)@;do`2(H(B&a-{6;W(n^ z7i1|xz+Aa>14PX9bRcrC2^><o6Q$w=sZ@%m<>qG-F42$N68sp6M#11xkIc%mP^tcv zSSrNtowzt21=~3C&z$p{Imhh3km#GFlSn^v8}L-Q0a$cyfK!gxE7qb`bgyDMJh&Eg zGR(`5v1Y;xVq-kZYDCrvCFZFU5;~^oJsl9)%hX3CWy4?S{d61%-sCxv4A(>u9IVWX z)y3Y)!B&tIb3vFCv3(66OuYMPl=rT9^<H}Z`9voLMGT$qW*VI=P9R5Na8KyamAlKT zV_&QF;`v=8yvs?t3(1Y>tNr9H!WpUZL*6G7JV8RR@OJ$f=1BhnUu}gk)LicDAT6sD zgX<l)oJzNh0ugu^B;Wzmcf5sh&R0Q$TxN-ePQ6;WysT@?`%IXxm4jFv&Wb6+=rL31 z``¬P>`o=ztxB4JFROdBk)B=sUUx?M&_9X(NI}r-e<n`J1$4u<F3&Oj4LTj!V1P ze2EIv_EhPQPbv3Mmm3XUT7%;!rN-Gl<jxvj%$6^?B)>ad+qZT>h_GYvnmp+F@;eEZ zl>xBgsKnD2o?)+KM?IBdSlM35TDKOwb2XPP8QIO0EPBU<TX1$-!V*gCtI-keuvs%> zmX+l3_5S&SGL5@>aHu=)#n?eu#Ser-p;0w<Y5iGM|E0lHzgrpe+Zs1RlFLSl9PC5e zPd#N%SK`SsNDt1sc*xbD9_suK!ai+Psi^}}Ds@)XJ-u<yN<h@MG0m`oX=&-vYrOtp zW7Be;$!oOZf!2@~J`Sj`=|NX=q;r7oQgNX8?y%W{@OuI7lA>O#FAMjkht%KgKl0`7 z=%gd@oR+ZgTfXvnP+eE|)?QD{q?lOiL5!|y*2g1d`c=B^=kY&M*C(Xy{Lh^|{nP2x zS`M~l3gq1ILsy?$X{6d7tcKHjb#o{#B}WU~0HIQJQW(%G;R&BpGQ!fA@zec$4!n)Y z+63|pJ_AwImazeP(adMHh#yP`sEp=)(DHW|%^MF}Zh*TzJ6kDl&F;WrX&V{cxBgdq z?-|up`{oOys3?N;UZMih1Vow?6HtnPh@kW)BGQCN4-kk*uc4zL1S!%%dIzbI-a$ZG zKzd220YW%ipZUM%nRVWE-kDizX3e}aANGf=m1OUoo%_14@+&rZ>HW8DDqg)Pkn)|9 zWA_@*CMchg#Moh)PA*O(RuYP(Pg5kWHt4lA(H2**0oD#yqgs=bPf6P(B}d;NsG0j4 zJbBweN4$-EXCz;B#)+Y}U4^F!O<hfUai**@qee$4VqE_Xyamo_wiR(zoKHlKWx=Dc z;7y#WH#F-~r6o&uLBKb`?RH!1sw8APA{fkwvoAtAXX5pgB5mxK>-lcg$0$v`3w=Uw zQRHn`jAuTFhs?{%U;lASfnXP0;)a*SnY`%~#WS^QqppMY-K0`R+;pIeDu){OBAAJ7 zcc|GqRE!xxzFygg8Y3$DT*n!Mg>Xv^8Ma_KWb^T*@bx#0lSD!Fx=vgoRP^Zivz@tX zb`R&b!^TzcNa#Fsy(L+gi;42+a#@{Mb&;@rR=8zG^zioiw5TUkoTEWGHZyEkvbJLS zp9Dp0Z{cB(L~i?OY6&<=gm&fu#Dk7d(48O+0{92-_Ngwf6NL@rDCg8Kmj_>e4ZdL_ z{pOp~<(>IUcY33qJW;;nUG_=RGznM{{e);&DW6(TGeQ+Xjq_UBc%zW;XG|E1X+4QE zMuf-jQQ6Qf{pTud_Pe*bP<LX$t9$n&NquN+eRvZ)nrJuVbn?_aT)&(q@g3A%PCRsi z<38VeFegY|y*LUX0h+|~l=0jCayA^lViY`XCs_h=9pdhTkJ{t-l_2-H-(Qn`-gM5D z4Mj~*#OHgtTY^zSKNCJwRhyJ5Cv#t^c9V6Bxxd<ON~v;$0J-C4eyMd<o{=PD>j>8p z58jRxS0MalB$o2MoV`7``Z&El8XB<CR<pe3{zP)o2~2|zK+G2hAZX+s=1d#gN=dSw zj#ea1_Y7U+?kLIT^}kr=8z~m5>BlKETu(aBOrXKYYG{kl+wAtU8^U$9yAI7WU+aeF zZ8lGb$B@A>HN_Y4c3L=eGbq^e&C|=S9`J{bZ!3TKo_{4Ojl3^?y~tBXBUep3a_o#` zDx)eT;G!Kr6tPRV7R@+ha?iF54ttJMS6X_b@p8mw{gznS#wod9hjS`%WSb8De6&L3 zy(uS-A#QdeX4Xu_BG1eSz2qH1TG}aa&2y7^J)*Ty;0p>MO$E@bMjNezFO`<#9-u2t z=<#pGH~HrBfOA|~Q3VCKRn?}JrNKpqDmVncxiRV@--2`ouKPd68J5USFr4R)W#(qM z|HP9{b$`=!waKTyc;+!Q+uPfXaKb{oHdy>jXIKJq9<E%2lZ<=ak!zQ~Cg>)YF4LhS z#ui!k<lL5uk8|4$6>;$N%LY(&&ELdv6i@CYHtY&PE9LG*AqB*$eg!Y^c&jdzz!jK$ z>(z@^hT%*lb$;!u2$}?Y@Q|0j{Y$dLz!%L=DIDKPulBw5(hTuzK=~2DF!wf}8!_r8 zj2f&)H(YLb&}%Hwm2Y0yVSd`k*Cg3QwM%dTO~mY&9qS$Wkkwv`|BHsgb0d!CyIg4M zZ<6A>zdA^Iqx&Wh<r1%H)8ny@!Fx7Emihynt+(ZixpByjGm^`_8EXfr9QF01uAnk? zcF4=J_~BkY;Qr-1)ON(i-pS>+#~|Lo+3_DSl}Alg68_epu}Q%X8DTrEdii=Gcw~nR zL(>Mfk*0mKvb6GGeetJ3C$r6kRj&D9wAGqNbJ5r=#yv>*CGAvY#%+}hw7D25KA>>; zQHyF>ReE(9z>QkS89|@6E}fBJY=VHds_2fWoL2wt0*yM~^e~tjG#15Y%zqwJ$DNTl zh#_#|!#J}Rbz<r966CPp_ou9v6uSDhLg)1`{qOY`PTr7i+X0Ne2>~5^)?CMGQ7gV3 zfImzCV8GV+16f1CD9sQsm9uZLAyv^lp<Q1S1k*T%`DqZWNRI|x7_BswUhszhrUbgT z8LBLJFcT|mMQB!>&PejcT*l8x+*Yafvt0zmbVKtQGb}}p^Sgl|mzxe`tRVjqi}vMx zdXn;gnnwO(DtUwC{pIfx$?j>^haXQ~$7cVuDo%cbB(&QM%Cko-w<5@<mwN~7nIOB} zE@sD97<5GfYjr+!MC95{D$95WX+?|f_1RUY$44D~{C+EJL7nR80*pw=IU`B%49Iks zKDg}xGE5brhCdyS<f+2;l~8X5s(WZI?}V=Lr8he>iiFDaegI7o&yU^E8r@sXXaE^g zunrD4isA*<%rEw3-}>l0!|`Bxj%)4F)bVIiv6rQtH&eWqJ!PBO&!jsGJK|gIpQmh+ zzM%4&msj1#s=fkhYw9)$N`#SUJ;63;BP;B+2XWZaMmQb@-t0&blkeUm{RN8Gd^1Ze zC$2D5k&K|#FDBHkb(g-q&uDpqp03XBISvw7@w)F}noNS*ZZ|qr-c(Ft3b2%PbCeR~ z*}kNDc}49l+2`i{LI?$1Ygj^XK5Km}5zIOERc&%+r*xq>{yLC2KJX%a>eDX@!t2y} zZ{&xiJn#yunaZ`Y`Sc2PsAF9ltMEE{6?g&$zhP!%9|_N^Vq1<AQgC#S!cU9UEW@-z z!Iaa<)CJiiMG6=8)_>^CPQF~xz*ar3E75WS>69f<vsuS-$FtlQLxcB6VWpiCB3IWL zwR2uomnGe#^8T*m@j90=3j>`f{)U)1aCbix{b`WTO(Pyj%lqoKeCIcE=_13y!}M|2 zql|RgL+*OiVqZo(gc0sO?=4ip;XH3NIU#t#rlXENpYzE%@932L)Me6{@<w*+$&{4- zU^oJkoX=9c|A8z1#_;0qi}#b_JPc;|5cZgRd+VmL-ZoH#<^7Zm{`&>Ea_zPSfi>mJ zTmzvyQ2!*;lDSha6TEkxfi06MQ`G&A*qkHbL2hWufZT^~=CSWIpL#1<6y-v#pzFs& zD^nUG75l`dqVS!<tL+F1xQX>~e|v<_*1Adc$HmC*Q&IEbVaLnS!EwTF?^HV%kC$oz z&H=22((2nReK{pWzh<o5TSPO#MCN!AlJp{m>~xChTx5s#Ij-zVLFZ<f;^0O)8{U?r zaevoS_a_w}&q&H#DQSNihunC5`$AYX{q@-K<5%0ZGN@1qR3K5Z5vLpr%`oiD2685= ziB}a}?TbylqCfQ<i7TWWyGrU#>WOxc^>N}Am*Pze@1w-!Bg8jpNC4`JcriWU&VYmn z<Fu_*(x(LLH<qDWj{;GQqRb)B1HLnJf0dRla&DU>W)%4Jf<?ZLZKWP+4OD<Oa)nXb zPq?a)cCkO_d{eX8$HrTb9QZIryk*+zE}qwTw1{<tQf6IKDQ438=m!rR$%m|UqI!Io z4MTo2oJl;nhmYwnk`}4g$e*E9+nVW+j5{jTWMg<*!)qJqxZ<TVI-@(aUwbNoXzL5a z0;ykClTaa^HB}VI@4B%FV#79M@4>JOlxl)tjB5&ijdMxV^Sl=3b^6i#OIA;`Vv*lr zOl8M~l6%eldmZR7@d!0U7=m2n`tUY)dFlHNlP6InkFGvQBYSbDMvejlM1;2Gc|;^! z%Po~-v|{_3-S~MT<IoMdUg%q84%52|lbW)%Z|~!tRg40a*x@cBev|0NW1nq(I)*50 znvcwHo?*g)0^uSU4%US>nyX44*CG&N5^a@l8Rpd3tYxXVmbg~#6*G+3gK0GFc$yAU z8!)p>?N_LB^(7W}>#HF$FOFVVrFAlI+0`N_@ZIeYDuO#WTWC5V3v=#_g!WXxE2i6q zoGL+%u7ah*-Gam801*U#hGE1$H)X&sw-(wx9!sxa{9^Q7dsfr8a`WyL=S0%&7yP{Y z1zGMs2c~u4MiccgwdtZ$8JFD^s82s@FR>0A_OM~J=v8gqP5mUBYbGs^y6TY5@1(a) zQ+|tF?21ejKN%56_#gXxEYLU6H#5|9WREsh94t9qzhfSCH}g(2J8R+sIz&zth$vS( zQYnXAub%4S2+=}W8OChS(Y?1u$x>V8tX2(mW0ve)PgTE^m+gu*O4`!Gax?Ju^Ru(I zNgq{4UYkDCET>>e>3{u(gsIWH!}|f7OIbx#Wf<64^tN4UH7V(lo<Hnr`=q;{OV#`S z`Q^5{<n_Q^D0EBQWZ0y8dO93se9doHVw+W|f68UbtN5lD0xS{RnIfsmDKBf_b*Jz0 zer9~)@~IoK_6J)XA?VM1?5`0f(6eFC1@J}x>6Je0La<DJ6_i&oWRb4xHHD`A#t9Rf zF}EykULN#=UX<rhzN6uJ@z12zW@5H$Ht~pbRR8bj+Nl3mIfnSr)0E6H?DX_hvrsF2 z0C$w2+P7B$7j}H&eg4vEa6-@!T<zylZczzOS5P_-&B|6MT~w$XMfMarjH)=(>udpp zTo94YjA<P~4K2B?A-Fmzkg6zI5|cGPJ>>`qUa`&Kw0dbNtmC91IKOxNYsapM&LOZS zzhsspg!{(?=^$4ywhaJ-)GQt|RMbm;<(B-4rr^6%(((DZH%YE_Q5FM7q}8p%n>AH| zYh-}{8!@1RRhs@TDo0|Ob-C4QsAzGF$YxAQr$^>Bab78Yjzeo<CG&tQJ53JxI6YZ4 zM3z0fb(^^rGj-Khs7~Kb@~#JKAnHc<)OSeICw3`BOKt5=e<(Hb3Jj%!+YF{jS4;Bw zn&Asb$qLkY#@SaB$P`SoRl(u*t_n)@I<yjlpzXY>k=LKSu1cL8RBrBvgI0C<(q6-p zIAi&4YTRvaChpgQR&`!sQ9(e*#vsjxin#v?NO3F+^Z50Jonv5hEq4(G*vj`a%bH=p zJS*=R{4Vx!6Qbq#(HV);wBC`Pxe}v-+uLVw5SC(Mc0ON$xbVHHGvFD*7tqG-ymg;) z9;&}>1v<I!sX<?(XTA$w{s?;SLvMyZjMcB)+o@o=1i5P~hP7>nEDX1&idODj$^C`6 zS+ORr)m29(CYRetZ|evUQBlZx`P5Wcq}kF%<=Ja+8zzDmTu~TO2S(EQ)Rmc*DX|Aj ze_Yg<+YX^VBgw@H`vG)0lZ}sJz;JuSKArH~FApZOs&dF(tql3l?nOyoLk+0*ZsP=o zPk}{xGADDI_#C*QJ(&Q)GmN0-(G@rZM{ZF(!61wf%1%1y;`_A=l8T-$#Bpi*5g)Kh z1#gIsZ_3yeL2eH<<20MgB>OkxLrx05s(m1gGi*s*98Ia1w6CiQblr<4r5cOW`zc(< zX=Jk?LTNH*w0>+Py7Fa})JJitaxO_QxV2e8EhvNQ66iyr7a)-VdjJfnZLxik8=haw zl%PE%cIOx7s(dq@c`9Rt;?P7xLz7+>P~e77TtUl9jgo~rHqy(%1;Pf488}e?sJqIv zQa}cZcoy3NqAlMg24I9p^|XL*Jcw{W6HgakM|fRI=}QP*{8f)5CW+<3gk^uAXA&Pb zDHAf@h~^=2ysD*9D#a6aGW7lntPx>pyg$Ojy@skc`es7AhvXh+tY7C4d{$NmnGUsl z4bmFX%@So!6&={vKRv%TbbRM+04C}!?jDQYvmbs>eY$Hi-Dk-Q3&ulS)<e=9XO}Ga zAeu8iftB09l&WKaSW~rAEQo)x3x{C0gXCU>aHq15781J){5wzN!US*YG4FNIpOI)Q z31vA|MIf`Rb2hU>*Jk)PvsBaKAEIyGv2^9S{$9pUTlWj_XMaj2i(}G7KE&Q?$mw}| zqbwGBuRC5K1H2(mu(Oz#j1~^ScDEe6wgjXKUJegK*7iuz;2Ce$FX(b%5+m&iQZFcG z7zP~}7Yu7aYg;^~+7p^M=WawW%i#xr%Qr4zV3H!f@apd%Qv+5yT}YUC|7H`_T9RJM zqId0(5md@yGkK(F<CPUl-(7+IKS&XcTLoyAd*$N~KVe0JfTEEB8yNf9J9`uH!uQ24 zp-IF?cC!<`WS8;3C5dt?rziNi3qt;m#Q0a^1OA+wB=Fy?SwNPuHp?c*l4V<!<oatY ztx7lce*I^cqYq@t0zYoNRaRKkg*S1+$$Ku>jnpCrBoV}04S&(TNj#(oA_!7M{|ta| zdOrco`;!txkfs-Pu1~L-AXeBQXC!o{C4VVV$+Z%hW_=;c+uCO&%|J7rprIs!Mv|7D zuVVeYK=h40Bf&|mG5lffl6E%Dd*VgP)|OYl{H_*9O5ko->t`M19-CRc!0YjXKXa5c zDxj7N*q*;Ou%5k|r9ot%Z(=9#_3ljK-vI@sd@iEp@0UqM;Z}m9Iko{#AE3^J22ozT zH!SzqV>7e+?^lQn^0FohKs0Wug_=c7UcEE*8UOY7mxTz9{Mm>H$YK#K>w{h}L_2yl z6M_dW?ag4zCzxG_M!!AB83{1b_)BldBZwf}D^e8zf8&z`gr*23k&YDUWaMT1LiXvo zWS~7>{PZs(IVW(3SCi0HesCAyod}%HZ8Mh52cKYE=-z&NU`^n3*8dWs;3MsU%sV$v zeKVs*4l6<kCcsxdng|4g2#NrCR$i0X_mA6!yPjylriiqKWQu-78Wg>Cqf&PhkSBS6 z<nCG^cV|c0?m{mTEO2Olx$3=X2b+yyUmLFze8~!_Q^x6#;09GBz_4*><ejiikUS1& zg)=5n-5#oct=w!(nRn;PakPOC)oE%y)qE^_qFaoe`@8d&O*dk}gW<xEnV;S(q*2&q z^r?Eg59RpKdJAie=Czc!y*V~{mQ@k{kKMr=9u~-L(W3SFi+KhFVQjfpz`1g~Wy@Z7 zjx+uja>3*vKU2YDzHXCqnyd0XW=!2ue?%b7Qt%rGnZ%OW`R(52Rt@19oeB3Z+rvd% zTQQ35Z@qElM2QNHp1ckB5iKPb_W9ot;s(xVR0PZf`@(-dsun&pUu#KUO>oM<72x4s z8F=S!sdV+@4koWhHtlu!;@#D`WNsNbcv38)mL6RKMNW1Z;IHjIs=VZFVF)rWEf05y z(QdnR-SZ-E>PK{;M^(5SnWP_fXI2k)8*|u(bhdcICaun=t8|sZ^xHK}XOxq3tj&rI zebUIWOekL=MN24O+WAmE?sGBnoL}CN3)Cl$MI#DoeIYYZ<oUu)8STU>-QnTW^*GR2 zRIh;Gc_=H)6tq9XJ;-%N;zPUv6jwU@jodI{kp@g5@Q<j2MGzvG6kfZ=>0NciIe&Xg z(G*gEJV{2Kktkfsk?Pn!PpdnnQ(X&vblVEL^gi+K7Fh4cUXoxhV*c$aEdreofXScL zEL(lzd=&i%%sbl;;3mwd;4g_N>oR>rGOKiyQ`yi;vBVjP|3b$wup=k7%8u=wDCbf@ zdRQS<jl4e*x!BjyR;VNV=mmg`)p(KpyR^aTbG@WZe2wd2Ce|brv{l$v1Of9y=2`Hu z?q?(*Q*deNv(Yzu!L$3$(JE3=Mm>AAg1wM=gSAAn9Y%ZrrlxGf@JVGFk(?5Q&8ioI z5CV9{D&+Q0US|mca?zo}Cl?c)Yn#ShBIM{0dHROF<Nei&Kq&n%;&U5Dr`bO#r||-S z8KK;OwYEYs{02-5yVmAM75K&ov^(W8d(0v+Y%{|-=+c9Dy~^tPMaDaIf(yNPkrQkE z7SIQ(5m!eYom(ODjFlZ~M1I(y!u-G>@fvXFTy|>|6|XN9p=wklhO$YYh$B#XTGAUF zUM)g@L|77ZfQoD3u#FQmz~wOMonc(fNrbW-YfH88Nr;}2FhE1)REly>2Gm`D8KF*I zH$Y_e*7$yzJp8yPcd#(m|1PUspGiKHqrRX-<L3A5akmW=S|Y*~yWDn6a~CGtw&~#@ z&FWK4k-uk<xWw5CitwDKcS>8b3aSd*y8XqQx5M-C<fSZuyWe*1#7xdDWI9EP5j*!u z3ti35Z>H7bh0=^nTG<bFrn+L3ZU*W-<9dHtj@k~2`6R<Vb8bu}gz!0As7};ICVyd( zpE%<f?vBZW(+_XC^auE`S2TpAe=>r;GJpSh;?pa)d*<zGx=~a0k7?$S^sw_budpm( zy4YQ*50eawS+D%YH+E9dCxyqcx_S%Okgp|SiI2wj1bq)<wwfn=@;Xymi{)I3)zkCF ze&{tWC(gi~uy1r@jrBWo*Hl>(A(mfbnK_;q^O1PQYQ7&wL+FIk+pL<2bTvwDao@R{ z>Oum29y`)TcW6SMIvUQhW$pOT;XgLmlOtvosn$&s=T?0B=QVC}rkZ5(A1SAo=JE?s z+O*$Sn_<8%#m}42Qt~R4MGke*w20{5pFrwb{7jnu-}~=Z$^TG=#Q$H*(!sLB(h}#9 zNMX6h(V|$lrklcFES5K<ye}*w9CvqVW+5AXTNRqmDykZ*62=ZL7YTiN5ThgzsPe4& z0_Wu~DJcU<wPn)X9_cmjr3)8T&7&V?1~<pY?Of){$sSMbodeOsSUqmx9o?^e{#?cc zrCPhou(La&AbQfIJm)pS1fnYz|G)#K!&9{<Ir^$=+?IU5G(p0`+oyew!=v4$qrh}o z_RHX^L=Wt+gE5tMINd5_TSC2Z=Zy>ABVh@M+uvSU)_K3BV)Mo&`XG6T0dq|ZW-TQw z2_Clz$xa{ow#9@xq~}};sRCqf+S-63UGvwyjZ8+;dz@OFDdSX#`fXH?otXFgRB}a< zj&C?k>6&6eep{QaSKykMRqfq&J-hPwB|+x8hnF|VoM%x^?r`Y5mk6^ptqp73_*3~E z?)OG|S8kSmeW7qYGGnMJC$F?3q^_YKh{6FHEGNW!xCNlOQ-C<2N3>*=L%iB&B%nn_ z2z=6B9>3G)T`?fIKeY)uwF)Mf@SaTOBLevF(({BR4V5-+&_RW?;E;7jJWymU64<c? z!P;|cyB(gHNuSz4In$3pq(qRF$Y=40wZkehpg~uxnjWe1LmK7WpMe*{5+QZ-)l?_L zYO}p-J!V%3S3uL)zEs4F_H56%qY``mS7-1tKUFL8V*8<-U^?n>V*$D(umPsP)-&(G zrP9&JE&VnCzB!Yg5>?c5;!USkDP1*u`z2(<DE*A&hTiCdcl0D=^8D6c&H>vtAW-lR z7TT*_rSnXwiV<WuSrR~@!JDCf7C+u{41?I?IFVOCEr-#8gtDZi;zn^i`CuibtAphV z?c|KXm1FAaQyXb+SxHYJhF|D~3sMMSzqvCKM%y=@GWzsL;xR9>3xM&WLOi4TKF+D% zm!_JI)gIAl5`TOrt0$<d|7qrH@~jW<0M^ogUOXg-D7elYq269A8FJNCL%63)rSerX z$WNQM*+k~l4`0@1dILv#sGi(X;zKSCIvi#4rZ#Z3QpZq*%uozSQ9f*w1l#V1lI{Kk zJ;;rLu(lyFoOPIcxH-+sGrerrBq)tf6MiDf$!4Ow@7U#F58n}l=7$P1RwNmBidErh zJ5+c!%O=BB>oTu^A-V0xcn{}XTiwZ~Hjd}(i33s}FUIgDTD+Ob2jQYmmk9ZTeCbKL zbnfJQ=^v0Xr=Ph6u}I?x=>+EsqB(0l!`IQjk`7Y9H!fysU5m~!`+m+;2<PiF?Kl^~ z?0f(KH4H<T#<+P<N8|@JzZrQFdfOc0jkJH4mi<2PuSG8ZQ)TjR<MaQU>zV(0ik=U1 z^}pn%&;C$z<QTatKdY>reKZ#Er)5ThEB2LwI%3gkmDLv=_I7#vR}j7!r)omwm~K_3 z89{2)0j@M}WFuTJ^_J*Rn4u``e^~v_jSHXnne)fg=eamyD?T3x9aNwMwy@y+eDl-! zyO?MAd*>TyYcsZ$*oO{Zb;)oRS`@A7PA0@UbM(fyY)3wqqdC32qOiy*u<v17muUvS zH(CT&omaOJhReR4`E!==f1Cw`U-uDRPnaBUfw@v%eQH(KP*<UoI{CmqL+u<3yWbDL zfk@k$f(qV*6V5^<k75Epk^<D=SR^&Bok*|-f}7Ah*I{7Ztcfy|C=k}zZ25C9?{}uE zisw5cDJW}<LR92vo{_X`BY$_9oD&BOAFhr64pX*T9&VIW%1Qe3;hkayj*$gu5+4cG z{RRQZeo{&(5G&vM9gnksnh`;RpNfNSK(>t^0}*Tp&W&Yt!S;rnk$5>0Un61SGE6wl zAm9fwC|Xq~?k`##kE#N*9A?BAKI6f&71zsUXC$9>=<()r{wIx0L&%c>Qo`DZ85Mzp z=_j@*0QJnA29h_peJQ}k=NA4JkfEml&9%C>g@il+pUVs<C7Z<w<pTXPY0wdWQH;nH zyc1|&=2A51q|y<2noz8ewR(U*xgSxWPh1pe`mVKoDd@b96pW!y$W=B!W0lf>`6w-e z$ix`NOYpRA<g44;5wDUa;7sH0F>~;x_g9;^$&C{kQ`(wv%X1($)otNVH_329Dg<7l zKr&H|$Yv+Bx;(8!wcG2d^H<giXo|Gx;5ZTckZUihIKq#A3LW!hY|b(|G>ur#&;c{R z;{4n!^eN6XUWvWY9KqEKDFq7v<Tf-qgm6E5ndp6+)fh3`szSJ)DhkYuqEKxOs=Ffw zKOWi>%1jfL*mI;mE}^iwLPgS}awZFGgna|o2XxqT$^wtT<S-g`RmV(q50S$y=q=|X z2lT2b^3M(;7FU9Io||H4LtI@s;$Yd-egI*-Fs@{2ZuPS6O`*_c;+Go{Pkr7O3_j^3 z-vmhbkAF9Te(x9mILJuzcn{F%{tK(CyTZ|y?s<KUeHTPg&G{3&zXeqw*a$?W4;Ne> zcjG8Mbm>0hk24aN2Q_h2FMBAx=mCj#yGVx}D`STA;6L!kf6~<!Q2U?U@ZVh?nNtj> zqZtI|gGsgfg~|3#Qto<QxrSFgn7O7|;XcwImdXsigLf~qp`n=Wypk{yc{v3P!+OEH z$Oe*@eb`|Bu>5bw%0+D+yAK#0hiQwu7C0JTAjoIJR?Kx=Wejnk5D0~<{-PBYJiF2G z;DBX&FQ!?GTaT<ijQis1E)|yfU`<euzGwKWmJ)*^@XwOJ_-{EZ@MvRz1;jWiksMB6 zR%P=5g?RziAsCV33PX31Ay{C|X<~rh4+q%V^=ea<t>Dx~6ujMNBALW1^rNg`JcGMM z0iG?be;%M#dMZjy$OKGQoDQ15xBReu{p6iIL2zL1w{?Zf>Yr9xbzkY<cjf<B!T0}0 zoAN&+|Cw@RHUp3vk@jh$R{I_6nVXdPHmGUUGm>8zph5H5cYwlkylng&uB=tcbJ<T% zYhVCHz4ELmD?6U$c4xQB7K@YUPhTAtr@C04o&twKeIhmPA}SOvWnQq_`$0H^s!Ey< z5JafHj%=wEEco?1g?s;3*XVcW=wEhZ)T9a;d05rN-+0d>!X4SHLr)Q-x^hv~RI`{H zMF)qv&Np-M`io0buUTHXbg-P`$A#BpdE*XxK<CD)#|ve}$96pHBx{Y3Vcu(VX0w}& zy2*HbcM%P=IA&HipCN-xuQ#&o&jvT%jL|)5)tGqY_v0cN$Z*fd2xc#oUR4o#??uQ1 zZpR#*r!pjD3fe=zw!jYPB&pU}n$-m#UmTT*K^!4J??SyYXV7&}M@c<@fh{K~J~7A6 zj6uj(CQj7m!1GsHZ%VyIOgI>Dh`SKmp|BZi+NraDRnT6!!%G&=Rb7#yrb?g3Vxsv{ z!1BUqX!ypKo7SV4wiv}VTs1{O_6^?^g}t2R-w~e|d%U$UzZ_wXl^|?N1-bIXc$0f* z_FnZ%tl+I?={l9)_TBtVMJ##^NJ{u`75%fo&rk99@#Md=aZG4FAaC&B*CvFM#}VKI zlJ{f641Is$HQQrZgid(Ad?sj}Tp{C{_5CKXve$SYo0S+LC*qO5#uNSeT>3{afwq|c z=84}?c;tQyTmr|lKWvJ(YQOOkryCKx<t>~}yz0MCA&Pq7^@@BZj@W%CpEGAGCq49M z+ix|uf6dW{yOl1EpaQS}=K|sjs_xxCxB*=?&i=M3aw0aL5Wd54Xy|XHmHPO__g#)7 z@QE6*dcizX4dbm%bgeW{(Y@}dx+hn_&tCZ4=Wn8Lpk1Fw2mC@>Kl@_zp8CakuBY<H z)JmF@f}e-~vHNhk<1oN_n<q)stGEH(Q~%&9t0!j<`)P$bR6P|e$0z3XG+iPn<lDsd z+er^|Vpu*qAzApN9#T+ytQ+@t)WuccpJ6<>1?VTr5u#|}2guO?Q@r1s14LQJXS-YO z&GY*aRTS(6^6W$%bQs?3^LXI2=ZDnGcyn;W-1JqwdVC0uMi{zbUK9Uv?7MuQZ}8Kn zpI#l_DzFx_OC`gXW#g4k+L&gp54w8s3k#WPne&X6p-05tUFBz4JR^DdiHaO%gmH$8 z4?^W+ag03EUle%0poAkF7@l`zS=>!Q`*vWYw+)-hMk^v4ymIg;gOPm>bu?q^i40M8 z|50xw6+gROOYZgES+aM|dJ?v!>WanLGl@SpM84K<7VrOZZp0<FQC)?JbaCDqv){;! zvb=dCGYV-EeAQx>xe(Z26P0mK5`kKl%2x95psq48psai$<653~@Fiec(bt4&#cZ>$ zCqdrPwN;g_iy4B`j>{#BFF4L?la!v9Cne+Zi(WNR@pYMH28A=FULS!orZNpd_s&Re z?cCev6V|mEEscr#Ap2EqkKj>|k*Bv-j5n^a@p3sMk>yCpz#+bQx}}Oh;SVo7WeX8{ zcc;s^lcn{YuRHaXjtz^Mlm!u*iM1y3{ZqSEnRHF61IxB3lS%dWMI}|&BXxX_sqF$! zlU!skyEM_+u2VpBOzfnd))$<!j4<*i``XFNv8(@@dvL!vGKn*pIY^Dz#gtD@!R^%P z(2gY|l|B`e+}1St3B?=N8uqPQ3i?PA8+UVuDrET^>EZY5a4I1cg)&}=WfP8ea;cV! zzn<KTe_!&-!TA$uq=u(RDBL^^CaYo8PiLEO48PXK`!S|Z<!Me@nGeZ$06vlUD5x2A zLcWSPzN%+wN$-i?6e&{8NsC&kRlM@D=-lO3((YP6c9gx>4eX>CritWBc8~bM-8i@V zye;6{1NG)#5Y{j|ZNH}#WOm{n6hVV=@4CE$8G2!5m~mgYL~cc32}V%An0h0IFM3(D zD#OEeO0%S0)#5YWSp^@Ezm`xNW3(<4pVTmu${X=8o&5^otJHGN^d}#Q78vj#Q4{0T za4x)y6?Sj@E@;!_p*D5qj>8QURjn%1^<C7PYjIUSyo3<N>%=STKAht&O*E^HYcd>r zOk>)wZhx&*R(a}8RkD<Vu}d)=Ze@)lvX>g2n!kjWPL<^eeuuH#mjAZq!6DcMd3|r| z1qwsYR5JO{k+b1?4BPD^nntiWr^i$@`YXex^KZ7g+WM1RA3nM=_m$bXep(#JzusWJ z_`(dn77=WCNZ_v2{j%fbkrlqW(IqARfU=qtuRvBTv6f0dnh7v}(${vI7uCJ37`od} zTh^G7W7WWv(2R1pL$upXqCtCRiFq`w{BA+QEk&NVPKR&#i~IdmS#Jm|IK>%Jykq9N zlrC=SXICM6N!uOY=up~>ud>dO6WL6zB8lCIbt1cl;q16Y!CxW;f5xYocmH`7`QJ6O z`<wrOJoR`A!3b-AkWYyUA>3jBq|{;#7Sig2@SC$EoQ^BrV$rLkf8f}Bq<wN%G2I|$ zYnvSXpwQ<Q47~;Qs`h**gVh@<Yl5GP=rEf86&$3-vFOx`?>c?x5q7JpEo*lo3o^>{ zq{*6UzB?dWe7J_;dY_HfCdDy!20_=cgH<l(6G7Q~?XG}S+tMd#jZnTnPNIOOH{ZYP zpj9-Wi5|Io=eplO=q{DtFzTA=8W0Y*r|di81&wWfA;l)Xx5o$NYPW$N`~Dcu(Tj#d zMSrdlIm)m@-_J;>{pa_Fp%H^hs2P@!g{}_IlY?n3AqfGQ70*qEWq=fhJZ~0p6@7>5 z8a_d+BE9;^eoK__w#!F5dsuq2*ONS|i{+7kvI56itfsm67-mo-dWdgg(Na+>;q^UW zx?}253|F*&_!L1!ko9$)r7rm~^hG1<1CdaA?cN>-e!ZDJ=BJHR2~!&h+-kvu^qa1J zjEJ;^hGlK==F1nlVq2T<?t`BVjAz5u28H3Ob16{e5|~A8V>yh=Cgd_p;JKIQ?j4gF zm{HVIQu(=b2Ub_h1lA#LptMFz-VT5FqEa+{=W~C(=J)W&apd&ZmaBr3AsRAQDByjy zd6F~M4cE{f5iU6-t{8Vam_DAHG(2)nkJoEXNV^*|iVCmib+o)7<#>uRDzA1^c<aaZ zN+a)E^&xtieaCL(j|kYaxKG)Ahi4>r)CPa8cm!t6Y0~2n?Vd#q*zxB>=_V#^RkfzY z?14XeXP#RVEiHb!@2^~H@*~J%!n~c<jd9t*j5`O`ilVENKKKp$L}?zaNR$4<53eXf zH~+!_7GByPIZB@E__$!b*H5#?d1X}Goc(xu#QDb>_WMg+T~NJp_|{K_X0phOo~B6b z*j9hD>7Dn>+DI*&W{7#5$!u5ZbITq}8WI+PFqHFn<mfcss?}37P)Jy)TvoHR+41e6 zcJYVpTD_Cr1n%&KG)(z#HCpWqYd#MiydaU!ra~AhOO)Z%%qNJALBSj_Qu)wjO{P>( zLs?C&TQs3cUx9*pqmx(e6;(?G+<mU$r5UpktgcxmAU>gqBeyK|Dt=<K2-7F;9pn_{ z@%^BP_@^)YY;?8{=mR$05Mrn=JWjQrYa(5{v(DNSTyO7XgTPQ~|1>C_;1X)60ODbS z*49y^CO0tYHs$rNr^E5Kc)q~S1`!nI`|&DVR*Z*Lx}^=9wPkeg-Ua^%|MZH;kw5w8 z*=U_2%*J)eJ`vdPUO5Cv15Q!T(BCs4FvBHLcejjAR`l5ti}JuZUgl;<fNQEW_E%yU zTnh1d&AhSOgR$e)$#>^?^R3Hd*h1SsE>RLa1ufw9+I=}+xHRtZYO8!~n-81G^@j)0 zO~s1j&0nJ78$!PV(eRV;OqfXP0hj^X8yS+tC6tonVN?ApxyE8kmHQ-V>JOv$$+nrO zfp^<qapj+UmjeHeK9Z;VA5q-@!{n|$w!`PrsGL~7F}`)a{@ikW)mkm{z0gz7#Bc;f zMrw*r4dq6~pHR)8ZT~c>aXAGq)^2DPRvyqaWXVhRYT%J%H}v{iWJB<&m+rpP3>fcv zf2;tmHRqMHhhj)OU79B%b3_QcDJlr%HONhVyc9ZWoz*{yv(DyL5at8fb~#|(3$UL; zY9D16sq8G(-q2E4cJ51#%-OUCg!6m;R2iAkW8zlP@6!PTlDTthc6}m^{7T;55<$M= zM*Z~DEpDAcWtR&r9GbsC6Csb0o*J3r$@g!D?rM49Y!{(HYS@67(`|tzj0YQNy2}ye z6E}`CYolaZ9w#td%dov<YEQfEB{IZ&l#I>l0Y4k&D{^mc6bBM6+np8nbc351JCAd! zpTUtq(fur|+a?U+TY~z)Wxf1AHWU6*86`GVE`gixdwKvL3{{DxD%0Cz_3k+u8SuM0 zMvtbt>v8Oz$*}vnG=RP%Xm9n$79Bs|*7KLYO0pI}EYm6lD)Z1whF@|q-2t5s5Y8`A zyJKz~B;RE{Q|~K#B7#s<g>X(`k)ka#9h?w;8j<%H>_K%%jW|z1!MDfh$4CUhKx$ZU z+`*)Ej8`21pB5>)wz`$M)i)dR4bZz*z3>0KMVsX8Bx7-rj;Rd{8iK35cvG<{Ge#Ww zbmZK@MmFM3_S4yzl|YW|QD`cWfY7dJEk<)D=gA15UQ9Kdk;v@l{n1^z$Y}=ph9d92 zDOux<;qZ7KxU+x%i7G?z)`QLdLic0kkSx4Uhb$$PX64tiCyKf_f8#@54$YdSqCfgN z|9NZ||0h>l|G%5~Ij2ftZ-p#Ghzjk0U~#x^e^6Dq`Rk_<<S@f0<CEj}d+d%Te|gi9 zVAmTo+nXq!7l&hF+rZ3E=uAqHhgcDXgWv|p6w6`fy`pvbaq)g?o59(YssYnUi0aKZ zbH_9zAd6nN$E5re=g8hudG1nC6FAE3A80_odHOdpjO+jE%?qrkK8OL;9eKaiuwyNB zuoSS_a2?T0R$maI!;2r8(bmhYzOF~uuab{5W+FTlPjE!8)Rq(+8(LCjJ~BHOrm6k@ zU~OJ|Xrj=PqWg9MHKO(#tq1yM<ueTvmV4qGJ^9PE_!7+32FWpum)ZH%(hnj$0hN{6 zceq4z*@MSxIGp#;xi+5OsqyBN3CZ@9g_(kt_tNF_GFAj3&8eVDY*oPCG;8N`C7a6+ zRB%nY(A<}r3r5d}*4kzj0eY1d@0Eq^^OUCZuCtW=r}3C6om*GGH)=_N)|NO}G@mG- zheHPd_2NNHFRMyTrJayz*E72u1ISFE<vPG5x~~BAbk^2;B&yMYU`gwwh_C!M7;Em5 z1|0~0D$g`s^qNZdUa$J#&#b(X)xCeF8_v)TD@HeRU|TP{awu>&SyalhnJ*hZLZ{Ib z%bm=54LM9j2yVDabi=yQhEf4?)xwK0>(5bP-jcm?Qy)857|7&~KaqD;mf=Gd!kN>} zE?7f>nB_pPk<-=KdHAxwsYfP!+-j~r$dnWNDn|3mRJU{96EVkJ)6Qk%9!=w#Rg{9S zfm+^*9ji&-t?_R&{P<7{6y`$EL+t#E^)v(Zk5(KilRO+v6J`UU``sKe-H%GDz7B1g zmzVmUSEAr!p!PcV>`$;LV1w$ui81;+x8eU$;rNsH{AaWP{;xqe`TLOWKaKhAb!r6` zvK)yy5GR3VzEEJdMP@5G!_dIADY)QsP0|Fs5GN-3CU0JqNAJ_?hOug7L@H+ukiVY3 zoYK&>SNkcnnWm6%x2^db=i+&?F@K8D^b$A=<|>pPZ>BN!=A)rt^}{HL@A2K!hvy;W zlSRYfe|GX+vvAZ7E#CXr842K~>7OvgvRXz}hwBhEjW<<UFcQ*3BN+C2nOa$6Y<6I_ zml|v9y&Pqf<f>^tYs(9j)7}r+%SkR3Cg0fa5)1%7&xJ9cBPf<=t%*PCJR=#%dqm_U zD92e6`OwV$SeOqfVKRG4vG`Tr<mbA&s<Pm2ce$s^kVrjph?sE;fwQh*xeY`uX8=Ql z2=S`loBEN*zkti#B$B}{O*#s7gdYcVNGBYx8FU`M|8Xi{;_%%T>GpKYhaV=U>c~EM zn$@;AZ(e%q*N@|ucRY?q>2XaV`^B*>rsu|ZbWf@bYmAtKPUIyGf`*g!PN`ZAIo~C_ zXn-__o0%b&RgXH4qofPJ1MvNk+EU9rRnGfNt;LKm<T4H&wpdvSZTN!s)~H7)e{6K) z%s5|Hd=W<0LZn85L4)=}R*P&AbxTm=?-Le#eS-Oaa&m*pnO~U+|JR)SPY>n4h|l-` zc>MeeJbwNKZrXCPZ8F}oEh02C>x_(tOs3~Y7R6>-$+zqUoRnYbupepsy!PSPYQUvI zxnK8d;GsrM=-QCwG<(S%OLM@aM^Tl_dQr#ag-(2+0$I+@dbBoP%I@OEG4fqS-Ru?9 zo~f+RQN_v_sh0y%-A6CHbid|yXNaYM?X||WjHR*5?I+Ob<^tH=eeia&D^yMsxov*0 zze>#lPl~x#+5(}$FHy=2AX6#Hhb^jx^o|m-9P65PbdXnQ*m}oiwn=tSy@-P=f@wOJ z@xamCpB+)~VNZv{qr>OZxB<4mnSZYE%6xT=-RA4#hwiy%keO7sb>xAHJcM(p00#Qj z2*PS4x2EYAXm|G<-sRZs^ET<{O?4bjB`K=>?X?xYv}n)F_jk#u=k)&#Etmf$1p6mG nr2T)2m#X~n1OIoPG5(>X{Lh(4{sYIl|E@ndU<7mSY~p_bCer=r diff --git a/doc/modules/cassandra/assets/images/Figure_3_read_repair.jpg b/doc/modules/cassandra/assets/images/Figure_3_read_repair.jpg index f5cc1897e3e37979f50b44df58f0db0f06fab9a5..62246c681cdf0c328d6349494ed17d729fc07537 100644 GIT binary patch literal 53091 zcmeFZXFyYHxF#CKLJ<L}q7W4Xq^f`jgk&ol5D+3ost{2TX+{MkKuA=iw=D=L2$5bx z1Og~6bhe_>L4uHiN|!_>EJ<P3KIhKNz4POobMO3_Gjju9AXzIbdB5*_+w(qewm)st zA(A$h)|QYRJ9a=WfIpD!8Ho8mkN(};&Ye4UfWJF-{@2@o|8M6%7x}+k;Qw*je?9%* z{kz=>krLkV*Dfug9mgO$rFIBO?bz;xKp~JFyZ`mtkpJtpW2exr-Fx;5i-?MWCp1bz zcJ2@o+PO<;_wHT0z_X*l<B(layAK>ab#9NegU{Y$cV$jLd{QW^a=xxh)^UWTs(<a? zBN0(Kd4+?D$JNv|PG}k!o-sOWY-0YWg{9R6Ya1u$%Py{N?jF}~_}=up<sT3d8g@S% z6A>BnI5sXmAu%Z<^J!Lg&a>yaMa3nhW#tu>RrL+z#-`@iZ(6#0di(kZ{(k#zbZmTL za_ZAGmBF0<^7Y%o;?gqv$Io9IoK4{O)<1IXfC&A!Df?H!{s+0FK)H7A+9kAW?>}<w z*clEEA*o%vkDl6d;GDx=pS#k>PCpcuIsc@vu1iEk-;pJI?cRu}oT>riIQt*b{zbC? zm|&0oKS}oA1pDuDQ6c+;c7Tf~Bn3f01OknsB*_0i{ui8sif_4xZu!1`ovoomPqHpA z@R!r@i94>RUH|Tj_FYXaxodyyuvXpSrU*@e9cM=99%D@<VU3vwgAW+%&JS<2>$_x3 z9M`Th>vr?H;A45?PU(YPg+gZybUbZc8k@6Q1QOq`{<3bD9u-9T1Eh8(C7H($PG1l@ za^tT(Zm|wS_j-3M+vz}L94!8i9k_k?uJ_dZ@2T66U7q~fMjeD>$EYsxd&lnJ@z9wW zq6vq30)Gn?``2f(QPaJWSGSuoE?-L#arJp2jmkyUeWBH<Wo<)tYw%OrRKQDU0(tDH zeu6j|fU5`4SdK)YZi+nKUoQnw-~XJI=LVo7pglV1fyki&itLKw<@mm(j^qB5e_`$; zB)n4pC{6w0bw0c9{h>{~KunygfOXi0JbJVZ=`zEa;NDX9gsnpWKc-Rd3L#GamdRA_ znp3Zb(IcXC7hVn~J>Tveu$3N*)ghuh$+=c>>&4AKLT4Uqp6<<1DDumNZ9@`hb;!PU zb%CfU*F-Q&YZyrAfZmb|M~H;2Pg6Kd79zPZUB+LWjb`kAj*{Z3uu+eW4Y%o3P?w=H zuoLq+<Jg$WZOF~ix%3*lt6{dchAp{YKPrS<hN)IYVTYtnv;<}pm(<rB>#>s7i<<cg zqy!_XCSK0(wlq9-Ht}+bbE)`GW6LIoxI#P4#-&UB-1;l;_7k!DyleuIekrAKZdHI{ z<=YxNg>&bxaNyEFIXJN;z*;3HhttN!uQsxrI7oK->QAl&?;Lx*M=v293f+bTM!>~* zQ2>{@4T-t$AH`0#<L7rAn&zFEsCn5ird!l87RsebExX(abe$}jt`+L;ps;5Iq9bgZ z`F<M9R!<n$huGC|n0F&jAezo<W>R`l@vaEP4i!ucLpPHih2O=~WTp3+X|D9O48>uO zGEc0n>ij}U1`JhuQkPt(>aAZ!$MwNhp1=^NX!tHAia+!L;ULAob!a-CPT?Jh=cEJ1 zTs!P-Hfe1~aUh|2pc^W?ybU?H4e`@Egmq|(cVsWb{W7HqV_}oUzz0^p*$uB{lQTJC zRe(2h)N`ONYGhwJrY(mj6W;|@f*-<nSySjVy`8v^a<qojoPKcIVN5w|qcE*9OiZUh z4e;TfS@tN4<r)H)v3AANfDz+cdBrkHtWCXQscGx8;??@-V6sS<m}jE)#@|HMhSrp_ zER$b90?xj5&c9+|Vqc$R(-t`LHUwtxchK?_ZuIe8@~hG0QsLJlrFs=Z(wfAPs;4u3 zej&vQEp8Xjpgocuz5B*WHACKgRd>qjFDj~vx*ZYq^XcTV{N17f_=;`FtN67BwgiXB z-iEB@ag}jXP$8T$Pm5o&4Y`4r1hl#d@-Qe}PZ7m)0rc5=48nlK9xn``6K=ptt7ei! z{N)=KO$uW76N975mLu<dQ0zPST6oIQoytq(?&b*t$GOI!js#KYrP~m3a9Xz^j}hW< zVV+q$=RE)MMg>-!mDeda1Vk}}!%|({fkkRTrF=Z>L>bk+ui}b<<nwQCe`Y*;+Is3B zsf6MK{Z!2<2lUwLobUVwia$~syK@o|v<;c@-t2_C#^5Av**oVsk$?{a{SYOwA+ZZ9 z$BG2f`ywr}Da_}pM#K=N4w8HNqqVY&=b@lua=;KheD$BbJj>Es`^h%td|<ahS=J|l zw;{JsB7z}=2>cWf-c6Ch2L--qL$C(uwG?JlUOS4BpG{&#PTz(_gcM_@ync*1kMA$o zbh)2I^GkTYkVZUc3uD-juT@;`9*%h*&{r{<b<xs1_v&Y3=I4NWHx+V2i1z!PD?XM~ zf$MYb1(1W}B23HAU)EwQcqDgs7kN)@D)5@66Yi#Frm9siugBzDC~n+&Re1RP)?h`t z3ut+Iu-3p5i+hD%P}`v*cu%Mw7(jojUEuWdQ$p?bZ$tW^2L;9$+xekbxDlqD`SHn1 z%=P|ZK&-1ns(Cn!FMV!meZHgl(a&BYI_hh}dgz<WcW?lB-8jKaIHC^GBN3M)5C?A3 z6mh1w;cdv_wjEpp!AHDEDcb-pJi@!q4q?J_A5%jx&HW8a&BfgX`^G|b<h{l>wE7xa za^8;iwK<5WyI%)91)@`2b#M!~;DhpAyYS-pn<o(Tf-U4Ko3#yLZh|J`+@d07AYBnR zB7g!zQEWXrI<=-PGF0i>5M#5)3~dhI(=zd^RU^FTSb%u2L2>AT$A4|bujz3Q3T6mR zMysV<lNG2;n=khy?>Hzxf?yCzj*s1j1fXJb2fE{cIwoz}ab*Cye<NLHsQ}7CB@+5g zeB+bKShrhQQ@u`~0&`A!=&Yr#t!DbOKX-`^+)eYShN-D3Ogs0`C!-ZJ?>M+U_sw!~ zu#)$X8TY7kk?fgg*aW<&JJ3DSRZ&&%EIlSyx{(7jYN$?(xL7&NN%QVL3s~tBW6NCv z=fFj+Vkr5(ue59oGQWLz!ebLL?ZnC8HyN&{;}HB@d@yR-v<EK^Gk=LaIozR#*~^LJ zzku1YO_`L{a<-V&IPY8tgBZwY+lENuj_@>CZ5@)pP$|mF|K66;sO_1;GViw)Guro? zx;+UW)>A1g_jPJK$A_Kvgh`?5ad7-poHF+4I1UCN8EPc1nLF-t$6<CnPsX(iC&FIn zeI`%@kf4E1z=>jPyP7-X#%sQCy)%ikw<3{A=<ln))|}1Fm}bPa_s~*k{pLdiy|jha zOG-o>*oFkJC!-)Rc{<;d!B6kmhJdr)puSohT#Fg#f-6o=vNyy7=kdun1JHOx6k!N% z?2WFcpWGLl?oYPtPK~X%)AU|GU@#)sIR|c#eF%E-CBffLf<6-y@Zm{aGD?^i!CL4K zgi?FlN8tt@xN#aes6~ElM=ACy7ul~9ImO7LV&Uu$6R|Z<0&;uh3r-ucK{Kj-$Adbg zRvo;;P_cI}us-aW3NwOBubeRL4^@ADk9p;*{9Bt+*c;1xU#pF645dr$&rWzK+};;O z+D9-Gd`KoevysJBET643m#lx=ax(U}@uKtlQU&XYvMl4V)>@XRbRYg&)>p(Kp49v{ zBoQhGL(%b|=n95I4{&GMO8KR6sqF9`xEza+!n2}-Mz&GgZ-PXIN?^WquUofFs*U)J z2kVq}kXzzD2L5W}8uobB=q)F<6+Pcso#nNh;w41CE&Bdt{lZz*omOX+PD~8Aya?zH zEe{td%P(QBO^znpMV8%ntkw)E_YgU8u2T0_S#9H$;avaU{nq6TUJ{+#5I{vQj)!Kg zaN<FgXvHeAa|S5sC~^>97!gQ`>yTp6dXZEk=01v4hssb9O)%uWoBfz$q*xOfot2hT zTK0w+?6nrk*RmfY&zvoH3l3shENpgDz=gSvnm)v|T4i&s!9}tXoWe|bvMehZgc%`D z8VlLC4T;N!%a1p%%!Zq@rs4!50F=?bk)B2jLyNn6F6GXz6ev=%HOO3|{Vef#oYkAO zn-~?oOm(?Q)=6U(qTuJgX06yT-$U0<eEsBh?bx`ZjRQ&XNv_T3x1YKt?~rmmzQ6dn z?_KE3ZP#KaXkxyIy4p-i#k&K^w|T@XU=}5QK1t){&i|9fD>P4wk?QI>%(Jb+gq$01 zvtm5N!Hh_>#jWUX%?w7>OtRgjTam=RR%A(X-X)dST-cM}BM&leduv4|nUqhO!l&#^ zBg>|W11mxUOR@;LtGq}5C(Yl#B`XDn4@3bPR*(NBa5?CP8`_2l!-R-}v292bG||8l zC3OqcKzkT=A=DGO7h}J*RI}Ehh8xC9*OIyIMs%z0`MZZJ(NUEaAyq5-e@@OF9x<$( ze|^D}GQ;H*O(JfPrW<+@l2{4G3__9T$IeOQNw91=7?xTgESri>!NIXN*>D0K5tnVU zHhw*f5L06q?cPX{{8(Zg%vcv)ZkCMC)tJYpP<#o~&6fPU4(R8l9FK2>Yzfe!r*9wu zB>omMP5JoE7WM!bjHW0O^V@L8*+Yq8BGV;FgYRR54ZKb$xGkFK#HOW|TZres@b~U_ z$g(|Mt!`|BiFElEcz<pr-FC)hqkgFh_F_gq4P~z><wSDfRjy{tAqVT1`3Jk^Fna}} zQtVH*m4XpevoeWgWIJBse*;kA?go(ExSgzzsR325X8gmqA!PM_i9LY=7&~|>`*>8U z?FBF#R6rX1j=$KDrfLq_s(ng9(6wrA0BBn-_LD6KCJ_ueiX5Y2zWFDDmGpLAg#ip@ z%VNhz<?uNXIUoF<bg+s#s0GOE(WI0+cQU+gydluEC@bdk0XN^IS{t`rVR9{gkXp`I z^<^uI{SzHvcczJ(bFlozS}?>*mAAdsjXl6`162rl;8Q?Lgk@SZ!kS+K7sY6uXQ#(C zTJXnamQDCMX_kvr(>&ugnhDf_wZN03aGY7C#Gx2`&>;zB!2p3=mhg6eGuIfX?(&8M zZ<r|A^UK||K1zdt!V2j!slk&GJyDfKZ)}5+=gFR2f1QFU?~zcma~^+3??$QPrYQB_ z^SKv!X7%akr&NKv>~%)$1}zPLb0{s+5r#Vsn6W+P`Y(jex|~?6QzO!&t#|r&oEY}i zyG~jR&p9LSAkQ_J6x=tBzmruw6pNH_2pF9_?d52<xYl5&^}U5;zMH_Tv^JMM?wvRK zW7J{wdv<mE*!`J?=}A1jh9CzE^`7Be0;k6hH${`}w>6*FqaJMA%RRs|$esOLZu2gs zU+3Dwz=6)sr=&1@tmTYc-DukKy6AKY%MGod8sdi06NvP&_Gp#3%C&rK=UNNKb8EBf z_x*M?w|NCP1V%(Cob{0rScjRa-sm!c)~&AzMQ|@-cd`h*6iG~=KoXsBza=NRLkW8g zz>OND(xIscQLK9n&ln&w&gQy%yBk!a4`Bk`g3tITWUt%2scZAyNp~H4>hn5WhjXX$ zgndh{o%fZh52#xuQ^n!?tR!#3emqLiID3YC<>S(3aQP1x_~}-b9#3Yb?+XFU?z)i2 zSTLu*ZU-Z7u79G?Q4jxgc*9?71fBT6nl1CiaB6ARP1SestyQd4y54F$yZX|CH#j{6 z@RinKMW5e(!CvGOu!n1rI*m9b-1{w~j5q2#MCqZ6THigihU4tAdM?nJYqVU{mXqJE zpV)V|fAqbL^gv`-bq#Sv?4q_IM~7xI1)_dLzxAj5R`?mf78kh<c?+8~K<gn^g9m~f z^;QpO<9`V8n-TP~Ts8Q&^_dmUL-x&&&$b~M@yuwy5B-R*WxUOgcF|ug0{PE#bl|1P zZ3qPQ#gAW_z72W#jsQq+L-ZL5pvR`A$U#!Y1fn5eqH|*8<jFW>GJY4#`#OxC4!D6K zTUq0`C)*G^qnNaF#AH+)h>akGB@1?g0+(Pi`|fAcb@Fr8j&hF*J|Yf0NIa>Mv<-0p z)XT7X?c@9RC|bU8C#XKx{}4^E@P*}R6Z&Ev{c6Mi%1%9+_w7=6nyyDUnX)Gr?$iX# zfbXBc%Hc)3<GYM-uyaAL9b|LXPH}aYJu?0*R=!d03e*MrMI3R;sC#Bk5l!rD2eZ(w z%ZOV@$jNvPn5I4Q#*G#1Z<P2-ID~r#KM4eKZvlseKdQCln6vr|86@St;D&wG(1*2Z z!85W7j{=vLwUS8Uz7x_M>@Vo1Z`h`_?N6&<-DB^MOp(9Tw9Wf%_L1IZbx7d5+$i({ z)FG_hb-La@!I)qVE53KlVpI(Am~gm;XU8?hUTJB+Db<nh!y25Qi)DNWbm_1T*E*h` z)oCiG0>gxC9TQeL@<sWalJExUJwl0R1cb9~&*yb-L!`00>6C<ygRf^f>HC58ZYngH zd}32~@+Uc6((SY7U>5y*%9p#N9oJ$w#t5;_Hh!7_!Mn>(CF#T%rpLe)F=+z>_=eYJ za&JL+M=vE8N1?CpThdW>Jjc+I5T(^W{;{D@a5f*7VkLI%$ctB>5+<9ibzOgi2d>vG zUt)S3@V};Elk);~np>0m`wD-$h;l%C8xlkTPDrf&)YKbDpUz=4uT`?Be?N?7TX9ZL z-T)lf?}qFeKW79)l2)tP1AX!yu8HdfP)%#;9@>EdrnQT6!!XMysbiM^96>Ka)yuC1 zSI=Se8G3zrrif&!frL>K-ft*LQJ2Ds;v}-7&WYJ#J5+$9Y$Ll-o(0-}N1AUGpsj20 zcAxg-=2%~%#1Qi&rbhM6U76#yABfPBET+J&fCH6}>$A#}@Gy)j+3>EL#g1?$WE`#x z{NZ3B1Nq-}T>k!j&46oL&E-}{X&usvs9i_j{+31^9UY~f$z?7tGjBRKH>b&jhu@Qx z6*~oyMkV595cCq1EIz>Nx+h=-;!RCsZAK&@vRlCIp-58bg#;;bRB9k60KkU_hHl}A z<+zV?ifPO_Bz27GRc#)5fZ6|Ts?#HQTvEv#HF0>b<jlB+OYeB9rp&BO#v;XNT9>%k z^4x`FC1HO#(Ak-xO#0zP%NSq8(@VD|(<=m@2rO$vTyB4L*)uf$mIT$DNi2{39wsce z;=-2b!GGe;%cU}AFWnmZXu(s&umufahX;jhgTv9m+OCpS9}aIrtf!cyH3{zSCBhDv zJcBZ0#HfLS$)&&Ej4KBNyrHjC^#&-40u>;!4aNPFhh^=J4{Ad)5NX>G5wgy;5cGA1 zo!BKnmMz5eqQt`vGPq`XJ$D0KJj{q4SqqwWs9MCnj^lv*yk5`HZk{V}of)j(kCMP0 z1p=+v;W2Fv#UB=ysB;gA(|yt9bDh0=QgcO|t!0uj&5yzKRqquS31TuBn!dl(V{50H zp0=0edDxt4T&-o!`vupwR`ZnaCdHYwoicFZSVx(Kw|Ry)H90!Aba0~cTXQvv+B5$a z)gYMwt!z{$YG;Q$z@3NoiQ}d~%OKIAh;emerzeLE_V3UpM<4j;IT+g<e<yI(^19+V z|3JB=@84^E8k?=va}91#lkR1buhq^>A8%E(x3kn-Tg<tSb>+%f8l+Qg1)_NjJMD$; zt?;!%I#SwyP_eM7ku4AUforH~^FC!0@}mn+9B!ZwTp5ahU7%=h^)_7m;f1_2v89g^ z22CtXpoHOqpJ-Q`1KWwY8IOiR*_j*?FaRQhr9dWq_WU1P=?_bQ)H6B7SP&4aDuVnK zIHw3m(82g^PI-LfG==Id-+JvwgTZL@tQ#ZK`5N`>w|f~#%cmt@GQAY-v_cJg4VK@k z<17?tttrl)KU?i+6%H1%Zxa4TRd$WR-r+yP?Zl06B4N;jxPzFee#M;F?|3pyejd`C zySEn~-||XzpcZ=u$YbPHBP4kyQzMBdqKg7t<ZU8tP6y82s(f+OQCIdzxybEYJB0Hu zP5l?Dr6OmogU0+xmW^;r7A*-ue~GAf%`Vu(#^PC*^d3&`I0yK!2YVIu9^eD<qDxO= zQ4Kr<zv$?zm>!zqKAZ18W4!2cf5pbm$I-3K!}Vv7i$<%h+&ljy1dZfK!4uYk_aw}a z?b<mq15H9TEcDs!pJyx#tjEgBd&<eO2YT#oA{(uWZS_~{r$QITu}0OqPcKqG7VkL@ z_eGgv@AGruQi9=on2;;%`}ssnAQSzd^mQxX#U5f5P%5x6;47=#@wJsZ8VBXsvA#SB zP_(Nl-G+=Aye(L;n7EE=SW&!L7%(J+okPSey4aVA@V!YtT3;hR415jyl3v)!7LU|5 zU<b~O3VchZmh`%QH*hSIc$Zj&H6qs>*DvjH3)LJ{2l|~CQXK<6+z_5I>*@&4i#54} zJY&nf2EwTM4Kw~8!j_@0fmnKL=<tzS9V1Mr7{<QSy0Uqd6vwk>&<<kof3ne!4f*nY z-m+}ntn0OyklwjN6G34^5-VG3mBC8ip4Z&U{rDW#M6J!b>t}Gf>+Y@$bAx*|Pg=Eh z;`F8BNRKO)uAOLc^5O)x1X{L@-P&4(tHo*NR-Z4b+&dG>HEh9Mq!uM<3Ee45(t!N$ zG)VjTMPPa25uqO6oi51>1o-o_>WuXs+WzR=xnSfad85d*M&h#W@GYDaD?Xn5=_ZAp zliFxH+(h+GE%WpW@JM~1yt|ZKf7n%)ivq)V5m5O-?fij70tB^&nq`lV4L)>2EMAl+ zebsOql0-Np&<td{@lLQv3{<R2)(y6N^2VWO1fkIT*>c&NCI#goer40Mvpv7<SmhiX zKx7k`1+iRT!Q7MyG6{c6$r$||x0knPWJ7iew@1)haA-2epS9kT;2{$;fH+)hKx^n> zk`i(*<;_gbM;rC@*s^||^8CDIeBTpws`{<nJ;&HL-#AHp-|6Tz9VFu9`&+)h-YiFL zsI$7z>RZPr_;1_yqk5Gk0WKIdo*8%QCp4+0GS?q$L+X-0_uyFr97wurG_;Q*8~RMm zGam}3x6N^N4DJVN+mz>#))>a+)ENxkM)^;;R$i~-Q+-blLF{6rav~%3kPT}gjvTGk zGp9UEUNKWK9}YExk6Dk9gVm9bOg+W*ZAMp*riq){Kbq7ijuXCqfthCb=jBU~@gj<O z*VQ|Ks7;oz$2LTU(5F62c^lNCLa%%F+0vxA(mT9q1hvog2F0j#xtnsb&76gppIW^> zuP29f<frhA<w11p0nUi6RtkcmsJh*gq%t7fleL+IHOHYrNFB;CXX6vX4h3s<Z#5N0 zo6VJ0V<bEZhI(3R3-5+S6$cbRDJ=zuK((i86_sj?ThDgiojmPH`e3kCmK*I5Ym$~1 zdEz`u#`^<K9R9&z$}&9`_fN6|X4%)*6Czh}LX!e*e}WMFbpP8ul`JvN(-iD(V7i1> zC&n>k8TGmcm9P!0G>2)6j?2``pKmvOqJ8chJi%1}2D`;dbzA1NSk%SzdqHDvCf~CS z%3F6NHhTRr8=({OIOG+4{VBVv29-b)h0F1LhE-YReIU*V0ZJJWZ9^cyE!C#PpdDH~ z@Vn{&O5cioCO_;p_b<qF%9nkIlYXDXcj+MLxwcgu^$0qcWYv{Z=RSEnL#|G-9+C?s zHN$lu&q<v*5%MFQrt>;4!-)RkTSuc3BHVRoE}r2}TlY?&aQp+;$9`=30dvsIjT<yD zexR2i>a8Ax{l-DTy6m5LvP6I8vtQr>pc=%V0~_qLtuOj<ahu(O9l772yQxKeS|2Jf z89yyXi5X^_t}LyiXag3hQ!kCuIF!aa#LnS-9u7^9M~K3qK%9gkPlg{)&UYE?`|A^? z)w(rqp&KEK2uN4-m;dRH0x{L4kU!7{Xt6~QWMGlv)_voCuk$QwJlvP%5O|CbBc})b zcx#uk4B*b<f@rNcOeeT`bxx7B_wg_2o<%vX{Iv*0M0}7Ak=YH|^Qj-w$!twBOIbEp z`CudFeWBWNX*T4{OjfJEGTUd($6nCA|8&6T(eNCCPKE;|+Lvm33AB+fJO3?KNnp{1 znzX9L<-Zq~4|*v`zI$hR@Peu&=Ezj{BwiG*kLy+ZR>?N!h}Jhv;S`n?^VbtPv~it^ zlXZcGSWR|G&Xgl<WloF#%#disKg78Qyfql}x^gB@D!mh@-tr{VWsK})zr=GwZ+giN zCU-H|<ov@HaLlhL8@)D^6=0SdXkIRsOp2+h8YZZ=)Mj{<w2+qE3s$(=f=R?~N)RG0 zhUwRpAG!5S$;hRZRi54^!X)a>cT;3KB(dhBE~l7!aR_;=P^DB99i_a~@OSlHzk6lI zMHyNvBI}_u)wVO~B`d!}`OSu@Rt%#x9oAq!<<;Y@_Hv%=-u}qjnWtFL7-A@w5{FR0 z`+>02isaJP?9|$Psn_zPiwC}QQ24u^b|_TV+5mTYEG)UCo@I)9r}t}M%hZYg93%}; zPkp3-?*G#^L>~4>ZpzgUTr=F?@Kfx8)l9AtaDQ|@D92kV#{~#un&K1ia>sT%o<Q@g zm~8a@I&$`wTt=p074J_VN`ZY@<I>$q8|V*qs|Bj08T<BjMx?<O!EYZ|_|^D|t7Ew% z-bbrB`1AcM&C^Y!o6Q&bpqnT#2SYWG+VBW{(b3D6a~`l}il3J(oAPGgBvfO1`p!t} z9z+`q$pPgqZR5Y%|GX%NCeO_F-Q~o}!_iboM(s4d%S52WGq^bpR(Xd~361KjeoPJ2 zT2mm)e4duJfsg&U{OWbV=t?i4VW3~hwpRdQ>rkjd$xkzTo?N+N5q9U%_Y=O-Wn&C% z(6!H6PA_t7-$sAvr$wju^u9LIvQNEWu|Y=7Jtb|Th6OPdq<-{cO7j$Gp{IWibVvap z8)Nt3F|3$+yx<5X$A*RMBF$7Ybkif~v_}Q{Z0|<ZCir0>f_oloJk@rt@`~UiMP@^5 zG5G43p>U&}Ygd8FxM4z25Q(f~ZyS67J)O(a8Yae7YZ@-KTvlnp=SCS$myMZG4Huaw z`iY`RrN%*1U$)9;O%2CEjp+U74kTQ?X%FegNw9i{J%P+VoFZ$Z@4cNNS@BzI>5~4H z^R*78kF>ZIRJWK%Vj4%JQod!4HqO?X8bxV+TGA-9I%booZJ%LrO#8{(Lwl{II&XxA zsh*ZS@@T)Xu?+p3>P``-ILM=Ook37@a?Gz|m1TK$?uBZ(E#VhEG>ZP(E#vUVf2x5U z<9Y}tQ2P+q(T{SsA^nsjy_x{IrsgfHSeOJ|PsA-Xkg3<Vo&?u!RdjuTlj7;IM&H$A z@&>lPzh-+(dlb4c&zbR8CtHH^zs^$Xc$Ppi9rf6I2Ro-vFL6$D1mpm$GuGo8(-kl? z{cx4`Wc%SZLk2<O(}&7H#gCOyr)GA)kaPKIs_eE(O?rQx(2aO}3HZsG2WCrfG@3Hc zs0L#WzP_e0)rbS(7r>nSkC#=lD2#!`;M6n{*M{dba(j;s;uc<MF<60I?@ObTglBsS ztMfgb*!$9HIaf8mDI|=PfA0!34o!*r-CXa5%Azs{<}0si*pT^4W-7|{IcSU4X!-mn z6qXSk-Iork^bLL!CAiWc*pEY)3=OtA54f^O=f~~aStO1Y;7WS-r3HMZcG)chA%(Rb z4=y5u!Pvq8{9<?{fA;lQ=!hSq+SJw>w^)?K*AkB_eq1anbM&n}G%DIN+7qOpnE!0E zJXWhoUYX1>SlB#1a(qNLc|%mAsQDy#_uxM~*b-FN`J8yl&GZLQc>x?#G#PDs0dSy0 zr7#2B1MgprW9_Q^vkx@K^NyD6inGkTGHJLuX3#3*9CCb)D7yQ2oi`1Yk6<|u{WerO zg~ormoXM%<KPX~vb%7YB5lfZUj9gulPbePgr7eEAKMb=y*$#g8K($usN5ZmM5%2Ov zCsf$yPX|t@jSz)3vntP{_I;J%A96Jl<EZfSHVBWvvOEZvdA_b=TdjF<|Dbtb9rzXS zqb0&)%id-aIu|)8ah@sWr|of4clxfSoInOe)PL~AqZzvq!PM8}wetf1j`5<TsynG? zEu~yiaGSHu6V268O*4^CjNg@fH8y>R8nvyr{j)`9XOc#>>b&3x56-gc!|%dWGf2H8 z(M8-@mPDF?nRyRRmzADq02tay^eQS>3VEXZTe4E$JuTRqI8>~%JJdJK<uICeV1|iK zZG%yuh^y;K9U2pDcjqNOReQ2jq5I%Rfr4$wrSZ1wY<-SB&{G^hUFbwe0)tKyN+nZm zzAtA%;6eVuhL@fD$4W%^g^j?B=E$e-%#u1h0a0HRpsk)_Nf4Bw0kvijpWVE1%@ak9 z?<KHmvH7nhet7X5e};dbNx-(}ef<d!NxbuG`Re8S8P(@)hN>O!-~BbsYe&9hP=Cf5 z4%@Ya6@70}S902DoEjSD*N_-doOsaC(=+bkyL>71SCP#;*>Gkpk?w{S^5-f9P#P|c z3Je9KwEb-k<;zC+h994Ke^P)j24Q-=kE0J%_e!B=Fnhw%n1h#2+WTOxUCv#3FIWs@ z$8=LzwyCzgv_>RP&T0TfRvPFTlE;07vYb#5!3WxYo+$$=u{@{pn*A-gh^GbUGl$|j z&c0^n7ZfvWZNWoUE>o-P1}A>CBb}evWqf`=X(VNJ2p2i;c=|nCrm~wDX?#=XQVQ|l zkhHdomu4tyBM;@$YcE+nh{_I-nRn5JZ%i$n^>`Ncxc^eGb0{OB`k~|3D?`%*ht6Ln z9H&Gmj5s}AYHBZ$PMsJ^qxc~}3_XP(gQx$6sxMEW1RzB~5=}D~$qNCNqPb>*L1hts z3Cz!rHN!-u8t5fJrLgEQE@GKR9_U3&QLhk$%+kA*6lYx6d&(<jkQJr1dHcejD4mbE zfoee9KmjLEedq*eqk1ak8i^X;g>N{M$*vls%TBGGVD$_T>h%WJT<52{(8_?s1-23; zhUdwM_E_ni+rtvn(P;^a@HhD4!Q<^8O0u_Nvunnt|E|8Zh8TiMQRo?zM)frb(CbtA zS@@glRC(r9CqfoCg?~lrmk`5R0P<|tUbqq)+PD4)D*p@r_+DUv0ucTZHv&Q&=(u0i zgIqmHtop>uQLKA#KucoW*870_&D1=(L_sbeg!69<QB@eODWG=P0l&Fmdf26j@Qb=7 zI2hn9&B~7F3S*D5VKm7`9mn5=t)Ne4;#22zS?I^5qiuTHXQX<daYU>s=D3dR?8oMg z*<ixO{>HOb33Vs?>>IUVck2rs>%Zh*QN0(5p9ZqWug#|x_^oL&8K8A7Kj=g>mx!-i ztNiVJV&cx=QO%RZF64?9@*?71wq^v;Dy7i=v^`>))oBvSvN?Sz78)sA<@(s)_Gexv zg5?FBiTlYp2%x}#!W>BC$_j>Z@vHh#^IY_9?6o#l&YIMGAdM>4;|;~9Hg7|OHqw)o z4*_8nJQYP1z}B8^L+#_|Arfy+kUmroeO&^fa~;wYg>|QS^*7g)2i8U&x#K=T55q!O zt_)}|i0p__{3N7Nykh{7Nr|76VXLR(l#g?Yi)sAgPWb5wa%Gr=Xrk|ptkl%rL!Z6} z<?=vC_K`2HpVGX^li4c}&fyy21~_v%{0x{BxSSwSqk*k5klWm+2Ssi}#O|Z0?|M4n zyZLo(;q!QD5QQ0v$+RJK=N8me0#n@%jiiJxlVnC<X}||RwYbgu=fO%0UjV*E2X2tU za)46jIiVmOGnuQ<yy{TQlLIM>$!=gd(XSUuCM5|>fN-W&d~I3BE5$#-tOLPt1@?N4 z>DE|g`6cFEj(_joDRMYr`U+-U8>ztzBIp$w=8lt|GhXe#pBixD>qxmx*utUnF>d9F z<)Xu1+sYlzwHdr3ZULI)lWpg0ZkH6tb~3BAos~+@jTAor;ZtlInHAU~`JyiL>j%5@ z1qU3Oj?W*Xm`^S+X9=qfFTJpPN8!pVAaCXfU_oDkLl<eX8#GDaI%BewM2aWl)yDef zj<5=1sscPLGlqXN${WT=YL=h=+<#i@XN*9Y%NBBHlTtuFk_oC8I=xPx>cpoN2=kja zl!WH_Q=RLHC>fjvpvP?P<qm<QLqjet;Y{vApBKFUN4NS&qt~%_rdnQlw)L^0MeOO~ zK^3E6$__FG&?LlBZh(9!Z)KpX1k?nCG^d*NG2iE<;A1vTyzMgAG634}t+?ct06M&? ztqx4o0L5MR3shc>myBPXgsrdYz%ki7f)@9sYWstwV(IdKsjT)2KK!G@J3vJB(KaOB zABC0WlPI^$u5%s&KL~qxVO)n5gbY@TNa;?M=!`s*%u2UoH+Q#dS?AcA5MtW&j}K3x zeV<=#r(nt=<mMFD`kX5h<`2&`o*=ER-WQS$s3%{mntXrfY_80_=?U&H4W_r|(-)Oh z<U4kK&Z)+Tf^Mh${fFLGILElX*mEp7->~MzE&8cYO=ON+Q`LCsVc5L?>&v~WzmFS) zdLIEdR6g!AUbYcMU11un>TqRvIsl(*DCnih!lGEHe&RedwZj-h4`sPlp!G5J4-8?0 zNNIynkKP{aLAH46?8oPLgRPs-TINbYo3+tWMi8>rXhOci2jHn(PCQG#OX49EEKvGU z9w_f5qf_oH;uG4U=20{8^J;lqRqP3V7VK0M{co)6P98qSWpqPNwmePO&^9kxR7cEm zSnp=TcyVi{%bPJe*&&Rro6i-;cR5mgQ8I!_nmFYuG?`GpKMOSJlQd<)5y6OwL@z%5 z67SL*V2P`S*__8th9l%}u+)3j_I7O%;^1dM)>yB+$v|B2#s^!ARK0ca+Il_h9#{cb z5#GHAoQ_Bw=s}fuKA_J21_u8DHyVc4j?WW5)c})vhp{UB=3lAE^MDRWqwQ5TW+}%r z4&n@#?-N9^d#k4bTjuvCmHzTuacM_bpIf3|6sWgw57tYnh94ex_C{q_e!N?G_UqhS zsRj2FQ~7}7`}x&H5`KT(I|CmLu11B;3)VyxLr~|DUaHDS9~jJ*_1T-NDZ|imD1cHc zzB+|e6fu&v$ocMS%X(<hSA9OrNyRjk89e<Nf#qO#owJnAdCoFQhiS3W&jSryb*#$> z{2*qb7u7)N-vz95&3G4HvBMv0w*@mKlz3{uUjpUP?|S6qeBI@5VtHDv1}$|hV+C4q z?o``A^n@bs4mcm)xNaJGXf=yn&G`+w&_BI#gU|RR+)<t^@PvWhj~k2(Ps8dnpu4UB zDvY}<qeQH_dQ97O#wU6EF=DOGudnsv!5dm{!@8S2h+pI44gVfH%1^ItJ71P9mhrCS z)w6>x)@9k6ZYB~6?UQt3Qds6C+y}E0g7v3AS;zzPDO>NFb@dc0mI4{@H>hQ#%j}%Q zC!>%2qRfD*%g&>Mn_#q40dkvBt8W;nT>>h;nZT1l&&&5B(v@{O?v@I=L2_PsqCgX? zJpoo#DUx;!6$T}SB24kq+lvunCVK$1d0Zpg#9~}^3TkItJ(FB?JNW2WFsT4Zl>pQ` zo^Z`?Azm%45{hw?9&yE0f>Dp~PIMfHa&Wz|x~pR42e6C~2i-?td19F4dqb#<<=125 z-;F2)y!iW$8+n_}Uyy5_Tp4`Vb-3C;0NoKj39ftv-(`T)0gX1aOgD=w&r{)-w}o)+ zm(FZM_88`0w;oerlltvYVC20IC-M1H0PV0Ix@cT`bt9rN+R{wj`p5hzqrN5E-*r=q zb0?FNnMYrEe1l(GBiQQ?GH;}nLIl!-$mt$O_IDEN-VvBZwS+jnTcv_$HyRYy`e;sC z@enGyep06;*{igJ>l-_hsadx8RhF7l7pLZCvCkmSJ{D$h_pfpzdE!NHy_t_q*As3# zRUUtQx!hGsu`wWQeht>2^2V&>70unTwtd0FjZ}U4DX8K(n2YKY#a4T*HM;ZiF{80C z(_-20U|)+{_rFEG`d{Lh*3Slks2A56pmBnLgM}Cr-NQdJea(k!kKtPJ3lP2^R4z9w zrsbDm&$ELI3M(-2%z;FM(ljfOYx1B&$zNHA9Y<<WdxYVs?|H7(zx7l*;MCKv3g<%X zgH^NFns0OjJ-0cTQm#IzP&N~#+KB#w_2Rl<x}kuE`kJTuYI$*+$-wGe?m=Ln$6J?Q z{r#!oLOjk4H=*dbB+vz1xu@*^kfSa<!U|_B!u8o7stw=ED<5fLWj_p)*phn_1piDD z=jRD_V2n6703-T(3{He_c_IjFgpQ6PVVZlx(Uq7Urg~bNRizh@Jn#=!A*9B}cE#I> zEuUg&=I1(b`-On>(<NunIre(0**nU`cdw)dS#_oD?%c8HuhS7$zCWYge}D8#%SK3R zwp2NcijkwX+G(@HAjI-39rmw4%YVGHORF32zy2`NXicAM1~P#}YH9E%fox<a{0NJf zo^UnwD;Uuw)UM8<MKCiLFMLbMzy2rtqT(6i4fEO=qN3d2eNl}vrLt$=opw#hv<pXC zo_*Wx@WQK9E*BT*j1A@<5R7ca4-CaNM@sZY1){_RV+bKb2UaGh2ne|SGV>DHNrIO6 zLuY|4{!}>Oc84-hjeG<>fN^z96|XHUT45=a1|~<ne#uqq(BPL)ZsglCty0KHsx*CX zF9Ns(Z6rN15CLr-SX+@?bO3U?C{m!&v8<vABW~&K)cINnJYjX)S$ju0DSgduWN-(c z8;T1#HDr7Kt%vO%;QV5jckf$7G)t^@v7_iW%gc-ULfG-33bzGv9R%;;sy`?pYRzT9 zI>YF3ZJPya-ah*h=?iAQTcFI<cr7J1^Cj_nUV)*POeOP>jfrMDBV%2i>^sM?<LP<o zCOTe5ne7hOVQ`AL7SR=)MyiK(5N*j&_V+f#7j$u+ph@>bxzpxIYfD^px75=oH=8_Y z1IYLe<;kMp+M(OId)jMfP*+e#Qztz!Yjc<qW#=a)k56>v2t>2ErvS2Tx!}VYiaa71 zDw7?jWhTvW2Ar5f2_1(rqO#rSUf1Vumim=pVXUy`XUnR29_-r%Pae=Bo|d{!zM9qQ zwp)Fgi5^j#`DI;?BPOhz_t%+8wvKGdK<TfO6o7$lyaY_0iBJO8dn5H?cp%?g-hmBG z!a6a{ET=9EV-YWx<m%^^BRfcybMKzGeLCmAuijQQrK;+2rZ?EhvFjB}A_*I!K0pEW zsuv*<Fed|A{a__?2EVyWVh@n<&Q?MOqs1`+OiNd?@jE~jo~hu?3iZxuO~L}f@VvPt z;@%`^g{6OeIJ0(uYybK(kj^>JLV<}B9gGmk$o}X<oFAjbfM=PYk0uR27^sNgNylFt zH$W<2^!i3H#a$d_1lVb0As9x<)C~8q*Z;@?bC$>1G%Hrqb96c2WW|>6&Kp{L$~OC& zMeK!+rea;9bNRRNPNnjq&NqI)$+UNJ^)z!#Yh9jOa)<gn`AWDPKe_zYhIotHMl@7t zHJeOrH?%8hX?+DE|KElRML~Q&^>9*cT1{G7;ggYvE-o$#zDNE#M~|^D%Cs)aZHYAC z+>-X6H8t=7LVA=HS@+We8SBD8lsT*R;kDa}1Lmx6nLdS!oZ#>fov&jaOS!&nmpq@} zUMO%EJwj9cxR!m*{;SO8Gc&=hO>(xrXuyQh3#$7c|JnK#WqAVZ8MZv{O!FFv9nP6* z`6~dw3l1Tr;;*lNDpW}PMf5d5O3lpRCQ)RUG?9!GdA$@Fp5Ar(?4e7|e~Gl2vfRHH ze&}ddf4+(5$*RngQaZ$NZ@pDno83CDTy;(?zZC1hNT0UlgtVtnsAPOH^swI>)-u5P ztg>%_PTOfdpMLnt!dE%Gc&*V+xKk%j5wN2VfmKcsoFY~R2rCy??<9!A#2NVbIq6sY z>1-vj?ntw;<Z{FDCq>D5zjaneZ4(n{u%FiY8Q**Pdg52Jf-bVdR3b6KDho#2kiUu1 zSKNQTut*8L!|Lv$Vv>JGoC^g@r3D3v1z8p{uCoN0|Cu%O^<MN)D~<4}0K4L{@D=`! z2SZ!AvnHd<Kkc5j7-w`ZW87E2=pFVuRsW*D+<0f>t)ldCi6JzV<kG^=grDVs@xh~~ zz`YY=Jrv>K;{cR<^OL)|_7GF=6NQOL(7qjpR+{zfkx31S2HXEmj<f|!9}`hwEwQs& z&T=DM1doWd!K7f~y7u=YtqrY@wDLTw{jV2Lzfm!hZn`=PiAf)XdKdp*wyA#w?tRin zv?KsB6mCAO#klsicx5*;h7K%l5#l#Md-6~>l#lvzGN^p4(Mwk_+dq@6l`Qt0nI*A5 z^K^mu9SZ9LO?kxywFBnIg<`laEfY9(TqjBjjkr0OYJT~({))0T01Yl#66o=>;K#i5 zv!k-d|EVlVV)X3W-}oQ@*>;YeAE9WjeL$0EC4Du^WSsrVB^7@k&$I<00yH?Rf5Hdj zPo<S=J~M88m)<VHYOVs;3AD^_+tMehG>SUhwyqZar)51PF_~f))LX#vbztDF8?WEI zdHv+*qd8%RlH7=IPagG&TGqUOf4-_x^?`;Ed+)sWE0DJiX!a_Kz3u>_Koh>sCxot9 z9=i9j=`Gh$@CizN@*$F4sdcv+2<P~-yvTPN{_<($7kz(JzTPu66$ZLB@I_zS-he(A zJwRw|s*`;O#B-K`C9tMR6vt*9YQ@JDoSe|%LAhra=c2o7GN_*f@8@;{X1AX^r@yx@ z(3rR}h<o6=^j!DQ^7MUW9goZMuy(&HZn5tVgz*P9)b0JL1NQ8)-zrN=N^N<qzhh+_ zybHg9_y2RH9qhHOLEJ|Q%ZV_Z&Ma*>L!hH(JUct|_$fGDxQn>%Jd@BvROy3~B~~k$ zW>!9z!kov*1gd_|CEFwSbCC&=VJ}q%A4oJ*Gll?-smCx|lHhOJfwn-_)@M7mA1C}Z zt9;Fib*GknSG`YSI;PJA*-zWI$oZJiXIYPx4}bZ2XHAAXQ1$KL-W(NILM|g)ORsmd z@k`NHDBEfd6-`|H$ZL?=xN={axF)abj*s&&-2#cxDx0-Qj1GAo*zwaxizDn8Xk$(< z(l7`n{$bm3wnTSM`Asy;aNbJ@$ySRE#WwrC&-E@3*oJ7G=J5^~*M6huA*}zyC5`_# z<&S^w+#h-%iBP#?zu-Me1SDV#&M)*rC3t#ZfkYf`X4{9;XRD4`l`jo{_L^DZ$i4D( z?=qYu)p!Sl-@l)&82waUerhx4mHV{kP>6v8qIQZgz?1v^L>o`fk*HOo>>m?|vbl#T zy{_%57+-@@G!)-$GN<jwf48Ipwl}9nsIBm!T=P|T{$}Q%M@>GpRqwXHTMipj%Ar5~ z@*)dCx_$?+a?LvXhI`y={L;60)1-Kq@tdr^vDkeBvL2@^ZMD0uv+Ex#rc`O~7wtGT zn)2et%Vn;_&8o9@=LQKG*f!F4+H|Wx6w1{QOtv)lBE+!b?17{WLIPAQR9vsm`!wJ> zPl^EnxDFAlTRrPzyX@eivg#`!kUh{@aCp*?vL9s|s4dr$mH&cs;L2K)k(RzDRFj`f zp;ypE@WJ>Ls0>^PgQx4Q#eg7!3t&YDS^G9@WDFt@{~(9918Z&+!walgZjp}<i-w+? zYFt`hC+ZrC%R4(iCmw6Q>#LrlUw=B?cgmAvjGmcdYz2xJHm`k^Umco{fF1Y?dXOM9 zz&&j{&)*I6-W)6gp_Npa0ZmHUTuKQkAICoB)0qqQLY@Nm(N5fF6ekV&X?+{wvOb{b z7Wr3o%3C^pcHgLRic(4GZl~R`AGIt?HLFf8*keSUtH<v0`!LKq%a@f9{<>Jd#=@BQ znwB<(cP^j)Mv^~x_WCLJ%Oc&SMknrZU%#8su(REK>Kk9heDeZ12~9vL!lYS)eH3Ay zG!w%^>9(_*`;9;h-vkH*WB*j%^)f4>f5)K-?B$5NOnyjr%kHm)2jt_UzoPy!R0@bX zq*ic&;htPKvmy)Z_y=7ATl)oK_%2H@n0y9R=z!uzp^dfa@*pohXa<DcNUW>4(QSxa z9w&IBO|}>`4w8+g(Z_n|GB{PF_)^_qHDGSt{$}Qy#fKGS*y*;P23g<mtjp=?2y%Hc zo?f&Ksoz|MvZlK6jg$nz{uNXMvNxR?IM8u`pVVQ-)92?148g6h7CWDeh~GlQqUKS( z$Yf>lH)9}iCh{HA&dF^QYfSW9rlu4eeHAepeP(ir_-1i3%L9YyKMb_FAfz8%vaT50 zs=XZdA<)=5#JIwz+@stiC88utU09=Kbm@i6U4_!2AYBQe*uHnjEE}ZkGuOdFg?`7h z`rhs=L$Ri!+1q~am&ODci=;2%%=I+NKT)*^p8mo_{jtCRnBf%h^I>5=;A92yvpXcP zdw}Qjh*Xpa49#^$3EttQ1lm}mWScRZK4yhJl!CQ>iTT-kzaGf!-LhDXT+LZxiiyjO z@XnRd%<pQv;cC&-<3QS{kU&g}?Znw%#|Md&0qaR1q6cOOYG69FcB*;_b(nGs3dMq7 zYS6QuoQ)^%0F0w+c}l=TVt|k<;OQUENWHmyzhUG%F*rC(nEqyrNtSE6>lu2L^lXY0 z*R8r@iAivD%D6Jt!}>0tcwDr0e^K#SI$OGDA}P#EJ6PU2q}#qmzM)d_@`}Ag=5A5L zz}L&gmQ7)>)XSwqe=qkjuiLj9%-fB3k8@I;DrIf(t#gt<A!~}cpEyt4+liM1>9BSo z41CfdkUw;Qj28t67E>F@c!9px=^jtC2r$}X)1Hmmhe^4$Rr@@QqctA4u)6hFKLU59 zVxWceO-vvn&6VZB#}GjzK+`pW4!q_1T%lowz<?~-sss~$ahwvgPW5zVfqR-5+_4L* zGFjLL%rla~me+gk*wBd^ug7bzW^eZW6)>gmK0PztMnwDXy1NZQHm2=tyZhYI`nkIQ zw@=z~rv|d$Y01a*FpJR<?JL&zOkZo1ES#CqC%{fUPlIMfY0Ifg$2)#&efkQh0wTQJ zwT|29SR<m}d4JK)1YG~6?ITk#fU3i%w@I^_Igf$ldA<ESb0C>vhaOOtV5L9o*nt~? z9@<EC?e;!Aq5aDX?Ug~Jqq+$P#5MbA&g0rXA(+Wkk0sG+Eg^3PSb!OaLhymKI7NIv zYPyvf9}m~!8IK!^n_HTEe}rDs1sgN`B@_b$3T$M&VXKyxQ&)!;+5aWiJFD8=J>X0? zQ{Rky?N1$`lzSg&<&?3&;%5v8HHE){jG>5ioaJ4L5r{N!?UQLjJa>Rt!-Ec&vWl>g z!10g@Z-2P@2zD=1z6JaWgVK&<bg-{sX?t6Bg{^AytB51kzhla9qlm+T{U9i)COC$h zrtNOCtyqGR2|U?&foK#1wV!u?1ipW1PJsnYKwJkQ21~YKc^5tbB?^<rvy)NHV{N8v z{KW>~M(#<rUXm`EtHHCFe-@T_;2rfWI&fcutgC$XNS|}@n%2B}Qla(vU)vBWWwPos zB4^mSq{^9SD`ok#`r4_d)YAA~$IvTY7cG^OY0?%>{r0IQN%@Bn4f(DHxm#`b)L&d% z=6VN&E{&HSe`_UXI??NxnQz)38}LLheT%eU#8E-cgmA)vO?K4^6ZLQ#avK?gdi8*x ziM`Gi^JYVNGGd%E;2`%JW*fpN9~g@FD0{9Cv@*cDG(KI~fNe1pV~E^=!hqEhw-J(8 zXhI`GymwN}pEJ>fe+ZZ5q4_W2hp|_|?Rr5ljMxtg0nNk7@eU>23`ohge8`&Wr%2<2 zp;S)>B~2iU8>0#1PGa^B2u_ZL%?fcWSmAwOaPE#9^5{yp90@~-=R?g|qe~t^_#5r% zAK!SIFEN(1UG#Uv78^{OU_4+T%NBdvR657U$NR+V(bNFIX5#9yy<rvOu-0&)M?m{w z1R=Fgdk^aQP1&VJq)vH4?dTNsQuo!0dG_hjqAcstVsDvp1xx3`q?F3bB%2o%t|Wc{ zHUb!A=uMB&bNYtTJK*u|uUUvLbiA@YP&zLRGxbCNgCqxbc0eg?-Dc%fpYW()+3yvh zYiH=^WO5``Gh|(|c6>I3OJb2=oBYsY5E$+_je+YpN`U?xz|%oZg)78+Ha!}2PwtsB zVhIRLFGVwtkJRs&HN)d$jiz^DcZW`^7H0YW1sV7a;}?^WiC|;vP**x|R(*b|4^7Ph z>HlHjTxCo=r<DI(ASxJwiVIG9W1w_gGOL(jwfeIdNYg<v<-1lq-3<sy(wHs0bl^R2 z6c9JbS<}{4v6EY{f8EefFt%k5t`yh-`u3rq7s2s|HV_0=P+`Z;zH9eZy-mtnPh9%@ zROqqX`#lS1uZ$hsdnV7mq{4@lcIm&c_vS%OWo^GGwgZA9A|MJvTA5^41ep^jKtPDh z0zz65nS`_g5+Nok0y4MApdh3fBSVY=AqtoXVQ57}W(hJRX=WmcLfo3cyZXCzPn|#B zug>@0Q>W^js`H1XRV16Zv-Vog^P3)2`;$7)xhEuxv4`E~Y;hYmEB7Ug&uO%MF3P`B z+w6Y4_B#4g@|7TcTdw2>{C&is#<U5^@HB<^8uqb~<p&K3oiL+6rC#HAAk*7SW-yd2 zlh*_VxK>1bsUiPqWs75SlM$i=wGFFFTDH#Tey(2JeqtJV|71&Q$fqN7;Ora|DY*!& z#EotC9Qn<Xe$*cMG%S8b9p38)EdHpyA~;0k1+cTT{){kn+?<nE1qJh4+cox$ea7k; zz58&YrCv3*C7xh?Ec(>@C;y0X?|ch{dXeG>^w45J1Ns}MvXS?8qj>K0j~c75CF*=Q zHI7A=Tc#OP#edpbn53HB#CQ7md#^EfWW>}}J+|<@?-%$cYE!>yfZJaO(D&dfq9AdF zC=M+C4dPe$NF+0rMQ6b7!YF8wqJdx&KmImukK`>()||W(>(~VK%@ERC^q^HD7Myd* z6t=qBiQ?|59WVG6x5gMgyLYRa-ye$G|D@+;j5^~$|Lldvtqv)PSv`*84DGJVu}hYT zxhZNJ7~i8KWuJS_pKBR8{BSn8=0r|oJ|GY?o^TwJBXSZR`cn*MKb}!T`M=wL>u=A| zBXwDQb+ey$<^G<%`*GNS;2GJkCC<R4u6sRVFiBFqK;~EofPO)wtjoqF2{+@$H8%1> zMM{or`Ao@CtRv*fAuyBJ7Txj_Fo8Q?enpqBIV0c2_0fWgiC7Eq!ze^pT?XOq^(nPM z^V?ThTbmHF;WXXm7d%mcbr+>;cBHEzf!b9PzyXZiOh07e$PFe*KB6e!+L16^JIzd7 z0hh+TBgo*kisHs_%D`9>&yv4GXcVMzq7-{HeuZLK)F_*t(q93X&z5uMQlhS@^*Ibq z{uZ_Srp}kQZzGLGXiD$G+>y?qv*ZV7doCW1eOyj-+ly^rdY2q*b*s(Cj%{;zkf#^p zTwqR7PR*!q_<>-?J&Wb#&Q}!qzGm?#X0tV(8vLiZr8G{K;pT)9D|K(xxI=Z1-nUga z{`gc*FEJS}=HR=5k824mq2n0OPLvAmV8oO>R0sOpAU>*-tbvL22kc||s>p(`$?i(- zSy{9Y=}L?TRw<zrero6GDDz41om0@7FNP~}&b6RzjXJA-O4-0QaZ~hWX$E2(CXo#` zxdlIn4dI7owE^H%S{A76)Od)j{d~cLdd%1i;`QpjX_4h~ygXKMOmYNM1P)^3Fr)6W z3h2_*B-nC}k6@$Jg*P)bKXo-N44-?9+qE<9@E#}5la8a26FwbRebkcoZ@%=&M{{pF z@m?lLS9V`-foB5-H0s$NjdA8D<u~6hvDZ~%x$OPpA|}PN^=d}!p%;FGdC?`#Pi;v$ z_%J2kukFH33j}Imdk)i+U0ujtm6D7D;_l+BNCAqkl0FBXc3_C3%)dr5i#riJfTv<h zB-8e2kRC55Q=}<8hM{*R#4`c|+KZ|wiEK4>2e#bq?r)58_M3X2g=5pZB=0q}#O0D- zu=e6Kk+<+S5isW7xq}6!Zh#M_@aB`TF2aCt1kBc42yRik0X^z3nob0>6TTY?XUV4q zsX{FU(~Z0H*?$i=-D^{M(#{hyh{tk!do5J1{uP@0#3Nf%`8K1YJZDUtr}=I$w=Q7s zz1Ks;zaEpVycy0UYiCqI=Y_fp#Je4A1FxFYihQT{1!`*dVvBzi)###AycSq`!4b%! zvGio?+I;71gipmiqQ7OPy{ZE7UY!^<g*^*J60!CJ@SP~pF<xL!7k(FnI>+xx#qFI0 zH6oS&`PEFnJnCaOurOqp;kWiBn5-jq_(dc<eT0rr+%6x5K3cZj<F6e3jg!7#Hws-O zT`2iB0Ksft7ne3g`EpFSsedx~z5-Qo4NfP58YC@l1YSM7)RJW1Xe>Endh5zXUO&SK z)qzqp{EXWxvdW!4TN*&vzTk0gA>Pw(;!}UH*66@#!jMU7>}gwbmUtVZfZFoDDv~8u zJ2PNJfhj~<7v2*c$8X6G*?iZcW%XWM@guVN<mGcSe#YeJxq`W+8yYwMz7hEF`}fQH zISr4G!fOTI%Ol;Mzh^&3dA+;(QHJSCtbCT<U-<3u`|H&t*T}S>p2J<gE#khU>~+U- znZBf?cJ|Cm<*`@$&X8A)-Mr<e0^f}T-2BOLd!@4*{^O7G45A6ebrsr5-oT|Z*d|;x zeq;wi*4F4|jeXgIDJJDhIAwJYl#8h>1)Y_An|kHMr)8k@46)t$PVnPs@`Wc6g)*D6 z<SNs%kJH+l=jp!l03R-7$KQi1;@ho2ti(<Xe0~LP$wzi9(zapkhBunNAz~fH<#CfL zGxCf+XSv6(`si6W?FG-yBps(;z1k;}X2wuJS;y36A1Ll0CrD&~Ik!_oEx4h-n@-8; zl21iE7i5qMkMYkhYHY*UMsX(85m!e0G?#I*1DB=V?WdNWzIzSuV8J3F8@fZl4bwJ5 z!85Ny1<K;$HUtZ~8FZdf^e~fMiyGU-ugvLtp~9{r$JeNP{>aWKUDjViqF~{EjHinV z6=%hxr<xz77kUSruCLD>AYFcGYWl<MT3ei-CtAkc^MNlmm-(aBf7Lbi%na$#Z#6Yl z>D5d2HpU^ZngS+ptHWYtC~^iMEqYMQjC}HxBK0Uu8H$=A%R<r&pUPj4PQiHBlP|Ei zs-Nx+AJij;kM@4K1rL(i=NpMCwMGG2jYYq^E;eWf`&nTJJrnT^lse|Et#&5I0kad9 zWzu293++;!oE;notqfX_1_iQzN-1FU;&fjW4IdA!kF-8Yj5Yo+vH01?U1i~7LrYzS zLB*3xar^W0Z|T2%&C0v8RWqqbY@F}x=|j~1>}TGXbBVn&TC8*b`^agpq*tfuIIi7r zD@8k2#Jm!x<nF)se;WBa854!Rfwrsqv~$vb@}v=^kJ1@#{;I2OVZe10@2{$mWX@$D zpG~IEH|>mwIJog0*yS=?`ueCVMlAH5g#6AiYwAieAH|tpi96D#>D<FWG*Y|eRUj3X z?en%w)EmM~36T+yw_~IJ)aLFF^>}XskhCXLgTZPFJ;65SiQ9JS`eG8N=QW;*KGkeX zsUmZwg_anT*FU~!DoU#HFv)wGS(}tpxvPg~Z{cNe8u<-B_En03g2Sj6@gUh=r*H;W zcDIiE>_q};a`<k*`c+rSe!wqSefXcD7yiBXqV|cL!OBj<w<B=dF&q>W!cP?}h_i7@ z*weiF(dMax&`rL9pcvAdLG8mtvxUf}k@~20fHgoB-2&<l+o5p(<m7iwSJ7%t=)~=R zZNH{@Nc!Wj?`s%OU;=3uinSUAQUHbeOI(8g&T$70-izSZ`QZm$5WmTb&g6`b{gf&- zm%8~=D#)GBm<jDLVIo<fnQ$4=b>La1>0%A|pC7lN(nWjOk)bb!%xy4F_rGc?wTha} zzG>mrJZwaYzOovnbLY$3&x2}pPOKN{R=%@%C{abUt5=fiCzQgoF9G~V#0T1z)aUqi zL;9jN;hD5V$8iP7==xR5tE}TL=*Rt5>sgv@>Qvn})Pgr%9IkN&29xNL(>C4iQBLNK zu71_66-4nP`b-JEL4GCmCTB7RG|OBU2Y!5~Sk0B!pH15jaoDL{{h4$n^J&?I=GDk@ zj1x^)6vCgPlNV`<jmbZy24Z{d9HzQu6hd9DTBk*VQW@NtcF?!Z6isiZ87vU?VfXS< z*#ZI|b!-@GRO1KrpA|RJqhY%uR7f+V3+A9%38!f`^DVgQH=@APxx;_yuMkxq(&o>P z4i=pwHWlox$`TVL8eO2kY4|BsS%7;DgC4;e$v>rRBM__=6v&7C;S{<8FtoJ$LhGMW zU%N>a9+d`zogI~BY31c*?njB6RV<YEk`wfeW3D>YS@n@Ft)@0qE_yM9mO@$5%ai%n zoZi@59Gu8+F55KW|HR(z0I5?lK;SvjA>D7>V15ERds+jhBy#5CyQD{AC$IMh4bmf~ znaeCe@t?~H$W&6BUPwLU(gBmB*@UZ_B$>Pa>F<a(>V81<bG<s}YQ*R@@BHGl2uwoY zz7*r16MAw|P@BOOJs}r7+fH6^jXFe<Y=#=-(@}fO<g*alKpKZxuj>ID!ZzH8(tZ4O zVvZ5{so@nv6^~M+La*^t!J*#TS1ZpeGSUC_WzmqiU-l5=g#Mm{nB;Rq&L6UzorjkW zlH<Ipi_aFF|A3qfn;d>zXM3u8HgG84`1%B)p{Qx`D>mr8i_-~FgTL7T3Dk-0X24JC zsy0JN0iwMuo0*%S%Ep@T>pJjFA*HzzTc|9C?HbG}ykqNmfrH!ufHQh$23ZY}hSS76 zHBkspxH8nwmP2`}421K>r;YmZoT6#7x+{G=2Nd(-gn;p&-G(04i@XyLYb4P-k&u(^ zd0To?e9+Mhwoy+Kouv?n4MVnepS8as{*!rO#G${D#G~kJ{7M(uWk_T~0DWdCzJ=hM zB0^CT%rKq-Qe6sAYq|X&0idO*DTw<iHJ|10nOEZ-x9BufKWk;x#dPyNbpN&2gWSfs z(k%+@eN$gL94v1HrJr?&T~iKsoFw`mzRPaPWbJe5b}$WaKHNo3caL}3H5vPg+~z-q zn0bVH9{oyqHV`+y3v0yRohm#YQM8(bb!Q`$As7eQgG$Bi<5Sacu&zk-j5v`EW5{p7 zIuC)@Dta|g8Yp+2NDEX6)`#B{tLt@)o-p?BZ8K*O@h1vYHRe>aKIxJ6OKU5A|LxVs z$rF0x22Z-8ltZ+te5!63M~rxQ6pwni)AjSH;epImm)7Y1>UG$-khPn{|H^<h3CdU| zP1tL^(2i9l(H`D(HiFuQR1u&VC{^eIW6GFs0lsgRYN~&r##b(UKd<`sRY7kSJOIMA z&jw7|Z)&vH&7g}E#{{N?c0$t_1E7PqN|i=&>XMH%#IQskH-MAkZw8O3$bJS76P+F< z#kNKTbPJ|-jm0fzAzn+gUlgAjSk-RjacK_>B8o1lzC?3RkMgb({+%s1bR}FL<?B_k zVpkGz{&C%d@r#EMfn;RCs34A?c5(T{C(2<2y(k}r9q_JEy<exA>mBS-;;l7pTIZ0# z7^tmksd16lJ4>|rdLpyrAY}_XLh^o^=ZgB+D)4*7$JOy3u;l(O8q3DYt8{=e0!m_2 zyX3+Dv`OV&H54cnm&3vpl0`T9-_ppoHoQ+2PYO>ARZ|8(%e5h6BF*kQEOSsmyFZgo z=eeSjC3+BwW3olk-_ln==ONe{sr-{OkC}uXI`d>_5x#v|36n7I(KbZF?Oq7df?DH) zXAoP=r|ROhK@x3mBw9Yg!N$ENyfE^@#pbE9|Ktq%mj{*)zsAEd{oIS;ysKVT`}C{D zmFm_FB-f7kxeTIzOx>2Et2I{dkLDF)T4HA%XsTWuL%6o9SM}iUPl!%uiyq_l>J2z& z<~{JJ%(uX3Wyf&7iFO^4$jl3oSXchHuGK75<FJMfpWa1~9fxe#@0b>y{h26v(2fS2 z9&Gv^OWHEe4!sRBo=yzLA~c|!Gtcj<IFFhKTCe&HX>`AhRQsqqw4zNB_OPaJ*=q7C zUTbWH0%zoOg&SYfci3vO`hUS3IVr>uYw^Jt-wuEK(j=DHK#25z^|Eak*I)V3-+q4b z6Eemme-3Gu+h~Ig{&((;mxs7-?}aDykksPL5=-H;YpG3tKS*y1NNuir%WS(zatSE) ztxfYG?XyQuv45>nwn`jwFRI#GrF%pB;D_?c2Tl2^PLrO=v|?yNh`=5YSD4AmV1V=c zE}eDwgJ>Jyku}zhm;DHsehh&zJ&xK%Q^XVub%PSvNkEWY_Bn#WheGKcgPfIkQ`ZY| zvo|c7sfUi7y1I@U$Itjsgbo<Ipoz}grz^P2>+c}N&!iCeyZTXlpGS7{9e9Pv0*Ur; zn>-t}W3gz(8j;x)$KTu#<Oc+(i{Azp$9B0G6O8JRhab%M|JRR=>Yd3pLdz#HUUlB% z+6ffiK)X*Z_otMJ!EdbX+I}MMLjzedRA+tJ%1;kGuy5UoTS}?P`ah-os-$mnTofe( zZZ;?As~7W;m|2_#I6r;`1qWW1roRZ=W4l3U?GMF+-i6Oy?Lx|6;IK6O9x{9jWWc`s z=inPePMf^YA_BFu8oD{tzq@j1uMs-7e><};*wj#96lM8UJ_}qB0E#o&(+%IaaWe!a zl?_Xl><SmaDAXSLTcyUl-30am&Or3rKMjT&B<O$rgLa!&7z7Q?z<z;f@^V<m;!`8h zX5AF}n8=P4=;wGLnr}eKFK4X9{~ITY-^D|`&I>rAu6}H=Y~JD*xUaLhp1!HT2rt)i z^{$Cp;<LNg(_OG|{c70W^}qA|zm!<#+m+9Im*_uj()2^H+r6(9XHJ|S%%eU)_k$ec znL<ziOn0Nliv<UuaAAPR2B^g9*o2h~-aq7GlI;=BKL@Z9AiNCuj(4#xAdiZH+YBX# zu#jo^t!)7;(zW^aA5R9(`1*LBH!3KTd(}og8g#{d&#O?-Eb-3#l=^t`HGU(`0{2d` z2OzpIf9x;f3R`r&=rq*DD-uPvfQnY+%gIV~;l02A1d~1H#NW~wO9QrRj0NM!1U^{Z z+D1)$z|P~1(Yj79_shSo#47%ix^1=lqHcEQ+{2=Q^nw`u!Orf(vqW!S@^<@kZmKn& z`3YXNFI*m34OUs)@XR=Gsk+UC5%S`^ozs&LrG+8JTaV~i&WT+_r*41Cw}(njRjHYC z_5I&oqy3ax830uxTbxb5bThIOrwi%w$X*n_Z#q^pg!k2(kM5+W)A!Kqpaxi~b!>Hj zNvbt0jb;m&hfeC03WUww>em!#n&^5ol#=3-5gw9afdXtUxMVuG8(6F+l=TmmPTK^@ z#}k+Eaw6}m9CET``-sFu6vIciBa$_ayeR$hH-^YUuB@|H)8eN#LE%p)KB1=UlDpGb z6Dotu*@-hplRr^i|D8?ACxYbdU%KBrm+Yjy_bVJzW##2;N;h?`ob`Akx|S!=8OOaR zG}N8EpQy@x+0k1NHRM&Ccl|+ZZsi2>Tlh(qj}44TmB_3Mb#P2(4+E|EPm;471+9Q# zur{>_u@?OKJMe>|^N<11Xk4f1Qz;bw(Fa8AF-Vs~mXFjuPQUZ^L*O|ltRUCC!R7BX z@AEg^XUNhEFnKJDh1vjlFhBP)G`8_RX8~Sne0o{6{~nqY>$(w19c*e%>`UzQPo63x z1#0KlZ_^%GAL*N9=oPSUgv4KmivN@K@vm1?{i0+kZ(&e8soE&Wy&IcJT!qIgSCq_K zyc=1yid?lB9<1?DzWj1&C`aqkdfo-%i{cT-$()vwS05M4sncu!X8*Pk{!^-)2D+Jl z)nd`j@H4e|-c{+98lWfc2k|7mLLE_rI2#`gQ$LT5<bgb?K%NV$pX<uwbixf0a&aK_ zZOnr&ogO;|MLvxzEQKWoqE7O-PlltLiiKO%&Yzp7-ne|{7NVSK@cn@}12MA#gS7En z7jYJAI$dJIN3KNiH9Zm&tzywR)`01my6F~w_DvNwgjdrtSRacSJY{l|eXi}$?c8IU z`n6y1&ZQds9k<zBX$L+s*}M<d35ZOM!V6C^iJdr&;S~x=I%oh42q4Gy1v`wajn7g? z%vy`B$_lpK)5_H=&%cy4zGv5lFH|mF=(k9JKHsj(=Ms~RaoQqpeouy({9SzGbo(pB zSyuxusEgm*V-&BFWUIJ?WVoyPcDdGW>t-@JqM=92-y#)jQWw$Sc&Kq_@Ana&uE2XZ z7NOQBJcApOm{7%Kh%2i~i{F}meCrB4G0L@7;T4`5)eY#)Ndh~de3H3SRSwId!^Y@Y zz~@_X@tv-h%n^zaDr32Z*C*&rhN*1eVI69RYm7WXx7i8RB#b{mED|N%QK`8F--Vhm zVb5nv_DwAVVa>La<Z<j_ap8J~n%2V#?bJ?^Q<<+z%_H1%GKJMf%dJ@d@rP&K(P0y^ zenw-Yu&<ij;^I2a|8z2^cG*_W%F^?W+x4MGlvu(yk4V?zgq1t#2g!Bt-x#1Wwzc3O z<(J{R_&KRfh}}R_m=HIW!-j2#ik+T~wIRsiDtz--g2MWRma4X9W_5|#jzvSoE-equ z{J3B5qY4f$GN(l#sHB@F*EmQi0VXz{8#0Y{ghyvolws-uM$Ngw0%a(PW3zy@m@%Ns zL52QBbBW~flYxBi(WEk$swEXmo(_q;_)Rsc1B#ei>U9=JkdH!dcy%3M8o@5n8OVeq z+`5j~B=X_urUONL^_u~$BVjn>0h3s}Q;fM^G+SK~$8z^>QPnGV+F4h7KP%$=a;XM4 z3~K@e(l<FqMc?b3begL??|(r~RUp#v+}@@_$1m}>gTF7R;?X*3h)m2d~H4{i=6 z-4xgGH%gQ#IZ2|#y2e&yJv7ZKkpR&`7jtCc!NWMU{+*DEdrQl`-RGi>XVWz7k8uUB z0<CKESndY;X-)~|*YuAM)mWo3M+vppWksXnYA`k+NM2k7Fugua0T%WP&Kev2a!4Yx zgn$92GiL^sQs0J-=UK37F)jV=KVWGh-<eT*Go8QVQCtG;*V1!lp7{FqaMpKYqqRh+ z$RTO1Alzq9)uYbL`wm|ky7oPA_Axi^u)OFKe(ANP_vVK)2eodadXXzN0Po%l;&fHQ zrg>+lCn7l}o#^dIVXy>jplTrM3YLO;-dPxpk5EHMVSlB(ghGXfvGO@HdE<QbQfmFQ z8p(tEz%Bc&m2H@Dw7TqR#@lySM}8l99Oe|mI`pFce9a5zicU)h%PU8BDb0-djgT_C z&eo}VosSr;O6l|<9(;c_wBmNVqr!<Qr@=To3ijx3Iag1wSs#$ROL5Kxp4hxY56_>u zNwU|EUG^^1zn^ceg+DXB19t#|frzBG{rr<GV%i!(6^r1nrZ&-)F^DdI_17(f6z)bp zO7>T`;VW~m_iM=bUbC*e{~~Pm)cw8E;i&%7)LPsSo_CrIX%?=0WhWZxvP{yzb7jN0 zF3|#XyAsxGL=;tpN$LbZ*B;wo{8Rb6z`7Vb15q}8z#Z_1zOm)9?-OrQT}EOTM|)Qm zn*Z{cLyYz($${EsykBsMx4EYqaSzxzN(!=At#*;!a9xnKYvIUwmDurPI@3h^pH?x& z)|*f0z8_xEd37;?Mm4@49CKCq>9VWo=pOd5CzKVo{@IixmSt{kO4o`8s`UEmEIdpf zTV2jgK2@>1N<Zq$8(qtxEN5f#;~GRrQjLu5sjHHK54qJdOG8!0er{ebYn|^hOZEy? zvs%3)l=WN8&*T0qZ(J)0a2{*wm5V$%ye1BH7>P1kmoxDhL`PB7k9a(5&ALo^^^D$l zpWW7!tKyh_Zu0&}lEKd7Bw5v~(WXAR1H=l~*vsM758M`)ipKvePuJW2?|NVUdHr`F zBOyo>av%c`w_4e8%IE(DjtH<ulk}$tHcH@e_&bOu7}tc=no+G;6KKGJr|n$ZHuhMa znILk8PIr$<w!v5-3+6z^6yLq&rxX?n00jP4ggTw~5T}8Dg=3D*-59<RbLJxI9bsDO zr<CW~mD2O$(F%er+T7RzKwEEwR{ndK>;M1pM9SSu_#HG@r0<w&>GK)XmL+EXgl1rb zDj~(pMYvp$ZI-Oc{=C%n_oIS?oU?w}%D-ewDUZ7!)4>0f0&#~Zu_7`R*ry#JApFR0 z!go|dzCWc-4S|&=QJ{fT$=Rrc%}N{QLD5@7QwDgY83f*V7oCdu9^fd_M<?UgT;VW) zZK$#OB|c++{@)A#-&^s&XjPydg8&tINGtwz4y3pG8nzYJuW@MfDSRsibpq;V!P1$l zX*fm5qWuFjc7CYM1OzEAC}Lb(N6c-csUNxq_}AW}b()5Q)7C0IFB}yQNXYKTV1K}# z`-*nrB{FECG(I>ICQT2S-ijf!-7?L-v!Px-%~Enm<f__Tbng#LhQly6a8pbCEefO2 z#3XsGZ-?n6>?zwGxgS5J?3Fe^Il>bYz|$;|ot+^lE_E2lZ<BnSekP`Okhh~`_NU9a zuvb@N_@Qj+&X*#)7^Q1<Nb(dI5meP?kH0+FV_=ucFgkrw%PHLS&3bY|DoB*nUVt@Q zwBsS)LJhrPxT@$BFFr$L!jGImbVqGF^K3-q#0%!2Z=qpld8nM&5)1bO>Y>O3Q7+D7 zW;K`6)ngc?I-_Pzb1CZfv&g=jTLg+R9nbL*7vdvLWc{(5#1!n|&^nhI5#ohvGM&+% zU8?rMM6OU|ygDA*t=e)Ar;9ZnCb?GqE!f#_a?e)VZmq~7tsYoKa+khxG3dipD?Y!} z=jpdow=Gmwejb#3iE51=8oG?d@0`&WhGr90tYe(qCc3IFO@H%mpgT||VUYv&%Il%M z$u|hygbesjNDfdNsonjsshdgd2pQ}~_VhiPUd#47-XBEjWz&(&%<FZU8_d@V1SaZj z$37Q?2d8a_Kbt_`jiGkXKpuW82e}XP;5cX&M2D*bmxHwU$aM2aIcWxSOU?yFWMXZk z_4^uIpC8c#CH+3Fq0{%vGaF?^)a-A8pdT%Tre8C&3RBi|<Nyj{joX9iVbFI~jEExt zT!w3m;iFN2V=h@INZEtEP$tqTGKMI5+X73|{2)5PEU1Z)&yI^yJd=TJf~z&Rw2d4l zdADCXIYmMw<GKA!!L4}h8)l`$H8t=X!lny+gqRZlV1b4{#r%Pi$c*p+x+jglXfjTH zJ%wltwCLdjH3%P_6Z`DlY9%z!wwILgHM^}{JD+9<GI~+FFpZwn(z8z*`l7Vb)9aN0 z3cnE_g&dC;WQ|c)w}`Z%E*2@2t($DTYS`w>s|d>Q;i+`PwtWi!JTOVviPho@N~$qA zE{(p9<8j6hn;+;G-nkLXAUs)~TV-w&91}kR%H0fg_z-8hoxDYKoc3Y-60ax2c!fAX zJAg4|TWueh!l<xAQ$8DtPR=CGJ}gvRv@UsE_Hx=Kw@N>@%)YBO+`#!qWp2<Hx-nDO zK>s6VsK%pmIY!r=iWwhop7<8{vhCHv#Je93&n|-;KVx$tQuz~@vO(xNx;yvdIjA5C zhAc!iJb$cUS5n;|m}5^3wA*l;l+|pLP{ni}U}>qoE3?G};QxA>%2~Z_gbSmGu1*XI zJ-fc~0yffIOg7_2;ai@eb}u|VkZwr==aJzc_F7$AVJY0d3HY=iz4#=W8p;n73F5BI zK|uFzzIxv~Rcz2<r~)0rL67J74@X~;*@V*&*GM$f=-b0ToUYGIV>4KGCv(ZfZcY?( zbpP&GZR#@i3$xJyl*pR8uE3-DpDqnWbS28XK?H02*$t!nqSPO&S8^+^dpME$cLN>S zZ%F1>&T6O02JC4FDLlJSY7HgwCOAC|BvYqfU@R^ZnN)|-w^CO3ij11G)weke!OgLT zyg{g#K~of7I_V|p9W`53_ID!H`=YfD8N8l43@Yj^bjYSwGDUbMg7Mq~P|^@yaD?~m zH9ehX{%nw@S-{s_8LO2r3loTU%>p%ne%CxO_;kkU(a&qEDL+y(p4ytxer=?r2*T$R z15iy)+AWf=|IyU)ucuXGiaUxvXR+~e%h&It7N&mX32K5#VMVX*zJ7>~?eldlJ!it* zx3T$jW$cyTyJ?%mf8u9K5~RCxRQ_T2Zqvh2%oxF?2b|^{BKTCdKfr&1sO`JqYEUcj zW7r>1kX?`v2_^A_o$@m|>I518@o?9tP(+}{k4UvW7uzh5+f5zk$2xq=wwv}Yo>Y`Q zSM-qL*7kVOMZtFaGxvPoIG?kbKB`UnF+QcLG-+v%WZC<77m=;<8AP8dzu<^odEJyB zwIxa{b9=UC(AhgVpNAsO6<td;vSf!}<eNo~q}Ql60p<Y*v`#oxaXKOt$y5=HK&~t~ zv<DO{yoCG6%y~Tx>H4m`f~DDBEf;O%Id;&c`q`02NcL3z%*}01!xcbsVdQvj5O)w_ zoOhKzc_TS1za+o%P{&_9Tmy~)8XdZ_C>kQLbMC-Y<LKiq-PW_I?FI<u5r7=3(}Vjn ze5Vl5Ha0$`r>ih%EC1+y_Ezlq=S#Q^{OVNp)?Za*V(dmpv!sbWT6dyLhQFXcW15Gg zr<NK(DneVV7Np6JauHnTtE?nGg-&+i8+tlnI~I?Y8YARvkIzi+h|rL~1KSDt2DGNb zfUa@GR<}Mf0oQsVE(UE}v4d<@cQY!!Vb&H9Yf>P%K@L*n53waLc`AY{;yQX*f4bsg zkP^g&t*r94d==aPlx_cbu{Uin7I}}R(nKY&@%Ppd_YjKpZ3c;KZZJ~C|6Qo(i!V)y zHeuBPep*rNFJtNMb?fyz0^KaH9O;DH`y?ASr>Jc6sb8$k&JUl<u0|Xt9ds~z_Kei4 zUt1YI_!i~cb0YKAgNp2)k+@}{&b)qH#GJChSe2XkzWpALO+PX=CuP*=dMa4HtqgmS z7=AOKrzrR+zK@S^4d@Jl0S!U8d=g<V_BiXXA#MK$+YOvPX^^`4C{(WVdRCoF1v5zP zQLJK|nY&KeK4h>~uD;{%?~ak57<``uhrkUerkw^CSBVsP<TC>}eW3TX5^l!o55-Lb zk^!*p^mau0bYp5KKKX}9cC0uQ;KPVuldMCSvGyPs6{5#gL)Js~G-^wd+pWC8Ukkmd zQ9;8~aQXI06+u7L2cFg=U`MgTz5gi%y8=Ig-68%1>sO0=SERz|XC`{T7!Vnepb55J zPpU*yGBBk9p*iT=rSi|C_S&l5K(hkX?D0wq@FQXGsQ!3D#B=+2^n{Cw>d2QCGnU?9 z4@22|BtmbfB8BiwFmkZlTYIzNrNs-EYKA-AGivtP@AJ{!=8^3GVyr`GITxoi_#=tI zxIaBzcQ1$H6+V15y~aElx$7nC1i95C>%5(8lR+(haI@0(nNTUtA3KiO@qRZoV8D+` zh5%HxG7h^FjR=#^#LLs<J_5D8VW~XtavdKAxVF1%u92Ph7pc~bi^1ndX4Btypq??$ zw0|bvIWT)*$K`^9)73G_W+hHf^qdd7Hj(w-oKWzc@_byehcv5G{*>uxw%<DI$??7_ zJwF|0=EYf^(M1Qnx5!HmqTd|(v{2h@G3sBEUtLi><-eqQQ{1FdU?H$j2pkU@|J2<W z5a=IxY^Ct%;E9FB14j(%Qap}tA$d0qh1Z!^AY=b>H7klodX4sNR&qam;lGK}{y%Zr zzy4eLEqzP*KLEv@E`Qf7MhA@N3RAT&rTK~s(aaoH?``fJzJr87H75FU^KmA)w?Hi# z4JrtKdPJ7nM;Y#aurvrfpv|^ND3-nbxQtRjgux~hJMrUReo9@tVTRwd7{jS?@HnOx zBRq+EH19FjcQ{A*=H`IQ8p7eNW^VyY1v9-GB`E{4pMr$r|HY5J;eVIc1<46d;d-YV z=eeOX_F%M#gGdXQ1nANeOV@wQ<zHZzSiX4zg?9R+Y5CxE!4yHmkW^0|BAt~Id5<+> zme^}_V$bK0Y@2+iOFrL<4o>`rP>%*iK(KX6H0a|XWd`KKA-LNBk+3cK%Zg~D&=rzr zk-2zQpX}5=qx{k&pn=)x4@LGgPS|g{ddiK$35>WAJT>SRy*Ct@4T2StK`ZN-Gmc`R zli<2bbShA_qZA;|a{x*WHA%u9dC5brY}36(nU;nU1-F3Gd-aYU=OHCW4OAFMhCY`K zu1~|l9#AS<PTY9sUc}YRc1a{GI?w^yJyQphZFAxmN@RwFL6{@$M*+pu(~OXg>%Rwd zIcsY^H{#^MzqS}(!5zlh4siuv3O%8r*F{S1mcraKIU@4~_jml^$C~Rc*?Img^95rg zM$t_x0}NlIb}!Guk#M0y2|<5~X;&`^6MX>-bmlRA+d7bCgwbroHE?ZFu$YPsWli)( z1u*X82CcuPtbz%%$UtiTQ(u1`ab*#h*GkoSas?#yv0On*^0(ADqoCfEvo{Hz4mpW9 zkRhyJu3vs2`P5luZ!f9Z+AYe*q@*nOJXK+#I?+3@zd;vx^X##tr|@ymYr!{y(Jshs zfu9|O$W$y4ogzAm3}-8}&(Vr&0uKLrzhH*l`c`b3Z+$jv(O%oe=JV`$^E!^y__gRk zjYUM(l&W?^PHcI7Vfkr**nD`WUnL=2sQow<UR0G|T~)obduSZB_)YL(>fE&7r|u7* zm_h#MH63H3w2T+{Ee+PseKzgVpH$lLy_F4vbWk1ga*+uHz*?E?6S;`*!Zjn}uPsr> z5RKPsh~Yqg-R4?HeOCyFiyzqr5F_9l_$SO#PTGD5^S@qqu++7s-mQHMCQ(6(lxzf1 z?+`G2D1EzR941%qFb+Q==|=FKL@3I$TU!g=!D!wRKB6<*JaJ6NN*CZD7Twl;42Aua z4@y_=;!51lugYzuMVMmZ#l6?)=ge4SpWMPn^iiF)+Wa{2YR0U2c=$jdS#uUSE)-0N z!J-cDopfY?VQ3P=u$I(Lxy|P>WU#W+SJrb<5o&FF%?^@cq%+}i*keO${kx#Vw-?_I zoexjD!d2T7n!km!9*CV59+UiqYINkfgb)LO55{U{6}F_RXjrPkkL!E<1K_QjKWU|e zt=h`)Ef~H>gZA^xzr1siSu1gSAJ}8FJd1)Z_fJ#^V*C16$U^VqU@v=JKrll|A<!@` zLOJ61e~N1uD6WGx52a?9g(~#yI}1|@lUVmWbou1i>mhiUj_)LLD@4O<cuS}0pHdt0 ziY4w}&en*mv-9W$3FJIBeeFq?@ru}*e20FS@R1lG&XXJl-X&=UEW2oOdWFw#>;WYi z+-w;T7Ss&vHr8a&e8;pJPG|Y3e}B5I?+hwq)S@7vl)y45zK2!~l=f^G5%k9FD`?3t zcQmxvXIugf*+V=eqgaU8F8PFzrX7@Yl<K@}Npel?qVJBnf<+0BEGJ~WOaH@>MQ2v9 zz6N1@-peQtROED-Vtn1L^nb)g%Z@E{-)5_C>`E|x#71t5EJCDVZAkw1_=@EpTZkcm zdZ1)-Vl7%O*AS6fDuV2``unFB7!|6uM=A_YbPkq0-&y66F+yCav%Xehoj3U)qeRtw zSB*-qN51v5*~gmN<`;aF0%f1MWzNf{QTCdIySg1Nu^MMOD@PoE`}je=b-v!47j9P* z)<qN?a8z8r4@PVmu;;mU{9r*aAI-HA$^f-<n*umfD%e|iF(Q4wY8bwW%+=GqgJ&W5 zs#q(ij7`!XV{h8c+|de_&=!=vT>qnLB-uslxm{c18~Jqukkm5(!n{n{`?C=Zh1_jI z1L$av6Qo`)8Bm=fadH$f#qGE~L#w(}sHzUCJH;o>cP5tXO1yVqgtFM?!s42;S#81_ zm%`%huWoABak~<vK|1fCwv%9IsAS;e>3+Awnd0^PdsA=x^+mA*4N+M1-jaH^w#e)6 z>$8++&j=%Y*TJf+x>TZn?Yvv|ZgV?|e_ja!KcV0Bj8CbOVO6CQ4^k~*1pIS)7Xk=< z+K=M4Le28b97d2HBo{73;zodf7tF5NgJZy@2jW(9P%1bW)@XPoHlP=lQo0?A<M|*I z)G(Yxbrgy=xaO&o#Cp#DYlSutW!4*oA`^hZo~y=75|r}HrgzfhFy-i-3q|qw%<)?> zIc_6O@(LK6ZVg#JF|{fiwi9E}Q@WX-ogZ9}G3XjK?cMrvXx+vjGS7+qQ1iH%2jXuq z`_)9?%BPhH&x5j#94%4D4Nj}W)v<8#V-b?){2B4_8ymFh3>4FI2GMCi-_3*wNQlK- zW;D}xwjty@?bLh*O96H?)`V<|$Q%`nW->MEuIQw+bw`;5fX!+jxB>B9p!uEdBrtm< zvW;NxL8z{^mK+3=(l_EG<Wr`UAV*#uD>{keVxk7o{i#SPOk&WGXfNxg&gk5#b7O`V zy)f{>iq4RG8pX3zMcH4eRD+~qIa4D>%5CO`dW$_shP!3FALlKNm1=HCB|l<cmCdbk zE;Pt@^vO@SNJ=64d)3uGuR^KmW^}%Jk!ZF&Sd-Z;wCp;eRCc#>wCXm|El%5~o@Mnk zB5uJ8H4vo?DG22x%pcPX!Y+LK$tao~_7t=nCDf60B_KP#lSP5T-}-qIov3{@MG<nO z39)l&S{Wx70ZwiB4uD;!c8RUuj+$)xwCJlBSGPot<7^rnJ3G9W>(fF=15<fbM%#|? z8ru;YfPRvYMc;vlfPom-ZyoZ#;QIf`;=F=NYFTA)5o`-EaRw^TYrD!P9!u?*GA<Uz zyl#OpxHjis(NSC*Iiz#3`=HiHtPq1H<<3`U+sNE2_@zX)w(80ombZSy`)e5H`xLd) zFYUYA%B(@as<ZO%O-sRgZ_k}`uAM7YRB*Pe8A;XNr$UX3@IU{@xrjsLw5S&~x+z(w zPb#P=Tc1mFl(Ve;l5B;WcXvJ8?q%ik`=2)Yc|KR<y3RH#d&Jy2cYjEiWDRs8ijSfw zfDQd0j9b+wt*@6p3b|1eVc{dN#t|Z~-n^L@uYWu;G&FqMr@+AT{hFIorYtq%<1Z?E z%V=#lrm-IgFS<axdzZ?Y7AY%o!eI+P=v)*}D7eX6%RHh*QxiYNU&h~csxftzk-tJv z2FhyhG_T((|2VTZK7=6HLN^38MwYem7fCxzo@Nvwcrd<<prAWZIjCm4w}FmYN2q{0 zh%m{b*+T8SH<#|Xw``Q}fGLXL&0BE`&o`X-(+f5HRzy(tow#^anH=YHt?<no3Iuz? z3Qyd3R3pEBvE%Uf$_Jto=?9<2{2w`qrZU9W$MNjj*L4O`Z{IRDpVQd->*JgR|AQO9 zi~eaY!^ylJ7PT8TZpyQ@pg*_Z9_AAS*9MJ<M(Q0{?TqtO5F=^vF6#W{ZNWkdAZA8x z4{<rwci~{#UmkVeCb5tB)29KC_dB>0A*@vY(r!>ig5k~x86%NVT<q~LX4A}8f!jcw zk&6Shz~|ueNldFCVr1Pr3y=95*O%y}Tiiby_an$9?cC#5d4s9g!p6dSGZgj)>YZrW zg3)&LLeSBs2sq|@O@w4I5c>}y@AdVU|8615CS;comv^hQ_>eAy)P^VI*9*XClqbyg z#0&V4IJJ{8p^2d}YL0P{|6NsP@Rfi9pEm@n#iLPcpm_nDzuMXV@g!TLD$cm(rtZpY zZFn<bi>+&&)f5$OD}IPY5trb<@Kr%YW+o`oDXeqWz1h;_c9OiPtz6>lwKTyz?nEl> zOK|cd&J|tGTN<vyIlUmZEQmb^$T~%cLjdZ#{*Q_carx76;c?u1$v*LO_`Y)`x!4GC z`Zo-k-AB5s`^MF!8_|I3Tvcc)%IC~k86#M7{FGK3MNqlrZN7I?SBSt(Ab>ocYZqS= zhS~}@3QHj8m`O3Z@LNU5$U3YYG(PjP+&=Q2Z3qa$Vd9F}D7if!8y*=q{WWtvEAz<d zq_~@&sH3<Ki042>)i3m5hJO&nLsl$#jff%zUmiVkmh$ogP8(y)Qm@KheN9lpn#6^$ zO*T%|*PnN-T)KWO+R88z5gJ6`)e29O8$H*Ge1jS?Cq9RB4|++oL1!|TGWAq$Vs`KG z8!)G_{^=&^l}3PrR19K|RAR3}HzPS4XR=}>yU-h#1=jQFw#ppB#z^i~YOxOw(TS2; zE()U0ERU1!bp8G#x+)_2qJ1{!oxL|q(`02@7$<puh6-z#<|3nnC!sCxQc$-L)R+kP zfw2Qko^wvYKo<xm+xd#LRnJUlo+kBYtReQ@pHtKx{OwiPBA9YX--NZHHS+qot-K62 ze#M9#S^UE6Bi5ULZM3Hg)#&r`$)~?|ZQ_k_-FXt1X&L76Uon|i!Qm^r6nevVNHb3V zNQv{2%nO<2e^wlPlX$x3xbrTLyLps5d6)D4oXE_}4cEE<JJDaRU&gPGa!|s3(O=-y z?S6I8)!FoFeqK478CBh1O0&VY>o&>DO4>`|;)nE0O;PdZK)Dc3-UgCI3sQlcmYgWx zowI8JwFi5c@7RfwqwU8?<bDYt4p7uJ8P|UpWr=4g+;<A$OuQh!Yp%EMQIu#G&yfqH zvG{i9OX3P}F~Gi4{lz5`4O|au!$qF-3YR6n6CyHbh$g;nSAWi&x+8nKleDp`R5?d@ zRJ_)|8H#u2ag6Hv{>F#><?mGx{%eN7;>1{!iP8cSeXs2ZDI9))Cut1dQC6|svb!R* zZh=Wk?ntG+8j7=nhyO}ziv6|nxlP5!{EG(ur7F_MZTA>sRUX%>l6|Y3J?4lq^(|Ij zmy5qmWN$HBbTEBf(;ZtgmKp5l7UTWgljNxATxD7N;%SISt*h77QuE*?N1Q$Pu*MkE z>9||$@@SQnb1_jh1aGD~&9!+d8JccJaliA`SX7bH+abY8&@q~n0oNx@Rb+$T11e5= zM$Ww;9iDm$1+>`N2qm0_Xcv@>2A~^H3ui!2>JSul=fLmR7zU}gN%}|29&L&f{|ePI z&W~UZv%x4q)xS0NfSFBY5dsYu-)QuK_*E2(8~qjef;vcfvj%#4B==#IN#&jE6rnej zxoL|DTwQWevRUK+_LDOp6y{qJedYl{&TKVFq9m>{%Ng$#6wv8nqAi+(`~|Y`eZj9z z29x7Jj7K$;!FTBfwgDU|?kKt96E=b`ow^=BG#YqeZGQJyw04H;jpfusWm%mUpDoMl z9@lec3_PjQTD%yu{nQ~;V3Stf!t9H1wmv&-Y_`hZGCg|NT!nR>^)pPSj+#tHept&; zUZ_u%Ww+@(V`i~;uc7H&25v$<s6Zn{!z+)t!~fILAkmq=)U%=}JOT9x$l@$ete}}k z*wS~nZ5ueSjA=O#X&6b{JJhwbxN{P9V?c+*g4V5qmSA!b*Lf#v7N1Yr7AkyP5-gfE z{6#Vl$D!WF=ZKWSh&iyPo8!6Tb(q$6xFysz(?5>x`Qh7catGfS+Acp4z*bpF<M#?~ z@VW`SuM8$Q{So1?yOIq9>v9`AP)eWMT;tg;t@n&YNOmCHvY2ra*_bVunlyBrD%~P( z#e*^|8NZEw5v=q(tAZ<h1Q+b)oA>bPD-$f>b0w%tU=f5}ZF_mn3|3*1Sw$3U(xnHl zg8MbXe|K4M6{Z|WLAM){nkHP~j>G0QHhkKMro)4l(#X`Fp%l2~4VK<qbn+D=WtsOe z6`56OT^`OmtE;#NFZ}+w+4#_1P#94A{nwr=RP@_EZB}SAp@PWVJ<gWt*_FXty~-Cn z33TgRH)R7KuQ2CIj<15Zw%g+o|08w6M0>20&>S~4y$K-L8$`$WnK|`XH5z2acZS`f z?`@LLbRj|x)2~tL<xz@mqi~gWjzI>e9MkXB%y)PdT-g&-F+Itg4*GPg8EHD{{|Gnn zloEbIdH=!TDxX7-LuDL(f2{AB>>qQN{V8PcV2xX)_U_>_OIM4%I^RQNhVob*k88Y$ zdMDf;hk%zpxXZG9qX*s+BGcjIte9D$Ras=O<bHhrze4CxGT`t!z23jGOQ;Efvn=QK z>g<q23pdm<b_;O{mNK@xl{Ov8o@N&JSWCAnJU#bJ^KG%i2iwr6%MH%&>NeAp=Sm)K z&AK^`QaiUWFDL-d%0tmsC_ws0D1-dbrQLVBS-R_bmJj<aJ{zXIma`AEDg`37C)PXn zzi8%Fjd5kzZ!TOmb-c>zmplKk?PW#G{N;~?+F7WG+VuE*utYDhcB+v6*Xmo|s$w={ zh{CR{GU4_3;sXLk2o#(w51)$Dz*uwyN%2xjcI2=S8!@5BdHCeD4E(w(DiwRA%0H|{ zr(_F$Hd8e!zsW`N?nFz(LGnBKK~VJl4=UdOeSa?ho&f2mzDDiE#)`AC=$pI;NDE+N zfvtRh3Y0=M;^MNjz^7!K8Fs^PPSzS?>%?8@*dPPmIyK!^?Rvee6H#Q}n$<m~b%Xwz zo>?j@E|Y8*8A8jvIZ;H=7BHz5u}yTG*YyU&j2&!quk>H~;D@So>zMFvwlIpivB8G> znWS^)ScAh`gm2yV&rlTWs&I$o6J7;zah#(X3Y@22(6XRQobH!8zLq1ult8>^1OVzt z@_3K^_TZFHVJL+*BoV_oHAFfvu_KwE@MSCgYPnP6A_lGc<YLI(Vv4OVW7eE1+0^tk z;^6nG{q4dlW>?Spma5?`?ErYHLaw+XmH|HR&;4otGHs6dWRZz`ej%|cVsy8EbzzNv zfAEvVmaKY}W)*?2dy~b{_~!V-P4PYrCqCz>)oIo#|91SqQ}<(0&%d`n1y6$+KrMa! zhTG6W7$jt)R<!vW-CK%1Ux89MZn$Xze}|?$QmNh5lTE)2+Y`>rxz%=zr0IP5MZ-XB zNf5Hh$~8#kfPD`-3ovo0aSE0U81CylP}<>nOPwo)9OQ~uLl{DtRDYoJ@DluxI2Wgb zNuC=v{`mdcd3?uSj{MF~M;GOu-bt_9DYhtoIi+!UG_yonJaJ7lGnD<x-x>L`rPFEL zC;ch?6%L&@<S|qp)pZ^2CnnIN(BpW)pD%~k&`E^dYjWsr32eKqAFrZw_Yd+d+!4t5 z+{ktw)IYPDV<xufq^TV2Z+dg1s%u+=MZRIWh41jlS6}XCp}OQ9onq8nL|4b{fLwZ1 zyYG05j5zoMs|<o-n~L?3H|k@zae9!)=blcSOsikuW2dT!f}GY|S8M~|Gbb2=YX&hw z6Ui_L+rr;6%>EnNEWEBJ4(+_}SMxRG8PAt{6_|k}yIid~S1uY{=VvEzu8UV!$!GO) zvlG^UJRiyI`CdkUFE=&K%A5FaSY9g<oyR7Tzf~x;G%lflY3n%=K%v5Ji2_u}CktZ1 zR*PWk-kKkp=jCK$Lx3qgGRSZM(SX((ZfXrc?+iRwTw}Hz<?Ltt8>PU{sQkqr@9!H{ zzQTp!+fNzd4of-!hFylAsex^_b>t-lP{;6>37bWGpk!VJ`*R6<ZFF5;8DrChJAk_L zjV=vPm91%gpXoIam2K4gD4r!;K4*k3iTbSK)f9J)?zTwad4O{OK5U3J5?%pW`~;ya zRQq{J?x_4OQ82)pU@Oy4mwb0^1z4YCL}NqIw+@A4l1n<4Z7wman`gh5p<J@s-M?L~ zDi<9F(LzRgl$*M*@ycy7wa9Wz-Ki*N218=aE2xO63JO?;tMN7RqRU$vTT3U-Hd9)j zx249MG*jr`l)hAN`FzhAxM^NEOJD3pQO&Q_8~Ud2QL3MOUPr$+E5FVQgi{Q8J`V86 z3kB1ia$Hr{4~^vU^KdUyYgEtX=_d}<Ai>~Doh)s;C^zLr&!V%}ot*r-Ju>N!`V~aG zKHx9sfO!q-_+S(>hb^5VfnlV(rgy`&B@=YYY8U>V<X{tzoQ%`K4F^dzH9{||MTbU3 z-Xvrcaq|4M$RE0G&YlSBhfNsbR8Q{H)2e-P^Pfpc-Uh3ws`K1c;mtQ}y8w323K7`6 zMF+*H_=~G|;R<}$Hn<G*l|OMhQ5?Xw=nRzaJTSO4VD`NY$qsM&HdbHpD&1B7b)m;e zWl=%%AJjD`G6v0{=|59NrLC!}bS7N<G%@wGgdBd|`c?Xr-<vsRV20}n8BLYL&Kn(d zCKnj`$LGM9^I3pi{D2(!VCFop=k;{b7ki;aAeeo`sFV!gH`)fzsLS?xWv~i+qU5`+ z>DgRtq)BpIKpvGJ^ymf|liGQ8=|@N>uS%SAFPf|QqRD9cvo`zf-c}=}GLCYoakXAq z`U=#Bb7>)v)!TgieC55p()U@*za6r?;asZjmA$Ji&tND&6m#AIxWV^ftI|j-EZKYm zE6dT?s;nN9NPTy~hjaB)l<f|t6m#2CPh6Cpcg4=iI^L0VO@sL{_h4%9Ly+f8YIl^T z05wX1Wguj1!}-*i=`O;@F%B2-LWH|5qqliqJExR^TU4}xW3|329X2LWf>JA8IXT-- z_o)Vjf9t&wv+tV!C%-ovaakA(kSefxZ^{G&CQYZ(_u`bG)t+hj&zBnctG7jJe8mop zgM9p*Xm;_+7beuo@Yi{6x0>?Vhx7NPS9f;%em^bOg_ZwPv<<2dT3`Y=-?&*E6yVbY zKH_qGRFp-#hBk$_KDP3R=NSlM0<ZGpUpG&GZ0RnWq3;5>Q)0<4Sxplq)=WYPYWK8d zJFG}iO%ta>wqJViMN?qN3ta(<eI{9O1599+gIYTPoVSKZ1?UzieXqc76V%|cVa)hD zh%2!1inCxh&J6H|@<E{HWTACW@;rT5B*WI`3y2jrnCW2&k{XR%W^~=9=!)i%x+_QZ zJ$q&!c?Y4;K97n1?$@FZmt1iTdt<TZYuizjL;VsWbh8h}&*SmpMFUpp5!sIkhubgj zS@&?y)IXtgk)5ZOrSDZ6W3G2i|0A<$KXU59*)C^_bxcJH$t`C+!ya#k*bUz=nU>gs z(us^AcKibabl%kE5Zloo>92nvy3px#DO)Gb4ScA&>S_-{qs=IhwJIZ$c?l+Zn{17A z7r7KM7zx`;O^1xIzSZ_o;ZGt<10~f1_|Rh5f~HI9kL5E%PvdMh4^j*=)STAUjf^?N zimtSxDJ$cZ-I9X?Xdktd-2bP&?~ZCR@At&QC;~=6ihyL4rh=f-1TlFnh=737dqxpy z#^?Zo5E2#XB_j&VfDkDG5kjN|K}bZ}2!nKxL`XqJdZHK}NSXcl?*6f7_n!Cuv1jky zd(QqP66*8t{J!PWjvgzizJGH5x!NY$&X(@W`!X{29&mr0ov~aI0-Ohy7jiu-&yh*z z+(WH!z0aS`1Z?k7J?&-fJQjR>H2Z<x2mj%ygCk_MM!t<ueKc73XM2-nyG@JP;n=hT z<iGysosJw#84Rq`+j#U;>u~N=z`#Fv6$z{Ifn5mwiKso;OypVKTJq{bp|8)_%IBZg z&2KZ%q`!u#2)9H+7>CX#6iH2d6z8|v2z{_;hH6$ZX`sr&QmuNHP?XS?hCpHX8@jBu zV|ge-Mf^KWbNT|SH6w&xPe_z##FtDy2*&`+kdE!yb|hmysK|j&KY-kOE!uXF*kO${ zTmdxs-)sP&#q>$cs5={J{C<H$Nxfd`oU9ssq!km6=R0t7XatUq7fH#E(`MZ5dgpp> z=ftWR8jTJ@V}gS*(j-!ug0ByQX``t!K0bJ^_7n!Q)m}wSQOErUstNE+X{@bMpu5i~ z?_zW&EEYc#ZUfMJuwwt%ABh-9K60Z_{rBc7d-^Lluc8Vp-buhQfmLa?`x449{=om5 zvHoA_|DLmXqTLjME1$Sz$y0S0$8GnU^RWO?ffmrLEvURfD21@E@sBRx({Qqlqlb+p znlbqj%aR!EmKnn4`RPMETuZro(g~!R?3$g<i@UHv#-Y+50}tHr>4@Wv{4ujHT?e{v zJ-*spsx#t0GE~YR5cmK=q_)r<tH>jCJIlNeE83gqDH$1TMYCawrv<oUkSqVh8=stk zWCX}5;)8s~OMN7$qhs0_ScjVn4r*td7Z^lH#rdY_;)@N~M5b!{9H<G)VzQ@Ta8TRP zKyX}Ki}mCfgB>)ryg;J>1p~89X(Q|fbb`%Vy2(dw7^QNMyN?c8EAuxmnBOGvcugr7 zox%y1-^BCVf8KR!>*np2TeVJYjZ&kG9IJj~nWSQ7>R!A&>Q9=UBDJj7B+ptF{p2Ng zp>lrC*$cs6TG&f``~5Vau+C}S{1^9rlY`5M=~TFL+pt+CqQ_&n@r3tc6p%kO+!s2J z8rGDC0C$q>HKj4jOIK5;0xXpuj-|Z-I;O%{5OQ^fgL)7ZZ`)0*o#-@9CB7gKqGX#J z9p2-E$+nKDdlH?opevv8nz+7c6OcJ6L)L!PKkFB|fgn`Ti@MB0h~bnu`TxE)E!yF9 z;$;4}jGQN^{LuW<Wc%ok1!-~Qk?+puTJwWHJBy4#JPLe~BP1|D?(iT!oREAQX-L#S zUcp<Og9<T$yS@gFS_cVrM2>tVe}Dg48QmVyz9J*O;nHWz#=*e@De(t|NC^X3uex*+ zQBfw+fSNes=<cF__Pq503GO_2H%i(;y*MAPF51hVN-E%Db{spA$ciU*DAf@U+3mhY zE1~2AO%m+=njl#Gj6H(n+!1N<h?$LKsbn4{%R^&89sp@>^Y7-O+0JP+rB}6l_<f^s z+iQxUKBUs+kB_{CR-k1~_GH+^=NE{|b0)-XdzwDbVofBv7EY*s`8Gd|MgYJaFxH;4 z&%BbVlkJvfmg`Tuo|aZ;_u{g3&lcl*uiqZuYw+zjZUEjCz4*muVDYW+qBm|hU{n1* zaJ(-8_sNP>ZoTU)SRfb(UPAGuM1{~bY1iH+j2aZz0~2y)QRh_SLqh3b51KqJHM)q^ z7lvaEp#soxQkR$I&6%CZJ&TKGtjWGk^_<SMhx))@GK{d<IiAhglKS(}4(<uM+<e75 zS-l*ksso7n?Z8Oec&D)U1N~rCPe5v;zNq;79lL;xoG1oT_1(_lkt(~cXfp{+0#I`m zoD^<?u0=D$T0pPZ@hG<KlJ`s%Dh;(qq6{r?sFl%t5E;J9l#omf*fB!Esfc_AMd9Js zt&MN#r5CquI5j?H#b>GA_=$K&^HD8jaQ?5Ms5Xtxe5XPR6zw@;wj}G>kZ^H0imW1; za2t+tkbSJ^8!*Sn%&0pP7pil;I=gc*t4#TdffEIGE>~4Hv7N&$PHG>?Jambb{a7*c zxZd)I(DnnT%-1igwa3-k1^bjOcvhuUlEapI(FPHY)K3=5A>ZCP_Wo*T`L@~qE4mI; ztMmUHDGic<*cc3aMN&rzdx0Oi#r=$a#naZl+^H=cNuGtlF)Bx^!*;C%JNRbbz50Us zTgHRy-n@!Vq*TnZHu9b*g1xfmAKKZj7Kzr5B~aSF-6pbkt{w|jt>2{nK|w>4<F}+^ zvAekx7ZB9X*dXM3eEx)OTPl9<nT?HsSiA)R8al~`%~W(1hJ*m*YY%keTPJNl?R;0) zBL8(@)5t1Xr6-n3<~rJXynHdT+NU;982>CRcl_p0`H`**QR_>*iZ`-gZ>A@13>-JQ zJNmcOWg?`H@BFg@tHwGdCLu9+xt_iKnfZ0g5+)SPyKV6P#<kA|;UHX|)OyyxB2mwW z{Z}ook5^ANup*qdT`v=?A7*7!m-aT_-Vr0ZaG-l$)|)ouE1K@>7c>j>MT!$3Iy?tt zpu?5$VHk#OM<0j6pbMCMr%b*tH~N8Y-cmFO@Y)k%WjP2|kt>rV4EFcjJJxWvL2A(W zo_(2m{OAmk^RMMzpR+u~jwNZLUXtC|lX<s1>v?l#9CeC`JpVx3X;+1G-*%ns#q2s1 zH~^KU0j=);%%=Z8`SUfSJB^d4w?jZSr3wbUJPz=wgu_|^mn`rN!jy+kHQYD43Ri@J zQMCn*<EH*TZ#ee;-hN&m95pKUbbLDh%XX0zgqQ}qAE0iJ;;jG!ml;Zm8AVy7#&1w> z8xSeg3sU$tADSj;Mm$t9+2?O57x6=y!9w1)&(J3}Sw=QQb(qRhgKVACK7m*S`_U!h z#t}+BGMqSR%kf+)<(YH=vF2fB*o#oB&W+#{YyBB#S=3%tF6<2<6`}gd-}g02g{Bsh zf1H7m6`dY5gV2=7g+DtE9(@}ct=ykDHdQU2#97s#QZi#>dqR84O2=Xa@$j6XE<(FY z59ial^6cqj3wt7Qh3fOIivv4LXfVJB&YRJHLr5Evbg-Is=exlhZWok>#rCvgveeCU ztxxiL1k;c&r!fVEJ<U^o_^_B)(H+0P*H?RHV2eyZ{Gqse<S(r2y#7TchM0pQacWI( zMP3egus%1i;`}_k)Zeo|^cpTYrDOcfc;OEI$0vx-Ev6e4LqF(_9-seR7Nc8~qw$6D z)rBRv&q%ddytZ3IQHSerqk}{I8KB0z@z3H~ePbr3W=SV?>S~tF)u=1g+jZRM53p`B z1HVOsJ8(Y)<Z9m_KZE)N<^U%!5h(Ec#UD0EpDGsrmU<RihuzNCsxnaxZAA->V#a$e z@r<f8OdqY+&&4_=S2Zy-d#cU?!bhACBbg#LHHnl60zasOuL=yc{duSv^G77e)&n>< z!WrRg{Z!A_#^oYszFFo#GEOx>1V|kSgK(uO=44ZbQDi};T`QbR*{w`fRRZ}@-jnUt zFK}Ny`{9kq#W>+{977^Cj8nml!sQR+3}UvN;rr)hnGVRYAVN1{Z+#Qrqky!aemLpH zw-}2~tH=r0E)taBovIjmHAQv}V;tiVU<HL9az|7cSB-~$on)xS`=NDSHK|o179wi0 z_9Mm%x_X$yjuHL=xBYihR6jm?M0f}R!SYtn5+~fX))ij&0iJ(>VsaE3_L|<<)vzlR zx0|0%%1UGpFb))lu&w-C6>P#en6#v$J3fSb(QX(E3XCW&LmSMgAV<VFT?W3r1;w*H zY_-h1ID)+D+i&7h`!ea9^C%!CFf0V*M*7GK`KtGlarKQr=u->h>%n2m{SR?hW5$yA z)2~`<aMDrIG0q);4LvG4L-vjdv>9p8IPBTgvRuj~D6uBzZr6G<J}gLd$Mtf*b*jHQ zC5MY&`dIHjW!Dn-2dVJm!&2iUM2>MV;mM(1^>a6WMN?KP)aL1bV8&3MF>~xN_oyQm z3lH?g*xBg(E!8qgs(8dK&o|yDF2jeSlHql0cgdpgnW35mus}7NI?;}o$L)dO7n~J< zHE25@?te@lI8$Md@+ug`m|akf;5_LUz@@qEL7h;~s)!CMXZTN_`HJ}ChfzB4R}Aao zQkW;Wz=}W`xKsu?^cx6F#5t0m=W%ib$Kk6A4L+Vd1K^5{_{0#os8sc-<^A+JUern= z)=c*N;5o*8mkMd0@X(h&p_62osDZx$+w*K3i{!DoSxU$t*n5`K%M`y5pnr=7<eI>C zmKM!8=B87DbGiu4f7F4{9WEcce6gQ{Pl`QRSlvDTb#)3?=ut};k6&+Eys>_z@Nw>k z;igUym(tNYl8qKN+oL?!e$Jzg<omvCq8P@R2}t}1f4%HAnl>*_q~s})Sz{L)wjdV+ zh2{bCO$LWp5D4?!j#Zsm(VXYrzW3Q`me3kqHt#duGN4O-b+ncKr2cfZ-R1lW?f|6c zEK*wNR#gb|`~jE0#>hfhQ=QZz3&@+jSDW3_87`v&tJlfLbW$E%0^5Ls{q0}(CAeff z6_$BU?e-vgbWGa5Ka&&rDU-e*Q5ztdp2kkClkk6u+<?+OXn?uEroVxyK*3-@sE`<n zT%c3j?bSb2+=rn@cStd7f(9spH@xEh0+XVOs-0chH*@$(cf2nbBieT}+}uYxxigL& zN?wUNzjD3RF7#GtC_u$pQ9NfyPl3ZIGLUsY7hWq*`=cE94yVj}%S%oWnS=+&lforC z$FRsdw?b#&JFw@6%yRr)ZJ_2*?!eML?lZF-^|up2G@{$F07t{&Tgp5^dK`Q)2s8=X zpi6x14rx==eI)PVit;HyMHocvX`-GT1pJPKGdy6p?JBeuMKz_JAW8c8#wn|q9pOZn z=#=&C<(AHGkj|8K#9SYs>X>!z3CxjBFSR_*$$f!-`HWh_xxJ1i4G;G@_zZmcRkQ0{ z{hP4|t~IOqb{0AfF4Q&OlPhg2QNj_xGw*_5`H#{o|KI*T*a@Krt`{YPzm80_K<<zj zOook@^m&e8FZ0&8stMn<IjmH`kCf;^cgl!wkjk{(G2PpXKo;UR`&T)PX^zTyqhE?< z9Y@KOhAQTQ2C;8I3L5~Cc_+B~VA2diGGjvXQC<B3;h6?({E|ZHT<TMis<^hH&3roU z_ZpE=NpJ$EX-Qgz5VoeJ4aG$tDTr?yCvy%)5n+aPbF7x{A6c3RX_2nDjJ78xkG;|j z#C%`Cx{S6rrf$3TSSzyna3@-MD)TST4aBSBc?Om@4Fz}k+^)og$C2hY&|NhZB-Z4q z_=+i}fBWvYUp`IdU|T3BE&A<m&n7S&emG04O@-W9Px7}tLVD}^p%Ru94jg*&<2|Q> zE59Zx^Ud3k-4sY41Wuy_4}sM)0z_%|;s)5pOy^7W>Umj+onFZY1K9)2E39)3?v8rX zTjV_ODrofOp_=T&G^>Pv(A~$suhd5_**cG#3itZj(d)-d$u94hmqyG^TFuKYAuo-r zTy`ipv7o^W4q0CO)Rb+xKiB-~LmiKEhfYxjd0vYMM0VBbXh{e?&A1-*XmT~UJ@AM- z&)xqJuPM|x$)N?GKNV%udw`+^&44`-dv<Y>?Jd#4V))AhHcUpZwBk#yXKWg3UY(MH zwXNz!gGVCf#G8%1h2vE-cr}!h+-)laAt=6by5(_WL(aV|H7*&hkIPEc=9#zc?$qZV z{qc6Qy^l@~6L(&qZtyLDDaaMK(99o23y({>T8zQ^Qfm6b&UC)BfD7e46CIq0LYpaD zEU04^S%5{arOiJ{Y8W;p22uHlQjYxjvVF_dXzQ^pZXp04^T;^K#DW@nKPubpbiNd1 zEVO^-`Jp;WpMMKljjnk;C)ihVuulfxa70dDIwhkq(emBNQ6H4i4d0r6e?-!BJ6Ydl zSNxMlctk)rQvX^@>8ewjXyTRl86Hi%*Skh<U@dwE&rqc17W=EQ-r{7Mb{JjLIP#T> z+|c|+(;|O{ur&tL4GUHoG%Dte#Jmw47ILUBflY|;47;Mo`2oHm_^v4{4Y-*^mjIj* zQ5g|eQBEl{8&y@mT~F#4paVOKw1>=2gqXo(pqlQLBlGG`mEpmaUQXq*eO|iK(r+5A zvTr$dhX;7;@9}MMv0w1@_eWKll)5N<4z-8(tCUeD8>Wbl>c5`60tUJZ_x}ab3wwu$ zbovg0i-i+yPMt)#)ML^Zm{hM_e__K(A~f`{9QyLt{4L!9k)B7(N`Bv0(}GEai^1v@ zx)CUKX)J-ExLUGL@(y=0#uR=JPu<oDfRRY?W9x{B-GU;`5t>zu#5;;Ypx5stS6X+E zoacLxxWagc9-KnzI8zvfjpCj(A9Ri@F^q;k8yiTxUA)8DZqQya;FcX~*nZf5<N`~v zfNVv)Rv(f6^-&vT>O|LiIXJ;0)ugc+!g#|+U+}f!til}rVUSA?JfhxPn;hctoXdak zG~~@zlpTM&l>Z)UIc$C1=!$RpwdRJ<rL&Y-aX!Aaii^B=+g7k7Zn56O*Y7wet`O=1 z@Y7mkJCoiA(>(T}k28=-r>><~8*`l#Ev!86mGw+o+i{E&=`fMw%=;*`w_GhuJ?1OZ z@ddsS3U@!c?mfQ02yWE*t+C(7NWtGXn_84g6`y#&`W1W99ul3nk8QYGp+9us)x!K( zUZwCYIQBm$e*tax|M2SmumA7Xh%@q?$Cw{%G<nV%9o9<w`{V|DNjhm>AC$Ar&;8PS zYOCV*F5Ee+gE#@$^$%EVRCI#5{JKPQ#QJ1ZbX-jg6X=4c$>#TfdRkT0QxP?vpA3x1 zVy-Wv1U^tgglUtLAsT%{0-tYkg;PNWBJEWR-xOc7>M?p$`Nf#FN20BLSuEu(Ie)ou z<S6BImfn*}&+_M|{o$XB<^<txxJ~nT-X-GX%p3Yn9Ehaio&mlE$WHpb098QLNdW$I zZ$T@72DAH#)2%JEroq+|#b59+-B-8x3T7)RtWYat(s!b~|GY$<|5GX1y94Y~PT%Z! z0u=p0ER~q_u3RJ)h|<dEfXz%*t1B8&%tmeTM_Hr*AHY43tRFSA8fZy$oAM8UQHNum z?)1>LTkB}j_(|tZ^|B^&zQ0idy23}Fif{MA%CIqM#0F21B8Ib)ZoQY|e=UZ79j+k? zOzv!<B%6ZN4aF}!+yJXSuM3kByYnH3uHet?izqo0!M}vtgPLx?vETQ-wa(Tih4q8- z17lHZ>rRJiLT#?xQb-dhezFUA7pnJpWd4!8fiEI>%$E~!!OWUk-@*CJq7qCnQ7Vj! zj*(~!{h+AcmJ^RJ*~A5BDX%vQL3%mJhrGolC;+-<?X`4tv1E8-=kE+k7x4kZ4Sku7 zXNJGt*sAC^jkSgnN+y->A<S&cdw!+K@Jl+Q9OO?u9X7M1U{6Nh3g0$&Sh>*WQk9l> z$ap20yJ5Ov231u=GE6I6+(`X<U56?f^4uoez+*g?f8p!EefS=%wD1g&-^TOnIa$v( z==Tu1F~QD!+vMdN`&3#IK}3Pa%K7odtyz&2g}IFk&s_DKN42@Ve@i(vHxwwoVaFEY ze#N!#vSFiA@Vtw}yXukQXbvHPMSVx-BDc&@XKoNn!Myc51U%OTqzThPgm%-PR8$&E z4WaK;YqR_I_}nv6gGQ&}?COminO@R?Uc7os4)t^Tt(?}H!q)o@yARnPh4wzd&<(R6 zV`i^WK7(^A?Arg{ZUrduUkUO7P2Y0z%Nj^Z(+xo=j<#O<m4Ux6(juby+dA<nJ}=}v z&p>G4il~xD`x#27P{)~pC814gj`z=cZB2c`+1~ulepl8~L-u#fS+{*w*%u%9%^uEn zZ2FR0@yc@mr2*KItVH6bWvnG<K*4W8as;ZJ>9_^!fZqe5d9th&q$c8kKO;eqB2wlp zoSBNgMf%6<c)E9+xzA6(WePHbJx0i9oLZZbtPcvYxN$8%nmjLG@*1@j^8(MgDheAk zMZ$68(}zZM7`P!ea#C_)fv^*+e5|B7;Ua%Z(p2>iJy|^O>$J_xsx}y5(`j39Zv`#i zS32UixJ`f?H-$!V2|dKr*s%eE#2V|}S}*(2FGk3{)=>Uc=$LeIDX?TR9S#dwF!247 zk0=PIPT|7gosSPxy0#lvyGw$jr044M*sYVR$F;O5?%m}_g*#BaG>i3TqGi#;b>MpI zY@AMPdiNUzH26x%5@qCT86&jWzzzvmYHkzZK@NlrdioWqUmo!7u))IfxFLkhtHvT> zm(MwhbPcII2K74HJYOa6x!0*%&!B?tgS!=&LG#2NbapYSHknv9858Udie@(MosZwL zwFS3Osgt%7a62a1csZSiWNPLNMHZ@aRBdN9G<M~b=BNt_t0C7;iy`l8E2qZ3Tk5nn ziy0-ffIZa9a%XFS8?*P9m0*?JzC2Fd+*NXPxcLC7$n$QI*74wE`zK}|E!mjpem>CL zE=~8Y*}(_Oh&)&LWK{=q31JjMZ=fu$b2VH#TzLaM_(@s869|8YWTT#(zY&kaC%S&| zWSCy8duvo0ST!{_bh^(v&p}qZ;kO^WH;|lAMow-=CZifDOZ&K?UfZ$8JO{xEaq0I| z6+_Lbd5|2L+)=3XhNf2gO?`6f20g`a|FyOUrN<B)@s^<g;>Hl+Q&SzOpOuTsJ4u1= zXhm-{-TO>$sOn?pBZ;7(qD*+N&WQ_<H(1Po^VcPoUGYD71>(FFJ#Vv>q@vAGY8gZk zOhT_pwNpJiiVi}hysyhM9r${9BIR<nE*&KkuBFfStF<vBH>Frx>SUVXx<xvW0}rIQ zbU1GnX+kI2{V>i-2Rs=8ujSG2i!?ZWsJc*fmDCE`I@u8A!m{R<chtRWZUkrhR=p}n zt*QB(|2Z@Wf3~Fcf`^GxM&;5iGk?3tFJP-WWbC*xRrF3nCi#u4ASO2KN$eWt6~^Ol zDWiMdE&ZL;{Bu=NH<<d=5lqd#_~%UBq;YR;bj|kbR+a5rHGjEpU|-kx{$s(}@hfQ- zx!#fF)5l9Py+3B#HvHjoeCU+XrxYnEJN?bA8N=?EbFUpM`J_Hn?A_ii>a~CJ#1Bn5 zcDu2W)`L^F0dbC@$E|*A6Eq!bTOpB~Mb|_sVj7<PB6gcdF>62~duj&H%oE_DM)u*K zX1iJ{z%&&IONG}tCm=K@AwmZNCVm=4Ot8vL=MWkCoC(79sO?yMd|>g(cNt_(4>f$r z<;20In$9bB^s)hls$|C*p*8NWQug>g{^pLEZRv<3q8MH^%>;@Pp1|&aj`HC%vpt2V zj@TQ8fzM*{P>(o0Z&<J}i<ao*hSFze<UbeRrjFBM?c@|441e8^*CEy?ixfS(Slf_n zK!<$cyfldsYGHrn_blZIbtNOy`p{c`li)Z+E)o68&#K}R*bBHpKx);l4hh&Lenium z&$F@x+F&n{^;oMJcR+sn3r@AJt?lKY_UMJ!%(0KI^m(AqQ)N$dQ+~p?JL9zQ?Wf2M zc<2B?cIshWtoMWtka|?lMZMqB?m^oRQgCFTnKKaKIo5u}l$A=9ZlotOSSg>Na1e0$ zGhr#S{L2zpAH*K(sxnltRi9|fKbL7Zu;+e-OOdL)gGuM*X7>tBGj;DMKlg2#Zl7eh zH`kL6y0l-Uq+WU)KG^9}R^bwQu&X^j`e<cx*?fzGs>=?OPnmegD@$pzJFPXe&#?U< zm(elBeRd_Jx8hI<Lws`_&j#>z?X^td?wAUJ1g`~UEdbK6YAPD5#NWZe%5XD{xxk`e zFEXnElLVeAQ8aI)(?i9Fo@nh&f?oNLEPXdH=nE-pN^@?c6ol{{wE?a`hYk@JiNf>P z*aSfsuNg?)*0`VB*o6+<;10}A5nfzKGr>xsGJuZ7G%kfRDvIFUU*Ps6G!)O^VO_fg z6B@-b*13<&MlafJRdwF>4G3hVw?c=3oG}s1RTw}_ui{!fK)`Vy0=B*sm*T;+1zu3M zcEkv=@J*pa&DJPxI6adtC9;U(?|0q`S;=}!;Gy|l^klMmXwlDgBW8_{oI{rv50~W2 zvx*(Luyi;m#n7P&ky0?X=|27{P&x?iKtbS)5c2YtPH+hXxFS7tCBYMH3?j+loh)^U z+835w>tS9pC$|*;Yyj4k{xZr}woiP<W$;FnF)Tj$ZF9ru4erY%PcJ{iLt~{&ALqUS z21nZov+vPz$+Ew@<yRWSop=4qpcSbzH$qm&y~~Vo?c#=X@A1e84I1Smy)H!|8{%{9 zPQCTWRoZhpS23+Z200yAIp!uNTfWOY*td{Q+;+Oj;-zFlJ#R@vXbBEdJ39M`CqhwN zOAnF9bM%+eh0dZ&JXK&h(9<=J+OjO!1;xFgsf&xSQ8Q$Hd!E^NIxnoJ%R@mwHBwe1 z34NxM@i;Sy5R<V8$4jR0jpfNgUD1!UyjswU8cFcVA(CT54H!+_0ID99Ne``}B&w4L zNycWf_!NW%kC3p+Oj!l3(%fIttUsRrZpzFZ$!FXd@|)#bfV4#+3L0J!NW}MVwEobf z2|(d`iUk1~i+hEa0J_}MMF1*JaSBa8W{tC!391?KuGUfB&YWi=b$)X4UsT-5y9<(o zBMHsfy}DT%37_}uKMEglW||15G4hhmd|-V=dCvl070P@j871zp4(1HNL<b;m;h~tG z9=tkDIjra~6j!<T7@UBTkBsYx=h1_@L>_DV*-nE~+RU6|cPlux$DaggDR3`%sF-vL z<*tGUnfu9;;o0HddkkN7iR7V_wJ$zct+t`k!!_#Q6@8ZyZ12EG$4D9U$f(r6JHIli ze1ZBD+)wAdzsJR9KPWw9{C+4`J)(T(txNU;^;@y=hlOg`;33*RNhiF1t)~z_lQ`Qg zzonD78KjxO{KUo9ci04Zk#Y-vI$NSa@Ek+v#SmXxX_x2m#tjO*W@#^9Bz92?6%Guf zOfGz;+-LE;@$VC6nmQ>-$epbRtFSkC%S8UH-<Y+D&<bevi-Dl*2IR}zCP?S2Ji^4^ zF;C@r&LFd}N<qcYfJ^?sg;AZl)EmX<GmA9$JcfR@8c8%OPC(NR;U?g`Gb_OPuiYFl zeE@q4HW>&Ui@@d8Cp9fjMRAQ8Rf2ea(`)0+P!-oW9Z=RP+C3`4@la%_;LM%Un(!XE z6m{Z3L*gR3G0vNm*t(jr0b8sP+Cwv3+l~Q9zhdz+mk8;=d&sWb<VE2vUhsKn2(ZTz z*wx6L;21HUJmCt-u)m1(<M^{|diTm6$k}g*W*GBT-hL2ykHd>Bb)h$me{Qy-5;R?F zg^bn8F|J&1#&g^GWtzfZ#rD$$l;p079z!R+Co_Gg3$6EBsf9XxNYYB{axK~)8S>KP z^B1SqV;5Q~43f0xAxgDdK@@_L(sB8P`;qstj{!XE5i>c_dCvSvm3xf$@YCm99iVIY z2WV;G(4h&SlQ#j{CPF(J?;tM50Y@xguP=siVcleX0a~0dF~JSq#%~fSbK@LsU3e=g zG}8sT999<}$oo3Jc5di9`t){8jpE|mMjXnlh4B_+$ijd&qJuXgxCeQ%;hFTGME>RA zQNVP;Wak1*c={Qp_abxE0nZ_-DyZcJElqYI=7!;?16_Je%}E*sp(WX}zh5b`)JDxJ z3Hta=OA(d0DS9KS1D{#61w3lc!;qV6TDS1V2L#&&!8xI&K^p>|w&!VMtqM`9oy-Bo zwi(oer}Y>ZG>=SCY%9A>JIdF};Vj52zMqoyFgX=r@r5qhMR<+cg;2q_>(H-YfUkuz z4j%Ij7(<}8icH3>PlEV3(FI;~N~`n?YG>FuY8Up37SoH@l-97|J7sORrQ_tQzRjer z#!-tSc}ZSVts*%@ZSIxYDAdZqzO8dihiYdKx#e^#`ch@`Mvb~jIqBd~&b6w>L%&rx zQ;t*}+S9#F`t*Zbl}1g6D$BTDvyd}}kxz6UyH=#dJh?t+S5_fsl4%_B%#G~%J=wKL zz;qlniexHO&YKG?BHo9RBlc#`tJa><$)z5?bL?@0;=9`JpkF+YCUKiLQ9i?F2upiG zp}8f_&;~=1jbxD6NMD?r8PLLph%0c4A`P(1VLztB5IbUahC3j40OD-Mw1zsS9iK&| z>7(|>jBw2#vcpr%npYZS?=;-@)hI7;${PRh7P-1o{kF=cEUfN}xxh2_V=Q_#->jT; z%dDs{s!F$-T;V&fdYkuFm;7jYyN9=;<zDS;LAJl_{#$Avu`z(c#|Xy85pwe^xgX!2 z*a%-FfS<C2Hw%a#3FAH!OO6#H^)XZD1eT0QneDvyy_d*=|5KIlRw&`}Q|zyhYCJCt zzAH3$!`1P*-Se7;>pu?`&YTY1vNGFz^7vab>ubL1z2z^w>}CQreKRd3pUZTer@F_p z3oq={8k@6na`dcjH_HxIa>=ngBa;<La_YtVWGQ0go&but%(>`X^(TIp(>ireKY!Nj zZ@j3IX7fS$%r4(8x3)-8Eq@Na_hfj`C)$qjPczgl|E5ad-^-N$PkbHuGm>5N8!?Gt zycjJEa!wH$2=__GT7IhHCkf7%g`<_&&P(!aq=w`#EhImw!mREObOT&I9Is*x<C2~H ztW`EJ_iilL1b!upzQm7t`@0Y242>SM`mMM3wQ|reH^CkX<mANn&lqome7W!k*6wVW z<`;DkXr(kmlwT-<@skt3_noJ#0^>j5PKN(@I#+&+S8btvz+7~`<I+@Hr_+wN%6+Aq z)h52J-+K1X7&`a1iZ{*-``NA{Yw?4~$_6ovt-4enAve7*kDdIW?5Ca%yfpBlw!FSJ z1%(~f?yDPKbx!^2ON51uyLj1f`^ZYZihg1EOC8OHh9;E>op!E?-Vx`VgY>dPX?b?u zOOY!rh;U@zbFL+bmz}k`^y#t3vIW|-WwI%%tqnG`VF2%t><7HwbW?we&LZUrQ@m14 zH1J3k+D?d&m6PFCMpZ#Z*s6XD_mPTh*#(bwrjCo1`Efm}!LpL0|A}(!HD;oXHP5Hr zY;1JM*VorZXH8Y<&)?d&?kcr29C6T=PSefw{^R$%#dbQIyd?U3=OnBWN9@~GWWrZT z<iKS^%zq8DT1mw^SIuo~OJ4OyXx^D%Q0t$6ut8UNmvdUY%g*{`yr2v$_`*QD?wv#1 zBhn&tBVph`-1xI~L}Do!t@E8nz$WlvELm+Ebd^X!{Ai|aSC|R*@Hi*?YfmRj>Fln# z2`iuJxqux7mzrNa@*RFz`(A|DoL=P<ur_%gw@SSFDmqTPUiuy->v`o+`X_pualP^s zO`N;(lm+Qcb%<<v^d)1VclEFM%fMfz5t)TJEHZ6wz8_+22D^`hiT3c;1lJ);ih@<; zBm0R~4W89RmsR+thB+Fdk@4<j;KnJG0TbV@8AW_xh0ZDz{wUIC@Yg|Bkh*zlV0c+f z1c%bu9|ypCR(t%mQzK8am<~Ka4hK&t+TF(2dsxEsX~#&nsz+yB->SkhJfik^i;Evf zLgRT7r91Xg34B+H%qQXLo{$n#8b7#$#efUE$D`bx9)-C=!QF6W!zCjY)Zcjjjm-Mb zTM}k2A5WL6Jgw|D=ZJOeo_qW1ZH>n?xv6&HN}XIp<-y^bRojAgldhHIln{?OoX!5Q zk(~xn*ShIEH`F$(F767XXP(ZN*&mg~1uXRIVyZ+grcv)7y><SRbF2p2B#FxTU6sMR zT8n<!;buae8gUdliEzV%aNsI&1kWz^(S9n1>cC&zB5CILp}Xs%3g9*69ey}ME25!9 z<#NIct!*T^1%v#SZkyOj+rOpk&wrbJeC^^8M@MW4`uj3!{2w-0W8_lTo$^^BUH~rT z{hW|j5%fPnBWeT<Tr+usf74hyE?Q0&2Z#jkY|LTIKfW8UOW}Y*!lgev`olNC>OyAW z<XC`}s%vm+$h!J(sS(HDSJEbgu11ZLSIz*}^Au7dj2A!P#hU;H;5B?59(4(&#>9={ zH5)GxVfglA;1}5#x-6tOLKcD$9^xzR3kLFLHR>f@9uS+#W<k$>fdJ?zZO(V^IK!qx zy5yAv@<%ka<wxTmEO{r{(xgB&_w%2`O5@X_FCN&cl5uueo^*4kf#s~_Rpb7Qtiomz zWMgtZXwPGbrVi~k;q#Dyr2Ms!_!<ux!ZVahM@5%FOa2ksBtLv?eZ%7qp&f2ay(YKr z{+7<BHMk5KV_f2~6@&Wu&nHzq?X}_HV;@7W_!^3mF(l7*fC=Lo>ed=3Ag?j;$*5ZA zj>@Hve@hVx4l;oy$+H*+cj&9z9omUIn@?!lvol-GUgjB<6YE`TM5RIBehrn#Es;82 zZV=Y8c6&alk$joL9r$jXhSH$PiFOXbQgNoB(uVQ~9mOEVVIZ~0iq`}BAg^;lU%$hY z=5uS$_iq2wLtjVkY8te59^AgWcKptC<M*xtzaL_4bf}SCS|1nrpjDGDO}}=vaA#2V ztRNc_8YizJcX{>vG`UBlubOr(=;2oL;aATmroczBv%r{n0X$AL-6sCT{lBHI8YfEj zK?Jm^=Pj%Y)Th-ZJdM3XxRkQ2PHyT%W-%sG8sm|!Mt<w#6F2MDGin#lQ0ew#?(Xhx zho0?t{#4q1%O<y-ccqRP{MmWyoL*?P+4e!_YN9pG=YV*ws0N<&f*0NCkrO;=WV23{ zIKorZ^%ub<mOd<g`Z)>1&=>pxhL}BA^VYt3xD3dRBkx?`61s&`R&&vX`d;EHgk?-D z>jJO;V~C0+i8tF3Q<L!4z-!@_a$3a3HlT-n1FsucEVxw}&e|qX_3q*8cY0K!XPmPT zC&$fJ(ui;o{*Co)H%kK`h(5&BD4}Pw=O5Rvj(R#$vx8MUJZcrxHUw$BqiTP782P<V z`&#vJI(_)i5K4y-4SUdW+5N?8phN#*RCOV9{RNN%{{H3Ps`H3o@eAZtN&>M-e^EF? zn0z#|Ostuw@4)Tr=cKkeUgosjUg+@Y>WH!JiZ#wyy^JP%AIu%{^cystcoclhR^y!0 zgxP9g>p}EZ6@GAEQ_}Ik55F?^+hkg3&4y)+Hn^X2$+Wzbkxx35{d^|6Z%KDY<fjwp z{zH+Kr`iDd^yL0zz1hWPX7<zaVtn26<IT?mvo{m+a9*sdU@?CFUqO?b{{?LM|I}+$ zQw$#z<HLIt(|7v)G_ybSPH?nbw0YC*yrT^kDih}440HrvPcLU>?Wk-1;hWY4eA7Ur z7Noth_$Z<(Y+4W6^h|^iCyQKb<`w4|jVp_n`4PSllAU?R?T_7#&r!I6mPBeT4}jus z*+JieTVC?cHd&aM*hLw3v7*7=XJ-@i=T%8B9x~8h5@FTprarMT4uVaD?kYX&uR%}} z^2(W8)_BHY?L_mZvR9~oyuUZF@JV^dSv8HM<>QtAxRhb?*z-gFk;=pl_dGMdlUCU) z2W;qV<pX`F&7i`;=~oXMbXpl_B98a*^8^!oN;0NoEOKw4U<hjD!ZIbwgWprOsF_V| zV_&VT5SjA?^xf50sq_~-H2Kq$2aQk0*Db~8lGMoDwC($|w*Q>DFG=lEV47O##fM$H zL!`3IY&N+`Nl9%=%TDW#D73S5y76lqm5GlV9Y6A5f9dvTHj#F^Yd@Ey9kTzo{=WbJ S`%3Qq|7-9ce+`g-PyRP<mfcza literal 43021 zcmeEt1yo$iy5?@&-91Qf2<{ewg%I2=1PB(~oj`yD2pS;4B?N~M+=D|%aCaJa8f~C~ zX>!iF_sm;Q?z?N&nmhAOu?jwFSM93$>#y3s)$P>n5`gzuUP&GRfk41B_z!Rk#?n)g zl{Hn<P?cADA_so}09cAoY#baB@BqNx!OcZO;US%#z5yNT0Du6X11JCw0GOG(I;m*N zYXR`2BqvMf3a@mB|DMh!0Qj2#V3bo)jgIc`<^K@EH+OPz0|1a3yf&YOrK>p{zlP(N zo^DQe<%e)g^xW<aBOu*j7x)0-nCuQ){f0Sy)%gue-(d@T2Mc(eyD>Xi*jwD;PB?z! z;cf}Xh)>`+%)`dg6OPB=nDM2%y$u{g;h4zY(##bAkg)E`-7L+m;rIa@W4UN)$ilHW z0H9%6{Ruz+6Lz!of{zmbWF4KnU2LqZ-RM}%S?PF0M1<)SEj?day18+xnVH*}xmeK2 zI@&v#Id}uWZ*#tD1rXfPmJUv2K^_r7K~7#SIQjpU{?mtltM#9QyS4qT#<BXZH3O0S z_<Pyk+y1@GF$VyIPvNwQ{d?K-j{r~`3IJp?e=lRq0s!3C08lgdr}>cF@r$*ao0BLv zx2LBkmyM-4*BwFsmj25Mf2;W)gFp4hb=TkDXh-+R@`aiEOE<bZqMAFtbaZ#2b9FK^ zx1{6zw@my`H~iDE{xl8_4a*mnE|w1Pu5{pBX5(N5Uv38r8#fzA2Ra*v|7eB($!32V z!yWvMui=33(+xmm$_3yJ5&?*NKL8{`OaRd+1O5c`w{?4ht^?fdJYCw2-}oMm;m?2l z{@;Ef#K8YWaJ8|byDOH}(4sSUck#Hx@I7(&3mL!wZ~-EK9H0Rh02Y7~;01&LF+duS z10DmafEJ(+m;mO0HDC|80G@y!@Cpb6B7s;S5l8_tflojoPzHPjYJn!84d@2?fgxZV zm<5)AHDCug1kQjf_(6sW!Uhq7$U(FqCJ;M_7bFal1j&J(fHXk*pl2W}kORma<Od1? zMS|i%DWGgnA*cdW4{8JTfrdfTpk>e&=m>O)fQW#FK!iYza36siK^Q>>K^Z{{!5F~` z!5P5^;Wa`u!Uu#Lgc5`rgf@hJgmHu=gdK!aL;w*Jkp%G`A}69SqAa2+q5<LyL>I&W z#0bPB#2myj#0JD3#8Jd0#9hP-Borh<BswHcBrzmKByA*fBxj@mq$s2mqynTGq)wz! zq-CT-q#I;xWGZBKWKm=#WIbdnWKZNU<Rs+J$koVQ$m7Us$Y&@hC?qJ%C_*TTDEcV2 zD849BDCsEWDBn<qQC3h+QBhIJQQ1)?P*qXQP~A|&P(PxUpthoppst}p(J;|y(Rk71 z(G1WW(1OvD(2CGn&_>WU(5}$&(3#Lh(N)nc(7n-P&_AI!pbw(2p<iL(V=!Y#U}$03 zV!XomfKi6gg)xV5jERX!k12wwj%kA#g!uvU3uZ6oGUf#q0TvsUES3?LJ61GSKGrv^ zX{=*xZ0!5k(%1&rZrIV-h1ea~3)oN`LL4p}B^(Q!K%7*Z8k`ZFeOz>0CR`a@6I>tM zB-~2eLEK$DbiDg`vUp~A0eGo+^>~wbr}%{UJou{k_V`iwMfiRAn*^u?_X*?(EC^l` z<PvlctPmm+G7!oVniGZ)ej@B5Tq8muVkS}|vL%WnDkT~qIwU3_<|o!6_99LtZYEwL zK_t0PqD1nNB!=WG$ppzIDK)7KsU_)K(lXK!(sMFOGHEhPvIw$rvT?FYayoK3ay#-k z@>=qF3M2|P3UvxkiVTV_id{+~N-;`v$_UD@lrvNaRBTilRK8TXR6nRrscEPcsGX=k zQg=}A(vZ+d)7a9yr)i<tq9vr2q_v@aPuohnO-D>8L-&&I16?QG;XSH*iuc^_W#1dP zcSX-kuT39JUrs;IfWaWl@PgqzLp#GEBQ2u}qaR~2;|vozlQ5GtQ!-Nz)A@bo`?~kT z@7LbnWF}))V)kJ!W}aigW|3lXX31e0V?|*VX0>BYXB}cgU=v`oW=mxoWJh2ZWVc~Y zXCLN3<`Cg<;K<>a<iz5X;q>4v=3M3?;d;Uq%vH;^&&|kfz#Yfk!+rBW;K9oWpB~Ke z5b!+a3Fc|wIpSsGHRny`9pl61ljjTMtK)<4v-4Z>XYfx65DKUWgbTC_Tnh>bx(JpD zZV538Jrhb3niM7yRuhgA?iE26c_<Pn(j;;rDk$nE`c?Emj8n``tVnEAoLSsb{FC^K z1igfrM7G3|B%S0l$t=kwse4jpQaMs!X$ENv>Ce&|GORMTG9@zm4<9^qd074MTvk-p zU-sK0#77E`A|4IM5y)xFrN}MF)5}}Q7t0?i2q^d}d{abLR91{roK&J#GFK{8I(#hj z*#B{-GM2KYa*8te3ELBwCk-kHDvwnXROX&CJ#~0ms|u(psV1n-t1+uNt2L^lsH>@` zs;_JCX!vRLXcB6gY8GjpX+6}6(VEp}(RSBv*TK~>(kawA*L|cLue+qjt>>@TuTP<G zqhDu$YM^8A+2GVr&M?t%%}Br~%xJ=x+1Sgt&xFF{rAf0XuBn-6<ul}Gy3dN9!OT?6 za?DPjD?LwperPUho@~BjA#L&AV#`v(GTw6Ig~W^a7n@cRRtZ*H)>76<*1I;cHYqj` zTSePU+jBcryL`Lbm%1;@?a}O?+1EP|IM_LKI?_0LISxCqJB2wdIg2_cIv=_yyL@&9 zxf;9Hxe>vIsUPkv?xF6>9+DoZ9#Bsm&q^<RF9)w5-fZ4)y*GU1ee!&fe9e8^{qFg_ z@>}+o@y`wb1w0RE4`c{@9k>>x5LECA<CWd3pTQ4;<AYB^bVC|mQ@;*+y%MSrS{#NG z<`Om;E*75o2I-B>n}N4{Z$G}hiLi+H9{C_LDH0ZC9`*el&$|!rZlhmB55x$@WW=Jx zI>b)INyQb!<H!5NuOz4>)V!yAAMySq(KN9yi9abb86(*}dFjKG4|N|IKgNE9rC6to zr#?(APoqkEn|7XVkv@_kol%xaof(;VnPrnTm93CnlXE{ODHkQzBX|9i&ZnL{;k?4n z6rUqMU*|jIFBNDMbQTH~78X$z#S|kHyBBYj7?li_J}Rv%<1EWACn=Bo0(^1*vR(14 zV&bdv*Y--`%Cah^s<djt>WCUpjd#sqt#$2EoqpYL{p0%f2C;_fMy|&Erh84P&BV<y zEf_7YTW(u@T2H?@ecNrbYFlYHZJ+JX>lo`)?;Pk-?&|H9@9yZ4>1pki=xyo~?W_MT z{Jpkcu)pSqz>n&m0zaz<1P5vcg$C<}M1~rM#fMu)q(<6CAB}d8DUJ1yKOG;Q(4Lr@ zG@4wRvY6VOemM=9@tC=q4Vpuqi<rloPg<Z_$XR4vEMF2_YF>V{`~$2Bo?S6t*;#d6 zg{{3_$6im`pxr3k<lSuAlHVHIHr!s@aoV}s4co)tOWSAJuR4%E_;ILrxCU{7+#W?8 zlON}w@Sn7ws+}&J*`HmVM?lG;1sB2>J(s$d8&{q%RM>}W*6W5F<(s)%``g>wWol13 zFPj%|4MB+wK!b||@C-uwB}nWbz~S#W4K7(wfRKNc&%kR@{*J+L%mPCERSvHX@PH8D zA_M?z!?SBRDxe6L#tGnUZNh(hrvtn^3Z4-Fgsi{!g7`bW>*X$?0EoZ869xX?AN?9& z|GnSmaQx?fL2wo0uCXn=9N~A2jCc1g;7)=1_3V$oYGyVLZkpDPPIPK6j&wXiTo3Nl zgnuvnrDMUV@|TYFZ&>4RGC1&;j`g1^XMdvGUpm%bI@Vu0)?Yf-Upm%bI@Vu0)?Yf- z|0Eqt5iSSa=_Y^ty`8$91!MsfBxDq1Boq{66jW3cGz>fp40Lo1A{<;SJW?VuGEyQE z5(;W2S_(=ADiRVp4myVW%&csz<g}a*I9VPrv9Pk-y#xfeCSssr5Mp2uvQUsvu>7~v z?Kc1)4bTI!5JC3<1bh%8KIpa+poI?@8E&+Nr$1yk5CS56yr^jC7?|)1)p!5`2oVth z2@x3?32sFO`NN+BNchME_jqJb2-VF{>79vqU&UpkF+8kjBi0x`WaNA95{!;PLP|zX z!E~RQg_Vt8Ku}0nL{#>XoV<df(qm06Z5>^>Xl-s``NGQD#@5x%-NVz%+b868Xju50 zw-NCP?-P@fKYUEd$^De~IlrK==xb$FbxmzueM5UkXIFPmZ{PQk(XsJ~$*Jj?W$?=C z+WN-k);8qm_~i8L9C~qiH!l!evHT<bHnV>-FMRmC5Ri}%kx=jE1w!zIXGDA?<a<0Q z1TyNVX3m84ysyxR9>!%?w4pQbX&e$icNxYYVdP(Cg4|8**UbL0i3R^tGyB`b{ywi6 z01FWWUpz#7Knl1_$hZY|#MB2JPt!Mn%fN4&E47y`CxG+fwW+OS>mddpa&S{wO?7q) z=r07}=EHEJ#R7{Dq*1AFf$_84om&9T3;&&0V42lyFx~>Bd1U?e186tg27u^B%|gqG z6?-EYBGlg+7$EiF#XY=KAEiEbJIg{Ov#;-y>e`a$oplYCikt|-so$$DZzlh~-woW* z8UTEu^HCu%$wc`L?W&p-uRNQ`F1(L5-*yqwr!b`NrLIY_t%di;9dj9|Y`&f`OykTO zjE>?n+B}}xJ~FoT#7An)V63UEjb3+Zq(NTH<l=u=+8omRX>V%VQ$v+RDJQpBpZc*) zj-`}2OYFlX2Lc9_`tKI;viHQCh&&@(ZOCR7d|T#Q%b^T*^NwYihsruX!J3ZcdD~my zM{%=HlE&~%=q0rijpEi3a1_O!E;)%14{iK5Q9Ip!OZ!`3xpgUy^!_}Yyy3S%5PAE4 zx6BRgV^Q;#{fu$sum`iQ@qv*eQI<6uNP(oK02$&M>Z*ha8R{DIDm6Ah?Q;v@&Y$Np z!aDs{va%2v{l58a6y)Zsg0B|FfM>YxX=6|MpV7Y0jCJD&+;?D7o(q-L*d_}jns_MS z3#6A5U%p;MQdX!lpfF%|q$Lb=W5)BSus7Zlvh*-FncqGy=FOY)u~ouTn<!vP5yDO1 zK0<g+{B6cuSY#CvA&PHZa<8hfDeOQqnE&c!Tv4{Sp5-T&R!?7tm%fuv?tOP1@as|` zSW&OHSJfC1@VKI0kbAFLm@&1i?sxc!A?UFng3{X8*cGN@V@zlw$LoXA*+?<8J~L0M zjF;9QrLme2(i-EPdW*?lxEG{LRYitAjoNHDy`gP9kXO2RbI~KCM~R|%5p*%AuJAOo z>%6%D_(sbjb#L4Hn-r@Q-KI70i;;ivF>uGozxWvVjge#Ed>n*x$H)r5_&5veouSr| zLwxsyBE_+F+*s=%7PAw9JrI>-1Wcjg$a~<qEQQ{aE^JBzjOQ9_05$y&@jU=myejeV zV^O2_lIjX3;)*xw(!`t;$4cX`CtlCl)M|iSZI3`_9^&2*5@E!~o>*M2I{$ud%fZ0Y z^x~vXF9%Xw8gP6AOAZu9Z&3A9P^nwT&vT=1f$b|bP+HmUGu-9VTw&v>Ua5{PY1Aj4 z^P@9Se{3D#kG=ET)&YLsJGPiR{r-~ezB_#c?1~I<f20%}eXxGa7r5JPonykBQrZ0$ zpyjJiQco09+I*F$>!kh~DJnKsL_|QsBhS+OU1MV?Q74faNcFHK&ysi+>P$0(S*)mG zXlFfuRxQOA+o%PUl%#6aAY?N-Abc&UsxM_O95Lst6=r}oTkmAd`dm%&4E?b)f8Utz zK|j3uRVN)ws#*Lb58GS=^MBfM<+Y^TuTC((P^GdhL7!*lz<7XxJ6NryKb#A}yW-s; z&|k;pXqp%yaJU8B+B&+;Y4mW^f=fKjrRXj&CetO80t2P51b6c1$JWgb2cxn_IQPI1 zBxTR*gna#e@%TO-!w;{)-rdrDoQ&MZi0f}6B!_0hx=KvAm2F@v`C@(b6H+y+_dKtH zWnK!ulh=+Q=&Jd$@`F!+l`&_pe0i*(D%IX-7$T5S?Bce?ufL+i$@OGi#L7B1&HHJK zVWQ6W_=QZ?nOi{cXADI@*qqUD*DX)&klkF}^@vE)4)?@4NoR)00u_<5XCzJY>5&hH zDRypsr1NL5k%Qe=2YXp+XYm8b(=Sjc3#P|047(MRi2bj=3O$f6syCN<TmAFtnFwbS zGj-KhIj6VI?gHPUIT$lrLV2h{Q_<AxK+1=+iNV{=7uF>&uE&e(pNk-+2KBOee36JT zU6p~^oKCoe5&`Pcc($~VW!I|<&Q`a@mbbFvezYHDkO)MBWN6^Vazp8R58jv`z2(rf zZev_c^V!HOy_eP;9^zgAQ}PFw;_7~>vwzaY))a{+gwNFc1^Kh0vfmz7ftDr04<<F5 z2c=`1f|3mW_m8Mwz#{9`QYgie@x|ZaeT>rlTEvFRTDb<qB&s5Rz+tsrnU?UIH2>N^ z-RGZQJ9i5#kqW$@bT~=vUHUp=vbE$EK^gfGi)-~hQjkLYN8Vu61&n5)*ajbdU6L8O zC{HiD-la#Wb|{p=K~4|p#G|u`0>3r`Bh5qXv!*-GH^jaK+ydq*Nm2p^ym~y_+7X#Q zT&0Oy+Rz7RHe#A@5}2i+r;jdGN`exY4xcWW{E$i*LbeeXd%nUcSp4M#v&a~CjY&75 zUDWRSD%~}+vDUDe43?LLH7~o|665P`#!<V#pkK<&*R5;PEOKAyiDe#>m&ck%?aQSv zymJM^u`=#wVbT20<M)zY<|JZJ$ktVbyF~HKZ8!tR1|x|XpWPYUv*YsHB=bBjYp$*Z zEUYJl^^^HzXpZt$+$7b_83;hQ)#h_r`!b<mTr97MOYNp=2-d_*i5l(w?HNY=_8tf5 z$aEOB1qM3r_bz>{G7x<Op`Dg%RYp`rj)CE3N=9kX;?50Sm{?GPk%1jyVRe;_(brE; zb)V72r<%tF^`bJ{<W(LbSBIrc_&C&-O*0r^N*o$xJ<EQs2Fs>-ruT)G@;%&XkA}ZE zzec@l#~ikVF#4mZ`QzSK{0)o5h>JL39ga^m;_uaZHoBjJZvhF-<aS=kh?(cm)K2Hl zeu9rlTjIAsoM6KGdoD~WK0Kf32k0maI67+!i$NmBOz{2j)>m#w>g#%#Y|*yoXBNL$ zZYF)cqlc+)XDFHb=Dgc%s8-@$J7V<+X6Lq90XMzMVORbMj~A`%9KR}Kblzir@w1yY z7HA@jS{*G|WRdah$#$+~-uEE#v1*+^E8{B>nGZR?q7a#Xb$*os@Py5#CBRJH|8W`t zzmB8f-_Ox3#NQ86g<t1r(BEVbE3dGQD`my{YCX<<E+UR0rViq$pOmO?Du7tTBB~%C zHwhN$=d<4{XS;YgioUWqeeE8uL$bmuL1aBeLJ-B*&~@%6eyBp<wf}RZZFNq(Pdt9M z@!=?M+4wOA{yNZP|1uht3PT#xF%Y4fzL}IxJch0z%E3uyCZey34k`L^1)WdYXuzT~ z!iy`_bOeLz92@&L83NAdInq4rwHiFM40C%~Sf$)OCe-8fEq2t>{haUj>?CUA=<4*f z^&`+8Uqm@YOJlfJv^F?5E{hdZ`x}=prGSM?_dGE5J-_X(_qrU-_D1x_FraOeZOIGw z4SY(74vURxE}XK$Y$Qj8`m=0BUp4sZ@(hu6*bgY(a6jfrnoYxpnN<Dz!b4QTG5&FM za1d79G3YoWCpuj!^1xRO6eOt@AUif36rd3xGc<(CP^^Fw;zQoAU>@lDnx056`(y8g zG7H%;RTA>6A5rqV4V<dRTJvC6EuC0(1M*NJP;{&)w81Vl1V6MSH)2Q-vlullf3{ni zp?6Sxm&&i&v@=8}!-EDtW!z83#>(~n#5)RAE=5MYSUy(FhUI53dDPlTwuvH44rZt7 zU<MQvXSLuxPK)?O*(=}DF*;1W#0fo$12gUWmX^uAZZ1E91*}(@YU0AJNpE<nhe`r# zPs(ey2hL->IGW-pLaiR;BqcuHrzBRw9o=Aa8Smnp<eGn4H=DQ^kP&vWesyK|1iuto z`JSW>ebrknQNF#NIq-*(|IfMZrh@raQvmh}7Lxrb!7#*g>)DEf!rVT~ii7IhF6$c! zzQz43^?_a5)~O-Tz%Fj<-0)8HqVGZwj^L{Q061E3^?IN^h`zBV_-Mk=aO!zoO*)n% zQ2{wM*|TEAHY6%MyG&7ui4@M#lFr(iY(3Hx|54^2f>>WD$5>xh3f=;-R^HF$Ml;#z zo!8~%KNewYTn1JsW?EKvYW8ptF!uA&D-wlaR-$by(xtWFjJzhlS*X}y%^ze8czM{| z)lgvn<Bht-AlXtMtw`Ve=o;fGv;Wf<rP)4Nm1)}u3o3$qp7!iP4(4WSY<N$X7JJSW z+Z|S;mMlV9S&A#Vxt{ipjaH%2k62ZXf46<-g*#&arZZ2L`o<uFNk)aV5v}xs<7eRG zQ4Tj>mR+}Ec0)?*BlEBcrU?E%#$_#lS4l!_lEKr{N<&9SuEq12|IK=gk;m1-Nf?!V z?^Svim3`k;dMNO3B@4i<$A4a;pqtXcMHS+s28ybgTZ#dn=9c-|?;J>@xHMJ-Nrg{D zEqj{N!Nue}fSPB~Q`Pp5lL8bX`3ACmPD<hD(dKBW@6cb#PJ;h(K{QNQQP+V8z_iG~ z@gEG~bT2O0KOJT>GAN0Ye#yPhIjx?_`H{hmgG>RPP2s!NQmw)0jBgF!>Uo*t{g`}j z9p7DW>h2Q?lr--_N<d54v)`eO!Naw0BW37!y+%vO|0h9B+b``V_jlz+5s}=!zKynP zN5SZ9c5)#IYt7Rf2*fS@LmdEar~k!F0H8F49YhfQUnT&c`u}|sz~}7DgP1%8O__7t ziJ*IhSy$mZ8d`kUkdrL*BJvyPrItHCt<$wN0!0dLT|hOHtkfdyV<6?PMt>3FALWRG zqAE4eFaG*eV0E>!B~AE?zY4akC6_OUkOc1d%l2_tV2k8?kiajpMt=_3r!56!|7!FX zA^uU0;MXo<0e<n9`q+?v$7vRt?HzxOC1TwIe*P>7ws#DcNUef8CuIPH{i3Uq+TeN5 zNYMSiTHS58{{z%8B8_Qbw7J0=KWXM#QJfSoyjEiL_2MtKOz(Z-aG{Jiv=QWm@>QWb z)zCV{SoYh3R7KU3s#+lmRz0xEZl}mM7|YYk&<2{!U;Dn$%hPbQNg-!udxmqwc*>{@ zn)h@i8;M2+l`f>0zVvSCQdPuwnEloN%p&(<f31F5U?<Ik(Gw{msIN?8*w%qSPa<aq zlO_e(7@s54D=L}?*eZ5q{<WPx8Hv})MTXa*=29;+7A>p0A!&~;E_ye;$oM%uP)Z#C ztCK3?3ygGylc6h_;)f_>xu}h&lx30=YBIwf(k@-vt5I<m;okKaI-!3_R?I0HmPY=y zAL4&~&H<HQ`yuz23?Pm7i;jhCcQSw)()h1k#3p(t1XO{r|7rz4Q~qIUu*XA8W15(k zKSud%yy4$bdhd$)ghCzeJTa=>lBTx*^~pO%3lM<`C7Ba=WdPq^CD(q`%{+@RKP|fj zI#O=p%BQb$AC2u+qxn`>_0!QB(MVAMkie)Gx>0d%Ncr+YdD+}`^3&Sd=HnMf(`WlJ z))!g?-gHjvpQLD8X!voGD5?kYJic9cc+B{>#hb@8WXED<x@zCh1@?8p9^3-u{<62g zqm)_b&!s{?l}}vRS@o5W7azp0MUNP&&vBkSI@j3>>2^Ml^~k5(E5CXF-S(7!q5nuE zGkh{x3bO@uyW#vT2hH%py8lCJzWpkCT?CfE={|;9@>L6px*5l=`VvwjN#^0@X6vTq ztiTtDzzRK|<N=>V<*4M`qyX67U;}Pr+JzLRqETK2jAb=dXp40Y)mf=E2RuVZT%?w> z*lZx(J@sC;KJ~3TxzfdVOd9YvSawhkx=8qb>`9mw4H~p%%7c|UDqF+ghDPrlcFPUl z%SA`-or+QDB={X{5%J^}AosF!Pv4zX>*5e2d04tm6fgBraWxp!?L$#pHV9Fb8R8N* zyI+}SBx_GQQ|2egA+5Mf^|iFyvoiD`h!>YSh7ZESZ_It>^rhi2Bi#SfmVLqOe4DwC z-gj3n7nl8jmlm!=5j&Ffep-had55za#>_f3&12QdLX|b6X7-cvHd<NoZ~VT?h_B*g zKK>~9e$G?XF@9vBbs`%n4V-saec3vCa6#l}USbFh0b8}HnDk4%Xd%_b4;e#w^%S>m zfIUzK;*iVTRsM$JMWGL++;C~>i`qBck*i@*mFeG5M0WZF+EsD}mlug(Xd{lo4OL<5 zBGMuq7Z`d0NO6XeG++F6S5Uh-B?V{BecVehrn@nvwWx0R*Y@WXu5N)tjOaoUkC>ap zf}gb7(Bu_W9HkX%MBUMo;`_&pqbCYq2>pwg7`anf#S&9d_pjW_z^eFOwdzpkd|MN$ z!(~<cMQ<CX&slznUzx)_b=kUfQ2`eh9QG)vE`+_Kg%qlck_ncOpH-PSLHkryKkC4U z?qm^_h7^_7A}?y;s^3b`qztwz4-LR+hRD{0kdziYN*PP>X;QJb-R+WB;;1*S7uo3+ z>C%cHSFkJggTXG(ww}{3L>ulvy}jKI)(xQHMKBqteQxWU@+RL;cC1)&ANjpcdQ1Jf zUI1^V_39a<Vwr2$XYC#2_7Y1+3biwc-m~u2S<#mI6+(QULUnUs;$!xHn2w*feb##A z31^;8j!3^6Yl^yX<S>uU6>F(JC4J4P@5lM6k9;o129xy^ndHq5>FSUK#c}KAbLesV z2(<BOh>HF5E0xh7spSMeUdOMC<F7ex7b$JPZ;$LmUT0MGOHk@U-+tbTEg{iqm`ucy zjwgQf43yAUZdpzb6}tr(!aHw)D_o(QAU5+)i@7kch#e7}pOt7D|8y`v`InJvBC6#8 z;=jHNa9Q*J=vPuCKVoJ=f1gn?`>k?Qv{O-*MZ7P-*5$R>J^K;9NsKX<_tHwKNH^WF zjfa+#1_Zx7m1==mP5GqQn92y*KiL_?gNB2<#wxsAjt)8&P~Oc5j?rImopMmovSgk* zZ%7GgUKA`IPIfF}dbtSP{2VS7hx8l$2-g2_&NR!M(d2B*nO;Sr6FW2WFm|>*Sa{`~ zM-dAdfYU+D=`N4K%>Q{+hBLin=uKWtC>RKe3JSt~sY_Vu$2qjmHK~4hkh+}i!285= zM*T=tLYD4AU)A|lron-Ak&Q&3N@+Cvw-(m8CR6j}gSS2o8RDk)l<DmHc%S>HAmK+J zqI&MpdZX|SQX~7o-9>)>>^~$Ky|l#*mkaHm2Qp~Znz`oBvv$ZhH5DU>VjcR&l(FYP zWx=(O<e;i+oi*`N+4bNv4)gvDzUpo<i>Oyq_(qlC>_?(zTo(2=mx3SjjpyfWF5MIJ zey%vyr=XDtrPqsiAaZz%XKKQ~#qF%i)2G~fA}7@MjTHqCs+-g7x%@rjGAC+ne_26D zyl;V?R_yHswnMzpDoWMr1^<QxNA4erH*$Uz{dmGOrxUoin}@c0MY5l^AloinG<%$} zPZo6%onbo<e`y=&24@F>ZQMf?jjgA1KO07Nxen~PxpwWX8ml<$H<E~L?r}Wu;2}4P zt}86zZ<d_7+`d`po?zc8%>l0qYg>F^X%BzKJ4MYmQuUimhD^DLvza{eUi95rOuo@J z-pcoelyMq8$n_)x!=ZqcT5Z@qbMQ9Vda&FKq&GV;JR3Eb^_LtvJkqxQY#M4VT`{`P z!BHi_wa~pvoUuTuI3gmmO)lMmTl#XVW=MZp&fW@3&G2E4F!(J(4htDZmJ>i7s{gwR zuj3|a|1PxaM~$G)#K^o<s?%X-!KjqVfb>O7{d)etvIr@sofCxdmoe*~u<Ge%LL0%E z;o@RmZd0FA^(N8VZBh)pj}tbhmL<;d{NK681VY5Rlw<f*4|P&tdr_GEBNFJsa=j<3 zn(B^bB24vL4eRxJ)+Cv>6Q}GFl8agK>y}Z3o4nqV?R8<$>E(|pihe?NrCv$y!t8l^ zVnvyr_C=^IDwbQB7%&0ksg?=)dI(NI!0CDgPrd6c0CIef+uj)I^}IHezV<EsTNGma zhea{Z*tO+OdEI@fw0qvQFxw8k!)=>+`VRf7(oSf>hGFcJQB1zT0E|m?ARD6Eqr1SC zF|wjl6*187Uqlqes^daL<9zNd;azS}&Oc;$#k?{Z!FeAk(7~prcBs~TH3Xm5#5Lw| zH8ROZE(fLDk<FZlv0j#(2!Op4#^Ab=&d|Le_2rNwg(&I4cFvKZy*eXhh8va#uiI9% z-7Nc0Nvztl#DjO@(jNpblPc&9Yc(K`5ZJATj$@u^sj%wvg6ABDt8E~BU6+pia6%^! zgp_tix>650etHx}n{KlOmtn>613fq$1Oj)9mpcq?zclA&pD=whov1AetIXL-nXjEb zeJ43`NxN_K=qu=?7o0t8Om17e?fylDxRX_&=`F`oA@*Dp9h$7@YTG+|E*Sf;&CSMO zWBIJL$jf9Yy&KJfZ${+Q+9)<qQi62_j2rDo*KtGGiqloRW!GWbK%yTT=%Z+uSxAPs z@HJ6{L5_dW>DP&1G#LW3T0S6~_RsG~DQ(_<mHf_-K;h<yiuZ=T!RyWtL-PNwIqxsq z(m!A`8~MrM^zdkqUV+F~f4&zZ%SDMftDIs|lFp7ss5Mm)YF5LJHv7F1!U$O>8+5PL zys2ou40txG)Qp!}Q?l1W2a_9u5fV9@DsirtI(L;Nzi6@lv3k;u6Ee5t<ebpuy*nW( z1Pxnoj5wfezeK#AXHZGLzOQ|f;;^vYte}SN=*9PKMm03W@B`9hLvxRiLHf!?En&hW zhYzmVrs!!|QSCd%TAoD1-RDr#_8I2Tl9v7~o0>EA%qDf0gZ&=?FQhc=Y$Mn;>2e}i z->?TkE3&ppA8chX9nP6py>TOda@dI@If_n}MBC*PAY1BrqG7x8w3|7^PbJ^0QMz`# z>Z6b>T}Ss~04c3Em513aFu7^x(6rNOg49|uc^^XCu`|WrY~tXsyP;vv-8`=_?|2r@ z47p$219`Zih(JpT)ifBnZ$h!YKwE84TNfuK{%mBuW~KT}LNMak3{u&Kdo;VaF!xln zgCtYL$rk~`oOwr08GJzQRZen)eK>b1P<g$lYplxs{PGh6)7xEDnr7>DW5mSd05)jS z7DHsa38|!Ggn%1QPZh)leR;zbMu+w4VDvMt;j#qAXon=#=!v~JzP%ORJEiTOVxDux zc>kXkP~0{W6T%4JuoPnj!={UJcd-o1^k%U=Q#qP>yOM0K63Wfs2$mwoy900;rg_*S zJ?SW)_cCE~2AtkkXcM#ggmP>p3?}ct9h&?OW_q~X6hkvB^~O&)AG2&q?roOor6b<n z%@AxVO*eN)nl6|ZSItgDQ-h1oS<p|Ue9MGsBxkIO^2+y$RdV-BjkM%zchoT|eBC=A zf(bPkH=NL>Au0uKATeBoDxA*k$Il0@3>;s7WZ2}}MqSQO&^{amziVmrTcCjmWDoh7 zqFt}O-awbSa;lX_CGU3Kisdsp3W?r$^dhff^OuwBHz!0Q)4#fZLS5TB3*j#d_Mh$f zMLPx)>C5v3JK7Uv;L0yG#z)HcmV9wmGlFoGmej%T0#y!MLOo0DJlXBdY*SY~w*854 zc2xn-`%R`#Mov{DD4NS7#O)Cctsm5;ezN8qrR@-3fqsa7cmiBn36Rx`(OWVRdq|p> zz@8;?xxd+W3k0FyI(}Zw>;4St^?q|zadJ<=q$RAGqI+PbeSlJ#jJ~!e^q~ULV`4QQ zKve5QSL=Ch#ZOid2O@Pj^?P#gFw6fc8!f(rk>*0bx6zGfTh_fksA+n=n=yaIE9BtZ z^OXHlPJ%9lhp-e1^jQ1QqtbC%^{pp@-nt{VG6bE{yn#CR!!3Z7u~8nVxvL3x<(AQQ z8qlPJCFob4OxT+@d`&Zp>rs7(tETW=gt!x`+1ZfUKL32M!-S{NcM*NQH6hPVlQw0# zH4HOkYjbf_Ok_hfS<CHVzF^c@(Vqn8-l2a%H!#~c5%6Cb?f-vvFRGhv3;M_>x);FX z+h_F+8I<`)sMuJ@@0U*oL%*N4a6so_V$b*eh_ip-6pB)DUVl$}Lv*e!o*4*5|C~4< zoBFkp;Y;)}c;gKxOu4*`sWqg;VlmgoC5rMh?a}dreW`|)jA_not*K%9cjoW38KjA| z<pIPjgmh!)h=_`QRuQpKB6TfAbvbw(>OYuu0$OvYyh0QjMo#VA>b};i#LF*cqgJ5C z`r;c9sqqCtUBQO80K(4M0wUyH|Dd_ZY?kfOEzobd=Xh^-=NnBE?&C9kxRXLR!)vmG zM3U=pZy=$N6`gLLsQ-K^)_<Dl3?2cbJ~S4H(q!gaz;qDmr_&~Gn6vys1bjq;V8mR& zKz_np<0N7$o4D!r(69I8my_f7NQWQ)8a)0#*M;5>p((b>-qKC^oW~OOeS{=MS0Yh{ zdD|b=2Uuzmg67;zEIvI94_!ZkDT7gXIEp7mrUFaTE}vQ8c1Ft?AI5~g6lY()QlV9f zv8--bVLnh??A-N@_ZH}Zu~jrpl)KZ??m);+C0>H85M9rd@7<K5Hahi8hK!}0L~P|z zvZtLgc|ObeO)O*qaw7hFxeOJY@yc(@3OtDUE`xQblAlrmcIRs0q<bbB%=c<1mM3k} zU+#WcNq2hsPh3Q3N>8^W@o4otXQjP2p8xCB?s;<DhA#|<r8al`9q#232-RsN%2W=V zhw65@e6&}6RNZ>Ra{W5Zd=YRH*mr`PBFh9zb_c5V%l@ovDO&trbl(x%MgYGIR{wJX ziJHt;{5!nzJ1<c?W_9jk<_K^;E4-?StNk8+eGd3O^O-C$o|Ch&6ZrWfk_s<7Ucn(E z%OU}a0VxluP%bH}TuGNhYlaZFd1*Iv*pRal1jsT^hUlUl^D=X3`H0eYN4&{amq&ON zgqp`mq^tiOE>ZyhY<9T?2w)Gy4VPxGd2srF=GZB@hEQ#Ne;`r6l=;vu3n~S!T%iSR zU8b$P$og2*@+OLGpW(s?f0lW@Cl{l_#Cy3%5(_%*p6@?T+XWGrW2o}1z+C{{T*7+B zmUIMi$7bF)&@BGRd117^`hP3jhgb*BhrdOHgseyr*aQ#`e;%)He1-f;8V6<FLjqsR zMemF#^$U}$&o6Jdx2E4iNK^zjrzGk2k@cGA#n;6)>Vd_}^R>#ov`Nk7R+=r7dxrw& zZ%#~kt@jub^Qu31yL2(y)pf3<mtLF>@(X=W^#`$6C2JP^vW6Pzyj@W-G=+NRK$9OH zPR=@1BZTfsF2r5%B;CvWVF{{;S^?4~t?t6>KBw4AzV3dkBrvJ*=cpCA-(B+T@8S1e zIQ&$LpuvNx5pFkKje>mQS-=r)a~0x{Ti?!>LabZ;_}E<xU(q>%x8PBK)UL+V>9=y) z!<eo>Dro-05Zxh3Qvb@QX<wopp^DF9oHevyWkIK?)s(Mx6YmUzc0W(J-%fUPQ3SkN z2}ED;dJg}B&T)LQE7)K97%nE>`3*aI@%rT@|F6)RTD8`Pe;((*7hPlI7!0$#1<(bX z!RbP~w3as=O(Yps>8*pk%pQX8nnf+H?Q$khRd=PZwr9!{Hm<`a3^Un1=CAPJVZAHD z-A*=w@4XugZh-=DTHg2C<`!SlUu!uTvnfRwK(})#@wxaDjP*(`bndwxyr0(*^FoE2 zPQ7R_xtCBYuw2(I5CJ}%j1(90UM}_UdNMJrENUd{RD2%%IE_F=xW~3wY$~<b=%nFK zK|-?s&0t790%&{pVo#DJ{!zZY{PyrcZ4i<TMS4@(YP`ae0Df;&<<!&vFx1W4bPu_3 zkTy8y@FwNTU5db1Ax}2CC_2!m=Q=hh<mS}PK>Ia?$wwQr7pa`3W{UKta%uo=Y?+vm zGnjS;pTWFkvi(x+nHbY}-ww1B+0%Vk&c-#Pc%n^?NT*xoztT1S!{@~B;C^`mn+3ul z2LXGNhcytVf)H-@%!W5&6}N!62KkrWf~fsAZFwbT0M`KG7%j<vII^XqYw)5uzaPv= za!GlmXqNng<NK6#pHUD-LaaZsjXqS<7#jXgx#xIpYexLFKx=qu9*dxd5Y_|zm~*e5 zTKdLo()Jmgw+LOIHqS`jNNv2wKtE?DXPz5t?9<D${`xr%Qn|ceK#%X0GJ@)aGn?ln znEB9H`r2-|s=v{9R<7=AT^Ja3&EjoGu|FEP?pj%K)Lk)yQ&qL9>D|rIAVx3l7`_ql zc}H<3j*(KaA9t%i2@LPZrmU;Pk!O17dPIRTT-BjE^3z0jQIT)P!LPHcE+(u+G^ATc zS6$Wi(xEwDI9zSilCFt6Itxx68mD=P8termtazy$rK4c;!PHNGffK=q+M+!&+_;45 zS0@L0wT}G<Lz8&}>%WwC??9Ldq>mQ6_k`AAy-JcAx^UHLkW`rsO;A>0?@SS+s}Exp z@6A&nS7MNG^2dED;bo`hL-Y1e$LJryIqad(VQ}gqGE8z@V2_L4#*y=pFTt9AuGlAS z5e%8lp62_zWj#czyV2Xfjx^VJ6)@#Hg@>uC6pF*EZ!<LrFERoi2WLE$G^rohpJ&gz z3j@>CnR~zg?@JGBEeu{iytp6MWMGnxxhr2)RebrXfWGnM8szZpS-dr3;je?vM7G3h z63iTIKWYFOc(>digMCqu%$fU>r>;^*c%fY;9Eg=PIL#DbR$O}<(nF7;LyaaLqM-ch z#s3yloKvbUjqG*Mxki!SeGpC|F*R?7*D!|lgtwf-=7_c^Baq@W;&?IAgm#>Z>F^ea z6z8+$7HGvWs_M5%qb#8I=%>g$z#mE6W9i$?Jh@Rdjn%IjnJ?ctm^xuw6NtgF5oHdA zS^4Ho=T#5n+l!w&l=S>e`0kjBwB#<qV{`~TUaER$*5GflsK;<4!)udpT^xAMQ|l18 zq^<CfhHcD`bEO>L>x;2oN0H&<nSdBEi|KC_50JwTE1eY227My>&s|Rf4;Lq&Z?y(U z7OdPK7r<sdGdpCw)IJ@5zvpdFsbtS5--T74$fPJN(FMPmN$mG*)K8}J(AFzwwmk{c z@@!sK!+Rw{7rEvW4__KPZIh}|KuciBEzno)g?rPwCv{>sX$;|$s~l=!H-PqZnoz}< z9G1?9PdO^<b1qI<=zRa77VNPS^=fAqXB0YrwjOx-xEAa>+{$PVi-XO>^{LSE2u^Wq za15?&4(yR6*RZ>CS4Bfr^H_FseXZ410z>)FDjGBfsw;@g$YN&;D|BTg9RruxcE%*c z9=o`=CL!PyJ<80CGcl6c`2`{~;za2t#YV><o>#I`4b*dqnpyux+rB&F2+0T~vWe$` zmy0PwC|Tb=+rg{A)#ZTuaQi4G?lrMS;%dUp`D$x<M|QcAz}iLB;iHtn@EdkJTHFJe zT#+4x)}{2AJy*FUuBU_YF2m1mrPWx~Cs;Vu0|A2zL|5F=;*Orz&E-F@Kb8#It^V|G z5Nkm@W;XIW@|^Dx80LnhYu*A=Wl|?OMX-AlC4N#6g&q^~@zwZQ{qgFLjxV#VO$j^I zT9&3}hddc(`BxhJjZBxhkI)Sh%LQ(MPpxBT)Ur3?T!A|YfefaiFeGS71{qZS^Hl%R z0r@u1Y}Ni`gGcg)A)dtxVgHrSPX1^DCuP~U01c_sX<utx`TgtUz}2_bQU^wXo98(~ z=PjlM%GQ^yy<L%d&CppeExM_73g>?MvI?adMN~uRg<QOi`Bg=C#LsS@w3Q?BY6xN1 ziVifvd6`_?4E42}zv%e=mPS(b*}1vzuo88#!z@;$xIx=&u%+W`Q^F+(bpa2QuT$fV zfMW8sacSRGrTG^LRT$<>4kRP_lb0!Y{5-<XaJg5Y6>Z0(f+M;7d2MjJ^*73hz32Gv zaTJWMv_7ZA*gNhxw1&2rn9rs*CpL4#ug(NQ%`Q<Ro*ib!@MF_HfowxtWhRVE^|{Ky zjSZEl<O;`x@lgRr_+4%$LKRJQla3aUm`?R&DIDtJ&$Bi@!TN)&h#TIF?tLOR#JxQ` zn(Y)lK`<Hc9<Nk*2Er)!p1C<^cU_9cbeDzh{_cx6e0>=mH1#{tk4(G_I~IwW%!Imr z3jZvaUql&kaOA(>T8^KjrGJR<1<5%>^c<XN0S}K+g-UVD*n2d!nnGzu1#fVmz2%+r zo*g76ob?dhu+o!*S{Yxj1&g7Euf_Kg5WPCABt*@dm6jTD@%*$$Eg+#Ak*D(n6JW4< zp5B&AiMkwth&FF<{!hk`*uXLrg<!`_z3c3Qwff<GH-Evaj#89#?9PXz@yKzK_L&{= zRIet<N@>GP_(yMupng#1?x98f#VMo5H3SM)9DN1d`^Hb?h~6y<K5&kffEe(nE-YTT z!tCwb=V+2^{R0lZU%?`2D~oK^y~4rbKD%-Kg`bNe#VbCK#|3{`Kz(`Wd|;9Zp}bKl zkATO=k;2c39D}BfGq*9WQ&R%i<Cd|Ps}=Q`+6s0y(ztk2v)UCfX>-Zb%}k=e)eLCv zO`ERiC2KKT?HOEfV!H)K)V=I{wH@I{fu?x)a!gl$AU2F+D68qv>fL%x#aDHv)0X!X z)9<agzLEmvt;B^n3${|kH*`?Lu9^mkfrMpa5|O;xjKi#yBPCJ?1Xf}7==#&L)}?2t zOSeEM2nc!h5fxHK-!;2*(B)pq*t29x{MaujaC~aPQ+-;yHI8;(psF>}s(HRSu(aui z8H~Mb<}%Rm0v{UE<pizmNjZC006mgI-K5|v{bFbbE#*33?lx}fSyNZ-4-zTG4R7H# zrUgs1zJ3mQPs2IvPN|zQu(uDctFKAoNs3lbZNdna5f=B=p~=qoa{NY1(4^@KA?y&W zU7(uCCq47&ytHmo_C~v2RXwWY(pwEobEiQZ6K5!g)>|A3z2cxBp?mcK8tS%I3F>9& ze@)gQC?g_5Mt(NnH0@&t-x5+sa%1zCTU&ugix3ts(LobvRYQ0Lm!D~wwU1&`5;pyy z;zu)eD*M_Wsce4jidXqVXTsP}xk6}$DFsaV3q+@ob8RB^%sO{wm)>-~@3{N*GG<W8 z!<7Ur7$P{5_O+mH3m)W?XOa@~>Q|d&8#~6~ML7l%wLBu&QaqaQo(2qg%)w6<LI^J1 zn}2`lSL<T%d|}z6kK$$Q{pG2>8O8H_VlM&OHJJZmk8dLT{n;DG_Jc!F)9DV4mhl_B z&B>I~y`QJ6F~Z*amNphIA1_&ch<5^rd!Y(o;#(lBg&6h-f|Kj+Trf9fcdj#G=gFr? zlt9s)mVrkyfFeA=Dv9N_^Yjq+wfUjKn>_)=CiVP5ac3KdzB2oKSlmYg-r`zR4_-y; z<B0Vp(&bWx9$KV;ds`LK_r}*C*j??jNl56io$KN!tHJeK+JbqkkHz2?J%{K84<h1{ zyyjbA@KE%Tp_1Ej(ND;-W)~;v#|nW+_56|#1A~<(Y`EkDzE_$}DiBSbu@-bM-%cIP zz!%E?{?yTh{*JUC&-iir84E;qs?5uTaG*sgfsr>v3ZLa1OQ00rN4qd-rrk7`zl~Pq z1{XatRAkO(3?QCa*!CHCb=dQG%&i<WrEehd&0kkK7@x1pw)4P33+?lhXR%tRBAY7A zlhu=-udgIqy?I*FPfur$cdeje^35u`PsU*A5Q>e?8TC|(yt49w(y602JLFnF97Oko zQR}HM1_`~St`yEvD{A4INMxr+tKEpn`H$+BsYCEX@q&THB!;NOQ|6Q|2$Bg3gsU-E z<<O*iAD1)_Ca+(0rL<>O{f2gdJOo*qryJPF7irs*WZFIpz=d8cFXBVPlsMh=`E6c? z4u!6&3w?MxT1D7v_7Iu_n}JCULWba!=?mO4U4znWivDH>e;=l%KzGD;=Yk*k{Sud{ z?zr1EMc0A5?~Wnmqvg-_iRRM2SQQ9O3EfhZFb2Vx;u$ZGqhsx{0Jux{I$}Uql|Os- zcn^`By?4+6C_MB~-3y0UMa-_>%uU?4rk$8!SnIklD;_THk|gs}I?N1fBF)+~X#P2o z?1_XC?_V~cH~K^5#YN2uWu%Kgwl7X{q2Bg{jc4A1;etWye9D3hJ#DA6u~T(PwEn_5 zr@9=m1u_v_`uk_Hw@E4hIp{LHmfw4nLK<o*=;18H&UApHgtqzE2{pUzv|J-eE`m8; z_O-NrOcwEmROJta8)An{UpDZDS_XLEV_Sd5ik$g++#tDFF)ycn>ko$w>PMJXZ9@BH zQaXfCG3vVfv1O!2ge3NSDqczcSeXpc$<CDa;>`0+mBZR6jTP!HwAz(Pc$C%&jEUqD zd_9t<<6d9;1h_GGNHhv8>MB#{as7Mcyk8WO<bfTh%fAIqpLu$o^UUcNGvf0{6EItF zNcVli5GgYlKF^tSzsSuYIA?6iTph^mk|Oc*T=wUzrgmL6oS9}vvgvLlDyDe~@{da| z&e%X)sA{3Io=Hz1eB&1gPAw)cKmEXvcyDy|_*w7_KNGPj^et7q@X3xzg1?Pppru6m zE~`Wdq)MZ6g>73NjbSL_$nkZGNyXB&uQAlWNZ*#!gfv#7JE7o$Ynah7eMtPnH?xEw zRW*M(^RhxmE61J7k<D2Z?&{N<*0QLhQp~PEEI(uG%fYU90sA|~bstxw`>%P-Mn|2| z;@-UH@(5DuTfJOwfJg3lwPRa-TO_K&meQ-AsPgAlO)?g}T$M_B7LR&8OA3na)&Mc& z*RZ>`iI%@<A+%v`Pnmq}e(m|R`r|E7>5fe>XY}^o3l5~nIwHD+$xDj^ThYu2fy}TQ z!4`;of@7{hUp_p7&}!PE`g_-E>rb9JUFVxkMCxNXWf$dSJE@(?8+5nwJ#+ty<c*8K zZz58mndtoGM6d@#0^B6?wn8asN!BkdBlZ=;GD&#I-fD&Rk}-Y}5h-)+oVm#@3HT|= z{bgc5<6M2Hns&F4EA!wfLtTtr!raoUjC`tz$@Um(XyhZPW!Ba)l-X#!jA9I1bVpS# ze#ZJ@ipejAC8ogz_sMy_SIjqqh2$$?XneP!2zQf8@dEba{e^D9_ZMGPDT&qUnQfz; zH+{9o7j$M%8?LznI|srJTW*28R?bo=MF!6;5Z`|oc+-3!b@Fi3%@KDRW&~-CdI51u z*7d=O^u7LQ_BroI+@rFP^Z1%>qzd7-!ISQuNd9+^10w@5xamd@7^}Vp?;1aitx&{# zn1%Z6LA^K@xZoD(+*l5b_EY!BpdPP2z_FV|xFH$2NBGmxNuJt-S7t_4tieeFqFFtD z3pAe}2PW>_cre-LTVGEEmbNZO7q=~tLTJPMxWUvEE1me5a^5)a0{H|wYn+Ie9#dM2 z;WDf<YzmhXkJ(G&tJ*QOo@F$H7fHL&aSQbXR*#$Nq(q-xJhRH`BipxY2*iT+cS+&G zJSFov=acflAh@o5&E^~5Z-arGBu-dO-tB2l?R^#)?xzVRIn**CKHTUiwR<v=Q%&;O zuvcSI^;OLtFS=_o;=wak=F^fq&ww+NM#&aq<;chLW!Iu^$D57;gA_k+8V@67o5#y6 z8XD>KQfzRH+ZDez#<&KF9GIlvT@^d#o-Thn#vl0lGT>v0p0S>(fvPiAoYAA<itC*R zX0g1V9<)(WB(L<m6<1om1u(laut(f6uXP}>IL^FWBTB8FpZCX6GdjerwhVOQ*X`Y} zl|ENh9`m&b+7fsb=0fe47Z+_)J}Qj8Fws=4!Xi%}eDM+>ZSn8+e@N+8SzTKbDQU#R zVV6;dh<Ks<+6%L5#xumNHh6G(qq8w}C+yhKai8CK)VP0sJ}T1)C*)Y*fI@C?)@|0e zoZdH3QXru(T~LWeT1?;f!2sInr^MuqYmb|TDauAzxYpezx!(epHOK}2WF7q#(fzM^ zU(W9BA!$f8HYOM~+I^?K)MU^oX?idIzBxVGaEK$aG`0K#6(8hZ*J*5~7r!qiUm%X_ z{n<wy`|n<AN*upV|1@zpKR?^X*+CS_7|Z_nn?)$nQ9xqAZ?~OX0`Xz)_Dkvaak=KJ zu^nzqE4OCc6mXrnPh`3cYiV*CZQ*{V@Ebx?(w#s&#|jP=_N^JVj7NeM@p+T;v(8d) zHf?ihte#tPX}hSfFP~hl9oe-KI)v31S1eM$Wt^Hu9A<cP*y+h!UcY)tT-_x2g;DTJ z4;IUVitcPJf1uF1ERUEv+Uou}eSK}%Hfk6=rZ=>kLWGDnvp`@2u5??Cl&wzOP#R$q z>Z1BiBgiE#QLC?PemY~w&k~P$l9Q_RR*WjY<AAIKJd62?vr*4Zkk^|cEc0IfY_C*G z8kJCBdqcy~U<6(ax>u$GWbX}r&f}DTFS)M-s$R1-`n<QVgQ3cp<SR)=*}KC{iP{82 z+7ec3k%}YS)s%@I!2_suappjUmPRL5EZn7K5@-kUnefWZyun+f?0K<GWmJ>!p&dVc zT1VVGEo!WuDch)QQi3<4&6HTr2&ye`eUGc!ml?xc8Lg?$l!ItlSOn<^mjQL8HuU#6 z3F6Gs9TEek6k}W74`A=ySBe^)jbF5>eBvbvv4(OYj*H=@-I|)qnp`uRWm8hy7v0OH zd6}xCuZJCtEKqRwMVF41`k2iMYKtUDBG#IySNz4ggh>Bad*>b2RM)QgASwt*6_A#o zbZH_@kfI_WB1AyxMWhK4A|MiKC{m<FKsthS5JHsRkrot?4xyLOdqRy6V50AwnfH6= z%$akYYfk;nANeEUVrR42d#z_Z_kI5s$=<$d{5r?s(~jY)KKjBCN0y`9w}bOF$(eV< z1V5e$Ll#V_=h<TnN-3TrGd)tdJLx~hMjT)>dZi7ef6%r(Dbrw_+CQ5n{T|R{c!mrR zo#vx;Wu#H{W!hCj93%@Cwxt%{lEyz&jMSA)+O1`f=XQ?!K1I3g8HFw)7+_gGU@scl z>)MZXM;4UX8g7(BB%QWDuIk0mDwVx|U%?S>@*(bTRdl<FH5UD`q4-#lYzuRB-p}Yu zshL*J0w2)}#AUV&bNeIZ%25R-*N^vQCYFsOhax_Co&t)*IF?m#K85{j9d(A*1dq8E zEFtqXZmOujv8c#O+3vkF7rJl!L*6QwH!LAH%Xrxn$;?%TH#2EdEGv*=%!kdZ(G+J} z)2lync<}qO(7I#Akid``EiawK85KMJDd&W1Pt<%y{0YPD&5l<|!GwgM-L)WU1$_oK zR*+kw6Aw;A)dO9m-X_jDPH_6lRwtY}A&M?{Q68FjP?uAZj$MjB1<*@wUH_$9I+WDP z+oQkzIhk7`#a{$xu}>fO1r14|Zhbp)%dVJmnjDucAd%6!B~z`Rtp3g;CTkf*-Eg)q z4w0L&pd(rNg=a4c+&u!1oiv?kTKeg0PIMoSMRZ}*e~ssVL>=`UYfE%BjQ(u&>Sy~G zrW*gH0w>+Ap#~Y4<bxro)LJL<juVpK%j53K&UM;|C!g=mL<D{-`su4cU`}LkFL-qd z`1s-!K-BUK`Xj%*4JDYLoIeGm*K3{v&Xt}5@X{g{z&ff=cWMVo^;NG7{Di;$*&nG+ zj}W4G;~99*RL&_N;QJ|nkQ`=#08-eVbCOnRKPDGA1z;hAztxmpidcB_9?rsUsAu3| z3W?!V@QkoA)0-Fhd}mzq*L#%VAMSr3$=^sl-OQDN0S`P#Azl;ml_Th+ER^HP=1%-| zN!@@ypC|D0h831)x1aR&RPo|bO6*i58d~DcxJLY+&k#hr-$#7xn|v?sQX@whi)Ez> z-jZ}MMuYs{6V*pLQp+$1lezz^*NYgq566?vZ8R(BCJqykk~Vhh6cFjz_6L$-hmK&D zYyqVrj2uhZodVdyEF$GyWq}O%cJLv&EUDFMp796lLLudZRT=?3Pgp*dvL#)GPlgSW zW27UhBS=RSB7IOe{72G)A`#xa6j<rka0Qeb3MbTrSwuva(s(Nq-VU7jr5}IzhrgDv zwx5ZwB=Q;4zNH}YC<*p?hbO=#4L={<(1=4{+LTioTAZRLw9g?ZJ)Bo+CI&|H&h#G~ z;H`SkFQ^}78;pRsPA*S+gG>5E9QQ6u5k$p+Mmeyq$bc%2$4{B|u9gr%M~W~|3%{J$ z2Ji3|;B2R_3cpX*Wm9;4)lo0SmDwJ|VL}*@1C@h$Bh8z}B|n?;Xy9=tl!G#H(`RJ6 z9W-TShH0jLTuLyTeV#={Zjb8>T1j}cgzp5QYc~w;2kf-Ck15uuJDzJRitqT6|KxUu zfuB4rq^sfFbV?7&1X@>EvZXbhXR1EQ`!;OmXwIGpak!QAWMnHpGi+I|nKW|JDH09} z?1w?rdx_<0*TkPeV9Kw%XvQ^jU((9m`961Z#d!Lnw`MNngt2d|N4LDDmgUh;Ja0$8 z+R^!$4%#~u(n<L7M=};PdnPbHFQmF{IuzvYVJJq?FxigF1>*vn4wBv88yG*7FduJ@ zJ}&7>@u0ru-*`<;1MT9fgOUkaaRlGZtZP4^w_sg1C4k#yUGLV5?0pOBPlrl*cPIrU zoG^hnpPRz6n#6t19Iw@%vAn5#-(-pSL&*udu<q7Ke+rntjTb)-bb{~dJx)7EVw?%I zoC5ag0#5;|NEb<>D$YIpm80f8`7wLh%J_a0DdPw=iqM+4c_lI)Me-*;0$)$00>NeF z;aJy-Bu-66(S}GT<buB+%fpzPTR-?j&1%MRjX&e~Hqpo8gmGe&*OQtQ4O~y_TqamF zn;!D=viW4kCHLzd8TS=jv~TZ853Ke&C`sWLk(s%TpPtv4ho7sxKEOx+L!m!l)Q<bZ zM}R~GqZY)Rs8|9G8GspR*ThfBnb~+%hkW3TxuCD+KJR2JjNw652qOqN`74KX5vr@O z5Pe)!j_4<~!+@3a7iIOgwV9q%9EA7q(vVN(9EAxT`m(Tle;@a;z`iZ};AM>KM&Pe4 z=?&Tp6_?pRHmMt>QXlO_b@sWwGjSr5Q(Qxo;8{c0tt*CJAWwv*SG)SkTUigE$}kD> zzzi{oXvu);cII*KE4bCEWhGJhq{jLI>CTFAnG8^z^q?)D{<~tbC$^K7{(k2iRvG{p zq;+Lk6@C*X6Zk@}qx8O2*t1VC{VIzOA(s_z2%&E*YV9M4pw(taBNXXL1}vQLo3Fk; zxls+Jw}5qO;a1z349#FLIzHOzDUJZLXYQ|nAM)|L5`^?wXfX6DUTgDXVZL!Iw5YL9 zwaH^hR>6A~lfP@3`nq_jHH{g6ZVD-S5Tao3lO=NGy=E_(RTsW+x$7Bt&-$SK>`qwl z*6dI%jN*i$Q&f9#%K4?Z!6P>#-lN+IKX%w)6r$r%LN2vN#RVtI80fZl$LeGwc&&QG zULQVsXoE~2b8@N+9adIaIOya*FMG?~Y624l^C9{GOXu8iFMQeKzq?g;^5-mh{NVq# z&1y*<;fTJ!B)x8YkM(tpSrL2W(m0AIQXi5osJi%>v!bbasaM}F$XoF^P^Ma7tfB8X zFaJ9nA#RsY{p7Wq9D1tGZAE`*aq>|42RvG6OlHTTppD2UJH<03?rntSSkoVV_GZP! ze)zN0-FC?4tCy``o;mk`??QSE+LC)h$9$|wv*?egnIGaK3HLth5A>1K`xTr5Oa>MO zIOkuZUIaG7t?=t7yaOoT9<Gh5;FHddiAtJAm)oNphP3jW2<`GfftDY}9oq{E;&Z|q ztZFq$i}6j4S5vIyIz4n{B?2^VvNcIx)EFPYH?B3h#6CC$h}Vq!YR>E8&o@eA{1xO} zv0AQ=a5+DgK@JsVWzdT5*_XN!!PZNQ0`_y{JPqP4qMq0REZ^qa^Ep@_6*Wg8)FVRB zW$>bipADPevp0<LZor{&Lc-U9RfgxW&l|W&cU|NaprH8To!6ViAC(h%kLG7wlM80k zLn><wA}QXMTQHeHWVem#Q?b}dhWjGf9+vkh&(n1Vrl!T|dv6?`Xip<Dgo^K_CoG+q z=LtZvVS%mFbelv=@vyy{sI8*hG(|--)qr~{&-f#vw5Md}>9xIFm(!_8<4H2X;-snK z6mTeFASc^=R8~>n|7wfHfhCQ#+DLu>kv~VLcx<`N5mq6+N+KHVQ9RdN1Ky_#E~^*l zIZ_{S%&OdZ{bIWOcJ~eE)K}m0&Oufj8>vqLiCR;W?#FZz)ua|<`w0ssLV1?-=^OZo z;<|6&F>(p+yuVIzNzNGclxxmVU=$4Iti0xKA(b%txH%0*dzqIdFK!f1%Oxeo>zgOu zT>3%+4Lradj3}5CufZL5l*@T4!2+G_aSOsUn>})q%Mww028OuwF```#EJ3^e=;Gcd z>H2TmmB`8JrA{X`N`>GhVDS5A11hPp^>vMWd3!&B>L^&2E~XBQWK(aoE*sdgE-SO3 zF&6Uk@MhvkGYhxgP#_CiF==fO7*x10TPXKq5rJGiOxS;s;4ahfaD{OTDy0T9u)ec? z;pT?f`NA1RT}6k3M>s^-W}Q;an@q^KvprXri-sTdM|f2$S&U?I|DXZ`G#{otu2wU+ zC+8%Z&3b)RSKEA1u8H^lW&A{B6`J;%6-$=g0$k}7;9DQUpIThEu`s^bJ8##&hoFLO zHF<jb3Jgmt^6xLsd`k{O|8y6jd1**iFVB_DNiQ*6qa1ut>Hd03MAluh`zc2JN@(X5 zsOY80Pu0s5Pj51Rn<l`Y^f&Oq&d*ukxw>mT4`g++5l-=VbinzW40&vdBik>9b7E&V z){Qr_loGIo$I7aMOOTXFnVVKtyb;&i_>`qAzjVv(UJdfP<$>yG5;pQqKY1T@g+v4x zA+q7ZTbv8c96ZYhOf*~}BtXIb`RGJ+k*{5|pJcBqiB}*uEFErfd1S=I*!yl(zc|aC zS0@6cSPCj5Gd?ad9(%p^t2IZ^;QcQ|<$j{HSjNp1t=OKjK~YRc44X=+upnQJ6L)~N zswu8LsC=ADN4!pbj^o%7t8e&J^Htd!yMkAUW6NF9f&SvDBlu9!!{mj^z3K(lF?3e$ zTBR$jy){h&xA>`2jJt~d!YfzcDPYnsOv$nmFdw)ILE_B=YEWqCg~@j0Y9H^gq;_Wi znl&t<^<@^tywCLY6Yg^k4sbfR@#%6yc(eN_I@(F7QISnWn0=Vst#_s}rnE0_Tp$k{ z!VsB7hjV5-Hz@RdfR1bzYsj2Ri$2SI?R(d1bp<dvmdQ{Oc?gfSz|70lRys8<HloCq zY;?@V?|48&%O@mypx55V2wl2S6CFUQK<{8ItS8yg9;~1<b0H`8tF5R>OV^aubAy<; z7&?p64lmGV<*JsEMTER!;waXP$D+5GVoI}?!sf7$-<`G&-(N2lmSXY@<<lcxM_8B{ zLGo99iPam7krwmqwB)}opIY?@u*aW%4i)9gZ=xF76G#+r3EL2QrvZ0oq9G<?`FJAI zd_Q8#8)P--i2`^@&#_CVfQC@eaJ_g_?{Ru_@;ya`+WUIjk*YSjy2k7SVV-ey!m~ew zn`?O4wFB+Aa^Ca2F)#8e1s1heOV|GIh0W6!|E3U^|FT(HJec*4cnb(<3QM|bHM8M+ z&+K7)&xq6Eg>NTwON%@W@KK6MhqOEh)CVexTxT#fc^JeS_Jk%Xk92P_Xx+M%A?;@C z8o#6Jde+tCTTIK<5DkVvVWD|?bsaEDG3Be0AWmvq3y0n0Qc<xLn*t453ok6?2Us2W zMt(iW7gG{2x4-^iN&p86mzp~hpKR!O?nepW1~@hQjQmd6N~B?Zx4TwWzkLFes^i&M z)BDjX><J!R0iW{cW59RJ?daOxmK$D$(!hAy<fD^RR7Hk69i3*P=#n)quZ(3WG2qps zY&Dd)Csp#6{SS%*^Qx6><u5FPyq4=g4#kjlu(*rpg?SPKq!v!^VGUEi6OGXllh_|G z4t}XzQ}q*!!cr7H){pNs8f$a3r+I18n!yt$O_iVZ5%BUH$-#7i1QOSIsOD7paYcR| zs9l<CG_4*%=1kR~l*H-F-1;4dcu*3Zi;FKTA~-#Bcl=Uo8-zI0X}ANNFv!-?$VR8A zC;x;q5tTc%G6@w!@F2{(tky1%#L25dyluO-B`88l5!!-CWpi?jsF*)`kVLVK(#W<k z&?RwsFLOr(2pR2K)vQL;Q@47GHL@mUI#d{RMx<Z0acO)+t3GF3-UgYbt^*-8mwGar z9ADGwFMdn0C-+>Ch~4{P18-))9y6s=5wNSz4RUg;>1*^h79(zg9*FfC0pnCXP(*&9 zOfLbi9AJLPe1!2li+$<)t^%}&<YL8E2JUH`bh}v(Y~3LRa|IngfHS83z`e<G?_5f& zGgB_Tm4fhUqc~^^g}(8O419V^B}j^)|1(|+Yx{5Bp_&z4q!x|3=+Rk)k`Dtx@sP1s zgf}eF_vF<w;N0h6mFROvZtVe5mObM{RR>-R!*}uUdEv1oQc})z$fE95KSATuZ;XN4 zZ+9op-G45-?@zz=FGWE9eI#;dbDkyVhQlp|THPOgC--4{kiKO62OLF^9NlnsvcA1V z^xD`AbL~J^JX6Lhf+?<jp`!M1r}1pwKB&zCM1T=LPnfDBP3X2RBrv8oePv#b!%)Do z;vM=eNln*xe|2~>ry!rpYu8f57Olj)<L3aTcTtF3bx@GK&g~wVgjrym4Ug8WsDTk~ zaDH>#>NT20c|+~zSuf{@+rEO!pw(h2F!LD7{_&`3WiMj##BJsM<L%<WMIW$~MYznE z6(uRly#Z}Orb-Q%ByE&#h<NlYn5Fq+BdJ>@Li(h=I5uzSv&AL)803=-w%2KL49`$D zQPevpla>vq018Nq2gaDIOx9>ZOjHue4K>ZZ;ai>cn4Rkb^CBGQPe9%KS;se*viR21 zXiX=)OliuFCTm$pc~%U+7}WmOn?+iXb<1p->1Xxf-BW<_n!4=Eg>_Fy=BDLomO4;@ zsZX*a_(*K`<?vXl<BL_F3(y2*6|*iTRcETsv!sCgjpB&Q4C|^=10!0}RfmF^i2=V` zy1&}GfBT}Pcnsb^Akr=j3@n;m?bv$q!6%i{N;%M=1WaPf)xl@kPXRB8Y6O3*%8n{? zm#65LljR^+L10@V`@SGaPRDRg&UaS{|AQ_2kC4Or|DJ#URk^VWcGyqpwBC<!&~OSC z_mnb@#a%`&T<k_2XS%~yK{~6#b#F`DmT~`8w%Qmb(E?5ed2js8{?GSx86@h144x8i z;J+Jb7P_<eMr1CR&ms<e%lqpIb3JQs^QYBUWjd-(&;I(P{q?=wnmm63a{MPd`h%Sh zt4&pqHTg<(+sjfSsL#;`f}mQJZx1fkAXzLKk1DoDP61_+@6;=qS03xyOXT(~TG$YB zNJ1guprOy(z@f3lm94azr<eJsGSh8BDe(PWQie@_DI|0@59T}YLMbNCqN83Wfb1;8 zJI}xeJy~2wS#&K^qNFBhXfGF?;g|Ynk_&BuQ%VH~Ay{FTs%d(n1nza+qUI#k@T~wB zf%5Yg$o3`@zaK7sCT%ujK}aV(c%V6?>NE3r&t~dx-%gfYgLQT<nSCr#ln>hh5gs&f zFDt+hO`lhX*A9xy*bYF=tUnPe24#rwDZ0k0Bhldu{%CKdy^2C66ZvCJ??xo&H#wu_ zOON$2^pm$sjel_VR&~_9U<3DteZwb&y=tUbp3}qd!H_iHY3I#aS#EaAzdNtdev<-u zD|8~W(7tj%`9p+kvRkWxOsv>4$qp<`_w3q_>!fMX|C|+}-88(#oMlKlCE`+PrTyJ> z>{F#DL=$p6GHeI&mUIc4!B_+(flKMR8+Bu)gYfwJJMT7=(G-AJSkf)wIA(XNA_B+9 z9@51Q_Pi%MDjzgfe%xmnV3WATedKqWDBL^q3w8Wz<qwN@cgg=N!tk$$75#e!X_<3n zH=P0~#oH!LS{%2ZPhGuOXrDf*aSB+#UO5G1`1jIyVoEV&_|g_x&!s>Eo!;Jb`enW1 zoV++Tk?)=An`|x;6aG)wTxw(ax{93o3{U9s=hDNxWi5+Vek6)z(N)R`k^*#9+=%81 z;i3i7UG;yR3jR7ZoXLl<%OoL9ezG^D+xyTRh+Q!P99laYEPu31o*;7dKK-!iE#XxN zbJ&P-U#WbN-+r9^B0w=%xJCJSb)9B*L&9t$JuFJmX&U78wxQ$mgGLWl-zhr$%rRtb zY*xVpSJ*ryim}kgAf3dVl1Q0VF6JkmfJ+F^?ngDfRoy*$^N8R^Mz?9eG<3jbXZPSm zkCh}y46LhMGrd9A&J%Wc@D<KTyfr5Z)EpOXt%#qJ<QppQeB<>}{>!?-aYlPF9rJ_n znZ}kQt3;yQg(c;>G<)S;0qGXd8JKGuNpj2U+cjfHx3?IRljZlEg~e>o?95ZFGQ-0u z<mP(chP|hXjfg6!gJ8jIU|tZd)Uwq8Lzg4U;dZEDU{5T_;1uAMSFWwZ1q&cRF}_y` zVHze|7iDc?=$>BU^nvs_I?k3=giQ?#&Ia&NMx!&H++GH7ozpaSIgngj_sH^A!~@f0 zmqGM<LrLI{nva^Tmdh?=L<mQ7GCE^M`K@PgIbL^!sX&D#-LBY}W^^1aC)l#3UjJD` z!5TO!y&h-rsH9@>g<WZ5z^!nj!R+V<8F7RAS92d}8RmC)FWFprE28nM()R1<aJdY8 z19}kk3lxBb(-m(j0b#=+a*OT(n7gi#KyTiAVsDk4m#&w8Dr%6H0G#}_j$U-=?poGf zUH-EW`R@DwhdTIA$@zbZpZ@n7_5FhYjx}Iu#1)+gmgfU4-Vlu>Vx~)s9YAfM&+l9m z37q8S2+g@^DkIoH^Fgv$rV-0icA?m<rfb_V;#JCt-?mHzS}YhX%%r!1y>(f{nV*kD zE%mR8&s2dQwfstX-$t&0*Gs9stVXSd2_n1h{-q-PKZb$7{G4Ws(Sb)Dw_i~3?y`|_ zr)%K!GnSLM4sY6ik}XY%?Rn3YX}5eeH@0~+4MJi%cB?F(C~So_tDW?|g;$VZ*#SMQ zrY@U_U67%e>rcU<XT=&gVdjSWLGm_LSIz^jN}r|6IQ#fGvxZChf6D)4OK_m5#R>fd zH!ocl5~8&_M6cWxjfq5)XmqdA`)k2(J&QVV&{p<}B!&Ib4-%VKl34}uF-d2yH&LXO zcM&1gnifGy0x@?pv{KY}I9=uij}4z}h065U9Vs}ll-PL#L5AZmn>t#_dKOG!jX{T8 zFh@+m+ZjYK)7sR<-g2*=&~ygI#S=6~lD_DRH#EzagU($*gIyfZB;8>K9xe}m1FgH` zgT2UVP!fY3+GAlfGg0tSxd|wVP6O}CeZ}PkH3-PIXtC_HG%`Fh8?$CKw%=YJo_+Gr zL0?j60><gYleSlQq-bKehN(fk-TMI68LV@Dp<IfZYkV9rGP50xuDcdPr-*}5NW1D6 z@|Ha~^jY!gZa$*!9y<k?8{FDr^S+I%3?OD;9c`yNG|uZ@W!IqG8}X}9#D|UG5N}^W zFRj3#xcw|B98zl!Gr6TLe|<G?#1HcIgmdjgL`e+aU7bZBQ_%DhE0xY#r7Ki%z^Mb7 zxv9B%0oPq_$^qjBVXRtse?pT7_M&=|{j3j3hX!v$c@Vk*3$`CjgoG{x)}BRy+mk!w zb9_IMH0sXygi7$vmr)00Rw6~7J$C<~^|~rqRm=!E4aty~dXtjX8IbCPR$Q67f#*sq z#Gx#P>m)z%lIj^-Q;l<z*;UupI9w<f3_%&^tDvKKDxX6%@iN7q(T739rEb(6Us()? zsz`3#80Tr};AdJOW5P<t%9qersPzob&mWuBMdH^j052ue`%9g6_a@Q5PAI2nvqHCW z4!X7_|3O%<BxP2;WxVkcEX8;rui8s7fu;2OqPg~CtdOBFi`tUh*PHX0OhO)-og%Po zs17ev0L}Zle%B*OWZsP`j9PZ)BFy3~d_L5h?G*62IDuqLD1WW5qh93Oe>k5Uc7ZlL z`*!@b8Tw$A1UD`OuiaF6e5te9^h~jBSwQT1Bdb-vHxKp)HJaUSKP0$ZP@4MD>r(Z1 z-+b8a)8vi#n_|M1L}9Eg#wmsn;MMXlJni?ZZ-K5Hp0e@CvLnvN@$#S$DR`)HLjk0K zuc5TSsBm%x3mcTb@-@Aa+}AU5w0+K0z+XYr<M@L!MYD2<8k|nc^~%Z6p!5|gFh7<J zca$CNLMONXbw$%Bqy%$#(m0b=IlJMmi3cVCvofad_}K8@?WDH0cBAM$0@_dUR}Uc- zBnrT@oqUbMEc{2Jwep#DA^tI~G_-lq{dHOUR#A%=POjYX;>wm~SnI!VrKBgCV{VaQ z=|eTA7nwQcSD2T#xNJ26>winuH}ZrAFU~;FY)fOuX>En)5rR<gW=$Ihs6b`1VRu?r zmg_z4aX;lZ^qxEB1+!YlT3*dHVkiaRy~<%{(>jX@v5e4T2ua7qO)6M%R$7@1iEdg( z8dIxW`~J-OuF=Ezb*cui>5->?PgD9OtvOH#yLPD?l&ik$Q(q^i3MJd+jBB^ZD{pQj zJG<-t^sYQ<oNXOK9>13F_|wFs^0zGf=Y@|Sk^LjN!{1BYzZ)pH$8=lc`DusGqlENk zOTD%lA#o>p>TmF<$>Tdqz>ons;D_Ir$WLMJYlFxad~k4bP$(=%O~Q1nvo*dSQO^<8 z>YG`^KC8q_(A4Rb{;uFLV@Q)QtG?gxK2<D>a<${ugRFO6N2$TfL;<X#Y?~)@12RjS zzdX@}?jnb~8dstx{Gq{Pfvza~miHqh7KO_7%zr+If{a%+nvF|^=M1GHCF(jsQ4Y7T ziNc*6S4`OJmZQ@QcN9}2j0`>FMDFZ<<BIeQ@?b^paCM_*$CWRNAn=shAT1ff<Rzoy z*~B!3B0E&0;}GM3ZdP$fai6)@>~tuOkI-A^3czi>wYp_2jEJ9_1GUrSEnM@FOq;-b zETGAFACPU$FjXM6&!pufdTEsFYGj()V^ua=D?f??>80d@R<%KC3ew=$QpIY9r{;>h zW<6$v*y`k-L)@8;s$=*rijMVS@V-rf_Lc5cIA`<r>Q3zAVUzr*Vg7SgN}D6(0}F~5 zJ>6T8j3fpB!tgGHHTYKHntFgt;yeGur!3q|h>y~7xX!P=1lfmGknq5KNaZ%$9+xWS zqC&{$UDaiZxvAR3U%ebI@Ra$|V^+Q**AuG#yxuylj1^fZ$AsZXizClHjl19cHFR&^ z{I+?{#l7LokVu255uvl$o#l~=J7{#TRx`w!l~SVO-dGVe!rP~>hrj;rZ|{|yMu+zn zeCI3@XdaU2V15=44j~$RTARM;rge>PXhPE~XBGd=F=BE-4Y8=N*bX=MgAkNk8U!mb zV&y&TFn_4H?Hd)~hrF#Q^r&mhlgkSvSGz~>?Qg~Xzk|dNdZA|Eqy`%MG#SK0JTPPC zgPs~g4{}mnF$zT!%x7bH)cU-8qm`2$He-;pd<wYev`%E6qZD%!+yh-A#IvbR5(^g8 zOwl^@WHI^G2a2!_v*}=O*=)kIe(bnP*^vk<E1lc5pHW9S@Ve1w_q*k3h1Gu5(${1H zLSife8nMACx{EuFa~bo8jV+3{JF^%+Daztkc#qkm@mnc*_MlF;H}p-kT`{g`(t6vV z-Iscp4<A$+UdNPgWCX#FrO3@RMy8Ezip*1txC>FFA{@3>e-Rpn6wQcR1e0yXK9i2= z0+MG6r<6terH~Mg##AJna1V*LrHIDd7#}x5{ImgXCC+b?mW+E?m}ZkQqqjgV-9-Xv z^9N3n815qD4E1t@2v3a{O5CKRj12vb2G5SwXa4@*fG~T<ZcjaWni@lD5G_a3msH*I z)$<J<WHsENw<IJ>nUyTxI|3M;BbW#VMsw-3MzG2j_d{SV$9iS-3tMnqh@3uf#t%+4 z{5$llV%pKgvssJM`^MUmeyKZE>Vks$Y<d4#W!sZMO{0b>z02LrQ-&|xd+I&a1Bl92 zvlcWX%gMSoMo07FQ=*J&<CRF#O*Qg2f8hP*-VSrZ#Gf09_pPP>1|9I9!>;Y{Z@2*q z<hz31jH6zQe0fiV6317dhly`tMqnrkQvhqnb;zw?*4|BZ{vM~3Uk_a<-1L|ZRCtm@ zsJ(A;%&+_Mwbt3d7ZuyI`D-HxOgYp*bZ#p|Z_=T?4M<Ug(7+qhO*w&<4+BU^HMv3e zU8k(=tLcH>9<*QbH8R#JVxId7`S~7cRV%}ft5UMVJm=wJkWA)FWp)ZHRjX0Mq?N|Q z=HDv`4K6D+kIUBt`I4dlz}!0Q5%#?SX21}}q*{m9nh<yqGk!g6SZ1Bw_NVLaw}b34 z6^+t<L6kIWI!mz{;T4L~NPKF$*hwJz%ke}W+rf$!oSAsL&({au^k`M)qs?NU*g?th zRd`Y(m1)ZY$z#3b%`vA%jb=fKg?RGCi1AmxyOhkdyzIe=PfOeD)6>S52ZRacl7N?g zp&F&|Wb2g0+{hFZ>1qd%4aS;7+^|0&?VfMOWLx;2lttFn_AqoMRR#`Og0}lk0oos; zW^cqqEvSi^u$xQZKMZ=d`m*3`J~^a`RGGdPIFJz{Z6IoA*-H~xSui(fmq+lx9b%fW z*u%{QQjB#v5_6c{vh?m0;7ozz{@RSVvn&CfJ@DT}k*|OqbKeToRzYR94`(*hR<+kR z2&uGzO4+d^zjiCD&XeWIy{;qStulKo(E9p(r}+R9x}njNpkVpgNByNwyL1!kD~hI- z<A_NQFBSyUU|6Zi$}HHwSUbIgGXCvkeWbI1Ctko6mGD3krmkS(?nA}(2Ir#<qvpjX z4l(fT=gAf!>sFt8caI>{QuC_N%c?yt&Vxs34jccW_7VE;I_;m{lm;U=2S4*X?dAaE zw2^;2_WU%&%W2B2jpUp5lwFl;=M403XCD3y%=t)1TRTWJ!L>agbPFk{%)XhG9v3_X zz|ZW0o`(+{e@XX_4cjcc==u<U{sY%ZxX-zbtlb#?Zn_MvJNpV4>#sN7eWizSAE2(| z>6*r626=~x6#1I7W^PfYWBCvhg#DGH#stjS;V++t7j<*f+Raow${KFql>@<Xq-`>~ z<00+)Q}2+@4(bQl;6@NuZe3`aq0ZIX_wuS|!kaxoai?NtQPJe#M_nZyO%z8(WCE!! z!ZuPveX)VsT^$WxsBj>6?SP|`Ksin<MJQm~&;;V(dPb5x`dg$|G)DQMXM955=!!o5 zsC`8*iSJ))6~;^TNhzR|$ToWfrG{B(^%ce4(|`Ad7tXO~dl=7qB3-T}R;0E2JNEH1 zO$o<6w8?6QBUK@z=Az}gPf7krCg+o4%f9!!^PE06Ng-_|QJ_d)wpWU6$<TU^OifVi z@v^OXBaUehgexwtYcYv{S<TtL%^yD#pT-}c$uXq2orE>(lLKaZH$EJ!=jt#n1g8Ja znEWrOm1$)dkyL5YTo2J1WYpZhW%Z>3=++wHNyfg@(?C(rzfslS(5`1%8Byn!1=H&c z*whd!*C135ycqaVotozV$hD{7Yf&LYIW+BNQy3pkE%+3$Jv@7FETV8d{k6yuu_gb< zYhN0I+bk(0%6sToS1F%8C(9nUp61roy4RVOw0oKP{iovKFF6xfNAHW+^w3<n6Yl%3 zlxbOLYtFr|j&~mP$<!G?jytAAnSL24DhucEx+pfxp(#ku`>5GphvGA{;_tDdsbbIe zJKvW-hYYU7{s$*j{x@j$U%YYLvp+ODb29u^6Og?hrNC3XQX`!cw?+u-;7CFybWAfz zvUC(*ih1IJhD<_2N%;G<bVhISVbQW7+d*XW)Yw@zE0KqsWSh6SeV*U`VJytrl*V0w z2M6s=+J2AAtbHR#P5@<z4_+tX0Y7crBo!FB+$d?n_fx<X@sKHt!!3e_<CnCQ>qm{8 zR?I6;=6$70hP>VJ77zQpB?|b{1%f|9)(YDznX>gd{n?p$&y3&U8A*Tew)W~wagS+c zl?CCh5<ifxRg)Q*buS|Qm?|4FyVYv1W`T`xjv#6LAv4N9KKYbA*nhb3iKDjG>t;ly zYui~X!otg}9qBIdrnTBB?>l(%wuBHuO#iFW$ojC%G-dX*=^1xF$2`V~=RNtRR!Pv} zoYeGuQAB^v@Op6mfz3rbY(k(^0>cl~n5nil3&&L_Gej;AMLJRk%t1&u{;9w>!}aV5 zwAQUK?n@}VjNB&CM-Bz<B}!*3U#Ps-w|lI>5V&vhDk8mrJbB_5%t*74-#9yZuFqQ@ zZx?z3=Y%zALTr!5pdo|f?Ys6VOO~tGg==sIe%|7<lsmJ>Ed9g2cdEu4Z1=h%<CPy` zYyRMYr8sUHYy6F|?yvRuf2Lh4RgFo;qd#%Z{xMwJzn<3lUj|!E-cqNMH%YvT=i2;Y z33Yax5!F^vT)$qt*q_I1tE$uUe)8)IBG-pPbMVC;gcAB{Jh2$bZ`RkzbY351yqbM4 zMkY$tK<%87p*6i{fN+<ac;M^L<~{iYYLj|g-1?6iL*!#kd!E;#<lc?(6y*@A)EY^- zfyE&%H|7?S|G}c-c!Iq>P+V{GXQlp5E8%3HDdOl=uCkvJ_nHU~9u)O-07?0CrWYx5 z%ZCr11Sgv++y5fD|Ij_E&S}hT4Cd9mzhr>-8H<M6v0T2>e)t`ss)I?jTlwLZPD5l& zqv}oL2#;?VhGxrp#_b%hEIC!30<2$=<2N1J{fX4Gr+~Agb@J!J+C=JlT(3+kEDcR; zdn|Fu%}}j9C8_F$-A-zK0dN02*WJ1I@5mticiXP;zpOv@Pd(#*O3eJXlTH7gzv&k% J4|F>EKLE4eUr+!5 diff --git a/doc/modules/cassandra/assets/images/Figure_4_read_repair.jpg b/doc/modules/cassandra/assets/images/Figure_4_read_repair.jpg index 25bdb347da5384b32fc0d5db7337065288bdb79a..82c65c3b81535fd16246cb2d5d5399ee8394c7d2 100644 GIT binary patch literal 53143 zcmeFZXIN8fv@RM8q98_^bfN;HAVj4p0$GZ5A<{b$5g{N(1f+#zNtI@nAfO;-Atge< zNLN~<FK{8!AqXKsrAs0ZrlxS`+UM+j&OOh4o_+8BbN4BM@5xN&WRmffG2Zcx_sh=Y z&K%^hwYil!WY?};5KHg}vNH`a{pZ!c``EpE*DmmP*Y1CP{I{ds|J>xiyTSkMvVXn& z-+kVB4-wzL>yN#fdv+a%>=xg(M|{^#7X$`@>=ODf_lEq}$FALb_6qG2-hV(u6uhAR zFl6_xJ$rWV-6JHlcQ1H#9C#kGS6oQq*x5__B<+2LkB3T~d+@kuzslw6c4>zpma5K; zd+`TEj>yQ$$*Y}EKY2<+S5M#I`~^ePD`w^vmR8n|PR=f_Ztfm_H~nt~1fqk&!XxfS zMn%UYBqk-NJWNf?%6^iQoA>lt{>$Q$(lXMk@`{?;y84F3x9^%dI=i}idjIO{A08PU z8=sh*qA{5BU%q}@SX^3W|Jc~v;%oyyfBj?EF36sL>$3l1*#BUcIB3`Iy?giU75>Mr zUArT}wnu!g(6O`oBre$t`-Vy$Klfn2)aA!T)$IpVbR1aHH|`CI98uL}sImVs?O!bW ze`i?yf5@_bGwi?FMS~pNvkM%&J>n1qL?Af%G7a*7+W+Z;|9o?xrhe*X%9ZgG$rfpD zbyb($N;A`}6d%*B_8m8TPELlzU)X!o)dViCctS=p?YY-{GRZJgi<an>?>fAusbcLk z<55?x9^hi17WXdSm1par$=iY8gat!X#Wv*~NV{r_*bYSK)W{BGt`<>$tD1{j@lj{r zDx8B&4l;Xt){|ih0%>drTRDkqwL%oDLnOqguMSZFdAw{;OhfBjp^EJe#QM#+vY>nG zRK!KJR+aukd?l48DG<?ng;fA@Sdp9nAb<<!dGX74Ah!u=sQuIP_bFTiR%HSw7iA4k z+JT6LFRgBd1oWUJP1LZy?6RbyC<}}1?g<$WyC+#QRuScN^p8#{%geeilGP8f>e;eZ zeS>m|T{dFpFH7b}_*s`m$IAbaQ9hn&s2em!S^ct*REnHt7Fj@Dp)NI5c21$vuli^E z)4Sk-Uy7yPOg_E-ZDFu~sp4r`-IDL~x~@@T&7TV%(x*B%XVZ9rY!dHC_FQ8;b&g0> z+<{=kT<5mC2Q=_g1ZF+uJ!lyxl&y<X7-~Di(*l~IU9f#T1RI*L74T+^vRP&6(U`eu z>oWYK=Ld7zaTs&`ZG(TV-`+TCI|0zRA~?&hG*QC+MX!UdhH<apdS9Fe_;W}(o;m+n z#9&4~c$EZzpEpln{TiQkVc$tz;2i_<m{8yn?wC6e#GyMBTpFFi9Bp{y8y@nFtLy4M zT<TD7IZ~2iq5E0w$a4*NdG8s!oGbi7XQPAj>19jSxCv-X-OnO9y&S7p4VW8Q&Y`UI zyh&icPGY(1Bl(&_{TyptulRX;%AhP)K08t4$AxahxQOOYHos8-T_o<o&%L6vtO;rO zgWx2X?)<gBrpNW-`Fz6Z0r&7U#Qb+Ei`*lxoy|SVyTaaAP}<A#>_SOs^Hdnk`y*O4 zI@XR2*V&vD3_+fhW9xC*0{RX_K>+6+0u-2H!Hj^_L3SSJVa*s%XMP$E3qtG`Xz`Q* z(|LNrtW-TKxhOJ{tza=OfJc$8rp-GyB;E1p<(|b*s@AoRJ#n{gaU~dyXPzdX`#ZRI z3VE^%Df%!S-;^16{f+E|nn(X+#KDdy0X|iRXG_Y*$S!9}WprM@karF{ZL%Ahf4<3S zWhBdoOL7!X@0=QGtH&fhFSe5`AQ~MLKGo*<Jw|h5nxn=qGLa?#CtO#5a*;s-9iYgU zb*q!(#+vA!LG{w>@Q+YTWcM@*vZO4s14$$}vMsuIAh1ifN!_z&ZRLlV3yv+xhqcuu zqQ$Sjbp1dS!Ozvk1Z1?>a?dT}rSUh2v=q)g{&Q*wUNMk(2=5Q21v4`j+Jgs0uvggl zjLlXH55T_^7-Ve6x2Z4VP61ciG)BVvrhm)x7=8>_UY?FlKA_Bm-+QgptJCH0R6tHG za?!p2)vnYd=albK1OKn_Zl_WQ*`Px{6XFh%q|1702h<iEyUe2_4))`lJ10sFKllW! zE;%IKHnS{w_ti$su0-QT(5qM@;|m=TGv_~Q3C8VF&xm!@H7+}Sj@}Jb#-0b@JCMCg z{MU$}!ix-Od^k!O>jDtxD1b9wq)iT;k836^$NViX+=&q9g|X==gS}LFkHsdlE?RCO zV}FDxX&^R$G<at8wfr!wit10KAsNKAB<@AtQ9cQG$ZL55bx@!erD?<NP5ekDz0RR{ zxGnFte63p*Y5VBg&6@GbdWF~zHgB0PqFk8$$P>}bDb0;x{+34l5_fd5Dac!%^SY)~ z+Z7#aNOh&GR;aBm$7(#!%F{Hc9EXWph`Rr9?YiuIwDDVpO3L#!wWy8D=PPwdq8sfl zqyg!_32`s~SP+)|ct~f&xc1Q$W#L0R{V-y#cn1PS%vGam7TO84bPgGao?lNwDN%3H zX(Z+oSnUqvJv9N%8ev3s4tO$>DXeWX)?aVgq*Saa86Y#;D#CXlvL83RmLokP5?zmU z*ceO93_f$Tvo13bd(f!L4(+m*#0%>yh@xaPVtl+dic>1*en!JZ?e5--f9zf%Yh<pE z%$JR36rF@;@`EKU4jd?1Q?_)bAZb3<R@Ur{!Xdt}y_vMB$jO(#|8LU<#zVd}59@h) zhHt398^Kh~5$=`!EUV8wKN9xF`9n=c7UZMV?z&f%XU|97U`=Ldna?974~%4IK6w%# zb7TZfo>~eY+rHZ5tY%$mSZ&q+{7DX{DbjV?_9C$twz>{Za!24N*BIY^4&jH-fQjxv z{9#EjD0QwHR!!{Pfk^W1KN{|zpEDuO!wIu8ZNOKy@`wj;jBPXD@St*_EQ)Ytd<POh zmGvwuI8nruu{|D#K-PN}>b)#H&V>m?px}Q1fT4N(^asWuPs*T!UW0rHMo5@Y)Ck9! zwF6n*W>Yw>to1dg9Y`W-9?2{3yJc`>fYs|7B=n_(C(BzhAzdN%k<GO_yv$t&3 z`#o;14#OpFkq4>6RnqjduF-Bkjd?|1bH^Uno(}OuZw=?O=Pe#|IhGzM|1m^P4l>HS zSL~qCrW_vJA8~?g<w{PkauIRt>A6-hl64P-iq28GG18_Wt&vr-9z9%8l=iDzb|ew; z2nC#KT>V0<-R3DwbFruat^yFtQDdXJ=@~HLw&OqoLy54a#w5`2-SQa&Z2H`zq+)=? zBzDE!@g$!#IX;8_b=3+VP_*nZ`)sfNc(-0~AI%rG_$YxD##!dqQ|F5DZzyZhV1)c@ z98o>djtA89l5oZ=`W_8M>>UUrucykB^8hGkkGwZTc3M`p!Q8MH!a|F9x+A)odRu0r zSPS&aphjXy+N`a7fB1|2)Zn%YSYZIcwZWca3%3&q2xyBj+lGNj`!cM?hCSdphp2r^ zsO<YQSYP?rQ2sC4sXsn&;xINNul*F>sMVSuotIt_w@^IWd(c|q<YSf0eRm(fOfr*O zmD&Zkyv6zz?YFYnetX;2di3b$QtK!FIoE%@wKf_U<$OvK$vkBj?i(bLc%0E+=-$V; zJL#blPCwx*vL%rbaQ>F`tI;ZJqw-6RWIO4=@jDt|q{?z!cv1ly!1JU$76c*%Y#k1r z{|7IQWg|FX!9=AIMG(GFn)*L8um!<BMCT0gvs>h3)^;&KAA^-)8+6+El>nYSv+$ZR zoT8Q3c3OU<xApip7jLb3@m(6$h|4>WU_u7I@lFvqY=J!09Y}mAkiZ=5StsDdc=ilf zI%<y=K8QHA-tJ1T&D>6J5ska~BM2{U<VT)fL`aPfx>5CO#6%WuR|yv?{1p2J9Zl0* z(7XwF+1Wn3nX}+fb?_Uhu6yi`Mtw9=G`anTWM~H3(XCQjB1vPmC&%EBtC3FU=##Q- zQhK??xz1Zpie;VKFFBTq8rX?QlW&}C^?Y6^GPc}L)csOA@WI_V<3`>_J@+8)Bcg!- zMpr<U^aw<&dH4qP)g&^m#~~`1bE2A`({hZRa1Hp$GM$ejbn*SV5Qlg+bKa=#BRDDG zB2&p6zt0WrdS<aP%yU`h_V(jo6Wfv=%M%e>(8(Q051r)(gD%2qgjY#S`eX`2{XJEL zCp^p(pM%m|Ic020&)_beJrKe+VBk9j54JP(kW+b#ln%PAJ3z8x)6-m$!|Ng1MxG?} zugGMk%o)1-vg1<~%#4TTqJIY2=#Hks$?lf$H)EP3=q6{fuYX#he68(<`anik*a+$T zwTiMLhbKj{-O-ObhLUAOlM<tGMF-7J8(%423mocp=oqG0ONK@|{Cc5W5@2svrBU~D z$tmGUw!jy08#Ou4=>sykQG!urO(2ge)&~N5X#mRhd%puYxQUqTb)LWnuSw4JwqvFw z=B-&FJ(+9;TOfs%;>Ff=Jr_hv$^4q<LF@v(3D1iB**#i5u33AjDr5(84KbHNhqRp& zd_b|TVx|;0G5kMU#NeHT48(z!U1Y$uM{qd13wM|uID%oY2h-MXe@p377Mj>HhF@!F z8pfQ~F4o_!sYAz9+f{q1z9B8mNDn3#bj$@`zy73rA}ZjN?i1pei<aa46g7hjarej+ zk83&DE2BXZWCx1TLR5uRwR<v5C;x7judmN@i6_sULNyPpWi5ndrANt2B%SYbC_-OO zJmrvAYlic{-sC@T(}|)8L>9PGxDlFg0*=lyV8B+%TsNLRkTg#KEU}uwJCOZ5kW7;k z;K2b3R{Z4-q(d+EwMAtRJ*1xXv4M`R9{C=k)z;ncJ@$T<xcaiU^phLKY3<Aj_8Gh8 zcrJWJ*2r!j_CsWlTSZo8e!XRTTVVNR59hS1<St+TQX`wrcY16%naKkFa$l||CzIr6 zE{=rR<l0)RN!dGHvp181{BPHMkZkIFWBdA4ErXbfJBN(|(ixZ};=vY`5T%}0<_<(A z$bRWpxP+H=0}c1gUs*mb0M*)>rQnUQtx21|Hj2^qt*vf&+mwG2^&@`)bcU@|v_s<# zq^5Codk#6}$q-vT;SHU)SnDeVQo4y!EdivNn%>~o0Fq+~xXy>jZ?ylq{y_chgBJ9> zv7~F>V1tZ_OaoFU(j0amwR<pI%-s2AW5b8BX0?sLg{~WK0JkuvMumMWDsnMr`sWGU z<Pa*!fp->klaGQZoI;d9q>-&@+H~-a5U~?81>+cgs@rpSAv~87z?NZP(katsE+WTL zBg|Udrn+c}-(8%jeq=Dyb~BMot%r*WMp3mXYkq+SUW0Qe8k`gN4p3o=jwkE+kAUu+ zz=>fSOd9D(4a!|QQ3QejpTMi%E)Q!wBoF67zda@xjEz<e*@^KdCcfOYF#e#Xjk+_B zdo5wJ-axzvdpqj-4e_1`XsCA}<bWDf0;+BYlB9>34CZ9i4`Eu&ggfDcHVxKooG6gW zm^pd*7d2I|2OB;dfdb6i&MfBId;LaBDv_Lw%C@?EYuxA}X*xPm;%n5lK-3HbD_>Z- z)o~2C5D$l>dG>OUoKV&_$B+f@g3_kDC6|>DHDB?+o5)SntsMalw<nWcxs}{bQhlr# zLOJ17(NSEgaWB(pl{w--4Zaiem~3~=+0Wrz+15AP^V1W4Tl#6dAc8<7g=+zPpP}GH z@a-DZAkRz<i@?e)3eAQ|Se@qTw}-oy{D`^*PknKkZBiKco_wl_wW$+wTUYdWU2t|J z7o;NEy?Fl5S`e%~f&*OEk>NYXkczw*fX}rEZb*>bfY%X)c~U^MpGxEp#<*+*%Ha2f z*DqY3j@2d~DkZwgNN@M|s$Cs~lOj%gX*XNh>_DvEab3-D3ZPGgVkUZNTZHLa4w7|s z8=^}?bfPBrbZxuH$0{Fmk#Fq6OvBt>tO;M+m`-?6Vc~&fZ3{%a@x(P4pmD3Z><*XC zN)q%SYE5z)C&1e!_Fe^pcZeXMEoQ<i3a^(7M8MoTH?uGJ6aCYJzWe+^T#>>p51|s% zSzy1Y+{R9$08LEq#lF<fpH}{5@~g#?$N}HG_a$(R(2=5tgw2~G`w%UQLa$oX!3bb6 z@|Ph9Av)V*UelY5g1HB<@G;YvXeuiiYsdQjeQK(9T8a+6MGg|jb9e}s#9ChUx;w?q zX7eh4=q2^A7B3XMNiqv9>dzh#9Qer|{&e<_?s?Qy!aVhTA^M$cgFLt(b|C0MnW^3* zLxMMpx2K;reD>8WspmIR=gLgPMKSY(fI9I#GPO_q?<V@{7B>_rG)4>UrY5vq6nw-< zrwwCw-={vjd9{IVAo|U~=*R7k#O{P_kPa%{fs{0an;Luw!k&LiVf!F{IasXDNCRJ? zyd$RVZNe?Va4>N`$!|m4(zNlW(r|{OI3YKX(-Ju?o{SqoJi`0YWwF1-YKp%{Q|eaK z9NwQ1KF?L*;K7fagzLoDn{|0$R=HeP-T|P9c|oxo{$LaKuuUn2X_h#*rvIzA)7a$) zy-Qy0uj?Pa&Z6H~y_38D)$NeTlb|7NJ=*$7v$Zz-J6fr3dAQ<Txl^X)cDe&9<~H|8 z+R4g+-DxNP{9nFR&{6K_T-W^E(&%8)5-qN(dyCKC>T=0-(RX=l+&%X7>sV*y{qgEc ziUie1{SP0yKZNAhqY4l#H@x(UKnxrPAHnA)V$#;dDoLL)d;n1qQ10}sj<A^#Z&UNI zV11u=uf25{H--*oKKrh+f5_Tra{nEz(o><5Rt;G*$-VFK`>yUlSl0&A<WW2+*u3=X z(h44IB!av`2h<VmgA%mUJ_;;o$Ag3S<t^6yDuEl{#5u%AK8Qw}Do=|xszo!4l(3Zv zw;f5!<O`bBCKku8M>%%<f<EABF(^WSr(J>tt2a?sOrb-dIIMNaGX#E#B`6FjS(1-v z%Cd|XGZ~&$-Al|p=DGE&O!R<_P%`QAbBC`s&$ga4^=&tEb8`RU^~$A}XLLDH?CH-I zo6L<RHc_$7Gf7hR-T|jJCe}>3T3AVbHSQ3QH;3t^3cCZ%?Iy=F0~+#fy9kc62oFOX zkgFK^C>5_jACy>BC-Q}P*6@$RQE092DsOAwzH2C0piD~kwy6w<6H|8})jN>xSkxgJ zHkALzRu!}}hC^YX4ld4$v9`OZi6$2kT*h!hSnM!Xi}7K<?mE;9Na@ViuJ(J4=Gbsz zq0&(bsKy6=wMWuXMb1A5*as9kP%LN6RD#=xf-f)u0;+KzVRlXv=29FKM3w3InjJ_+ zn-YCS=nsDB=Fc{5o=yB@&GFyb&bA0meZOJ$txDRi)aZ(H$!A~qx;Zij#XE;44TY&S z8=kWISZFN~nda~uTPhv719@PkJK@^@3s$UaZx;TXVJ3BL*S|KOJs>ye$bW_ie0mV3 z1g6sT1iZ{s^Xm3kHk8qu$a5yUMPJ|VKfG0ivd&$%dY7^5>p+N+L(gw#n;!e{&8`;9 zk>tAV;Kf#ycR6y_V%8F&S<8jQSpuPsGL_9#Z~tH^!@0~S3oZ%<S#IX30j!mt*`v+; za@_7g#ei*(aJrZP5#&);HOD`2xA`^?%B1W&O1nFcE%=p=pB~|?0Ec!9Mqt&r9IhFl zKbqSLP`SH+KT*I-QPZZgu~mXTy5d$L!o5~xk)lRTHVU3E7=JVut)FFdhqRQh+1Gv3 z{l3QfB=<mD-5QLS`w@}y_tu%IS;V2?h`1IRD=z(~K{qh{*CuQ#T_CcHdrUB?vz%}6 zo|tK(Jb}eCy3~76`>^)tN)X(#=Pvj;85{-{=2E)64Q$xRr11-iBw|FVT1jO_$oCUh zFL0Hqa@0WR6qJMEzX}Pj2WD8rPMr2Q&v3;c(I_^AvjY*~IkA+!-1%Yl3TpyXGekc? zC8D6$tO}pomMrJqapQF-U;1Wbz@z{tg;#Tr6W$9>gK<fiWzri2V(1!x&Qy{FGtjkA zhWA<hSB)&0HBqGuWkq24hAUnL$^K3(4w~3~7$Z1_RJqCfsFQN`Y3$&kagF@NoHFX# z3}y#%r-Ao)Gs^C0hijcCDf!i}#wY|g)&@Il(~P)#lJW?D3l4_fI6fJ^wJr>(yRo4U z1iFCsN740%#9BfR>X0{22jH><!(X$`>)B!$ocd#N!5#PhUe0Y(`E*B*a1ixQleFUM z?+NpB9kf1|9-0-JHDfHlWzi~UZe^`yeQr#z7NpMp==fjKe`~T`IS=@4ZKnmFh~h?` zTy1>2-WK9eI(^TIfsYiw7=MR;c=7(M4xq<HtmJlCkg?~0vwwn!991X0rXHe@++4qb z>UeSZF#{#xg$g=73#$seMouNm>mFTpdeMh#mGJf)l5?ThK<%8#wvHsRX<nWS{YrfR zm|WNIKzLGn=J2anfg<ZpGb~l0h5NJv5wBXnCs1p8+p7$xyXi_M>Magddz-5AQapZ$ z;`af&#{<g@iD^93h|;LAtB=M}b<3zgaZek7!8I5BMXgWaNi{QsyVGIQq<Nw3jv38f zDzwG31bdvH0lEr|!iop79h2yzI#y?EWhBlV1UUnV@ab4%SI$cyyLjbC6plat&IY=B ziXQ0xhBdYWkx!)G5`4thAUiNqRapDiQT|w)c}hA_LOY}$k%6d#cW2(mD`W2i-Bq4f z9-F?3+Gg$}$>ztI2xVv`XZk8VN+>boW~$p%r<^5{g4b%kD-~VVC_6WqHh@LkFxY_% z6AOmIJx*ZnBM=n#x4!VpS$LW0ydB8#mRt&X5liMu3qDEWhQZ0)x&F|DU%N<^3ZIrU zf$?Hl&NDS`;3_+W4}_)s-6jL{&KdjJTvM=n(c8n3r%Xkv21k_5xGf)94uNaUv{g_( z62Lxy2y}R|KsOUfo9CPv4u?wu1--entEf8q>N}>G&q#-f4F4sAHS;{?P6AuzJr4f% zL6txmH!OeW!QPnL293+%2+5_~h8Uk#wS-CZuWn)`Zjjz+u)5B@vK(n5%34puv<R_D z=J7KvzTAsg{~-)mD2!o+j5g2bi#akH)0qt|<GNp(lbKRuoA@>3Lu+eOl@S+Tu@*j; zx@PBQM&BFrII-!h#d*mI;lEm!`Fe}z@#N>biXpGj7gbseT$Z;)jJc;?Na64=L7HYR zoAjtfl11;#&&$9akLvAc+s%5Exi`;)n%075DyekLDh^k@d}LMncC>!r-Vaev*(0*0 z_oMyIoHL*Pu}h_>E<0pOrSJsJ%Dc9lGJr_@l_mH@W!VpA!sNm)=}mOPCb#FluCGQj z@P|)ds-s9^!-3*?j402EUs|QxoT|eqWwm9qM&`6rPk&0X+zLT<$%kiI={*_p+%Vi; zyT}gX%-K+rcOY*xR(zm<X?dkS>@WheE)9e^ernMyUd#)77!jw#v1C!w!_R*O=Gn@& zdYB_4-CDZ=1=DBDW|_8QO&MtQb;<+k-ZmxNN4yADi50-G%hXPt!|z?%prQZpLCprs zw-v{|qR*_D9A&Gg>dlzDYqyv(TRpoBAC;FcH)W1ydC`{*s|IE|>uN6;ef}VM{$B9q zf^YYhF2Iv4nvk>iJiJv4wyQ?^SuCPSPKCC!a&`L5huYv7`<SVQ;cd){l)YJmV*9^F z;Q!%S_;>7m{!;-|Fy!(Amd!kp5Qm<J@AX0c<18BILLyyR!aexJwxjv*(;x$k??xV& zp{<yRjoPVf4395*pn4BCj8<W^<kskau&8Tjt}CwFzBFacH-L<rK-C_3)}X_6=AC7Q z>+eA7B0IeIsf$D)X^1(4_`0!rFd{YEM7p%GBRfA_^uD}bOZgjEbUD7ad8~JA)U|-n z|K?dDsy;={zut05ZH4!tkC$((cqkz4eP8T_jp3C8!7{V-e%{1@&KfEOFW$0;9rj)z z3p`^r_Pme<Qdt{4O|}iK4447{_Z(@lV&EkLC$1O7*_e-S7Nmx3iJEJyG+mI@?D=al z!e6NnC$<dRh0_!a;Plz*y_iX157v4-I1rgWcq!g>{u3i~L@Y`0H=@pmY@*;cKG!h3 zM#|%0fv>R=+I0D-`@^2L)?ZcdvQe>J*DXTPzwEgx0vb$&8caU~6bQs)QHQ}0{EpS> zP|KqCVx|_DX7#Vr*$H!b=@c#8L~tq4Y}(MAuY~Mu<dD@qCNYgqFu9$x$7>uTi<ez3 zvnvroPyL@hZz5^AZ8+q0>anPB^)CWf4<hFGzuy3WPxe=`^!g>94R0d<=hR|cw3(Fa z^O%2wUH`+N|4;e;wa=fYr=|xZBaQSMa^HLii--u*xP%G8?AEPC8V$d3^R*m%_Uxgv zv+5s0)_+%&K235KdiM19ia&2=A%7FX8x!Slr-3|yNFTEh&p>vD1h}v$JW1hp+_6!f z9@~?7>%prTw8>F0jd=i&J3XHK3V4W<;^~ZU{wyY1^8X0u-|GMVp-XVQ{;@iHYoM?v z>E+cn1zQXI(WbjkJtk?aqRU1jd7ty#4Tj)=$XZDlv)buyxW7-`J6~99TdH;yNMqc0 z^{1wxU~MPx?T&hMajXHKh{Z55i71vWU)ZUQMeV}V6xwCB1&Tl^_ij+W^Jk8-IuOY< zifZ`i_8q&2d(Jd^FGqg;(W;ytQl?M(V>UyRY^;RQBM;UH6nOe9;r8|20E$8A7!;0* zeGh|TrAww4aZ;?vB;96jh2K8akwbbe+K8Ea)U$l>_)C_>(LUV~E8W*W{`x+1B&BGv zO?D`pAO>J8M~n~*I&>*l{zSr`i@8UdTryuS(wr+@3~yzZEajfHI8<0RJ#}gCnM)$7 z_GTiz@)fW4I8e<$)X?N2w5@~lE88x#e%XOoa!_%zq=iXdN1F+`(Bgv~d?uczxrHHf zE9*d1)++hCW{2Pc?r%gbX-%hiDiBo%1M`q1s+{iHsj}=VFmWTvKcil3cQ(rGQ!!ph z)(f%g{?5~RbvF6*XE$UB;`_w*bCGT9Q`Tpndj&|JY#nh$e(C+MIl&YuV7quD9|5T4 ztyYyCp9T;f{3@Ihn(`DLj!dC84z9H`P|#(08wt<O;Xr*@oy?Rje^0PsRaJ(5s=TF( zR{HOTm;>ODou&HXMOJ=ArmdUoTEa{6>|cJ&^N>kJ1QO(U*8Ep`U;zKz)zA7Ig>mB- zw*)ZBghPW06;EgTE#=RhOR|kyFuU>Q?adLls1ww8O2)dM#+adE?h^$_?l3O`?6!)B z6%W&-#eZ{lzP7vKm8>(U#GAWOXeRyc;lIZyyS~Ra>_9s4$pV>A$h87!&m_Wc{Sm!( zFa>^-MObt7Lo03G$0@Y7HRfUx=|_x?AR^f#j1!HojWfg;qE8oEi4Lvt7|}jorh0u7 z0_XZQrDOe&1vdgWVsegO)OM@~WOt^x>O6NgyaPXVkuftm7{)-!f>6n^Vp&-l7~nbs z1iOp^gv4@;1>yKai_)C$g{E{f%--M^N9!yj#5{&$?Yz8vU5H}~wl17s_xd?CV|~+3 z#Z9%yf}9ftO-mj9;T3T=^pw;t54`>U>*VXayut~%4!Sgej@9Cm@pDzEhJX&r?iGnB zb%roc5@R*0d(o6RsJI++sG3cm?<MG6yyt+Cz-j{**eX^-o5bIJ4vr?G!~8Mxe_@@O zo`G<&*F}H24~Wcs-h^`x1Ej8t-v#RU_TLb5B|DJATLq0=bMh-RwVs-8vS)I}`;Yy> zP%*)Wrho^VYrFeY>0hU(JWb_XMDO0QPNuY+d~IZ6k;`CacUBCqoZ?RWG?wX$`MgSb z{!RJ|?e$`BiSPI6Z}oVC9~K>VaetxYTf{V$>AS(2RtdgwqH_Q@c0}PkP7UkA8eH?9 zD=0YM=T^5C*@3UyjyFOY^WQe!%6wg=XM52t@3?z}#arj6k+YyqDLzRsgsQRqnoIzB zJ7rcte2We1LW#E_e0qfwK4fH77A4b>E#X()%CA5Kv^uP~jG63RtUax@05qCT7&$U7 z1{b{j2+|1q>kBFqmY2zKets2hv*`T7!q>%R%leH+43mj*7_phmt<2AGOk_Yb`fjGu zSmhh5<5mh}mExT1PLJK0KNP8w`^tyif?CzYB(?}06LFHOLs4Gqu$26;3bxd*o*4SD zCu!QHxeS+pradB(mpFzE*>M3CXphpuhT^iIn25h+nsCLW+4p(+k>durABSe920~8X zGdq2|@3`gb45tKFP1XoUWdcLJfuvP*at>tu5^Qk1_(eu4^Y}DGwN`Fvh29b%Jr7S7 zU&<%+>_DbGt%ASBrU0Sv6ulLMY?Q>rK=@E)9sWq*8H+>NrGcz&ZXRk^TBlQoD><d2 z$|1Y9VXz{&uPpyZz-LCa6x-?4M9I2HngG2zvVk<0a@|M*J2;DNLD-@LCt*AZB<EHg zzm^(^oTSc?)<i-2@{C5&sr^B-;#4W1}w<aH24q9)!aZ*;Y(59=u%vyRH03b{r{ zZ>x5`{@jlFiZry&r?PAjv;{Agde1s=Dxd=+BbWwpWi5A~pi8hHu$rd}<EFrhZ#`5f z>O+^0$e12<FF@`hD&WMUM&>b{P_ZQ%qQ};>lZH>JjJWWfQ5R=pCm0PD9gp@o>9>%$ zRC)TWTUxr9UDav-z{Y4ZhxZdzms?isMC?1wH`Pxf#aEYXxrQ2)aZSdn@%1iWvi*xA zQgdvbu1--k*%J!`85ASE{g;olx;OoS{X`l#Kho;sYF&sP5u}}@bfdr?VY$b`WLAi^ zq=y1^K^L)J9proAf=QDLtc6a(R33)}6h*LZU=BplyI+_jGZz@n)XX+Tz=C<dm`P3a z2uMFxulc9<N{o}e_lH+*<cGyg=eh%!7@qV*cazK-oQuMJ!0)DBhtiN>HI}}L3Sjl3 z016qa&BnCD4gvR}2e4Ox`zFWGO@+^?k}dd`GeIwo>31;4doK@HMR+?p<XN?4oxJgA ze`aSzq3mc*V3S6C<?=>%_orWtW3$K4=cvArkBVacQ=ola{xL^mba^xb-FLO)iFYLY z`dDD8wY7cFweo{zR`x}PI<_4xn&XL!_JwkG(!t$B2|3RFxKBf<-MowJ$kicmdSLiz zI2oSHr*)hLNRB)LrbuYq`C(rcrE5Kv4lz*=^yp2|3+BU}SUt_m!FXj8kn@4L4fAYC z!SeUZ1JIQ=enzzLSA&ta{UU$b2=>do28dqZ8pqvcqcU1T=bJ%pYLwpK8XrD@XR2o! z*|SBv@vMv}bxtnum_ZOlM+&sqFRJ3ogNCL8<-L<C;SmwK@tV#;A2n2$uMM!DX`f5) zoWDDXwL5W2PnD6>^>xyE=oOzH;5vOv{7Z(jn|0wStI_nV_(7WE3&{hSMTlTiNsmI| zaOS*W-Ps7{aAb?qnJmX|U!DZ$DZBPhagk=YV_02&WlJnuSRgVuhfhE$AOfh<P|gid zPkC;;*VvEIm=r;$A?MaJ5dN5S69^061@_cZKCwH_f??1>5MQpPcFjjtES*8i&^|X9 z+?RLW`djRJ)?a^{yyTM+b46$tY@YNx;wIP+U<s|8IxWMgV6S)1VCE&UihvabtIe`) ze1L-n%6IuF5}E2<gokulUPw_OPI3f}wNTB+J>D~88&U4#jyxFJbH)7jA~SHJ!~boC zLj<DUm|C0Ycj_Y$Gg%QKndtc3-};3_mDg4E{Wkl{#~&RtopcUHcU+@9wa>PH<hh#T z-fw+|&`_HE9DYhq)UNDAj{Ko>SzUJFt?5t5j_G`bmsEY61c(W21u{HeJ^=^EjVsFn zXSo*i6+8rsV((%6BNUtp?j#<(#ok`Kz`#Euis_{zpg{KAOv>*q!mpO~XI0sn%P}tq zYE@qgxl0yy<HttVDYvzbw+7q!&B1t5<Sv{cpVXqnJ&qfuR}*>(lg;z6ZelX-EYQd@ zfu^>>0VOaZQ>A$7tRD;33B4{yaO$8wN!=&C(?n<F=!9-FIaA%i@`R`R--P2jp1q;t z<sY5L<@Ix{Mx3vYzGS!*bPmeM*;f#2D@zj-tL-_C1-WvuQ6#JKmLR9h(+1<qvpLcp zZqZ@ta+m8ghdVwNdW2{^DLv>M;}#uj*_qoirXG}+#<zW$#{0wvS)f~Bp^)<$sAXlp zS3bml)q>_~;<{<bL~QVD-hDQ7b(#%|C_yn`D0>gnZ@>@DTJONgu%`jZYbo#PqFSd9 zRmGBh(7!TYP$KWKXfO9Wd6^A3loV;4&A3qLuKR_4`j~7-gqmSi?N`ESMZcS$E3Bo4 zTfWy@8`Z0ya(A?v((Jc$%#iX`AU6+O@Gg}-HBc{GyV;h;vj%6B8qXJKWrOr;gL-0% zGrP8zS}*e+`8&M<-i3@O)<mvdV4Kg|jbr23u?%FwCIQ+DqV$lcg{vbW#{u%8^!X{C zS*a<rND?1ha4J-%>a8VsI{5y}rRysr_rgPg+u9mLBd-(XH&~n1PC1swzV?n)(hdju z%!90To;zd@T&Q?jq37G@DfF<5QK9<1^|g$+tC(xfl!36rljjN<p6Pb>>32YJx`y?C zetjS1BL+CRzUDoT{GAFll4IdHI763WuAg8mAEZ%gId_J)EN@VUTV%8vo;Rg*5oxfV zU%_{Du=+sk+|MuLlqWq)<w3Q9I7Kh7;D#?w$277t&s*e&p;a40n$@<mFZ75%S<rU7 zq(42SMh@u5H^O=-Qn)GN5v(IvrC#6@d8QSMhyz#^WugpE3OT=!v?ZEpxM0c}Z}vSz z^5x25wb(+5>Bo8A?8ubwRo~bzib9SP_eUAH96qqGm8oQW8&}98Ch~OV5Ta3G?aCsb zbcrGWqC;Lehd~sL3b<6=IHHS$0(jG!1lys_5Oupcv$S5jhBpoIvNtlFRUSz@LqvX1 zrkXjiEcI0?E2R^$AG?QNjt?kNGIeDer1Pv|Pf+IZsrbW2R&n}`^Yk>t;g(Bxug<El z7akhfN55M3DE(4t^BMJ&;a2rBet+vY)w;iC(tk2SNzJG9`PT&=nOnUyGqY90Ti5+N z;Hm4qaotlhmrYQA??76$TXXVkYQV63ZR7I)M=D#yuw=WT=}vJ(*(5$_S%2IhY!ZzR zO6%oXa}=Ug>)&ugI-aibmd4NXCWKTr4Bi}-2cUuj2Eo1{<hI}_HC4BITO$t(28g1( zfY+$>3#WZHdXx=WYm^LLSZL@$(z#hgM*U3qpdmy17CcP8@?HhuQw8tiKXGjt%`$Zc zI>GV##kPurcj0Rz)39dq$OpL-m97Fw{Ny~dF@txB2~Wu1fuy`9O-B61(~OR-jDmWc z=`rbyoeS1~+p(OBL>Tcu+a54hIXRk!pS>et^Q}nz$Hqid>}1T4XU{dTwk+b9j#pP} z#A_k02EI&NJoC?LGLhw1Pt^^kAy~F_sijv4#>r3ke<Z)VH`j!Uc5D6w#k;U5b>>7T zUYR!Sv#UuI1-L3}u|ozv`)P)Tlo}n{w-wG`r*y!@+&*wU@a>j3aRA<#_KVmK7Qnq| z6TOH>hCj${36Wd}o;46!35bGRknvvb;|QquBI;n<QS2plPp#rD|KS2hmZz77ZFy(y z_I;`$Q@(EbTZC}|wI<^_P2f-FQUbmasDGjDa=np-p0o>A>O2ajZNkTH0W7@mr2II0 z*s!&9FsS~M4%qWd*#V$H21=mT1iUxp`hgjs&C(7;D#YYS6y-hb43Npfn+=;t;X`<5 zfOOnO)XE_H`{P?+C1{FfSI#etAeqL7am*fFeDH-Rgr9pxmp2k+c?v{8vGd*rl$hxp zYSLz&lgGZ;MXVO?$&_(tU1^|;fnE_kcra*<tuyCS4C+^`CXP>>4#({0S+(VFTSnKl z`^CLnu5|~m(k)l2!JCG=CO8xtzLoPIPIC)->vcz>!ozU1dh(rLkYOgqB_|T<>ta-s zWEfhiZWvZ2Pc<|0cAt2E!y~OK*XDrXp$Se{hm8b7wKSYxW#k>BSERc1Q8J@=ze}lg zB_a#7L_u(%j~!r6<>v@Q1YpsDWznoaIE)oyyLQ?Tm2ngD+dQf1UZ3Oq!tdV&P~(hz zcn>83FLb)?c#uH#ExYyWY<6&5Sro>Zm7G+D-gfLmN0igt9!^a#278D!`rIJz2u2|K z4$sxVbx$A~>?p67=x>yX<$?LAUP=;u?>bM8c@nHgAS8)<m*w9BRiOdsDnR}_GsIWV zrqD!S>uDRTV2UYTJ~Q*#3r!!7i-soPcJV+tE!%=g@1Y7|jacg`CYrox6H$b0OJv!! zjWU%jDv%z39PwwAVpQ7KZ@%HW1M-~jv`-p;KGIC~3G)u?vXR&jN)PEg+cAI2nS*xZ zjD_a>pyj9;hFa&GG>qCu9P_<wPpr9CfBs}A5h{QFs<uXwh*d@U9+7K>Uq!-MQT3@A zUegg3?qTL8<E1u>-=-tf5dDNn(wxj13LO1%aC+JC#DKG){wF4ZGt4i;%EeU%;|u|_ zIZOr-+IDEk_BGE3ATTKp+rY|4Vl305yR3frwJrgQWD;xC3%?G$Dt(YUbB}?{!nZ>= z;lN4aDtE5ck>mRh(krAbtb*CFCgmZ$uDADm8^RkmO0C6r@z2&%GGOxcgQ*_SPG<na zYV7tXqi`lSLE-Dnds?yKR4!|iaQyNe-!y8Gh^Bm2L~X{c2yczImEIbgDhl_8i&DC5 zgdM^IdlU&-dT-7Ljn23`m&=)rEQG5rW;>5oU6!gCf3h6sdt-S##IT-i(8aahDzMXz zaDj`<r#(N_UcO;|wp?}quT7Wc*@Np0I|_bG67?V$>S1Ca+LLQ>rvNH*gjFt^&IuUa zi~_e0p(a1rz6BEe*fPw3cpma4?`m!Hm{EX~Kw$)wbsI^@E=Asl-bd6A-OQCc7B#(! zp}Ee{*OFKY0ucm*0-O?Fo#5(%Yu7U>mKm^$CkgstN+!DlVbX0fJUCE3r)AMW6t9P- znd}2n=ZF$Kx3Ri`J>5RA;MOo#Fsndt?S|QIp7d$aw>d@W+lSJ=y}#yuM^dZPx1!Ja zgL=mE;5Jv6;=hj=tW=al$0fL>-a6mcW_^a^SlOCTD(~cH$2J+1afKOf`f9kRb#kp; zuxV}=u9Fg<E8HmwNX|tieZdWtzp`?5cDCBUoxly5><1^uTih_X)ND{R$xFegBw{Un zt(^_`d^>I;3%VLO8|L~c0=|cLgwP{U7@eNOQMs4!7%%t6-Vh|CH6=~AfG!hgTN>!{ zi}27l#JEapF2uTR5C8EF#1~2sfScCh3j|&GdN^2Znal0v$O3Y#K?Zb9hqW;;EvV<Z zVd20U+s28VLNJnMyv3enQeo{R?4>bnz0Bhw0X8i)BkP-d8=r9sX_SKDnFSPgGz@>< zfjDov-cfEG3%%E4Un<+zbJxDM&iSgCSDV#<p3h5;9?AZ4y~mUJ)F<q-@l_rz6+h<h zj&2-Oy&Lt++TOX?^O=du!oiS(ahi^(w<$GUzDuwA3s2e(*+3l}i$w00xiW~nBMqGb z5fodq9WO!+piYKy(CmO7SUg@@PYWqfQ0T#nAp$Y+RER*^j8%K}-N+GCJpGW74I9%F z<IAd=NMHS~chTFi%f+SX`MA7yYj@pYv(WolgwOq&X>H0cR3>{URP#d1k4|4@?Q;A@ z>#bYV<;+WEwB4Ob`2GDR{Way6BhXH3r5@pN<g(8iA0;w^%pH@g?L1<wbrOp;B3@Ef zm(>TXku=A<WRZwtKi`2XeU^>reJfCy1@~kT!Iy(zgrp=Rgs{eQ1W}#@%b&6aWvR@U zr88S4EVmZx3yd|U=gNxiVo(3FAE}GhzBT4HSTV@Ved~VWg=QagNlD7}zPk8nXSC~c z2eQLQO#Z#F!aQU5{*ncChf0b=aTCF)v`^N2$g3`}6Kd;F@-DEW|FHGh%=52a^&Ga( z2$9N7ogvdWHD*%wZt|d%<~O1L#7&X%s0G7-C*uyVQ@Zf5U|M7nRuqhg9kWN-AR<DN z2RLz>oiC(V4LOmoqf)#_zn{Sth}s_S`OXYzpH^yCRJ=jmw*&E;2DzYNlXGkWPpWv{ zC-v2$j|sn|#eftDs~zlYy)Tu<<;3zcPDjY;ttY+M7wi2aE~QtXofU`;nMWVF3{qLN z`)Thzqk{f|g1$TcWmG<jhGeF+BY_jh<b~*#v+NTJP?`?s^Sk;;(K1{g{Sek!B+8MY zU!);*#*60&beFm<5){KuX9G6xee}&UA3q_Mt2{dIXZi9<+R4X$kIs)doS!!ndHC?p zy}$hlfxO?dyR9*~X6qCZ=3;QI1x=^Y^7>-Sx4$hDa+s&XGW;HpwEWRdo7}TMzw>3x zO+chtYjWj3vh7y$%Q}s~aURA8dSdngot?_sqc|~;1Qo7VpdQ8^;QwiKmgE^^qQcr& z6{lz)j8mS_jeYhqrI#+EH+}wiEgPP+Uh1jj@kyn4YS{%O9Rs#9rF9a6%sPVD__Fe{ zdRt>h4^qYGD?6tzrNU_V^1$tpDN?bAG4$_37>7TO+g(LX0C6HrjDolYlEhQiw`DjP ztor2UdE{g*Q$pow2=)>Q>secui;p)jxFq`FCH7L|r!RH+$Xd7MrqxTBn_Myo+<J%! z#3Q<E2v@+J9l#)VwW_;VsSVd7Qv^n=#*Df(yYqe<rP;_<)^~E4H1jm#qU=;W<?p)} zDAp0KE*8$=5gLng=W5;@%XyM<?hnhOtT2=!_BU{(BLwe>(h*9g9zZ%*O3>d1pV)F& z$#Y<9*Rr6DxZX63&nctPV1gv!px9;P^M<)QciX2tHM$g?LJtq?WCm`2Yw_QB;)n!= z1U`{^_c{k=ujL%s{#sZ3BO^a}2eJp}Vhm?Cu6c6LVnv3(j^lK2pQu7O8T62iabw{; zCAIAkYqU`&ckt@~X_6My)D;$F_%v_7R>*u7qvW_=#yb%l6xPrPlGP!|M>ruq0pS<Q zL`|_x&hbouB=#N~el0$bnE2ux>rpC&XADF~FqBk!O*y{-j`M{EwynSi_Kne~?JF(y z%@gxOEqZ>w2WhMJK*h(I$`Y<4!*$h}tH=`_TWc|8d2;Oe1p>*aM;}iXHL~HI#CU`l zR*vax=_js&c!)(X$gxwUlO|~8`;n@(wP#gX)J9kb^r49%fMKB0iMzDcKvn|8JpNFn zC43!nm7!Xzpvs127#$112QUW{np$~gHwhie2q2^!RPoVDa5pZOGZ!&8f3Euaf9>f! z^Fo69hAgl5#4{}|q&e8%AXxu(&ddD!k%o8gz3-GZop#LexMO8-BFlB^EqQe-GTRnq zx`pA~;6H6UEclDKKO6z-Y=t&2w(4}o_|F!<UTON?{A!R1RpObm=?~RqX3qeH?2A1z zoohIKfKavEW7F#B)a<7^Y}hT)juxt~gZ(z~5H|oSdBFUNMxKLpU}&0*kk8vbLI8EH zP@yMA)pe*+zU>h$Lm>hq3MBu1))12xo_R;f#%ApH<j>rgTtY<fQHjg%W8-i)zfWVr zHY09kd4j&z()WMXVzIlSptU&4^YVBLl4<Pl;R<UzMYqtxXQIB!nt9RoqUBIs`h|Ki zN2gKk(qmecRjVNjx$d$_mdsl>AbM4(EF#MdDz&Vv0>D@#jt=YV)(q((sP@R5Rpu8O zAy4FlbDDYzxrUR0xT64>>l`#*Qw*(b4`adKn`p8oULe0?Ow`<RTpmBR9U*DdpBj#D zS++8R>#tIem{~tAwTrm5CW7vxs=gk4wgb8Ta0{P?Qv&L_VNrD3<~JZ+_?3MGlNn=H zR&|FJtSrGVs4I|SrL1PNmIJ>Yo$B>xnc9A;Z=Uv5w1GVor~=9NGi)O?4d@XL9UZ;a zZ)%1&K?TJ{+mR5hN?pE5|Ng4mmA7v*Y+`LL+5}2F%#LeD&0GHC&>uKKAmoT&4f2U+ z-o~7d!s@;(UJ0R-BQXQSr#CbQC4H)*sWFuWo2#u7H9nT02mjplA93UV{onjd;0R9$ zuP$+K2}WVng;V;7LxS@(AKVNGmG#IH`4M2H0hGNMv{M`4ote@%Pv$Q9vN857_#=~n z-EYp}hCC9f8t><DBNH3zTzpU-ObB~{<;Pk2RQ$~75?7Sx%UbWINcEwmvOWjVYcUz_ zI2qj7to%F6v6ca2+yQ`sVk-s>{UzxN_!ocvU3tn4|Jz-p(CFdM)3(yx^<DOZn9#kc zx{~eZ4jj*y^HcLrE%x{KS3Pt6($R@;KfjCn>WDnA7+n$kF!37LfIrC|sV97@@p2u1 zBlWXz`P#M`C~_?A7la1;RGsrU`LZQ!{Qn;=kCKmtaRYI^^uuj%P*zsE{W~>?GWDT{ zuC#ezE8QD2*PGa;k2PS+blaX->71rAj@FMF?J9}N3UL`;?hNKP7}P@p?UOosoEY5~ z1{~tQy_!FDvg+o%P;~zXAGZ_FUkrIP#QUtOx%jM|e@jv?am@MY7^|*n;@n(Yy;(2) z^XDs2)BsLwV@*yofpd@wR>Ln~xL!6)W1Wt(JQ+}RWf~XKVpbXDGt!fPr-0umlQ3-w z1;cd}W^ao9kK7CKE$K&Z!UC`sz+(_l&4DUnBWE^(Ny#+QVXZSJ9&83v4+FhDf^*}Q zM#c=hSYYm{wiU<mh$_3Nxi@vcIo)~O<31*Hy8qyXc3;2DFdHGgQyLRKeXn|SvYk!# z6(8tvKu?+5XAhOpY?*2a*1+3)<qkWLywrY&49AqNcclk<3_SYJ*|^m;duTk$t3Wr+ zJ|hKt=ok5u1Anz<mX((V<wyK&(1a^3qfhrn`n_`V@w{je>91E=Z6#G*tD1VesoFuz zr%FxkvO^-1ab7<^Xh5$Rgd0fn7hRX{mdk#Wa7-LVm*u0$W%)xO5DENbZ+50n_Kzz! zFKHfod_%j<*X)>kM}XR%bMc|SyB*q6gT74LfTdFva0dm0vj^V+1ojGZp%WI5fSz7o z)orFKW4+niNjQb5a}$nZxAcGc6oxSgazW!RXHcI%ig<R6q@D<E(7VNa8F9?^lv6(G z$h<;d$-7cD*`D>vlF=KD_t2URDYE@`L+RO8t<sxT4SWgP#-S6lDI|NJQ!FGW2mB5L zbO_7^B=NtpxpTx`{+>&1=~pN82Ep_OE6R4Y7^*+@Qm2!&Od~#sSOLOU7Lwe@PZg^A z3&lSki1iLW*)MjgySuJ1KiEojWTMzsmxC^=JAN)><Zf1$E$LQa{{3LP^2PC`MNpnw ze)1$m9?;=l;wkZqv~b6Lc#7<LV&)o7lIOyvfU=N>4pU1_-|~(0*p7^u0q~U3v3pJ5 z@`CPsU)9EFWm&Byj%Uf?LqkvQCAg01+3J{%=qUwzjNaCbQ?Pi`Jf;`o>D1gfvN-_1 zI<oeqnK~Mwz<np7skvXvH`l!^XZ(C|YT=Z#m864xDY#);%`T(7r{;O#%f*lRWRVKH zzLCOH<|P3*cgL*!J6dO7^*gJZpS?%6D_OEW<zr8_yHjX==VHYl<#LYC<-cVx8LB1u zaC?*n_9(xZ>I=qa$!;)0|Bh-vu9mYqdy#%&<)rww!~Y0o_qVA5^>x!~(CAn26eAfX zce<xKb^p2kV?XAFwcoprMHFYeeEC&IS}&?0-`1P_dU=Wby1Z)Sn4QGEjGLt=FIt@_ zuq_!5E=H3m<}dyR-ELO=zYxTy2&$n%=PVy5M`YSG{TXmiTHHq2s~U2s<hckW?*A#l z!hy(R49|!K84tIX$e48waX;=ovDPBFO?t6mL!<5*FP8sCPf3T_*wwtaJ3yuh^L{;< zzWZ33)&7VSX2fRT1GNDBXKMi-G4~?oF+gCg&lA@W7oXk1gTNA0tEK~Mq(1e*Ge9&6 z_=b;1?eWc;58ZeVh_NCW+3|193=Y03GDp7a9F@M^|2)6tF8hffXb2USg;(hwBy$y; zLhXhrYC=nFM28l_alPr}OX?Mqz5jif`Hvod;l<7o4`FRs2AL*`QN2i8&kn?4BXuSm zti{`8?>4sSe4#nCuB#P>%)a-yCghx8S?3RP)x{WRDxjLuMU{x!;79^U2K1qP`Z7_P zdaK}{FGB%0av#Bv)F^szMcfA;_?Y0JK^okv`M@nUG#RVTph}!h`@6<;UbaMbrY>A> zgTRyAz6MI40=N<Qb~BJ#97o(7YE!k~8sa{`*#AZ@IPx7pni~k?gadUA7DTn|Zep!) zN`&Uy)IDp;#v5Z6FUEfG>{AvxuXU;YU+leUP?K5sCu+AVB1AS3P~ry2Zp)^KNxOl7 zfP#R65ET(a#E5_f2#K=qX+=TVB0>l|vIG>62no<iBSP3wLJ}3(<c&gjB}>n7=H9wh zQ}=)Ge43j7)O_HBqE&BB-gD0L{I+M|OqL<=2oHbdXZ^F{t63#TtWLmxEZ6p*RbMGZ zzmR4=OE%>7{*uN2PfwLV?E@`LIt-NI3S<z()3D4oT@|=Iwm#iPMXX194^ExTSH)X= z9FGYV6QJr2>-3C@>2FUTS6AYLc=>v5qkA@v@R`<tb6gl9i?$4+Cv&Kfqk>Ck_aygj z2vG-Yrz@uE<c&b*=Ltj??f<UQ_#nAG*kb$?w?L;PS`TG@pIs(d2z8m|<Fic>e-HcW zY7U6Ln?G<+AWh+#wA1)eDj)?WvA%mdo+aDa&rm@4^f3-yt2Y1WZ8un{`+Q)r=V*VJ zewhE^M{J>^R&@2Yii!=o%-u2#uxqxU1E>kU2JpTxL^_o*4D4sfIt*LhB^g<lFc&BE z<bS!x$B#BoHl~HzSLs108gE{^mnWqr5#0iK*c*FLTWW<?NlV>_RnTAGZ?Bmn9CBsz zfAUK!e2AGXxXOSd{)E(ahVSu8z4H}8)-R41l{zXVoMAhsZVN<c=y|S>jkEl0q+>IY z`*nVsohTEz+Lm%?)Rng6FfiK~ksQCjut1|XI~Qic22+werbJ0z)^7#8a}|BUvL;}_ zH~vJe@4Cmlgr}hE?dE31ZnM<i^rZPDIiefL5kRi;mMBT{W%~(^f-5VxbBeiN$Mn)J z!?O^Vt!LL?Nv;c9v)aMi4=oyqpY8o_%#PqP$P#7KBhso?Yl4fQ3osaXfCyW$Ub*q; zY<YAtdcDnB2oU;nKMN%XU^77^ejWN=z>sYFbiS$d38TkhC^Aob;audsl9{IDCre`1 z=4AEctx+l>z&S@>n<h7)=Y*h07^U*9=5xe12;Q%)Jy~hqJPP}x9d9^Ywh`;AFD<M; zjCbk_{pffao{i+=?^?gguc`bS#MZQ4jq~G)gH2u$Q9Yk+l)Qg3p2JT*g=ZoWZZkhK z&9oBPeZ=0(47UHdP1|%_htp2jwyu5IR@&)Msc*YKILp*lI}|(h_=3-svdL?g_VQ+Y ztwKaw=Wg$E&@b;8q@|`}dk(x+iLKrnSxZKf+w8!H|7EbMS(d1fC)GQ+3AY!@iILJG z8z=ET$yz8Oc?S$UKc`$a5V(SdM_5aSP<tGFxVwO)s)zqw<-u7jM{Wk{R4vZri9w6o zoW+r+_jSGPTQ5l;l7mdgJ-B?kd3}h*mYGVQldmCSvN3ZQp`L)rS^$>HU&wwaOcm_~ zXJ<9Y$QaVns*dAyl@Qe+Y8+}89O+RvI5`}8E<Ve$&>!4LOY?DaO>t#%c3*oMCCc4A zMW%Q+fg-X104z&XAT1$j&qOT5+t`B#+$ITv6DDWLshIT^<W*yE6lt5U1SuSD>y4jK z(nhpEz_HeC?CAIscc{D|JmgkZ#>6>ik^2x)3p^8XoM;YKJxI7-!~*uxKzVKwT5bs) zU3gmh47CsT_Ns$C?}bI`&xNgrsL`$4Bo<I_d=(LAbIbd1SCYjm_4C8Ukvet*s2<~v z`thm7*~`36ic!h5N6*g3-M@9~IhYqFzxB6UosjGRN5I?qZ*^D?98STuwYW!^?~b8b zdKAXDLyS#cqT24vjaxgpHe~|^l(<tKI&X?*`a3S}+=i~HG?+fn(;27vIQem+Y=`8K zw3azpgh2qL7v42I-zPjKdQ%Itc+Bxe0eSAe*qRt5&8A87NSg@A*SE0b`s#$RoNFkZ zuf7MM&g6U@4oYWwxapMDa=U^pQjt6HvFRL4w&UCnomym$d5glO&xin;S-U8uUbit7 z6D6T=5Ltj82@n+tDbY*4ibNgh1B>#7Sd>C7+y=$>FZ1+b=ZSbR8u+1hLA1|<pC1j3 zB_z%-QIzaVGL*e>o7;5Q`(-78NC?OwMt_`<W-~9LM=z7NO0Kvha3L3R3bn4clN<xY zH9H{(xSKt(SDGEPq=o-|gy=GW<O$BrdaWG`49W8{HgxX0p|_Xn*Yy0im{2ZbSfWz* zn#>?~R0Bi$I;xN(`U^lChb4*-NMdMF<C%gDUuTs{RN+;t&R`I&JKq$2fU95o8gN?o z5DyBS)2iVcL3YPEraRAv)*NPhEbCF7^hezA+h#!|t=^tM$InTo4pwgM*voh%xOMU( zbThi;GTrfy%H&c*hR0TWk3u<zNBHu8VvQfYXGOY*KG>emI;N)o(n0y3ec`XJyJL-Y z-j+@7j4#eRU=YH#eK@oCT~=el$BFG6hVgFJ%9e}@-13Cs;NmN-Qyd0HHSl(XGqseE z;aXc8JtoEs7@5n<duwUUc9+mzWNI0D<3Nzs6`%xg-;H4}zz5d-T<itb5?dPbE?#D6 zPY0*}F+gO<kI1%SgTJbPRF(W;!Egc{vl<J*Wh=vunadt~jduCiewpaVm!A00_ve)` zX%XZZ{{^}eBQfL|Qd9^zomtvjp>JtVT?7ctiF1LdeZo^swp9Ejy~GxGGHtrSW4Lu! zOJqoJr17h~iv1TZoD_Pu$N6CtrKM<pasp7X#sWOSjE|}l7^^tK0m7+C<~-u*2g2^t zedgn0rlv$**qJ9-ki$l_G%Y8D?VW;Nb=DMnca*ek@>mY>K15^8=hCiT_@kbNK|XYk zJ6k$gmrK=fE!w%2c5H3#Md70vH`?A?KBo&@&rTn>^>Q!gb6w4WJ9E&vw-+9`Ro?A< z9~*Gc*|)!S-K&kkdCmb=5s!A!-SWD_e^`}D=4Czqu2Ng1D*ak1bkM^1EQ?!fC)qa< z0|<y2X`I9lDi@_mv%_qmbtyuDAR*mCddNxotVMy3S0&g@{?oW35ASEaCaE+xybo&} zSc=S<O$&64JlB87UKw#nngPt|fSdv)RN@MK&POQ2D9MH~+k==`ZZcKD7<GX3_Ahdq z5z2tDT67*h2ptlT3Kd>a(ok}wVUK@Tt*fkB>YI!@bgr*@!9q&zXc|i3N1+VIA8+Yb ztIuQ>riF;M!4_4d;fSpH$HV!5aAZ5+Mp5_x`6@bjVqFarTBL4hwUF$mUdZx&&*Wj@ zV{TAKPiG_FY0H;o<zn`hn{jMq%B}^6Xdjq4DVK*45-T9nv=0I{;*Fr?W6)!=mrC zQYuM}(AE+Fu|()5&;)skNRSiT%rT@8b{$%9gboAl;RwWXY9@THfvV~RK8}AXsAt`| z6n=wvulRiT(N@=*&B5xFWOes<0E>iMzl23B1&o!Hld<zE{rP*b<r81ni(o2Tk) zS>8qSL_DcDJfWwiFP6>enSZZXr?PtQ0o(m_qgKX7sN|#jCiiK6AaC|NOTp#GYT3p^ z;&9RcM&8C=OeEUE^5R<(dq>JhH*2<%5F#Q=smzOVtyZLW6B>gpI+Na*?X%jyOY6|s zYIUN{0LMX?$++>kn#0mIg~kYW4ui&5n^Wem{cu@Y4;&B8qDE;2h%Yi7$r7QY`K<@( za{=o+&?CPi;JuhGbTUZ|_MSwf<nJ3a3JW5?mTiETZPf|SiGF<+PKc8nb$0&Vfocoy z(Z%LhN9K);cSbv&F)AkGU_ix!v@a!Lmanu-Qay$i^}tR5*&AXGOB{jSu$R1EW=t?; zVK%gc30Jf86&oOrYT+tf22|%JTt_MYFfzU3)FH~lRd?~E9#NWOT+hCR#cH$@bqHky z-eaRyWr=;g^x+a&UJ@)VZe1<$1DZe$Z=yPUNqk&_5O&>bu@K(zflp5$-`k+8U<CgZ zX(t>|PVno@UI)&M4ewIOA)8vYl7bkHJMR>yZ2r#D4ZkHicL5=a)$t9@S9Zs?<DU$M z-@qMH({UI7VD<L|ekQ!&l)5*;rwpARSwAwobXC~$ql<5F$-wqN;KM(jrv%&U3ZJ-N z*nRU;aN2qQ#kvbdbO)u--~*4(x(6DB2Jic6C{vJeTFw^{cYqr>dfL}UXhIdBwXjsk z!)5$3tE1JlpNO*2D~1c%w<8u}gO(_pM_`{Y$3EUTt{a|u;jPpB()&#Cr=os?b?9m| z10l&l&#U|@y!l+&Q|%qqyQB%gG7*P)ZpGpzAr(=|Fq1!(w#ZB-ZzJs_z1P+sCRq^P zak^w?Ib9eP#lK`aZv4lKT3VHE&e%nQzrBSebD${mOSs9B7X7B@3|^aX=oF|i@EY)H zuYB3Ct*|#%a$<7cYsxWmLcy7PN@z(5f5z>lJKe$AoaWLh=PKnLH}zE{)A;v~9Bb<! zIE*&RU#=x;rJeFW6(gTztY(c@vC}QMmsjy14UKocP<cW}-u%J)*|&x)x%{DjaQep- z+zxj6JqQ}eE=cSN-ujl<)9~4CWPoBZPF!|sER`IP^$bsoU5L710XLI6z1Ghi(woZ= zKAS*{c8i`sNI;8Qpb&5nNmpVeeFy|+Ty^3_^zUJmxTa1zAFW1Ubu_IjxJ)UWr9AD` z&8c`Lqbqp_?29TF4$C&)6JM41NS~qcK2{<!K$*`HxArmpFe$!!;NKuUp4DqStgFX1 z9oOUac43my@{7iYJcXn}u|3!gtqQZ>VHf+M7Q51_48KAG-UFH_q+R>rjpJPSXwgxT zLE_^;zr?+3-8;7|R+jZ<%1q%7-dE6y<?F(K@OwwQ`BMo*yGb!JHNvtS@;vIY5i2Cb zTzCPgj@dGba57cgM>Dp_@R|4M%<b%l<*JsqbXxwpg;8ZrJ|ruEjMq9ov&#h>>Gf0d zSz>KMf+$Az?uE2KVj(t=ctiAPuqkqUIOelHTM$~a+3M*iO*l=WCe&Eel9!j09N}qE z6lXZz<5`h~5%Ywoma+1VFX4;}I_EhBX<Y$Z6J4@DjO2Q~6-56N;BcnYHuq9o`%s%l zn-rsDYw@gRR^B5_;G)o$)xMRey>V-@1C5AMF)Gxdy-L?C_OtQWsC<EOWy)JAfz4r; z!4c`(U@Nc9Zh7Q%8#}SLY@E!Pnl}W>{5Qx=M9?b0dgMfzA?dY?I7-GDqjj}WHV}pd zigBfliJ)#?=@vym+yohlfH~9OqGv%(BifbsR2>#kLMl3fioIT+NP2*&CzcnH+m4g* zhY=!6I7fioLV6!TP69!xHsICyL9i1kW$_g=ph#Xoy!pJRh@2vrVv!T9mt*H`)Jzk? z<ef|mr3jCqu5nuCvJb<$VBf6o^?i=WdDCBpgf;zuTI8_Nb;E4Hd|i_klLOFc<T}$> z3rqr1L3Ws^08{CXLO^2jD2dlYb4-;0;g(iOyar{ZUYG6ts^0%K!fCD+CsJRYEgh*U z*v|i8e=|VxO_~J2*NkP9jx>dQV_{@U)bt$g<B<_WXYkN}5s5Gt%HdywGm3kK{Gi{v zJ7w#HciPZn$RjZ2de~09a4mzoaXhf*!;Izi?>@&2xYJkV@<(}MgOS_|C2hQd_(!L% zJo_j}q`Rc<PYpBJN(yH<xFmc$Rm!6sD65Iz@4nr^!f?2xMs@u4#T~_ifwk?tsAm}= zPd~E`b|RI$n&=y>pNGFKq^xuqeRg(jcT0&od76g2E-fJ603Fu9tA0aImT{F^)n<Hm zHxH3PI}?Y5Y{@C;hA(wciVMVzQxGwP-n_;xY`xuZh2%o3RtCngv_-)$m~KZeemZsu zckfo{w8`sCrYZ9ZaBVOps%So{RIE-UX90c6aF;iJ><*vKa&$>Y))pZlM?`Y|DrqK3 zk*ElBOIUb6flD-oG01Ey$Gr+AAf1QIapE2_MtFl!08?F(MPJs|#|7zBe?Uir1m50Q z^hV~ja^_fdCwChf%N@fBuD%qTfnLo;n9mfSgqPd$E#8BQfZBmN*l%+u*uMm}Vo?<q zgUpV+Y;5UAb#2nz-tdUli!Jl*DV-imFAp;*4hMZB7qZ7BpzA1!lJ!_<Sj3RFR4@a$ zalZTT8j^id$~WdUM|UZ;MJ0Gs|A((KRYr5a1c4J&Zzk^J+?Y6cooHAA54*l!ed6=S z5npfW4qv&xLhn6<tocl3CDFU#*q0#^u0|5gW53hT%8+|v_rChT7uNNc;s_BI-xsAd z!#N)Gw1QNEk=4MAJAK1m`qst+pGI#hJ_@`QokBA>pSFo)Q0A^?{k_AM5pm!p=QKHw z+!!z)AP$viOMx{j@H|SF#6bg4<s$m-#BU(5FBq)9Ez~=AjP@&>BcbD56;Tft#y^Y` zwZnp^pyO;zM5+GGRW>oLFQ5G*Z1UV<-nzGkz1|5<XN3ulfR>60Ji*jdWDRp}r!a6> zR85v2g>yuO;FZ2J0>`kBt8CB^xeN0<(M^c#!f4oR54$8B=MAT|>Jax0fCCjfc9mAr z70GM9Gmvp?z-Ycwkx@7!Z#1hnZPiJJb<|nC$@vFCTm}G52bc|*xp^LX?6${CX`xI3 z#1Z98Dp>J6J9=z?u!IB`wsp_D9FQtzm1N<P;5AXli;RrietD{GtTEwAYr1KDL(lP? z84-m-g`5{gIfCW6J3R78w2T9^L?{IDAcv%KUb1r#rAjEnD@u;S=n{#1Gccv75VBtT z8V(s-Aa}F9EV>dx>o%>#wLZ;A{NU3O{N*Letw8_t%PTk&;#Kd*p9q!Pr(E+_Zz=z> z(KX{tSYZ3U6`H&C%~IdNA$5a%?X-W6x&Lu)@MxfF@Q6*XLbz^e8^6Sr7R@+7zvlL! zMCdp=SibSbSwUh~=PR6aHKfk#VpHStS7YRf?!dJbDRv^1hUM9vL93AV5YUs$_aqK0 zLHu#~!@n)M+~X=rObepo`wDV9_D|#P=#7nBaIqX1pI7^_JU0%R`2thyZy}RZrC-uc zBpI9MJnP#$d<9Mt+u|^YCFVBP_QCWVKV-sOF^q!*?JGgm2^TekGdD2nyaex<FEb-c z$4*thx0v?8_E;R~xz*<T$7!#l20c1jZ_%4*0lj;^eAY7wcw=j_58HYAhTH9jA13ND znlf$?tCnMLJlr*)dimM2<h{>oE&dxqT4tS3FaDZ+qp)JuIxYQv;q(@(i2Ta(z&p1^ znm3*lk9fbi6IC)}HGFH2hsVCL4x<dG-4(%IDu-&z9;`C;{Bno=E8%Mn%R{OYVtCao z{AQw^v>b#!dx?Uq*uJZzlvZVURuq@P%FcV9hBs$!m7IWlAIRk3sM5Az^UW@JK2jBH zb}ZN`W0z5qK^K(C;FA+EkjMGz77YItz5_tL%*iLq8BPIE_2j!4GMR_aB7N{$Tlu-8 zg{h9Rnaz%5m4YDGz#+dRZx(7TBfB53`t^ry9seiNX&|V@h8hMVUz0Y#{yZo3*Hi_` zb{=^ZL63DCOD9COYj1iV%G^W<IQBgCL1u(=+_Nzjjy}Hb>Y-gp>PF7{?W>ErU)*jm z$i?vkqkJC@@*lK}eO>C;Uwp;Qa+|98DK)aNKf~gPmUhhRetecaU0jO`!0CL<R}kvS zHWmnT&cacGadI3Ja;%2;R`?pXK$zqT8@Wn2o+C1Pe;wqNm0Zgzoy19xIg`haI=OOK z%&}{_%opO{S~&e%3!_kuNkeLIfXGBzgAPE9=7>V1nM{95x+PZ)NIOh5hzH>l5Os3S zN$f1yE?jN{5*I>Yr?w_^BjahH@>jB^MWu!WQGp4JkfzQ3bwoW0H5?L5sxdCfKM~Ko zT&~V0X+aCA=Ek1wGUNB=7*#mHr!LERNOFjO*r3b%JtGYj+z`3l^W?x}Ciiq*7>}QA zWxM_CaR0^Hi11Llo4l8En6k^5j?R?@|L(sBx!%lUQ^F_pKQ^@I>n5L$X%Yo>e4MVf z+E2>#EzYYPWGpdW%|BwR-@bG6SQz#W&iu}nVUJ;@isu)+dsS5P*C+MeA9&<{@$JbS z14};*1*WkRK!+QR>nf34m1fp2s1k-n=6&O?!!m1GhliQvOOP12@n^tkK>7BL3~4P} zp|q_rkAM+@tDyYpt2Lvjk-|F^q;>vWr3HEbh^@NXxtl@6I~hqp`f_aCCYf4<NLyh* zY>U&gk9LtogcufSC#P~LNX4svVVBO0>n8Q%&DerqmfimRrD;ytr_XX-ql57?%T8#9 zI!(a2!s(;YBx4T;Tyu4L9lZ2#<}0Pf*JVK#brIu5fP-+`@vPNfS`HdNatyVG!t&AM zQ#+fvO01_iAiN@~KrGgZXN=7srSN#C?`|Q566W7u@3WibtN6YUsKD7-<(d;d-9eWW zdIVn6SqecxA^VGz?~qrcP=G?DwE|g>TI)Ijh;d4pJnZph*(K$CpKKJ50`}C9s2opd zIWhFG#}M~4)R;6^MGlLk&D?Itj>}~>zidd7c_oMG4}}RFiUJ+wO&pSqt(-4Bq+@9n z_>BWaRSyD-f?kH{pQUM<l{<0bb*!A7%!@N}F@_Ni{q!@3Fq2tVBc!TlJpTOuvvB`M zzxVvxI&4Z}$b=0sb4mQz12xj8On>xPuP9<jExjXyO+`rT0Y7^e4&Nn^hV~I6Sxn7` z>kb@$_19fTWhTaxjjjjMS13g`z=RyANx^xwgh<iCD@bAX!BMgw@uu`XGtea-p(055 z$oIjuH>(Q280`DmXZW^PUwo>1OH+qFd2)Y>B;&A6_Kkf1IPyOq$k1NK3bke6%n8yp zQZmRBS@l2d$Fmz*e`oHn7)cslCMmZ9wCgHR=pn0Z*5fP22>Yk_)%VOzHJo@U_|0#M zd#a8+v#;=S2!*RY_boQ$+TGj9;Tz|rP=Mry_Z9;I|JI6~>D;}4SH+{-(30|}2+oQj zfH$_$KtT;;YelzYe<NW(3212FHlhE||M)*^^}p9!()jPHYnYTwY8{B#F-=Enn=U$1 zlt&~gFj{EF|2&moZ0g6kAvuDDxNQ;K)GBkwAX{9xYdinjcC|_Se0>ky+(WiJ2P%s? zQYTYP1ecF9X_)2kzU3H5>6T{5_Fz&7m^NztQX1vnqK2uK(nrfYA>L^K-=b?i`10D& zdLYBt_5JgW-Ts_Db{}gI-^GP@T1ixnby0r@sE`)VbWUHq1H|N%3(&DlC17#w?czdo z8#4gzwkrwOvwanZLdok0OtIa`KAmRHFRz7Nc=5zfQH_1D`bf*)N}t@_V3rtM{%w!B zHPrU%IjV|mjHyA|Asa;L-Ru9dzyl8*H{6DdY4(ptqU{Iu`og@c)!>K6Zf1mgiXs*g zHkIn{$Nt@R`!4y4wc@J+0{uk4#EzGSvlk*07~NVyE+>ZTx7+r5_Ju`Rez(ZxobF$8 zLSc6d82QHy3NrPztvohW99hRS1S16^TIp3X;DlQb7;7hM*@Ato^aV4pHx^ktoW?Me z?;e-uLD4^)h4e0Dx(vx^uFW@-13C6ca6E06n3;(7UKHk**L=LMwSXC&5(NN|^-6Di zQvj%9M$3P(gCseLMSpSR3mZL1%o?&*h#R7tTZ)!JOJUJMA}110*jQ$Ehm>NvL&!im zV5P3CwuVFmOLO63PW#xMIfQVj@{`m0U#?><-icmH7|mXr%W7RGeTq^i21*}jogK*W z>+mnCrl8P7wAkrG&HbPaw|`XLFUqstcR#k~+_eGOwz4CR5)(i!-HX?Gci{SQz%?rG z%Smf&N@?Ur>gIvqbX%%l(X^#ygzqET=-Bkw)hBeXHveDoq7UVQ{Qdm=ucvIM<y!XH zZ98LIcE2bvvEWa&Tgiq?X$@$T|5|`cM!`2m9@A^bXvjvnH6p4qYmzd`5=F<b(d#NE zcN*Cogm9t^$Rk16dlXX4xi%G*f1kIY4U9^@_lw(O<{DN|Pl`vq{%WA8Vu}WTz7Zo& zwH}mKkSB|%8jH_4u*0&zsvR>r%WA6m{Xkfhqv$4lj5bO<E>^urwH|kd3xx8W5k`&c z*H#nF!tsqkWpyEg{S6w`NpBrxA8%%mO|G%N+<#@tyX*#sUj|u;pAXP}e7^P5l9bkW zVO^FG!y`2cK*$4eZ=Y;8tbP_QoNPX9c1B2MMWH*g6}S{2XvcY2j%89xW@SpzQig(_ zbMy08jO?-S`F{QWuxlUhRmmEi4vkKX5r<|!UFLX~V=x)x3utB39zykzU)HMU7=%oj zI~%bBwgcDL8`v=z;Lk_v2Ern>B0()R<|(j!68w~C;@yM#+?9g|Io(X?K8}3-@AS6| z?B{ZpFOMFq9y!#RZz1G%;9)~50Ppt0W==2_YUXEka@P>>qFa!8zR`0yY!aW=dRR)8 zsgFp|aJ5jKHE}RXQPkfBc1Mc9+hVx&%^iHCUM%~;q3aKhtl)%GqV3Xj(9v&qa~@`1 z!;fZ7W_2PqPyfDQaRIbFJZ}6FQ)O&y-1^Oy+}1t$tohy|yu0buVFjG-ss~%PdB({` znd`{cXEAr@p<bCBqk{AgQqFuF6W4kG4nnXn+At{5RX8ASW?tLaw0&YHm@<S&@VZ+A zM^=9QBjrmCIL*w+G+(DkBsKWrUllRz+~6l9pR`hE<?>Vlu|1bm^=!&hDYe6`!%pqY z0NpL~Z0V(&FT*r_yOg%}(G&8p1MwO<bM3`iTB(IcO|}v(cb(fXKu93CvUu2P>`|&= z>O-8Lo;T99-hek_icMr4u}CiY>?e~Gh#lYz(OZblUKsNeALEk~fVha>N#}86Nb4ka z5VQS&F-na<pM;yCfQ}R}h`<TPR0b*&FF;po1c|1bH1lb<)`Wajm!r3-6;>_u?KrY3 zdc%&*NB-XNeZ%gO4wC*g^cE0SqGu;jdiz#z7M25r5L1=3T4Km)T~9m=pWvHrh7ny} zUFjqnocb2u)sSw@GaOTaks~%$9VO8DAdQxB%w*hq*m3sFje4=4#=8IztG{}d3Aq4s z{07n=lG9Qq87oF0h`+&pd~PDTewam30$uEvmtqtl+sVyNh-4#^zIX95;ag4Vc^1{_ zj5G(4!lf{+o``P~thu1uoqVg9fy$(U_-cUr{1txVj2&XWX82~IzH^%WyNXM$A?0eC zp@qq8PVS6Hdw&P-kqg$~{;eOpgwjNsZN()E%0D>Ibfnx{nrCKeX`9=_X<Kue>k{_T z*MZobj#^&%yJ{b*UgdsMP=ja1C~})OqBDmZ?Q1SX1D)M&9)7M%d>Z~f%%lFwH*^sK z{G4qV6xpRYV4c?$YmQ7oDo9rIwO)Bo_6*fGo%m#_JIzpw7o9Y=a4T+^VJ)#?kf(7$ zd#m-gR2zpOJvR(uyX)V$wBm&C@1B}d(evxr_TPWDVzQdkMk@<+T(e6ulBXV7X`E;; zGU&`Tc>Bo9?M$gdJ=UG(aki{eZMQ?Eb-IDtPQ4yilRP^Ir-OS->04-WZjbEy(;UKf zeJ}uy0eAgQ+Cc|J+QBx*?EQ{cllPWw{*Ri3|LE^M+mVgOqOyO8whXM00s7l^ihOD5 zqm(<pkkUX>njJ?&_k)(X=}XH|L>+NRCw>8aDQfL4j5j!Cvt+FAdyxdndWlahd|&sE zkz<pk8q87KUX7|v9NCEx?#eRy?{D3BX+cb*L_+yO;R3-k(**sWAidzf)4F{pr*qek zv}FC5but|Za7^%Ei3`*Vd>>2IYY1H@uvZxQQM<kCBLkW^VH-ANog6m4y6@xk^85W4 zMlk2SC%o;qvz3}wI#hp?h{4!1mhhMP`+kuONZ7fF^I3w<q{6W_Xth98{spV8AZ=>Z z!k&IbYfL2?N0Aea$k%!--wuru$&-wH9H7-63D>2Xwj3@!VL|=6Hp))8APR@|#iu0} zgR;#M5*)(gCUFsArIn@u?-}q7<shr`9G~L878hPxh|`OhGY~$tUYuz*ojr8_j(<5e z%b?LZYM#+v5%WVG_*tWgq%rQAPuBoN*E=5L1gJFm^EN$ZlT-7vkoW@prhFQ*k3nTT zBMl%GTP{6?tGG_K$3$2gmm+vVhOD=K6CS+j=4&b@D|pW(XVqX~#!V0R-Ic@3nd#GE zp7^p?<3CTa5@^;B%A+5;W>)3&_lm3C14hR_Zm-8U_F=YwP){aw-eo=P(qX+q1btQX z4Z=f00IO-DzIGbPK+dD_6*%OmrvJNw&(kFNCTV)MhK2qqtsOO4>@#-e757`|*K0MN z{Y;n~PGAOAkJBf!+L5Wu&14*PqpTND_pMeS_YT=%j0Tn691eOr%K~=zZ$v!-R~|cI z+`s6A7p1{0K4x=op>Gt`lMyd+QGi|k9Jpjzy*<AYzguIL%lh)-g5SfNxg$v(K1DCD zQ}3YMmwiO1pz_a;a}82!F4XU|x-$NS>8+y}K=w8@8wX=_#v0j{q@7LD2h1zMg_2{^ zdJsv|bR<v*Y@;=@ZrPTh%B%NKDB1gE(erotTF{sLbQ5!*-%neB0gHc(PdEA0Fdi(Q zu37N}#fCg6<wi3mnfGKH;avjvfyL?#=O*5i!Z~a!TMz*5dr+8=80Nj8LioYgOAXVV zIxD0k+5}YKnmS8w-qdLwqF*`dR8j>f`GXP-=@X)c)jiTiShu&`8dX?6LC9cd$7YQI z(#l6b*%Iij2rBDDUu&U-^h+J6N;G^P3&hUl6&PRcsolrJ{|SC(+A!>I9r7hCuztqk zN@kyPPtinWLza0(wg0o4ZtMG>&z9<zPRzghBx-g%p4n3NZ8)Nt3X*@RH7;|n#AmvT zXULkQ&4duP?;r1fM{XpZ>ANx@Q4mtvZn3wup`&H{2w=j1GjQqo?1|!C4?B7J>X|RL zlb2gT{!bhN;PmUIIe!F;50d(ukn?!xOV=jqYUYjAmQ<mE{DwN42JKDKDw5_k1#&Qv zZSFDENm3P>7oB`Ib?)>g*3$3EdI@y>gGJK=<UmBG%g~s^$D*Q^Xp5N>dmGA*L_ION z^2)R#>N4jC+F}t2>(kp8HUy60feq_=Zvt8wkPTd9p!#3a9vQOtmYSi|--&i*Z^w^o zua}k*HNwf0K38>3*Jmlx%dchEh0h@ElWGeWagKxLJ$UDuA#)$wF3*08L+4EEKe{<K zH!?X4Ev=Fv9Ijhy)?VAN`TQ>6uClxm_Fus}%`b7!2stMQS`PD_kPbq{dlg<z^qCdy zm-Ml~G28M>x8e__7&|&K<4@u%_Z)TSKhoH`ZU22to+e$Qf_{xgEExhZ4ano#g3^<< z1JJaI>iPm&Xv^tFD3RW2%4jz~f6C~<)HYHITA_$`-WP;vf-&)rQmT1<MA41aXUX-P ziIK$WlR#?rR|dEsu907qPQ<p}O89q`9_XCz)h))aw0F5R;to5!<(#e$-|?stUs=%H zhz$xzpBeLr4)xz}7qV%;E!%%jlFv#1HH7N<Ly$U)%0SKug>+G<@J4Cr*ZgfT2jX^+ z6&XHw+szH-g-7^7x&mh>ul7TC{qkqV!k^{&Wh~!%Y}5Gj9LUmu0M_3~pA*O^VrN<R z5VF4UuF>~eNrW_Rt%$}ixjvA1i7Bj}qX-e_K=eTLyUE!7A1Lby37yqq&L^xePHA`m z3@)2b6YrM|{56Av%mnJO5^UAiwHY10s~>pxZj!h{XTSdaz?(43b4jU~v$2~jfAnZ@ zV_>#Vf$?82JNy#k5&KyDTavS;j#|#4Wv+QV{q~{@x{tDh4m7-c`mxE;&&yOLy3$T7 z;eqJ!`OYE(Th5u>J+yTkCqsL^?gO)RDRy77ezEgk8Uuv8U&{_i3z`0ik(sV8>Z(d} zmn|?-6fJ#%(j}gTwp0s;`BQg;{Q6swgl7Km+~**OdmNgX(J$&Pa|sS9C&aLSOlPM~ zD^d(b?Tt!*eirK!w@KrE_3n4I)u1dkl|tP_(jvEABUwqV3eglBA0fuQ=J_x=h#XC= zAC4ymml5htLmia3q9G%qBhMv%#WdbVUz&uaJBAg*m@|C>jB(YL6^opez6x`Q8DA(| zGhswwHK~&HU(`66kd=!wF1GHVo=RiipXT5U@uDZ!-CtjlKk2vTiV4n(;J?Y=$nkS_ zbM9Cyc2M+<Veo_P8v0QGOu5%NqB+hw)D<Te`|xa3D$mEr#aND$>XjRni!GLd3(Pq$ zF!ZZOoWE<>SHKE<-WX<ZzyhPP@qTu>^Vwf}aI%=Hj=oWe7Eymi16;*}d9^Ol3rh>B zcVhUIwAL+-MO+C=nEwRo5`_%SN&JPYeP2kQP7X41T;$-8&Rg7TBt>tnh1D}R>z5-7 zn~vic_)*ii34I|xjkuautpfkxRlS7Botl*oAAm}>F_Ez#M=11wstVVB>?mq*Bc!|@ zdsjfiS$;Dfii}M0E>`yTVRKI44`xo7#<m)e-k`UG+=lUmc66iXYg0AJ?!j*ia&-PS z=@XJEh(Q>>#P7DKgT3hz`(~kZIkgo*_|YbuaC{0+I+pxXJM9(!$TFvVJL;nzlk-M> zAL&tFmRqUgTKA#SgPv@@Ff72CMy(-DoGmrjKjd0~S?eO_)t@VHs4wfMX>O}4s_Q)9 zsA6mGr<TCF^n8GKV8MI&OsV@&${qI-en8^WqQdoFsrnuzv(BWx&$AI<Qh1fS&j{Bi zf|(es6-5fxLRMKh8xdFhs0e%8^6QlsuLi;le-_5S?KfYWZ|`&F&+dI{B)Q3^XssOV zvlk67k`K;*+i_nvnxK1RkJ~Y$D-|{7*DETD;x}#BqbV^3H#tvoDO>Ow{Tupn8iIvX zAqJby`w1<(TwxxsHwL9P)DjGaCyLOpVcA!0bLmqH1uYh8o5!ox2cox5t4}H&-PGu@ z(y?eM>^*P}`Ib4Ek5QJ%jRTdL5^9&MJ&USD0`no-bQY^e6V3=R?6d%Wah4Fxqs+bG zEsX6#Bx(l+B3I3|qNc})`H0$D=r9;)%QN*s%(bdTpRD-wVDGnfCzEs6qpabwMOwEb z!j_PHqx?ZVsxl^XKP}=7O_QMicK>Tv_K)tI{dBht(;f5!iyp7vWh$o;N}U<!`l)(` z^+UVWx9J$1Ey@Y$C=IzBc<H6eIrUAa0`~kwd@W1<DV{{nW5i~p0gs3!P<>1Z7kE~k z<9K<W6aIWRh|}e(Eesu7vy8%zq8p4uQ>Ohn1-VBKHe{YXSGC6EV)*ZW0241sn3}~h zr>XaK%yDL@w)Fw4oj@vKe+>4Q&BsS`Ccof+)Q*c$A`J$U14-|JDgm>}c2nIf89IYe z2(%qVVKsYSV`{7CF+$BRUpo)(J7yX?uQ9&$=v(LY7NU6~IpnGhumY?-FO6-{QcmK* zhR|+NkCeU?CHTZy(yt&PNIlx}kL<UK-QVR2r}$nw{>B@$3$FS9u>2W5FtESHFX}Sj z((l>ZEgd$>R}9ZoGI@XpT8tEOf%rZMqW%)_we5fOQ~WRX?^gY}BYHn#jdpB=rVL<b zlTR$2CVNM(BJVcdZL1=^MS=kwm{H7m&f|U7LDV+WG+!%-dDmnAYkW;J>`Vn$we2dy z(b>#zOv8<`7(?itf)NqivC+?D!~d?5Z<l6)xkYyE_-JlJb@V*%jM~He15jl$sheD{ zy07iE_lx?41a4h=VW_ZFxkpfER5^7>eHk>{w8te@{)w@LKRf^v*6-y@PhhJ?Rr!9& ztk=;XQh}Xgu}Ee&@(|E9(ZsF)cc%zO6@v-Y=f(R-11i|2>P~ztNKcUXKV6Gbg=q!d zWdwDBUNkKgeIFg&x)0v+?<yxyr)g|oj;k5mnKWvbqjfKNB|Z5`yV@?n0sW3eti|&F zf`jc2r!!)+&dvd5^@qAl6(dnq7u{e3V7767X8~yl#3NVSd;{5uKKux+PHGS-T@Al( zqb0sTG=s#x7ba>H)Lsgyu`P~f;mIb9qK#KIvq8EWhvyO-mS@$5Y_MZ3B%z;^+1qWu zZ8v!XE<h)=Gjer{W7*p_;Qj(>gUCqoq|<H7fS9+Sj`QqWx-CAMWd0a)3(+6=W;k@a ze229qjmK5F*5s1dg02{IqR(n>X{Mi_S+Oj?bdzRc#4t8H9e>iaa=RQtwdCt--oxas z5ohG<3O+N_GneEU==iq(Gv;m3e8FKuS6A6$kQ_;_<L87G`JWW)8}#B?uc*xG!kI1B z4UbC4+M?)z5kVi+$4CYKVVXTb;r@}h-CcKZdz@1r*#47Vp<%y9vl)|Zoh4B@^qPsV zkmst(*hqEMUQ)ZSep55mmC8bHFqtEs5nJNM(%6-Idij)541dj?R^{mR%BQqlz<|mi z$K{;D0l`hpQZnm=Yyw@E83e#J<Y-{Ghyp2Ob?bHYjyTS*3@7sI-9Foq!DO%9{B@OJ zDxxkmIPssx)om-wY2(Yxm$j*T(<%79WbKS-*|mvaiSmOVw~~`(SG7CMWD4$9lG<gR zWlO%~K?c4?q$TU&ZY+`(7D>wk`opxUQvo7Rd0l=uaM^31YB5ibH<xdK+TZfL&juZA z24+IX4ZYYI+1-348TVptGWquXOB6gWVEzx#82tbPY5<C-1q9BAU%nE$bfDJ)HBu?K zP`64#`^W)KD1jH8CU6<mCR17u!bxGzK)kWc^t~x1*(RXyE3h~OEv^`j1eTYeV@!;m zQqJz*NfnKQ@i8^%tC+FOaaa63%+?kiL7LwYqkMhHj(UgJn=~Pxr?LCNa2L=ndt&|g z=rtcJs!IF@L!0eBzFyb*s4C)x!`5U~UP5-9wY$E1v}b7Uv>#9~X4&lAd+=zn{@^2j zjrNfE-J1(tl}mfIvk%NUZwd4BFmbL+DgEa``L{`&|Kf*$z1<H&pEJe^^)w7ZXh(^) zA_vo-E%0_ebqztDe}e&z*1f{4`A0yB_sbPA#}XeREQ^9(32&E!;KEbV+}6W^Uc>1& zAyAzwgosAz5FDML)`Vi>wH(`l`qFx-0YU-xV?7w1k?Bz#HNAYg!;G0>kc#NLv?vrg zHYM_d6@*CsmuBGvW{lzq=OB`0TR=Q{49W<EDyfjxd~H>OY#W8EkHM5lJz>!2&r}LZ zrAwRY_bl+Qj~{0}hsM$?Z?M>?hz}!r^N4wQF<N3GEs-JM-pPPYpLOKTpy=wMJ=A36 zWtNtF^<w0GyI*!5H@2?100+!eo?a$QI&S6alPLeNHObJ2X<?0G&iz(R*dKi9<HIPR zdz?5VEjDEal2gIbu4g0QtKQ7fACo@4^Xp{V!(e@4f=Xtxn!dfN9wSiOR(vVADoc9F z5#E(-yFTNk6Dda?P6RgPW~ILWF_3At)+2VBH<i>j>E}x~J{aO$cC*WG*6m@Sed%DQ zU}}>AweFGES)oBK+m@kswwxDyAVT=)WM2p0e}mrMJx{X+_YuNNIjP~7V%p0dC;#Od zXTV_{%^Lg&+=hz?RZwwwfJEXA5?eS2N;wNGT02m?B)j`1{!s#_?ZA(v+V%8&An7+~ zh|XV7BUp3zPVesKs}IO_od4=&NjG0v+rTiX+3@_a{grr)qmiI0NcyGHtRJ511DOTU zV36;j-k31}g0G(TMaA%j3acq9fhg@ho8#3{yxvBf{4;DjuDtShC(ETm@t&)Qd8O6r z*G9*GX4S(uf$0`fC--@yL@A_$S&Q$ij`u~wHA2=&^4KzeJ%y;?k7iW{urTW+XMr`V z6-{)2>^kPrt}P5<ej~r~m`i_#<G{tgT(&nbb@JGm!~1A8vZm0F)+{?tg?XAmKi7q0 z-fZY|$oYAF(j`Xi^P1>GI1nFkdTsKAA))a2=~eE3uK&HccG{ynq0^J?o)%ixS73XV zKg}tfD=`?Ua5~WQwy?mX*dS9g<zXH@n*C#M&x>PJ<VN#Ap8jdN_5NH#k5bJCsZ$$o zRU2&ea(!s*GOiQPc4ylf^tnI(rcx>wa;V<(v-<qFa8{JkDy5;XRF6<ZR&W828q3v` z8T1IK>0~w1E<zzsbYJLw3^JYj0nzQK&e_8kAYFR-IaGmqGHMqtd)=cT9h!_9zO{d0 zPeW@Xp?kPe#^N&Wu&dKad!)svwZt%}@!letg059(1MjaDAd4umS=oaNfLlwvPv~lE zBB!?)@)y>Q!)deuo1Ia*X(iziz(>1aVVI{qvx+%(r^aPlGW4R1@NziLoIf2IuwLie z5Iv#X4v|>yVE^L9nGVl$=SvR^hz4f&avts-$PHohKF6hNyFJKiJp1$EjgJ~0?sQwV z4986OYkgt78`s1J4|ViU4AV4Z_rTEi{)_)#0t>YtE_0VIayS3#mq6c)U@y7pra)RE zg7gv7KW*3NKM8JKB%?u;fvg{`cD>~v(#*8Yb^&FSZ@*dPrO9O@)e_UisKLR?w91~P zdG?mu9=Ct^_<a2E_AhU*vGGpMg$oy44A;i5N!p$_66fw}ze@851>Ox}RZYo$2gujb zK(armHOIlR4I+?7v-n>ull^#zRFXj^^6vY6_ti#&$X6+wBwK1-e_l49Ksh}n4yz2G zZtH(aedT)xJnzLSFdkOrD|~~3GGSb;FO(;KT9l|#1_Sp#Yuzk94b*q5{xkkm12)^B z_yL$qoh7o7Ht}7v^+5XZZJ<0^-zkpMwWyWmqH#ZMj3H)+_MYYk`+VxT2x4F#GXRXu z;`(bJ2&Kr+G%9@lNxCd+^L!+^1s|NGj3FnxI>>jq%JmCi#ti`MfXAzgM@g+vznD!{ zByj&^Lf<Nabnb8U#?Xojj52@x)~dlzzp}IM<WL-zH|tuBNPqY+tL%=L>Si!d=$qzc z1_yR@NCug!KQ$Th5wPAco2dnli^3bgm>-2dXVCTQ=a>S#1_=~f_1g!cPf*=q8l>ln z+1M?>uBmVwha1~|anD=FtTVP{Vm1d&nAOBZF)`(R<r^2V9x*Kj9rh6xWG3^;&$vL> zLJyiox%0zZOc_{(ctGdYoBcOY<^Kc6qs{|y(`D)?S@iTJ$AX$lmNQqtcgg^Cs3*j* zmNl#IlFVR}bAxOZ0>E_cE-6^CTh;gWzOV}@J0V|-u+&!#s?H|!?8t3L$yfWPWWDHm zJXj~(LV^d{hP;Lul(wSYVHmlA8Ks*y$sHT#WhG?uMN5f<`QB8?ZobBNgnVW2pJoWo zTSJXY({G)z)?vu8l+XD83q4FL^eu9Y38YzwuDpD(*jmFR)JuAMq#zC*15w+X-aVP5 zjIV!<R%Tu@O|(uMu=al}eHQxB5)qxhXYkWb&GFaNdT_ZdDcHUC-C=wG+2$jWm00VR zXZ(FD$4yS*R`ltC)})$C{$<lvBCq1CQm<b{q*vI0dV{*i^GwCiaHeNwzxf@H+M{2m zcbV-jU*~#cUD<4tVjMqD%N1Hks`t@j`yz*ebU90D@;;o3r6?`;=Pz8dws>VeDLJ-B zeJDLPA^B|R#hY(WkZuQppg)&2#6ztuLaVtG!aIEOoG~Al^KwbQ#SluL>FU7LdAuz7 z@VD<Bq3K+;@b^Ppxv?WV!jC_IeUTey0ti<I1Yw)>q{Y<h$32Vm{$2IAJve{rJ;jHe z;c((-gv=ShlIzSCi<2d9!MB|vZDIzvf(C%Gey$NKJ`Qi}O@Noha%mut=^3a_yP`m$ zXV|q;E!i`6W!w)NfLTA=YE7QFr<Qy1dUXjiU}1bU&^(j|S!lGbA^JfH6bq*fwS>a9 z`#u-GqLo|JZ>HDSV+G8(iM@<kq58cA504F(2J5>IT5VfvahmP=@TJzL64^glMy>V| z$9Gn~SKh^)?C3Fxz|EidaUy~NG4;>?M5ZDe0j7%U<SN-{fg;UO`dp?%xN{P|FSeiY zT~$jFttZC@35>zaf;RMT5(K2ySw1Ny|30H>moR4$h2%FT+K|f#O;heQ{R(sHNGzj1 z{rimi6&HQ@DRZ-I9S&KuJa+^XK@vDbn8P!TMp+V-gy{J<;tPZvFcccWdcp(*ebN?n zSArqSx`nh3juT&)r6|4<8p@Q(D(5h`x<F4WO)<79ikANP`3V&-d+Bk{f0#fKmjpSA zXfbWeNR|3(wQ10>1(QnysR}3qkdzrGpvEHATer<<_E1DO>P%xux<rfN#U6}9eM^l} zSLBr&S2znNT8!^t&5o?L`wXPAQm-=vU4et6{Koo+uc!_2np>xx-UaRt$v4$XY8MEH zPp>SUTBGlg>+bGBDcIIAlue*l>`wf21J%DT?;20vH8?i!+4C1(YS4@GCfG}Usd<5C z!t}onNZ3G+JNY!5{+fH2%)qBHab#su9|%dnK<fqEIGGA@oe<xJxVsGcM6hYmsWDxo zH4utfAlW-eQlFM>d0!|59Ec-Y6MnV;6b<$BdM=Eirwu7?_j_u4N~|Whb(DEA<S9xN zUM3p6d{5Gs^>XX=M8?a>{!*a%!(!IbS%`GpWcN-w6?Qg~m?~D6jLuFdJYwwiqnzPb z7|vb_!QJsU>xlG=#N#KaAfY8jXt~^uc`j;%R@*b-`7p5(>HU~`rz0f}qlNkdw=6S& zy9!4Mrk(jpR#wQZq&>Z5-6ce){xH)mM=G$MUeexC8rA8VPD?#TC+oY%4^cWy<pa<f zC^Fq#V8cWxH(yaWoHhw@B`PtZ9sQ<Zy@*-IN#oyHd}DfhUI0)$J;C<BDef-v{Z%&) zFdWOfu|+kblZR_Q_;^=XDax2BG;a`aOBj2hc`MHPLhHL?-F)P4cWLo#*HV2nHt`>` ziXfZq=ent*ZD%X33~C-ZRBbgfjyM;X9NczH-P*HcO2dujt(3}s+n`;sdu!R*ec`^E z4b~Q_*v=cMx4D;`4U}XxU<Aln;X*3Yj|<f1PH0$np^dDGt;JFmTQ`9~h=utOpeq)R z%;Jk>J?J$=IlXv`?SAe}gFZ#?O~L4jJLeb=#-y8^K!3VH+P>SOtXGZ;13D5+pT>8f z;Ulej9pe3<3>%)q3Nieu49JsrXFjQHhP=^~s8t1v*}f(@uSu#I9Y}d|4Dte@m$wkT zL{-<xOMg-k7_r}$gS(By$GABDH9KSDtW0Qne8E!Zvigw0X_|R<obQFB?MKTpma*-v zW@8tjnTi7IkutTf{jnbpb@!~a?mBSoa^RUz{#UnZ0Pau!=r8r*JT&bwsE6LC8&q7< zC?f<8;G7-II3kney_Kh;|6dZ<5%FMlMe9zXIq2tlx?W=v)Qh_W>hn1QgrcmAELXkT zGrazAh~P2vy#y=#%~e<NB1CSN$0wowcwB=20Iwa^xP*D(y9z?b%eKNQZ5V|Xt&7>7 zu_$fWkwsb$6=!L0#I|Zd`Dy-7zuw;9{-?fT7JaO9-zqlow|?eUW`M#r5{C+%MQBTG z?<*{~Ghw~;EHr#prjnbsa-%@(OX7ecm`_O>dz8jr(-l%iI3~K8@_NPIWPF}0iXNPA zFiO!!zI6Y|-F#VAN6Fo_;Ii?hI^38emU6|R;<*(0CE}(49V{PnXmQN&+puf`+}kZ? zWB>i_{h#U2=nZIGuRL|68w}os#3CAJw<Q|rAm{&Gb!W1Av|RMpx$jK@G3Y?<CT!g_ zv$m`A(BN-9^(bSglCjK+^18Vr799KcwqP|zGDNSU<i#Gco{8V2)yx2N3U~=H?2aZ% zJ5t?7i_FW(^K{;6QN3~20h-BI5_03|MyNOZUIg6i@_y-+E|Id^^}X>`*+xmeGy&io z%Y41)7C*>M6dONmCw-2-vY<RpRF%?0x2}rjdI>GkvlN!2WtIf5Q9-&rNsD+Ca>KUn zegDnoF@LrDE6%!r{%=R6)J=k)1vYE#Bj2}_a?8YLd?^!-kq1NV{tn)<J~Oz>jnaI` zmY(5di?Jw->i-t?5}>j}RHOf>$x!>=IKNt~Lh1%p;L=1+l>`Gj7R<ETy#JlL5h#>8 zhAN?DK|u7EB#odc2;b?lYDN>EL^SuiT(O%mAh<VXp>k}O0%OSuM6VhEIgBcQ#XMX5 z<aF-z8wcf^zXmo>k}d&BCjc*_<4HX)sQ|-yW^+({nRuY)kN`Fm2i^eOVUG6(m)sS6 z3=`)PkJ^T{Fzov0K4?1H{U@cjZ|nXMR#l>~_apQ#2+3#h^y0(vOkaayWBDFwkE|V| zWMNnm+PoKiZQjhe7PjL1_{i)U!1&^WJmX_$AVkuA+8>EKUJjf0_@1$eTuk&y#jT*v z18O><9-;QNi5f3)EBGqYfxa=Y#_=^zaV!3rb<bvo%+l)42S)f7uK5cyW)1aqYZWS% z;gFywT344WJ3kTp)2FoB(D4yjzp()v_i6T^Kr#=KBhJ0V<O%-ychzH`F?4e1u8U0# zgNr}}cYSvY2j4P&*wKE_*db{y%-A}(RW3;H<F@xF<vbBuG4a)E;CL(-Tf;6QVvY1+ zJvCNao+=9ENUYGYm*Hdl?dCAEU0qxIWoypa7aWEodP_HP<$aVZAD5Z=-G0i-c<PDG z`zqAg0rX@xIJASK!ugmHvH$_#*%1u;j+NPrGJK5Z(?dyRUJb&XesMqlj!=EBMkt53 zF{ZceYOZ)dIAfz%k{j>jV0C)XO{R$!eUavo`vByK8NJLE-Gg-K-KJgKk>i0R-NAx} zt;bu1^tmU3S0@SOC;!@W%ibD3c3wD6f;P+-iXT1vunl)iJ?_He#FDdt1~lvY6f2_u z+6TXRJ^ur@?DB7ue!S}3?@BW=(fM&E%SPo!E>Ev}!OquRSD<yk{>}F;CnXLXAd?Fg zZztN+N{fwKRV0=Pyig_s{rYR`DuT>K?t{gvhDVb)G6JUmnM!c&zsMccOs#>d_>nJ( zZxlXuKJnkvGG0Ym1NyN4(%yLnHM#z89t&at5orQSEEJ_$KopT2ML<A6ij*MaAW~zD zNQ)8@73n1k2ns?DU4#&+5d<QHaugBi5=uyr76>GYA)0dbKD)CsJF~O@eY?Mzec>>Z z8JQ>K$#Z|pb$zaxESDhKMh7$j3fQj%!I>qR_WND9c+>m@w!h$%FoK`J$@3Bw8B-pr zALQ^VIQJK36IOtDFCG?<!o=^{!XO@A^B{P0cel#qvnv}D&U)tjEsExE5_pUI;0*@~ zN}=!ZsXnsY)#Aca)=d0%uc^r(oSabkPR90v{vMn(#vbX<glvK9RVlPr24~;>HXC@j z4gb10fm&s0=D&?#SQl*lj9L(P-SWQT;gvaua0Ii)xLD^@Ia|jquljZMwY}B43Y?$E zBl7Ae^XBKBZA*0&bh?&<t%%YF>Zv1_KMyYsx&G2rJo4+LsB~4l4dcaM;-YvX-28=f zZH{A#Fp%N+b>30M4j|n5_XKcUyACA?znx#{$p?UXRlYnUAr=#}lb^|eBw<j|43eA= zY@7~5L`D97K3rZq5}wjTL_$GnJ5vHodeE7_Ahm1llySou%F5U*B%Y#72}a>*32~Ib zZ(PlVI6V-eh3bMU)Z43F;X+fv!%UWHt}$Y0rd*2lHZugdg1wc&ZHG{AmWI%^9Wx;r z76H5*N(J_~=ou~mk_g|s4oOo#FUmLH229veadSg~;Q8RFgwf_e60wIDT^4KgXht;0 zG-Kp8q*aI4*=IfjMNKZZW-;4N&jYb<H6>t;cEnxu26)3J9A^v@%oQ;=nneUNNp1q8 zt&($&NluUGk6ZgDgyUqa?{Re*Ir#8?Nq+p8BX^{cP}u8x9^}~dprXpA?WLo>*fMfr zWgmAk?!1VE>(o6C-N!oFIZ}+!QCM=gi+Nc&-m8{ap!E#)Cbr^&-|>b}<IvELiz5lh zD>p+uK3Sa_@2I5y$+FmG+gkN{$syJMtj#g%4yyvCke73+=!7_uYIxA_C*6zXlaS4A z8p{;Lw*5;!x&YloNhWN=TqK+P)@qo6r{+dI2DOJ8cpM#QA0naVQ4;(-_ko7<TC91M zfZEm}(p>$!N0tQ=Pn%!yR6~@G#tk2N=DbL3y_P7MWj^qB%hQ+Avu6}o8=BhMpJw-c zDDC@5R7V&DDn|7x>o%yhxeniEhjl)(J!biV>f;kRnT1-dIGuUdsrGL0S)&P?Mo<$~ z=>1RPp6)ytK=Kk}c|Lr}S;yqY=B?t>-1kVecDNK))#HR|PZvHN0`e9OC4~q1e#IBm zsxo~LKIcd4?QB#J_922b`<4we5xRy$ikzsGAAf;dglI9nkBe9#R)B>&pwsU~mv>BW z9uC~JE}Kdrf37R;We`-+1RL_xg7XL}ZU+{!tU8^9LJwFKcN63yv}&_$T1@7(^Kxgi zw@5hySbFc&qc!c(siM|}rQ79&*?$(CQ+CLNH{)APXu1*0`Z>hkkP;}sA&uKE@JvNH zW^$amF4YxtBYqc&4jMEiO3+%;(Fb<ia{yylQPB&^`ILfFC-*%iTtUgh`+Q5RSgJ!8 zztq2z*bMs%ato~8ZcZQ@SEq@s6Hsd#jSM%tOW>qE^#3~C{=Wq8WA$+DPUgGNWQJ6- z@DM2EE%=fQBJi5+fd=-2aJYV2F9RR1Kf2%sy!bb|hqcEr?n?Qkm|}N6k+b0Zmcqq9 z=6#UfaH~P5P#N>$adI3kcuD2j$kP#T72Dc3(p6Q*KYm2(TsRoJ40Eb<j)O&1*kzZV z&c!0uSDT5_tif`hti=($v1w9S3t@SD{XXG^Sc*%23_A8x9biKtNm^>t<yfb;oGs$N z>rmGkSWt$!Cz=^E^-bw+1IGELGn?keCzr+|zjt4jx>bB||HI+2`Gi!nouDzhOiypT zzx6~f=vKCNipsHvccBt;<CoMdMX|VGNTTYybC}~JpHf|@gxUqXnPZIJwCX!aqqXg5 zl1rY&Tf6gy9z$CA^)`xqZ_Rq0v>C#}hN!JxlY_Euj&aWo#)jhG55K>mX?4GAStUFR z*LQ2odM@DO(V};s5B^^Uom=Mc_<tm7tij62emb~$ql+G=YEU{mwaHy9=a|x_^ln6a zKzNIDeW1|mo=}aO^IkPRj|rD~{<U&DG2#q+s@Tcs<k1QE8C*v;sI?ydi(#VhB?T9O zq`#9N>i5GMEG0ho$F=FxzFNG1J8+`d{=j#6JK|j9r>E^;Z<rI?(3@cH(LDAA^&Fe$ z<k2td;2dT%^h!Jo*Tpst`eZo!SPhVzPh3U1mbHD#Iq8!#YO<!RJdeGL#))@&ftW1V z=(8CVZMhvPNAYz`q?HcCyyME*hsu_6qdFk+%TnJm#@<8f`{RuDt>2sLxIORIXU~s? zvv2r@Dv8v0?C+!$oi-VTl|CygEt`h1Od|`ClI%E9TBk{*dt=lIFd~c0aa$djoUqwd zT$1>yy87QWYatV|8y($6(<)~W6c=G8NkS7M&Ca)qkLxt`PH!j*;mJ$e`&TT%vi7#) zf@Wm?t>pZk8ULDy0iXXE2T%2y=$8{yfp6$2mg;lpZ4=>YW;;&0TCXKkZ~~zk_bGKl zmL%?l?h%hM@(p&#jo-v=2Zia8SCD6!PS#gPplqWJJ2^?(0AY$_BU~bZuS?x9YtiQS zM~LvH`$yC#k!}&?B!rhOdJ7HI0D&?glyM*60oqbyjlk2k@Z`k9TqL`-<WKOR>88}; z(jzu8*)rKj{j+@;`d&RH_mV55agjiI?MvdHMeyzetr?;}1zwlay)PUR=d)$3O$Px} z2Aikqs9__VG<u_*L$@gfxz7AoW|lf8kIeikiS;JGk)0JNe{CLfk^Iy%<n$B!$~iOY zF5B>m(N~MZRV!IT=@n7;y^1n!>o2~jI({biNriLP`Kn_E0nFk0b%zbskgZL~fC%bj zX=%oRwHuEN4|sWbS!!%3%m4kPee0fL8@(aBBa+FQX`Ud4yU<2slXtm_zT&UP6fnz{ z72IH%rsD;YOJW6%$@@yRUbeEEs2O__jd0^a;xL{ue&M%_gMfrC_6<CE5ED?>*yQy# zryJH?T!ua~?E!K(xyWWaa!HDK2YCjn+;d=zts=0*jzjAk$r=dFALvTnbN(|<bh}}U z4xZeQ12T-G+&G5uvxB8mp*v|l!L$cTnGN-Wlkrc#EnJ&(7IzTp@jpy@r50D_47G}w zzfVP39Tyur1{%)U<swGBH5b2?G#9T2Bn#&=#XGJE)M3STi~}aThkQhpmPS2Sn+l*j zyf%(QlYfMoR70z0I4=FzwnDSLo4*j(m9KVx?zH^TwxS=#&}MT5C%wD@ec~Q}DpYU_ zeZ-M$;!Xh(*unQ0IBB74Gr3Vce3Y;Q8Uo#fwsyQ6@#1h=F}apI`mov}%&1}koq6St z?=@_M1uV-@os)CE)g@I!bhw26Coep()RdPK+ymMw<71l3+KTh{wb=xaIh9`dN@yfX zf%^Ad4<<C=*+}!cLNu!Di<fyZWMzqCaZxFWTgNNsYF6Fte&U5jT^wew2u)-<zZj5a z4fihI4>!Lrsx}g(it;FdAONd2;XMhc5$j0pF}uWa^mxJ!%rRevqm0Lg!;>K<!KnG` ztb&8wZGY7q8pXK3UUoN4FR!Hk-4<C`!P<zb%QRYb{06!<IcsX$E`A&XsJR=)I#Eip zs$x1+vBgRDA^1Lkz}@9sEfNEX<JsUl98Z_d5WbFeOmE{FouH%7F~$<Yr#y|fH$e2t z@u-HTmO$Q=)u_N80iR?Ef0%sI#&>;m3oW+dKK4m7AXaB&y2m`ru~@6E+Isv<P-VqP zWw3*t>-vE9=rJVSeHfH6-|<)eXEF1i{PPDr1-e7LAG9xlv!qy@a?7D&v;YuMDbgX+ zySY8Fm<zXWhZFXoqr4h2`merevh+t^@6uW<kDGnln0sbbe=Zqa{1hiCR$GQDfYQDI zY`Z}csl0fkO>?9L#R=<frt)2Qa-tfH^||YiL>R`Y-~6G#01*0n4|f$|hg7XnP`sF3 zZqK^{xe&WS@j&n#I(N)@s7#z(`N7;&xRY-$uoG%?%R#4}ZoXsC{1E0i|6%C??at|a zqe}ki;pu3Sk;u+|SZ_Xg`2trlz5*E-Qx+0=%8RAyFD4*yzG0{+l=O$G2R);`%ddy? zhX$RZZO;=3fdjw1&GjO;rJJXeUh^dSX|}*|4$Gv;{;XG|I3xI8jI$`tER+F+_V0M_ zxRG_U{g2y@0{y6=;Fx&ANBeR%z2}DL6PUj35?n=`eqtR6R$MfzuX>TWR;{Kq;GNj} zXSzxiZK8{S8YmT1Q!aP-9rX+9ZrBlKx!T;t%UB$n+;F#Ec-x|R>p-v*NzLZgZ<{E` z*TTj~5T@bo^y&XJXZc@WzXMflQ(pqbrg$Ib65qWNQJ}>c&E#sgs~_X2S2ATKSyzLm z4F~G`m4k8sK&%1{_h_nVve$;ke-;PUHy^h}E?jzXwRWc%u|}C0G~{dbO?@L@#r!F1 z2p@j>G<&cYBoKM(GyE^P1$`UF1*Rm8zZl6+0b>cHgxe#uN|CkG^|z@?$_h-b!VyoZ zB{pXz{7d(ZH?QqvVXj#Xkt|G?Km*%H#?_+Q*=jS7;rp>CFi{*r>?c#)1EAnYrzJcF zKN7JjoM*(i{A|KZFYwjcb==oMXIj-FJY`bJji_jp<ll8BCutM3E~!uShEJw=w!Pq@ z@C>L;u;cXUV9E{y%vP}7V7~|}ou2Ywsq63!gPG%)BS7zaACxv`QK}A^+$}xL&?LIg zQ;m2#%1)vFSa=XUfTY{?8wt;c)&;JCJNGX*$9dx2w6PsHTm9^Jx5MY2@K5&7^dvSA z4_~tAg+cbA@yxwdvEPDy^ChO_Gp;>$J?hIOI+SF=-eFm(rM)EU-j8G78y{uO9UH5j zV!7Z?M%qo^O-0=#M)ZB6v}8grgvUTGb9hfn`8X;d?-wqbs^|L^FXn3D{g7qBs^(A2 zye$38mhSuH`R}zTUmE4zv5?+#F-gJcVomq!WQl>IWY&aVUQ7p=Ez9=H(!SFKJXbC@ z4b;^}d^`X1IVt9wgBv*@U|4lvRD2Fj+r>LoYm$Mn9x!t%TXGxBM|4kb8CpY`A!nv8 zdfaTPC3MiT<1?(w`0+y7MeQrY>uiMmP(%LV-(xO=5ePMVN4w#w!1Vdvt3xFAY;$c} zT~ku>byzP_&)JG)+k6>RP406&-IWKnxbRK52z##Q12p_9-+#6jvRAF#gB|}cih*>a zdzkD!|4HWwF3={d<wen^yH{%PFQehYz27%F|JAiwi+9YuLf2?H%EPf#N^RV9n}J`D z!DnJ9=g9lJberf~qfcVee^no#p|@T>_-tBn^3^_}i9DlXu=5LPYq8!X|4%vYYSGg~ z8^fo(7L&vZ7g<;;-6+Vo7n|%pG+;#gY0d)Rka^kS|D?I|e|4>A=!v-9LMbllJ=iJ* zaxL1Y)%XuN8$GJ}YYAq)GbH6jo+;R+w%8Q1PV7U4w+31d7iR2QKD+08b%ONk-uP=r zxYpg%Tlo}Fw~pp|Hk_lYW53`yj?d>H<88+8#FJoq#@-}QTteRp#>s10HjlKKVn6Y0 z6}<SB%2)$V#B*1!=`u~RwI|7lFuhZ^QYx+UPlV3bd*9nCS3X2$uqcYn!7tyo9FRt8 z+qRs|vE~&Cy4G>A<7N0hx(q*)+IHfL5lWG7?K6OJVW8xevhR;2wAe8EoB<stVhZB) z(4b!AZar!=K1Uv2A7_`OUk;47PIi4_dg|D=#>!9T2IBVxx_p183~p8%Fz`J&WU8zJ z0``oaWPV8eQG6=r>Kl;qd78CR_aUlkS=F*J$v-Sia2n&)mg`xX**UFJ(*#^MAAA<N zeB{U0H0^DN`aoP5hztWeHzI<eoDvS%i;)(UnyK9+1LJa@xqjJ;VvPNl70g&8*vVCP zu*+G>rl>Aly!>Q0v!|;-xpgF5x8P95DotddhZO%Kq5t{L_@k?on$Cz=pyrBO>RW1N z5i2<3@lHwa$-2=Y2eh*3OxSnvv5mHbA>&`aW+5OZxxIuNN#@W2B+$K!KH<)^j=x=P z)UOl-ttG^*qtwvW@0-6FJTZvn%Fg1FG4@61UF|S_bMRiI!CL36N28a`G}8DI8g+Oq zfD070dCVxh7!qWz&$WI8ogq-9TI@)|6Z~gP$Yw;8k0Lja{S?^A$$_@u)26~@(fya- zdRtr&<(ZwrkbfH{u7x+<LSfvQYIY)}K~cE9p_6IO68=FOgs;Yc$F`j3Y%t1Wg(fqQ z&3wBn4FfGF7_!@gXt)I7*}s<DLaW?HRo$MJwks&PeNcU%xaa1#ml-yzE|s2?n<J(+ z6Zw@*QQy2YX$~8WPa8`3O~(ev4GyLDzYP7=C;KA>U9c(t=q}7DWwsP}wCx)fzDv^u z>8&#dRQnpQ_acFBa*h+3LhA8r&lu6eY~eU|xt-*vZjPMDcM9hnk*Pe`Zft-<5@>3S zw%foz&ew*sl^M9*dI8b+{n&mVEq<`TogZyW#-H@ItK_@C|2>~K8$0JEv0}8gjpg<C z%?F=8p89>k66$v?q{amq<x2ZSlf#99=*Rj3evsxjmiG4+nh;XRq->@H(I4XK#~y2N z>F*trefIE+8I|@Ikr4MLPIp-*OI>x0_`n3K#!X0Ab<?+QFF5*G5m+u8i21@z3Hi5P zsPJBcF@2gSwU|puf$&nmU*AazU^Z_CnMSm>;kk~sjZNT&(`8O)miFpTkvj;qsQ15b zJ#7bofVjyFyPcEvDqPtFjV9Hc_y%3w=OvGp2Z3diK(m#l`7zSOluAAffE|_04**Ix zZio4K;gObM7k3JPK&G^t$W5U0s7E}~r;h@P=5n4_zM=b<1`j`v2&7bJ?(L!!mAB{{ zcPfpIZEJ0d?Yl_v!6*bPcUd{c*ugE*rCeDSVME@!GDJ4=qVXDrmNc@PjjBmMWfYve zp8p?bR@fWDd!tC)P3VN(dqTtQrnz9a1TL7cjSuBr1K?&S&B5^?x{SFt+OJc55XeCM zDJkYB?i~!O5*~_)p2a7sOAY@#TS`v<DCiRJHF1tfK6{faIyUk%Mw|NEt+aZXR+e8B zlxv(eT3)$q+RARsD-DJrQ)R=VL6CObw+flG)YAhWH0o@Z_Xn$;dvnzua|p*ICDA!f zC~0&a0}xp`8`RvKmATngT~|&*Cw$`E?AW8vWPv%WO@4%tX~&hsuWWFNVW&6X4e3Pv zm*g+)6=WWY`urRGe2XQies55enM%B_Y+Ny0*UhzDxo<b|L%7paSzam;q4m=%|Eshy z#vT=Dm07t%*RjYc#_PUTeo4#VzzX((6I>z~gg&tI|NAsyEkU?rW|wRdf#XE^y3j!b zLoxI{O1+sHggwGB?Quvy=eUAsg*r8VS%=V~7@b#NpMN++x8O#m-KLB`3C3zRJ{tII zfKx717~?5$4a&9S8}PJq?+M#mq7V{!&U`<Cl}!cMIw*u!$X%a;(J^NHALRh*|6>MS zZj<U(+fBUv?egCy&)es}>fJib@N{{#@i^Dn5omPw{&4?weplBV`jf`DQk&tj#+NRk zh|d#HZ9dWQ-78Xt*);(c+c7OQxN1yL#*X7^`xDle5&I&P2+n)~r&cQW^Zc;SxQ+iU zjI;l+I&-*~*0b40^UG7o=F6Y|`qiBy@Dh58%J9M?1b`G6pXh<AlnuLH0Ii)im(nq9 z%-<q_ppOOkZ9(LyALo8f@M9XLLjvv{%nByRu6z3$w4bmi2J66B`pdH%bkNz+&Wp#w zo9|+ux}(0O9{n>E{o<$-^`rgII-C0+#trl2d0ehxV%uThEvCQ0wWqnwL4!yWcMte6 zZf6>#nCVssHTbeon?1w>kow40U0_%7<wsMYyGcJU8}%ma&?;oo<<K~*H(IOfX3!R_ z3<sZph43vp+O}}wb5MnRyLNN6F944gqx40wVbJUxqt#%{YQX57|BcIJ@{k^}=ntE7 zH&l$dgcWbW?qEpm<n(zyVV0f<BZC9@nh=SnQfC9%QcR@04QHymvW)LT3u-w5LT(AM zM*gQM(&jrC>!w-2BBmhkJ^3N5aw-HiT2aX&pPVMN=>C`-JCW^|#7j0%2(I?9TtiO8 z7)<lVJrM{>%SX;GLu&Nt;n%~MAp~N=|66m^$WpYVg8PqIrsD)zypzlYqc{~oIQ9df z#<~OlRmzJWE!YDYpI`1Yq12bBLW6)R>>{_1)B}$<oSpDCq^g!vW0buI-CgXalshAf zqL<I@ijF8~pSs=N6eN60Hq9*G_tEEL>T=(!kG2ZI!^O5?IXZW1D}1t?lkeUR))-%Y zed?LC)gez8YP7NL2Wwcv5!>ABeYzh@W)Hw@H;@`MaqKHnU;oz$@lGc|<bEQG)5C$$ zYwFb?TrDBK#pD&{0*BBeW5~qe&CJaVIe9&}otUUfp$WGw2il2v=<iz{1m|Qft{P1+ z4!rZ}c{O3p3tF9=z@tUExZCir|48uu9>82<%8ny|`P-2@WY5v#T$qLO-3#=Fe)d^( z^hHUJDV<!V8QnHCdS~~XSlw6k!>hzZ>th5t!|Ku1bcbEEx1;5-eATNYoh;VHQp3vO z<b1u06%NkjdgncCPnCMw3|8zocF{rq)UGAvGaoba!6@<Nbxrn?O$Twer-rEEY*%-< z_{;~RfuG)rzxrnS`MMOLygXAm9kYqk-%cT0uS`_)|B+y(^%3z8gqldM?xh46cU$?Z z5EE0YoMOx1dqx30*IR~(K58JY<M8Uw^4OIppA3+xZFb-wea!C@Ag~_^wdAff+7E!V z@8Uo^eAGc)i~O?Kgp2FwPl42}C1Q_-K=&Hk9kBgyt9ZaHx=4uP(D=K@Z&ly4QSw%U zoiwdV9*{1a`*b3!wngSvzlw@WFK2*c+K}ayb`*#qOM`6;@B8#wT3szweWtLW_#uF{ zNc~cvtGHiLtKj7A+_cl|W42XU!NhR<`H9rBS*d-!)6F*GrU4%{qA-F3>7b9VLTI~Y z@u^0Tgy}6{RZk^u>4EQQ39K?|^x4mudM3^(n`Egzn=Se{nj7Lx-;_z|b(2ir6!MGA z+a{_BCry}+@e;UWvDkCK5~U81r|{W`wK(kVWq92sv3YCDzrS*T0pHDPzMaISIB|<p z5QG%uJsd9VPV;`J;Lxu=9*^m|mq+MsSrvZVU*Wmg+WL=QyE#QUr6Q=ZYCnAjOG;N> zX1Jv3j7G&erl+CZ+W)NKHWxm(JyF_MocT<}{r;_AhZNZgUZD}kreN;yzp~t^K!poh z%u>KfSsNc?&J#Flv*qk0fjQ<Fzx`F(QPKl56bK~ERc|e;ZmzK@GD&2-YmwtS8O*#T zXMbhgh3zQJ*G{cBsE&!|C5tM7%at?^dGAJ00JQvcs0>z$FXc3X3<pCIpbIv0DMR0< zdW{((7g`$z>J^*w`o<dEYwF*<RpN#AA8)M7o%t*N*nxYZlJk#4uP?KP^2%O&N1Hg= zShhPd!kn{iFbOBKGD6q~_Gd;Es9b2d_~_Nr_>b4O4MNHdyZ)YenMLgV_>$xwwy2{^ zg$9yn2oO75g@ssfy?DX=4<LLSM{SDGZB-Pyb7nhfsHE`T^C9iN85@^IWFGfv;fa<q zeN`DS?-CPT#mSS;AK&fnTaBn+`kd9Ib73gghGtV9km=CxWi;eHWFPd2amE#Ie72&S z?QJx|oP;&Aa)YbWjDB9Q#2Sf*2=#$N#ad`GU0t4ME6RiV13Z5}w9e|GFOh|l!`;Fw zV6N40rPvl`*u5>>sN^c_W*A1>1u>&9e?_U>E+(*(aw|RhyGA;7CG|UX!*Rt};JgHF z99u~coX3u;8b1E%xY+{}#Si?8zm7KXcq>4z#gihoe~Wb;=q}R#Q?)hAw3Nt%O(!1o znsAb#QTKk5yAkV=tXMe|>1tlRpb~tBs!$^<W1{>zEh4V-3<-o&3Ot4u%0E{+7@aV> zQ-^D*u6%1;b(7P-yGG6#W@5>F!@~JDXFo1sI9af0M~fc)+v{)sPx?mz9LV|t8NTN| zP{^=F<!yREhDS<h0S5P(g!X<2zn6#MUjNQgSLcpV4b#Z?HFCA71SO0q|0$z_9nqiA z{0)1O^2KbTRo7`Km1D&lBRh@ToFlPDFu*=6X~GAY0W>x74zAz^$RdxqO{VM-`T^^a zrDKAdUTrfAvS<DL&V|mlEkiF3ZEc=JldT3A2dk2S4YS~wOSr>Ooder*ojy0ORh`?g ziuoll6Do0fR%QVwU-ms}+d*|z0DoQUz)B1mNMF=VyNX6x)eqhNIrYb?Rt)@VTm1ZQ zUSME$bnC_7xu|UGNS^6;+=G9uXqo~Gj6C0-JMlhhSJ=#23ects4{_C~<^AzYM2tz- zZ^dl#Zo+-eff`n18JAeiJmjO6XXE?S`I5hS4ZPar8}AMeN3ES}9Bs`01$pL}R{*Z~ zz|>mn8c__!f!8t=f3@8r0)21$wW{<C!i_;F67qGStC^lc^!r8pDVq$b!FA%}n`IMP zEWaHLNV(R+_ZK)!E^Xw<&S_**WQ~HPa=EgG>yIcVYqyzv@C1?c<a9M=4A2+Bs&H(@ zpS#Hn5hk51Z_UpX_z4~P-q45j$C4L554$kNR(faEmJ_{XP)KH)f$#U})E`VES0q<( z`L0O3A$D2#b$vSiCy5h%HY{d(P3-#Z*H;nPQR@En{(qZ`ld{Y~Ivt@FwL-6?p}m+3 zJ+e|kls(2Km+KuLPeU@D__5q#5MQE_y|3jgCy+50U&&01nL5oroqMG>=lYw;QwzC{ zU3C6YD0@$Z@(NKPxeVXgasqQH66hMh1Gdmd7xdek3MUR8`ZgSnA0KO1DmcVXW1isM zjo0<Q+ta=|SVOLr{nk^%N6*3UVTZSSXZK#Xzj-ij=EOU=QKav8VSvswRrZ*rsJsNn z(Y`6ZB51QY5^s|2ez_<~)kfp#vm@z7&YxFT{<ySrhsH~ZlZaUr=&hq|$IzK>{4jw6 z#*hnP=Wusf4QmZxEJ*xa*J=QQe1t>yAV;LLhL|Kav}(eh!9Fb>ZyAUxLi?L}GM6JC zSE3P8cX&`KT>G&V@!?<h=|TnhS)c|3oDtE*K)WRDF4z)<Q%vzg>cxBJH~@futnrHl zKB>It$fQ*-_eiIY5jlUS={T%e0(z}QmjT_$m#5~M#J0#WNV0sB4)gtQVr@AaaV@&V z!PDxT`PfogbuZGy>dSgc5HkGT_Mt?^qocDII$kOM@R-)l%8cKZqrI^ZAa)utbfXuU zr|h7H+B>Sb69iA1;b()yHij<W7v4iS_C!BsTTQ9(BACS{p*N`Xdh_x!xaYFDlIXda zTq$cBB1*%mhqyUgg!;x*p$)y8GMz*vLO3#1X?;X^_vQdIOLH`3reD_k(6fIeJpO#7 z7UbL57_S>0jV^vPJ-vPK=%4W#a$V9VtTuP;EqeX>)uxLNBsP~8wZR(he@$v`V3{^( zjykLjeTcNP{PH`*HTdeIa;x0(?JF5%;kni6DqTF6K<$^nSPjjBy#ZRSU3_k^F!OW5 z6)qt@9Ig<|wQJM<;a5AlQl6$k->QA*YS_+~b5`ee>K?J^#2f)8{XVh8$e}SaDFGBC z$8l&t#)MW(u*7nFml>!G>q&@vRN&R-Vc%gR<TjNXZP}vK?^mid?j=2aPU-WRvQ&`x z*G28l&6_2S?*gAD8(149jexZyd7Uzz!+R?N{RDU^RjkBE2;@zueI_&b^p<loj@vO# z+-joW6uMx63RCXNcg9KjEPn~!fh&wj9|g?8(T5QMZQ8B5o+znjdvO7-ZZ4&&Pd**d z8m@V}-)i?sd!MYzv5srknqXNvclAysmpU8ljZmO-hW*FmyX|BgN{-mOM(Fs4M30dL zr?s-RpMfs9yzbk4D&uHanB|lP4iJ~>m4%1l%9NQTzy-i3L2K4J1Yi<DuPa~a1By#P zR%VxS5-h<G&XFeX!9Vf}bE=IypT?!a`GuU=t0N91Zgnpes%|XL)Q{|U$@LOB)6mA1 zwQ|?-ZIl_pZ*qG-EsXb%1h6jnU`)W>lNJPTB&4^T5U3&Gl5GdUVmwonEVSo1&U_Oq zj0oYyz(dqHZO-pOFb`N;jG7-^${Ec`D_e|Gwy&C4x*s>8+1grlcUAo2b=urxcUa8o z%ZJEf5DDL?uMn~|VD;=?n@fJ1^1@SlhJTuk%o~R=eF^Sqih6E=dQ^G)iw@@f^3VN` zf2`~_sH~K{=o<We=z0xNHA35=OzXAloMU_(AkO7DT?<%Z9EziUyaqW}k<bwk_UV=B z%jB1(iPv;YU{O5dkZwgre2J(;2bXW%0jJ6G0{Iod-7+gY0^VR4=6Dmn3(9e#CqoXD z5N1&Acv;M8E>5V|i%&WU;Oh&?;MPcmsW%oV*@ZFspp5uuRn#cva`gBuz1abhkqyah z^&_iWc5zm`TSwG@3&201aO_buLmrGj#Km9cP}p(;3+$)qL-l-{8LvL&t|&4cNLpnq ztcM~@h8h3``^VC#l)y$qicOBBVD_AT;>hX|%8&7mazK>m5YLVe7ijXeyTS0d({Q&? zzlk3kRtAD+B2@zjOc`sj5~LHah_M;I@vd?3>ILaLEOVWL=@a|Z3Tu-}KINswRKmDO zRTVKL@#O1>;X7x=+G?gh{p>rOi<8=2-gln~xtr<X*<lpym|H%oP^@TmbuR9h%UW?- zc$mlhiK5*1%+QE8A0}Qj|4OY%zsAz+b52Rd*HTG=Kkq-&9d^lD9(5MBie6A=UP0eL zyWn3dJ1O-Pu=Nc>$pAX0$;uc5x*HufTaKUMkUO`plQtn#p^q_V@4fa2fdv#Ywv!sH z;wP>RP0>0ASI?6V#-F)`wFbUjWpHSe!{4}fLMEccJFPi&@8R{3_z04kc#J?UYvLO6 zkeohzoG=<;(z&*I2~tO(mJACmN_rz0`-tn&9;F=X6cg);Uca-ZW^p%=Toq`g&M~*p z&QNu2^Xj~K?HzFG#rE$FdOo^4QafFFM0;cQn==tsgxXdehn7I+`qvjrAv$TYzL_)M zo5NZ-*9zSfHL7!q<2!wWtXzle8yw}GMlxQtmJIJdVx{Ah!~Mm{;iZcbDSrL3xOQLY zO&tGNPn@v`0lJ9;f<W=0`kB0$6uxqo_%Odt5QdTDk2c4UR(_UX`^mo*^Q)%MXV~xb zWr&n?Wda6KG9is9jT}|Ail!76oPQKNp`;k3aU06a)B9W1<v@_U1t;VOwcz-ds(&Pq zP+Hhb9Eq(B{v<oeU&PXA7_|ewb=@1)rG98kA4EInblrFP@K=cWp;6=f+f%cgxW%{P zaa}#?Z&*-KRJwcm>5Kzn*VN>G<>&9<y?0OM<;14myc3jtwA?B=Xf{`?dvVV4soK`# zlj+;f{JDMhRiCA@smHB0d^79M;JJFW<ou)8il5EA{`1Y`mr`eW-bY1Eul$klXYOb8 zOgqN9e3ySZK4qR5@$K7*roA9WpN+Q7S`HYQP3(N-N>qKyGvo4Z=asXyEi&Y*makHo z$8aYpoIl|E7pF}`;7dAFr&^uDjrQPA`7$9i2oJ^mOyHjCgzQANZ3D@CRhVm>opJ!a zamZZpD<5u29$269`EjKw+S`_Xg`Co<G<s(_WJ4E2J_!IA!%ki!6+5U~Z1bW1j$0nL z`|jO{@G#TduajT6{h}EwnV@l_Sjxa-@o7&S)lTMTk4|-c&GVK3eR3ptem8$@>iC{j zs<~q{>)^M^mkRgto&nuME{d=7KhBS>WBhg#;I&r*_wFU3qu{dmgU?aE3xnte-o$)) zFq4vklU+BY<y_>h#p;h6(_<UqyKbY5ptK`)DKMpG<FU9FyRwk7^KL3^HOmM@JhTGB zu|c}eu988$P7|d8Cu8(`9!^Gl`UA(JgM)t8e%f4m$b7%`U*lg>QXo<jlfv@@^~>iR zH?y44hbO3+-b+e6H>cn+_ftS{^*m*z2Ag2!^!L>ds+avjHT(xWdJalnO7y^_Z#n+= zf^<yq&-C42udACr&Z#Foca$>C^!axE&7H2HYl_hOBQ<he*2{Zcu4WcGX5^na>QX+C zj2}JWp~}rFwA&=@dT84)!9d3YmJyp@EO>2v{CoIe=EA<w+<e>2g^GV2z(@bLdb225 z;{Wy6#8|8@|6C_HXCX;YIkQmZ_@6jn(r!1#o<PrbK@^SC9NV#wk%%c5-gbWZtlbBp z4Oa^#hqn8p;qBWTo;6oH`L;0y{Ev{O(Qc^3^8MBF-K9v`cmuYf5#?=*F%u`lAEN#? zNx>Xo9+2%SP~@uTBdXBRozto#RmODL<z~xuyYuN4#RyaH&PFgnH`<L?;9TZjPFLCY zNCx}VR3mZV@7Ig0zZY352lgrY`}@W_@7Suo-T0Rc7>;S++L5}Qstf5jzqMrCJ^;i+ zsiNao1#V;Yr13E)PIe(jtqV4{KF{)Y?4;Mpb`f@>6WW#B4I}g?$tGBpLE(u??+-CX zIjY5fxAzSCV=kB-IK$u`k9cKqxq)jLSvP1CK4RQJtA(F`ar1bW>obhei0aU&4L#ej zEhjW)B{zEf9GO+`{%L_{3{KhC=zDGx7ey@;KSRibgU>O;HcebRRD3k({`3pf7Gy)g zS?mGv)ah&(pkBM2Z4RiZA@B9HLJ|wpgOAuXg7XwlTVQS_`3WxJ+8xA)>=-*j53I#e z{3zqgjJ1`*2qFx*OS7PyE8C6L`FC3g>Dy)*P1ubnP;axmGn(U8SLDz($<H%LMeQ4Y z`R;D%23Z#yn5MqJrwQip@(BF}r5)fIpHjtNAPpWINb))z04=m1<Jmg|{`>{|<hA!u ziI%f5hpic{hkvii7@O;3Vz|a>4V`dFvE<T*yHjs8krk3jPlPyfewGm{&D;X}{(Fw} zVJLF#!P2Z@B3z0xQ`#Is?>0;`*MaVX20>*(zHAmolB+v2mX1Bey04yAjIf^(S#+pt zb8^znc5#>!SH9H89Jw%LU}jI0QL`M-*C!6?ycc(1`=GUkD|3Q_XggO9YK6@qTkxZO zndfbc`&E{MBdXAOJrp@Ji~aVS(RoZ%CuLs<EFlrAiGKbETa~4{TK>wr<kH1Vy$e@k zAn;S^pSjPQzf?G&cI|YZOTD>Po%Uhr)AWR={@xM#BFFQz{4Oh3mgS8=x5>{oUiQv* zUK&5wu<tbaLCsL1BAu~v@&d5_y6`TD3Ny6yl@oD+{j}n4_|BivUtRx9GFK6z`3ASv zdm9fvyD*#bHfs;@>{;JF7Nh~iR++gofPsU1!y8NY*_$ob9AnLBdnka42Vq;RLX-69 zGs-=EQ69}R3%$s_XCIt7kszHSpj%!uKh$@uBtK}x?pGnCPXn}<nJ$NV114PaPyZv4 z?f5;)dIrQlIiVgj@;(@*v_Y^mxXkU3t8a&}pxSFU12JejCp`t%rzgdq`}_);zOR1U zqN1_+R9-0P0j`K*!@pa-<FaJ8|J(MMMQ|1X+b`8SYysC1IHLwRvhOUNyl+8KB<bYs z#7w=*f90H0lnW4Y;~)8mZDQ4*?v-G`FySOzw*<c}5WF|!Bsr%XI}X|U^WBrkR(yh) zwRV#6Q37Ggu`K9xl*u0Zo=Nn`FDcaFMtzWaN~rl<^2Zl2aXi;Gm$=<*$G>zZvH8h` z@(eQ-j?X_5pZk$7`a?wjNEB6Y9Zr?0=VCGa|47`vc>UDl<-}K_v+iPA|NFBbpPDWk z614-#8RFT1Ye*A%-}~eHZOjS5Ch?H^wt=!R{1hpv&5SSGPP>lG-rvg7+RdIY524ra zyTD~jaBFO^`*H2M<|h~LBfHH%RAL9{^{5pR=dQAp80Hg%lf`Oc{ur%a?7nc=Teq*B zdHDS+(Ns~>Lg$MtvGPu^0tLvb@*_q2&*xZ8evE*28Hz102~R=lNJ1rD=Hz#aN{p@O zG42Lk+Gp>Q;X~jiiRKOSE6Xp8{Ausf{S>3m=_}>_iai&sTBQ@&T=l>R`7Riof$p4z z&ckFVkR`E41ZeRH1R;T#mWrL#C#|$|T->fa8ZBp(uay6VwqmTv9_+bCq`#E7^zdPT zh*Z=<`;`rq$03tnW-b!+aGZVPSgDx(@3th6YdbphBM>jZ8SVT|==dDWZ-JeP%)26b zV&qv59na*ULCmPzG|y1i1E5c5rnK$3j!BNsH4*rI=#iNizi#)w!xe}$x8L6*kG?B8 fXJ!7fD4BWg;(yX$@c&=yf7T%Pzxcdl{~7;piMDSk literal 43021 zcmeFZ1yo$iwl2IHcXtmGAXsqs5FkKs2p$3i8iKngXaWQX1W0fR!65|q;1GfYcX#bZ z8))EfviCk`-#f0Hd&V2%o%i3p7}foCt*TX3vu4$rR@_Y9ECK|N9x6WsKp+tC6!8b# zEaN;;mX|lv&{BV>tg3*x0RT8ks$d64Bmw}icW`sjQj}wOVqnOCJ^&yAm;f5U3jpR8 zu1;#&4|M><rK}*&;EE`8i~oL|jsu7%0l)~Ck_H3A-}C<=L}=mU;syX94Mb^vODk6k z1P(#q7oKiTxA_nRCV6Iei;+-nu?wPu2uyK{t$)LuzsmfEWp1&hy@Ms9%x#~YEbT3C zaR&mw_HegCU}RMU4)Xw8c_Q#A0yDpGw+ACI41r1Pt;}5k00rkZ-_6Rx27&J*Fpi6k zmOKJW000J#^`G#wKVdg3FGN29K;F^G+XZZG<Ho>d!Op-dDk{RDWaat7%FT^S!`#Bo z+{KbX-qGI4+`$_FejD>`Er95jwhRa&3-O8y332gpBgp@^>%YADw^IK#xSiYIN*rna znlljDkH6>rz3$)h9J2vH<OD&R*uUpJ`v?Fvp#VTJ{r5cPOaQ<S0f6ekKaGd%mS1e# z+?>RCcsxBlxxrQz+_wb%+x6cT_*=>U9Q>(0?%VeMMmvTFR?p4dU$`;c64k=-g`>L* zgR7Idg%tzWzh~lqy5gU@^{0MtYFRzEa<OtiG^K~&GO&X+V!9nH!ERtj2L`aie>TJa zWVJu_;THbJ*9buT=^7w0;|2%@NdV-%9{>t5Hh^rLj<^H*+q|h_>H)VaPoI9{H@-(; z#Qk5N|K<ZJ2JsWg6>QCLn=P-U!(id=;&F=+YvT3;6~F@U0TO@`paYlyHh>G@1B3x_ zKn73%9s%lr4qyP70v3P`U=O$eo`4_l3J3!tfmk2`NCq;1Pe1`s3Va1>fJUGd=mPqH zAz%!c0TzKZU<ZHzr@$p*BSQz_f`~zsAbJoBhy%n25&=np6hNvVEsz1|Daab+0CETU zfr3Gipg2%6C<{~oDhJhpT0y;_pP(tw5@-u_2)aN*M#4cNL83*vi^PK@f+UNif~13F zf@F>4jO2q9f)tJP0Vx})7^xbm6{!zt3~3Q*2k8VEK*mNUL%xH|g)D+BkF1Vti2NMc z1vvmY0yz;m8@Uv@9=RKN1bGp87x^3o4TTtm0fh@i97PF57sUd_86^NE3MCmOAEg?l z17!qd2?c_3jf#s(gUW#_hN_JE1l1bV6EzGq5%n`_6>2By80s48DH<9Y85%2^Fq#sY z0h%qEFIp5@8d@3JH?*H<D`+R^=;)N_9O#nh>geX^Zs=j?AJL1^ThNEm*U(`Y*ckK} zd>9Wg3^5!qf-n*>3Ne~7hA}oUE-?u(Sun*g)iEtGy)k1jKVjBm4q~ohUSbhqv0_PL z>0sGny~6r{Rf^S#HH&qGjg8HSEsCv)4aR<n{Q>(6b`SOv_Bjp_&OID?9Ag}JoM@ap zoNqW&I7hg+xOZ`7a1C+YaHDYxaNBX`abbAGc-(l(c$Rp9cqw?*c*A)6_?Y-C__Fw> z_&)fF_!anr_`3v{1a}GK3Csxs2vP{@2qp+l2#E=K3DpVh38M%L33~}QiO`Ae5-AW_ z5`_@u5VaGn5F-;a5z7->5C;=~BJLz!BS9l!B~d1^C5a>{AsHZnkP?v!km`|ok*1I~ zkuH)Uliej#CVN2^L-v(yoa};}mRy$Hiu?_EDfuw@83i?k422a%1VtIe7{vu81Em6` z9pyX98p=5;6smhvnpB=t=~SImyVNAq;?x$@5!7F)r)iLA?$Kz`_|oLi{Gd6ZrK44( zb)x-9+fKVnM@A<@XG`~<u9<F&o|s;W9!&q9zJ-39fs{d(;RVA7h7JbE9hy5zciit} z-5I!Z$;is6%NWF1#yH1>#U#S?oasGN8xw??o>`6AkGY6>ngx?ZgvEv>iKUz6>@MqF z{k!3JYwm8cQm`ts`mh$U&a&aMNwYb#WwVX4qp^#y+p(vy58Xq$CwR~1Udp{e4kQjC z4lqX=$4^dFPEk$=&TP&JE*vgdE)T9Et|e|VZdL9e?i%iW9%ddxo_9RmJlFRH@4vYJ z>HZ8a5$_}3Al`c3L%w@_7JMmuqx`u15BUT6Yx$u990FDX=>n62#DZ#q;eu^~S3<%< zE<&Y3Tf)r3PlZ#3Cqzg@G(@6AdPGr0<wOHT8%58>gv8v$zKR`)bBWuD7m9C6uu51- ze3Dp^WRx_Q%#vJ`Vvu?&l_|9-eMj0{I$L^KhDpX!=CjO(EW50&Y_aUV+<iG0xhlCc zc`<o^`EL)9A1FSEcrc(qq@b&itT6wO@uBs@qK6PgK}BE1Z%U|2DoXE^CX{KFEtCtC zA&-O~`9JDV!BNpxNmf}_y{GD;TCawr_DC&WZT2zCV~58z>VUekdc69a2CIg%MuR4r zriNyU=DHTImY-I)HnFyucA@sEj+{=6&WtXbuDfoV9=@KjUV+}3{sa9u{lzCdPyC<s z8BiI34QdV14fPB^8=e>`7$q3384DVR8IPN=ns}M?no^m*Fl{o!H#0Y@c#8T||7qb< zxVgG{w)x33<!7nSAQti#NftYnGM4Wxx2z<s;;c5FOFoZ#zG*FK9dEs5BW;stvkR67 zCxfB3O12rcXLjm#d3HB1^k0<OW7t2nuX7-Buyg2eq;vFg{OQEu6y~((Easfx3~^C$ z`RodEHF2$VBS8pLKit{eL*17=q&!kQV4ixO6<&m14qiXJ?|HxR-tc+olk1D(YvJ4G zcgOFQ-;%$qe^vk};8{RhAX8vS;Mz;Ym-(--UfI3s54s-|7jzt~A6y?o8}c$_B~&rA zC=4&mC2S&GJUrty%4_iJfj9hbKEAn*u#ET~c|S5S5*}p{_5CgH+YfJVqMt_(#0bTt z$D+kL#E!p{ewQCd80QnW60a6t{hr}{#QWm}vxMG6fy9g?tR(lO#Sf|<YCkf6jQt2t zwn-jKkxMB{rAd90dX{FHHk>Y#UYbFh5t(t33C^6%Qp~E(zMGwxgO=lwv;Il%Q+KXN zZoy}&&yk<6@*MIO^R@Cj3WN&^3TX;sija!ji?)i5i-$@cl+>1Tm1dQZl|_C5zPNwc zE`M4+{#E5`TZKqPX(dZ#Y87!+L^Y_|yBbnsQ?poWQ2Vp)QC(ZTczsm^cSBy|oyL?V z(x#YZtmcsBn--szlW$Jnc3Z7mSK7?lX4;>$k9KHw40Nh=_H;e$YVVfqZt0QiY3vp2 zt@|$Wy{1p7ulk4JkE(vb{;C0?f$Blw!P+6wq57W^Kbwc8hg(M;jC74EkM@l{9{V}2 zJ3cvKJh3=wIk`FYVhTF#F?~7nau#(qVvb-gah_&Adx3qSY*A>jY3aezk7e!UnH7tb zomJOW_*%$1?t0<|{YJqi-)7_1!>yrhqwTdFr=82)usy=P)P1)7$^)5$ACM=IHK+^p z<}m7r@+j|E;JEEX<7EES{`B%J0!9hTKNmUgzR<texb%dh!#`ZHU)5i$T+iOv-`w0R z(Yh*lfuAEZ1m$}G20|P_BoN9kL1G690e{D-2+4v9g!(Ig8c~Y+cf5?iY#`)c`H12G zF9-=CLIA)vBDsd614;;KoCr}D9RAxg9T53Zh=c?nW&XVt<lpgaE4LRKfc*P2(Gb7> zXx9+;@9jQA;6JwuLZ}$Gm2DCENWWtc!R@nvTLtRZy+6J+%)t(B+BS|(3>q$u47|eJ z_ixpN|LFQl$3jr$FCFXOvBuwIaNsW;>%Uaa{zSLGbgaL0tiN=uzjUm>bgaL0tiN=u zzjUnsO*)nmLJqpsP5$`4nY@_+<N-7kR5Vl+G&EE+baXTfECMVnOiU~iJbWAiauNy( zauPB!Dq0qLDrzPgGBO5E2By2L?DyCy>ACK6vE653V`sa42nb<K#KOQL#=;_IqaveX z`yXF7-vB}k;0ch447vj#5rU8jK{p)$J)*;?2%|0H`a^~TAt59Bi;jVbg^egsMF1dy zkdcv4kWo=l5LRT6KjJ=sLWoLqhgTMjSkoMx(V2wr)w?VVCb{xfQmvm5X8va`L6}%% z<P?-tEO%Mi*zXAl3JHsdipf7vc&MnP{773zS5F@yT3c9JJ-4<2+q$~Bdw6<z`vixC zhK0X=6A>5xJ|Qvb!^h<8oKLx*^YRM{zgARMRoB$k)wi{Gbar+3^nM>6866v+n4Fqk zT3%UQTi@8++J+t;9iN<@!Okyk#|1(tmVaEojqKlzix4p`Boq{66!hD1fsi~A37HTD z^$sr@k*p@Vxic{%-zy9fxp!IRt(Z*wS`gA_E<drzm<5(tptnQ&HL`zfU_t-X$o@94 zzmIDgz(EEfCJ&hqkOnT|({F$sam_);leA6XBJkVhO3g*{G2py#Wo9eY0>J`A53Vb! zXijecgZY>Ed2qaF@xY=38FbnkVC*z!=LSIV!vEwEIOf$G%r^jeE=8aH0LC?sAt1I< zJ>Pt6&Cx)C4D+`E2FN}5@F5qPBeZ93r<uqU_I16|om&t2W?X}$BgbFj)$P@kHBo-w z?*gvr4FUepxu{^cRKmjz-OB1@uUv3sC&9<+Z@WlolUOo$Qr4vJt%di!JK{D}+YFgD zO6AHOjE>?r-aMMzJ~XlQBt&URXRfZOiC%YVphI2A;1-Z8X$tQ7v^Tl!sijV)oSjo- zK>G-sZ6$5N7Av>tK*Xe0_uVp1{*Jg4iDzUBm|{lJw|TCm48~+P=UAEnQPJyP)^;q* z-QHS$@NPymag@N6QA#(#_}w}Zo|5?EMJG|xp^e`LYNy|4Wq$)KwJg3PzdMH@Z}<)H zlCo{TOZJ-nk(fpEe)<?{*!>yTxWLHaD68rXltA(lfC709eN|G80)35jl@=GE^|=A? z=gxAN;T?V}nVHDUe&75y@^kXkmoMi>fv5QI>0?g>p3=Y1h;<VH+;`y8p7Rw|xTf>N z+5~9J=O{16zkEH9q%PNFLSw>iPmLew!j9ur<7l`eY~^8LI=6jR#FsnkW2;P{F`mzo zER3JFeTWo7`fb`mM06DzAx3CZe5bOZG3-D)NZ|6tyTUB*Csv=>T0DInUieO^-udo2 z;Mb`}w4zyOudX#L=y6Fqukc>GAboO4(+~2A>E$CKB;~cQu`4V`CfKk9&X9wWnMiSr zUUN^H^cOZCWpEmiQXAr&dWtBYyXU7#S4M_Dj@qn0xu$P8c&L2-`n+5A2{oG1`OEV` zO~uC<oo7w?z&Cm}={wst-=x{488&TzUyS^VkAYi8{>8_@Z;Tv+;NzEgw~Va#i;pvL z-Wur~I>dF2D^VTUyc=y9#9?(JvIn9Pje$vYyoVkHE{maeWC|Kn0h8H=YCyx_LtHn2 z6Q@r4^O2ZwTX9u63u*alO&L-ysw3sGknxaNaE%to)%Fl{>LK9`B@;nz=#ItbuJ!NZ zu^J3KNh?bH^kN{{r5?{Wu=qf6<Qm-|8J)Ie>?|kx2H3vT0Hv1hKE+=;$q_M`?2&HY zl0jGXoEw>r`eW$;f2^I~mJaay+Ofsn>GPLr^WEts;!t8j_#>t7F$NjLd_lO~);Y&5 zs8!r=0DAtqM9l<o<;_<K`c9f5C{eLFqN0M59=TQ)ZyOpyNjgY0K<bd@Tr1KUm^0lp zc9D{nk)6!|MwRrv*ajV-xHv_p8Yzp}0qJXTWnBqt!LS8ajR+I0$z~^g#^-W^cj%9m z`TN3r5BlNNr#|6WT*c-m1!;8+%<H%1&TUS<TNQ6{u1;fHj5){3iFF?ff3Ql&;Aak$ z;F52L$Y33xvvGWw$l(TXYi;kcpnHO+5mfAHA<b}(HIXKj7#JvXDYTO}H@a>P8H~yr z=Gt3^qNsRY#pfCHNyPQ?8hr>^_U@AD<znVJLSBCzAvH7;)>&-IqXLGn<caszjZ0Ur z-toK)l6@ia_MvVBQD^m+l^^_q?9ADFWlN*^l_~bdKcRx@MJ{ez0tPF}T->VbqSiJ! zsosy9jS}>}$IWN3Pu~DS{V`O1%NEQ=yKcD}5Dp7X*FzF1JN#qkM7?PeOLS!B?%`DJ z$A>;xX1F<Zk<OpJh7WdM9qeUloW>2HPCZAX&Yv2?GU`%FB=x`iDtuq2u+BpIO;!Kn zQ&Fx)R@%z13QljF-37lzb24W%hw{>dreJ8+f>a<g2|?RU=QhR9uf~e%o{6HQyzIH> z@kKJmY*iKxJ{fljB>^;L2yE%0ORksaTrF-3&2Qu-{OCW*q7aF_l%+!$%Z+62+<$Fx z_=Z#4rj>a$)n_B4<W6c+c(8juT-krQ1YiG4t-WgNy~aoaVM3OsFQ}iDRQ&dE@^!3` zez0iJ-7gv46p~`{zk5jg93EM_mP{?4L@4o=;A52b*TQ?~>=kQ3OoBS<2RwG$l_^QT z35&0NQ@#FqHM2LsBDvuE35Vm9p2e@jrdx|{5!8_%aky9SqP$d$`^XoBK9AKT99!=r zpieff5asD**R%K_#SV=+=%te!gG6)|N#NHeV7O_BW5#Uf*@pO+fE&O<Em2y~knaia zwr)hm4_6t|=2pxBx{a8o>v&da*vW$nwc?lYERe^Graz?Phfu*1;?GvNgo?f#V;7p> zud(RIw~5(ZU8cEaG}IV1QNVLEapvTgnqz$3%{go4nG8x;`MdN@n?&ylt6Jr<czLXO z)Vx^y!Z(}$Ggj99G%Q-+S=?UYi|hm}D*4*VaF;0F*$rpl$Z$9z{j)ohdse&!&P8gH zSghUE{aJ6#j@%?)d1`(%n>385I!_e;y%hfPuZ45kDr1iQm0Gqg3LyTHc7yxX;uQ3p zQN>XvkLPuVh~s4GS=>4+M?^ox>pj;?GRF;FF<Y>`W~zAbio3L=N_1Ss@_|OMTZn+I ztUMFODoP^D*IK0GYoh9mC-$Zzx?}bZ%zEq7i!bc^7*GndD3I^2X(#BJd_y{PRG)8A zYXOyc{vd8eynUvo-P5`XicBIgqagk6Vtw2z$Zekd?DO~^4#IkRyJtZtiBeRFU5>~8 zfd+Q2jUVz-AIpM6QpF2i*wdL}ikN<QQ9d{|H7B(YT=&>h&*B&#w>pCH$w|Khlo%*E z>B;ay(+)E54D8eFVOVDCa>Co9Zs+aKw}BXZlm1Gy)VyFhTr)L2vm4h|j-fnN;T*mS z@Sv=S3QVtsmI@qj#mKEVsL$@Q9S;UJ>>1}Nxi+95oC>%$ARn9w02a74qO7ncwk{(F zCq<~w5NJZO`s6G})MWA-efy3Kno~o0Y+(3s^oN6nyefdA-F`s%n&;5~6gT`d@tv#f z7o%BYHiJx+Fb>LOA$-DTAO>_DQQ&yfN8@E@O@ehzgr$M5rN)xg+Z-p10m>(6+S?c? zZj7HwSEBBPTqs1<sJX6s>+>+XC<``J^03E<qeO*}YsMc|>&Njbp?5oa65}Ry>4K*f z$qCsR`S6(JQk;Wpa9X25=*+8%r#r71^_t5Q9?ej$$4#C?bE2LL0+=)>OoT_#9WQ10 z+{$QSp1NTt#g0GXrXG)17<m(6hbHKL$P6j_(2cPQ<cU*X=215j*GKA)O<!Y{GDXZ! zJgrLd!<&B*&coICy14f6ezed2dyA;lZDo;7FvjI{TG+p#hbt{8N@#X9WBE9T(!Q>h zz7s((;9T|`U;82GT=^VbUIyK?rTxr*vsapzUSyLE=%v3;FS^aPH3DlsLs0YI3M4LK zl}<LoS*evC!e=6KcN8KhQ)AwLeHFvo`}7G-PlrQa_UOr&+B~bE0vMGv>w~5Y#TIf| z<mb9tUJAqSA|=RA>%VwY`8Qf3NkknDw08ID!j%NU8|F)tCHfs?iJpA+Ha|Zdt_CT* zLFS8XC?!K{C&jtrkfoHnf7KKj)0vf3W*5LcJg&@jEL~GSCkIr$F|m48dP=JGT+hHS zT#>VfNiSN?2u$JeDsY9`1grZp@X@71@qyP_P+u=$zMd-<J6NZ5saz-aO3EdV%UEm9 zWp8xnZJo*4)~4fQ$Cj$fR;q<aQCYi}>52)MIxjflJmH<)vvWV}zQ3J0f#82v?-8R? z7OWKxNW>Fb^&eP{7FxX;X!{U6B6)pKMYG>}dG!{c+5dKV)lXNuch<BCtT-sn?z8>I z+ejP@HauKbRZU?Wk>H^&=%ZKfeuiOvksy$5MXNi?nFD7}PybNywIzM!oj1ql*Z#B* z)FhYK$M1NK=9$QBB6|3hmCn)<oSZqvlXpq-=G4eoLBge#QMH1P`IXYshWpW<f-w6Q z_t<1J_IBvLCi@*Kt`B0*8Wo1k_JoqDLZfls9i^ttNolETd5*5zPLmg6iw<ORt&Ug- zdkoj2tqaj<Wuffj!UNr+?(QoMpebp^MKsVnTit!DCN2}ib=OzY_;dcm%d6Z{Ml?}Z zI1+zs`Kxb_y;c*S{g^l|;p<lX8r}WqYgKu5%v}-tXHRle;~taBAT!60G`R?SdbqP# zSsHD4pfBv(oOC-j9VnT1&BoNtYIe<L)J^`g@{rB%s)PpUmoAa>yDA|=!M_<pSLW=T z*ZqR*R5x+Afv?=0tbZlnA|%vfNd~A^Ufh_4@l6=kM<19F<#rbKlAyI+FV7`V9!|vx z%e96$#eMS2eT?m1BAM4<A*JrKerR*ynPA;Kz~0=^nhEYsg?>Sm5BICyB@(ulNcS$m zS@N`CKuGGj@}9FlIbP86E_kZj@`lZ%#=}kDzvNAvNhO4vR=N+`H~DV@1Nt8K(IFQ< zQ@M97>6eZ(@o%R7#ngXLD$))zi19D=AE^3&U;Y1_m2nW0tEeq|hClxDPC@2n_>Pti z{}uE&6SI)=8g`-ME<o>eWrIYOj9(j2#Ud}gK>r9x{;Seog!o4}BEPUw1N4i(KIL0q zu58H=|KhLwZ5yejiy;)jTmG_r6c*Sl^&TYni>%R~gZJr60QtWv{Y8j>lp_Rm3)z5Q z{G~ZM<llahiD7%oU!w^)H-Mi%8<OoUgC)?aq0dSi0%5=Cs;n`1);;|4?q9WTm)rjV zY8aEpG&6&*amJ3DxL1@W1dXngnSH$kimcLlR2|M$kcT#2dZB$)>`F1RNj8!HHZNUX zsajbhOvV0W*>tx<^c$S*@kMAoUB<6<U*P3wG}5S;J-t26HEc3zTnfv5ypn}NCyUMy z+{0LMJ9Q~);=HW>ntx`IJF&lJzcjFe?*7QJGzrXCwjpfmK(IT3E1gA~iei-Ck>wQ) z-F;j&yHfv}4xjXdtE57st56H+7wHRDRb9~32j}NKn_d(GTpnn}j{n_G74Zd5zQV=S znMCzNjJZq<>?v)P=!Blcw1=|Gkosyw!bPNKJ%&N}Uy>EG3xCR>{#p-lzh38n+OPGH z^GgPhA^1hdf_t|zfCkFguT^wU>{bY<1mXTw3$dsC!_?r2gPO%OvMzm$^4WMTu%rCm z6<d``6X7^9uGx~Iwg2_XJHiMMg$pNI5P78o-(DrveALf4jj%W=y#d;juiuqTUFAF& z-L1m#t*Y!}pf{$IrUIaWQOyh^5<JkdrTMbb*{h_-H8o8~&kv_g_hW3%b%?wfoH#y7 z(>K!z;3ZO34di-!JNNLI_HT`|h^fzt#mR8hy=DmP?S$XI0m}U4Z-58MGqC=}0zb7+ z+*z4*70~A&B(B5`nX1n4R3Dt_Z3TBZAIN*;QSX&qzyEG~A~4@~D4G#IktB`XjJ{oe z_J))0=lr_A94-HTC8ItPTi{eLQw`;+B~|UTV`p74xv>=M&!r}t#-+@_=g7bcBfr!E zzhuRT)a-=dvc2KP@(r*HEl9zjz6==6Y%JFm?-;7J)@TZNiix~Ht6;fVPriHNy<~IZ zTYG$|Pw1F9;BUC(p!xDV{`-+9acVSZ(3T|^UgD@?14kGcy>~dQHheD@9C>!iM_?1n zZ{Z8b$2S0_mz{gs?u14sr#P8h$vR1#^hc%DAW)YNRZZz2R9$w6Tf+QqMXs^DJ^gg4 zpAe^v(h|+rk}l7R(1VwJ__Q(nP+kEOo>QkU^^o*%|4&=?`7?8^7CukDyK=j@><7HC zbRCM=kz(}Ifu!d`X0%LL^{Sgjt5k$5sz=Q2Cmw=zG9SM7`z|Z7ikI=|qtN?VPj$z* z;rW*FER@vcTu9YL%gDhwiJwKW5iEGwx>e1zPx^T?xjtd=DB7#X__YHZfwEAC9G=dy z*Hq67e5e(EmXtiNdEFJc8WvTN_6<#Rr&q8|Eqicjfdq~*>?l%S8MZDeBieqB^&|l0 zosksX7k~W~^e!$bq3JUp_hPK6E^HYcnrnfz{W-<U8vue8T_EZabDfaiPp=D0T2aSS zUZF+SA2}|%d&E3)toVi4zmSEQCxu-+Aq9Q^(yeq^ozSaB6Xu*}Yf1xIQYT#S2D5z5 z^h@~48t$oouTvi#aB;!oh=S=uIog}aVJc`D%aRXg)W(l7K2_F@I51;6Sw^LzM5Q)A z6ti?SXd!A;S+*++4Zv%H%2x-ImE=E29!>UXRI|6;?R==rS!Yrwy3-}vsS`J*XjkM1 zhhLm-J!71YHrj!Cd%GR18^Xd1;j%FMoR-&RjlQ4k*m2%{6!1RoDe>!k4!oXvQpY46 z%UaDbWAC7_mrycXpqozeo_(*@n!d!Z0P6b~rk@R$7`6AqcKp2Uv)1#Fc>3|gu*|E` z#;9{gPK)Rq@#d-%@(^YNKdw)`l(R8lZ1xjWve!H0t3#4hM=hJrU`K7ku!hIMYWB}A z)kc1#lo9<1iCdQ-TyxwmRNg?G9NCGyO0Vpbq}GSM`MecdOr}>qk$@u;NBZC?D89GM zs*DjPegiOtciaG%_`=sO?^%3W$bm15+L0jmS(&ckPaCu9zl>ZHQKbNo{`Fo!$eRC0 z-;$zv5!2%a`^-vNZ&VtioeDE8<9va8on8xFGanI)!~}b3FSUe*eA6A*WN0aIK<L|J z>1Md~q)#%~Ojg)lb!U(O7QWm$TJGg?c+ftN_I6rml<}PVgp-DzE#t&_Lt0q-JbwW) z(Y}D~<sx|9|Fc8_+GqSD$l${n%M5FJqq7NDS|yoY?DVu;>`Ysb$jV!fLN*Ejubq_3 z{UH{sz~@z2uC(Hz*SXc9%fQR1moM>O=o6RtaSiQrPiR68QkK#j_*6ZoH4oJ#<r&Tm z)SX{t7#`RZf+c&^N}@TwHM75KG_zPbc;n-cE@5U*oyK86@VRdi8h-d8s{0PTHyZyS zEvgT~RpjT-@k5H)OIN~ZslfhOAd_~Dxoh4Wd%LVtV-b=V4#YpElp`A^zgz=NdRcj; zw<b{{zaDhTY0;O?U)3dU8TD$C(6}O;<50|;+tMC<A@niNWNr?8;hvD&zv5Vzj6o)x zRwwF#%;_zWp^bPKx3Moxo$&059@E}6Q4%_+YD%@|_V;|3F<xW)%L+o~eFJp2;BL>~ zgAj~VQma?Z``6Dq^88S`R`4tDBM_lG8OO)pgxKyC%75B|Zo6>P?QzAbF6bjW!*}F< zX&ab^rw4(pJVR6sEhn@6^~1Z|2lhPNyY|)%m7Mk)iKO5=ocBF=DUG9R3yKArq^2*n zujjkQId)32m)AvfEkCfeg+Jw+q~#y3{7oi9rCz|>OqzZ#_VzR;&v=_)<$Hbd7~K=- z)dUk0M9^BJCTyQIXq#d^NMRb<la&yjg&xHIOAh@x+`9gB3T7cwKC;irSt-dq-?d7b zK2NPQEGoNADbtQ$@?xuc$Y4sr-Wo^4NG@As`3+Jw8wFOT6F?hk@Vg4H=O$+VHnj6c zwUFNU@SJpt6Qm=5L|Sb?<~*iuJ?~#xgjCSYeu?#$G3%eO>gi{|8kRG{CB(hlCO@Y< znZRrVCmVVn#cxh7NuCk-zjciXgo<~n#PF*_^pfFwQP_RMl9(b2J;$rsnvUk8EOlG; z>vg#{WEr;OCmfPe3z>22R#C*8eBM%RwPCQSrH{!<e!_MoUWx7^9Jx=#3o|_J3(=d^ zthO?+;DV@=&Et%9P`vzrll5}mI@cQj<oF)Hts&CuSxqQo%^SuyXrzR4g)vV#bQMnc z+<j?uyWcjm+77<OZ=HVp7W1;gPIw;7H2TRnCQonx&Mh{O1y%3XpTCzryrNebG0^8< zND{@a=R!i~eC93bU1nG&Fl2Pex-t>Lbr&Vj0bE@(RAaFkOh|9)8uO?MmFy$8gL2OB zW_HAA4_kHwz|jF`a$QMd>YA7S0^v+1NxZ+EeQ0E_$xNN@hU3BKwiRtR!_hB=Q*)Yd z@K!?RgV04{IipdH7W4rUhxO1=jH-?ry8++wti#VLFqFUZ!m$rQ=%j(*lCDTs+9AhJ z55nlvz*`6zR)R3lgUdlMaJOiw-N^P!Q%=?~%Qv&}n!>P(?5*UvnyHhwQo|SY`^FEx zf{uHZvwoUT+7@lQe^DdtU>9tB!}(a4BL_{7E;G8y_ST-ej0<UX0~>BEowgKunJ%Vv zVOa7{i=NmR#|BDCvac-TNBc3fUlX_Bbrx;ewcFN{8N>$qC>doGP$18LO%P>L5Eyj& zwIdi$1jDVD4k)Jl^V*Y3nzmmhy)`0IygsDiyJoESx;4a*{l9C@`^&cU57^9x`#GKD z4hI<(No)<~dayEGlvy*&s3xQs?C6ABQj}ok)f^Zz-y5LJ&~=JI_Zsc%@}`S`rz6Tu z1S!?Ud(8}Rg&{aGiL;q9*LsO_XKB*&X8Rwj$8C7Qvx`p7@txkg<5I$~uzAOb1NycL z<f}O*wWO=Ny2r^5^V>~|8n}*L{NJY4Lz9g@piI;^bqgD&t(?~o$4_wj;G1uXos<^V zyk)N8O)%Ph1~Y4$W(_TF?#l#MpK4|_YPuZk{|I<4tz~B$!J*BN9l`#Z<0Y&-bDRAB zRyqr0*3|m78>K3w15auMlOmD6(<eZ_#Pe9ocI9ywYp|bMo>zlR%~<6}VR?r3u7v<{ zdT$yJ^BZ7d)6SuBr^6JbrF`Np6cL9oDbQr<;IO-)WzW+zr#R<$8qNy6ThtAe+fYKH zr-o@84&ODUTA!z{GOVe6CoS=Gc)fb1>QquF;>a9Y(TaaKvoJsVSgf5aL)6I^3Cn_Y zM?+=#fYPgs>>3v`dm&hHwWn{Q&hzZz6BEmuU3I!9n{^Z9grtCbu*5B<$Tm}QDaQyw zH{R|_C>V2T!xhee^Xg#aGrrN1B-Ti~6wS!7y#%4XHNjiu?d~GpGv+w|eoGjBE14;A zgl|}~iK0>Cd6~O-x>Z_}_@0>p-JD%<7I!iAW>5rMA@l79xCGaRbjwUQKFoa)zd5~} z)>{CMSyiPTT?vCf^xqCmdJ8v$EH%c^%}BrY6UoCaom6;}X?EdAuy;KKpG?)y8IoZL z;=|Xl6V=w@=64qI6D`{^Wf{&Mt)#y6y=0f#{ZcI>72=LQN<*lB>q9W5UB(Y52G>U= zBMc-)3oyl#+5Nb=z?FfctB*{Z{M+bD>596Lk>$6|Eq?QKP{FJrKQoN0wU7-==}V`Y zhv<}Du3NGE#)rYt8xNl6mT&%Ya{cCnNMQL__fKf*+GHaAWx@WlJ-=|ra6D~kj%Y`B zymYzZOSQ?N%AG}Dyw&uVc*={K%WnhKAkCqk#de+?_U5)Js~+3_BzU{(fal#tGu7b} z^$4n_vIq%#WFwpVH7TELxJKyPC01Y`qUDZ(3u{4&I&nrT7E%u>i(>fG1a9}&`)+|C zG<?U;tGQjDK|S8DFUybbD4I5hHBog9Ot%eCt57i3RENqbk~|{S@Bzehj`ek(<&^ic zi#m{KDrnwOK!jNScgc9+Eu1_D_Pv#1EX%4k<e<7SWH){8l26#dx%)B4r|furDi0B9 zG}w_Y#G}G-N&Sr{(vx*Z9u+7iwM9K`&W9TSCw-$VP<vM!;mR$g?=YlGTb5*8Q60Cp zsQ;R3{;peH4qrp@nJ8%oOuM5#qiyclV7n=AgYN?7TuXeeoi=^)R7)6k@Yd$Sh`8v6 zdXkQtT%J(WY2lv)=boW|K{xQdF%savGur?E*}bS~wk_oIFu^?^5!*gvV8o;%Fidlg zjq+~UL=f!zNi!#G4le#|-;Xrw2VQ{~4cFE8)Yl|ux)K?IFwD;hbFnF38<@UCA1!aZ z=7Ot~wX(DX7h5jmfL)@fKhqx`-QSn4Z%&`$+SZx;$@tddtuB)csqRAnITI<(#5p3O zypLT}Jd{LJM@drw5rg^<CcS``oJp@>#rokBJGa`eb!u@B7qZaH(PMoH4M{ZkU&36M zjcx#>ozr<_=-a+Q3(=WO+rt~6&uY){&hE}Px<>p*rv?Zog?_r%L_3)jH)L-hzJMK* zVUDElY%$h<isTd#dZRfs8i>|t?wikY5bCGbDq)np^jviLkPgY1HJ^#{n6=tT)K)%W z(@oB==lGYC<M&91AO9LW{=e6S-Up>C0%vXMCx6am3;RAymaH$CAj`V#kM093HVebD zug4c2L&8JX58*1yXuO<7<HM7IC8-xrE%7^|6-*#8;V&dOmM+!km1C@`>Q`6~lomR6 zedD|ZyW#iB8^_Ds>FIZ%6ep4|K-S2vrz&@@OVAsfx+j81laC{|a;Z5|Pgp#kX8$G@ zvH{r<|FcwzzMTHbZ_9dl5c_R9`%r}dwIcl1)xyQ_R4j=9)lMvL>V&_--O}Q&w6uPF zWLR={mlWwp)f`uay*Gh>$ZFRdC4T)ECP)eRmcPTjJOW{QEhHH#fpakZPM44N>JO?~ zuGy|aQY{t$H^F@;gekIAsCajva=-M?%9fJl|3&v5sci)C%V71tCy=PoV#UARE3e}M zy?sXSE_Svc*VBT_>UTBY!>`T&-={tk#U`@~U^~J7ACWW!S#gRE5t)|pFf3?UaJfox zX~jyK0!9;*wAD+uz8y@-P81-|IuWdoam2^Ut>Yuc*cI_QOY<SptC#4xTqODi-w`4O z@Xscf8-NIYU&3f{=86}uuRq&P*)^DE>-&Aly2T7RyG)q$a>WWgXzL<%<$31E>gLx` z6#Go)#)LDh>)kn6<)+?C-BLKPDfc}8Ir>hh;4D+6XF2{n==uWQJ-VnTm@_*4zMgL3 zPtFVD_0|7d;Xb5#2tNERA|!Z4ng|>~{PXizRl_UPPcnFD>mHJXIxbI6Nm9PBxcdC^ zhI?!K$w8wcc(|m<w-0UBJkP%_w9*bNT%4^{?4?d<FSXEZncjg2p1nRc<FnagO31DH z;O)}MY**W{l2&qlGAJPYJ;fiyQJJKj|H~R`toLR`&BzSqnGH*lgG|ggR3U}#O3lAJ z=S{qm@xuyK9<>6bPFUZD*L_a5mkQ~6q%1h0_2;M+h2LHB9PbflUO4^K3t>Tn>Je_& zoee_#5}Cjuep4m#kX!H0mSU`1-Pq`D3}4|Hk+;xcU(~MF<jJ=(dPq!XAPp=}E?9p^ zirl~Aaq5?7N0`#n7-uaVc<IX%^eXCCy9u|3LA(BAo;MTiom2s@Rsu2Sy`CZ7ptGEx z?DF>)KZc7-bbP~&UbuR3A@D1-rbeSh?$7=FkD_ag9fROjHvpzk({h^dF1^)tdm~x8 zby~|{537gJ+a@u~E4%E86ZKtbobBnd_>HTuaia_lkGV^N>e<B=kuE2&(0lKC!y6!f zIW_ltO;fWk`LDU0h}o1T4q({1ko;Wq3C@105IXzp38I~lh&kbcO{X3VxWWsV^|C_e z4G^&mnTV7S_FgLS@KPQBsUl`9?^JXa^eB}`RHWOsNPIG-$oRPaPeDTR|HEL&Cq%He zu7&PIDZ;}%`-j^<4{BbbfT_|NQ&-~@RRaXP(N$7T{?kx5Z?iqrf<gMAY{+%;rMond ziDIsNbYXO$Pxn>q%i!x1H$&YJD$|c(^XDmCCFV+uW(pbreQc??vGX$hG$E5k^F-T) z##3>YvECh62dbz0PX(}RdeL~R0*PLi?0=_g{HOOx-y;0-MBsViAqPQwQ@LuWQ+_ZH zM@Ic?@$wr$LW}atZhq8$tL{T(Rsi1+>KH9002$uW(>HwHl-IY+MRq}bsbrq?gY)~O zO|S7wtoT@eRImX|+XNQ=R;BxBc57N9M6e~iB$rLdLm212LCl#~cMW616?xk<-W#OO zPn)M?ucbHMV_}}LQnJpDHuOHpwfXw_9kgO;KcA7%D|r~*32!FXODIFmMCQuwXJuc5 z?~Fq2*V?dU^fk*j?M40=%e7Z3N+a${>0IjSm5pz&4+pV&@J9)aNzdAgGVqL*i~M+6 z{K?=1hv3rAVn^Pop{rp<>Tq?3s>n~{U4@0d=?A~|uG*NeX0hNdJ$+4e+Y5)LJdtpX z5i5p9p6E;jb?BVtpc?S!(6FM#GPL&m&HI!6{({Fs5jBN-6!`Drt6m)+JgIT)I~bbC z9a#URw0j4_O`*N?xIM@84(pXtw6OWh4#UKXELgmX8b?R6I73|+yF^c}BBe5uq?14X zV@WSN4IjEUf7(X>2+rXMh5cMkSwMwLjS22?bATPW9{3Wi8RUq6(iO#$-Ry3<yIa~# zvbr0+{cB5ejZ<5ueyb>#qE4msbM;Mz7V&v{z@wn_$5N(s1N(Cvxwm0p+Iq9^_y1$* zVXc|T>xUPQTy;7&`KY_fWo7x7uZox(s#hR~Z%^ZFkPCiobf)sfUK7i#LG~kt(1Ev0 zeKEM_`AJ+k{k*jmdLr{}vf)6il;KH+Agj{ao8WFnG(B1jiC{&QSI_@VOmTLJrVOgr zLB|?ZUe`f5mE`1{IYIp>jw)XnC-{)QFg=j!B;sfx(v*ITn+0+ML`v}6@(8xz8CUj! zQ>pW5J^H9J4hV-6_SkxNGmfv-&0-BIhv&+64knN9tqI29fyG!u;nu#nQ@K?GdG-=# z4#nO5@!uU&P!`=Kd5s~kqs7X%=Jo!j3s0D?W%<DIS4DwmyfqGii@J((boWO6xK_#t zy}p<{X)iQ-G#wBlZaMX>{61<pq{2z*bkHZF@67c$5VA1wY^x<eDu3nfm>@3esX2uC zLic3s{hqfywX!|`!%m#C1QsO`$xg(nOj5t6BYv_K5L>V8nYKi1tJ67oE$@{Gebnkt zy!@#g^o{Dq0nLHMH$ZQh7yfn2p7gQZgb9>ip<<|+!w}ZpVM-HY3MrWjpLA3);98ip z)cgKJBgkVV>ebFJ-Uw{&bUpCmQO&aJ&lYBT_&fL<LZ1pPi{O&LU5>$*&xSvc;{NHb z(pg?#*)*EfR99nt8P8PKUrC3-L~{vs8D8jUW{0h;q+#Kc+D@AW+v67XRL2LLU`Cmn zb0tKwIzLBdMIJBNq}pg7B=Aa7u7-Io(y|)-Xx(>b9wr;6Mm6<3@NzL@3Z>}XzjyE| zaCIr*F2X*FjekX|m9QFreYV<C)}B?SEVy<a1$mG>7=F!RM~{C1S17cj(z%cswdXFg z!uNDg*=6eQQeKT!SH;1r8VDGiC%NQ_mT>gEYAWl$`dB<@x7zPrFW!uC#A@t!=sDLd z_>%{orhNlUmP#LI7sBt17yC&=6}wF-$5!KJ492QHI=;xVF(dBKXkMJ08S-SB5m>4B zH#S@1Im9$dC=<K^KDCUV(#l^;a0l+h2Qrz7!BJqz=@c-{&y#(N2b9~qGnM-j^&UwZ zMg*46Mf_JjJNaV>9+zg_0CeQiC%rB2%I;nz1+Koakv=dE+&s$`K5I71SFyQh>FJDo z(gd4XrpGk1N#@#5TT-Lepo*#wJy(bWTU?fRMf7+1q^=xNRzZn7SM*@<&P$XM=I9}A z{$gWyn;XbAXJ%)=!;3Y=A(`wb?*?tNmaQB^%!n7EwE4U+{tm6zf=WqOCMCU>6&7D4 z)#2FF+0gW)PhMurV`mY5MoT?{Ef_l<<(x@n&uW6&Y`#%P>^&oVkEdvSsq;BG#@=zq zp(V80)M6&3DWQo6ab_hDW`2Pl@f4B~BY;b<3f+da$c~$o7;u*@H`G_8P%0h~$3+Dg z6Lz|p3YRz5PB>aZV>&dKr15BrKF@%Cf(!=PkvF`V-Fro^Nqf3?wA-k<U&3XV_xPm4 z(~(Aa_AD&8x@wcPraCS4_jjMa=I>2!r>onEeqicl)V@H{XfE8@FVdesw}3Y6;3#m; zy%aY=&nSoV1;sgC>})y15)mGw4wL4QwfAUjF@w>M2wmgBddfQHJln}kx$2<$VI{{0 zHL|{5^Oi&PUyJU<BYU-5ONv=EDK9qQ6Zq+lSVBWLB2VUs#+R2hbDwOv6l*FFiR$tN z<@GZM#|D<7DF!)aJh{p`SgZTF@8&Ob*<ON{hT9=W9*6o)%08n#j^@<_MG1YFsldoJ z3Cs`1+BLLburO)-sG3O8nzJ{*Yu`jwf#mIi(0%7<NvNSf%KXBmE8O1BeU>h%#y{ZT z`z1V*zM{}p(<^*g!e{qgU%}_XNQv^#WAB2#%%i`6I3Jj1K&h{l%OVi*apZ`7BHOTW z<J4`G`^1b0{-}BM#cFw7hOVL=SO%Y9az?itE@L5evY9~=xS9^jxo*`ryI?Q6S96LG zobKHK!<t@pzPgTxtw38Md?}{0FAx{bIh5H5v3|Q=UH(;*<)ryN)zo`y?yuxPSqo`F z_Pnh$=`{n)sI$6Wav*-mgiJKICLNNQe5g$BfW$7M8C`c$+Oqf*eenhe1p&cNKcYiR z89Qed4?5i|n7bFvNFVvV3>=%B_tc!yZFxsOCs^4MY27r}6j;*u!yL|0I(-pnbWR8h z?sS6HbSIy_&4(RIqi<62mwYj@gOzX}uy&a=cCTrw_q`M?!4GfdF`-|UYzcV=eNV^r z)16vBePC~Yxwfu4kvB0~QN0l>NLEC`SC1|$&&%-}JyD~!E0nlhsAisKJdga;r{lt= zQN<hMYE}KPg8Rv8V5&PE@~8w;8LZ9{BK(SzahT!N2Uw`vS_P<wsV{`0T}W0`l!Eef zz-h|I4zVPp4;4n|F1EG;4;P?pUSfl$u*&-I2yQ>KQX3zo#zb7kL8Xu8nl$z`KT_`b zwJBZZ4V{YM!W0T%>1I@Ll`l}e0<N|3lvA6W>0L&%x!$9$kR|Mw#d0h0I&kFWNcs>V z{bmB_C(lG>)YY%xB(NRx&jkf0GL2jk_+lJ}$etD)b=1L69!k7ivN!ks!mq}~@Y(#5 zM=#Zj*t<)Ud(%p1d8A%~^lNbcg>K&jj=MA0j%^1JF|(<5t>&?7g3XEKlD+<u)ff@) zeJil#i${xAAL5(<(jJ)NGU*Kv)=Ub20L9C3cg~-kv^&!qxAWvzB8jK!N=+vq8$c5o zV3)%2+IbAY53ztKzTOj5YShddlyC+^4OBSh!rpx}<SVK{_ux~aJ&IUwBws2~?50Nv zxU*F*b7yP~ird*XlZb*D+p#Wjyc$%ur7M)n{zx2Q(Q}BN_aGrH&TYB@1|eb(jFjD$ z3j3i;+MQhFAIk+JHS>x;3=CEr-@~UI@V(S-RD){kjW%O?`F7}O2R>Kv_ot06@OPyD zcq)L`$DA*^Q)y8uj0Y=B4vf4eQT(jnSPY~3KGKOzH|3_i^lhXvC#dj&krHbba{%e| z{I<`)D@gaFQMWSCq=BL2H-CMZAVU65+l~WE9gNSaPh)jXL^st~C#ohsUtLPIc=I-= zot(@Z?OMacA2zA!tB%4kp;Q|k)0!z%xus?KC6kA*cPMp!IEd|upx4o!4-$JxUn-ua zl-D3Mk;o2@7Q127vmaH>laOUOiTr_uM5d^O6V~KTD2gc>l)E8E4Px50k58Tpe`rv7 zsk~=i^_qU3G8k2cw+q<F6K&m-V%a_oz=xeLEfB)Ol)2mt1i&vsheB61g+DwVsU+?( zmxE=)r{U6r&>;k6dIPu2)?jp-V!xTe--l@^G92E!b-@q+eu&FVf6VQQs`J3zcgKkO z!P4it1Pd8ooN}bb_%3N`IFnFJ(X^Mx;gRlW0K%nv6)~W%E|4{Iw1-T|(KBcW6hJ)G z_rjsp5i{%8v*UMd=*Op-);iD4i+&b$N>TVJLo&h|$ul<%oBGF-JW;UX{7VO(jQkLN zeqOyo9qHna>x-9E@ML=&?3p`nG;i24mpm`aNZ;XX;#5@}Z7_evrKvz_iAsWy{{ETd zZJGi=4?2ym9_~FzCJ(g|@^BXBU^zfj#@Kx1gr3!UQl^!t5WyNJA0lHAlS#TEUGYQl zn$#h~_ule`M*8x;$F{+=H6`oSm|;?pQf_wL)*lWV^p9|zn)tSh#55?eQq)!1BdbWQ z2r1mT6oTTs(NbBI<DE&}h3RLTYLFV$hH_08dfkdd0&1Ih<^)Pf{%)z0F|V(^f;`we zWLo)_wH3*X`2IZ#-p>oja+e*a%Dx3ooqBqn@y;3)F%t?z6R}!y%JhE25-m0Sd6qrl zex8#}bjIA6u{w~`DNW|*x#Z7PMeDj`G(E+E0`6)cDWZD}@_(0Bl)iyHU)fA!Go6+; z_}VXUIi-lQ?BoMe!kv-Tqo+aB0xYCvus1YuBF8(X@%~`PKr6|#U3ST0Xr)%i%DwG} z7)(PEhmIl1rsa!Qz9umLLIYcJQ}S5JuK4_O?w`z#X+shpzM03rRM+rVuqZ8Xw07LN z7~Y&w<Ec8iZYhmAEWz#!#PKt+xftwx8?e7)Qu}c=y6=kDd}PEK<K64`+#WBLdsi>k z>k*MVUTwJ6-xf$JaiyQsjaT~fs3)3;U93tcKaE4bnjr^8cWHr`@~SypTgA#=Hxq+d z+ma_j+^;+zSADzzD%^32W{uz6dCrLvSxdqYKXGAsU@MjpA(#<%Ez}Hs81I;4*qeul zAhe#ctoq)$+S1QEtM7chiA;N>pyHyEWGB5dagFI#wrAmgp0sfu_)SzgGy_whj0Apv zNRWqY&Q>@%HPPmURm8qhSOys{#T%W_9t!3!qN1g)9n;r2#R2_NJYUB5)6X=As_1tM zxHAqOGu6h}#m_FjO3$MipJ<Drg+)GqS!He=!B~yQN~uP1#dg#c;-+oRCt3Vr*kbBk z@Kw+9ykfo?&L>@pz~Z`$M0pz3iso@2?ay}!y+8k|PED#&$7&nxyy>euHm^5xQh&u0 z*f9_WX}$q+TewPKRO!4oKwKXr@Ve<h`dDtn%@KbJZVYXSdJc6<()YoO^u79M{yF!@ zy9cGgXK~eCDCHupgU4OnkpgcY1x5y9@i2@WFjsyJ+BJC`TdssHmx=!Lew_r)a{dj_ zv9S~w?WgIHPCHh0fM+*>bWJvVhq&L-=^?EtpX{`{c)gP(RJ&^I2534v3QX9$_F%Ek zv$+}%ENNMaE^3`8hth}n@hsC)t#lA#D|q9*4dfT>sCFV<d_-*{j?c8tv?)?XI%+RP zsBXv7a+=<>yg=TGiC^$UaP_FMR$A=o`BUr6UW$FY`am35U#B!a+*2x#Yc4T&8HCWa zukQKA^?|YQ6D5eND7!o@XuVGZ!~L|E$sjtWq>zpF5<AuL>?*R)Mm<^!>aVKz_%L0Q zkPn`+vz`>^dIp@DHb^y_s6;-RE4>nPJKA&%7^LdIZh%C}H;t8A);BOdNe1JYv?+aW zh;e->dSIG%dsOU*XR7S+s6b%IMZm}6CnitK4Aq@!-WfmmS$?&1-#nJ@)BRQ&s-%_f zHxkNAHvo25I_|JL_LUwK{*Eg*$Cz5DyZ`POdV0Hr^_HPt+`7H{mGbAxiX;AJAzLD^ zf*hFr(!zpm@<+wd=cd}~l{l1XgU??8<jwwF{&LiA6;(CWky6IIoObE8$jImVAzs*> z)1JX@H9><*8yyWPJ7GtTj{5>8BPM-wb5R+_c)>@42UH4!Gj21!WsJUoQiAclX+p|$ zGU5il_XjXeJ|!e=TzOpAPf|C)!*y;C$^90%tVJpKC+q01i0*&Q^Ky1?AmqVSxY*0+ z(e68SC8mSMiBo%VcP$t(eg-?D%FsT%ujYgL>o^T~YT^4r(mC?jlRx{YWB<cLO$lRH zX`jX+b8|DTT<s*G%&{DgzFCH%90nu={C3*OC6EyAZoimz7oU5s3fJM<tYT~0O%dOj z=UBGOsD>`P!4~0X3cn^cBi{)ma4hFk<Jg+Mm;OMgJT7-)ZpK;q^`>nOo%J&-Ze15O zj-}&^wL`lWVu!H0qVfgWH_Ve$$Um80LpnTJ%j#AyNUIu!zAy`Y>BeEZU*46a;|~<r zl;)DsMqA%KW2~zQ+eQyV#Po)CQHhf9W#kKPAe3(F;nLOdYieU$VtsVqDI|r2MOw|3 z%}=LHd6^P1s@W;ZZ^UWx+7Bq&muIkFaWy=#6XNrx3d^|DH`61XoJu1c*j8VEI2b_? zgXxu_2;F;4nEfa@;7iUc!OD<(4L<MfYvJg!rg_RzQTFZ#Q=%>riKeUw<)>pHyRofr zg*5k=T{0=y&(YO(HMBPT5z)hMG(?oft4==pW;tFBs^f5R_hOs%<x}ye*H)F9^p%{i z`JNpPU<7=lt0SpPDAkU9{9}_XpOaf^!otyYwI4l=s&`Gb!BRD9&wMgEMCU}!NX`?B z4H2@-A+0{nS%oXF^|kVS<Aq-&;{TAzC|cp5shhnI?k>@WX1uZYW6i2A@U{1Lr$`jh zqZVoZzPn2*SHL9kZayB&QJ(9M`<<iTXnF1bYwx^+nq1$!9}XxgO{7T;0s>04&;y7F zh)5BXUQ`4G1Ox;E1VWKsLJ?5FAiWc*p@d!{MS2mClF)lXjg+v_^X~heGjry*JMZlN zW_Qp2kv}q%d3c_Xx$o=#Uf=KM%DP}R47oDrpfGUOs<w9ULi?oy>nma8IK;iO+JM7e zsycOjd)t8*(JC_3Suf}bIv@AX7S~8HK6<F6!UR(c6?uv`%MF>9`MU76DCs3eEs>t} zae!;;LWrGa)*x%CiZvn02#MSGhZas}AsnozG<n*{Tt2z_-Df>kdEJS(<4K=sBFf#r z|J}*~xIndf-}hZ=oI6{D7d%PDh5$7x#h-=Q7uenkTQ4zq<4DGdQ=*JfqM+X}W-=jt z_5a%IQF{Oo`l@KNe<Tp#_~!iC$&XZzYD*KeE%t(1ea`vPReE^QmRccfbykf{b4+9+ zJYATCs)d*YDB+*RK7v1Y;&#v>?f82en0h1KNh8A!-#$UCTvR4fPJ0-x&UAC&d-e!d zp^BOV$Uq;j?QmU_qJT?=K2Rv1m4belEiNyH)Cnd3NQfv@OD8ax4_ixL9PdP?T6HBn zN^|G-l_<rf+)uhou)_GrV(#`ay;W5~Pg%2v9#heqI~UU9HPDho_RnXXE-WLdoQq9i zdmfh%*IwlkY8D#XsE>Mn_8*dqN_vx>%l;_A7e$>f1hyyJ?9YVkX$tjjD@@snAXAX; zUsny>Z{CVrVrBCBC9m?w>yv0V#`fvGV3t-=uqm9Ntlr-+wBPvo&Sy`eMN%W@{`}mI z#6U9%c<_55{qdyZ`nx#&dw=SZ2LDk5{r5-KRg6hL)jxC*{;C=BmjPw+-aV%%IQ3wq z$yeF!H64>eb@;SSEu@D$lj+sPYF_SbAq5MV!WnA{vsE+2Wp>3=0E2n@@6-lR3X!GJ zzj0|x_Y~0l>J)ISslkFq{8kX-xF_1UOOtjA2tyeD?%U@gITNY2n$mU-@}aY3P$GW@ zY47qM_cw{s!Jh)|be;kT$$P)t@fBqAGq`#P%lMw_?hr|@!`c}($boNPn==OO4=$44 z+mHU9L!f*QZ8^Y&ymTAd9n3ieTnM*__GPsG;Q+Y<J_USw+=%((ZH5HP84^NIZs$W= zvLohT!2zkZaDLJSt^Ra_=+^oQC=)Oz_Nt0l$HtlW4*kc5KQDGKVU_8ZZ=Jle5#zR8 zk5a_W?G6HuNw+5X5~Br@nB!lnA`$fF%u9c%<$j=6B8zp>WJ`mjq9*Gppo8?ZTB%>r zZz0Y@M8B`6fOdntU;E%EI?@vceYKoyZjdE?vXM}bC>FV&t_7l>TU@FNhY||*!hZ)8 zECO*gvm3Kwp=v7`K%yw=f_$%9fpkQkH>CR$AQhee`m_qNCwV9m-|@VDV208u;9Ot$ zZ3~`-nVwN$g`PMo=ODRg?MhE%0PzN4u7%6g%Mp2d-?piu{^Zzjm#pRyafNB05)BQO zvS3)R&|vpbrBR^L2uZXxzs38s<m||0ij(_Ajg-Vbb4pxiv_>5XJ=H{Zb0(%U?PUiG zaLZ9)m+TAJb&i;gZ9KcS;)bD77YNa+wjRDca&xkyg<mes9aM@wct7Tb3m6e5vVe^R zTl^*<H!)sT!!pM%Xne`hlR<72ZfYOq7(1!tY=5<^$c{<pX2L9OVDgTbzsKSZCMU`1 z`g+}v$I2M}fjZCgY6Zdslx-q<@rk?Ks$fPf?dzXQL@)sph)ZkxG|}mnqPMMDBlJbC zuj~Zt!(0X-!&|m^9P%evc^9c!<!2XZ2z#tP?S92dLUwY)-%F(9Cv_w(sP_zc$TllE zlYLM=QwvY4xsE)5Ew%I*GybL4Q8um8-0@ZMiAglz=pISma=SbK!g(;$+<|IvBIUIv z2VVH|{kVH`*O!|D<U@1Ek~MMA`{f@>KvxUw71*i;+=WHox1mq+Cb?m+!krTE8l=Ty zb!h-%VppGKQEH0phfNXRY>qDLo3XA`L>*`kqDB-%j8=Mh@bI8wu;atoZf8d;q`=7s zJ(=Ues{Uh@OmF}>e^`N$h3HL4SVGsy9T{L0|Bxnkx!Nt<Euy-02ywmfP+~1@G|NG; zy^#U<z}=6OJ5proK~OndbGhVDbZ&X_ZtC9b<641M1naWb8(H@xJjdipfBZsAEeG{u zW<I~3wmpG(Gs(1B4SPB&`(#s(v0F3t^j)HR>_zIr(aQ$T<m4CVE{Fq;R~~EGR*n;e z4xzk;)dfjT8bqWFr_!F!^Cir5l2eJDB>k$_!(vZ)RNI3i%OK=$E>cUp&J=!MqcyD| zcSgn5Q7X0=K~WG1kz}|s&@JB)coBZ+$r`FBoVyt*_N-H2u%2zKg-bOTqtS?TB({{r z4cWxQ6uo#hd$*F<Z$*3U<)JKQC!<XC1C9IL>Y^yk!WmvUW&{bSKNpZQm&AN_QOmI_ zD6A_c<(o}>gC9?2#BUG{q|K;0y!PW~omjq`j)OzhvRle*kqX@9+a*9Sul-!eL=)%K zYtNG0Rs{w;b5x;iZ2+NAXXI10-bubwzDCVWgDb@&2J@{@x`$kq`8focUjG8yY5q~C z$`y&*H@<hv(0?+uq*rYe_C>kEHLaHw>9#i992Ax;@S2-Qo9wC$*-?8i_F~mk`p2XX zmw$V$CEFjrc+^Oes?3xs9YCi54c{E8mQw)jNeEUdTJ#i9C6NN-!9U-26h)%>36?&+ z)eA{-kN3@mEas9XpqLAR>s;ECiWTdqG&u$HQ-HkWCR)xlaxPWBDhZ=ee+zA#Q)6FL zJygk!aL%c$jj?)sjzc{)nkDD@K_rT_zMtn$a<bV^EB7==Jn%lQ|FNBAFc}$L=M10T z12xP)uQkEF^f*Y_1EJfK$J*+pCk(;Sb1hzuNd-45d~|!M!c^+So2WVcDE9dr_IN6N zc`Qr`EX@s<ar=I5sykU4KsjwcZI1=M7o|~j?7g!46!4K{IN!jlW0iFC71>l@tL$NG zDNb}c*X%6J*>tMUKYJL&p1s0WQt2$3pCj9CUuc3}co4^bw`$Okp<+WuQA%FW+1>st zWX!2=bWYPu(SiV7EA4<tjJicYFB{wTPeAUZsn$hnD_Ag$lfrZ;KSzCawL7m3Z0Go% zm!ixHl&*~+L_p4Snrix$whx_`-Fp*nFm*n*y6_}m_UzT(*P32Q!61?1Z>x8Y+Tdc1 zV|@2sVuPM)+y^*jXTn09?s~blyV~Q*pG0V-DOch<G954;wE54NvhF_C=S$cj;#mcd zTGwq@Z^upd9TH2Vt8+c(?;LKHW20Cw#Duf?k|_#})?)@^qs{sbrES@X3SCAic`M<X zz<c%mzPFb5lVrbcBJ6J4$2*A}34IsX;{9GUOVcXN9Sgs|VX_U8qrm{_Y-8<qChZO{ zky#Z?wlTd6ns8<1W8wwmgRn`h)8zG^@ji$!2`K${yTeF#U8AnS$)X3C5BCKN+&^6g zJ)MtJFSOgKuWcml!WOvow<q16zS(_|7TX$-N2dVuOC%8=BhhyeqKljMxA*adUo;b< z&qC@Cn0Afk-T6&TSM|=EH04V8p5o=@G+Zpf<nchmz`hgGq=L)+>(R{{A8S|-Up!eX z4K@>7XSrWKS)p(*d@jVSnj7Y<-af7A>6@vmul|D=l1PZgE@8%6q7-ISP}Rd?h^<Rk zyzL#!zHG@=!1x|5+44C~)S!7vQFKLX`U;|hv!#`Uc-Lp1*P@SzmrR7)FJ5=%xv#Kz znO62`nv&Vqq}IInrfsl=&WNb!D{R`=p(T~=rE;CgOtLZ$MtOGf=o@-q?-q0J%8{pa zc)BPzn&(2#A2JV2qHS5fptm3mFVKywON@gDh@suGp~Iu*M)xVVqoen1=4nNQIsMT+ z3&I082Yz|QM~Ph>gY6^Fxb(?v-vur@f@bH!(Sv&=>m2I}i$QWi=P<Bs&FOY&?<wG> z&*%I*t{>meao%WmXG`)!bkulyV;UJ}Ub@#tHaMJQ>8M@rA~e4ouL%p~h=qHW*ki_m zM2?klkmZim#i7pOO+I98SUI1rnoB!SApgr^xq)u@di-8_6p^%rv_JT&D0N`=V8(fv zFmbp!iJm4$B5UD7-YF^BaK4y=E(!|2rq|y<JLa|saHj4UO1qd<t?fm{(AJBJ3JYrE zYkr;(241*X#ET6OS@^EJB8HgZwn1e)cC6=GWG8D{CT>%CUnjG};CaSH#`yrlO@Wsj zdX*MrtFdbawIOdI25y+W;j0YhlM(kfE#n>n%*anVxT#}PN2Kdce(FaSYdVV-V|_%j zZoA~ZNH?)yFZtmrU!JN2$7vD0b;%YQm5s62MwcXyie|3JdAUZDcGgynIuVyvvw>GA zORP|F|Cu9&tQ?UCO)<*S@BHiX(3G~Eo+Tv*EiA>DHFi#^cD9k6OzTCJVZNk3Tf-#< zJE^X<z6;>XmpiV(>2Yq;Nk!Vl;1KgP^U*9MbHwZY701}tI`@$%X;x5te81j;L)Qtk z`?%A=r4UMLT*U?*fov^E3kL10P2ZV3GoR1pKMZa;MbngYhju9Fc%|u?$9p)OStzO+ zdySSy73kj2(&-|JncO&~5>g;7NECI26dYD4I}A?d6S1P2EOM|5`Vh`SDc0>KT~bm} z-<eSJu@0^ex`BlOG=<^hH|(altasS&qTUt7c%8uYzL5%xMqRtE<7Wfw9etNp*oZ-| z6}HX3$%%~^-Kut5icVO0^LS#C_Tc;WQD3w<P57XF1gnI+I<M~1detRM^8D@~0G+Na z&X}--wW=_TBq+9c2$l?fph!A!7Vqnf=FyezTw0|SGMRhRpxpvxkSb2sK3euau^&9< zB&mAWvXqCqD}odShxMOay_#Muottmc(!eo@63S4@T9$O4FuE(PvWl*|vD)JuZ}R#7 zYC8I7+x~Q%`A=rH3_mdz1pKmp{11c2AsF#*E*;;%FQCKX_!Vf!=2eD!ASc0FpHg9) z3ck<fmQG@PM)OzQs}4rDC#IX3+AF^3-p|(gvI^!7Y-SuPuVpaMw~=wWZ6L1pGk$qf z@J;OHn`Lk%3OB~XopBU77DlekH$y9m8NMy~d;@c$dbh#cE0kZ+WBEIsSHK+BZ84sr z7z-sMlxvA!v&|0fH=X1r=ljuhhZiAT@_r=Z`_h{UdHxU1xAptgCECImZ{&N#e!E@} zt<Ja0wD|hiFy%=AKMoxL(#vOkcU$-CtE$n>bF)9PVxvD6+`<4E!HX~Au{6W7zM(*` zWm)g|Maxg{X9o=t&HDw54Q<;8JpFL(Fkd+&o-$GG?Hg9kH-mFPhuUQu@}ol7yF((k zSG$U>!<mmo`lB)EGpGlbq%(C2!es%JfX5YspHr$y8Gtt4f?6gw>LPI^Rtf{Al4B8^ zZ>dxoe-zEfS-Hf;rW9Mo!v*p4hk<65jLF*0Iiu)g7$vw*=?6+=PTlF5MV%F*eJOwc zmMDksZ%IT37}IhpxV)LA>|VXoZS^w~K&V*AqdkOrMv~aGnHuy<AOq-1N%#DOz59Tf zjcrxkS5$hvWW!D04<&6E?{Lqvq4AW#w-qn8mzxg0LIjJmy;Qz=!k%|b<BY~UmNgQa zZKPlG<$BiD%(005=JPyW=6HGJY)ICCxx+ppY=Z6>^6h2A^-7Dm%6H?Yv8zwY(){wd z3CIkSo<yQx*$7I)1r4y%k1Q=c1z4+e#cn#Q-CX5Bq%T)I5fRcKs*HH%oHa8s-2`38 zMiuf9pN>qV|7>yvgbRN9?caFO$)2S13E^O-Q8J0H6gTo#dxp5t3(5N*6|S7gdPEzW zXYqy1Tm<j$JLNnZb=j4~bEY>AV;SYUn)c(ue$7w!xP-wL=k10|e>7@->Hnv4(Edie z5aoQsocxTLZx5U~hi~4QQPs`;F&gwcF%pmPmvVB(bw8-8KI6E%bM@`AS7zIS^ACp- zB5JGo+DQoB08Qjl6qaJ4U(;_rYO!j|#l1e`b=9qTT5ut`;{9K?M{k0kR|jQOM_mo; zVC7W4#x*;%{E3!9pP-EqT8sCveR`-u62hizbt?-I{bJpRPs9kSD4%p!QpbXMa*cWI zz`<<|Ql+?70e<eAQO^?&e{66IG)ZJQfvWz1Sj<Lgof|8je%I~ItcbePDtNevJeI@L zwfOQp<4P(qs&AG)0u|49r)k21PlO7Jfb?L+)&@RZiLY!O_)zo*KII;u8c_V0hWVvv z)T$6kb-@_+WkIGd;H@h&$POH`W6Vv&K1XZgGmo^f`hgwk8MX>nt1N!l=kz+{qVv@i zbba0p_%4fl+YJ}$dqor>yw*ybIJ8sXKsi##Ru<l+8~9=BJ#XNeJ2Sqj-|d`hHaRtk zRDT+CKNnBQIHdjT_JDP`U=AtP`<K}ba{m#ABg?s^fH%88lJ1w)jo4V&{C2r-c4R%C z<ex?W395~giu4WK;F#k96<4l^SJ2ukrd^b!)@SR(e+KB$Ql7U;E)3k0_<z>ke`<jA z@0kUqV<i;iwx%(H*s;y+9~U>y{-~!2f$$wUQ3_1dt=hErQ3EeS<VGt(gB|DJcH5~@ zXr+O`@(otx%bKo@G^o3vC$m1h2G2o1(y63Z!VDIiqi4hwgPc@$#S3abiZDJhc~6S5 z{X+-B#i@zO=}8nzjEXZkZlKG5_(Ta??W2s53L+iq>#tK4pCynoYQ3=6qdkhVybGlB zu6;vBy!z(tiElfwb;ZzR7ForxHz!`>UM6dj)24o2=Pg%#zLCy(RP1_#3KpzjDZ`<6 zK5K;+#fqDV_!kRm!v>iH52oI4eP05}%%xF-<6jQA)d_tt$x^WHgl+}5`X&sF$zZj~ z!Io_)(UloSrd?Ck+xr@)0G|@}y%HxPQ*UnO@iYxSu{0;|s0x{7Vd8(%kLZG>XKb7e z$s90uKB>Vo+?_MTjiw<?5GGMv^Js3aoNKaaRuVSagmR2$iT;p2d`+oAaeJf4q_Uij zK>q`ZUzLe*DR0O~uc$?${B0`W*Jge7@2B2OOXtv6i#&dk1C~=Ej*QNOH$cO0-<hF> zol`xUI;EEYd%4c+6;I5>bjn9waW@P(KCi9#;Grh`kk!z~ZbY2a@9{rlNkE%Mw3;)I zNTftEmLqNNreU8dJUKQY#}|k1z`{tT-xh5w>>Pnp;iVf@BhpT2ay<spZZe(%@DBeV zCzQbCVOd0A|I|lX<bu8KNsY?}O;nx?JPWXTzr}szcjs8BfA%Nq_|3}i9`D|=KW>fx zZrT1nge&?t+8O^ujx^`YY9>kJV(n8Vtq$8Sr+H`!ZPTXIPXP<qE2n^T|9)z(vGOr8 ze0i&s*HWOtqyGN1i_3Z?*|`a<!kyh}o2)K3zWYC6b*YQL+Ee5>U`V`(Ka&;?k$P6N z^7XxFCLK~nkd&Y!DI6_TLPZOtU)B46Pp$Q<7Av%4X7-9IH(6z_I&j|@y*5Xii{oXo z4nOO;H`8K4E7EHAk}i+nj<308%c6Ns@$rK5MmP>Sq3YwU?jzP}j1E(*+Q9G6&)9wL z22QI?KgKk|QprafYD$3Z7F3{X*m~zm2j4FHx~eDIgdeet*P|KF@RH{rYuX5!8L_Y$ ztwKU@GgyV1S)o(Ft%oVSrLq3-4&8??-mp#Nkuv@ZwxJQn($+{z=u=DF@7fTtLPmDF zIk;fB0F=VsBun2*=e6hrt;yNG)}vRRzU+RpO9$~90>rnA)%rNG5Re7_f)7x2py(Au z;*&UA<UamP-1h9wJmo4AG=frQu0JgN-G%tb=zNqtR-09u4@i4^*>Z@!#{pq?C)6;o zFCJ)c3h>UY)KcIC2N0abe0d1r>Lyz>QdV(vPZ>FU><1hiFcnqd)1!izfUD<XJJO%r zSq5;P(J*#7xV5<MnF*1{bHSySffx5iK4^E<e$r@rw(LT7Y|q}3+>t)380HmRiP!zc zkgv>~W>aEJJwDkXBiOp7R{vQY^nz<#Vm-k^r>tuDl}&kLfLw&pa8~St^n~Gko*W%b z!@S<!B`d}-VRb$&1iN2Xm&;|Yx4;L{KVbn_C|${>0vC7`NN&+P1oqH166nu;cidks z<E`uMpNbf!C2&psG<_`E_4X`ltuFtuJdWus_yq!U`_+K@&&8qtedE5rH`%ZT%#FAr zqF`knm&My-qZ@HEWyW^EcHrk27kL5)xj9T@Zsvj!c&OzdSv13l`6=7A_^sya+i_ym zitoQ{nF_R8T(U5e*b0W|GK(?2{1&}5xGFYVt*z7gGwFS&ctv}?{6gt!^lG?ZanId< z<K*>gHTc74sJAZJ@v7nW^Ydda8yR=I2F^TZCMI-2Xa`BwGzHd|eOG4O@>Jbe=R2rj zH)i6uD)NqnEU{*FQxG{k=mzr+@L>&A#cce7B<0*-3a$eq+Q0!eH{1`BwW_{y7QiDx zL6=VP_#~s2lVmvf6{IIyf^vyi5HFs0^VVf1VOncK^eSCD#-chrNyD+y2J5uLyo$PU zPHk*eA3*!%A8%}4NoEnm$9<p>Z>EG-_8i;SXjlX(2*lk@*Gy5{;c%H3JTZK-6)M?h za|E(uF0+Af0SzZ#H+Qv>^(~l!8-orx!46~jVY9GchP7#${z~t@&@}o>i^LB05Begn z-cm1L4mxwbL)*o!BT%wt78)T7ed~l$^1;%u7?j0fNA{R6nVEoml%)Y>v2g8OnI>Fr zP=kO}t0wbKYa{)0vxyg%jBU4<M=?(x+UeiA_8rXO$P3>qJd!stTpO!}h3$RRel(17 zex+EBm}`6-`E7PPwgYuNj!qs2rj&5iFXXFuaOkt*)7x@H)jL6w#SP@PSRr?C$bjQ? ztb_G*m-<;<9yaxJd*A%3<niI(aImm9PK+y1C)|Ff6V$%W7HlG?B`dy~`_0e3iO8`= z6jl($_ts<*$Uqw2qU92qt8|6Rb~rUICT=QjK7hE3v<z3mFqlOX?@ws<#L}oW+hTl3 z&0_6sCtjFtz=HKhlWW(`2i8#_w9(03ve~``q(*XgViHn&r*zy-k>!{&*A}~f&?c@z zRvq_^oSJ0HOO-yy;tWW2?2uoXzKQ3A7vc~Wqo`X2e58`b+Eo3_R95x%HFlSC3x>e- zv(--HxyqkIH1LuopF0kNM$6r(x|)~`N2*DV-8knNiQwm&Kx4v6`buf&8^n6L*XK_y zYQl-@7J%2c(gw>NclV|`e%?@a(Pot{a?WXOOZKCXURer8t#z`I5u9Q?lw0F1pTu0= zxoEES7<<i7h*@<>rb&8!EQ65S!A2QaF@nNN<~!v!t>5+hAUyAOA)HETmIiDQ2AvOu zu$}@wmn4xI-<9tAkJO5M2M_0y!_U)3WZg-;K6^1(ImwL^#%D8KnON>@HZxmdT@et! z-pFD(2;s$krRrd_*$)Y>6qKOSaW7Yk`QpQRpE~!Ozv)<nf(XgEVVq)U&&5+2j;HP9 z3FC5Q_mWC1t~la&oG1$nxeX09ZpgRKzuHiqZ&Wz7f&~xDUTI3}CinG<8b{BW3iyLG zJWoD4Q?@9Es6y#9U9S*Fh9$0C(B{Xo;*PRnUFc-?n^rV@LdwPtiH);xBxb`y1J9Mj zm6<-H<YUEux0~w43pdLCBQ8?^ux6wfbR%CI(|xsZl$rlXq)s*i72+S)MopU=JBZ5M zw~SsqPrRb!&6y=dzczUON?Bhl``jY^(#IMOZ!+_-pMhS!l8V(NtiPO;ZxoRlFGf$$ zXos`mG`Avh2|);Gi-wh*lK?W=uovEw>3WZQ(oga2MXw$6e2k{CrguxNC<4TF4>{^= zin91FnjU&$4<lsbra+b)NK2Cuk<DjO##GAJJD<O}YxFR2{X&Db>5-RyUvnCx<{U7D zO{?6^iL<_^V4#~r`5f!z^y_luNa>AaXAj*U5G1h?(>79k;x60u2ds<opLzH%3m@x{ z{YCchck{>J1q|*v)82TNRLqSN(psMBwbxz~bCjj}0*#(JQCi{(8Is}p_<sy1b4?Ax zrTF0B<e*S+w(1SjiSD+<L0CO|benHREgMFGkD&3WU!oJ_IcrFrgi+gXc$X@gd2Y2! z?m=dZ_fcx_^05F`UaH-Tsi8Pii@)-{3mpx+hbrfLFX%&q#{xakwyp2Jk$4o!;u(Lf zXZ#hTf{afkmLv=D{&r6O973ERFN0gyMBs=gRo_X9PzT&_M?N*u$j~c6SZVhQXOvfv zCrif;XD<RXsYoLX!=KXvYDyBO7>!Oa@8O^#8$_eS$fY6O%#x6j0dsH6OepRup&#W6 zz-@(D${7p65~t^YXzJXB>pr*O-^V`XQ>VWRAlc}q^Kb7nXgZ28j&t%v!Brotuv%OC zQRYi5B_Fh@4ogsy7QfbN7Bf5*XVmo<6GpJDZtgjl(o8gpzEh+l-kTnBhqA+0q91g| zDvW0*{_&_uUi2vc8B2xDZ{$M@@-$u^ZN-<C+TiC$ckNk%<qFr-0wmwZ_`iS3%*_D% zB$0sYZt5pUJ*>8m2+XrbZnN%js*KTqLO$=REK|-+*S#mjBgQzjr_Glhvs^85C0-cJ z?MHE@uShwue>eQr>cD$X{q7fkb=^DCUpCLUcr^SrBHSQiMCfkuVAfIg0FLj~X@tCB zId`M#-b4`<4B|7;$6tT<SD;l!y$iAh-JvMvG7ougXHJ3F4mma`Sev17)4YCl<hzD< z_A361L*&$gDr`|-9t}13vnP=BGeP8-Xk{N8*xyOa`mHk8$K0*xj&awxCzlsU&UVk> zJ6}o$f4SWr^gEepe`uh#g-gO7;<?giKkBK{_Z3g6$wwoogRfdWJ8FB`yV1sR5j<;< zy?hFwaa=!UnmZ@zCb$P=BqXw`OdaPhsG4>>x=0q6S92f_-Y}a9hDc=*o)2Otl`D>f zSy<@YuK$QW%7&uGpWpA5r4>^BSxaA+3<!y{2x!Cxr|2&3G|r{ZA2zniTkl}T{BEBs zd4u=F98Jol<k|we-QHeortOJy?I0bw4bZ++qgU}k$Osfeo{^C~ej-I?wlNAmu_-)H z`OV`RWh%^WYxO6kVMN~Sc#B}NeQCg?YbJkjnhK|oNn2V-i0BxrN+NQPh_t7O#NC{n zG=crF;@WyYzfC$d?qR_i&5D;G0y%Uv1kwNq4pRKaMbg>H+s$5Rdb03_^tp3Up`9Jt zn2Gv~U;i86*xrfTQ_sHU#*kWA>(LCOid&vqo}rzTx|@^Sja%hR3eVm-04_O4G7t=m z=F(`5z{pmQL#`b5b!6--Yi(4Bj6T<_AN0cLug;gUX;%;LW*tKB3kyk_>P@|U9;Ak; z9HdaRCJt*DHB9SW?roVieC5$s@1+)StZ0d`p#HX;tb22OJU1~V+Nds3VLnc~ruY|w z;N9ll4pY+iKTc%0|KblCBWY63e-F#{FXHe&X$XsbC&*^jL9bP|vM*ABy@|`y#J4a# zFcdMC4@Tp<WLC!3Vy3%)t<%ZwBj*b@J!b+zPqGPh_f1atbzi${QUt!L+NRB0`vx1U zbTSZ`+X~T}vO~9XQP#rL@y2x1j=<%^08&~_X4qrbF>{+IEfC^KTbieyzE&0Y()XI5 z?~!JWBJ`v>B`e%(9x7^|!Ngc$16rwGjUFYPG#<A6Iz^~+TB>?pzAkw6Lo|RZ2ZcSt zzB3pbG6XZIpzxaC1zyEXiieL%uCrPHaNYfKkTs#KUOp&@kYGt;E>R`CL7X!Zo8B&Q z6o@T7`JT&qu%ZcNI=(aD>jQ1pS(W@`wKyPpP<Fxt{m^*9v~@xLX800>eNL0AgBc+j z;&nAnlph(xn9Q)e?8$+L!}kXCw6K){f2oeA@Mb-#h`X5~DBObvkPXM1+q+@E+M~VR zPDr(0eNqvHs_Ucg`G5=@c?R4bI0a~ZjK<uIi(XI_HDNQqf&Vz{)#l5LvnsHI3s;-I z6F88(M)E(|JnN?pMCQ*8+vE~Faff5gSnT2E0x8Nm6E$|2)w&dO3UH>xaW}QVl$LKe zVGjIv5#%e{4mn{0b=6Li+lRB8@Kvq#4MHkyph8ysx1STsYI3D`bHpFrfGA^51lrus zcAF0sJ8h`<C4ruO_ECH7gO+GUG$E+l*pC?W@S;In>hvqMnHl-}G<7pO2;*O#Se-`; z_~Y}qqB36lr0FYT3HP1E^oHkS4Ws8pzaNf4vtA}!gsfYB?%zGKr@B3_;&fT1&&7H8 z2yVCWUuu@n|JG^$creYF%pCN$mvHH!OEX6PiP*C<_TG-uX6@yt057Rk88k<rKbmRu z7c}P+9c|t4u?epI0ipLANE!1MBQYs>3V{B$3w#+dbW)lIi4WhbpmBYOKl_oB7~ykf zBXc*7zn3nZQ)wSG_M+)#Ow&a$_W?p2PuDytIm|bDOqr*FF>{MHoyfB{f!SU;YD^lV z7%eRrUDVBiqs>%2D;jR%6$7;sNTV`(6YbIa(=o-)c4`M%+KoW0%=)z%dX(!6-^;6B zNpJTA#T-kRL`0HDb$ZIWnkkQp$OKYdgsg6l48#X&^>j6OBO<uC>xLX01uAi(Dc1tF z4NdIrTz|X87W*a2J9bR*p;uy3?)ZxS#c|uJeiGxq&N7_uLV-d+`7tZz$cYMUp*c{L za8EzxEgzJ9&-yTtk0?>8DO#kt`z!kK8m@rj9@%6u!;z|xadVMM-KVIC1qNqg$+LlX zyYn1A(xjlaf(V!JK$drkb=k;zwq$Kk{K>Mlc_WTt7>Fw=LA9Dhf-UE)!}2D7ON8?W zXt0myZGXU;4ajh1K^h+p*K>9m7jmWjo-_H^u!AWw{Fqc}Qj3Q?8os2lFK1a=#pTu( z=|#r2)7L;*&%aSU*nrkEMMk3BGQoP?0h{WgmFfiK(5s=ZHK}la9oN46rlLZ7#Zb6( zbNE%9YVavwdlYkTBC>Ej&0YBDxHa#qyDv4t4MPfw@*O&$kg{2GQfvw9a5s|Swj2Io z_cGJFf|B6U?C)3y2n{wZG)IQWegBOjEi-NHnRhjb&ci+#k0wtNPR=1rONWXoBG|oY zL`T^*1TXUGwD>=w{LCc(YprOm+OzJwTKdQFNmt_ki_<Fqo(;zTkRdD2$tdV!yaLV1 z@LSDXY=a08Z^=rnM0Ua&A-s$ILvd2q48twvt`f$$C!QVlQ}&@G1pZpuC5YIlNX3Zt zaB<7@1O=<5@Iwx=%{$yaFYkOc7Gi0Jb64TDgLbE^I}<YM-pY^@fSF>$;v__1!G<)c zz{uf7NDw+t0awIArY#P)2<i@{@D%Z*Mh;7+l_&GQ5@jP054^?00mzMf{xpH$Pxfnt zXe2|HUbjCR6W?!>O1$6F9)#8L%$D?-W>i}c?keze(Y0wXab@0%Ogp)d1;cE$*{WJ# zBb_5p0hA%LieC$SDjw`V+$dnLtMitQM7p+9SP~XqXYNS!h&8X(O+%FM<n2izgt);s z<x%zF8E{25_{^+_pF{4Y?=SoEOf5e+m1L);<%z%svq#s1^A4<NY_Lgzx0C3<A|_0= zw3yj>9L-=kyp(ChkF?ndX~sW5S7$k&KXIyaD@-U2g;tPTz4wtpX!jpWq%WUG-W%9G z0nrEUo4kok%O_9%{u5}VQOIwc6+1Tok;U7D5}_R61`T`bqY0;w;YswaZOYQKRdJzO zoPi%i40CP=bHY40>Z?>e*<iib6P2j=5L^2@6D-AH(^&o0KdQ(7J58?q|4u3WhqOEY z$DpgpTT#lnQ^!1b&dt(iPR?$#B3jDw>*6JggSmXxDv$c!O*O5+a(pN?hF|U3pL2RM z`M#u>-)x|p;jBK;cs1)@oMg0$f$A9}!xtC50)%?p!~)$voA>1rs7&f{3F}{L4T~RZ z*z&rIkV6_1DJ$(Sq}JY&8Co3SbYp5wqlH*hoqT6Q2a4%!{y^&Qv=NB=43S4~auod( zxYvYv@xbV(L&fKQ%=Q;c%K2P{et?oqSL~PG+JEQ~jdGlDn;^Na_m>RtJ`=G{Hq4im z(1)Ecl}BT#HY;D<(x{Iw!7uc~*&`AgMxC;xyb^X!R+b#APXRC9kmEP)(Ei6%m{S17 zI7;?Rc)KvF&-Ka-Xldm8w&xP3%<P4=r=-TZVXvd=KtR|Z_d2CNC+z-{Evf%Z`|CgY dTmL(6hGakd`@moSM6vtlIii2^@_?t~{{r*mjY$9i diff --git a/doc/modules/cassandra/assets/images/Figure_5_read_repair.jpg b/doc/modules/cassandra/assets/images/Figure_5_read_repair.jpg index d9c04857f76bd84ac0d19c82bc4b8448579b26a7..c9757bcaddf218cac2fb082a00cdd75cdc8fa482 100644 GIT binary patch literal 54451 zcmeFZdpOi>_%Ax864E5ch^Z*2m<o~8yz(N1oO7JFB!)DJ$;g-~IfU@$SWc7Ve3&pO zr-?DV9VBN)j2R_4&XmSCbFjbfUi-J!-uwDp*V@<qYh7!t;kjmv@AWnJ^W4vU-=EL@ zd2CN^&p{5@S=(AecI?;zxd47awx=PM|NivfeeB%1V+Z)XW9NT7{@bga|K8;Pc7y-N zW&d&cfA@L&4Mb}1j=y(li|jZC*(tR{L~6%&F9ZgG?AZNZ?hX0B9y@l5?ApCYbg!7W z1b9L70m#lBA|g9?iR|9JYZrKREckoKE~(uIkDj*JBkg=e^jNsenfs4R_Ntz5=$3UE zVyo$0y&WeeE+?;WNb&dybq!4|eS@=xM(2zz|Fp8UxnOJO>gMj@dD+W5;9B7IpkP!; zL?q@;6gK+qgZPBRq=(5VnOWI6xp_~X=07ieQC412SykQGL~d?ref7GnhuTZ)>;LEN zyWx@1vGIw?DLRui|K;n#x5e*EoF6|oesMQ}-&_CMwF4sZU#IN98210NOA54W=dN8M zyF~xBYsbzgaEM6l+I{r&o`V+7qF2JDkDa-{SLXcVl7?<ERXrEB?A6;t;&N*G%;TJY zP5VDB`~PNG-2X4j{;OgC%`Q4*pU4hy@kFE`2#8Rq@jM0czsLWcgZ~kIK)YM0jvw8I zG|6|H;U{{ChbY18_ZWNmuCe77bzay|RupX;k~F(>VqHJ9rl7_o%zMI`<M}X5S6Bb7 z?h5*<?XaHo+aN9N+_X2FDZ)PxbERGCf?QLjZOHCz$b()+^ZE*b<uq;2^iSS~?7Elb zp2Vz3BwQl`H$sV$p<C!xlUig&uf%#%BM{*BpyzI2aY7i$>pjbaOeQy&AF=mZe`f!) zD?r24e}97aOX^U4LspS(OLjo~`|#Hf?B4Rm%quJVwlv*b?rJN$-CC=cKTVR=9PPG* zYV>Ftp7ZRVV|zs1)u?r?T&?-i>O)#?wH}WoNTIy(?M#7>!$hQm_qfuk_T?c`9mY)C zrSX-8ntVRKBv6|g9kOVt_PAYDBuC97hj5h8u)b;$3z{&z4T-Z5PE**<h@H6OBm6_W zb9}s@90Px}4PiSHrh<^1+#Es)-w=r4ta}0k_BZAjDdia3qnQxEW>5`h5tK2UO*P<S z-IcwVtBFA=`7mrXQWuF++CuRv&TC2-Vk(M@0kU}h_;=foR`dfxsIp%Fmd!&0=oE!} z7f55<cnfm7w6G@KS-`SYL;EPsYF==6ng#Ev^EXjOUx*WOb+kR?TUuNTu=Al0zbsbR zVH)(=kIPKWk6ofxXIKA-u`_x%X>@fda7^~xXp6Sw$4d9MG{lYXBe%ghei8k%M$t{n zGv~5rnHbAA$br{<>&cBjD^KDM=Vy%7^yzjgIYv}D8pNj5R(`bEm_QoihX~CRE8V<f z!NtlP_^JhGnTdYm1O?jNUw`7CuI0OO@L6*LL;vv`tL6+$+4rdY&R$ehtn%My$RYN7 zM_W#b!9KPW@GyjKU-OfEWpIM4@FU2*=4bJ5g$isZzH}(JQBaRp!A8w(rf)-nVA9x9 zBrr7xo9gLz;LsKxu$+?H5PizWU_0a42EBL2Bc_7$CR(>1wZfyK1I1c?Aa1e%#&h8q zk$v$s0wtlm3#Ay;kDMgB0<VLfA$N7;S!_e%w;^*m6UoR6sIL#Be|W#GvK7sj#SN-r zA*W>LgN)w&M(<lFw6aUd<NAw=K6#g5U)5o5vfJ}?clkV)#$r$2<-{|!NKlvk?o|eF z#k)H9?JACle-|!(>2im^o`~X*XT`5adyb{c%Vpg`PrCOqzo9g=Mwj40CRJ^O0Gu(9 zI_H#HyqitzT~Ace`GkvR|LngjM`RK9U<Z4)(z-}YhBOf4e*b3VK&aK2j`H(S{i)R% z70rOF>5mTC?y31skm)+KT#Vp&_QcAw)bBSFQv7!hZ9oqI)6AAU*1(=^NPidflc^`S zn;DCynICAT#9IC_jDfH4wjnQlXQpYAjLEM&leY~GzbT-bhc%C*DRXIzm$G?Dm|%VA z0e?J=A|4j0b7$GX@A`z*_)XK@q5GfK_%5zT>|FZf<KH1|){rk{+;jCz9}LTe1L};_ zZ3wDPL?|B0i^hK>kk|iRIz}2^pDnubo2QPmL@47A3g5RuSH9094^0C?9;+-ZBgr(z zjFfI_TZyf49J<kQwgD9yRr6l9ddUzL6t_cLCq`k+MvQ2GBD+x4_*J!A&~ZMB$)5b| zS|Ptb=ul6(IOh7&d6}<rC5|^=)WM|1{4YPsOb^wlFwSN-6$WfWbcGE_G0S{AkB-9) zrhk9Qt~jKN>VIsGo=MSWPxryd1ip+2w^h)*4Y`4wD&?vIRcv`~5cr9e%lOAXnFh`i z9uanSSaFtnk%#d{d>^(d&Y^*wkdT%{bCRI*O#oB{*87`PYx`)aVsZ6EYoq<{j+Zj& zFgsv^=Z>c%Bq?(RjQw~+T;Mh&(ME7@8xmsX>cMtO=rUVkkSo%^a{9Y79wf)GQU{?x z6AiIXpO8>|bTp%naA+w%EUj)cj<1Nb3DoKBI?>uVhKY!%_&n_wQAMe?l-J-L_hQBt zo|t#UpymTLm^IxWohw#CH2O-jm#jugJrcYdDjb{7**6xt-|&5zU0!2+lN{of@J$?Z zi|tCXwi9<(_I4`HE_Ey%?sch2q>O96%NG!yQ+5#M@{nw2;v<F%I4cCYD%Sw0;$$%y zt2X~eIVm=Sdmo75{RzOiR{;_a78;e?rOUZVnj=Z`%{atvLQ0#kOxq!_>hh%TcY1x% zt+1P6Hw*36T6aHd6l~Qe4`*v$39T5gIV9#-SZ6eIzS-C8&v~OBR8OZOc{=<nQQhpE zV|Iy)FF9Pl#-s02Jjrv1i_FrCd0j8J5*t~k+0<;L!9FX+Tu08gFZpQ~K1$WZ#s-#q z%`6$`e61%|O!^E&#}Z=0Wy^|+$~wYyXk#;@YgK6U;R_$uR%G4nPeY`w(thPu1O`>1 zT65bHiz>aR8JCfhpRJ+-V*{fOI)z3DVMH2z>F;+v?zpXb#@HxO?La6SF5IsmcqD}K z!@+Qc7fujic=aWMoT*hEZl55>{Aeg_A6|JHcLhjhlH&4V>5+Sz5eHC6X#;eEiuA-v zBuNSQp_KaYXv(sGs#|Lu!=cqD=E4i>oNdUpZOC=fZ$rdA1lP)3b{n!J@oZedD+YVL zcqqpBcjQa_(q^d^_U(1MgST;Yj#C8vXV2$lU9{<TzVA#}Ws0&t-nlWF%Q7lbvkEqS zgA%-AvSnhI`+zn7D2-Mvmu3U`L-zE==P4x#GR}YO{$FL_@rV`m!xXmbYr<S<1kTa+ zY615+k!zGUS6jTtGyTFhpg)uoC_F;xmE)lHCxm@0jEpGxE7^P$V{ZO|@W*0x**RyD zleFWzW)0R+x#7&Ov@4-=4N~Em;UVSnvi?~MPI<)Q!_BH<eWMGNEDcTXTum2Ozp=ws zhuzJhGIUj;)@7^_Tw!C!agFz!m>vS#8j*Mh+0efl3dUL?XM#pL0F-hs0jr!m8bc1h z1B;;g%RhKCdq(iL4$g_igC=jG9|%<d8sE0s2gq<>Z$oGWQuSE8t$vz;j`@f|+Dsk) zj2<yk@PrbGoc_rq_9DcW8DtV)1}q}De*<B2aOje;N)mgphtWvr$+$mZZra|~DE#n} z1Mi0_+prJ^8Q9j#O-)n!w#<L{N71?^uxZ^)tnXyTFkN$~%2@7FR=C@Dm#@V2BV&h+ zdqxUNT=UP=S6ZLSJz>kzGIA}lYpQaTPRC@DJ}$xUmWBNKT)O5<wNaj0WxZvMyZq z$M@+tr+rm8e=7U=HYA-f`Ccft$qUAh%>JA4-|$bezpW<EL#ITzIN&!}K3)`%O*J_I z%&t<NE)dXHtOew-YiJF?u?b%evO6QiTx)`yN)aI*1niwh$tFd1_6TWIxumf~r;3Ec zKWO45!J?p)auu4ca#WCmml3|Dy!^HrSSF;Y>;^DwA?!hZ=f`H=Ilz^hOOVGMdCA`8 z4;k(_lIyR@>AHtIT}iznZnsrdyKY+lp<G|hSeWB{`D*2*zCiukzE*J5y~`JyB>Z(< z(FYqpD4Qz7!*tOo<1;CB$K$iVCRI2|Kcx2UPxF3E^t<SXT6pz{euj%pc~Xl%k!gGT zQ|^_ATQp>l((UTJG@0ef>ZTlV_ruvtdDvSLpsv0m&qLy=$elOAbeZuG(JD;k{mD0D zBYU9x_?m+Jl@%~VIPJk5ZUYo?@Uo>gzl?_%pJiCeT|qxB@5i@K1uJcZ-1kLN)IoLn z=l&lP-ALfLJYR;$M6af?pwsfb1omZ^IIvD5O0VE0u!i$p2~fUtDl;#h1r)Fg=DONz zmhK|&k!P$K^Q&KA(Dj><>j#0{q2>ZfWBXU>m3dHQs*jB_!NSd>-q9-&W>A@P<Bsgl z^O%E`+4NrPh{%O9M|Z6e8cSomO5@x_iR+|C_F|9Bd{32Y`u_5rQe3QEi=)&{W2>QC zj{@b&6h?AY((IfcWuwNDn#)UFT(c=h%nb?MXYmO5swJnCTO&w7gf;LE10&q$?5H?X z9iAN)-3ua?)<i-uLlisc3xt+_27=xoWpPp!q5kIV6(Ur4d=l&Z!{<<$UdY|+FaGwi zk1i*FlFuOoI|0XaOE?i+JJ2UYur}OwV5$@sCP*_s1Bh~y05hH|KSY9e0T}FsDRfA| zlhyM40UDEXDD;jq-(m=Dlltqc!#Vf{utAeMjapZ(OYwfYC9rZZ87i|*)5?m!HFoP) zmBG26!^H<{YnZV`j%bvl!rjc9IU08QC9>inDjy3~m8<Qi-D*5)1KcOdJkl<_I~}<+ zQkQhuySnZ~XOXQF6c(uY@eEPB=tYt3YrL)S9fIvh%pmL`gfT#G@LUz;h1l>Ma5g|0 zPY;XcMsW-i!w5mheS8J>_<i|!ysAHTow}7l<+3L-eu)n8wRb5dmWq78`{M3+MR-ir z=cO~)M}LfO1-32+%6K%^XgX6bHKrX_3$ZQ?P@q}4lYFFI_*Li5O_01xvkS|j>HKQV zE8;R$<Fu<sl|1_G3YXrkzJ8ahk}g*e6{hQJu9Bvolk=>=(b0k5z(WfMVC*SmBWzWk zr?m~~0i&Dvyik6#haks?gEjU~p%ewUh+s;CuPYq0oP?jj7f4==y>Rx2=>VAcghP!# z!&_IS1WTLn#IF6@kc4RVw>K&f;gE0=yi}>%T$UZhZJk0iZ9|$7%tN1Q^^vQvm$B7^ zhUjg`)t@+`F&aKFweYqzdNsKAS&m^!w!C&<wK&NiT@xELiDv(ZMN6@#3+OTY&wqa% z4tGDEm_7D2-hIrxutxe9A*_YJ4RJ5wAoHepg&iXcAYu;?*LZ&Kf2~muV<CGfN!yS; zT`+*kBGHfvSORk!Ld{4L!bUsX=baR>H7>2grU)hlijrH$REFf2mqaf8?b9!QcwcT* zV7fuHvfI5V8BeB41Mh-xn%H_RD+IRVS6CERMDQ4+kc)uw9XLp8oxG2r8E+Dt)lNgC zm`cqNRSd>cc_)EmT-wY=4M}>@dmHko{z@>KVae{zJTH3mUjW`O$in$AY(wU6rgA*F z#T>OM9y*1B+J<CoLzKFVmZF0B_NYBM^Jo>{;H~|CbM7C`zmrpb88xW*kp8LSz5jyr zk$%>*YJ<aO=R#gjJJc^}8g5Si!)fqZ6O}Y{IOkayiA&Ze2Avq@k)|evaf_sH)eNDS zJ#R<wuoY$wKDBNS7}R*5E3o5;zJV#>E&@APW_vbDOGa^K^?XH6aZ<$d5cOz=0r_#q zjQSZ}w_h!139Z|ZmH1T_;vP(~!-97cr#9kkQWVPd5!7OM5~N2XsOnU~3!=z2B=q}1 z`Y+r?xxK}M7lDj0mbS!Hc|IL^aPicD>`d{Kbfpg(-5(3mkLrvnE&M)_BajZ`>QOm+ z_K>sH2a?dWnO}dCXF9#@+Yxs)NL0dpLidfjE+bO%2676@6%{m?8v^rhijNSxi3jy5 z4|BI6qKNB6F}?<eOGKff>ae0L&*Ux*zLRlUQp$>p@Snjwsn0@(-pVY8qYMVxscK^{ z0C1P?&IU-)-+mKcs;{7!CbZ~)5gjrj4beiRIh@{)s|l^UvkX=rI77*WGit%)Rl5Ur zZF-QQ)*{Q3A5P0fzRb#u_5O;))l#HC)jgTJwdo#yDb-JUfe;C1JdsW76#kLdfHE%< z_@s}NLCjTeL&%uUc%nYt@C`JM(CE}HA$R+9qfV$7pw40(@SdJ~++r4v+yyhlLVdr0 z9E=kz$y_Z!ot8mY=L&#{qtv|}(Y#pU7+odf@aMHx4|)h<T^c};b!*Q3qb;a+Ww+XI zRm*=$lDzH;<QOAa#BInikOE<yY(qfJoGxrbemJUc?6xA9WfHoz30Dg0F@$*GNsa{q z2PUL@f>QpiYrlSGqzZNYe^0Gb?!=6FfYr<eLbrYy`pi$R4j^6an-0F^&CU$XFAST# z(CpyPig|L~^y~1YZ{fRsM0|SXb7(@!mN4hE4GFV`Cci8?^$SJ<gXi>9!TurSpfRCa zgD^*I=~Y(eVn;eox2_R-4$W^vQdRc;c>T{00`mapYhKoak?W`FB#^4x^+}Mw5!Y%i z$}wEUuqht#Qh{|ieL>eY<R)BfcJkmUhq^lA>YnjGrMkVs)Xu+lX}8^NpU_alB-#1b zgW>u{piS~EG^^CZJOvEcH*Gegt?=UB_;(C)&;vv)J5?y2z{6pUgS#*5Z$ln>uY*wD z28-*vy)P;phXZzyLMALll_tE?FP-`GK6|-NXB)zo<uNy$KLbkKE<q+Dc*nkbw#-4g zHrJkGLrvu^hrbVCA>P2!Jh)qd^MApgZbSBBMr|{};s>{PIkN(K{cneAt>?)1MnZ@3 z$dSRusVU?70>Z!{JW3zGG=xmqGUUq?a6bWIEl_%dP%LYHAmSofV)tgR(<EU&kBX#g zTfWM^g;4_nIj%z4{zp0|mG<>X=otFEPV*EQc$@<rA?_v2Rc%8WL}@d3sNH99e#ALa zufUFZGR#~GKUyrxnOJ$vIlB#U!xI`iOqhgpp~@#@zkCV)#^S+{myJ8WYil*A1l~BI z4JDwHinKA=-pJTm;m_f+6so6C66^lsQ5L}<hV*pnkQF3u$F2j`7Kh`Y=b_<N5e5HC zU@*L_aCD-dJ1poNTw#(dLkYYy_&04)J?p7N8NN0)ZrK6o=K15ChJPVt`O56p<Q3b~ z{#YC41l5QPQxK%gok3z#S;R!rwRkABXGAy`dUqj}S)}C7kxw>AgHwgDPaC{WAD;<K z=lK!4A#$u%k7yfZ^p~HbFnDfc1JHS%X<Pnu*l74{RNy5Cf+>>i2>$7PLQ~+qP~3qN z#k5K5FyW~76oU>Q_Za6Q=Wog`S}`Ujnc4?{S??aDDJT(V5>v7?9<b8Zm2Rr~0vjDx zze2b7`6uV*FzQRO*H`YDlV+yYfp!Ki*{BK=^LmrmfeN(hKOdJrCL-NOaE+Ctx~;aI z_+J=;-bQ(f25A^N7?+~)4Elb3TFEk%>jS)HGlb&ObM`T@Fb9>RKrCB{J3gZGr<^ew z%xXd4PE&yob;vWJspiLaTzJhPP+^M8bn^;y<=z1X|Cn(_a^&kEo(9<mJ*o)k3B{k} zYi~ok!5xYon|LY>MwUclL23TkEW-P0QV&|TB`(12vEiVgtt~3IEUu%#{bSD<H|oLu zmpB;Hl66!IxCexuH(vVK{mn0cr#r#Bc#-KD`FwNm^PNY9h5K^`p;1!a63pxXlRH_r zU3kA;e=WqcpbZDkKbmb!&4$lvTG67XpOTITHrm)*$#`1abGDL!{KxP#WUcQ3ELoyS z7{p%e&z>$NPHX(H_6SrhfTYn?@>ov$m$<bSp<zycj)z@up|q;)d(mKA7+Ig0qUfMM zHnNM3s<@t!SVy@U1zcj#VGU8M@|<NZyrV_uc*cFaGEh9hvRR=I>p{kwgdbpvH-4>) z0cg8m#GUU<3AEVk71iMN=h(3t(hwqS5YQm3%xg{KM~O}H@f0xpCAPd!V;m-2k-{+1 zDO+KWj+}4Hdqe<<LomR<%JY0W`Jz>{yZGD$n5xEOv#7mc4`aiuq7^za;52{7yoO5Q z7$R!jn<kj}Y38eo$>^EV>Z0fK^ftS`y~r&HMXOFA0@tUbxc-7h{kpq65b2U$U@2Z~ z%rHub0AgxT^!59AU7W#~@-g%ZB^j|77hU$fOO36w@cn!Rw>U5C@JbpNb;4Z->ESt# z>>Hf2pUdm-wE2_c^r%BG<~i;XJI)>qpwNC;V%L!ga|PTTAWA=l(ZP12F+`A;m-7uu z{;qMV5HvdI^4?2@oHhJgF5&ecl#dg1P_7+4i2Jjl9oz#+!omEQU2Pnv_zl=TK!W>) zy+2jqFE|FdC-{2>R*y?;X>Li>J3aIl+a~#i_4VdqQ>RKP8HRQ`bLG0MT^Wp9ZU5W7 zyfdI~8?rhsklL6$$&*M9Pe5}+NDh81MmD)s(24fIPY?Nv@l1px+mIcEkZs7$)?TN3 zu!CgyL%a%k^u&;<Mc6fXBuN&l+B<#^>?euhtl7F0RA?RU9H*|uhE9&X-9vS0{ao%7 z=;1QE=`Z4NHowWHEkES@%&F%o8j$}P2Px@aC!OZaMXlV$88Tr9Dw2iT_)i3>xuUN9 z`1i;?8`A`NJ_aZ|30Kn#@kS@6eL01-d2^?cc6yZW&Qhw=rry1FXE?UT#fx7PjrSE) zBj%nV4&YE6c^a~T!Iy4<NdnXJ>q%&*B%v98xL9>$x)<5RSn<yKGb0AgI*K;E#YYNC zX)mUR&0&+i35~BaP1X<jp}cP(u8p1}Of_*Y06{zhDl3*?*T#YMW~31guIc<jrxD2n zn%M<x(>%Hlec=3k7haz^{8h5|#QB!)47*SIlPhG&gHs;Ub|YZ{k`(&8AKt$oxL+5f z%OaBN0%xo+s{@TcL;5qG2Fcu#uh_lXfOsFP`b~XgB<f4Esfnwg*xU^N`R-Z)M@cB| zS`&N_XEO3j$&A4K#+UAA=C(%H;6M6{zan>JxC}Kr(Y1SjOnVRTy!aQ%nGy$wmgHKB zcaPkxD+_~DTW_CP$_)4Z(1u*G;oY{KH(5PPr81fjtITr#Ng%8;G#^8pG)1s8xMaY! z<TF-@`9kGL;zzoPgG3MI0A5Bt(bV7y_KOXg#8(|6m;d<K8Z5fxehVpsm&cg_6YNbI zL)=^&u(9HNqmopDTm4}ibbN<Ji0(#RkX^gDzviVgot+;d)bxCOdpN#y-)T!ssJ`JT znpSM*VG1;%RN;zdbwv4Sc5%InEh&2KcUE|8R)-4;rsN4jN_)GXcEucZHX@VlbSl>B za`IjL=$i%YCUor|jEk)T@;2nk91c-|jR5P!*3Ff=vfd0PZ<-*z=)q{2=$_d#GjokS zBgjamQ*8y+re;BbUzC*QYl3I>Px*A#jZsqI{F<8dwh^KDd!C5b$NZpPN;;R(L`o4k z>FF;fsK&{2M7RMQOi|yj^vg;Y046gQ%CiW#gFF+Xsaxl!JR9wl-+DRU-fnQ|=jvIo z!fU-In1Zug7qUKeGpo%|aP<Ck@ofmsvs3=7T1R~|!n%?)SmY!We+`zkjE0UZd$9L< zgK<x2K0X$3R(KdNo40pL6Kakr{^=#BEZV~m&GH?^Sw=M93$6{TSMxCO5()RcS<15# zPEeYU8G>DB<bAx3pr}K%>aA>X9nV`h(IwHrHm`e*`XL)Cu$k*$DPrv{<sdRltxuLe zn)C(6;d_IfKTDg?>ZW_f^F(OQbZPer{!22E`TSRwv!C}LFUML7l{K~@`hqcFNpP$n zR{Dsb<yX9*++3Z13XG(8fMY%mS8#sySlB#zH5DV1$m(Zqt~9aN`{;%#N7yzGdy!h; zj+QaK!sNjuE~AExGQUxiSp&Ar82g<k1PD?Q;rifI26KId%2W=uU?)sG<a@FPm4Ni0 z{V+K&Rs1n}g(FHsJn~13W*VV80y(0+4PMiAf**9tFjN34Y<K<B7pU>G^y^uHfmgyL z+hS$it)(3?At;x+`rPW%o?p|p=&P;c4%7DHwX9Lik?|svrOoUjl6GWgrhTL5IMYhT zqwlDd%%vUw$q;>qlmppF%3kbx|0VuyAdoevfFGbV?|E9$UH%<GMt1w8nOJkutI8*H zvK-kY8exA(uDmg{ADx8K5C3Mtxl>S0|D){7FymsUW;Cn}2l7I3@jyGX^%38IhahxY zMxw=WCpnub1_fvd+)+VghtmF%GU0m|`FaCZ=%377Xq$kqRS1glMzfGhSd-qU&9laJ z9|BEPSq^wFm7c-L<s44`eb6%vd(4lrMSGFr*r<LG9No-|cn*~IsGJ;HG1!Le$3C!a z_iHu2Ln>XmWxrccdQZ$&D$(r}6=IX|^x7-myV=8T;btdt*LlmUch}lC%x+K&d*{3# z;HRPdv{j+}$Ou}u;(LxPaEOUaAs!;kJ;7fX{R(98K={~gyt>!Had;t?zyg^n-htr9 zAl69kOTe6KAEyL6O%a?D<X63KtjEqH_gwCqP>d9+eyYo}8%Ep`!f@x=a9YMLtPcxG zg~<aE{o4>|NBHy0*sN}$K0C1?fCL2fw(YOl`5Cp<l3M${$#zdkz)4;4j+@j<qrd(D z4*Nr;?$<7|j=n$M#5?21+9J_z3<rH?o{l_C&heIB@bkRfH4sU7usAj8?C;4z%vW^7 z<IZ~A&Fjl}L=YEBq3Dkib<DRREJ80_47=yu)}>nkO>G>15=!~QQ_-*10jvHyn1g^O zoG(r4Q;ctOil?B7lO&pPt$c19ZYwAW5rV#-KU*Rx*chJny8J${9J}PD+L6yXRGM<E z<O}wxl)a`%_ILA<=QVUcI)grqy!?53cFKR68Wj|c(vLdZdZ+bt?fB-|i33e)+#lx` z7Z>dsUxEF%5E?|r+3J6aAq(b9NBkH<8WHA{T&yQZCtM|}gc70V`*9`{H*u=G>%wXO zz3o0>o_~(5yi1GX`U!GO&sTE%_~=ae1K1|oxDZ34@<Ovv*GGW1;OD~_1lXk2W(BQY z;XLRgyhN-#ov#aG4&KMZ>T-*CQTTC{eXn_lhaHgouQ^>MVLUDTApM87O(^etsJT){ z;&N}-k<<Ygl&s@1*{eiTEQ`O3ZpCQ;&g^e<Xev>ZuLju836$O-_nYo6^$AvyW*6VT z5k{U-cW62JW#`JUzyCml+NOL|)IobpGzMcZ^LfN~;3R3|=?Oc|O{3b2VF{_50pq1e z-qn~clGPZ|tT0GIyeyGw2Rzk5{!j<Af<fc=_QfE+0-(um;zk2uOj0~@kEv`aR)o18 zU*|t<&E}@c#GO|A(fP5YIwWes<!wYg5Xar}T5FcH+X_t;oL^fCZJ$nFJD%zBOG|^) z{wP49vQO)DN~5ipT8nL@P47FGtlL?)vzx<~rq?DxVA3n|_G$|DMPl%L{-IKAS&Ez2 zv0P1=7mwd6#i^V<Z?Wru#gRv@i_i86^lXR8_C_14N*6~q(#%ghe%>&0a<%p(3+4AB zr?iEfCUxk=_3KA1PWBdNdCNFkPybic^B?HvKY8Af%5pl0jp?3MnK1QX%|u*a_a}6m zei32}qCUP2Y3Xdfb<(rNyH+n+x9S^eLt20I?<pwv@v<--8Pt2>Y$IXPSe=QG0=aW! zJo2#cEOwJM*t<R@&Dw_a5drm0zI4HS9X0eZAo3>Z{76SE&l;$9uwMA)C3ueC3k3A5 zh_w+W3%c>ziE~fkk*;!`^V%15__l%qX3cU<m<yo-BqhUi2&i>>(j0Wfq>|jkR~g~M ztArDXX6r{XHmG^xZk#zMF_lTVM-(+kzHHRaraTm$R(A3pzwsdkzCnqLGIswEH{qJE zNxA6mF6%7oEOq|5hIn-~#NAy){22stc%SEPoY`UAZ|xD+lBC>$P|Ybr(uEhps8CHm z#GSKwn8Q}SGUpZtdO9&}cdg||92>*V7WgkUdAIko29scH$BfB5W_k&RrL4*g2f|qt zm5>m|G)1;c7p22#otIAzdUCp>odcthQkz2kkB;l=9*<Ugr5AEC^X<{Ag&Hc~S%|6m zInhV@UxrP6*m`rDDUnZBZ=MVo=G(B}cYDtk<{F06B**f-Lhh<H=+wUK7PnRKOMhAY z^5`EbR4dH+1jX<#2^u}h*)GW2HYaO53SU=vxa38hV9ZYaUIu$>j2_tTwY82byd5}e zPR5D^M|vQ414Y@>>pi$K!<x>0y0)i?W5RXdT>Js(`3=o1w0r52wGsP(OqmYCOKi&9 z9C5)APf0k_brh(YA3$=w1l4?5Ch7h*q$Mcc)NC6fb`h_{?rW%AV(g^^XC$={t~HI~ z_3#7uW1QU`hHaF@uwOp$5B%6O0Qvx6&03=?Uz)u<^PT>&bjY)x(lXc$lg4=gmf(#D zJ(?ru1(k%LPB(s(AiE<>wpMx;8#5nue`9*NPU4Raucua<_xbLBZ6zh(oGSk5QMUYp zld!Q*?h0Dp{6y$uCdaM9Q_{hiW^wkld-f{V+uLs=<RfoJ<)c^O1a&Ooq?L^i^la8b z)GZ3#VL5OtEUb3$b9Em_{vqFtDJkN`@l3_cDD?>C*$a8GbwE;Y@v*U!kq#mjLy_;J z;RZTLuW^HOtk&U;_LvKSxiQIK(lttxi*4LpZ2rc-C#v8~crN$>#!f=;x>)z|;`<yb z6LC+dK8b3|;&vVY`l<Q&=4j@sh0l{_2ji`^rM@M-pEltY@6z7o|DG3$gXmWSY%Rvc zZ$qY@;D?D@l{g>d)9aUC|8-`Cs;r=u+5M@;-Kt&FIP$B0!D3MzKIyxc$NqEasNAnV zD!<#n9+q0Y)qYjotSYKTZugzA+nU@mWD-`*cm_E17(5aY2a?%_Zy2fc@uwlJ^xO<K zg*A8&N2o?|9X}*T$5a5W$oAaf(YBWdFQmj*LB4pt_GGK_jDsm40uf=Nfn&%OCQn+} z?SDd$Pndhwr2vBGYb5%EenyK&C-3YcvQakgp-_S^&+d<ZCMfR+d(L+qJsFu6!F@f_ z;f$_ruC^?vFVk<f6l)n5%`91dvm^Y42!#RK$ld`<?xLU^yd5I}(E?u~@%Ca(xaWt= zl~MWq>4;$H{bGGViK!oFb6&m|w%2O@ZHFFnRhmjT5GjFx@b!ntI+j-CQ3Lf*f%`9| zx00Q;FRRkw0eU6&aN1RNwM*T}6Yx{d%Cn=jnT_jL#9yb$4y*<y=ANj#C=;+U<8`&t zF~`8eap=dbu|WSK<HqoKtt)Jdcl$yG@{sX61x@<$vQ(*~!rM@VehGhrY7LhPtxRD- zCF$!omMC9N9z<i{X4nOQOLt8;L=@QoE0KL&=E4bN>pB%Z+0IJLnJ4T8DrhjV+FyR5 zy+r9QmC#w$Px=LQ9{s*fBC|khkFm<@iu*Bg<@&J~yp(8d0e2{&_;-|M!__?YvQXTM zBO(-Q<^|*a1kQ1|^FD%XebThZ9J(JKPZVt?-1khPj#CaIndmej47<!+{nncyGfg#B z9GUk0o_8z6aUG35{W{b0b-IMXzT>FzDl4@*AK9cbc!PgERJWwys;s6qui-uCKyM$$ ztT4pNPUds<?ejiP#l?z76?QIhrPiJ9N_K9V6CbJr4`6P5zS$)17(aH-?dw9CwScHU zznRw!@&cgJ<GA1%UOSX1V!l^6$`BE1;R4tLl!rGvtl010Y(oy{!wz64{%m0vm!V6s z`C+bqyljt2)mSkItqBb+JqFLx|E|P*BEF@^g7zYlz;J*&#DYEChRB*m%z@+>5sEwh z7oC*S5d}6N4ld5_XGbNHX}Ly9mnQfQET!of9hH9^^febgJ1C-aJnt+(gy(RcEiGX8 z>jCKb0oUVZ_bMWR6M3sQhM!zb*HeY2fs1wlC9W}hd*JQhta&q>FYn|ks1_>XX$)mV zFyaA177?_Fm%=Fm2K4}uRk8Awv&nTGLIfkB{IIr>=7F=zj?H<#AnGwQAL-#r^$o2h zezP1ejjp>|<Z;3uANK7)DU}@2;T!u%^W^(wgM+urYy+RH^ozPm{X=Z#y32{swcG@q zm+i=?*%tv?6=8?9>k6;lUNjFBe_~)~*ld?7pU{2P-QYwgk(@F9`P{da?{8ffLUbw- z(@NUYW_BM(oJZ<cNgF&~F#HC(wjyQeBlqKyTJiYW@RI1Y*DaZ1QuB`wJszTJ9(mvq z`qPHH2I9lP$t0oJK)rCxTRx6VQUO^vHX2;M{d`ll`<nt=c1s<{bzYY;H<;wVr*%Bf zPV;!ugE>ynqex!rWSzIBLt1tVci2+9|JZE=WEGUH))OQM!Gr`Ac|jT9mS-j$M>Mt4 zGIjzV!OA_Uckls40u%`75V%f2>9g-`cj}m7yCdOpe9QzIlrS1h+*BVU7sR}m)YZ8< zyfi$^bBXKuaG@1u>ndq9-!N$(V~o`rd%0H2SS#~S(7dW{;Q(i8X%&(*dJ1w?C7p`g zNsiF|($s{DDxy*!sy+KIRXpULYHdza%uF-zzl5Ht&`N)Ssz*6iyzJ#^jBa)MBgfup z|Mms9Y66IkToIx0sk}Z!3-pocQQi%F&z3u5r9HZ88DtcaJCt|^{8&K+&VC*t0z_Gk zVE)k`q@%fc?4XDGsn97BYk?xq{n=e={`a_|o5Wv@OPd#N{*o2u;`iY!ol+^HUDEu^ z85hIk5TN*65+}it)Q@^4kA6{+1}bN;k0blrE|2<>C8{1iy6<)__0CY?{u^hu#>M8V zLhWFlxBad*xGjxRTaVwpjZr<{%oNY8vUdA?`aqecg;R~J(G@-M=rAu*?O0m$`DxoL zfk}zS-qKO7x!O?FJMkiSXS?{NzBAde8h`Ilo7-<|2Ki;cf=Lokq^b6K!xX;XoYMm^ z8OV1-Q-ntBU5RjIz>o<7<Q>2=4KJ2$_qPJykGU?fyy;o2lr@;CPy2GJXifgm5jO<+ z+nY25w^%?F>fi_OQZj_9eBB}ZDFE)lR(U;7ioZLa$7DR*l1n04>ll=?>tEL{=BKsf z5yH!wXG$H8XMerYmG_s|*|u$nSzFKwm3JnD!geQ3qUR<+q4iu4m5_MXlP|3}-LFVl zea*AtYp~IYeATk<a)84074BvuR?6mKa)6C}GctuDj~O_>3L@wO-pIG%=Bc}D7@v(9 zGoSxBQ*zGj1M07(^P|f@G+w=Wt>H~>xcu~|QJ>7IQwfJg9e5XQTDg~FSOJ#M)Y|ir zZ1-@tI?YdMUbgmseuZ6oIf=y--MI11qv$NTM9cTr)y0n(kY%~F;cONBpYag?!@Y;h zMX)cC=<_Ui{4dI0Ar0B+SXN=f3&h2;zpizHakXrXsM5&<s|>`PjPZj-b6wEMBVJs< z)!DeE1HRwQcMa+08+BN*8lPV|kx{ygM@|x&lU5Dcn_P2XH%Amy+~&3wag*oj%!id> zJ;ll`!cUxxUbHec)tSuxMqN)3?i%)77&nbzAwc2qiHtIzecH1j9P*%y*vST=KOSTf z_Lv{V>e+BWUAPndSgfIkpq>zdKIp}F7Cif<qZ`;c)B)2s#*W~X`8(J;oRb6W_u%J! z;5}_IEW9JXSyh9LLv9S*ULhjY-}_%lwnNpLX`ky7JSRru#SA+$w9kH!6M<r<CTcLd zr#pz*=%06N>V<;@h!BSV08Rk^H2VRwuKtBS99s8lt*jnMaxU=Y<fV5YNyyKk{(3;_ zzbVEKilC{OPjuMFR)yT{Dn+d8k}#USC&Pb-tCNSU@ELnS(b8Tk>jSnRt_#lGK$vHj zowJL`%6>5oLc5q2kRx#Z&$+8`qLleDtUOcweuq0pj}?@%fe^upbSvttKkV>gF%BZB z<%{|I(<*uc+q)<oMRENKmv&aZ|7WeinxtAyd}_W2Kg(#USk++-9>N&_1}vzYFV2&l z7k5K=Or4sM0vaeh!-sHAfD-!as2iqc^RW_+V3Tk8Li;#QBC7VecacZWFjpIO!czaQ z_cxoaEf2dE9fza*3y(XG#6?uSRrq+*u27}|emlqOxI*i3$5Pu$JGz#EveEdh@KQ~U z7~2@~Xkp2&B3o~%&83lyPPOc6SBDBC#>%I6w7AC1wB|QQvaeP%FGJg=4>>}wi<Zdf zX=$4@KG({<epS}pE%50RwhY8peVB5Haut?}kO!Oecfd0jji5JxPPxv-0wVLUM+ixr zHt!}7F5CtDWWjrvGaf4Dwz9NIV0kRP))70;fCcBP88cGI<Dc*<K7s;kwy>yT%JL{C zm+hx&#X^!d-#F-TAA_O3iOSOuj?J0@4f7e3jQNU{f*{T`=ua!_uwH~LV8cW|>@ws_ za<cs;I378bZ<)$bPNVhu%LFliWH(VURJRr|f>HW@l`?kpRjm0?sOZBw<7h4FaDv_M zVUj|G`<GJjSH?!~o-7)7WESQqKD}cIwR0{V@NYJ@9VruUEYeL%tgtH&X^;(c%yp}D zc-8Yu%TnG|^srw1aSiQ{#R#bP^Kao5(9rkB#RKQIA%6(&I(uz7<^_~|zk4&atOw*b zRGpfJH<s_@$0Ww}JGG<_4@-w*wBMie!3OyKF!^UX%0%y=V?vaJg`-_z;j@CTVHz!@ z(r`!Hv#oY6w%?;(Q$-E;wk2DK>0T>9nS^A8=)B<G)NHQH?Y5F}%a-xX#d!kX*q(Fd z_i*l<cGfx-5wD^op{?7_G3+-##6Fv&nP7P#cXRf{=xXWfCp(|1-S}vqAZ`ocQ_JwP z`Jyz3zwvL0O~h3s?}{ykR|7rBKLNn2vGQ|Y2i|W(s-}1r!Xd*hakh0?0N4{E+D%~e z4!ek6esQ&-)ePq^zmJ5J68Cq>bo$atP(#~DB=;gdUn)Wx$W08ablu?fQOqcF)m<>o zLMv|veu!Q{g)8AM0U%|Wq#`|n*GJ)xgL`0_u^VS=4~;8U|0L7|N^RIJt-;FHRaIE~ z9&c{ZKEqRf-tTqGQw?gD9}d=|D4^=17IA1yd4+P#X)=^*cu&=yWxswHPlqXn{ly#9 z!$$RN!k{>1W|7x@H!D8^Ew@1yb=Y@jL;5VJ$J#MRd3fu!fCRD==xKRwLyAyU<@KEX zV5$iB25_{}`cl83NWEup3SrN_GQOc)5H$s)dT@4r{!37A|Mv%ILK^FE=Wva6o>`QO zu3^eoyQPVa+KunQ#k0_({aWPeu`kc7-kl*6_3ohPLv0h+3(UP!`(Q1^l?Ti;5gO%C zWa@t@w@oMXb<}%j6$CE{K~6jMVPy3<zI5%RPskRC$1H^JyA%>F=LaP9V-~LyX$Z+- z7P@K5Y-Tg=hSZlL^G|>kCy(jWKe%I&AuW^u*<SGU4<vMPpbV(#ZW}0WF0<FSu$m=? z>%2%Hulmz%q-LgkYd!BoXj@=*>8TzOXtQnNI>3V~-S`6lX3lSaKUHNf8=9~I-HkP* z-o0vi7oF6h%?vv+V#($vRrISi9ZxHNd?g5d;QCe7)=Rdg6JLp|t-SrhkLG)ayG{u| z3wHuFmH0PAdA>B*TjbjTt-L?@XGZj=!M@`GGIH;C!@FG-7~VykXxHgQn93DDbNV&= zDb<&p=lHehKRIUcj~>oCB6~55w}<=GAHE=iD!shaZ{)5!gxMICa+9P*kA<rEP^jl` zJXH|zqwR+eQ4=_x=Lp7!2H{W(y-8<g9Sog+A*Pg^*@5pLU#H(mnj6euv=}6%>6`dm zPF5UF3+gkM?J8dLYose*r;qJO@ts(t;ZI|eJOmZZ;C)~XHhdXsFJaH|VL~t{7*!D+ zuSW!<r@p;W#Mecz;7a%pu)~4uRYosK5&)e}ZxD~y3ch151BB1l8TM=%ZZ>)^b;*u( zXv(gYL}T-sgfIDUpoG0S?@0wuMS23xtX27b;{J~4=XBsfkG~FE@Tq;cxkzi}lx@lF zYVPLg4_THGVP|Y2P!VAUb)%;~Tb{J#8lfr%C5(O|P%>i$j|9t8%a{|a@uhd%HgkKa zJ+IYEzuFbnu4O-a9fm1yu(MrRTa7V^%67KG9PYlDd-++4ZF#2Al|;i_uRu(?#`84$ zw`!T^cBs1T%De#CX-hNj+9`O63uI9q<+p(9sa-z=MZaS1^Ue#0w<JI2a71W#)y^<7 zrb4-Yji=3~aa_5{oJXI|e_G6M3bwBa#_wus&!_uE^cfG_Yx#LHa>hH<_q0*UWjoo^ zvVrf#LxiN~{$Y!0;Q6*E)f<bho;lWe&zBgP-c8Hvz%jC31^?hC*QO)z+6XxNd$1(h ztbB5I!h<)(`ascqh8*9tCGk9Bc=ak<x0$)fe6>5_gLB!N=v^h#D;f0mIV4|ybA<+e z)Q>QBHYgW{ZbKyI=P13Tcm(??;VPlW9uzZ>_$Lv2LuO^b)l3pfF;-&{yoeP7y_hg% zJ`=>vP<_X{sMJB*U?llN*u)+}((Eq4Yfk5cvd9~XBu<H$0l~u0w+6+;%J<+Uhfd;5 zm;)z7AE)Y{rJEN8iRT_q=@C==>6e4IOewKHWGR){Ye~OEJAbQ6sd+%G;*d72O1pe) z#32$ztYMy(eQEUQEL_rcighX?!R~fr-pTK8?GBQ&QpuTjGEpldHeW<j5Odje(QF%` zSP~Pu6Bnbh10*HDRe6&&x0**K&=uhcs9Hi#%p7Ak-)LAUheHAw77POKwLce0+68R( z+`SRbpxdw{5=XwaHa`xe@J>tHa#Dq24V?FmATrj)PcU9at-c^!gFYmXE0Xxq&+)^v zk{r^?CX-0_WZLu&Aek+x>WLkq9Cd4USJgaapFg0=GD}KQlXLmpUG4jQE3LD`KC0r7 zdpTjEO}kY#rFZ@`wj=iLFt+24x@<~~QJ0PJ<m7O=*3#hyw;E&pij&6n`)jN`(=^6! zmM&B{%2c_O>{Yi*R7sSR%5-%uO|0=~-u3!*=a##+TX-$lz@3K&X||y(2zsCz(+fSt z;65neMuVHv4yV=#%y1setM?%!LG0&2xYo{)!H<+O^QgRarA7G+M6WH{{8Wg_4tCQ0 z=j^W3XTjcyRYAv_o7bM|KHXhUM1@ul;=DssGd_V;in>sY!LejIt$J~^xhz2)A>=X7 z7eC?;2W*}T2hBw%$gntL`4t{}oe6(4pxQf(9sk*5F3;YdHSZ+l5L*{gd?vhd$-8KQ zq}`-7ZL#apvze_@D`TUlpAuCcYp_l$e7FuTI<7!nezI>r(xXVxHbzTwsK+h7`cvAu z(bD9qq@qQd)$&8a<+At^?P>*Ioqc*4`UTZ50lvo-NB8RtGb=^lp`HGcU3i+^f$Gl9 zik~!&gis8emEGq!#MOuw_(?jFFX!2VKMdr|5k<gwYtN<~2PX=}*BM{!P+}*sJX1Fa z@*yhwIzXl*SUbi^7D)|f4nliD!bYZ@@Y3YjJtsBHFC<?EOo^L3Z#;t`S}q`V!SIYW z#L6@~svjwV79Qruvg;Tt8FTAth77s2^|jiYYMc`VET3&i3SOlXxOvC>LgJuP<~8$M zfltp~*@pO5l2mzmyMEBU{;Z6-*p^53e|NiZbS&^8cc}K8rd>zqds+WF?0C+F36krw ztlM&*tZH?yNBWuaD~EnuUAo<BRHkGUBkOp{+3r-15%arinTE!@B8?agqv32O{4H2L z#)aWrfHB@4P)m|TK@ZZ`2=r>$!wNex8%-NbV(c(g<(<Z6FwZ__DiOp&VEZ$Y$gus` z)?T?SN-`O77;sU(uI#cl*F02o<7Qa?>{-1q;Q*sSe)TcPQl)i?05l%RFLYn%!0;R{ z<5bul?Q@Joq<EJS?r0-;box+e)a;3d2e#w*qry)ritN1JqTBFvypf;o@K_|R&Cle* z(9Qe>A@!n9O&VKR?(ARFJDz)@<@Cp29GFUKt}FKpwXMFWlQ^z!?I;_5xOld%L@P(# z=)Ch6H&nlbji|y8CY%3G>}+Acxsk%Es*1rC-ztk374)^|MFzXY?9K?IghvV8Qh4(v zysV(DYbOpoSQ(cNN}%>4uA!%?{Y0g%ll(sbXADmVkYE}1GN51#nV-!M;4Bne;k5d4 zR>^G{RQpc9zoIu7D?VXYB44HFRRBp2MIWmXx6MUx2AzN&+hmT>EwJgmgax&~Os5`7 z5<_y`)GMr{H8G6u5I#o``);m~hfJc9YC}ZNr*k%%Zk-$ue49$>&*4^aqwAJsJluk` z6g(ePz&{)P^*v^mKV!OEi+t)9H*2BlZBm(koskVOFW0+#UM^*{%CpY~=2D1n7Q4Do zCi5(t^DB6w&pz|;%Y|MK>QI&QkC~4;HRqfP{sD>PogjruWZSSvRAM|)ZcVt8e-Uia zo7yl5yKuoHe4WtwFiU3A!{)lPF_MVBHbMd%6SKGZJx?r1nkq6&U$~qT&uL7Ar7&I& z@()G`#Z`HdD2fCq2LW}DLvyi&Id$+1j1RERvY`fmftLw{l>z<pc1uF7B#`$-_Q4c? zA?YHX6Xl$V<j2hlH~9{CFgn&V>n_2=e>TU>5T2!IUMc8deNB6w*IHR+?fcfP!f2>U zi#%^^aIVTxD<Z2fP*kyasj@=B!g~3*f10tZ?DyduTkBa+N%%7D+&cqRRMl-OYaLr7 zf8WCTh}+;963hSb&ySGeD08JCA+?!8xelLdy3`KXhXvH&hbc-Ot|fSoNo*t~!9O_# zRpaggB`PH-=0J>RYp`D+J1HSj645Kv<LqZ22!`!%Hf?LT_^Qt}J3w|oQ$N$EWyAJv zip!$_P1_%Rnk}<$b*6s*nny^#<zWfk7N+iXpD!37r|gAdt*n2GOi6Oj#Wqo}12g^c zn^i6>6)Ao5S&&#Yov+I#JUqfqe0VJ(bA(o`$-#U$Q_!h9v?}~T5)(WH&rAAnk>F9} zi8;@BIQ&<v$s6OHQ%nBp<7%i5i+Q5_QeKqzX1eew@ciqSJMG~AufU<IL+Rrcns<J} zo8L|<zu2)hww34=mo6ga7$;Px8P1{!nN!hf_^>DQ$J1$}b=<RYtFo<(_MbPv<y<#9 z4~lnprf3x`8RMcpeMiS$&o82NwBAAuVs<i|LPefCL`aqp&e%srAJD7LahW?PCUeIR z`5XDeVM~;7?5nH`SJ>|m@4IB!k}J1x`7i0&&(u({;sy7_S*eq2)q$Boe_Da<GOf4^ zvOh_nX2hKj7d)xOL1Ae+rf9IQdl=`&k^C0lb&#(LxH2W}LC7xKrGd@3IJCyMs-48z z^yFWk8P8SRoiMXjR>^P*EJL+?N=C2O^*%;_tP6>}GU(#laQl%*TI)Fjh4ZIv-??U& zDfCG>x@2djguOu7rP&RKmx-P&J=gV4hV{+mHu^%(fkzeR&X(@AdgowuHdfm|7L2oC zr7kHv0xGlB<8iX{4kEp?CjeWPHexT|en>y{OWUZaG%L)~x{)KBo6OoPbG`pkp5<t* zZsyr9)H@lUCKjN51VC#&VU=<fp2{E--#CdZ-vSHl2u^A@qsgO={eYXy=2o$PdI0vE z_x<3pKIFZX8(oIMTh~jqzYCQ{N;%_?Ykl9BFP*U<T$PhBv(qd)W5J>{)~9j4aRb4K z+QRo^!&VQ?BNK{^#tCSrsfZqeqCP0L!l?q$ET;tj{S<)6gD<rp_v4&j*nlR<;w(7& z^GI_Dr{#eG?X&R{`{h_UrR0-0zDO!xv6oex<m)QRVQ*Xu=_|U%`u?%w7K~1wKU76` z`<8be6JTV0sp_P)q-%lu@i47R+{>nZ(E~P8@j2!paJ!U)eN;G6-|zMDeGc|stq3me zPd`LdKB7aOgpn=!oO{3u6RTWDotkzs@v#Vxd~D*=ZSanIv^9BQZ!ODEL*b=$AM|H9 zlrPgH6xT$ogjWBX-hL??a*aNjOY)6@j5@3nD0@zyl?O=hzK0uNF5&-0h&MN4Hxv6M zU0DY8d2rf4m{9v<{Zu3Vv0TscC51wJ{l~q8GI8ORRm5E8-6$HIo+K2vV*1(J0q@!E zw0w9!IuUF!8?L~3*5(Zy>*cwMIP>E;yAjWRbFEY=C`N)eH_bW8hvXys(BcfJ)KpjA z*-APhC5*|l>Of!rKiGS(sHVENUla=>MWgf{6)Do9(iAXxEl3kldJ&?cLO_fNNR*H$ zNH0-PP!u8{y-61YA_?V1M5GG{Awfl&tf<5#DgQJ5&e<2=7<-JfZ_Xa)Vqb_32EnYA zHRpVuUwb}q*u_c3k3WRX9cH!NTa0YHW)g7mdd9<jgE#ISaMe5iB_(Tg@$G?U*MG-e zwR~8JUY5NcN_R;Omvdf{_0!htMj_lXwH`K?b)7C0Rdsn81ngT*FXU|dhe{PEG3QC2 zsE`vtp7|Pr7xl=AP}XWUa-x{)KU&$o>Dzxq)LZ|Sg$2-f3VP3?y4KTfU(GS%0qE!K z?+$7-6=4~X7vy`0Q`rAIc5Pv;JlV6F=}>FB5L7hnd(MgiAS)bZ63HH4k>eSWJ1l4c z<jLc<pkg_|*B>Vb`3s(6u3=-dtrI{-$$Q~e&Tj@X$tf^*3-r6wJC7aEiec-tODv-I z4O#9inQ$yIDE!j+XfAo$G_rx*XvcisGQN!A{G3}50JBlxfFjWgSp$M6{8&u-v^@*a ziAn(O)-RNM=!3)q9O~WhBezU^{OGrPV$g=L!%BGd_H&Dww&e>YZYuo}tDd={?IV^6 z*YqN};APPvi(~~r2~o-QsO1NkZl*fX8?NMJS};|QbBeilIWkyVRCguD?SrNBtl**p zvLk!5vOi-$#1oFElhosmC@{P<+fobfN#Cn^-JJc^yJCqIR7U7YDG(GmiYCQrd@L%K zx@?IKRz0=<*TbUb(u)Tl>5B%qdf0`Q8rprl>S)P1&KPC9t-?xp#NT~%+m{|b;-Wh{ z?`@c?+^G3uI?_%l1-Sp5<PI36_=%ltSKsLo`ZJytL7$dkId-nXy5wa&o)W>RgIbS& zLlzUhqe@@`{(3M(&>BlS`2H{os^*rDX7w21$YP`X&}!c<xwuNU<0e?6%XHmph4}y% zmV3MRN=lV8*9{Kmo#opIK=2M~3h@uD&AZkxe-Et!^Jc48x;P44_Q)^PJ+JStgE<<E z7E+Dh1;~E()MQQ?wGft}Gi8aUzavnrYQ%yUsX+bprK_i#_01F-X&@a%gd|3^gvFsH zYlw1(Bu7NT_8}4e#f$BS2HVA91;)%q=t|{WY5kH1-?nOMp<Vq${aPDtO>ur-Xy~;j zC#$;m8)^%B(R;xWi1EY6wrip{jiZkcFr73RNt*}!J+Q5O6K8l%f2f;|(Z$@ui2kZ1 zcg9Y-xj2QrshAJA8k4q2@Va$VLo%Ur&M&bvtLM>ahe*?rn)=Uqx>?x;mhuPA?`1EH zmB)9?zF2gAWN6*13aAGuS91?Dhbxvt*T1x~Rszn??X%loujzB~B}Gk4XCzkNO<Vd; zVE=F>@-fK*Y%k^{GtwZnp*<G(tts3Qj^i7F28(vDH5?jXosQ?B`1_&X(Ij0@{vkNU z(kO!6rrm>85NZXH4C^>@%j>kbm9NqENC{l<`7`$Z`p$JWwC7%wH<;M<za{!Mz9Yv! zeyR(!e;Wg4v<A^i{FCF=Ra0H&#_DE+#%-Ir$IYa1=~G^dX4#F@f{%ii{egq4CnE{7 zhHv02S)3;BN4Tc}b7d9QcI=A6*tKj?!6srFiz&o?JPm@@zRPUA9!OO_cR!rT9W2<J z4xF1uOsyv`cxq(m=TZtubS88Ll{9q)6xO^5!tq(Rm@p}KDuQ-_b~jtCIN?J5p@ZK2 zfE(dHqjn!;enQV54=;jM7#SyP%ISWOnzXZ2nc980lX2?T%8Ip3`ud&hZal)-*;P!h z9%k#kcgwA<JlnAPdR=c{e@%l=;7b=(_Uu$(_VtJRhC{9wHTb@LWKR0!9k$TDt-d01 z>~-}wU-=;v|0p{+$QJs3xA3294Gy6`BNe|=H5?k;T!Nko?9XH$x;iVhDEaSV^Le9m zJ@!7Z$X`%vXI_^x3D{Y;hPl;AAo#i+#qj(D`*BTS?>X9}n}tQ^?kkPsmJ!hKiB^aj zGii3&6MK%c<%#U6sY?%Rc16X^uEHAg%Q6Ll>k-LlZKC!l*pQe=TwUd7b#p--X%{@l zEg9W4;^$@)uBO5uE3WlBGjgSxQ&o0U-OXD#({blQ(m^Baix&0Re1gC2@Gr!+Z|m;y zzYMe{^nbM}93NgW&e3+Lr=WeRs6`)-tD4^I%V^NBI-4tc>k(X3-om(0AUYG=E9iTK zYRi4JU8$&ob0`a=qbc|MHS@bQ`$|?QXqGW>e?9+zAgx`V^iEy1c~1~oNjOPXHJ*;_ ze(<rk)2IWI;rdWIp=qDm7MrmaN~gh=fM{c<dyiMJHb5~x`-G~7-V@v`D8tO=Bh`rx zK^o%M&}yV!v?5f!jE9&;p&<c8+x&huIq}Au@&tA|GNCcJ;Z)oK4L<dyta~!n>|KbN z>j9?}tW6|<eKqUIrUMt_T(elqthkgHEqzQ<``XG6;Cq;*@@1^K+XcC6a4QZP=AMTy zQg^ZmFOUkYk;Pcz1$bib%bKusaw=9~ZCgZ_`?jy?^Q4z=Xzx$XZsldS#?K*FM1k?^ zCdA^z_pGL3CD7so7jZi{87TTt;(W`&!u@xKtnnpqom+j<*9*-eX{P>~qsKDOBj(3| zs92N?BiK1(v|3sRb+Ul_q5b}2{WWSdR@4N)mAE&X1IHli#a3jZW+|H&Z{Jil%bc?r zH}O+7RJ-`-ZTF}B#cuU~aCFxz>w7;lr*B4_4qG_+^BL}Fc(4_!{a=A!K5bj8Gw%es z154UJZBWdh#W8EJ%Z{^4OChrDd*B>44IDp>+TE!Z@6ylU4fx`-kq8ljwe6_%ft@>g z+4vok8*L80RryT<{=e-2>Y%ld0dD_WP)o+?RQ+hFW~@pOPc#YA(OYJwL@e9T*mz`Z zB_$5kkZR&V91J4u9wSaC_~Xu5^e%q*H`JVajq);i>FkgHh<q*sUwR!4+izu^rLDHr zXW1CyyCyMz4)D5yI5I53?^JxdiYH;dW1PA(tY1MWeZ>BaX}i|oHOru4IC90U5i>PD zBbNj-y#i>#?X~eMPXD~T)%3vnEB?U-ez4%R<tg%2>ST2X_T{w{hu?3o6O&7<-z(a@ z>z}H6jcE#S8_JJ^U9RC71a4jKE%EjZ%60%wCb<b)nO%denyjG10~7pHZ%Gn@=V*Dp zE;yo{fpOSVP))nYoIoCX>H+nnlC((!q$BVJ$eEsfEpZN&I)-f!W2D-jg8ho&@vcES zo_J!OTMQwWe@eCF)`diB#^yFeg;ARA?AJAUdx=}eNSlaSrf4)rl$M4zC!XdwCXkFt zLm25`R{B=qZXy9r4v{aPs|<3C#%A}>Gf!tKBjfd7`Is?$Wd?tZs0437$8FkEE;W#` z6E{dN>942iPmLXLpAOX~>r8zAEBX%YYlc9XX8%wc?@`FHooWf3!B;-{wMM!D4{$Lc zCM$w1meg)4d?P#p6*GJydb|M5od_ff@2GOYCT;dyAGn8IdHMpnbS~J$(s<q#=oj>7 z{`FZU9%X$){pxq~<K{%tTe_A^z6q4`+us-JxY|DnQge1FqK1u?_g@`LHOc$2qf|He z$6{fDgOABXF&ysbGCacn62tutKbh4@XK>R6(X=3Dy!j`MzE|VutpxdTZa3^-e)>hv zJOS$_$i;^{%8~thb(>QgRz+b_c<O`srg(ihAfXYpfOirrL3joNh%Aqc?sAsT%ZG!6 z;tRZhNuO`)xmzh#V|dU`*xL{PYCBqFJI6Fx^wRLQx!jy8dsv{5U&kfHz6cSo`VoDD z$DTv-9}efnjDEmzQzj|Xn%e&n$w1zxsROk?47Tmg)flo2K+7i-xJO?Ks8#hy5dx*N zU4f&p;>}(a3&JY#gG+GPR)Uq{IFdYH1hy@QvrYkJ>TT`<$lRF2?wnu*M_Njj5H51J zWSR(q!6S8nZy?BQHQ}KMs5$@Tj&k@lTZ_tb{)U!^hv#S%q`c)n!h?Uqg*sb`?gsg* zf%G8I2fa(=l6QQAy3qG6k6hkD&4+y*%$ID2!cq_F6gHJN{6H<Es?C-Yd47BkLE_rs zZCP*rBQhC>RU?ffYyKnh_{42&ENM@BaWI*O1&%?d)twV>i1t+rAX`91(a$OfuMW5E z#%i1hrkQj;Hvb;0Z&zvl^2Lhr%H{qSmmWIVW5ZvYm5r<$s&}1rU1Uuw4&_)88X{lg z<)?xd%0>;;<G;(vp2aFg5&kEl>HigB_5Uw^#+^l>tg(1!$sMO#-7Bqj(_rAn-OPeE zQ@8T>e>;_ZrI_a<9Bfwu*>apnx99+sw;<0^gNLRu47O`s90<a64@gfO?9n}?w>Cdl z-Hi|z4p27>QT$t+&tI7xSkZPxfOJ(SYr5NfR#$_`tW@VNFZ!GJTg?nE6rQe;k{&WI z*?b={H^7&fV^t@&s|tJnBO<km*6zfxZW7P?khv+aAY!k9+6uD=F{ywZ0hS`0rnXz+ zw`g_YOnLhPuT@OTFyF*KSkdNun(=QqVrz}!nwuR&ufp?gEg;A2`P#9aidn}+m-AOF z?Y{u|zFVze5C26}F>wBV)29EAgfT^u_MlAXj0y)4_oZzl>?8kZ-s0rm?6e0_;YERC z;$<Jtc^vuvq|~Edb)A$a{r*xM;gapebdz*5_cILG5qd|hBgOCz5RD*{;tqs5L0c$s z9P;NvAJAe%53m3-7D=1n_##-01%e+ol7`41v(QGMUvpa;wBJ$@4sv9A{UyO}AJJHi zO3@$Vs_00`AXg8zZ-mU2MQ05*f#uLt?`t~pi=2qvNCWO(g#ceOj{BROppF{P1z$PN zX6AO|0TH6ZYx2&X>bEm{02`I&BV#>s)LzXr%J6$<pVV)eSGgF&D&^KtJI^%rM~7QY zbX|kB@^oK4lgEfB(a*TH<=8#G9gb@a({(ErztQactaK#D+zy?KlskHP4SG~~=xw35 zRnJVyK6l2&>u_R=bYSM5vk#$BSLND!{>nzhD;mlfv5wtLc;EDw_Bya>?T4Be1C#b3 z&<l|OK<^&ziKYxLp5wmqhhx!&-XzE*%5k<M6yfOMx!lq`m}~<TzJD^eN^v*ek~gbW zoAmP>G`$j+QCyD8u6On|JR(C^-w1yx0ohByb52J5eN<#y9e6hw&~a{3)*Em_)j)wm z^V}%7Y0h!t;8dsiGujTJn($wyvQU>G{Q7m(rY?bCeXN#qGIdT-YwQ<xORt|_)zOkE zrHtR04sso;qgsqF4bl{UN3!(Di5Mxga=@fvEGt-JPLGltaeo31IdNMyYdwo1&Gk9? z<Goyxg=eAW3Oo8x-hm!v-QU6koK-pyA^FM(nQ+XU;|AhB$TS%*SIiwxAzHCkHxV3d z8ye%U$}=a;r@vI@#J1k#T#&h~`+^{%Gn<#`Oz%;$9p^su_YE&VTPe8d8Rh!f9V&AM zR;c2wqm?nw4y!I+&~wROqNmKCbhhlS^pD9d=yiLfX?J%1S=-ZF<Plv<6OTyUUtdh~ zl2abC^OA}s-ajlqDYs9kM0$sT9570RK373I(SkER8<o|8{K(+0LhcM){p#K9fu1OT zHf3cWG{?p;mMo~-FqBqzwsTN5ZU;Q?)>uDid15?Mh~jK+xl)+FaNTbFNAlO06=Vz< zawxUb<6vi*J(NiqZY9j)x$(~np0PNl@eq(6n5l1uBMV@K4w61JJ$`QupfZcO@-T>k zJji6rmzOL^f6Q_A9RECGqSBWt{XXDxgZSPXzi(oovjfr89bY4n6ZkjSddx~)NyWf5 zL)v9bw66hYoRPQxN}A>?!E?*A^6}72zq-ryO0yK1go%GYnjudNwQ7RL?;<=<C)^1Z zAF@tt*_G87^QG}M?-<{H41g!JWVaf}o=NvvCWtFDOlA=b^C)|PDCB0U;i%4FoHlAj zifZ^soFQF^m9&#B&^+w=G*$bo)3yE%7Z6f;%2HQT+<M!-3&Sp#b&d!NpZbkn`eq&$ zBK`2$Ek8SPm%@{eJSt_sNG!(cJ{ox@XbUJ?%dsoZi3-PJ_(6hl@)hy~eNvA?L8`aQ z5VbPDSqpLs;kh1P^$)E_D{YN7M^7<?O0Q|kl!WR_H#T4QnEd3ho23!hTuAQNXHu<x z^_8F!WS<O74$plGMWBGt1*+>(_Fu+9SH6xmXLp1-PSBaBI5}(gQidvO7H80-t^Yng z@wvpJy$9aF+am17)b{r+lIwmi_d@<GO5!!i#=N;o$v5i&e3V{m)&x2mHY#zwp=}_6 zXnW&d596lwoOkyQ^xX*kGR`gwT-p|U<L6UAeQTz!{zv4l#$X=9L@ifCy5P%ZCVS<N z5h!=<1$D#-PUsF{FBY&~WS$ixY{M`yvWjn-%=LuzLKHZQk{l$r5*#I=)d78R8|N9> z+6hu!TN{%|;=fW+-0R?DE9YOF(oSjD750*A%;K&!z7HeElH@qmE3urUnQ2aAZ)+4s zi73WGAp*-%y4}@xa7@a26+N;8erk}$27L{P{}C~&j&yU`vwqZ@6Oko6b^F&r%a+uq zFP3b>pVEYOmB=^z>>=808ncd?c+*EXLIWIrAY#h@%nj|9VIbj7o&1^ul?62^HDjOw z>?Q-E%B*CcyyoDp5lfR2Mb;i8jhy$)JOE|3Zzk>#Q2EBWQUK$RVBtIPTf6s6u_&?X zB2YDBdei<=o1A)zIL__9<z3I0{Yv=9&ptm}Zy+K$#X^Y)-X`LP@h=dz6Ez7WhvS_I z*)HHbI1au4=?vKD(l3e8iUoi#!#a%f&%D39TS2I~+w;fs&#JeaOF>IicG}h7u(H|g zXM}&bqH}@Dv7LpYgPOAKOB$;ZdK@PBRnbp`A22(AdxMZ1U`0wZpMLRzfd_Y=&WQ(9 zr{6$)(r}3A4}#rSggoE&RihR9Qq4Jd)9D*BZ>2RF-QPDjm<c6raS9^%2KcGtte)jT zR$u;wX2^Qk7BaJb%?b3t9Gv<@j1-h;aza{K&o4)>+CygXMTuuG#rHj2^<7jt>6chn z9Q*L%`oJgO_O++JiO)=ao}(ecEl*w6976Mc{l?>SC{vC*U<zYKdN3TZ6R!3d{A=IX zGhMaj6WM8An3kNskeTpLWX^Gyk>Vstz3Vo8y?aUqv}mTSK47KC3ho(CyPtt*baEnT zub`42T6~I|K``U)hw*ciNm2klzlVCc9A>h7lcQLetqw}(HG?BM&Ci`Rb@SYLchhG* z6s$F{Tv`_7rfvX_v#83Xv6;ieRia;-QfODni8GdhqVJXXj6Ev-GIXlk+VpV-X?wc9 z@Qd*g#HqNA8{+ovQs$AZ292@X7TnAg;=@u={|p-cD?jcq_{#@9Y(G@bb9vtvq=xQR zWYyof#QHCUA<}~+U$)FSRMK%X`UaWU?8JLAH_i?O%PTH4>g$O6rz++QWz#6;8WPsQ zcBKvy#RT!hO+0jfnMQvdXLY5G^V+VR<KmY3U~?}ILwYC44r_fCe(C-cmVf+1QR0~% z9a~$MpE>Tc?kf~dGLc$O<;%LyIdU%#tZKs{Ag|Do&a9cga}ig2&A_?a+y&4h*-+U7 zRVOQs^k8G@{qDV2t0za?8;-~Scq@M`qgrzelpWFaq+f)!pYOUGhRRC1K#-I0)GDg; zk0gt{jNqvgcX71&+P*tsu~`5_B%T_yMRg4%)C7_h(MIQbB^7K9+Yb=EN5W|`=V;mX zIqRi;IiIhIP^ts++ir)I4p9trLyFbGi<&)t%sUl&hPpg0)gE(b`u#FLN%!CX#!<p# z?m9$G8R*DSnOmj%7zShKvX*fiNd~!rO!v_p*hgmfQDdP+J&qacob-uyKkGScwDLeF zbL->L4@yM;%@uFxFQPiEeH$L#-rsq){8)agZdVt7w0g10_e#C)``AF&ycY|$w#1y3 zZ(RQgnj}dRBvG0%F!59cp<2)mLOc+Mb^+19GaGexZ3(}oJbsTDl;;6wFj`(5*A8@i z5$KkF=N8sX?0(Se>VJ{&UbmM_M5__cS=gCXwF_B8ThU+#p<>r@p}QIdPsjn0HZ<U2 z%H%rj-~6q`xZQ}e8ohk~^Kk?nQ&%2*Pgh8D{~}jvm+JNKN~O6|2&r^}T0B&$JW@Iu zS5q#h`E8V*q*Kwq@xN%lztrFV&OdynAfGl{h~5foaC)2tPtmHR_tXY$uuaqGnC<## z1O8c1#n#MHMc@a*9kAHBmi9U*2Q->M!gm_ZVL9jycbb20ajTj%jJXg=uEus0I)r84 zm{Pp_^eJb6FYDc{eo&Cwu1uOBzeFw9u(4Aa^cnN!(Q8EO*#sd9q=!8vdO{2aNnx0O zCfB^3lK}Tze}Iq;K<|e-*SD8G_rBHg(EW$ap#BH%*3^_5*$Xk{lt)&oYGSyN1=+HZ z0^Rh*(qom%A9Aj=Jv!`ER(yzec*^xyd8ZRy7r$hZ{yE<Buv2Wn(J*eqK9^#r96g@h z{Zet~dk@*nMNPRM#mg4FGZUmqW-aCoV)C^E6g<wDLQs&~>nRCtxf1PTERNcI-0bLN zi29Szs-r~{Ma2j84<D?o(6v9Wt%NRyR@sgTKnPDx_S#YdBsCaDlm)9gbrzz;6Fui* zp_JQ*E`h1z_66k;p&1;OtHyE5BM|lEgoUBSS)-L<eOb-Jn)|N3Rq^+^ZF2o%8Laz( zOL%5hKcALrJ>e3&uj(?vwAg8%m2I;j&g#&y8x&0`E7j8~h?G1GF0QM@{Dy7SWv6Fl z9TSCnpY7JZwKt>Qr%<PON!Ho^d5F^G?Qh?gc;GJ`D_d4LtSk9k`?;af$eHgCN+T|x z@U2*uJ9&DG$FuD={3VSGBkmMF3H}NA4=ij1*o(U`lkE@T>DYThl@<=3LOiRU$VNz` z6Ir}{d(yuC2}4ONM6{%pxXvfsa?8wk7jfBdm+Ea2r5v{|d{7wFata*ALqKKG<2G#A zTKb%Knp)~@b;}>~AXM3QI}cY7oW7^!JEQF7t`mXB!u6c)jg(yghO;Ex+np7QWrnyQ zIoA(WrOloWgJlI|e+%i;x=+);xHkVm6`R#@NyAyRI&OXQ3JHsooXRyD%unK*3}a^7 zkRmUjnQD$-7q%9&@&^I{yD!A-TvZ_Ifwtu?8x@0;YM@Ol(X*?gNxK(mk|v;HB2do2 zA)G#6j%n5`p%q^+?b%)=RQ&P$%E{N>*LwoWoC(<;r^Cv}Qi^4Noj#DG>+sXUiu9vR zaqs76#ZCv3g451E#M#mG!sc`sCq94rqMUIS#1Lz->|RvL3O?Gl?c_bOa%rH|w$;^- zT>ZXMRe_pqY_4sKCVXjl-PGnW^F9(cnA9%|s9jU1<B1B_Zs-o~K6+)B&R_YZ$U$4} z=Uq6Rx7*LAhdlIlIHDXBHat;P^7yS$iof{2Zk+LO3vMwx=w$XpWmo<KPtW7~q$)P) z?GyF1`FB^^^CY+cV7^a^_<IC(fC&01#BP`-MDw`qgS3DgUq(-(aBovhErjG7&EE?{ z$YCdE;HdgrRBVQm@}hU1eV=2kNx4PERJ7REw}pWo1ii7!mBW4wf=9})lCs+r43tf@ z%iX_xanDG)l$~n+$oJub=00?9uN8XLQ?qcUX~Jj)M3KE+O=Ci?j@$Suf<ls(un&z8 zB-5_ZCdq8E4rV-UyHJa0Gy2zE9>?4Qvi9*(a1c9%q>)9pH{)wzk(Niw7nSS#SVss^ znZq`((ofdx3+Be}J}sx`RH(bZ)I`kJDFvV3y3eK3qA#G^xYXrYp;msumMN<eyEEly zPjd3}KlApv#Of@(zESG5Bgy@mldaYSkl_(|JVVk5a?9Vy`MW|%K^{o!O<L-tZnwm% zm#d{r7NwREf;yFftaQbf$7tUN0av0)<<#*A7E>1It1+}@A<45(7|nlGb0byXK(#hX zBrRRH<zS0>c|pMz_md*!dYj^$w(d&XYBK2bY~i%aq80u6H0FqVS&++EYG74L-l37C zXKL*4Ek)TkLEu=SeD*`H(+0*vxV#Z(*TD{*x3fmObvy2aT}*os!w_@Yy0~@M&(kIP zIz@H)B0i5cT>8f|I{L@{6K8C{Jel}M&Gx*Q*2DYH>vqU)wB0KrqO>>T!^J!DNrC1z zW#8U@TdV>z9DEcvR#a?Pve^}VXxDfGX#iVK?x0N;uzrCiTqsIt;0MhH;gtbj#rm{9 zl8XeXzh1z$f#Q&=XB|}*Fca+{%H-NdPq_Hls6j0KO>mXftK2c%Ea3}oKBjzGcDQ<Z z-aUPZI^mq78(P{&tBbY!g?oSVOUjZ{a-i$3$HOTull7V=4rX=F3J!ORue<~6V<aM^ z_^@U`Z*>T$5AFcr%YU0<#7_m~N=G0D%I0eca>$qTZZwH;hXv(AIq<HEs45%6Lm(rT zi}~t`8E3}yQ-_S*2yqjL-eb6@usR$Qa9dMzFZDFIgDe*S&+zpJU_Cs&TKp5TA%I!y zzkC4bToP-JD*1_Kbs{Hub81bDX-TKnOBUq%O1>CxKmTY`qxGMHCQj7U9r<vBwIflh zQXgC(ql)o1Upz<@qqNY}@vSgPqjKdyO+NMYF-edxmemSeme^O|O4g0$=}CX8+02z> zo*_|kY!}qig-WAsCz{L!jfoc#&62TA97LDTwPU&GlNn5j*8M1=HCw6i{Bq5(p~_cZ z^yLAyBH;&WJ+>>VlxIwO!>o@_UDiI_0LSu<AI%qblck8o|28!@*T5rlY@;cg{fk`e zw(6|6e|kj=Y!4q$<jczonKEFYMjOD)?v(dI<|XGt2u~xWEEl!M#wfzgE67fs{PUKU z#@fbLt!=NOhla+K3=ONQiqd8#Cud|2IvnZz@zUdh&FGUS39b^hyCfh3BknQqkx+;( z6MxfB>t^xG4S9AUAuP;<1<PQd8<TLo2em#DGdI4XCmj&HH(+Bo6DmEE<+tyuXv%GL zJisMMf|P2*;BYJ@+r0}jb-dHG&I$uu#aHpkz+gDVGRs!*JUa`kMBeg;nAU>zCtHX4 zDpw>;x9XOZv)PBJVy72l_pvpnc@KEg;Ih(lySA%zy?@{4amFW$hHCZ#q|A?bKoiXt zU~1)`uq~JyfigL&XGM&o+xf>G2|l9B7*UwXErRa`-|o71Q<)YIED4cVyyUq9Cp8=5 zhi4P5pI8oB{TwUGnZv_61Km-j5Q@7sX7>Bt+hQNvHH6bx70Z3l8_ib(PfEP_a<F7D zum+E(+<ckT=aogz+U|L3An--DpagQf)4B~BW@M@(;LzCtad;Em%jp&Xv|D9Z(@1)| z>aZWZs&AXjvAXEVe$%oPYv~l7UsbiOUVQt<t}QR};**z~3xB^|`kr(oN!8+RvA^rU zPLx?6vc;54d;(7w{^idxLChqJYZx38)+tUQniTp3Xa8hnEgM0U4xsONAG34whN5ZU z`zZ|PNOre8dg(Cry*|g0{`2iXvtQF>x$keNNhodxJ{2D9$<}jJ(_a3<q9!9{Ru1Ts z6=*mgzb<lvLf5@E!`l9mkNkgtZEsaN(0tpc3`4H+)v>dN><3n=9!J*=&LKO=NmMEH zcKFI_H%$Sp0V7!cF;|VatMYtn4mrMcC+n%ch7D=Fn(*{&Lqp@Aom2CV2u`;*`5IgW znt)Z`)<vN9a;Q$9AU;?cq_bWwj_91sf-h!}sv~~-he++8Ixg4NdiE%O^I68`9o;p@ zr4H!W<R|Uj;FoyTMfZ+~goublI!!yA2I~(jN0#-66qEWVBYG%PG2YRhyAF{4#cbyL za5Ua@yWUJXpxraDvnP*l(rEoe&GiK*{XZh*{=0U4B96NVDlxOA7=@-Th{p93WP{F{ ziFkJk$EHk1vWuE{C;5&^O0f3>$eDX97|CdixZd+>g`A}DpuW(h;AG`%J4LB0S~HAo z=80<!K<Iea!kV+E-9kK$*3^0IsSoNFc~*B7!UX97&kBXsh+Hn@>3y^=Aq@geq<;rF z+BQ_Wuh@fc1E;d^l0n(b$6b43811SfYsMp%wviff9xKZRRsIi4cXoH~#AdV<FBfoD z*}?@(Ao%n#E8ZO6?zsaqK&XYR0&<obJbhmURcr2)DW<NWkviw&c=};|qW<kBO2aYe zRqW%E!@sJdnKh2fSv(77UnO5zkSmlYXmmw5^`Z;rmK-G(){8sauCQGvG`ndWGhch! z4np*oti$QQopRoP?IZS}N4G0>kJMGWDTb{RZ(udS|M|Ixf=%2X_PxaUQ3P|AGP8=} zKU@xXg!PK=0Vn-9_oUzfSm>z97W+Hy_MA!@y#RQxrC(kCkBH|@rYtRR#kyml;@jKf zlm<em4QK1aB7=fwI~<=a(&T8fm00BnopIVF)JH%1_?>?Zh`aL%Q`~-d+#fM*qtGQZ zb`*il80$|&pY%daNd^U%T~0oUFp3JY0zJC&8>hO8`GoZEfBb%*99Ye;yv?^tD-E}` zMj$o&QSI*7u(OaBV7E65Kz^9RPa!}T>&<}A?Q(xO4{S}_iC5t93WDS8m#&c~yyIks zC24@0D+L*Rd9ZDd^_GI?f#`Dw72l<+Hm_H++&d64$c^=)P<0399&H<NO<g|m&o`u0 z>oyj3<HGg}PBpEQ<hZatkD`ZL@HwN8v-YOjYLDntUx|~EjC0G2k42ftQv1IPul~@x zSaT@;_S(FAZ*Rj7Z&W&Ho?E2;-`YY1B~0xMUmEPuzzM5v&=iVi4NUog5YR&20r>4M zQh#s+$DSJzxjKoNHO(_Z^+u3r2ouy~{)pH{XVsUwHoXd2*k)HgAM7qC@dJ0-ckuTL z3JjiJ$>V7W0q@W3Sv5j?V7Gb_eWBkBOv;0XH=8GWP;bbIRM|Y-{(roqRb>sZA52KM zI$e&WT|QhGz}8UW3l-q*)e+vO=%$7~mi+f`c4z(&?V(3m&(B)PvDE*@BmM6t(l)@7 z;DEP=Z}HIN4nOj2Etu9tRwZCqU~)qQomzth@up_9oqXGT_~bwjQmj=BGV_EKQuv;% zHup}2msX_#*$OGfIj09)etb$rpR!z1y;0sY4rUS10x1(nHjruXfW}yzB#9%b!=jJ^ zJ1vF-5z)!rUSiQC@de@G(OoT^?x*GOIAc;Xv#7BphnmokqwPLgfp={mEuZu?EZ$3| zeImVqSs*^^9kPCeq?XLRFL>6znNOzB6`W^aCrm0<+;S7g3P_{EzeH6<Z6M;GW4G<G zy|*JVm6r82;i2s-osD=PqODzB?&r*R_$tFeoIdVNSdK$w4MeZ%ha<%Zo7kYm7a>|f z?)TQVq3l{d)K9H$Be-`7k*zf!sttZwOidlqY>3v58~W}|n7HuZ12hNKzPoYzPq=4U zc;VeRZ}G0ka)hF;`dGy8X>9orqu9aLDTm-%srAUq_+=Y`Xverxf8f{b7dyoykJ6Kl zGu-+Ty83&RYp?m+LMx}YVWp=P33NM|QuiMj!mY2Pz^xkK?^NKPhMhT^umJo+iXtaA z*+Na-AfGibS7*>rKa451G<oxN5gc#F5mb0(x8E)NgtK-^A~!s}e$n&B8@-*gOz6Z@ z<b?k$e#HTRi;=J#@5mx{D^UjGGP!;5ErkwRyt)L(J^uZZ$O6>zAjCU6Di&@0tpRsw z%y@0=srRkyk#`T?W|YhQ_=s9DV^BYaa&=(~5PHy#fUS8*VINh}Qj-&NR9zl!V3Fhh z&WfOuHxnY*lsNfums%kAwHmZEV(EF~RRZ?(og2%Ozx~YJjcsso5rewinfC`IonRqH z#K%pu)oI44L>g#9E>J)IWT85#lD)UUT#8-%gMugR5-_4iT`lYE@b_P-$9yEaa^)b0 zLSe?<byvY!ML#}7DRYJ!aIXG_c~ebNwW?dh)348l8N=_dH+*wB(`4z7Rf+E|4f@`H z;Y&O>?A+)|LM7URFT*K0KZDfl{qbsqTh|-pTx#d!y9^YTSJlf_I>y33E&gEI$+~`? zX7L?~`yp*W3Hh-URwQ|62z_2)U%5%#C1~es@Xqm1W^^(e<5x7n4Hkiip$P6v!9(F* z0%FdzIFE>dol*4H_=$C%9RLs>x9)rK@f!&5TUpD<ho7Dt<^OqZHh%DU5HAL`Oa{>B zBnI-N!8+p%=HzSE%p*=#$8GZRao$#D|7@ze59bsX@x9fyw3kC4a+Jly{mQvAHtsrJ zk*Pvib~wGLgsQ(#GHqYj?{6^8=;al&@gL{7R-EeYZF<~$oUG+Ko;gT`ory<33D4(W z<)At+ag(cEm0GJ-9&>}&eyt|7P~!?mm%lF;#F^A(T6Cq4x?kO<xEJX$+^$449kRT| zCMVr?WH2`^kY!r&`2p(69BOB<1^xE6EQ+NMYe3=%KUV<&4T*eg@cEWFT0XA2Y%%o$ zE+Rx<-rwn!DX05NMnpVz<I?$qQmW^0`o&ACklpEw@Pn$Solm>ufBu7+)N|@;*t=+T zwphun)2&!}b+pS3jJyd?F5tyK8F7wD4h%`|fB7dP@37DNux~>JmL`(@UAhgIm0d14 zg;C~q@XaC0%28mCyzjuVnCrP`OiQK8(EO=)nGF(rMb0?o7i$g)t|f*w&@+G}Vjy85 zD<{&$v#Nfo4Z~vIGxkO_=;vPa_*POSJm-%bCWExa+4*HND0#Lq4HOgBF&KZU7;eTA z@@HTah*lVpjE)*_0>!l%dg-Hp<s5^LMR+Io`bBvhwVVsP>%aAM&!#2n7NR_S7@A|z z`Ks_O=qZbOr+ueEb%MS-3mao84vHN9A#apCHqeWjR5AW$UH#FP#|Gg?>^(=nn`{;K z)pDcUzIs2{oAI{4hYtpx(XJpTqQ^VkFm>4Fc~Fu^)p4TwAFU=Ew2SjKAqDzJ;@OCz z${}16dJpBzx@f}Ke53u+Xj2f?_)XT0y;gD`kwH7%<U(ik`%8bGE$k$_q!&>cmPf8e z*`?*XIT2!$^XDy^`eX77l{Cej`;e+fZoe|S$6z30`)WL&LB9?=#UZz)mLKy7G`<y9 z^fC<RnkR8RI^J*(G5y}H0_T|}c=vEU5U(^23&o$X1QwvV1Ur^U00y$frXiuTR0=3C zfR)_9h_y<$H3mPa$3k>es|lyD3dRopUF%I<Ao2k2J3FKWfkVj2K*4eJTCk;2P|1R_ zV{*EUW=`eTNB<FNTl}$scZ9zQDM>sE9U>y&f6x+yhE_YFE{sMi6ZDA&93yzZtBr$9 zL3Sb$;IkvwhG&kp9xn|vTYu18f1f$S><<tBI-s9!_C(+KGKJw6=oDv|Dse66l!<Hq zahcN|pSEaKskbUW)3n}cE#|z<OgCIr!{`0}oM)Gd>+w<$r_|E&i>FI!AU?BFNnhP! zo5YA_LF{dlkT8zyi|lhFw=S({k+rFf12LBNpjzti1(L0PvI1I0*o{>t{RM#@khb^; zd#GE;*D#YbcF*z>#IaOiSIYQeQ;<qRNnEZdjA1{;Cs3tVEUgRFw9k%fXKwQbjg<P! z498bfM>;d3#*<gCt^gl53f44R&fqx{l4m1U=I6kPSdZ^yZUniO&jt0w%Q&7S0fZ}y z8=^IrW+Yz<^4l%DpttkRg6GSYm{|kN@ej-O_SFb;LUNa~ad1dWkjHnGbwBtuPeb^Y zT#tw+YOs(LWStpcm^tNiDq`pmU+iy#B!*T442`9ntXpH_KE%CxiadPtJCku=Qbq5O zglCV0qCvsE80mY;%pJ5V*a-%EoG#FwrF3ELAVo=giAo=2tQ`R93FM|gQ%>C$W(=*> zd47}3hmk%X=7){*{I2xNQ72Xkc>1UOVlGPS5jq%~p(?Bgbv%Hr4fI{_PT;<paE8jA z_=}3A-A#4jhXdc5sChjM94vN{)#)r$V&leA53w{<Wf)yf4m^ouz2YpLp|9yT7<Ppw z$>mmdspjct?k$`?!>tvXSc-uy1JQx`s4d)X11T(*SHLnIUS`W{bCfuLmUcAl5@^lZ zb>UbjXx1=w4xC(QnJg})RX`>BmEdki3GV{etU{WJ#pA2X^>A2JgnfJ3TglL?m+|Fj z6{7fm9T{7}*H@sVl&z<I+_=ih4rt#GXza`I>uyu#wsxIq0*mNaG{fHxjrbx!2G`2& zR_7=%0LU{FBV!#>ZI9n3tew>J_WAUwDr2)7b;O!Mp6CH{YE5%BDJ;YratubIfRmfS zsub%MhnOsKw53$8-iKBtU_`V;=EIMO2l6IUBW#C=#;e7+%yHkVrt<p122Of6{Hjz~ z@gRm<kuN<ShLr_fP&x*LT+V1iGllG6KSu6eI2b~*@ORh#Y93~chbw=P6$8Dr>R41s z?{f=glSLF9&cDPSuaCcl!^t&tq;DNE`Oa`pL#oAfzG|*8m6VJNmz(pdc=^Xrd*8D_ z-%t?QH16@y5a*IIO>lh_;r8ehM)u{8TUQ}7=kNo~je0Y!1zW#*@D0tM?a9);(NSUf z5-9FPbzEKZOHV$fkFJwwj|L{q0iPHf&zF-CO7zZQVQn%6Eg68^X(`H+685NTjqo>d zyji9-9Ha+aS&nPbUQsF`7}1%C@MIKnL7oKCA$>fScJ!DQUas&u_i6E_t^UsAvfYl0 zX|JY@SObh{g>In^{G<Rp2Q7}BWL2E6+6Q6~XVWRDJ@LXESup@;7>Ci)gRN)Tuf1hy zoGp(EbAmjC@0(t~%2q+G1X1hu`VwFBiPxly_OZTL?|>SB7QiFq*|ksp-l9o#`njg0 z?0cM@B`mW(P01ry2kcS<oMH*5Q$luDK>c*GRKanz*sjwD7n@n}k4$tX&psT<a5yK_ z5}E{pC=kqS4i-8w5J#<BO=>*=ND5srx7QNrQJYq9Risg}65oo4A?}B3IP+b~?ukMj zGdf~2hz)?h4T}S_2&qL({WfN&cTnm^roFk(SmCS>=0qDJa2TsyMWE1`^}m;I@kGcS z_>1tiIr4-VEA4IpUlX7sU_>>!7Np7S<GB$}!hB|puz|gIw7z;$rd!#dClM_NU->;- zSk(+oYH_wdEyQtqMcaPIuZBxvme)cK2x}?Jy*yKDoUbl0p!n17&d)N_8d5_$!MG$l zk_?Ox%287QNGY?(oD%LGw{)GwEF+eycPuH#y%s)=3XQ-S|BfIp7`0{)*CJTj%PTw# ze<O5co`Jke-bC6#RD$s(pRm4x=nb$d2<xCBT9ry>0y7u<%fR=PprTaV<)~YsIN6X- z_8_-^LDAN8WP{J6g41C?zn$qAq!0H;6%@uNWsNQjmD-D*R+PA3^sV_k^ne~IfgYGB zj&qUIRrDGu_9~;h5w1FJXX`Rm^fSW>-NIGEYorU(y8JYg!&TDLP>MCWnYw3ERI?hs zmV4!3{gGcKS+qAaSbOyi7C7<vJ2g%U?tX=X#o}81!PMJ#oUG(L7|+oqD-@>$R{?#U z8&aI;#7TRAOu}EFcC`Rs@y-{*UD=yQOq`3_D!ZmTZ(rR%1;kGUz}t(EB6rjyfyp8! zcMq6t9E@O(<?upDAB0<5VgE8h-O(D(XJ11M<Rs~+NZF2%IEStLHQ5Yhq@Fr<-{`4m z&`Zl$oY=1(3|~5ORy&#;jQ{wPTPDaBs`JGJPxwYWGcVy#{~UPIwxY%0<E%e$TNRNA zWD%_@#X&7;rL#d%rdz9jJ;*=x)xOVuK`RM5Dl=7fx&1XqVcCo)M__a#ceL*&G;xm# zviQz(%;ei=K7eg>xDy0a$pz)=!ddx=VGC&)a{{sp^rFdk|8PnR`hNfRJcG`wVadnd zlKRE6CL|uZ1v8pQo_(sF^!VtK|LI)8%B!c6InL4PDd&d^vWD`Mx~z_wjQXGaH6xpn zTTx3qjdk0Ziu*!ntyWO_@v3a7<9y8_ueAIdH|C#l(9^turx_p3D+iVi2rpoQnFzuh zHy3QpG-3#d{F-99F9&g&Sj7y)gvKnXnYWLqCTPI~Qj=(Pc*f7?gt?RB-_d6LAlN9M zAUQjqxMCT}F`w#cIXCH6Sn+mjvN*1Sc;w=G7)$F_B<eUE4;6H^9|6DK_wT61P5e+n z6ZsN8iL@UaA$emJmU{Vw@i_#jdXg|TW-H??e7x6cmm~ZS`7$+`+<=JXs|DREwJj!S zsqHvQuEd#Tt`8)j6)FyX%l=eiM?OVWp<Sj-P`HnWiI@W6Aea*HhDMomy!XAR4QhPC zDQ2PWwCj8%1b%1jPWn3kPd%soZj?I3_~q1_^oEx1%W0Zv4*!~tttJaa_~P(gs1|cK z*Ag~>o_5nVvxkk_<WVs~eK?$j*hw^uxBf6DH)vc!+))1QC+;k6EC_$iV7K>CwWj;I zc12x1qRON`k(Z^~Zi+33o(i(l(k^%-m1|QTmh#9mW#rbvgyMaXZtRz%q(Z6utOn4c z+>aBz;GL9ZjO4oNH-_A69udUioC#$g3f-jy1@RiGx6gD{(=o7SJcBwB%Jv1=l;|Eo zQmZ{rZvi72hTIO<&9>51X_pKn=i+0@007kH<{(n}2BnKLYNJ3|saKBO6c9Z7$K*5g zW&~gsQTDbqnFD7UhpVLD8b3bpV-+@w;NBEO6P2J0Z_e*{k`n1XNk#Ayq#?SKJ|QK# zn=V|Kowk^3if=toKn#Z|ML6^pf}7QtetjP~E_3o~h-du;XS=4<=&d{U3sXQ<Ck7*V z=I{};x^MvW>A;4t9a4=LH_wMrEW{mvmow?K$*kAtsnVu+KJvxHWP8DZG$Z4$8v2DB z9(=re&Nme_)xEiD@Mwu}$QLPijPb8Ta1%zq(waxnmN?`VwCVy=aZNJEtJ}fIt+^F` zf@5`#RT*CUHF2q_<w3Rmr#CY$?>PL1hWxrDrH9%&CzVe9nD{Ys@Yz(~IX&ZfOt`hE zS9DA%uf^kSSx(;VgKsM}oi|yc^j-WOTdPUjuyYwLy_k1sTF2`~rB|lrDEG#*#WV2@ zkL;K$+Jj+dtu)T(|3C9yieA467_xbrkn|qRcHSu~f8~l_?3oRFb46qq0ZCQH2MIv1 zj>3v4IGHS<ti%TDHenaJu2T4j#hjAtY)6b#{Yu4iAHWl2zalsXb14<k?PbDt$Rp~0 zusl%mEsG}o<z=>kwM#3<AS-6`*v$`cs`fWJt87khm@$~T=tADx(zw32fgD$)_{>3H zQ)#?X?-(*NO8WcM6z9WlWC9j)jS}nh)fpG6k?7=&L_c0AfkDr16W#?!SOQv&WBLS2 z=GsBNZG|>q$;PiyCS$s?9oDi^iB>_R4;{DsX4O=>$IZifL$?%Vn%%7RMjAE4K&&X5 z$g!yZ05d>FE#x)!Q)k_T21tg<Aa2^oFPkcn)E9M}oN)k~nXXTNsf|lFd&;<Zf7&uB z-FxcA)hp{LI!PL;zSC-yM?k$1s(c`-0W-7|E;Aby$2VZ5YB7#N+`9!IPrUTR-8M}v z?2v1JTHGa3)+2S`ML>Z>ev2~iDrVbxTu0P}zManHuT$`!3;gkJ1F4-M@}@7SFV8X< zMM+M7nth^rkr|DcZRFV!>UNN4f=oj;Z2KdKC*$x6#4zYLc*olbw2o{Q;&s8pENf`K z<A_<;WJGKjDt<DvErR2_$BV22Q_tjtd#3MOUE`%@eEg@Za+M|q#F{iw!T?)66#{cM z7y}7dVB<kndHMD~Cu+mvxT51Ph&EX^QsIrT0w<CY@f!$aheca;4SlC<N8(<KrX)v{ ze7k+Sq8IV0VUDTRzJaJAC~Q{^LQ0@bU_CodPxRWYU9H|0Yw6qUkB(op>>h0OW~tqL zZ?j6=J?CKl^*H)z`9j@Ppm`KMvGUrUknFY63#9VX#!$wWV#fe`kHznTrO=}3wbt~u zKb~J*x}pMKlwF+bce>eTvtS@?`-8u2Z)3f;@X~Em;_Uc$q6u3CFV6J9<?eg+VJ*PY zq%<(II;se_&TmxM`E7hMHRXneuZLAhLw&}HiXxl2OH$(h!o}PTf^<-0-W2{z-9W>U zC(GyjS8TyxX{=1EoZNUP<=%Qvw>K?@x`nU6ai^~r70gxx@NxSr9K=K6a>Z&nAj;P< z5v{`U%~GQ!t8_ZswRohj+g6?Y%n&>zY4M%mV2*z`_9bm)H+yL{0r>H?V=B?WxP(*< zf9)N4B!ZhHcoqkOA)YQVl~we;rulftG3k5#v`W{Hsh+FZh1H4R!p`FGTCYQ;kk+8z z&~MlYEwSVIvS!><Pm;X&MuOtqf<m;~<mWC~qy#3IvYkvHYDpU!Ox!Wq17hpR+>F>$ zmrl5CcC~q1OBG8f_VPG&<hHA}-OguuPT~#a?&o?fEDpUs;@pCpC=#?IV|Cc&gXO2? z_Og`d4%+#*Uo}=$zpJXp6=gdZ)2vA1q<7@HtmW|+4$p-H<PE|de3_wZ4<&inSlV{J z_H1a`zvpZ<-#3J?rgdk00@rQhpBQva^;-_rWO}Nf)Q@@#hDZXWP*hzIi@9Rp0tbQn z?EsPKq{Bugtw>3LHpnpM$SLl_&o4p?*e(1`>ZR-)-!o^T8(U8Fgh-~od(MAUM(%J$ zYx0k86I%x4%VtPWBGmV$_2W1QZXz%kgmSfDhY-A}RnD?K=T=qwZl6gTu_rA|RwuLN z@^z0_Uhtxe-Z-&d=WV%g`CL2C08vryn=0DxRe?jT+M@awN!_mYJf52F+f}EL)nMZl zbNWos>y=}|uPPRLdiG&B|CZM+ufo$Nr#0OCBQCqTY9?%?Z8vc^E|lou6yJuIxOswH zvLAE%X*(L2o<oa?z$mqAYJy8!0#gvF_47{)4BlElObf!ta=$CRD|`F)&UHZ8br_TV zQOdqkLD>4La!xsSf+Je_&i$;f@FP=+kLT3UYaC=Mdbx<`@o_E{QGY1*&B0r@$@eNX zX6C<4#j5vjSA7@cIg)anLmn_7efw$g$3oqKw5xBy=a^G9>U)o4`#D<dz$>%{up(k+ zslYnD3K+~)N+zZr(b*^eV=wH^(<C@|Qya*Xxe-b8swZWD{t^k>MnXyW%BXxdAMJiB zbu_|3KPZ}b+yzuXvEWdZ`+byWLhi6cp;f+RJ@G~+lAGeYX#l%QQ_NUh+)tE-cm8FP zvy*D1cGr)7=Nnw-Jt_*iaQ{ekNBb5a@vq!C01${)(2Z_{v8R1n$Z>oXmOz`KESkh} z56-3JdmY7JeSP+%v<d5<Wxdksz<5bko_|88UEPZ8a~k)n&_VqONwd>#331Fg>z0}J z57_BX5z)4cXaL^eJwDDqgBt_>6~6?QY?$PSAz$JF!95`YK0Yh|U^~g<*B=s!@7%%x zbJY!5F4OJvcB4cy;|gBpA=CV%87c6}Yr<|{S&j){7-=zX7-h81RO%die#MB5RU{2F z<<KAB*AA3;!K-s$X>1jN{CC@|NV0#Qzx3ydkRJ6NQN31?#s^zz+whlTql3!N4gE*t zP2tV<Q=Fx*9?Mpv=T1Gtp7_h(_3{U0_cMZ}+_+E|9*>v3-m&?>i!%%<8;vBz{1nkN zO)FQ|Gyg>hXC~>jvj3ap^nYQ@GTK}43axyclf^aqt241zih%F(%!yZ^%8at_nNzN# zLUlNL+?v*e{J9of*7m4V`<eaKcrnH5Au#`z4W3(d%%#<dn7Pz<`H|EVT3t^EHfdZY zZo3wht<Au1BedWm(p18-8LGeE2hB%54L})G>3`3+UAZ=!Ir&w5i(j_*TtHD`Pgub8 zUG=zEAOLSHFC+pi+?Rqx<JYAxa>0qNI4qPl9(wk=x+4|u4cqV3R@>|=`H%T^u}y_U zyc~U~!=<s(#ql$H{Odp!E0Oo=ZNvoKe*PWxGi3zajrJ4k&;Lg)^Z)hVRTwrRt@Q}R zcq23b7USV6iWEVEv0j|G_LD`~3ODu4>`^|QiZfW_3|h*}8pt@0wYrxIaut4DHLgT+ zg$o?H%KD+&yk#)P@*M&uqLz<?L}_vm(DJx}J-zxrB6)i#|MPFt9KZda*ZiM*u;>3g z2YLVZJ(w-vegU?&TA{_?U~b3>k&HpKEAh3U@gC;Kp&kd=HHrlaH!s+m^Arq=U3%eI z!Z(Gg3g;A!r<jRo#IQ-~mdW`pv)jurX$gFVQt=C!Gjk627y8Ai>ndRLcLj$s*r%=_ zVcq%3Ux-+$I1Pu7qwNT@xQx~ww)BI@UA|iI<)RsbhfO~j$R2Y1B&m>9sr9KtS@4jM zInOW~U)8m`T->j>8C?g)C~y5smrTQdcozXi@mcU(xWR*e8_bPNCU2SSzs$jUj<niC zbzsbdb#y&>1Fn&a5g)S&J91m8-bcFsyF0di>~6Dv<|ps+t>bHw^jvAE_zwSY0c`O` zs096vvD9EK>HM1BM4yoz3}bGGLyr&V>tB31`0L1?NtNJ(z4iw@@)CDOsOt2K1TLC! zv)!msW!OG$<Tlgyi$X5h(3AMo{*XHRD^^$5KjZ3w=E<aYH(4&}qGSvfvg`BiqkYul zM#7J(^wWV6KN*@S{R<k78v5J!2_o3Y6>GMG|6EUUc+k@9@_0JBcy3|Oz2<Fdm4$7z zED8F{9>Lb}pZ(zaKiuITge5Bgrn@Wm5PC1EgDGu^1Brcg*7t4kw}smwQcY+XaU<uQ zTf}v%5Usmx$^;k#4hL6$oNl!;M4oMl_0J8m|G(P%?w}^ue@{Dt3epjTC`CX-EFelr z4kBGdy3+Df58Z$?i4YQxRH+9LID!zRij)W>pp--ikfVr5jRYYisB}q0!mBA~pR+SN zbMMTZxp(jG-JRdg{vnD63Ge$n&-?v;zMoPAbLjbYkLKYr5?TEN$I)pm!XqyXm9no4 z&Mv&DW+`Ws%q!&Mit5%87lO*u`7P}8f9D3jGnnduK^H_vLCJzWv*^e}$Uw%t_duZI zP`80Ry^5CvH+U<0jun_8I2jYR+xU#JpBA1q^LWcY4s~&|vX;dZN>s+-CfIID3uRS> z3!h?B`b?$n8yB-6Ts7tdS0R}tm4#3sA3z(h74LnQl13-?VHJnr+Gr$;kQrTqODo0} zG(sJHql-L78>8V+xEwQJx_4g9hVPD~a&bJWFohD}yX#6L0j>sZ`U%^x297r!-;(8k zm=4zi3?pzHGR-ad(trwHMi2lMRGujwT{kCC^|21W2Hz~UrPzh$x>Wh0{}`)LCVrfY z(X4#*K%=2_BBLO#I=juILFwA(OBGeFg<gHcF`ufciS`bPH9EB$H;D(($s=NZIc7Vg z!oknZlxRBa8$;~t+{(%k8+e+0IJ21aCAHcT-V`~abLU8VUhuEDGMhlHx~YZo_6ntL zot~p7y*2B}pCaG5lteyyRJn<o7Onus==J~EV&z}Ge%%RDZ;=OEF-o9CT&=`-2%ih! z<loHB@q<L~izRe6E*D4smVM$ZyX$!8vS@dMq97E(yER5x2!09n)gW_yRHUG|D#Z<? z!EeW}{cVMdjb9sz`*B<^baJ-HdrVNC;IvnJ^<+-@Wx*$G^{|yZH<kWRmoAR=z<$Iv zEB2Kz(-??V5?{%pYKZTQcA&;d-4*F#R6t6Uey0HlLFk2KABTEXw}??;GNDE9?NGBh znI>`~b*}mRT{GW9w#UL_>bhimcZEwXMZrsd)>sAGH`vP`_IU8|sB5fa7=EL|+o<CG zyN07NweJe9)Q%RnI1x5RTgC()&`!Re;Gpm^VS}<Oi2k#U%FI@iFc9mLnyg_@XR7TK zK7lJBdk&qpoHIyiqvu3M$9mUYJw_Td+K`e8nb^EFy{Jsus2_gIyi9Kvjo<)>kHGi4 z8uT}7&89nCTyTC&a7q-Igb7jc;1u=2cGzM^G;v!oKAbZ()#!>>*~@osH<xO7HlJ#E zn{j00wix>dVm7xOo-fMaOo=4Gz;5SJ$oEO_#oK<cCWy<Su{T=rzlsMD`3X_HGDv!E zFeDWHp)}7YB>!Xb9_dFtb;`oqD(Tn$EHv2rW8z!&I39njxJ|TUsId3e+m}<+>oSRp zhF{4coBZ6>7!IC>pThU}?cipDhDo#q3d1q-d2Jo{ECM+Otp1j2VLnR`rkRaU5Dg>u zt9}~A?m+K4P-FnFXEJ_fkD^#Y*p)8zj~LP&zWm2Q{vAQQumFVkFqV0<&?aYqif?IK zVRAQFg;{J0LuQk8(NFg|lrtxlgOe0#1e1*uc^;fK>Ht#u%Hqsn$c|NbZ(xPP_yhAx zRst&4a4^Dnai|-A7hD1WqK5Be2uvY)PRA>zc&B`*p0AV?T8bJMn0hm8L)=+r{>peZ zJ}fUZ?z_l~G%lPQijO(s|Dx=wQq$)17qxQ>=eu^U1dWjiR~H!A)t9dAw^6TgG1X{v zrr`;rd27nrv(eTJ+mHuEU0kEaS{=Y1Q9H@k5)GQHb*sSAVA9y*&>I>XziPqd`;3@f z160p|lB#_w)S-m(AkISBeqiM?VDpDc*7jL?vY~tNfJIAtVd%j9aFTwwOTmV3z>m7+ zpE`{n&Ku4?*%?GAp?gflwN6ikdrVLEoqJRHsH)?KX5oEJqMg2sm)74rykGxUx3W^1 z7I#+(3n3XQ(wzuYU?5V6j-I{B4FmfXn2y8sfn(t{*W-4OS_^XcL|_R;Paz>BaNT;8 zS-f%pDQu#>Yknn{gXsNP+ES<a9EqNKRgB2|T3%*+ApMgg;cS7*7dvCU$bFc<#iGVs zk0zD@m@2IT>Upakn<9si;imQg{>$^oY_c)*I*1~Uk$c}<3rBN3v!WA~sQx`!4Ub?S z5vk-oBXAY8*elTF3gBgaiL<cX=+N#s&3Fa#Vw+^5x4x94lfIpJhUsqWt*`obVvzD~ zml`FDu2@Hce)N&cUt315ZU|29ap@ad)IE=YB*FnOXz~-ZRTqE${$Jfr{TKcJ+qpSB z#%KvERv;J80|>mj9A_TXh)Fo2@ro~7@AYYU5+w)NX`JI<7AShY*h3sJP9@7@-xm(k zH1>^#NFO_w`=)KA09?}U>z&%qd_`2XbI=g06ppg#XEqIG!4F^=WVw6eW&?E@;gfa! zae+CaBSF0PDIR5x$k}9kpX%QJOoBS}FxO7|Moc@LP>oKs9U-mC{A9Wgle*5(JV%W5 zY-GHalmn_l<yVt8h>1e~&ej$yb>o=dmHCZ|grTH{)Nn7^c7Q0;f7B+-_eE$)UUXMp z4jFFQ0GP%R@|-o0Ayva~frsqB5}5pFK8KumgQI}Up(X;Z7}JjxJz;7%6h2W+p9?Q@ zBieu=TFz4+(FZ^CjxjqGS359Yn=MK*JSTpLM9W60JC#`xnD!t6=aFF(w@4&G;5zk3 z9_rxG0NYI)y20|x9DUZQ@Yj|DfGO>yMCGc`zWlj2Y21*R1luOszpEhKwv{PYrs;Zc z<TRG<Ie$Qi3x7CaH)AMMO<A4x-?aNvD5mjbC34BJ&b}oIXRau+{~c`ZIkgx67f81M zJ4n4-bJzw_L1p?JQ3uOOX0foCo@(Y;5rS3Y?#jX`71KP1jp}s2PkUbMO2W@DoUTQg zxfpOY>rLr^zP2-_0TO!!lcwCj9Skv!OYKE`FkoZr_@_TIL$W0a?3_f#rGhFjp+!Qe zw=<0|{v29aWQV67`8rF?I76xo75~!tyF=UNSBfMKHGLd039L&M`Da99xddK#4`u2! z)r0wbjh69w*FJ-b&@d~MAdRsVW(fl8e@IyRgt&wNF2dfhYWID2=3sk1JZ$=Oh!=Sf zIUXeua%qeU>nN(j0;ug(Jz@9^`p1mo@>%8VqZ=ngf&|503P*+FmP2@2o-RBn5d3Er z>A(CMy&^6pQ*fFy*A1R2sdK)5^j|pPcLgU2&`@9d>atwRFy=HX;pmz-ofO)7*?!79 z__ndD?Jt^|+S}UG1svgv=y~B2K_K4{xRerIHX}uJgzTXSvYAEDucsD}O*yYh8Vo>F zv;9u(K-iVVLN)D|iQO<sR$kCs*FkQI)p;+4?Ij29XUp#Mme#MeB6f6@lfwqSL|?o0 zt?zmeYp(06rl834n<q|Vl>liHs7^8qu;wO+&(*f+7>1t|eS{;pVymfKZM!j=z|qX$ z8rQ*T*bvmYcu52Kb*=gtd10y9Nlq%YP2nqTV9d-kL3fYlHwmG6wavj>w1dvpxmGgA zBCM33*&Gad;&OKQm)*bpvSrJb?c4Ckf=iLL=Prd8N1@uL+ir{2zb~9OWGPuhg1m+f z&HAE`Al%8$;@=1*-}=w+cSvI_o!Me=4|fv>QtZ}NCg=DWgQqQ_3f5I(hm>qECX~E` zoBc30lr5DCB(P?u59RJvJIY~Z6Y-ME#lMz}>GOWudfoLF!gkntY8M}${MVM}^E8mk z695^-Ji!FDv<xFa)EuBWk~M^-ln}(etAv8k8o^EInid(~fX=bWc%5mSC~$q^1I70- zb~M$d_{!2eju=N0a+>wnI7r7PIUI4_fDvc~Ze~6dHe&&ZjXtA-ogA1*LbY<TM-3r1 z865j<0nxI#n*QE;yP29|7ahlv<yLlAMzHi_9mdG-xae!s*s=60jq2%Wt?Zy9Sz0MU zrN5?Msl3b`kV~`8J7c}`ip=gqiRnT;ZCX86Qsx1{FRQ|6oJ6FoekL@~E!XR(wsFqZ zTAw6Iv^{_o0jH;_hN+RES}i$ezosw$!&~Y<^ZS9HtqoNRCLT47%W=ttx#bJ(311uP zmfEx4dc<KLRxM79_#u}RH;W{j7?YO4dz&3Jn#(mWzCC|0blwTl9J(8W$Sk5LA!chp zFizcoIgZzvVaq96>8l9!$eRj5O5v^<q{d}e8FUY;mVUwp<kku-=o?b|0np&A>0ij~ zhpUnDzl>DuS*NVp`4g~HFqO`e095FK{1Xy_SviUj&K`Qg#|bQk176-2DT!X64jG-C z5*#9qqcks!lC=e1<IxQ^)YcOZy}k_e6k@d}1}pE&mr%YNRR${q2hLIh3m_IyRk<*1 zxU8>XJ!05#<yiEfp8Y7Qym^#3ZPGufU}~zQe>g?wdX~FJw$_Eqk0qc+t+(>_xvyLb zSa$1fZC9gngzY43*i<iHP4!iK^yjQv75itOQy;_)h-TspG=HA{`@#JH{^(zxXa9_U zo;SNO3zBCr=Q%~`W;^HbQj73o9M7yy^LogkZ$^P$|9FgukU<Cbdd0lUZCvo`FOAGn z<rT$N&G=fFozk^aFU|dJ2Yr2+Z!pA$eW1(*0E^%q-Oy{ol1XV%;&)S~8uk@HIb5U6 z+L$W1U#ILktDqaP4b!J>8U@!N=hK!fVZi>y_ykriizzM4w2F#9g|i}Y!&k@oN@yn@ z9Xcm<?<>aUso*5k!QuCMAZ0-V1k3SLZoN0xkK9H!f^Hw6rI<yO_nmW_Ftk&M&q5%F z1Fr=Vg1kt)qR1QTb1xF#4@9gFaGe`qaZ+})ALW}r8tC}^K*<}wr6BelJ+&D?N}?1Z zKk=|mUD-#{p#h$yu(H#a8^8oAa9oZ^eiW;SorJaACS02(FicrtrBP--57OHvMtmU4 zIqjaQy3bcP*6zVi5C6lYEtTMV@p)Ls@R~Q@(xF|Shr$+FlgzAiJSvP#V&q4W6InMW z78i!zg*w-suU{C?(XtO4#QCZ%Zg_z%$Lk^Z@XLREM|*xL$`P4kBEabM2+*9}5#A@Q z>ahTriaiioR3<5`z@Z$G7h1V~_*sJkS6ZwZF!I7jW*W~J<RY%#T<Lusqg8LW)6k9X zu;t(c|HjbFpXOyTN3DxJFS+CsueXH{_%BV_>3iFblo~sC7%9=N+O5iym<xHRmHa>; zKa(S$!Kgs0PmI&yhJve5C?|&+(tMIGIs(m8)piX3nY61Ypo*R4GNI_dGgAKbkGuW0 zge;)pUS)C|tA(E_tBITx5tYFEQc$}TM@cawu|snp`|n*xQN}CJpXY`h;d>U`<k%NU zsaWYaS^S;Qy#o>!kfoP%<|g5)R$Sbi)v^V=4MdH-FUrG&L5mH3zxWnLaQDz^V<T7@ zCx-Z4r+;zOuxYg|PanP1macTei4<zIQ5zZ;-Pxc^p%mk%1D6H>G@M;niZldF<9=N6 zcQYwqknPStjU7?Qn-OdQiNgB?0V&M4in#(qD3@iRG;_t!M&?LWO^_Fnu6*;ZJgTKn z<z}wWzU=xtfONv-6~9OLBNI1g5FPK=1hI3>{mWhmD$qNK2@vYIo*P{1TuxRXN{>b9 z$q}1E#g@2_ZL~TOFE;&Ie7ht#;hB&2<jp|!Hk!c)JkL#75ujYtNBK>(i{5z+bu8A) zxNo1fBYYq9n&oQ^gLvbe2e|p&d5a=#4z@uS74+3`+JkZV^0q!S;jX}lGw%VOUvZ*I zwQa%%3K&iS-a07Wr6wJ^)%KAgS5C$pu1{~;)QdY0l9c9YURm$CfxSa%4C7S6Nt9IY z+Aj3Hk@<`7jR@(oTiH1Ag*X(cu|;(v5&p8#ik8zmcj6EG@oN?yk38~HhSzj0mV?$R zib@M>{KWPp^<A%ZEe*7mi37Zgul=FcGmNJ{d>61|xbAa6yxN82*+T|(ei(PU;hLIg zoI(^!Zv=Mo6BM}>mw;9-vX8m{3WJd&f%Xb*_HTh)!4W0NSoBlm6x@}z46lH~Omk8D zrZ*PzzwdhXfq6CLwntoH$(_)r!HndFu8m}2R<Gx6&k4^6t-glOrQ;pM*fY)Dn&Ai2 z-ezYXu-l<bLKfS*JgCw&*0pP$8gJbUAH<jUpG>-;)Oe*&GjCF>+~GYq%l`?IOUm#5 zE2Q<`7$YMLDdETuuD+144z0~A6qcBIR|;Ulm)IRa?bSsb0$YI)9%FnLtB2m>+>ztS z6<e9)Us&|I{zIe<88j>@EFV+cpE8cos<$muf3M_I^|`D9+>933ssXS7h=ZjQ6L3As z%x-25b9c-Oj5PlOP<I-ZO*kFI9OQC|iV<EuK3><Q6wp#0O#n&CN$zy6SsO94tN3V@ z;IIB?n@1~!NcyqN55V@uB2_vqSU85>tk``bGLh1n+LNW6Ap43ULo(enK-eL0;q0Yn zJqw|<Fjr+Of_H{tkenzNih+4UuPG^K_tG~)nwbeZ;%V^DHqwKD(Sc`?Zk?P0SeIH8 zZV(1veztDQUH5sW^b^A&p1kb0(r$-4*4Y8id9x5Bzi#siaM!2UVO?xo5?)%MSSOGW z){qZOUiAfNH6G)i5FFpQz)1K$a&=Jzhj45?LkC0#a%YU`2W?+iU)(RDb`0a^V~<B- zrb2~k1B-D&iL4O!#r}o^&mMefES?f=|5{eNFqBwlUnP-O<sn&WYihWOq}#MigTvnI zz<>1AhsgmX;}?-0w9Z1V3<IK6l1cFbQ-ypIq!Uws^vK~#(2a-DQumo^f~Y5gQ||?S z%eO9^H;`hW6J5D`%Obup5EAXE+=timneZJzawxVk$BAdH0emF+=dZ<{zmZj-NTB*W zW_-HqOBLqNZ_`WdqK>C;FBa48lN>{qoySNHiy2M{PnX&}^IHZ@n|^E;Nf@aId&RD} zvzK|_1m}RGx(;rI6vGaExK-iy?Xr{OU=~(KrN^$H_}0+iO2h5PsADE4r8U33F~)zY z!)DUW^p80PY$n~1|5Q2^D!BA|^RsW(=a|&W!qEpM>mjeTX9nD-9&Um~4iv^um*N6S z8W!7Avg#LkCghyD57UdqjRmy~Z{}iqSqUy&tAU4qdS%+0>-S+W^jH%pvxi?Ftk15m zx9GCkf({qR+~_r_p-yY_9taaRN^RW7zyG!6Lp(b+xzm(m<1)U6^yox1r}i6c4+ha1 z<n8E?Wsl<BpF2u=T55WwcKo8f9=7Wthw4}unvsGZ45vR61PPOFppYU72LSz@;F|vi zn^acp#zWea9(_lax@{GAD5d`Il#BFYQp3k2+rwRL93;?QrRLS!9g{pvm$|xn#vwQ+ zf^UE<_#*ejMm3|#(d*GsJFQ7%cp+2c`^TMj<K5Rn++rWG6qvCnQM{lhU8nJM{Xb)! z`ER}FA1b_NVHTkHPQ)pugc*U7Vn60AMC0`Q-1LJE@GS*riwEqX7EX9C{kf(hT}^Fl zO<%i}E;w9ku~$KKh(m4)Phc_kPxiL{Y{Z*dh8j`kv5DL^cHk?D0y=&+aCN*q2q7st zyv*o@D?mv8FOc7!0R)(m6_HDHE3H`iQLgQ!<o%d`bX_<So=Vng_x^F*!=F=D%U*?x zhd&X%ZamF~Q3+cFJ{+g}SY>bkA(KVw7;|o65&S-TXeWBqa<tQqg_908HoeHeHkF4= ze4AoUDcXKGmfK3t^1ZcGZ+?to*!y`jEPuYrAYF=u4JCHc2aTT0&wcv{zqaGqar-7H zpv}*z{TuE`Sxujt=RnyRA+UD{#(yZbnV<xk;*_y;N^@1q=^4>GtOnQ6i?V9PF}BNU z1G|@v%#QiPbB=zuL4Pw%aCmN?y@A|S>aM{(eR1vjIVqmNXA@6E5d?BDnzGHxHDHiE z-w&7M5R12Rb5^Ihl2dlvaG;b&Y{MQHd=?V7D}#T|AQj_yqdJr#feV7EE;F>#P)AVb zM;L!?q0Y#0-nG<CO_3%l9{aR?WJP5jIaOq&$QLR$-*)X-9j0{_-uLaeTET9*>Y*p@ zTG&id6+W9#@I?kP3_QMPZJJZT!f%6<=A2Z}ZL{<hQ%>AG?-^7wJ802qIyaQorl!{n z#q)IpD%^9PcHd!Xe{E5j8E}5VHO^RZu+rL00VN6=?+N70ISD>Nd9PK9ZwdB;^5y}W zdAn|oIaR~5>7IIcJIzd$Z;X8tr`AvzoSRvvHJ=5BFe*^lT5p?`Ej-%mQij6!X`W+J zFZ?qQP>r%SyP4b3fvln)$}T91wMkvuwdldnP6lK)^P{6y@1q@04WH>l-Kd$gu2s7< zpZn)22hE&gjo&&tZf`K2zKffvJHoWJznuR4bBNP~BQv_9`4ewA<w3C3@o)B#*v{8| zcXX=!D<Y>w(T;p0EVB%j<^vgsBs_8KI=&Gz@EUiSX9MkHH|%lCe|x%%l7?W}+HjV> zU!d)$X(f-BScp$+#zsk<d<pQlDKQ|jFG?Q(qb2e8vaE|PR>BItANEaK@n#lW5`AVO zHeLEDz&Nbf;2MeJyM>av4@IMOAMgi)&RlNy4QX$Ec(diT)MB5}xbIa}_u;X+io0*# zyh`zKIq}BgP-6F>WMVe4?vl*<gJ;@H)ib3(e2{+}UGj3T5~sa6ndp}-9y;~V?#^i# zke+eYGNbn?_F<*z-;wfY@7_57LK16D28wcPq8sjuw;pSl>lh`W6t9kI7c$*OkL&xV z+XmjVQ!hLxKRjux9YWEbq80+K$0a8qHT{W}ITxMtA&^Icvhk7|gcKfIj#IPJ1&9fj z<A=jjLA)}mhANHQ3j)6~*mi{PzMkiQyD<`tQ(9txVEKkeCSR&{VxPN@JpEF2fR<Nf z9IQV<cZ4^Ex(Qud9Chj(EekJW#=Z~~TUJJWmE#4)!+>R`=X&Vho>}yRF{{9wwNK2G z>y%-%s?sA;0=eF?b;}|Ru0rjN@*oqn{A+uBkRionLao!^TyV><>~BMc_vM0;OYhwS z&O%eSZDmlCdaZW?S5-L%VDv+zycz<G)@^1t?mS)eg7p$4W){T!hA-+iLY`MVGcI$6 zG6m*ZJ%$nJ94oGVVeYfVHT7n#uQ}VBn`_0o5jT*G&|ant%7`$z$-7T#<^-<Ra<pwn zkBN++bNZu>Bk;ougNE(azwvD_s(!M)uw7=JTKT@e@TMv`qMC>pR)e1@>U^*GcVIJT zz{?d9CvW1e!}eSSLU`S{i=JRp+R(x!Lcr~~w^S&(Bgz*$z}$Jf(Qp+xxBB%Usph_s zT_<AplyewS=+M_x*&ZJsRtq|@8^Qu$(xCqx#H3LC`B%E@+A{SKvyV@pM-KD-`sP}0 z7N%Y<>DR3~o)g2Ftj}L@<jgK4vR;i=<-LPRP&)=;M%hEp|G0Tkbq`y!i}9!-^4rol zp7FpN<vrb!7q;f~r27-P(^?HX^8rnX!=J)@WQ4U_+g)YOv}ZUtCRC?GQ|_EifxH1P zHg-hd$<byNtqNyL<Ohp-LAt<b|BQmLRCLxixOM1g2fBlYh3463G5Sx@X^h=k@mXT* zeWb2WY`50;D_c`!)0nNsy&sS(ZFr935Tt8VT^)-WEt<8Vt~#sydD!5uEo>Y>6+1>` z|17kX3mfxXEsMx^<lj8;aX{*cgXzH0KD+A$#^aZ`p##3K*Q77E#$T^1yF{P^4cq%i zoW9c6Z25WQ+rnxbVB+q@4*G3v<{qhdvRr1)ozF6}tHPXvbUESOwL57!qB8<N?zuwP zZa{GItzXf|{`r-y>)%Ovd2;dRnTP&&>XxFqulYwD{XxRtN6c1%6Y}FCb0_ZFsx~j^ zJ>d<=T0w;2GyHaDui3Y{)W+5>f<nD*Wr&@lxHNO5UFQD81Qjgn#7wqPVQ5{@v}HmL z{}_PI&7?Sbv=Pz!zqTj|;t4>hND_()VxQRYEd+MM(WS~#sbrNg4Kzt=8}=P@uVL+x z5z#jEF1GRSeiqdOu??ZSJi_-c{~Xwz>7yjl=ozlj`N!vt^tB5sW9t0YkuPoj6-)0v zN?P2nb2-&V`{3+FiWxM}ELg=hEVUnrrf@ES`xV<qX$BJqhYxW5rFP>qTo5yxxxdqi zZ-pJj9^$N6Bldm71VTCCmEBL#$ex>jZRvc$aZ1L(9-~dWe`>B@kMGiZrBKu+uG;tZ zPfDz_73Nr#?i<(qFAqvkT~l#w>+qSAnHoxck^Q=w9a`lUR@fc5?pFvI>=8(j6}TeK zLZ%{#o_&PLct;3avz^C<CD<Jpcc_+wWJ_R9am$K0MeH-BbJNECxInJ&G?5pFxiw|8 zbII<h*Yp)?L}hnG6v{9{4vymQ5%_SKJqFNG>n^^OKpQ-u?h3$c_TKN<Hxt0_<V+X0 zh)qcsoq<MKHT^?ri01izFr%=3_yiqFWo*)z$)f$ktCekBgVb=+9d%BB)4?L1^TKr= z^G)lF|B2Vv)j#XTD-B3zQ7wp{9&rx%?wyi8`Y8Pp0GA0$=Nm56zM*{mSn;+Jm04k? zr3^SX8Oje%e7LU_qeGQ{==P>oXWA?3XCFMc&v4G_Wn-Y4jqs2&GC3B(@nP;Dq-+>4 z^c;o2Rv~!shDdz69CN9Nb-RI&WB3SEGgx^JsJUyLnN4<B5vO{Y?Sh{7f>gujJiUWE zoh6fbG51t53fgd*y)$!sYh3r`jqoHyt9#ALN#MU>3Qh5^U~Wy|0um6s$M0{7<RI!S zGl?i}CLJ*+ba+*KpDu9VCeloHm9nNByt7Qs!AI!%uVdjY*Zc#64O{-?T=0GXWQbY@ z4KJTso%h+SNk%WOkG7e^eGd9*wddaB>6ON}{@Nay$-R|lH=7sZOV<p)pCb9D>W8)t z``P)VIdOZprFSQbGlKpO^D#yEuQ6`S)(GUod2Uc*G+7p+aLB`Ep>tp@F3-*Pn8)|2 z9fF*>bu6hqr_{JAQc;$fRQwzwsm!2l%*?#rLbDh(*jQD*tgvnuTx)SCZ^omTVemU{ zHuI<B+HWFV(5795@AHC2**O`&_s@y<*OnI?sh(vRP@O1{wFChZ0*%=*5Tcgo<0Sb3 zO$AGWRC*m{7{chUI%~I91^1t8(rhxq6C>V!E_Ba1R#P-^vclz5Cd(u9q>==fRegKm zmg1jGNUgi%e$6kxR<HYLWZk1G`P7=Utf`-K-EK+wNsI1oO;I7vd0$*>%;e*YY6s6p zFX>cQ{uo0<2N8fBxE`;W`?UnD_Cvxn_%4CF5X^&bA`;ak=pqE>A}96LS|&j9gAs}C z$Q3T%W?6x`)69{rw*WW7sB`dXYn`ra#O@D4abZtls}>pK*$;jXJz*7jaVcGj0c$N< zonql9rCcB_&b$@4klMUVVFgg;$S;$DNiW}2pi$J#Q#0W9(UPJq*`Pox@!%Tt!eoM# zVfW0oKUru9JziJ0<jJMpjM4L#{!`5+J%|mWAzXV$!$KN0mWGpV=(_V73zdn%@6!Dk zrJhdm_K{UCj{+qkv+T4KK1P?Gvy}Ff(zAL`?Y!{$NW~??IkSt}M$Q!fqs~`?DWL~m zoT*ZsyOS_wg^dvnAUV!RaqRn_$W`-lG;Ox1AAc7KL)<VO&~suK@K9WA&J&CkRK)tz zXl>^pcYC)N?d~T?w*g9_u_v<FQhOOQs?tQ=uVHInSB?<V-*$K#4c!x%flW_@^48cx zEA&|>hN(y#q^(`S{;9T0*lgx8vmIC&AH@WWVRwh@hcalMeoA9zfh^_@0kC_Wl}a}u z+$W1cEv}p|MHtkIk?@RO+M0e<%S&6wuCeDa){koZvegs{E?&GfQax7Aa=xK4nB=yk zy)tj+GP>{5=@`9%>8b+*mIubN%#1L;4x>*}VhU_@UcA|}_hpkz!I=Wv(I>lVyee{9 zsvP?(KP%av!ZnRKBPB#kFbN=^1F4CY<)9k2S-l#=SAn))<R%a@5W)CNvO@aED7%5M zB4lM5xQ+u)YK#uItoBAsAZMUYQC%hYSFYSd3%NA7X7CfO-9w|S3wgt-^xM;Q3wzhR zWRr*oa|hh(jIs8H*QdQk>|L^S9DC%CREUS$11oO1D&5G}PkYQpYjyhG7}nCPj*`6G z7Oh>9{9T+~AE|$w+jN%~d<7dzm23dj0QWz{Y8Izr)@d~ssBC_KiyaLz`E0tWoFU!= zXmhR#>SiZs@P4Q7BHZC3SjI_klR?(No}7#_r8Z6_`M_A%=R99)edVQ}GjH^sufJ;$ zN_)C3ejBB_`@B(lh8;N?aqDTt&{=IAbGMMlc(2orJ!G}$2)U}G?oYH|+c(#`$~zQ} z^tkaNgV)bHj@Ed7EFSTBR$k+N)#OT=#}mV6E_qgR&R13bqsR_`f&M3O|Np|*V#P%R zFSs_VKpnl)9;$n`9g%LdHdW@7%nZ%HRygnqzYP%xyXz;;L8hvsbEWGKk7;wUPw!sH zY=Pb^*wbb26uF9co~(VOPJ9bcLQsI6r72tU#VCmN37h<zpaO~2P5174$Tmpx69?{- zPGC&StArw|m-0~SW)>BTOi8R5R2lr&7PB0Gt~GCoxv606Al615t=apo$kguLsZ4KC z0PzR0*mtUFF7W-MZ;Z}#LH4O4$d&%}cKB-_z89Fw4LkjJTof(_)<yM#T_Zr95h(A1 zhbSYb9oTm`S@Lnn(?wUst^&P2o0EE6bP{7uUsVp4KQso$_Xk1NA-#3<oZh~^q^MIb z&S&w4k~Uh{$UVSw7ZuL4=8C0$N1+;wO%wiDYL6lPXq!G3H2C}@YCyed>P_t|77Z5Z z<(Llwe~uFs*A$!jbyB1yVyc~fE`0PIW%BL{h4`J26qnh`AOLe%-N(Uljk&C$-_D4R zKX^+UFd?^<Ur`+mIm31M=`QDNA$N;<@D_D%u$=RSjDyZr%1PO0=YH8{{S@#7wrts& zKA%WF+Tv}!KLh0V&g;chX|6KtN(L?($!$iJ_!ZuxR78YM97~XM_TBZbnE?@SIP?%0 zAYDo{PRHs9e&N_q!}ofB<=dnQ64+g0-Hg2|it#@flr+L>1G^_O*7n3XGtVCI7~{B6 zCR1sQ)(T*(>NN#KRP?#7GYeOz(KmQTkS80N>7}3fcGv3|X6AJjsni+S=f^n&hEce} zq!s_W|FIgQ<eQgZk_5j_-M>Y8FU^sEG78nYVnYo50Zsq5zHl{&9JsI%HKWIkO*Ru_ zt$Oy&oaG`?I}fnTCC1lCL$VOTYZPP3T2+2?YDZCVBdP8q6;<O^d+p(og8je#N`GTw z&_jUqVppY`Du;1_Fi0D5m$2*gaRCyiy_(N=tlaXWELu&DbGza(#)JcZyzp9?NIuN% zJki|IRsTxeco=9eXLE5ao&X1_5L^^inptoydF#;Ya=$F36_>e+BP@WBH$q}eitk`N zhc+vg8of`0MW+T|f2j|ArAt`u6Z$+_sh4NguY^7Fy&p_Gv(;pJU&lZ<Hc5aupC=d! z-$#AX-ip3Ylagf0kw2G%x|eJ-s0I_(Vl}WMBY$oA75fUeJ%*A~oG?J#$F~>Q=KDY= z`3HlV@YAK<Hc(AH`u3a$v>{`vXQfDO*aHT$1Lt|0aiZUGSo}qSDOmFUK_VS`v;lxn z+HCq%)~2LracVWeYy(NbDKh7i-&W!8SQY5rJIknQCScCTw2d3>7N8myH;)hAXb}7| z-g)C0!%4<3*d^#&QZ{0Ycp*#X?C_0jBex4y5^3q%o~b|Wd3t%v_PaYx$Viovxd?ie z{i6rS581gtHi`yGlNyV+?s!P;40f#<a!+^}YEWb=$iv<Q;x~79iD%oMJ>Ba5%PaTo zkGJfV@X{e1`+o0V<O%%kxd0v8TtRY;c~czv%GO!@cFZ9TU~*7)VormfEMGK;Py*`d z@ZC@|pS6VcMFir+37N;RIw5Kr;B~zT*C$44%1R@o7qjid`cKBfW0VZal!#N6r*!&3 z6v8!_s;CdvMp(6(%>gIREG^aSZ|1?ckOY?xM#_Y*l2xWLMjV9<{z=i;jN*tVi+LX` zH)IM=IUE0!I*U_6OQ`BO-~Wy}!y9#+Am(MaClHhIyf5j_$wtY9!~&fmsm$7}QgjC) zeD7_so%d?b*JWfBNFXO6A3rtSTGxJ8UUpQqFt_Wbl|9ofA5(Bbj`RM>S%I5>3hnyI z;X22Tr{hn#?i^BOK}}+viCNQwQX2shvjhC>7eEWNhf{G$n1a>Eb?Y3%-E7dNmg$dU z^q*iS^1z{E2ZA@Fc!9uAQRv#=#^7ZDcR1#@2DpYD+dvH6;qX2jHA^1GAZC59zXMy@ zF`GT|slLUhW{g6V$_Bls1ehG(kJd->!h}U@xDeGSFmb(i;!2PrbGMl(Hy(@$_Lw<_ z&Mx3W78>$BIfVQ0V}qy)DCOILJL&!5;sNKe?=SsF*N(5DzJ`UPu8o4ux%SOBX^zzF z2WSBo{DiYkrEn-JtX$|6_Gm(}E1Lqjr}jobosg2>f!3@YP_fS1MT=8m`{7^5>6VIK zN^cd8G+d1u{<+@ZQJi#?UZ|~2`%L_}__W@r;3Cr8M#6A@W$C*4qY7<1OPyzOoXJv& zXbq(~-S;*2WsL>$PKK}MmLgxbUA3a_uI*79x!kFvYt?@${9Cw4d<r=G(V0y-!09#^ zgXYp#gq6@dk0NZqg;28btwcN+R^20?M637Hsa}-6J=5EUsQ4Y~&>Y)9kJ=L28mEvf z*Vf)~Ze(UDHv2%a(N(D-1m|KP_@XBQoxg{z=B{&eflNm)ZZ}z0^btqOuHXVpyEcOF zk9~tZ*9z?`*X|w?N1vkGj)UYMCq*UzV5E;9nZSsL-Sa&&^Pxtdd9vk|dTX((P}A8b zL~g=A^{t!C6*F@7Rr_RSQ_a0{&h5%lVdtZ{qS<RE`3)wIUIe|cm#s@H@b`;}FpM}s z@Nn(D&Lyo&6ij#3bFZY{k9s=ZmFHY|T2{xx<?^M5ssq+?V~FA*NQX2ekT+%ru2T4i zK{|AiKqjY`A`h3t_P~fvIU;Q|j;%@oj$hj7QwEIfRM82@fE}JvNCU=~S)EA#XM(eN zDr@qFU)u{B-qyE!_p5JndypW{{|B(FYU%Xnf*?r(s3r6vOa7@z2tP+#dCkIQz*V7T zzElWBNpxUw#sDrUtOmAwrF#9v##CmOSHYOy{&Dglv`sHr^*Qd8_mtylACg1b%G#_x z4A0NE^8f8|^G^@zSbpDt?8ZyB|C^<EM2=zeM<+u!4_N+zdli?G(bh@Vfveav*+-u> zWM600y0|a&M<2Z=YFP2=GP-m&{f~B;jQkjPpW{a=>sU+imx9JFKlnI(Hd5pcdAotz zSScDtv;?l!#LNNs|EMrcqyrc-HW?scc`u}3puoV`Km<<7mHbyG<lJfq)G+rz4kv#9 zl1PC(LBQOJs0mR<;Zslnz7jn<9i~bzZAQ`n%u^>4f6f1O2_{RKEfkr&1mEY%5_1<; zT{M6rA$z0_iay|p^j_FhDl3PE$UrDR9yQZo<0K(I+oM3OhHL8jlB<316MaK|VhtFU zc%#!5i)jw;1;%%R<=+<KCx!A;DGeJJsZxK8)WCvM%Di1phT51l*Dt$-1wZ2v&b4JG zg<30RSQhK$9qfcw%*|-}eV@@ax|jQ1X~QK>1$?ufq+1&@_dq1Ea)iSk7q=>xj2NvU zZLc<>9y@0$z8BUC-m*Y(7QEA5ge35w45xSMab+Dlp+1f*O>`K<HKf6h3t%ssx`=c` zSQ1l)H1&aq+##?_bjf#k*;?cWoAf-><V!VtKKMx-PA`91BL=_ZmHnXhUFn6nz}QQb zA(5}`>`x{AxO4QpUG3<D$KL}lYE}#l?sPewIXfDOJg6o4D@*6i=^SH2tpnNzp1wA% z;0*9~3PB$m+~p~fP~p3RgU|wgLNut1T+y3FWH5oJ5k~wyBA^48#@q{Rk4D)}8J&CB zrBQv#ZNPlp4|lK47^^%Q^L>=$fD*F<bPPuiN4T!z$HIvE%MSb!EBTIMo|2iqNko<9 z_DgPy{l>$WNM0K%k&&o(q!5yR%<ib8(N8XA>W-$ItS-O=XuW>dY7$ZFOy*QLNbSh8 z(^59nwXCmX&!4letT_jWn*l-K!<K)hU-I9aK=E`KZ49(W*Qy~KPp)<vTp42v2H_cq z`?#&}b7)O(Tu&^=lKnO7+SA2$gDiphzJ)W4FC|Ax>OvR+FP|{=7TZOsjUmd8-*$gZ zdM`v-8GiJ7;GJ3iwRpI?HF#6j-mc%Co7dtLP_`h7U&Y;&QV_jm@JcvQU|PZgjyo~j zmA2a^+KH6leYzWVlp9hE>91aTruwekcm*Sem{dSY$UYt>7$QL${mqN*rIlHwYW&`~ z2uDY<sEOP(erz>H^~^WA^XS??vX|cc>}RWVn05p|Ff&$Y^Z5EYV*8VwzDjl5c58Yv z{SE`e7xuK22ZxqUG>7%hDAT|58?=2nn`zMl`k|H+#JUBqnazvi#8P{4)P_DhNOj~& zK-OID!Ma)(D{SC0k^{Io+WHf935{rt*>F&X1qv_m?816PZ(wSmc)?Gert*qgo)_+m zoO(rRN$m^Yi%uY=9WT&M368G4ZC{oC{Eob+LcDjQw%bzPa*D*u|D3<!5lk@(xBSxN zV{Bddkwfabhx2a?BG^Qae2e{|@gD5YbXc1<6}6qFh}?$V9#lNP8o_sO_NMvsO$0kY z17_KXzMWu+0l5PGOf`T`aIyhYW7pLk;W72-w#%d3u*uQ-4TrY2<`+Mu5wjV{=eKL0 z3v7oe!HDb^`I{4X7$VGR%9XvAN>Z`fVOh28x0{<o8^{tvj#LCbEZnntoE7VH4#m_# zzDFj4cOWi$M;31eq%biSjEcMmye@uJwG%m>J(L~js8@bU>AcxP?PDrdoc_|#@H15> zvy|hDtjl6mzT|!^G`W7ZIx;a^ao@@((HPUHOFhqX19j)r3Bl>zgATH3yj*0!QTj%H zY+Je0Agll<iJQ%#NWhJ+Kqc%pf&89s#2!k>5W|42)SS+=^+p9O17AAoY{Sv7-#`T- zK?$fU>1!H~4nADSb104P-*E-#g2s`nryEip23IaB<Lm(0C>)BJJvTDP^AkR&gd0po zj*RM8K?c;nw!rdvccH7-9(`fJQkktV5CUxJUlSGpx5h{OW2?rE$4Z0`A$ni1Eor>w z#=3V#)n7kVj)spLCcws@i{2UfDz#tsaE@@)T37>Rr@)^gOanmvjLdWdXB}|(Ioj;& z7a}B6`BtLd<88+G69<Pf$=gKl%cQg4VYf`fQY$Z#*Yx&$ZbO8FVpp_BmKFcd^9uBL z3A+eUYt)ldHhXZyzqYK(a}ZZcn_s;Vsd9~1YUUig{TXseBF67|eS6d8OtI6>zQ4bV zvd2lDauRqHi5@(pnN{f5a<8dVsGxgY)Q6>cdaViM)*XhXD8SWFGG?&m3w($0r9k>w z*!J2Ckt`vxOnG{h5=hol6@d|gCF+qy1b7n<A?6)DOkLCy>=@u=cg;M=q79n;oL}8p zJ*|0mNY`IuUUW6PV$&mi!N;<=VYy~`H}<}gS>myH=ZNo*C>}#pv>E99cj}Eb@uZ+5 zmxtS;;pc=cqHV$6p!heZu*6!pN0<p^ftOA*?NN#%iD`nj62_c=7~5O6t3Ch07#4(J zfKE8(1uFI2IuWr2OKscrnVT)^k`;Mot$PN6dRkkwKcIO^rebk7w#nZ{eL#b(IxrC> z)XeEqk(vY#5a!%3vV9bi*b0oJJox;Z4X1<W4^48Hd!>JNhqIm1K&hN`OiWcPpGrtv zSF-&fbph-oduFDZdAbrD?5gg|h?{FNiwk}8ocUM!db7Z_^vQJ5R{wP5evCQfd}mHa zKRr{Yjf*+ekM%FPZ70kue;E?+FxhR4Tc8|Yr7gc{O>;DB+iF*{SMzxLi)ztk)VF08 zO&}dB63^w#`=$09Bbq@d&<RLno@WQFyh_K-kbF02DSqtQ;jyA#9MLJQ)8Pyr6ZrPG zdePh5jsUr|En<te6InvB9)?x+Z>%g?Th5on>hrCPu=|e}TjQ*AYo=<oP%F{Xa1bDv yBy%gb{OOQAz5InY3dr)l-mc8E)6@F!{|X)M-=oX@|9}7YDTV(huMy&}$^Qlk<Zq_{ literal 42560 zcmeFZ2UJwex+c7voO6;K1tdw%S(1okkSr3LoO2QpBng6ml5-A{B})cTK(b^SB!>nG z4Ky%~-*?XW=KlB2J@>9zYtH=RX6^2W+Eu%%-g>L*P1QFuH_HIQLq!!u00aU7PZ58> z4Fu<jioCp;hL*ab$|D8D0|3BLegt-MMj`+JM<)+AEhRbnCkBS}=)(XKfC-=hyZ~Ts z;qIcQt*8SaE)@lNdUr&oTl|mfbP7Pc2>`~qlr`w-|B?PrAwmlmHxB>+X&`d*TUxnW zAaEE0J9vAz+@?bjnB<xLEk;7Q#cqfOA~3}*w)q2d{>t+Qmbt~2j!u?{JhyFjv2?V& z#XSi8%FELVfsr2}aJU!P${T?v5SZD)(-DloFa#!Xv@&-G02G|tbPp>FTLiw3z&LI? zTJi`i0RR{{Hh;m-{(?QMd=Tvf0C{H@UpKIgtp`1u1v@>jsHh0NvX!@km4^qHhPj2k zxtk@uytAW=xsxvd{L$yzQUK8{ZRrt27UC5Z65`_HMv(vS*Z=b2-*f%1!R^@ok>jW4 zuQ3CWefdY)Kg#|i%{dnUL{1R2iT_91v-bed_!0mpX8)1KoDBf@VF1uD@|XUQ-SUgA zhlh(850AIEH#gYIg8P=Bf4~0I41dr0AA`Tt$9-Gh-)Tqxz{=X()4_xOmZ%oa4$hu# z^zJU^7FP6J|CWjW+Xes9tiQB_Q_ITQ%FW6NQI#Ho%fL=Hh~aj!1bcv;o#??%|IrBl zH;etH4Y%+QzD5Az{A+;3j2j>rApwy0z5po1*Z{I|CgKU`@8k9eQxCYEdHQr4fABp5 zBcA{I{GUFM;t>BLxr1%!Z<FP<bm%QS-MnrwVouzCpaNI`K0pH80cZgxfDPaR_yA!* z9FPGNfQNuOpaU2Hrho-t3pfI9fHx2TJO{#oXdoU)0#bo2ARj0JDuEiH5oiTEfj(de z7zHMQIba$10qg)!;1sw-EM({)To5tn4u}rK0^$JifkZ%3AO+APkQT@Q^b}+RasqjR z0zfZ7(VzrSDkuk30;&czfjU8hpl_h>pcT*-=m>Oygp7oPM1n+vbQg&SNd!q2Nfk*4 z$ppy;$rZ^DDGcc~(mSMFq;jMNq)wzEq)DV@q#dLaWB?f(nGBf$nG0D2Ssqy(*$~+p z*$p`eISM%$ITyJSxf!`1c^r8ec^CN{1r3E5g&u_qMI1#LMHj^a#T6w8B?cuGr5L3F zr3YmkWd#L_a*c|MN{z~aDu$|p`UKSm)f+V&H5s)KwH~z>brSUl>M0r;8W|cZnlPF& zngN;}nm<|$S_WDb+9$McXsc)^=;-Ko&^gd0(bdt-(LK<^(chz&qj#W>q5nXKVPIp> zVenxnVi;mLVT58NW0YdFV~k;JU|eDnV6tF}VX9+VV)|mnVdi5tV~$|{z`VpF#A3yg z#L~gC!+MVO4yzKY7i%8tCpI=VBep2ECN>y51p6KK2kZgt73^~yBAk0T@;Jsgo;a^@ zif}&Re8>5Ti;H^~R|eM**8}%8ZV7HT?jkM>j~I^|PX*5sFBmTkuK{liZyz5Mp9NnQ z-xS{uKN-Ike*}M*0F&S@fjogZK@dS2K@-6=!3iNTAupjip(9}oVJYDt;U*C}(On`1 zB1@t$qCBE*qE%vKVkTmFVhiFI#QDU%#6L*TNLWc!NbE?WNh(N&NuZ=eqynURq&}o+ zq-~_jWXNQ9$yCT3$l}Oq$fn3H$Z5!B$*ssE$t%gn$j>OKC}b$CD55B;C?+W`?$F;+ zxMP3k&7H<O3zR67_b4?fy(u#(dntFRNT|fAEU2QWYN%$Zk*M!cYf<}C=TU#5KB1wd zQKE67c~8?#vr9`xD?@8X`<Awyc8iXfPKpjp_m-}MZkwKzUY6d0{vCY}J(Pi(L7Bmm zA%|g@;gXS+QI|24v5Ikl35!XD$(rdcQx_AInT}bFIe@v0d6or}MTEtcC55G*<?Jr& zUH!WecN_0+vQn_Bu==r<vCgyMvPrYKvgNW(u%oeyu-mg|u#etDx+i$g_Fme(5e_5{ zAr3G{2FEu}R8CP&C(c~XX)YWtSuQWGGOiVFGVVv*q1=t!`#j7%hCFY0`gyMJ3*L9Q zpMQUjmx%WvZzyjw?-AcUJ`27yz6pL@entLZ{*U~J0vrNX0+|9cg2aMqf)RpUf>%Pq zLT*BpLR-Sj!cT?Mg{MVGL^MQVL<U4rMdd_;MO#JB#e~E>#A?J2#JR-n#Y@FEC0He_ zB=RLzB^f2nC37T~rRb%eN@YteOEXBDOXo^MWSC?uWeR0BWZ7lyWXomu<?hS5$<@o9 z$&1Mc%71!*{6OhJ)PrFKA_ZNARE0%FMnxONGDWD8ppw7RCuLM+RpmFz(<(G77Ahqw z(1*eg10VLN;;3q?rm8|7-FxKrs96n3?V(zt+Wcdd$4-wM)d6)C^+fdr4OR_TjTTKb zO%2U7&2=qatpKflZDMUR?NaSi9XXviojF}LT~FOEJ$yZ5y%N1M{RjF9`pZvvo&-J_ zGN3d78+<fGH`FsMG(0g<FiJA|VJv7IZaiheYT{!uXi917VA^JeZ)R>*`xN!5{?pQ@ zaC3F@T=SD>D$mlNK`rDhQY>~XWh~!XZdplMC0K1(OIjybZ`w%OB-(7*O4}ye?t<mP zso+C9WxFi9GkbOWBKsQ$eTOPX49BOAO-@8k_D(&{w9Y=x-&{Cc!d;eK#axqIp>C>f zh3+7C6ZelEBnV;Zizl1sOV1TADX%myn75vHtq-A(lg}65d%lsr8-9v@1^y`h7XDoU z3<1vrRsv-MbAmuY&w{#wnS#TDe}pK76hFs$ZvXsi=>5=y(Bl{SFPg(>!a~AUUn;#U z3&#t03!jb<kH~t3@(TQFIFdi|edKkNWz^^B`_akK@ED7j&#}C*?_zIWTfZKT6N<}> zM~iogpL!$xrZ|Bx!7pJoQ7y6IE&bc5x5r6lNrTA($yq5_DV{0I?;gGT_@4QF{Cjw+ zZR%v2Tv}B+b$Vp_S%ziCSf)&7Wfo0Vbk;>SIC~~XDW@U#Zf<fOTAo+ldcIzMe}PCr zNg-unbm3KzQ_*s<R&h^>a7jrib!l7~QkiGjR=IKcXvKqykCj}NIaOp;(I0>ho*%ZW zpH@%RsMd7ViquxtvDBs46W2#IfEs)oppCYT%O4FsertNz)YUBBT;IaoQq;=On$||z z7T1o|9@c)-;n#8U$>q~-r%mT-ms!_b_ml349?hQNUe(@#KE=N7e%bzx0m*^ZL9xN6 z&mx~2hlGY2z6gG)|0?*kepqO@VMKW3<EZFp^EZia?PJnoo#PM2`zBN-h9)0Rew)&r znwd79UY@a>+5GPC{czT6_Hr&{9(6uyfnXtdk$N$AiG8VRS!lU!<-y7qh&E(y)navL z&3z62BWxXaJ$Zv}qhynBvvo^xYjoRa`^S#U&gE|S9^qd4KHGlXfy}`d=o9FVL$|}5 zqnMv}eij`I9Cw{)oGhL?o?f0s!S29{&qdDrFZ3@qF1_LC@OM}2SIyU|*Yh`yH#avc zG!GSgz}5&2LFFERfe;4}HwfjIAhCmlfPdn2gk(VpLj9FKi^xUwCx##}8wmMVIwCv3 z3qnGO5CE`^xZNYr0cC_VPJ}26j`-u9PKfjv#Ek?XW&fiV<UjFkEw>jMfc)n>(GdUr ztzJXif7JU7f&W@B2%%!!7PdpABmIeI@NeG*+$vDNp8f4x!yN47p>6BzLa*WGOwTLK zeg9TX`1h{gIu?Q|zjdsC!y12=!GYg8*8fsD`wQKE>sY^atlv7;ZyoEmj`dr|`mJOA z*0KIy(y^2ga?q`A^0)7snVUI49za7uMMFhFLqkPFM@PfJBEZ7J#Ka=O!^a^YC!wGq zCm|!Fq+y|>q++5bBctb}XS&PEevkbQ9oKy>w)-q>>}<C$0U@l3SQuEuSXjhtlw_1_ z|K;oE6F`UoJOQ$iK@0#AAqbfebkhUSAsUQ|Fxn!nzsYbQBxFQ;(J?Twun`&R2>>Jz zGBOehGAb$x!io$EL_7yj2vLa`cxBOuHO<i(T}k+!zsbR1lB@0{)%pfy=6~iEiit%= zPH~5l<t{55`#k|cAz=|wG5H4yib~2V54Cl4_4E;<wS}dXwT&&<&fUY)%iG7-??u?l z@Q7EDQ3;7}laf>3y-&@}%P%M_DlRFlsjaJTX#CjJ+|}LF+t)uZ_<3x6VsdJF=KJgl zWOePw`o`wg_Tkab<CD`f*!jh6zd#7Z@^9B4J^MHPB1H5H2?Yfi1^u>PAS7?ZjZBDw z%D{_8B&&&T?n=za_Z)*n?oCd0CngiW7L@dv+czvSW`Px!!`rU?>e)ZnvC#ik&;H)A zfAnh>z(EEf1`nAKkOnRiGjD($am^9ulZ;K^BKXthYU4%wG2ps%Wo9SU0mTAD53XzL zsZVbJgT)a1A~@b_@!+xp8FZQ(VDdC?=LSIV!hhuzIOYu-%r^je0mYExFvc~HAt1KV zu-JZV!_h*43=6achRMD7@Szu)<1}X;r`gC9j!lEoy<3WWbMB$i(NiIKO?!=1ZFfHJ z_W{>*h5-M|g_sv`sU*b>-MWTUp8{}nFTwkUPrFDNGgvYVX+Naz{fHQP^OM_9Z8L1v zD4nZd<aG?c@#fE&?IROAZ$gxgOy-8##@FjEEwrdhS=<706>TpD^7m%8y|vWIRC4pm z3}_yLbFHK;*y81uorsv!nm$`5$TNt$ka$OTfGOsb{M#2gs$fj^3(l2UP*uIJ5N+qG zg6%EHgEw=Jk|zjE8KrcSjNhyy;VFwhUUm^B9o_h&qxSkkR*pBoO2_gW^1BNN@<!YM zA$Pj=`(&@_9*SAC?`KY;hToraPY8}4i?M3hKnW(V04R`upsz`)QK0`|U8BJTX#8#f z{DreTW_VA)YIZg<bHJy7jpDo_b;#x71n?C9EnWPHz*D-nS@9kMfaeZe+Iz9K9@lhn zOq&1=a*pCK^`YiGnyOlp35^N6J3Vo@4?BTZjiZG@*viYobYc6fjIUtc&rXFvW2%@X zRTw{G`v@tF^wX?`i0Im3lo+9HIYV7bYxse7sKBMeo6;QLCsz4v9p3&<4*t`R7(TlX z2lT2Dt!g$os%wo2dR@{iD!kP$$(&iy41ne{g*+5OQu$F6zsmB{1RIvb8Fo-H7cGu4 zXzop&>0tX_2B#G%y(PhApzMydXK{vfUG&SxF`LaN*K{ohiYn)?&iiGbP@yTGhn$aS zDm~8XJ!>llKGCsBGi=*_l4h5t-?RmOG4d}y25uSo7as$EFmfD%k3;Zo8CmHUA7|sl z8tEK4CG<@xQ~tDlGtn`E!|Foh2*e~C12gD&ie3b6%P$#ZN?OwallhhgK*Qi&LO+0$ zpicVjp_p-3d3`ktY4s~j8B#9FpDL4KQ(^PqMlF!L-4W>2OTzb%Oa!^5KOUd^W8e^v z)kyG3Mp<&c!*HrwGoF8N`GL~-HM&76I!(vqS>Ed#VEa-7lwP^}6o2I;PsC(qK)QQN z2K|xu!uV{=-=+@mx4HAj)B*mSJ9gMRLxECV{yT$29Lh`xf28z1#!!Q}4+yv0I_H!H zm8$0rK*!&dteGUPviUqo-$gSFB_=*kR8&yXtH8=4wx#7INe_tzNFCZ<U`09ybETcd zE>qSrvbP<^sF%JM-=YJQm#678AmuPSA=Q-EHC3>dj9G9siZH?2Y<Du}{4S??NB=f6 z|D2d_L0^1^)Tf=x>)8UNpq=i)MPKc>3))lf)+bt=t5e&RV=l0AV%^8WAF0<d_?CA_ zaLKnrWU!9U**Y~w<a7ggbawYy&_2P_2rc)vkfuM!n$D0)4i1*N6xu0Tm{>Q5j>P1Q zaqU44QB=LJ5{nFmBoYRBjoyVpeEVbuxtMu=BCo%Sk{X>0?=3gwQ3b<Si^K<;rlcFz z7`!h-WgSFf6?LPCdK*5he&H8nXU^TLTA3)WOLH{-b|{!x=H{^_V6dvf#r<eq)W$Y1 z-S=_3QIg*0gvBiO*&9IUYaHbe#Ddvq*P}oK%3-1DencW=kALi%tT#(yiH^+NKbEfj z_{a~-3^%VS+O^PU>|po#!Ctn;X~Hn-cWX4N;_s7KMt#c3q=A<;!uMrLn=GUw>%Tre z73FGWrKzh?aEWyF6#Vp>lR2yXB`@{MGz`s;AXVsGQs{Quxox@i)nr){vv9ncyI{@3 zEN~cy3f{5_X>I*0^boHG`~W(6W9Ntf_A*<2!|^#&BVWIYi8HPr`L2UbBd!d1Rcnb1 zNBP>3{PhqlO#O3z#%b-ONW2KhTpZbqnY~zenrorqB3F-#TmcSXFo;b;rQ*z|^8S^L zI1+&EB=Qxi9~vOL!gX)<^SD;yeXbfzJ(Sn{8z^!v5*6jheM4;@_EXT^(OzO(v2X~} zoFEo9FgaD(u9sh^eeLGOTRle`aIuPY*O)m&yn>r<$imxB1;>0LJU6#II-##RIw?Ja zUYk0TnK~a*2Kfth!V?zcRSD?fn*~q!kzdrk$Wpv4RL*TknBY{wo|7I%3em0cwi#VZ z-#9E}djA3HR8JSl)2f@ZeM*j7@{UFS%`-Fzb7KQgH;Cz+k&}cKlWh=Iirh!qoPpNg z<D9=JU%vFlo4Qus#tp*yCYP8u%1c*4=7&%~HsMjpaxJKr%h;c_Ds|V(z+aT7?({MX zj<jBry0@6<OHF2|x*&n~M&8w(;pKeTr+ADb?YkN>m!0~BN95{?QII+1>emOjzSJ0r zj!(N?H1IGUpud@5j62oOrqpjn^|!xrFy8ROU)lxN^skpGs-j6gtkyn%ZHv@6!ywu{ ze|Xk0X1Q?#&=zN@)W;K9wqwp?e~&ow11eY}9E`0BC%tH&d#dKZ^GlC>l+=Pd+$-JB zymq|v=-$?W(uxCwza2Q0vm{^k*bC;b`+VM+E@JN6imA)5!ke$0+Dj)xGfJZ5!{BZo z{ni(wWm5C?tyYe_vNu`$zGk?1`aV6ay`fg^#Bjb--F*Qc=SsmnB@?n-3LdVCy{F+T zsiFZNQ>}@__Rg{Iye2q8ik*3;>FvE3Jc1p3dA$Tz@L<XazZ^MValQe9=Wc*kd4U^n zXyXmgv3CRXXn@*(Ui%o`085BX55l(Odjm*t-2exd!G|Ouea-gvy5|8E62|kN7j=01 znH;%gMc<r0699;wY!s64lWNONcasw>Z1K)No`y=Ujc*?g^ghjGZFrA`fywdXJdS*D z*tIEIXq{K`y~YA<V&8mL(~Ra8LO}7JZF-#ji`=#wUg%5vwm=`1lbiaXpJq(M{Bj)R zBpbQX-*M+a(XFLAL^@(B1ZTzDf*!HI7SQl6d8^w2yy0`{iEz`T3<wqe9v9>K<kQBF zA2n)aQgHP61srG@dyr)Cirqk4qsKX*Xy<z)&Ljghr2{!#LsW{(gz0=AaEc}fFg6ub zO!_G_WkREXvJyM8Sr0~|CairHO+rc)=BEWwxon(!^-uOR9+RZIs;-C^*9V$ZEvG_6 zD)zjv4ZJ_?tq-^z%?(5i#W7)QRBkDX3=Wr_Ie^vd-9P+KQd6aN{-3RV`?ux!-)@dO z)3Bg}wmYorOdB+)z#XP_<PAv$8nIyVMTE=+1Z2Y_O7Z}BKKw;aHlC8+Yh1Kr{^Zwv z*BclQ*sA6E-y9`t@UDw;Vk5t&i2d$inAu#?(z9M%L7!X4kHv;eL+^R$F)fUcDh!lJ zvmMCw@R`&bV?vbPWctpCoZEFoqE$%jN-znQ!a~P}*VQU01u~7}m8<Ekwa7iYKFvJK z3FDezCuv)1tagRf2?*{v_s$v)vb0B>o*5O(kX3V(e-K>kDt+{Mh$nvY$9ojJZv(21 zOn9R_a0~mKCpUoemYL&Z#sMwVz?V@FYuH?Y>t52l(7gn@qG6pMWl=_zCe0bHjU4S) zlDJ!6&rTcOF&Q1&uyMH%4;<y(i+$<#_y&0Q@iMI`)>pG7s2y~!a*i&qa<>ufV63xm zKCWqA^A}fX6ZqM5ES%<L%t!}UXCr6K&;>W;P1CfWwE@(Q1D6@S)Q*Ff87~0^Z%5Q0 z<qeG=dR-mGC>Gf@_{8pa>Y;~Xh|o_<dH7cd7#SHvIy=gJ;1ReiRDb7i=c#b9)^}@! zY)C(R)DvAKrT1X<OH3@k4rRgd4e;!Z)R6tFqSsHDRg$eA#Ixg&TN6&i%Zdij6kzaQ zwOLGQt7j_1zpwa5duq(RC&(Sl<n2t5a!+h2`mD13_#*wQk=%)_jpzhwYpR3w{;WPQ z-`D+Fhd!gTBX0jq*uvnQPlH%mn)((o#u;x)ifjoK@<(jS-cC|N>FVlxWcs;-LloV} z?u%fKP$zsbNWD%fe6o3}IkTmw*C6z5tsl2sl?0ri(p3ytCir<*D(>$O=a-j92kd{k zy!uM}aqp~c3rA$`&6#+V$Xw!?I2OR4v=9S=OV!z6x7<O81+oEKuP>YZHDiS4*RmkT z*~oVsn>y)wca)4z=cX4!aDI4O0Kxbbe-o2|Uvlz<1#Y7#E3AzT87+`-b#k9u0Jayp z__<~w#afS(@q4Q$j!S&yJR3kZC_#Ohg=V;;ueq?3tMjqmfH7poY~H9QPs)f-ar4Rj z*F-vTZ%aCz>Wk{m9Bc*;j2G4D9v_vTeeV^Xc_)A?@-tHiD0VqWUI<m(-AEh}%t`0o zwWM1)E_S+1WpR8_D89e7r}(GH9*_2?!t(l;z5@Jz#G;$;|6*D_C>?1B8N~Qos|V`; z@7L-JbFvQN3Y4^E&+w;07)r7)BX+cO_^%F+voTBWT*EGOJO$`nu56JgQ}I6r)w9S; zFVQ^&QhyivO^AP%BZ^DwG(f-jE5F$0a&=3F_!oZ_Z`(?(T#TX!-tw2-!|>pCskb1( zUu1n<_+p=~0+9b*=r<w$S&k6UEoB3K@t5YrXkhnAHiq3Te@!Ie+yDWAY)E#u43<Qr zhCVNC2!#KltBS_RS^rqb-QT5dr`!JqY8aEpwKIdSaVC%3xL1{@1&yv$nEibO%B(U5 z9yy(>B9Cr__@LD&^`#lvrkcoqT9mG?dsNpbOv(NPV!GQS`U%eV_~K<VZPu@OU*h9! zG~TL|JG(v0HD)qnTnQ_9yqbeTD~nG5Vt}#YcIeX7#Ccf*HUG*Y4Dr84zcRRo_Wt;> zGzrXKwk3S)K(IfFE0aZ=l4642ndLb(?R{J|`^vz^9>2__tCUistCtqi4w*|<^?irw z56;gAHhm}rxV+HHo&U3yD(VBAe3gr-H-++x7;}{v*jw5v*#$j?X%A(WKK=Q)gqz5~ zdK|s*za%T>mVT2#{WTvFer@M~+OPSL_e%zlA^1hdl6$u@fCkFsuUT|Y>{bY<1L6KI zg;-PmX=-pJ9Gb<ovaY<3@!NPMu%q(U9s3cbCc+<L+_)t}<M?aK`-u@G3KvecAo9rs zK0Qxqe6OE%8f9@(c>{E(UcagOewFuNVz(Z{zrJpWp3az7ni4n+j%lYKm*6?9T3M{B zoWDwW+}POm)B5Q9>3*E;xek#py$eUaG+jHb0A4a>{cwTTr*ki_*}%>Oi@4^Tc$_SE z-D~>b!Cv_N8=xvs{swrEItTl@ToRy`&z+OqRC{RsPU1@Jh^hVz@6m%Zy{#90t_Sj7 zMO1rL*Ka@Dod_%r9f@W|Os7a=x1(=2pG9)gep_4*l%wI_uVd6lVhjF0$kcde&64ut ztaEQuIk~YE>$jCQ+t!urU~6Pxm62cSfM2q9Txxz=5aMXK0l5Ko4@=T8s4jyhvRkWl z#d}6S+Gw-|J;g*`qEWEiY$o45@m;Y!@&9;ysZZ#fJRE4a;-ndJp7{BvH*xxF(1;yN z0ldOl)fRpOEc@<oSZ(-UEIIS+RFA`^A+hi!WQ0O|$H(3?V|QAkms6Zfu40`eLHfP& zS}3T`kFv3H<WOC9lv~35Zf$|Fyd&LgWq=T;jPeR~O+}w~?aPA@K75)u{zF~?6P{C- z56#fbh`{_U$Ktt#P7A*$pWV6L-1dVUEZs+=cBB{sbfB39&^av=R=tL{iF#Gx+J<p+ z$7w~dPPXE!fX}iLYj{}?-wVB+_f~gK7+dU^%0Wqo6hP}QI>ryqNdhd&jbJYzHl1pw zL(<mm<obj!CeWTg#{W3X5iEP?l*iLs^@`HE#E(khTSbL+<Ey^tweXnQj8AByJA;B< zYPlmTOC)fNF=vtHy6|;T8PV=@tS3PzZ;Yg9KLqNpqW5u03C*7Qd6r{+@57eSp}rRQ zvA>{nc>_SPUYCe^#a$;Af2GrfrL3yssjSi<>yICo-TlcteysF?IIxt3nJ0~1JSh!* z|I(upqE6`3s0ni|vNNTIuBa0(`GQ#rvjdW9SR=gk@Ac}V18#1395FEcLyqota+oSw z7DQ5UPHpNKBfqX`+=&^}#WE%xB__RHQOwfapo6GY6=GlYG6=8jP`=>>Sw-=K)QMET zRy9Yv-Cji%&L)#4(VafgUY&$VCHt}fIQ-&t>lx$XYoi^Qudm0!x*;s06fO&M%<Fhn z)#{&b&yMrvy@2oWKt({WHSp^DlO`tVc-97vIY%djy`+k<65ULax9oc#ZRjcjN)G)W z!}N3E5)+OA*v^I9em@2jiDw^AkI6irXpK2{=CpX7C*EFvLLSC!5Wtl`cxOHijLm+6 zO7?1pd~H;c@@L29GuY3rF<8sv7ix~rF4e}rq*W1p2}@X)ApGIHU8=HyI1sWEeU(`^ zBuS+Yi!9uVFDKJ$o=(D%Ng#dj6qGnvWmUxp6TbnNB6@CsOMKz$kb4&SOL=gJs67dS zpH*mE{<1JX`j?SwqUsd@(%<_6Le~7>x=V@{Ma@nb>@zFpM5?yFb}7xaOz;Qp_4+LJ z&AmrV5)<r|z4Qud@=Z@%lhKvrVWCftrQ6{)Gk&RHGg)EBM>`_~un0)+M759G(Lwhj zTI{UQ1mijP2`4ojTh@u|hP1HudGQi-x_b%R$4&71>$eJt!y)4@p$6~HSmszWTU|}K zGU~|m;%8^&;^(?TMOI_IO4%p?ylzr1Pem+Nfx<Oeu8i{0R|O3(AwWn>NC>`zK5<0= z*XTa?v?lZ*Z6(8r?~(Vc=8?LjJpH+Wy6f{S!vot=u;idx#cPgF?d)$_%`8?9BK@2) zCCnVDGB^wf3WsJ6BaYt1^fS=;qVbQ=p!&_<%Iq9pq?mnlC5%=|9G?X<X*Zg?7cH=N z%eu6dA&KEY1LG<=a$)k2#>145x+}dO5*706p{JY{Lz(>bed3le&u0jYYa=+0#LT%Z z9l;ku?~6<p7Qh#tNd;e5otsiI$b>VRM7@wXeI>HA5qohL`^xtdo;}fHn!6^-LI?G2 z>5kli-fyy|8tr~rLC6r!hyD)S?Ztagg7G>k_4>uY=0#_oFUr>n0o6kUBD5z{`1qSp zyS-BR{H?=nH*VTJuJ}hw`pB;E9l2lH2BzWZL2xI}C}m5>$^6&mv0d&1M;`87N1K*9 zPREU8QZNJOeJ|cS#;-q?lnb;;&0cI@FZNAw>{R4J)<twJ-?4Q?Jms6A;UBB}LncF| zTEg2*nSCo3dm2|{yiKtBxjA)`_Q~PZG!qk4&_<&%e4jORn_@jwVfJt!Cn+KaJ(T^I z9QtjnbN%Uen1xLB_&z6Roh0{S-x_J=B9-!(sO<I~nQr_FhpmQDgYOECHaHqaa=9Xq zNTggg3ao4wfaayapDMhbhnQpR%ib>yLV8nU3({#W(4OLPX|-XQ^SGw<qJL!(Qb9L2 z1najk>%U>u)6asnK(Znv#C<$w3e%oUV|Ia44SjzmZqBSoo)HAby2k|{iubC<@vB4i zQsH|s*h6EIm?8=T$7|Y}&gP;lO<T?DO$D}OS$0z=9FkH?*$L}bF~pmEzEWKu!(rc7 z-lr-D2-{cqBzuZ*6g&|x&GL3EMQ>NL+RDO$3!=`nPcb$f;uQy-tXK0kx!(XF=ePJ> zEzv&D8ecLtMlwdCkrK+4#y#cGRXE}E^rzPCk8NkQ8;Qm5oP8XNd0A^Oya;BR$TyBF z5*&teiw)--s`u+J-pd?Y)vJpd9ttcaiDB1sBcXLY^Of|iGOQ99HM(S7osQzVixTVv zZfG2Bv{-vVNN4IE_plz7>^--WO5WIJZq&p8TW%D<(F134U(I0ZTa^9)<xC|>zQ3J& zWaOyHOqJ<@<HhH(_1b=p<Es=-<7v`Ctc1)vp^M~dMx#cp!v{zlHlsh|9_gsD8}LEq zoxatB5BYm9oQDvEP8oht(HHGbGwPiGAe=4(yoHcqB?yDPxSRxocgt3~jqE<O<>ef+ zd@`GAEDf*C-AY|({C*NEHFiO_Z~UMJbUXmb`DSv*u58=$gBocMyI^Z1=VM`xJTyJp z?AP^nxAt5JF0|7FY`C#<+EMCbx}4F6VaY!$dSYuFA1o!wz6!yA9YEiGP27RkTefB2 zZP!d@5FhNPY?M_(fxK9gB+8^9FyiuSMKGRz0k>H>p!gnG)SX(<w*5RM)`&>y`iPqE znz7mE)(}JXf3G?3w{7X4u$hg0<#drd8evo-u`^g0z{+-0Va=|hoR*@urxor<Q-+y0 zaA3@RZaHK=T&EcEY}CH4Zo3G2I<C@2kk(MX*G>;t7=;s)xSFYOtyj4AR;F0DJAPR^ z?!tR9zwF|g*z3DHB_#|CUv!Q-pzFFozFJ^XOS!tMdz|XDxZS3tf$QwU|7lkJWvbCT zl<DTSeqqCm)$>N;#A!}HeDh7QlgiS@Sms9FB%|GDFte^%)|ch&L)qYlQ_ZYaO}B&n zFG1GQTK0BP9NP4`QS7fcLSWU|+vNARGFhPWrZ%rU?mU9_;7N^RQY6#$`US~Xcpq!o ztv>E!eG#Bm<kKS4I9c~zSf0MSZz+hJ&X?ND{05lbw0COV=`ls=sGh!iNY}kHBhY5* z<g~k?<;c^vptRt88o_#ax2*qAZbKP~jtZu2ICj^Ra($7m-mvlG8)=EBW9tp8^{0|T zQ9sQOYdi6e=9U)cAB%O9Wr@1@BVk#v?r5k&4(|9=kzM0L=Pv|nulDp!)Ons=<TJ5E z?yA$a*{+))C#3}4gC%b<MR%E!OF2ggdhqtw9fC1eHr(O#IL{Bp3-OIsB(cW3rKrb` z9VG}IZ3tpjw)@L?&zKVezgoiZJIPFmqx{2DO_YpU&#OGeGp#b(#P`e;Xcz3ubGXZ? zHbbM>N||pLz!kVQv|nc0S+T$&adQ@uF<1hQTYE$`u^J9n4BUR15(_tjuC&I{&Pl%t z5Glg0oKc9(HoI^p*t;Hu&!p?;jmpr6^5JXPi)w3e^ScTKh*oWxvW(?U)KOjfU$RT> zerS-93iCvtpeEG6^&yzjK=31o!ObzL2m^`H5=`l2em`L$cy;*a)qAE*{%!P?OeJ0D zI3%{cBVdvCP%vjSzzpN+N7x3Y^rcIqBKn;^_pNw-<D(a^Hy&6QRB!%ra{b|iNMiX{ z_fKf*+GZpDwqXC&o?p6SIF+%oK(wPfRSBv6&|q?;%CPK@x0V@#r?RXGi49hRw!ic) zxA*37G`CAz^V$w1!P`{_yzjP}JsLYvkD_d=ijr_dHnP3nn3iwLHBQ$pu?l<lTJ9LQ zuo0wa5@)nxA@!27D2G2y;`V&C?-2|_!*?!RE9ff(4fwvgtUhK?GHnlUqwE`=?HZ<1 zrC@Aqcqyku@{m-+4-nHi*4KHKSN)Y;)QLn>L6boN5jpvv-Ns9?aPmCZ=T7>`9IKCE z2Mw)ZyO|4@e8Nty{f{~Fa})I`y+owZU_W)CUbW6E>XF_^Pu87zR1Yz!EShQZ-rWE= znHyEX+Pm5aS8gR;k0EUaM3Qm!(UhY_b4|MWn|^gUd<~^%qNF`A?VjeWu7zhK-KM-P z{!5q(9f<|@+H|SkJHoMFY;7)$i;Hfkr|5Xd6$!<hmi|R>9vJ->bOYa;Bmw?2qy7J% z-HV!L+d_VdNuI@sko7qOBPLaWG3tA4ckWhAhr&Lev~$81;Ns8r14whe;FX9`b6tH- ze?@YpE0Glp!z@f%h)=6&Vfyg;CuHLl7hJWflcnQDx#dzG*e!;tknZT`{e9`?_RQ~G z+d4Df7-KDBb(v&Hbrk{RY@`em*QluKA$C#mmn51x%9;u!z(0HI1$E@j_`FbR9y_u3 z_*m1VmY}$lgI<ju?@wq*qQM^mbB7>80Fic17m*KRhej+!=d$gNZh#@HJ!gj9olmr_ z_zzDF5KapHOrPm)GAVB8-f&_GJ0|@C$<WzyeBgJIQ$%c@=IBH)TC2H#G0VZr0KHBL zqudp1QOFT3k}+#B)17111{YB~`J_z`xqyM=UrvrcBOSi{Yw-C0yDs#hL)tQM&X#^^ zVF6qC=P|NWeaR$Q*6l!aKVZ3C7?yiIwe%Po@pAnLt_nfp<t&>Tn+dK+zj$hi-}73* z1R59NAi=S6sYa&~XI0<4%6g!@)U)fK;49b<zgOKlRpm)Xw{u8wBIy9KL3Te?Ww@?D zZ*l3Leld}H9JN(I#gTr(;{7!D53!I9$c_51-IeH&%;y1HHjokQ*i81(S^+90_^qpj zi~gxtDF5@Fc;58sK!v-N<$W0$U-6M)sr`LYq~rAqT(yqA1c6~|eG7N+n?Eo?E5Nt> z9pU2@4Abi%$x;nofa&+Ty?0c9P~UOQb`_Rxu>^Ps?z<pNk(EN_yTf(+m48*Xlr8@c zy6;HsqJUoptN(WbiCQgI1G{~SdM?nr=k)Gk=L&K?ExBxX)A%{!>J0FI>Nj0(GOqx( z7ySAqnwlUdLCGm9+cFV`by)SHTD82gb~Qr*qwSEi(?_?t8+?bIC`g`l`h`BmPd-*| z9X~O~zNl9@nu<uzL(mJjNc0UpBSZ?|zk9pg07UTn5=P5&SG;&bUvuqM++R>{eZDW* zw45bppAD0S)UMKjwl30Ft+U@Zw7-g>*k?L7CY)nk@6W@kHuYWUm%@R4_bdurpzA#p zoM)=@uEt*kU0=ZaCzkaD^Co8BHq$Qs#d%@8zV?4A+=o;T!H0iDguGalCISZ$e=D4< zZ+VWIFN24+?j=d6<M!l~B<%x>yWcNwxUY7A++j=<50@1A_L1!m@AI0aPMYDRi?biK zd+F2KD;>03rVLQQvscGve71W`Nd@)qeBFAP?LYRcW>lP?j0gySP74Ha)TL+_|FVV} z>qV}r8JWSnb73iR(CIm+dZd@TQj2fSd6OBkzF2{(V^)FmX`9>dy24aPsj$9>DuUBm ze~nsE_|qlN@fLBEg)=}M5feS49_4Y}+akm-kqsQ-x78t!dJOJtDaCs<O-|g#@Rgns z`3fBk#q4U$oP4UHgU0m+Q^ShnUg(cXkq6d3PXF-Q8K(R+&Q(hXUKw(NUQhLWH|f?e zX#e$?Cvv*Gmon)2YB1)a&ojghI?tJJU%bEkK0;ig=M(Phr7MREfnT9DjT#+te{JW# z7hPlQ916F(0WgKyAQ{5DbXM2htz?-t866`7tX@K~ZDN*J_PNt1>buf7+p|@P8&}~| zMp+zQ3zr0li2GHMJ{PdiTi<5G8=x4HUhuiGt=*se*H})+ZAudd(eGSH7MA71*)J7d z&OdvCs3$CHLAYeoWdH-N-~h9MDD>U{Q4r{Kw1lwlN`;rtqp5GIV#e|=WoMxe(}_ey z`t8cZXVS`ykDLDzBqaY|42FC{1ncTs>Q9y;JSuWj-2QgZ7=i+(%xFztOHg_gB;bp# zns)LZhPwHh?V*;8(1qqguTw8QrHM?G3glmxz7F>5zlsleaed-ps2fIU`W|d<oyJvR zuFPnrpaIauSBe|ELg;1*nJn6;yDl`IinB}(?!bCby*<AvfZa38raBc!^!j9f=^DP? zk*)fx>{54!R?ILIR(2JNxPGdXaxN@^fxf_gXH&2st`Vdm!a^#q7~D=cvBxz!$km>~ z()IYWD2wXhrx8->?P>qu<d;X{Gbi^rc)DhPR7FZinwLLux)fZK)5V>1pH%-SO83%E zx8BBD&U@n4D8QEcv5Lf}@=^sBy+VVu$YEVO**loArpL#V{5-`R1vFy(LP@9P^{nvB zo^5XHTUb^9?DXqa^<0=94>n6>ET^)WmQlzqjhm>P?5027B-f64fv5XaaTVK?gytc6 z7a3O>tmq+hF)vT9drx}i=uxd)*m67b^7BpU$MxGn-n^#YJ3Sp%mfw>p&v<L`;je*< zTm_%mvE{x)m@TrAqBuC@uIGOW;_n^oE@Xk7s{^Bh0^(MF3J23wisbX_MX?nkj6r|h z@%s*gBLxmHH-#x1su+xXKI+J(B#kUQpqaS=GTCuvymS=3pO)q^Cx)O`_%!y@QTl*A z$~Y?cO~-#6l0^sNUgh5(S|_t|5U{M2K~))$3au-Lvsc39`wCXi_dWJ#<tUfBr^enQ zepxV_g0ug!)ne|A$#wLKBl?5q&&E9bu^-U={EUinnHZC}Z)dVru2pBCtql7PJ<}n? zXjKCve?)g2Yo$`p;m!`K-<-8tXxq!3H!zlWMLw!|urk>m=4u>NkPmLoS0^hfnl)9t z0eo#<TB2`bRAODOOpZiwGf%dYFSSPpVfYG6KbsPMx4Rj%#T46J)M(f{W^bXZD>>r* zhpAVo+l)qbm4l<Ws6J5GbUa<Y7+e!?O5U##9EmUD1S<~W_vzM!>c23b@St$8Zcn|S zQZSnIia{S3oc(m+U2fF)<80G@Y{UImm{)7ZHIL6)T=<$Fexth~9Uj~{5W=NuFbY>G zC`^K=(Zm~flVn<I<;CQ`$^0l4QAFD+SS*SE;NOec>w?=JsvU%0)7hn-rjXV(Cg(~< zx|M&+7`vDLlK$9YG0{ukBFfE6$@ABa)HHba5VtQcN3AH9+4ktcNC_r<M4Clbot2Kq zc*EN%X{-|w5nWQ|^w9gS3qd~LpG%$F5@D;g1bL2;I%@Q`Si#lolNkf1u%CrNA&pP( zs|bK5e#K8!F-%F=9};&8ZB_VeC_;bqhr=gSlfSLMn>Q$<b~dN)4pFA#@*7D_%y;YQ zTz!Mgffq(An^*9u_Ai^J48b?O;%uZ=Z-`s0_NsSxEbXq-hgT+cXm{`ECkve8PpV7D z`t|xA^j6_-rx3w{do%CUlcz34a2s0H4dYJWwOGZfpS{|8c>31zRQ)2DwQ7KK7VI>A za<=P7>XzCcA&^1bb<N*4wmUYvq1}+_->7G?KKMiUYp9D@=DSLt0AHw~;)eyPD(7-S z#7UF)0)y9+9hdLs8l|E4@^bJ=?WD*Kr(d-`;Vw6y<SKBKR?4VxSbp*jsT*eSk=fs- zO~LB2=|URjyi@J5)Iq1?nrN3Ao_Kj5MihAOV*9gKYeQ^)!~48v;rgEw@~LSd&4{JC z{XV#z{yR)iLep_Y6}9HlDW^o@u$LorPK)CGFlpQ(Yo)ZA^i=TLbSwG|V0M~yA-Gcs zd!PG#(~PttzyiX9r&Q&mx8J=hTu`2Tu*=6`Wz8ll?%F4Fa*7UyJ6*a<U$Gb+if%3? z2wf{*JKX@!<)(YdGSLo|B67Y3G18ISHWxr<77R_dH5TXRYOe<0Os~)4qY%Chgf)Sk zA!a0S*TdnL=u!2KJhBbplWRsQ9e!bb_dg_m?oIqy3rc*xqj!w!zOwO3LN$LbKpcvf zSib)BOe_J64aN<v5<hDd<$2Q;V|hmE{i)-~rGq$l=ViooRLR35l^ekS$ZUFLbUHuv z2;S$zsAkiB1CZ9oMVwtTP721}3#7LbO3|;0XlZ0QC0t)+K2l~aL_;^+khVF%3QXyz z6>hm+oL;~FGG$YLWZk+zvxMTCSSjTzzQ3BfnR5NO1QBJWPj{iI8$__9Cq)O#Tdl&` zK2mWk-=6)_*q~9z6A586b8&Fo6o0dq>bf9R$pcL}N2&Ncjj*zgnQi;y<sc3jl1rZq zcD$*|2to=_Q^yHl8+WjRddHkl?9F=HFNmxDta!!8Pc3ygE+Q31d7-ay1H@rxU92q) zOeY2nbsRmYqdTjVKKB1cb-Bbd!BQ7!w)PBh;1=$9h#yu}r$%=67+M%xjH_rO2hSZ8 zsJ@z6Rj^)_eC&FTGYgA^9F_&Be1yhi-ha_0CY52AXl4>!ytnUU6{N?O0n;D}Ro<xA zIOJ>W;)#U%yq=(d-+{0{pv+gjuAExC`d&RlTXc=Or|C1II)(_Fv%Bn-;)1SGE`@ov zJV{oat~Tm47kG->>s5{M8J(#zuvA-3siE<t(X4BUhj;Jm;^)N_#L@c>`28#7-upi4 z{i$eig+3<%X4yO3Q{^f8hb9XKUt{vlBL@|wxiKKkJXAA_#`cXhKK&3jXjTN?<hWy9 zn=I$X+vh)96U5fiZvd<{i<6QST#HXZ^oL&W>AFWi;FQzq>f}{pHFgI`4ab=m|ARYg zBj$0&C;f~(5?m_>QKt9PAnWPY6HW&&z{0|*0Vytn_Hl0*Ku?fFMriyG)oX%+i{NxA z*$pkGW>s+ft+ZhD084#{Q_D&;57|7dtXuEGdoH`J)tS%M#6UNG@Oi6wZDt1VVm{O) z;+53Bts0-MpS>6Rk@l{xY-8mfO?qs(k<_K{WwqB6WCx_Dk(oM$N6Ldn&yMCd&%~tC z0#~M9JPwC2U=Jb<WC@L|4C2Y`#OTF!^H72j-3yQkAYQd~E<f1iW--EKs!g_>u<+(C zWQ<ZgNTU$xr-fBesvS6AHC+qUH}B)C&b4=V{|LNRi2FV%$kr-<NGs{3{9|HVS0VGs zxZObK6KCWGACa>vby(~Tz&za>e2R@&D!y@<x?hUfk^5Ce2C<mg3AsI)A1TeNuggY% z<=|L4b4|~i<#((eC4HnGfC%z<T2^&MGMQv_$rYRwyv5ck*jC#3Q~HWMQy$t6`Oy)6 zO=hcM>cijn)9A@}wuFnNhqw-(`BMzAL-d!TCcZff(nZ|>1OZ%-ITP9q6=-EN_1N5q zkA|8rb95t!B%Q6__+r`&dLJ1Y+!Ycc`{B_G*fim@J*YRN`Fnx|lqae{N=ABn*IB%H zw{&SBZa;O_thaV?9}xk#Xnp^hblNdcx&r<RE(`e>V6^(xUvVP9m+r+e?wzR_u6np$ zig)5F_oBs7anX-Y)Kd}NOXQVthqOywygr}t$u^r~%SLRx&RmU+=Ts?%l$eSZQDy1V zX(h|>HO5<om+>P_*_&XID``dtc=p=Smy*u{1x#65!-9@j&ka&I-K2-b;y~N9gwt^2 z8Xu}p$%J46nJ)oy&2HaA)(C%e;h3k<r>Do@T-t~@v<V`nMAkXuw$qizamTqc?5zZi zO;v!QiF_})6wkDPB(te1+8dkvM<0q`3(c*oB}thx2aa$0Q=`EdC*bNlu|X^|K4m(| zM>}el0-AeXggpCe=c{rqFF)xqxDJ*Efesbox~MpoFnq?uhnPc;3<QSi>Jw}aa^FO= zX+;IOsGv}Ll>{Me3rjJ;A`kUW_df?p88vb7I=i%3#tJDDyX<=HG5e24wNl0%T}vIT zwtY@YM4XTmUh#&}@a(RXnAzO`7#{Xt^-k9>j+D|R&iLwTgjihozNm;GW@G@<Ar``; zWQKLk=o6vY{Bgp`LGZH7Ua;kTK9)+X#bb30Y#U^18<H$E3C|)Q^e$c)Q;%0_zA4-I zMy5y|th^v;zgziB#U0(b7cF;~@y>hQO|hX<d8=`u(gepd*#z51uyDwV;}8<cz%zGR zKQ~$9<I*&(w*@p6H$bD3vp5#kl-A-6umPiC<jQY$nY4yMOCZV7UuIH!i^r!P+1Px) zAVrVaki05cybCKSvok|!m4tAxchs`9x4-V!X>W}Ca!HA9m7qg?GRr$n6%nATCXG;j zu$aaf8>a$rt-z3%aSqMw&}HzfW~VNq4&u=SX{<eGPe~gYm_>n|n10AEX9exDR9Utx z#(PNpGw+&vDB5@LE8p`=Kfar|GrKb$bd_b+`flm*#l_zFDL<Sh&$O_t&@KoC7V8di zWcwPcHLX9Px9j`BGp5v9n8qs|dHywgBeIq5D(_NlW)2Z)r2r8(tKZ`BUpHKG@Ns3J z)v<9NBJ6v3k@)o)ER+Y?C&lDjA&keu8$EjLnu&8ZF#q(!Y!ChcPG@K~#VX)Fp1ypV zMRejXjSdfh9=w9fGbcB5rV2F3ZOR?7rF~%q&tk6K&k4{Batxx|fNe_GvaEF+R~^Ce zJ!t;0Lp_vdV~Y1Bl|bmZ;6FpztID>UgJPeZHMgBq#OAt;8-K3OmJEOL+*D<iF7}!* zO=)sZLfqH4bX(dwStO{}hNCD|1JV2EF-dDS6@U&*ob|LDR@>|NGuX+VSO9;=k?;rt z|1h~Z3CY5dVuq=An_`c8aG9>}sJ4Wnsz^=SaDvyVYXjz7V|$yV!rLNLC(1qT1zN@q zxDL3?_P;vQzrlxyRDLv^QsO-DPcF`lszE|%EckebWCa!A;5xziMViszMyEO12ll}Y zER_>KWqF!9(c{r1uS(VIA?UG!CTgv;5**-_uNE55!-Rz|QU__r?mzO^#`{DYlI~6V zZ5G@(bSbyEt!CvWus*6s+RQ8b*7B#$E`E$tzsRt?K4j!Y@Cq-qMc#Jtajo@|-S|GJ zTZ!)o%^x`?NU7d(dVli7aKYDI8s?5TMj<?ImLP&Syf3V@;@wq&EIWR}&V@7HZ+<tQ zAo(Spp@FXAW4~oQMV~c&>~gy$ERThI>o-8>;@yb~34IdlC(iCZBkuQNUdD^<GQ^>W zh*n-|*B+wx(1kZko{txex0>adDtR@WeW=6c{N~xWETxZ5e>`$$Rj`1yN1e!ImWqq# z6D8b~{vNsThf)K@tHCsv`MfF$*4fCyA6%a8naM^02NzLf55>UK2Rm1Oq;Q2&L!XqD zvv5H~q~u#zu)@b!){ao3fwLIxE3M`aPx+|mk(7^neUqyKm-nc&p{FI#3c)l3GwpDy z=mMjxny*x}nqT@8{P8RN)O!MXjH7#Ku@40o<2_+@`EQ(e%bDi&lr$s;@+^`&H!)eC zpHR?|RKe9(x%icu$0eA`4Q(&Ha!}{17kZ<c-cbj9xVw&(ouG2E)meBgUbTMZe@X;% zT)P2S$D2AJ*qW;dY51(2Vq%{gJ;KADZT(|yxZW!hHC85HEliDk*Ub8t!7CI2tcNY1 z?t|SCQIpRG8Wk6Z!Ks=K5T;g!XUf_~={BMG6`r)LwUe#vheYZWU`mDIszN)mX!{bw z@5T0SEiV%oiG&ZIM|FK{J|vI4zVE~OB&FN0GA(<KYanJp?!l`^HeC0OUmbb9L|I$@ zrmjSL<_!;Ce&Xh*T3>gWvg1By-{~P~(POCJJMyXn8tY?r8T=)4diVy24D)J#%p@KJ z6R%AiFO=L_Ec+Tnv(-42MH0k>5SENP?L7rc8dH_uYds}=&emgIo$0D;KN1x8^TFb1 zQ1RMHFfptOA~=<^toV&fg2!dM!_mKGMW7HV$O7vn)e#yWpXj#OR9qhqfg#LP8q-Hl z&2U5Uo0F{3&?fSo<V6;MF$pEKCq$V#kl!fl4xHV;Wac|8uKQtk&jr=_X)l_g4}!Lw zcuh3uK}v_Z_@v)=g6Qj^aj!ja?|MuN<Q@nm`V?Z)3Y*mQ34?y(P!ms3|NO)|Zn}M) zA!RaM<yO}=ZR2d(0i`>qwXgI^Tj_tfJFO41s92b@Pd>_zwZBgOwsd!yhM6d~4O4Wr zXsjhJ(kq3Q0amn(9Vm6`Ptsc3#HS9LW9-sZMU{-Cwshl9T@0r6NipO#@G<K&>`nTq zT*fur;<z>^*L0p1@g)O_CS712V9+|rDaGT1wW?T4em02uU8u7s$jy;zg7}lTd(2s{ zK+@!C$0p*pC}50!MUHzHRXxtpt)dptnlpKXfNi=xJZmDyp{Dq?(lS~6-P!=ReB#g7 zeHCzxXv9vItP;co%T*Eh&Y)X2n#)hGZa^*9o1U0X1ki{U*=z045!XwStDE6&&vL)8 z@JneN-~PtQ!ydQaOKF*mlGQeT(Y2uoRqy2?+7VtPpR6BJvltXgNwxa;F##xhL4~B) zdC)IAUDK#kztH~UVsQ!D-J#o-f>;SuDT1*NJocnlv9?U{)J#XZg;OomPgQ>;S%679 zrXDbhSI<c`kxGXLuPMVuZUA#!ga<`oDm;li@}OOGD~MrA06PtrZ{u<D6e?Yt&~}sg z2486L7nM&VF8ps)R`#fLqC0N@vYZsPHkB&ps>~^&=P!&olwR`21boU2vk@dxPrP18 zSqGyvUAw|#=|Zp3#}HaNKO}lZa<T1Hklmem|AFh<+3HdoTU}$k_alNkf<thw`j0g* z_mvj=0hY$5s6OTdc8$C|)H{gNX{VLB%~nmH8*U9?@>pz$Skv44s7(m8x)jxT5jz_x zz=QSKVHe`0r@DxfKsfq#8DIXzj!9OSgjKFj%Y#7e2Q5Dx^H{8K3QQTr`A(?aB<<f_ zWj>RJmU&!$*C!VXJ0o{*)EXWa8I;~HoscwSw|&2bu>Tv}h83W{hwC&Zc3mWA91<(X zTva`^iq?vf!d*xsC@-3*ltnq-nbBREeYU9vZG6;Ht?5RmTboQkWt+&H^gr4=@2IBQ zeBVa_QRyNmU5bLVC{3j#0wMwe(vd16y+nGbQIK9DAR-_|ihwlfy@w_sQbUmvf`HUS zS}2Kk>pN$b@4a(ootbslyUxrze`G;eo1L8}JJ0hg-|y#PkrVHN?2N+8dL+SAtIXPY zk7~-3dFcGRmAtL<nRDi06D2K(iCr(Ry{ka!f{#HW<_!NM;hQ}zr*9M+4en*4J@&IQ z&+PFvq%VBT>VPu)d13rTDmmOR#*-65<aV9)XA3wrj`_!?7Njka%~Uk8+fJsY4utt3 z=2Hq-N_JlHCU6fgY&{5^l(@)bP6%g@yS}|@7VmEdLs-eBu3eTZ#8+seu!1YMPx5|> z_y&8HY*so)^fe{;zq@aL&g|TCxz6~!ebGTaSn4O)S1lItPc-lOD_Iuj!)#!y2SdwK z>fp*9Qgd<Cw<4+z1ht>3?ZH4tbil@{X-(2XOfT`O#qhAp$=I+LqMlDwdKL~Af!yL4 zyp6`DW%g_Z4b-r9tim6x`O;MSU;&!^AdZqab?#V9r}i=4+$tfrR_T(kCPq8{wq*1( z52BYwWzu(0xd+Xe&nDsA*23hGHD@oxqYoZ!+DoTLNT!F7q?+)z<6&7wJ-NV)Z8Pzx z@<Yc$bN|)P1}>y!GL9`JHCMGn2k0uAbjrPC-O_(QacLjXaswKieoCU`?9r7^l3?a( z`)kRsU)nlaMQq#&NoT%#K3q4r`#j(K8#jvGP?Myr!hn7V`2E;M+Md>@O0Knh(ezCX z-WsGs{O7r+X}RoU=r$w=A@UZ%D&y-Gf!}1bn016kdR0>?Zqjyt{GuOH09ofscXuxr zj%+cSNj$txi0d@I0d7z)n4wYKnCX;yv0tXi#&oZi-#*4=*;j9LMt^FjjtEAeK8E0$ zzJqd_MT!WlX;;64;(BoiQsX9cTVdE8MmKR}f^Uwp#=9pP_|QiB5|b?6=JdthDh}F< ztLh{#z1}*o-Psi>5v_`dM4Sd+7~bToC<|OO)p%Zdhe{!X^1+o_S?c*b01dQ+LB#lJ zd8ToURstCf`Z$Ssh?}t=_V_l3nV9^frcB*qIlNBg=yy=#-WDQZo#c7ZDc6=bhA3*r z#1yp7oWpa4_<`p+Sg|MuDkX2)XoR>Vs_HS*oGP2G3?1(x@ACB$rV%FygQmm9=H0Z$ z`Pp+Fr|9wxB^S0DYM{~&_8-_}^qk*ts6$W_dOM(WM_!Ozk?EH?^T&XO8d1<UuGj7~ z-Ah@<N|sJ9OAhZ{M5v$6JTp$$j0v~YUgU6hEVGjNt#P;Ztmf0Ib+J=!iDa7(F7xjc z=6D6{n$<&^Ow}S)r;CZw?pw>SfC1KiQavtGv2nEcQC&TcVX_^qY1^F-^+>lTH#W^u ze;As#z!N1JBNB*b2LVZihNgz*Mw%|{u_m_$ONpyjES`&HUuk7$O~iJE%c@|$gQz;w zXoftfi2C@WSdpA86E@wh{!62jY3;H$UybzR7M(nZDsL;wx5S&I?9agSvj|R@*;)JK zB;^rnGabzeYL?UiYe&$zCjU-<1vdBc%Ic~}h{;u9hqM|pvVDVRaH@_;uORp8zyZt> zsy<~kWD5q{kT4xK?VX;EN;jbm+LGL4RT`LbpYko?@kKx+<9kx2R5)+S-1oim@#Ic+ zLgEt9lT<g(R(}+#`@_opo4rPxMe>&(-v44(w?jDYTu>zq)jVa4$7*en*?`H*iS@Wk zmOLj1gJ2XlIc{H3_o4V{`m~#v{V<!jPc~}!tCH&Z-?ZK)j1p6`$8girQ>`NHXM_2k z3#qm|3MT&+kPz^jC)X9haOB}MmwJg#bh?tpnPgu6b<$m#&IP<J-)UUUnOScSs3Ag0 zjOOQ75%jQ9VRfOelY(itq)Ov*(9=^c(6D9uEKZw;R-$^Y>Oz?9gD>A4ni-u#Y70td zIl}qIC&&hQ!*D1s{*u~-1E$IbsrP(R@4Ki4u9SAZ(e(%A+ZN|Aam3hM+T*FM4q2fL z0h}knovb&`ilygCF0n4P+YA*ijFH$(Xc!GBeJ749CCqVXVVARad2=&lk#{pwRKjK0 zbK5u1x6e;eJr$`pbdVDBW(`T_?w#s}CckFChG?s+!y}QN=q4sETq?mSx)Jwud-dCE z@7V5o%je%g70CBN+Z+^{7_Q?hljvXxQ+;d~NhHQ$RlwQw%va9>S$h<9Op>Ia#;#zm z+qgi9<abc86)=?yD&Rv&6yWb5M&%<R5oJ=uxhN>^GJ=ro<sWSIdHbDEKLT^#9=d(6 z!2&20s{wP*AdpY@0@b9<U@YCyX>-!0$fnkR&m+HhG@twqvWM@rH(fmRN+8Cvf8B)% zd8I6he%|;jg<W<t`40O1ph5rdvGVdAmnb)o6O7n0C+Hg?iD$v*5p&w#L3jy7TiFO= z`3)SXa32RMtKpfO@9%s4TOBA1KwFiAE7@Qn5Kg>TU_@>hugcKMU<}X5ym2^0dHXTt zhuNF(uM?zx=_t3O>#Sy7AxkRtX_JN*&&OW7@!Xk1d6stt$Q(+@2=?e!5_9RNihb+k z$xo1pOBd>$J>6b|q)Urooik@gL?jUewFZVVkpRi#8~rikmDfXY&@3bd)^ezd6iL!B zsFOPpv2?^Z-w5rPRGxAt;(OW77du(ASyjF8U+`E{UNxmK^5O-QPM5=$0(e#imjlSr zl`JvjQm#%*EUr1aHq&8T(T;wXhAw_;Vu71$064o23x-&iz55~a(BmkB*Y+qsZG3-o z$l9@#wmOlv&khE>L{n732jj~mQEwQY46^lOomwmqDu4UhG3-|!f)KBQeHY$3iCB#x z>ZootAghhkoEF!o%Y6XiR}}U}r;^!E0fkET=qoHgbMmrV8U+rx?|7Q%hupyx#F907 zVNOKim(4E!+UrOC?mtvezpWU0Qd9L8`4VFP1|Q-dc5MD^!TmGTqpe0`tWA(ak^YuO zL){4T%8*U<3(<6O&~k?{x#-Dm*@%He?7-9ePtW-zY95>b@*5O7@8_zQL3at5qBiGX z`-daVIAbU<Luq&5S6kj?vo)%f5Xo!+CZfaGZ(cykSBSPl9RCq<#hf6V4t$yNatQc2 z&j<QgU0K8@F$C#S<8LNJkoUDCusOx9zX?&Ir<sua5*#x_a~eK|l&h!(p6*2ahT&P{ z!ZjXoPja{uCGwlwG83`s!7NS2=JcG<WhOAe;MWVpU}DA~JWzE(b6oyI0v3~z%W4x2 zoG$6tYs2F<owz!bzh<jwsp0G8t=Zh6%)q%AF8T!j24wjLBbfStVRVEVq6C09W_<~c zPQI>#q=EO;u4gq9(1kxWy%}@>tRQ$IXBSrzjU2v>zyoh{I9J>S!^Q6Y9h4!C!0W9r z{WP|b^)zGL40my`7~nV<NiZi8E0?E(B!}<Uu4kr`1REdzhP?0&Jrae>z^UdL4s3zf zjRa2E!*-<$n3KX%q06VfgC_p^X~Z<!gV_*mdKt^l5pCl0(dW8ry|yGH>B{+%Oe{38 z@#|(zbInl!*KwhVoGC^9a><WHzz77h9LQN0QK#hp6hDiJMxzcz&UDUd1!)EjVZJ0A zM0Jk~J*~Wb45}>z29nfr^wT3T%eajpb!QytZuNqg&E(bKuEQy0BgmDLo$wu$4=-u0 zJ1~VSYO7#|i&-jZB=1(?--pL6O(TU}h>Mcbeitg$puscD5mz##(y43Al00Y;?4?Hz zO{4B%VA*T_SyS1bpG=H;91B>3b*XdUWEqPMy>q7f4FtO!z?>txJ$>MT@*leMs+g`x zD!sERt2V(j<{(0#IOz2cLB|3FJejl9wt2SWg7!JhWRq)mGZVRDbOR(EFo{R;FkrmK zFAW1IiJS)kpQKSf(6(r&*GRE-ir1=$@a*f*xD9A9>_`mA3Y;*|L-I+)@{9aKCI9%G z%m~5ozvjySl{5d3e!e@1-YHJ9klT18>$&@!JMR9Y>$+Ma@izw`gy@X^H=hTvB~$}2 zt9O%1btIP8e5}ueMjy$MJ>4_L9q0~`uGuo$?e0l`OewY{xj$4AAFuD6i~7PL|GA04 zA*~}Uyg?GOzp|2y5}PFOOrTy$`vp1J3rnj5cVQZ0+Ki~l;eTJ3C-@%I%@)vuWA}=8 zlw?F$7SE-#H?T2fqnX9*ed2EC3`{ukbyDgio(6Gd56@NN2T;AXr7SqavkfiN9I;d2 zjd_doc=lV_F7X2+?E2N6U;WabkuJfE71H+x$qvvt1Zl2vg3g`L+(f(60a|&5E2?UW zsxF$^)xmx@a^MEHf>b^^Vk^p?J&qhjdRmUY+PhT!;I9z3LYnY*e;CE>-&Un;c6omJ zBEb^di)7@x>&H>j9hv`X|Bz-@{sz})zax~_z~P0%BAt<Sr_G`-m5qTyk#^6sV^!b8 zj^N}0({-+mJ0S7>=sIIt{2<1``;6hEdtZ63SXs#io?N-Svs4bJ?BG0+rqaZ2+MZ|Y z*fG_95Ebm#utR@0U~{mBbBCe}wc3B_lrf4#hPQtGKzmq?CZo3M<_VRp7f+PUTuiSP zoU8r#wZcbTY~HpgBhDfGjDV%~ChvPaJ@z=2WuZO{p;W4XTpa^Ue!1G>0=7BH6HY~+ z8CthgcsXQrZQ39L_`V)~6kDD<ksh-1$$gK%uS;IGebfJ3(7Il~fB2UTYvR)YF+$+t zx35;3<X`ccb`PgnzhOqQG%|ol__$~MEahQ~{lt+#eOX1|*l<^qNXR6)VP>?bt{_ho z1sM$32-wFor-IqD)YwRoof$81XFgitoxn1P{h0RPhq_SlW$%)|A4Q^bw3G#Dt^ae_ zf2AOQz5y9%En!fBwl)fx;6*b{6Zm~2>l2sk`*{Mp#z8{TZ67!1tFmbG3l&950!2Qn zSmt(1Rb{tl57X^05l1&i@aPu=xw%xBIncDX!U}l{H%MkTMAIhmB=~-fDn4AKOpP!; zrfFi6Yo4K}AfR_lmm1h9lId!_D)wdcIMZMMa<)Rth~68$eu~@tna_e<U->KUXk~rt zq!UnKbj4TvNfDkf_h<#u&C_^T6xHr3lH9gEy_yg)a;K@3TlhPu+o19XwXF7CGyH1~ zF8&vr(%9{1f1ED*PvmCiQ0<UZXC}4jgl)WuJ8>YE0R8wEdIn{*->G_EWQdVoWs8tk zZjvy+;wUG-5VF_FYiB9o6re<oXoBMK`xZ4dNTVvQ8Aj2I(4Z$mgF?l$cxB<y?6wGj zf(yTCTz}|X|GCGU$1T!0VSg@F4q`PVT)cB1J<$>(PLT8kwmkv(CtW0llHCCmER2&6 zDxqc4l{v+=5oRFvbnX7rl7~N#eJ=eocUJ=|w0`BWsoeS_%^o9qMgpLp>DiwJ8cWt0 zAefI^>YzJ8c<*B4T`%7Rljoj21(|B?*)2Xu!n?Ioc5>0~6!CDU)N+eW#<nu%4sg=J zl5QR_nrybi=#fltt~;aSNQkY%N9l)p`zDsveRLS(L4Rt&;#@UAuf6!Ak#p>yeHfCi zqIq!W;7WCCEQRp;<h@+NE4l;(bIXy#!v^fU9~2i)$ZCU7Y9@0iY_3m6>>l!QUDe;U zZsDANtMTEBlIi#=<ixT#0f9-2rP<YZ<5`4oH++y}Ez=E|qlcc}zWxK{kShIi?kW~9 z{IzcRN792ci?nY3Rzv-FHgo=te`^qGcLmW#h4<3OP5C8X3k$NJwAs@}?8Czl%MaYi zI4jq7TKpVw{WG<5qa*GT{3z+OtNC_cZmWs84jzV>M0s7%yKPdu(Bme=R-q4lCM$0< zpH4IGAnj&NmDj%hQT5tw#f_7I*DPZ7hQj|Veg9Pk|F?h7+pviko4LPXb>~62S`QF^ zDv|NU`5e<Wh+hT5%UQN*_~Sc07#iAJ(n$*PjkPNmx=Bo(#vEOro=t}&^h|X&@!?;5 zlgD8-LgcQpcTVnRp9~I@-M_kn*sJr^#bVH5;1HtMJUA>np$RwadH=%hG5M~sM&_{@ zHZ|L#^@lO%dG3X?+cW9XJ~{SDx1!b;nNqqKjoU-{z-Kf36w&%tP{Ea_V202)-IVx3 z*l7pW&?lseE3D`awFeqG)JFbkwYK6mu=5L9bd8+5t$g_BzK_sldj1L@rnE}aK9Wvu zh}dYyDG7!c=!{1s$xA-Wl95S$q0Lvy7c1T1OeS=>`+*O=zu=7@PQCCWNPP{#8*AuC zgOE|jOshOfS4h*@QkeU)uCj5$wfUaRoUi*2n@v)IjQAUwg3Mo0%8Gv|kM=(h(0i2O zCPsiPt(=fKmf;=1h=0_n7SLmthd3M1Cf~g`_n5{<4vh4>Y=zaNnz5}5e{}ze?OV-e z@`4X{n&wEByp503Y5Y|D-O)`f_&bITdRY%;($(^s12J-n)Ha{eCXQF<K+F0ByWYvX z6UeN-`j3`5#2ZxdqPf~9-@d(RVdWt5k<LHN_`u*YPDYK?N3hD04fP*;!!lwMe_l+s zD?qbpC<|8vC`~p!aGNL`%DB&8Y(aE5Q1it&7g810&i1U1;DYhCoILyW<=nILps*2o z$TJ3=-H2uPk-v}$C{vH%l?cMki=*pzjEQF!IW$HG*vzi6ED?tgs_T%8E9KzUW_tU{ zM_ZF0-MeZ+Y)!H{RUgr}lXLo@#qq^oL;}td(o{|dOF41%x%58I+OaTbx-cb|zJ|{; z>Pf}H{Vko^DTU}Mp$RVYB!FUvsWyAd9|$?GSpvVNFuDowGjFM-9G-W!GOR447teTj z_AEb(Tu%Wj_2kl1lJHj&Lld52FHFMJ?1NgQ!HA)Xha-N6dhYm#+uqPmOJr?n%!Jpo zG!n)Gv`0$eUUtt_>~edvjDz*A<{S3qy3D!M#Q2VttnD~imYA(T_{Titl(q34?@1q< zV~E<~mO&qsS#j^~$nLeyv5)o}6~+w;ecj91lc8Dz%Qz6dO*R{>@MLW>RNq3wD^q+g z^Re6sv*Y7MKRiCZ*C}M#2Dv!cpp!#vZl8>9Bo?pdU>w)4FSox-e|QWi;k%U(-V^G( zq!*EM<am*_y1p@F9;Aq-pp_l2`$YK3b5Lvd+=7pR)|~0JPD(7rQkWzcMQ-qR3K81V z5F@8bm5}V=n4NuzcI`|fl+!_ACE=iebWYUBY#REp%PJ4Rt8k{~Mo!Jrx1K#Qul9gi zEK%(C<GBl_sN*j6=Yd9kFUr%Wq^%KQg!`Bm-%d<&;XCIUH&;;T)XkglAmAA*rrA1` zwv7XfnyhPQTRCUcKLQp4-M7@o99t7p>GB(6377D9z`hNA+sQrDTo%027Icce%(&F= zSj$I*YJdGO+KUaioR{Tw6RNY(%_R*h?nN3OV8L8;&PQ7FP7WGrU)FX_j0X6Vyl?1) z<vb|7&-Qu=gpD_goKHV*%Sk(|zlEC2X-X$6J+=6#Ja$A-KzR0vBqrb@P%Pxe*^t`m z%t?Wx6Ze%kOApmn<Lm|aq(x(o8x}lKV4*b~@sy2C_Zc(l@9;td)+P^~)+{?RMXKPC zkrLqZcIlIxtRopCBJz{{oQmArRu{z5A5yrP2@@*k%xUbMEDGTfV?rnyz6d_=KBwgw zqpIOG7j47k0QNx=xBWp*arVoDk2%&QUb*4ed&{B65yCY4E7xqeq93OLpX1W52a4+? ziG$@xPO>lb(BR(4v&*t6VK(g>aZ25?OH*VyBD<Wu=Bo$aL4iKAp_S8Ml#hu|OsGG_ z&Wj`}VUgI_5iZ3w?_7BDl70oQGY1+f{PH`<&@g4KYHcOr8|Mi%`YY0D%BtUx6EQNc z_zA%jYo5<PE0^5i3+aX2f+1fir+F^8tWo#^tvCR49#=Q~&OCyAZTjFEK^)bfi~%#l z9a^t7iKfppHv;}kk%at)v>7`~X<xDEe)=G15Q$!XwcjT<Skg{%^Yn6E1J3)*?EMZ` zJmPU(2=7iYbUv}&9UtX79Szg1t__j26`?M;7#@4dO3e7Z;J%#O`d4G<2|ulCQ333B z(*2_io_k^i%EB(x7al!)mbXUx8Czx7F|GFH+br<@LmBm}cg9!$V#58|z`soW`vCG! z7}ot#1pxL!SVAx#vH8Dix0y$r3f5#^&Tgz(26K~SymlyG`*7V|@bUO58~|-0j}ywe zI$2rmnYrf^&Ud&ufX^7KLbjb{7!D$q9i9>5F(S%0(R}!&rw_!;gSjRTv>(?+@fHZI z3&zNAZn&~#;nKslXPK7u52~WCCy!Zl%St#+pZB!*O1x@%FJ0_LF0IbP2!SGMAkPJy zlK-;@c`Wm-M;_EsCogRCG_=ocFSRGuHMbw`PPyc$qFKEy`*vNgT3M{<mQoI1@RVF( zS}<qp^O9~3bv%+NgQB>v`%6q=-L-7?LLS1um8LZjt98}XRb%Qn>y^AWLj!nQ60r*S zJ~E&e9kp&-3^DeNLa>XazsTSI=<8+{m@O8feLD8JvQPH#xVmIbh0Y`Q2q|wCt+Q51 zF+k-Vo#rshz;<th3ux^CyWe@Cggqtl<L#`W&B8%3a+CqmIAy+SR`oR{Jcx^hfWC=i zddV00QC;W0bEb^pE2gN!`UiBHFSaGW&Yr1IOFskWHy$iBJHJs<#CdY^p~4A^Cx%Zn zAWy~&+QDRsbJJp&Vo};R#k1WuBOd2WB))OB51z~5Li%j{sLY#qK)roa@0q+T9Uus) zrYqn)L$jwFU@n%HG!@Yd7pZtCE~<dAyfsWcGXyH#GD4*d7JTaSeR-PC2%E8WQNR&- zp>u-K{=KLieot98dFS-cp4}Jz%Ui%Vyzfkfw-2X9KAf=w!i6_0DDjgVmB>ppa1q%^ zEa>8{68B1Ag(ey9aS(lumCONSLCBmnsYA=!%~LcqCNog`c&0bWuvvXn(X~u;BQCO# z@zksSWc~=rH+n#<30L33C*{OZKB0jg>gSaq)gJ7-(hXwg6B0r<8M0)m=^x8upPD6R z9ZA-g;|!uM)Q)700MaKzt8-r8auZnGSZ>`sp_;k$c?VnR6~Ka<iN{2$MLN+Aue~=h zksB#kl-WoiZCQo$USGam^s{e~H+h;<B>?bbM6f!kr2wkpn9%qY0B*FCG7xG8`ovQ^ z+_4n8>A2yXnvD<{qgN!C5mOmpA;T4!a}dCWiJi<dqBxB=R#qil_Kr(|Bq?8qn)2sd z7N4^@UF8aze^xGZcf7WmF|ckmC74h^ZVDXSau6lmSuzi1Gn+Ig1nE{YJ67H4g*iDx zzr65mvW)NH2k36$7%=&f3NG!H>@`49MGto)F?3IKG_H>pmiRgsqUJ8}Nb&NsKtt_! ze`~2C%+WiePnV!4NSrHBx=lYVrm;7iZ!GerFOD{>p+5HM?uTeqfK)oKXktulV*GAc z&Ck-zF8jQKmCY!XCPBWje=+n8Ja!~x^ppEV<0f<kS+~)sdBL34Pe=OkCq6*XhQ32j zOgYpNu|D*^YbitJZBaT4+4;&zg(D(^{lqOZ?e6s2Sf?2|->}{=Jd9I;ELN@j7-F4z zbf3Pt%in2||M`kMe{Z*=5<YzT(@IV16*S}u0S5B^;&f0PLd(-^mV@PL7g)+QFIMyw z`Ayz6rF*HQb}4`wub()-&JEj6QfZx#I73r5)h(S>@(^k^+GtXZY~s-LDBwB^=UkEF z@34h>*C4W1sDPagm$YUF>yX_M0W|mp>CQKoLW?I=-Voj%J6xAqAbtn!&O66lUX4Af zEe_Mw9(=v^7W+=dNFcKk*i{eE8(-%0h-O)MkeBh&`Z~xbc`I@=Ld)D--C|Cid8^~v z<J2-W$2K9usrPm%Uj~MGXX5Q>E)CqtY#1@~Gpc373^i~+YWDq5Ta$LXYtQwy&1{b3 zkpZ4eX@BQCh(a-M@@Znqt@@eX?QoLy(PcNf7uz3%QgN_oAlNc$sN-0Ed8a0M=BBo- zX4abpw@x2lw350^#29d0fAQV_x|%q~#NHmd^C2vwT$~+5sjco^gpQOf;aFj`ouy9D zACmgEDXT1IJwvDW4AHAhl=hT)_CZx9;e4&59lWd4OKX*ZVBGpL{~f#x<9aXs^LXyj zPHN~)G1{42q*huwW~qJEoH8<UwLWpwPc7h@`AR%pnj#U!)e=N@T8=WKqd9bja;SkW zN3JTFFKq+UFx9&nROg#7Q(is)Sep7EPSfJ2YUTCkm{+a0I`DL%+vyDBY$CoImCcp> z5EFbz>kdcj9y|I(^YI38df1w}CaUhNH=pv`m6(X!aj8wj$42{~LIZi_#4a3vTUmE> zu{k`{Opm*vQBx;Q>7mT?dbd?mQf#JO09R>DyW;SAq401lQ-j4hxt=ba4ex@=pK^x3 zcR&7d)z?3LboSrkI}}ZpqK{5x!m)SN{H;$Ihq3t=t=D^dF%J-zXY-cWU1=%V4UB(U z`k%;3Z|dM-?My=0>)Cj^5XNU{$1LIz)hGW<u;y;3|2GYxx=)h*;wV;kuLF_ob5+Qp zJ1Kyb+Jka@%QPPJK@lWB7c|q*j)GD;PTh+)#&Ny@2YZa&aI=3?<SdsRx<N8GVKnM2 zFiuiU_T}(kS{x$uqDV9l15^o}^!*a9DIbN(j(o0}SjZ6~Z0UWj-)($<EWt8=#f)(z zqp_|=x1!47zM@#K5INVzd-61w?xGjN2Dde6WW)vcH=2auft%`;HEHOUI>)zEJr+(y z`F$t*?3H;WH<V3?S67V8(rt1K6|LS*DLUAlUXKxYE|_<4dH`n=y51my@xOMvOpn1x zfj;&KHkz_x7*7}*m8dJdHb?<5<5b$<hb=({HKtze0NdFLUNx!`_znC*z3ZSh^tR6M zbC>P;3J5!)0$zSQcEc&T*P!gh<O;Lq-kT(!@1PdVdq4D{TQtsBG05oTOIePmjo`_r zgkgG`A8f+CZY}u4ej|!&Z+l&~FZA*MJ)zEZMhL$d>UX7$L;0QxczivkbKDz!=IS!J z1E3+{+)k;&svdHvyUvK=$D%QsqExkfb))?Av+CuVQj4xW6!HU*+~Z|Mg@9ocdqN<w zm6Z)((-{Xd+KZKx`Hz=xZqa5*P7uT~=+FihzWA?I-#)LzM}4+8DDy^(GonR(Y(72E z*M3*W+L`DUUWvOF5y10miSU&B)6sQ9>1Af^#rs`ayWz^PU4w0&M%ONlvLRuoHHmsd zC*h2nv5KcDZgIBg{=~{SoR}I-;d_btwT~F`FR-J}e{k$T4EB$*(Za~-Z_BUHz$v?$ z$>9#F4Y-*4x65NM4NB&P1e~p_%!T~!_8AM-XshTgv6Ofe^I~m6i~Uq^&4yLOdMVn~ zRKB+KR#Bx_9#htWp6<LMle8@~KLHP%>VTXZ_DH#;AF)BdKsOf<)qJl7*;Z9hf&IC5 zH=wcnN6(tLyb-I-n2S}NK921hM=ntYIR>$Wv0gp=Y=^(N60S2l*)NUtX-RPj=Vy&r zO;K`N-RD$J<j9Bqf|u4J1MB5nBQOZeNx~~QltEb6C;E&SVp}i{8rsZ|!-9yo+9`g- z$@K)-GChBHP-sHDD$jXieQjQBdf+Sr5}VM5>_cB^rYzp{1`MKf%w}BkYAQn?rFI1) z;79hph^I5hs$Q$PWI^7z#8@@ey>t!8sDi2{8kezVO}~&#k1(SiGEtUlwsX00-Ow?- z))Vq7iZh^6NftKm@34SNeBu$V*R(!Xtl!B^`b6{6VtcU>s)!p4&A2qps64!lyG=LY zG$a;~P3z$LgPkr~m}DQPj8wv{cBJv{AGyr$ixj17+BI?Mz_<qA!cs%?T8dX&&5q%k zpYaZaZX2uJ-#>wM-cLb`Nf{EwBUXM!-}+9~wRRUv|4#J%N49hSzr5C;kofn{2!nrL z`T8}A1AN}J>0Y=<lviCxaAODhN#zadz4vYPDXOnylbDntI$mUNAxYaDV6F1eLhi<4 z=ZAF_NnXmrcUcVm9p1&Bxifi#$@DyvmBn-Um6Ih`N1A9RTYVu}b0W@rZ3g91UmsR# ziDXBQSI~ST8FG#m3c4=$Eor9cdW;y*`rW_>Ozk~(;G}NUtY4fq9$>^Cj%$do54|FR z2tkGKk*I(hK^hgjOF;@1<LlcAB&#DzpW#W=z{J%VX?(pM+77cxHI-`Kv1hN;_}b8d zl56#n91|c@OV5=Y^ApFZ!i5yz&Z0(#_nV$iuNVyM<)+qQiLsyPx@rqJDRJP?BSvf7 zGPBUxvMss3P`|Bo1Gk=~*X~PCgo=I$9xQLvvfVz-_sbfWK=Hqw*8Tee-+v!M{;t{k ccmC;L!vyuK!v8;;Z~kkK#*dA@5Z@>M4`HK8)&Kwi diff --git a/doc/modules/cassandra/assets/images/Figure_6_read_repair.jpg b/doc/modules/cassandra/assets/images/Figure_6_read_repair.jpg index 6bb4d1e32403c0c38c74d86328068194c244808f..280f9a4856c704931950bfc354eff102a3028d61 100644 GIT binary patch literal 58945 zcmeFZ2UJsS)Gioo2#APu5b`0tTR;$yd=@~05b08*B3(oTBtl3O1f(kn_|YQ08<9{1 zB+^Axgir*86hx#%PE^9t6y|*YoqO-htXXqs&6>N`+`A_5a&lJAI$8U?``ypp&wgIE zKW)=s2ar}aRxkkp0oZxy2ev&2JNxh1fAh6thkyX|E3o4~zy9mH9sl0se{+NX-DUsz z^8e;>`!!5_ufW4yIy(gp!*+-Z>=YN+?uWr)FoE6w>E5va^(C-l=dRs*1osLFi$E8& z9DwZ**tv7ZuARGg@7e`j9S405+a<pHpz_hvdn6pL2p$fXJa+F<>0XsHO+8YMV}R=M ztGD8Xgr#L<<>b}WH8i!f4UJA1o18Q~d(P6z`n(O&$=Su#?UK8P?=?UFfWV;Oh{&kh z(U?0i3HK9|k{_g`X6HQ4&C7rCPeECEMP*fWO>NzaW>QOQTl>q7-oE~U!J#+9Z^tPU zlT*{5W@hP(#V<==m%pv7vVQ#hwaMNBfB*T{E&<ri|MJNGlVSe{yTl>8cI?`<bC=-1 zb_wi=hQ?0uUAvW!?m2kcLGVhr#NlK2_DY_4RNB-dq;lL5kh*$nOjuggkgmr1*R=nz z?Eju&@&Au3`!9z5SG#6m`*sRIlebeGhJf+;nq{f5|9|}d&*1-)K0q8Kf+lA#lE?A1 z40Kync_q~+s3WnuH&j2wmS^!b#B9Xfx*xvhp5Cj;&icN;c5&an{y9ZHK=#ujUwNg$ z{<sZ`zYT`*_xiHT7(;2?2!r%SqYY4V8<uP?#Y$rnz?U53Io!LgZP>mR%`Q|UH?SR~ zFqr7R0)woaq41TP7)M3@z%Z+_pMxLw*jhH+l*%(U>i5GwJ6Nu}PqKWYbFoA(Hu&LI zT#1CxA=NeCaA7C5q(bBRlB~_1EBu~Zr+3s1$*B`wS@WpXi@m6sGq^1a9VuO}K-#xk zLvwDa6?K*quKbA_;&R`j3|Crgaf--FV0Qw<^;jT*ZOuyVLx1ue@F4V&Q~zu*PjNiS zufyQnAh4DW7dy+qKOpSK2axxy5cap!WQWSjJ*6nPv-G|eSIw_(*@?Cv^|-L{3F`wr z1Ty#ZBv)}8wyS~_y&eLn^x$Rm@wA#z#t3>Z7|qy*^~YJ$<H&s+CY&COVF{*~yU{oH zg~W@5>NupiC3tWw@@u?o_a`3<%KPbUvn%Xkz3>z433ago*|$&RBlG79-~A>xe|gF@ z2uS~uQ*L&vy;93$@ZzoUMy-$i)4i@fZ@cA7Oe@dvdq=LwEiUT#Q2MlboRbGW%$wBO zOR0#>Py1O0+;%?a_~HARy1IDxe^PpDVA-ghQXc|whCp;mb-{EWP8S$`pdU)-SPrx% z6N6$%z~A1!67NM5Co&{0e9OLHmWdE3dW8OG8}@vI*~cnhtRAAycCyuYO%{8wCg5of z68CP_tuIVE*q3h5ujtFj@4=^S!}eMnTj}U3QDEiAZz5zdW@jfGed9US6Ol~hh$F=} zTq|RcdhEP1*v59?mGF1s-Xj3?Hf%P8zO~K-&~)@`g24PXtVg$dCs?r7gWqeulOC3= zB-PGRD(4`%*3T!5R=N;$QT_2L@2jzT8n#IxIpPK9e|;1G+;CpdUb>*40XVyy9}#^r z)<0S4T9kWkL1uTxz=vbaHYTHjx<;3bnVMInJY25GKMuI?^XESF#LY_$XDXJ^yHj3& zthbL&uvN@_eVJ~!KIir1+tu36Tdw8nOF};;ozP=P|GIU4=U>-~Zk>lke9ne#8Mc5T z+pyikU@412$M=Ss^(!6YJ=uo&;b%42GEE?XqpOi_NPU2i<DbMREJl$B8pGK4fz|}N zo~;#t@7oX@s95Wk9yfQu)8fwSEvM?`{W={R8e5IJktQ|$Et)HdV9&nb-N)?+!XI46 zDJ@{p&{t_N+WNv~e$vExI0gqkfq!%D!6g>ni{AriBpP&i*LrxJrhCg^qqp+6Ho|U( zu9N<mHM>(equ{qch#tgnK2(g2d@flkxn!FA`zA8iNKu1X?=nb9Gz~tMUT%6k(H+^a zGPdgIG&DIvqtqw&I1M@9SUpmD()Ui?Nza@a$D}=^Ra7x@f3b_{&2(i);{0Q`uVL#0 z93+3im`w7o<($CHAzsKNV0T0r+^=SA^igN6*=SaE|7beA*|l6P>p^JODI!;qwUOEt z3l#PlE5X25H`-cbGBO&<7*u!4>~~vrr^`&lauzcRdb%aNYVuVkY6vdbZGwgIGP=Y7 znb&ZMUqq2rxWMM<Tw3220dP1KL3eFlY(%^+2pDAUq($HFlB)%8ETTG+o!h2Mfj^kM zP**)|xvO2-vw>d=M5^tk2L0OQk7aoL?hG#-QKf4&T)26osOr@lmp|`b5t<9ri+mkl zpv3GeFU@AyoR!cVFItf5L5inWI&_e5O{nZ5r(*jUJ)>b6&hXj&%#<R{kw1Tzm9_n& ze>t9^`#bu;kYQ|+!nK+g>hw>{D+NEdVQD3jthP*?97yfihV8e|2uANQk7A(2&?`!^ zU}G5=(uX@bWiC*+V7+vf5vtRtbd=OS-cpR5abMUQB@&y4GSorsdQ+wTw!YZJ`EgK3 z?q7u$?X+wvn37)>CFjOY8s^Mb)*AZRzo^ul@OewrahiPQ?p_;M82sUn`s~c{u|VST zB40>_j(UI)-iFbh!huU!_xQ%Vazr3v_9}&rN=NwGqW5Fg*qN+9n6<%h>&38pf3{&+ zn;e~86vJX;8=y5FYt1-i*%<j=!IsjXclpwl%pJbR5WmDc?Fnk;ik3zWzDGWJA;Vu~ z_tRGxl5WS+uyth_pRdljEx9!vEsQ#9xFjf;TyGlF68&bq;<qPq%~K0G=baegbBIN_ zZ+@A7IIGu33)o8e0%WBaa1|I80Kd69M6eI1Ki;_hY!fB2MxNO^i1)+9&(1kKk4-zt z_Bzw{=pSG2MT8{2$B7(-7sm}z0hHM68ao5H&hh{k>Es^5!OxgIKO!^F>zd;k%3*Y| zox#T~jYrH~7j#&A9+_Fq92B|r#O!$tIsq>7OYD;yTYMa=OtU+ohzz^84HMX;$^_?o zklyQ;vP2js3h8#%gC!=c>hzXCd|W+P*pE9I8sId!F<-z~-S8;6eQRCxCL_EZyI;ig z9cw)Dh@76R^WzCW{o9K^Edfd{`eAh+`+mAs<&(l>5^qxev8^)+-mLeN#hTw)^PjAz zWc*5XmX_AnYi%@bIZBEv+$t+Yl1<Yh=Quv8{17YzIJ>iX?Ra_t*;j0Clo7BV0EnF9 zWgvnGvRnZOcV^5DIF?|W8?OwjP=k$PdFS_W;3?8*v1i>98p(v$H!&klBu{TcOxc!K zXH%fQLHXF%m+h_N{0VT0bB8~{*XF(GR>Spe!vGZZE_EkP7i-BYv5*8M7;Q`imI6Bq z9AQus3H#0E!dXrn$3;4!H%s&q*9;6{gu3~F33cC0T|P8WKKJ=m&dmI_HGKCoDXW#F zai*f?gT6~G<EV-I8H-<ZPod=Mw0^7bjHTTuQP<p%VKsvwVTW#Ij!Jf-2c`fw`#7h3 z^ZE&c;uXQOat|;U%qLoL6v<){jqf81xwS3Ee|VO-W1tHA7q4s^7HR-$1Z3gNal_j% z5eq4-5wEga3G9S0xQCbt;SnMbmBK$d{>vbNdZ2rsaf<m#R>*l^s~>%zzhm-ymo?En z_;!rTR`U1#`2nH$5WIwUcj1YptPjvKNyN+GX%z^0vM*uAmjP#@ptObw-8caX^d+PX z@j$o?X&aV~kT;LuxM0@$w>%P`k8O&jF37Wzd+QYiQQ2{ZYkK%o`6*E+RSKp)Xdb*& znRj0JVy?&U#9a(>GyQ(;=zAGti+SiF`@@MyBZ-@}_LaKEM)Hqx5^<5WCryxVD+AA% zXc@#Us--2Qf-BBFZhiDMO08Jx@oHDH-ldipulxcpfc$+^(rpRPl~Umgi?Srx0^{4T zYqrEByg((dtjn5XkDIa(;}z(2E3GUXCILe!`p>%jXxlLPwb^qea+6UZ4@hg?j^1H? zrcvc~wYG}6mEGk=>jMLAxvDxHPhuR>iZ8eX8)^+6eO#&Cp8C8LseRPm?rr7yp>qqJ z@-{V2{jNoiwdl1jjVESPGfX9yT`LMh)kgN(tWFKBCQAJdCbYUS%{a&S!-y6!rXS?Q zTA&($r+v==4EpkMX256yJ^+;(Lmf1u9t4BukU`1-yPSxr?w#Eij`_OSzh&zGXoq!( zQpu0`%F)^T`Xhov+ps+d8WB%YYa7g(5$r{`RQC{ORC>_+5jO~ueZV%1?nQ1+W=2vK zaeCMTV9qnlSqBz59xKi{i#f)j4`u`yvx_qeqpG@V>75*of~f1Rq(!Z${o0Scu>liW zM&`&rFTx$ukX6TLb@lYGefIXCl)DxwOH*5tn{OAbXubD%SRS8u#f77*i+jGI%J_XG z^<e+xmux5Jn%+v&oT9S)=7NF_$0dGz4Y5ybZkjJ_$GO6PYheiX)nGzsS?ktxbOK(` zTp#N~r*J%QpF0G5@EO~%mfXA?dMnEywVYRoQvjWM7L=1O*xq4&CxskmjLhq=a-3Wq z=^mVXI6RLHgA&m?Op@5l8eeE>F>J<(7DX6<XW;>-PV&yv9CPjwuyoNZO8{ft(}8+m zA=@@UR=^A~$Z1>zo$14Xi-94%j|i>SRi8$q?x@y<N+$Z9bgY&C6qQnWXJESb%FEE; z)K-g1<Rug2{_!-TsZsV<+nLyt9sFUH;D@=|-5;vka}T+y6k=j^rA%B0M*s0PvMN4r zGyA<Pw}xX?k-u)}@hsn>G`iw+?IpXqPiMm-;;Jm~gsGPXghh9*H_y+gCfQ=Hed2qX zCdMTPnHb^7#b{6i{Qz3A2cXp(h^JL%^C|!0r2_Ui_^OKkhWxxLr*Q=@$UnS>m*C5V zwBDX24hfN{QW&%TZcV^C*(i`S*RGE|l~-LpJ3sSz1|_{#vr8so8)oSui+s^fyZLEY zvxlMWdu2qfE=5<}H~saX*V~27^K=(?N_P6U4;3|b5tN*8r_TcnU<$uZcAKm}u||8| z-c~i0)^8gA*sI*}h|A&6{tXW3hloSuFvKij0Nv`=PyB@E3vmDiTG&9=19J0FFJTAP zl~;~a!HsRh<glJVGkqvQ-$EN7>;;+PK(B`<<3+J{b!*fEQQm^cmy-ptPp*?X%@A7t z|CHPu`-(yFg<l!;!$CDXRHejyBM4gGx6p_DM}k*jNK7L)lhe!{l!zg8u`GPEuE#t% zgMY{yGkv-_i{sn(V>QOED=(%<`uVU?5^>jP&_0H3ByN^45$+Co*v#|CbywBK!^Ese z&o^~?3rWjXQjD6n6;~ORjtOM-!}sfgw4E}Ki{8y%>SwGvl<%5tYw5qTm{w{x;%D0K zu6s^uvs&)e%7g>&Q5jhsaT!Y9NBF{098WK-8aTu<VE+wTauCp5Nn0FTu`v5+l!00d zT}~qhrqweedS`u_=<`mFV0bxrGjKr~c-9ydYtC+EK3nyuEWXzwf8W6yW$sF7*qlr4 zo$8q?fs4Zfe2S{&&hUS!34JIkH`x&K5&dDKRWWVoblU{@RA=(mftZ|AET4KTy<s9i zuGdKN=`1pwG`^g{m<>J_<fTb`yA1=it+|p442|`jmHapP=zaX1wj4`L7Q2YofQ54m zJ+Y@KU6w)gtb^X(n2{d7(KIQ={CM5!w0O`|M@0uUDRRxK&@s<U$2QI?WF{>wOxq@h z{gzja_ealYuuFLjWLhO+SGO2fajbhccxz(_CkKSBudq;_;7}|BE$SI$r)r@z5w#%+ zdY;ZSIP@7)vV39uuk%b_%7Q29CFf$|%;m*ek7C{Zh@*`Le(@vT=O=4b$G?`_iuzPs zZj5=6q3fcZX*ltvjC#yx!`N9_yZy70H_~ESF3Wk|#M*7dwKB$V$rzxd*U?u`Bg<27 z*;u{`{gZ2nw;=!b36sjx5Ik1H0f($ZTstTXrGcJocU}hBpPFI;t=)JG)@`Coh2ibS zKruxZ@yXqWU`yF2tjTz{a@Ibs36Pfp)O)rd4#@X@1y6k-)~(3@{quTRo3WE^WNVUo zqxqp#r-9)s^TH+_n_ev$@)+TE?MtJfCJP60h5ThtBK6&hO@Ue)=k&7A4&nXIm9}C= z9#!g?n8@tl%@TdtT6(eJ`jRl!v48OR_|aUspY1in#BXO(xg}EwS{eSJg*NW>Hmn)X zEat$m;($i_VO}{2y$4LYIDrVv%3P2DWdKihC@}c4$_K?5mBogG_rgQ()U3u>Br`<h zBBNV?4{5clGHuzq^`b2=Z2szpOb*5OAhBqUAJ>Lgj}IWGSjh28Xy5=xM=Y_Ua=G7N z#%ZyKSBm#1rus;M!=+$?E9Cq%s={ZC=lKcJE7#s_Sf58Xp?+n0>M+K?#eFd@XK(qp zTB}A~`;dfU<3m0dR<$R-za(1Ea4c<nCLhN&glgO0B<T4G*{*K#N+RpW8WUcBl?>K? zqC03<ZC}=ztaj&?<Kua9^r)G29=18HSi@w%wQLpl>Px^4E1H1LqlZF&9g4XIf15|% zLB2*!At*$_LEV4>C^WT9ua9u_F%s;53i(1m=~?pb+;CoT_i^qm@SpunfIVfAU-=Xa zptqKna!~*wnM4)y8e()3C4=3ZE(LBxS*)3Sop^RQLpk9yg7y>vxk(oF#V9j^Baa(K zG>@*6D{<osX1wQEDbCqdoDzk|SDxb<CHIx6f+wpm$1hTdUDk{Y<Tzv63dH8&)F+5h z^(0dFgm2r&De48vhJilAQpMh}Iu}{w?eR}nxMQWPehAO;nT6Ehg6OU*cN><ODyZy+ z0K16ZBaM!0_L4eZNvpOql(%a4Gu&<M(TyKpyWB5+Sw>sRs-T-j)Ks7DCA}(|M;el^ z1QOp+Gtq6^u%V4lvFrnrh(PL`FD<(F8zQitn$jWGi%O#I;P1v>dO3mJy||&kJ<mEp zpob-z+G0Jxgr}Fp_62PnpZE4zi^*jd9`dlu%#YIcoF^SkD-mORKyFgN4P*&2Q)u{4 z&Fq(e!FqW)xV#wMi#o*h231%)*=D>-BY4)r?vAje<KKpc$Rgb;pFqNE3;0Br#$uWs z+_QBeG)y^l!)D@XTSw}YlUhT%|6`Y!%_F}8<eRHM6gqm<MFRL)sh|jSu4{Qh@GFnd z+B5p%*~_)|5-tJqvY4y&{ME-_@?TZF3`orsF18j+n{gemaVU5EoO`W`_F2p2F(R-< z)EYIe`^-qsl|OQU(x~$D#>S>djc!BZ)z32T+B;r8@=ZGRu3Y!E-`!E8L?ULn2PfSQ z6j!HodDDqvUf@>$5sd9?_Me59RY4J$p#fu4t9s%bCah1K>Qy=&HDh>6%%dxFe8Sdj zF39+}uD{7OKmK_#tzsM2w#>}pgy4I0p@>SB!b$Q+2)mj1%X-IA;0&O08q8pzGZ(bS z$w3I&ASfB<h6~>9I?d|qBa2Yzh6{@Azz&d%oLH21p{6uMB?S4J194@6i!JT^yHJfZ zvvnSd*|cIMRnQjqBj^uv>$R8|SWYLVHY{<}+n}UzQScsCtr{~#H||G^u7)I!o1ZPi za4bI>iG)1($J^sgsk(7gn6`Vd>80zGBlT8}ul+C=nh*Ux<a79$0{l>K*Go;?yo<Gm zPVA97!l&d()w!`9bG1G;lvDSqRzJCXtUC%Nl}XI%eTn41%~TO|5=<s7OX*{*B~l-U z)WnV@%4lO*_4Z-E<Ws?9zEC3v8N3Z^6<ZhKsNg6{swWy5<aGFsE<r|?yivBbR3YX| z(1Dc_<B&^N)h@SL=u$T0A7yO@pE)2*i+kns-Au?yAY<@y@Z00{@%4`BMPYB3>~Dw; zu2{G-W;^WXDt?)#@6j*0a<6CcTUD)ZkW-`NY^YU(B%?4b{KmLjc<AlWU`YqdTg6u_ zB`*s64>AOor@2xn_C+v-6AopN7UUr6%vt)0W*S)xdk&<~JQ4dap6*~u%~}B}e%*k{ zb)xgazq0l~KCnM{YpT_sf^5GyH5)beJM-?KWHT|7Ysb6aeRKty2E@7^CxAbyqymOM z#jMda`fADkh&hQqzTOnpu51ONfbiqFAzLQ&8o5T%rJ#XR9Eu^B*sXfWvH=sLcSiBp zEYnD&Fyxi}psotPdCZ-@a#ke0#3H-(a0T!)v@KvW2J==}nvWzM1v{h`w_yfau8(h! z{M;TlnOMfxY@9wTu`6$F?O4x5geU(;?)5zfyOmsDY5($dG_AlJl^8>j<qZELLJ$!Y zHe1e!UT+3$`9eV)TdqEjXrT-Fk<zA}4qsRZ7#V<K+d)LYzns*O*5?UB$*&hQAU2XH zPDp`^v?wA!gt{B*6lRGI`$gXl4b>k%snMAe58XZ=YXnk(a&{tA|6;r4xd-6Q_Bkww zqLtf*1&S!{FvBO#Zo_)XGieU2;;i-gnqVc(RuULQ3lsNQMeOfVtheQL9ztB{npht9 zpOUzEVz&ArTLv+aczLx%3WD0&ZypOR^!I{pQ&@xWh-i8-+Ij33@2nHRoL|=oyAx50 zo}cULu<URBmZCG)6?$0HP3>=~f>KR~VSu!Tlss1C<NhEG$DP!D2JsE|4?WAy(Cj!` zRIX*TJ|E!j81FIhZ0+iCbH6~&b^Cy%cY*I)nc6Qil`6F-zw%EkIR-Z@%30}qeHSjO zL!y3^vlVz5_+YXCe;nSj!JOe-1#@UdL(HR`3m7+u>)nP4agT#LqZS(jXhXeWiTFLb zMs&KJK1vSU=$o}~S%e=5?qI(<#U|eT<6Uw4tGHA)=|WQ<kPlHx*QlUDRT>A6NoF&r z8=*Anp<urS0^A8Kvwd0BiH2DZAa9Es%&3OaBt-vwEJL+h1sG^q?r9Im8T?_x@J32` zc0#25k=DDi40Ri^Ujvp=biYH2SgGqqn&U}mx!zn=a%Ao$hDu6KTIh!=gEh=QSL^H@ zYkewXasv!PG>m)F`pzmQ4bqzq-ulB`nW%LztULKJPafl6;A$(PtFExha#)Hl-`k>D zMWAbNrLEcLq3JK>f5wRs*tdCk0laua^vsAe8B}HoI<w$VEiQ-qxS#~i2=;H>N6SP~ zQw8lP>wHD5;dG>yx!}!(gFunO7oD0TuO{3MthYY-W6hM~$ns~&Z5r!#G`ui>tVEZW zj}ynuBX*GkiOEnEEclu#gWV06HV@;N2BHsk@5MY{qwYJ*(uIo~@^t#6|G1V{cm>bi zZoiV~^DabvwMYMbM-O7XSgP1a+gVCJN=*<Ybtp=!n)BA=ZPbyLpsPW)>l2l_(o(05 zG;cYi`w402Yy|Sc*6Os4s+0c|pL<sBxtE@L|M{jPZu4Zr4XyVYawlDLz9NTlVf+!C z7-av`HMCp{F%{mr(Z4a1|C%fsxsjaQ<FlXlgnVrTFNOKpyYLq)_iJrH%_>~p;xFt~ za0u8q$CAoy3JVF1s9wP-jxibIQ?uOpRT;PFz#4==w<;8(_L6C3rW@;c&IMebk90Tm zMS*KRaIl^gFo=*g+?Yc!^pjb4wqe)dY55k);B*ix7Dd77ugatPY}3$!;8SyJ`!1Gf zNhRZ^Vdq}bh_^!M$imDv%-7KMah<JLTJPdbb~#!|^65Bg;ZnPA>B*C|$XiMJMcRYM zZWP>CPGW}%A#?XeUSSylV=bl`Eh(j`X}7zy>EE3DoW8U?c6;9FcyI}Mt7z%Ltv?j@ z*V|d5^W8a>bE`t3id^}q$Hq-F@Acl7c@Kmh^r~#^X9HdAUcZ}}taBfp9=E!mD{tt4 zytQ2Pbv&)`#)w~ZtWGO)miebBGU%+BRjzRKmvY%&<H=bqwdjR6+$CN<;+jDUdKdWy z9@H{s8Z%x+yq-<I3)zl7h<n#D)TgJ?H=9R}W-x`PX+u3Zz}kQ~_`S~&cwd?K&#A#S z&YVCsbu~?oIC^uc8OI=7)s!Wxuh44|2|{O*l;E2iVpft`)$?_ci_w$wn})%gI?f!I zWvvme<HCIYBOr?_`IGH8O}-umYHTorI7Z;PUh@!^HElFCk}9A!F&q|z=ogiTx|%dI z+qo2rOS%E9+bQK|&5Cv>MPE<9W&@xYQ-z#p-0SBU;_GPX7`GG)ee6CKx^F`q#G~&t z+~CV`{lbKUal65Lz}isj?Yw5MK+}z63+1ZDs@2wQnExGmoW512kW{(YmW%I-h0;eu zA)}N?Ju&MsKi2-HaE2nzTjRnDO2xDhr#{3?r!nGEtjU4ZnEt$&xciP??|h<D2W4m8 zp1X$$$H>bi?KFGK>Y6-qazV*3KuW#%M7d9{wf(I1nq;`1Yf1ChUb<wc><8YF0oQU* z&Hj(rmKUj$b-jNEeW<H1uw!f5QAeCl6&JawT{uH9i~Nn)8-@6nj1Y{4nw}Vla;6cE zVuon5Tz_6M*Ozk`lRVh1p3`q}0=_TG*5EEe0c*h07(9yc^gt&kyS(an`SF)+J87VE z?7CyfOO?!(^-?|4nEb)fPjl=g(2Ns~_2WUfL@Vx?Uax1IvSmL5-GCwbGE)wSWs@c? zbU?Gy2?Nw!*IODD=wSn70SJN(5;xcC3m<TepIH>v1sWIlk&g9T)(_oO4nJaxpGE3) zyh7%(OuC$EwWFHayX=qECxkE8ruSU|o;)xlFHL--*UZ9yd~Z<pDNKxstUH?~WK|+2 zh7z*$Ni<iI*-@@O|Lro%X^_6r98sM@AcY0}7?en_eQAX*L_p;DC+Q9L8Qvpo0B{_N ze+Tq%hq>+`YmuBmmWSYwkm92_mU*u<WE&<6orXGY!`xU)8wrs*)prc3`({K&Z1ql< zB^?{y^Dtkc!qFZwl>i`#Gu3?Ivm8n8Nicx(7dG}4FT?x}M-RIL$a>Hv4p1THxesxW zd##!G)ZF*!Y>32Xt_fNOdz>ZM2b!OxkJK2Z)n1;w*23D&(hGyN$%rm{+j5Run!N9A z@n}j-*SXk7pEtuX9eXIXw!3AkH%?z(nW`5SK4zO2>g*zU<?XBcr*chWILN5_X%|Cw z2}R0Pswt!MR!dsFy~MMksK-vpDib<C;e25Ojtth3_lye>kh?Zm27RHiuT9mm)^j*9 zSY2MNh04HUhEu;s=)%7#b&rw^t2?DWkS~;V8hBP!ke`07gQb_|IY%9?nlTSY`b{YN zoy+;fNY$Lk-DfW%gF?>WRK!xN;~eV~i70WKmMQH@&6CzlooAJ9mHVs|<nGsU4@C)s zF?nMgb4$68H96nrO|zT7t^a{*@IP<EUZ4}UVK>M#@Ws*f+{ITuTYcS%AeCmeA#9vN znmZr1!PRTDj0$SRcn+NCh<evFR5MZqmfMbPST+yKe-@Dt$~(fp%J;wx;Q>_E>=r%w z-`f5rH<VX|prw^40meP<dezew=Gc9_BCc&5#|C??8~B>{2Y<hX3Adaym!Q8lpoyic zk!??TI!)>s5heE$nHNl>9=1O2!12Z$lAsJ=z-3@;yfW<C7eepy*;$To@kA{rra-;a z5pXGWHC)jQLi4J{P$DjE$gI|-$$H%jfrA}wy}X8-zub=7NB%aja$eFpi9e2)!_#u8 zZAL?o2ZEZ!2Nnm<LKDISi)koObN3RvK7_T#ZUD#wa0wiR(ENc=gVoKVvp`?kJe0SJ zy`*4mX?DkAJ%hSVSU>pnc)X&ky}O-4{F0)3BQ9+K57?4tiRXEl2-;&;>YlD>K*W;Q zg`nk`THm7sm7^I3-G8m2oACo8*FdC&DAx;6YGUYV^yc9Xk7Lam!Rx2P18N6K<6cZH zqRxA~m-c^|{AWj5Q}4%2F;Q!gq@3z~W+RHm<&W`A^Y|Ja((hbNMsROgcK(!#dXv)R z+Z!=cjpkN|pB-)^Z$9v)JIdF*_rF<<QjE0-<OGhVA`hkqGLxOl%F4klju`(9y0z^t z`9|362z#3Myvvqz>~k=lMC=ipnWGOT(-1QselnuI5`gbpv=8+xEh5SGy)6Tg)Ud?5 zQSv7`l~nj(^YwN0uFhAlT0?i*wZ3RA*kT_{)j{5@Jd=1xPuQU}xv%4cc*d_NmJusT z;c-#rNo4EchN5LHd$+D7tu~D`dzTP5h(P$1<u|Gk8kZ|<66kKJIiT&VRd1<TRh;c% zFhI^AG_^5DfM_-Y9N|Dqr^hUcIOEnIlh5=9i1e(!(b?m)EoM+1)(yCwAeh#@cP7Nc zxR;uS+rgUdMT@LWyBUjvVP0{@&L6`JM*{ugUwh51o?%?h1OJ0^bH-YNcxe8!c+U}6 z6%mYp_^uER3OA!9%d7hr#~9tsK@Au)`t3%JHEtrWV}A63ky)&s4sy~+aVPzKW@$}O zp6gdLoe_#(MO@A7ou5W8_;d$%&R%nA9oUkV!@$Sd#L(fFLn5wiQyNM4iIwo{r6vPa zw3?5Vg7Su%lq5={>66duSIlh1<Ll3K=|3yJenRf@H<xVujS_KFA?rN-gGH4xjuC|b zYQ~Ko@W5gZ{~e;4*n^h#37pMJ#ff7@K?h)@KcBDI{;Ct;v0u^9I|6-VC%^wPYqUj? zUcMYxC69ksnzc={4Jwhb%3F}pljqD!y5WpvJmKjBi!4!ds}qgLNqJYVP4#6R{j>u6 zk+P5Uo-3NPBbW<&6pfO59ILGgZfs;1NvcFo{uuDyj3GdJ4Zs2asgse<j9$cp8u%2P zLdb%^m@fVrVU9u%VA`;h3=RdC9H!jZBq#It0_2QecAnP2g&48KD8q>^U$yC!>mCaW ze7h6Fi>Xffr6Ss@gmbw6w*nFGie<Q4B}yXy>an`9uRV-<SE6B|BRdxWy^O4?r7Pj{ zm%rV*MWP!nOILjF_-2psdYcq-=0=IJsC}Y|tyTFUBdoo8&QegdnwykM>-T~sI}cO- zj{h#f4XXkOLWL3FK$YW9LPwvlUHm>bYSU6x21|rJg5H4{y2Pt|hBFAVPz9svAZToc zD#>4qtergw;Mp&|t3yK}`%`d?QzA~J@ke0|%I!HW?X3kNCfl%Q1wG^DI`k|tYDhsz z0#qZOo9IGeJE~KRL{+RQzvea$tOh2jO+<}!nnmWgc8eb@3rWiUyIP=WARx7`0e1|O z9@*V=rW>8r<63#c(#Ji?Si66!)`1f4kXD`7p<ldr+=2bD)^a@e#8R<rgPytS&5%#d zJy)N9N_C7btUjc#0JnPl*VivgniGR{sg-UnZ;wFTukoVCzT9mXHU`26W3JOU-j@oh zYrHm?$%hbQN4E|YTH(fAKl(OIIv6d44{6ltXA_!<Oyek-geZd8+8=V}FRI|TKfKZ8 zF3SLh9{$o4*O+WK$^5l8!pqJVPbr)u(~1etvzVbAQ~or$wVG+mIjylCUUr(}f<4A7 z`h^hXn(-btbSr;@I`m--bb_?SDN2;$o@FEppRi7*|20;bFmOTwen4%Ya-sTGn}J3$ z=|R_%diG%IYB92Tz#lX1&|n{imPlz-EhD<gBv&R8HS2Hn6+n7~^0ztF+FDNG)sC-B zo9q*{O?0KvNTpGsd^+&S<E9<?^JNxFE;%8zbwURj-SVm;fb-$bna8!;FvAihUJjY| zs+9^{jOKM2b>cx)&8Zf`%vvAe1^I!wP9@fw*Q_>913fFao}C~enh`4hVS~o7)yR5R zkT@Y;@#g_0bx58RXlsYAz>|#Ey;-)5X|CinUuc;V07m!#qWQfWGX``hNv9(o%<gZi z!=%m)%}$lG7JHQ>pp@gjI$)oFh3;8w=J2Am>tmRLV+ha9E?c%DqtUnP)*C{#q$^2h zRP<1XVK#aO@9pgA7UWA$O*(AQku%7s8;C1tsZ=XVcO3YbbmCc%V>2Fj_Pk_PYZkG7 ztk2vmenJ084?XY|+DQ9A+H`><7b1t^%lJLgXuttI6O4~Pf~h_SAhNJ3EEW33;KqF= zeU{t(hORi;Je-l$JMY&LCf2i<wQF^M6Q;VyG$VJgu`+n-Ixbf6w!hF()L}GX8+L{I zX_@^G7{#g^WuPE%pNZxR2|-E^fRJDmDseuaeHheZj6e*Ik3K7p^!(BSrqV(aL+-&z zAcVWLJI<^gP?VlSvG#3^twmTy)SY=`g=TxCe(f7H31U>*Y3%bYTHceBR_i2oNO#pU zTI&kkJ}2!)aV>?QLAvkX+K-kH7O(qUBCmZQI@)>BogsCK`L-Sz;Sr(XTIc9cYq$-| z6MH}fYz#oH?};2Z?h_ffK%ALl;MYybP0@qY16+TWbx+s>xU?#`6w2Cotpo?ji>J;q zaFu|Y4=|nBr4wCxA{U+>rL(4K?yk7Wo3ARqqelmpHYsQQf;ean8ta?JKF3Qm4|ql$ zEcpvetHP=^gLXi9kH>)m&Lyyswb9R)L+#`mfZ?=8lm2}9DPjY-x@;f6cU)Zc_Isl* zE0WSbyYLuSi&{EWY%SkIirrEg_-9}vu_b~bwr?T0z%Q>fiavjb)aZ03H0(@!=RNBA z$qy-wYbX4_ejYm7GWor6>$ma2a@N$yH@DY}7r!H{@~$NJ&{$H=h5aaz_-eEm@@T#O z4Ugx2SL+9hKKsG1PaTF-JANK69kq<rvqvinOa`k|MA0&-)j}HUB^ol0d51}LR{F;x zR~dm$#pegZvlEI<k5%usKk^&S7ou>EFYpH;TBmxwoO2}j=gd4Nfky5T+X0#l<a@*n zwi^2wCY(>&8q7a|bMMa^$2{A*6p77y<aNHkSppsCU>Pg>+4}+ng_(`AgU=useBoec ziLChirK~X2@}!yiTr2bY9n2v$^F1PhcgyC=<16jc6<*vhq?JlfIQzz0qVu$F+XVgx z;d}aj5VH5blG0Lu7}Q&W(3>5~QcnY%=o|OCY#>SEw35--;&hid$B1jv+4g*5z3cd? zpDT~JNRYzX@n<|vhi2UKX;Zc;ag!Mqx%FdqG>AU0gPthOe(kJmnw_pJb8>h__1WBd zwt7Umiq;3@fy~CXYx#u%)Co7b8no=8g0&0|_a$89J%w~gA><jV<)r-IP!2}EJSq%2 z^|{KsmWcZ%peD&dVi`E$X{;~9+kqUjC7*fEk9MWQGIRC=T<1ffAYb;xSdkBzSq({J zj>m;sT#6!q7z3uxK})&a`vQ7zVrI-wMLdyrrdnj_mcRRXHthku0C6jSzrV}jO3mKf zKS-#t^rg=B%j{3{;G)y{PC2QMSLYghtN9le@!1GmAhCyMMyEm@aGL5xAN~h)n?)Oo zY(r5M6YB|^Tk9>61#{{1u2X-rKCeunN$}n<VK8=xFW<43H5`g>7+JisuN5q`??`oZ z*)-S(!CElhKWmrgXN(Gaehe!9rTOEgr)Q!ewz6B6H9GJG%nE3^;B$mmU>*PvdSuo( zXFw%?|BON;g}KQ*#?WY)<{~OkUmCb3?}!Tbnu*q*b6z}-`_?wQ=Kn`MCvN{ceZPc3 zZ-<7L<DKcRW<2Yg=k*)CRoJGf+{;sVS|eV(E8?psZkP(#WzA_UDxZu+%W)$qTq%wu zRu9r=>TQ(20yi3~Gc;Onxg^hU1;aIR55IZz#(4Kz{Hz3DL}XEx@sh$<!uOoS2Y}CV z{iH1pv%;9l9LL~UkKxfrxPtvDK;8Hcra)g>ELrV|VvbZ#rLE389UV%~`|8POE?aNk zL`CTCmI;1Rqu!zMj{j|*(E8Yo?m~?x)2h1-)Vh43c{CdD5>gy8mc}>Z*IfYLp7?qA z9@|*av+TPpnE@@g;O{Q~d?d4&uCHZ(;NB1vS_B9O@I4nHkE$q`d#QJ{6-?upg2#;g zRi9^A9OA$8INFw_%5=U$x_s?X!T03-UvJsGEeiVc9?xtuS?ZU&e!Eud&*^O#a|#g~ zNPHc@yZ_w)|K%hHQoi(D*lbwFjn0)X8sFbqhRFKtJGI{y&?wqW=BKLabp<H`_Z}jY z2x>h`wf=0t!Q(#ib!I+hS}0?P?gxE$ZhRlb#)qIc1CH{8L--4LZc6^2p}Tw!0w@eQ zT^6-N2RMWH^pmlf`luA&V+Zx^Of`0?*s$<4a@y?inR{Rs$9&Bv2>y$Lf#|c4IqXqr z;bda|z7L3TrH&7lXf!7*y=dZ9Ag+zhlrvH*f^R*duHyG}nE_BVbVMvX`ab14{Ci0D zt=GwahJtSrDyI4KzUNtcJk@iGO4kJcc=H9n4@iNzAuIx4M1n5S3ME{$LaI_Ib9(RS zQ~o&h1-rJU!Z)eCL`(wf*4o_pP@sM2##Ptr4|>yG=Ao8IAqR=Ih>&60K<l=hS8jgt z^0O;Y4zb9#1QUTF_An58Cp5)C9hl5mzHc7AyEZ+{IiV*y$FD!|#S#BQE-xAe<6m7k zwB9igVNjScHe-8yWX-TNEs<*V8_l=#WtXLL7ePoOvHJ#OcQp4hOE3BEFY;_aFMNNu z`l@{%n;=9E^wt=lHX97sHvau$NIkb2i{L%&Iz@kl@|<wI>X3tK>vYxhFwQnV$y0E3 zbBXke^xM}^aqPwOw=bU4YIhy5ZoFwo392{x{N2v5AR6_Y<Y1|sDCzL`?*GXLT$w+Q zXpz7{8B#MM6L)OILRRS%csELH57rnw3edb10>CWRC`6m3lMIsJf?OHZrZ+G8D(Ati z0T1N~Xga52>{@s~tRU)D8@G>_106qY!ve@VSEIWPSo_El_&`FMM+Px#8x}O0`R8Tc zVn0C^r{{gidQWw2%lq*ZLxMn1pr<WRl5ilkuWxpDVih4^$C38DxD8M{si|TWp-86X zKwY&)6snDab>Ku|Mu*xwyxM@dRr$(|Uiha%Hj!m~KRh7(q%Bh&t~d`~jTQ}4wmU<$ zg}m<jlwep{$U2A?LCs7L8pRYXd@y)nV>37XPu$R(aQ(y8q|!vYI&2p+I+R3QdCT91 z{au!-vxdqO{++;n&U<2R#BsnHXZ2B5w-$DS<+R3g;3Canki1_j4O&~fy7g9C^Tja{ zLq2j{spl6${{Ct7tf{gqe#B>t7#104sY>d61XWkgAH-{Agq{VCMDcK9k(-uhZdGf3 z*gu|G|A#y6v}Ev!ytoA^_6YYp+)^$6!r&-9RJks3MwlE{4&>451q-~7_fij89K@t< z_23UQ{*qfBna5ac!<;A*9sqa!-IfKMWsx7Qd=R-xcAZ|<SkKuPgJM=f3&!^2uYpUE zX2c9?b5_PzQrme(QyVKk6>$x9-@`5C%lT5vOPeCa#=REOogHsU*~2(})!f<8+CRqb z^d&7vukSw+9(!&KYbbD;_<t%fgGOj=(zSBc&WHgdRcG!Cl<Hi+3jKlo=W)x&2oSu> zR|||*k^$3bguNg{x4PZoxFusl=&uw8JQP2;Jmi(zOkb~VEBxXQHw;KpDJACXLA{n! zX7s!;G8FaygZ;fae@66&JI0d+F$3nO0T{xirI76fHdlu5cZ~g79(JGWE6_-8IjQoO zb=}{rK2<1tyX4kOZ#ufve=);__odJ$E9yt!LZ*gza=wx9cm=o6_G>HN5L+0;xeSGK zBtF13gKvx*g+l@O20Y1CSH=PAcPj@QBX*MmJw?^0Q(9~{X0jHIQ(7Q-!<ti|@UUu( zSbwb0*K9f0k+UOx0-Qp&4-}0O`NQy*&F|&(tOT;udO6rg|8$|m2;9knpP%AO@IUf* z<Y|s=9A+i=7(B?6kA4t25_ru&om!bb>-Ap2sB*L4^P0Igc7?SKV<UM*+pu6AMRn?8 z7NiIVDbB#>kto`R{L^lL+CK`T4r!I>q3Fv42Esm<z_iz<W%ki&8E~f;`O&U_J;X)J z^+{^`Trz!jpPc9R7>8n@e;?hrg2^W79r8BvK)hG5LPtZ~!Vk0)+dh@M6JQe||B&WA z2=F8%E|@@~u5xN~aaCU|U=6{me;R#sPVDuc>%qI;13B#6PEzZ5w_(|xW@HDV-$bv) zeQL_YX-JtG#MAh<QhkbUHfBDIdcTv7PcuS`V$p!JxDCsgeG}}YbGBiLR1&)PWyOIc zlmb_sWs=F)Gx$(q!kCKOV2X(?y!=pH@1uT*TV)`S_Ifm5+-BfZMN8VRY|2u(x1&Y5 zNM`S%@B8Gsia!G@YO5vWQv|EHdP_++xvBr`b^mi6eNM7|vhRV$9DS}P)WO`y!XGe% zClU5xrTp7sHd0!CSMK#Mv23G6hQjTdQ$5MzU<sdpUe9n|2Nl2OFhSHWZEk95f0<|U zWasqP0v$|c$sOPLy()LA3IZ-Mk~Al}vPV=Ns$@9W_^ZMMR2gJRzAm&PY;nr`F`P1_ zv4&3cexZfAR{7Qw-=W5c0eb_y=NDUt8zu-6nctPB-6<B5Jyr8_qnt}zkLFB;hXuHV zXu<oWxtF2%Hmohd%r>Xj+^GUmQUK1jqaf7y4r<gHA~Z*@D<2Kw?&e5Z)-Pb37*wWc z<@wb=(dp#S#<0HW+6elF&}WZ2R1i>oq3e<a(-sm4G=8rP7h@Z>eS;@qkJ5>|G12FM ztOV?-G7PG>M4qDi3%wr<=<{&NOs4Y$ttd}zLlHJ(q|jq~8-Kk&AriQlDf?H(;fw>? zniqG+S-JN$KPyEm#5kAKiP+fS@9-XTuxR=i!RVg=p!~h{?%-V4%-BH$ogc9L;wOYU zfsl~;0T>Oeip{pYo`#Dff(f9eMh_~5yq_F6+<g-3GVME!GsOCnz$l{EJk}mmpiP|) ztHkO98V_&>F$s(l_e#ODeYXox$&EK!>~!9@7Q{MW@UFOh+{@q{&fi#VKx~kBpMSUo zYH$)$aB>uLce>tfie_7kPxanMS2|mB4hH>c>|(@*%-O_yxi7enhJ+uA=sRps<@Obo zhJ#_Fc-h^5VKx?%*R9Kf;NpCFE|R5QzPJs`q&BI)Y;eaoisFY%-;o1ZZUe-bv?0@q z;cBiqSYG$8KD+^pa1X?4F+6+Jw&uFb>OXr28<XATkMxj$UyY|LDrNOtS_^F5&4NyS z8BcQys{VPf(cEt`rfZVY<)=~J4TOHn-Ao(TS+F0jBHPsv6L1Ru23GPlgF_*=`MX&7 z{x^sKRQmF@nB><6vjq!?y9n7|oS~W4snI?ki;lK7R>Qt*tB+LBpW``x<5AmeutZs> z!A#$v&w)^_%}EwIY3`iuBmY0Irz4vy)n<e(>sNQI4SN=uDkg4W`!X#f&opYM0yR}# z1(fOL9|49d&I!ny;b9@w#A^JbaW@$joS_llEe`G+#7p8-F=_Td)?J(u7~sLmO2^vO zzx#_qLhQ%jdgD;86VEDo)Kr?=a^q_P9^E}ZS)7X+CQCp8aU0eI;TQS9Fu*wvHdkRo z!4E%Ls)_*-c32AEoEtI`wJSow{XGqpEE;E=boh*7b@%kOangv%;vHgGW2x)IJ13HX z<BLRQJ%cQQMe;~pIw3$l8wUhDSnmn~n%PRpE)=X?14!sEP-FOD&N2j<35-TT$BuDx zx0;o$hn}k*sJo<~8(g!>Rq;8WkeKQRVU3|Q%FkR(EB$m`3$}O>ezK*kEiC1iHz`BN ziqcT5jf+A0IH25c+c4qPQK*oP$6(9`8-Wpq8&i>m>hmBZbZg>f;j&yC&iOSTvFTRg z1Fk5*aT+r}3GsgH7(g)Pa{B!ymenJUIVeKEFF#OMrr;fHhZNe0%u<PiwgE^rY6s_< z^jO?*sgG^z413?%_VwMW-9%B<B=N<!>@l@3x!FG;0>tIv|5;#``f0$F;wW=PdDWLc z61xwBx3*zOFol+%_mdtyWRUE!P!4P?j`oWB<vHzn36L4niAxD?K+8n*!QaH|&BIj- zhF4En4YGH9Y@;|kJ68)o&ozDMqO2z$WXN;UmMA_s82OcMvRZIEWUQQl-i4b-lakjJ z_1Y51w9>R#7M1PamaB_3g0N2ngr20#1Mbf16T5TUn9V>y?}-cM$9Mg0evEq#$ZhC8 zlz+gE_^m|#nBDDdSi6!q3xA)Br^oFagxnjE%nhQAO0AqQh2V|rNk0`}&yq>P+LW%| zUWuw&Gu_`=;q^YGe#K4tg|(`&^p9c58v5AGc_V=IXvpx^S;HV*$MPXv$HQH>tX5qX zAITp0=KS#Vt^+qiepd1Yuz|cJbEi7)b!hg&JZ4a|_bBI17JM)Z)W{<<b^{x)(P{qQ zZ+A^`?Kr3A7gX~I&G4a=vmb`l?uSFWIFA3m2qqxg2sc^;hrVJ>=<m@6USl2|C|C09 z1!z-G3_J8&oWEzku)Se8)5=pj?c6qOumhhGG_O|6J1~aU#SOz-je88{Z0WCt`r!$) z-Swe0m}uAhUbJMlG=^#ey^*!p%RTq}?<istS%xbRr`!odr|T|#7DoIN5VvQ9r~P9a zX4l@@Rk#}S`$R#gk$+b0zh%`0Xg~j1Va>9;|8oo?fgB8nc3oSqCjxE*RK@P2t8J}5 zl0esO*BD$zY0!_55<}KTdeqz-Qq~?w`4>e`^OpeI7)oS@PQ9q{dCKL!&+}CEt)yKA zm<+aP+fi}|VwOS|gmxt6Z7`+>m}gxIbY7wDfat!BWc~pPyq_S!KZdnuN!SP`@FlqF zfFCI>94gJ=t6NQ^+pSHHllGPpE>7s{{u(hGnZpkwlF<TuecTvc5-MY^;2^cYDnaU( z$js*G%p%)kyya4tC2*E)Fuoz4gNoC&KIp-9;#I=!A6bzX=h_MD)5oB;zLnl_>w9Xi znxTC;8v~>12+6Cwmk?36CV~J1-+S6kk~Z#b!*({dXl`yKQ=dcPMYMQBEF>4uzBJ*9 zNnX6&STz*5V8MDZ#Llo^u}>_2MNDeYqjhkmx|h|4RZgFkNl!M4ZupCx>_7eUcN~Eg z%DOXnYwhKgn>j7JE2`QEk%S+=c|_p5gQe83<@f)V<cAPhd^zk1u<0p^j!MOgb&0VI zdZ*30DRd$nM4e{k-3xzp8$l=sQ+vNIc$NP3J^jleC7mx&X|W61X&4`LqOM~YI^y|* zgb-WbU}bnSx_9?@`njghe-47@F1;FkhsRERy#b@%J-U@4RrGDVX2aB>(3?T{nxX94 zXgFz_9UoUQ{pEuoeY7_#kt|5s%hdvM=)}7g(x8uG2`Y(y2sG>K-uv<JkH-~0K3Vi) z)2aCrrOM&5uT5m7s1BAMNG;8JwXp+pp{?JkH_KAB9{gYbpNp=-#i2ojFm4pR+d_vM z$?ALCeGs&Ug7pCYK+C7bdm(dkaN%!l1)V`L*O2McrY&Fcnr3Iur2eprxFhLM-dUHc zr8)4{ek`>!E_*)n=&kH@mnDsBT{FLlPy%!sAS_lxbwLWDsd6ZlGzw}*r;u{nz-Zb= zVu$~u`W5&g3o%Tz0|eN6(l|k2#3k1LoC-V65%<wsc`UH-(!}*Bve;_dMZzv0RDDB% zH+3u;(Y(P7=bS&yYvbI&&EO>)bT@M)(EN9VCPeBcr2L~9Bwv5#K0r-0=+?oU((Ts4 zy54Kxrp0!yQKi4-xzRow;uI<Lc8;za;tC29;`Nh7xx4GQH^9ZP&|2Iyo<twVY2e0u z4rk9~3?;SbaHQG+;%lhHzdi011WDSlyGy+Xo(7$;^3ERh#W&c>Q-i0k@g{vdMvgow z7*$B=vz)BSEt^V-jPOK;F}>Zf+U&2moOex^Vh~P!4At76HPy|qx`c%zsF2{T+hlDn zGlwq(6{)+Zolh1*pso46XsOlZASG4S2SPpO=fK#zh9A5aVzS_<D|u3HQzAk&r{%4W zUf<_8({{@3>Q1-dpoT^-874JB_RC5mKwh}+#|Vi~9BABF=RkSj->98lXpgsTC)^-e zy(sIfGfSfnYIzRo?~Hqj#`xGgD{Ifwx^d*q7(c<m+v)Y?Y9*U1TJ4YzBNB6`-+aw9 zmJ`w%l5((#TF&8o-~3I`=bG}mEl%-AN{#?+(A<KWOKta{GJ0sRTY0QY0hnWhAgas@ zBS9zO)TZ@ANqAArSx7F1ZV1Wmqq(JQo^od-jShJDRL0SYksRmvw-~NYgj!?qG}8qa z1djA{OMuCAgg7WMggZ2by+R|)26kj6n>*3N#8;*|x)T_Rf^IE&BTnN+amUyiNgPzk zc#v<NmQ-%;o5Nj1zEH$sm=sr^B{&528>0Keq}F^4U-EM4fCjYLI6zDu)6Hja29=KT zGWkc1QyTdSficYgg}wI-YI2R*M%{`cO_W}gSm+3d(nKKJ0tg65M_N=w2nZ1Y2^x|u zNC(-1fT9o)A@tA!N=bxJwjdx~f)El^x+JLakrdCp-yh$3znS;UneTk3&YU02KxXog zC-?o_YprWt*R}Yo?L|i>+jlIWgrJ{qzO5X-r21;q+=Njz@Hzh5#Xwxf&?b65Ho`QN z?;Sp*?<rzIn(g_uQk2<Nvf6Ni<~74$%WZYbj^{mliz@54<C}QTSYzKfTj8W}&2e;| zRJIeOJIs@j!$CTd-4z)5d5SPV2kO7Ju~amm5v~o!pd-C|6LRg=U8tgnAb|iU!*GT! z6Pg|x#<(PYyz|b|fbz^4#m~B%M(l^h83T2Vwl@3OB7<+OPG4c6Gp<z^G?kh%W=*a7 zMTR2UvtR#^srw4_4k^Ba^TpP723aSb)&N?a?AlnyWw!RqPmc=js|7a?uV<$Tu22H? zXXpziYx}?ky!Re!A{c2E?v>2&Kf>NvcW2_`DJ&<T<OPuPQIrrS6sl*HFe%;ZXD{$` zYAj9BK!n4#HA#5FliyQPiCEbhck%UxP&P@C;`F6u4H_VOp?ETl3;G%G8wX4{>OjmV z$<;x^_mL0tWZ5Qt`U$YT0G%18opOP=tQo`@Mz|WOWRr#6kDi?BQQ6KC)XW27{4!_j z%hU4HCevRHXC2?qie0*&=b~t#=*nNWyZ9xeqNhcvp5uPV!*us`cUJBd^>Kq&7u2lG z{T1z8N(`4H9Ub=z?hpqOlk^&#OHTG1JX2fO8Ylet71&g7cs)D%=HqI&*qfwrmm1;q zlL5QOZ}MN%#`yi4jp<FkWg=wIF;pVDfm9IMI`{M?Y4pv-3zGx5NX5d70}5APi(d0x zI81rsB#G+`qo6>;_rpvN(A+(xe2C=4P2fOONI=MfB1(S2&`f?^dEpd}Cg#ctqS7<W zlBtq17zYT&+WN*Yr0rL-%oP{0=qajV29E0oj-ezvNTi4H>tt~M@v|5yegP#I_b<IO zls(l4+l7_Fxc^($eP~>uWMOw;Y}tL(VG*dR7bdYdg;gA6+Qau^%-DLbOP%h{4_YIe zD3Y^^O}}bB9be8HmD3jw#H>KC#`;&TkH>~GmPV2eN4VHkmiUOWy^yUqnSeAxjmV(2 ztVJQZ*Rf2c>Cmz(_2l)4{*V@XX%~{OxgXJClFAH7>h5$xnp@baxm&fskk%fbYDD0) z^t&l~^YJT_=<u-GD-W2oFLUP##8T9)vK8kGjCGFcE9>ZJY4^X0KV9*M@WH+3O%Lv$ zpWhEV8!!2O3Lxx&m?0I0n=is`3&-w37Ku)hb$kQ>Jr@1~#k|Cvp9H}GE$qt9zzOq= zw{|9HL~lq0x8@a>pl|t|`!her_)gsVyJKx{WvjowFow<wL@d}xda%d;)f83w%R>~d z8BZKK!nH;3SRlT6ilXi4LHuE<!Se(XBgqUEEw=uf%p<<Qbm!2xKnH1dZR?y;%)J>Z zM+6yAvV8kRR9o)zyh}=7P?3uF(7p@i{l)Om{duxq7Q;PFM|v47WVuPW&y-8~!pWg$ zg$mr0UDYcizuvW&B%M-aRL_+fCFA$;{mvAA9M|yq1EIngD!Dzb>2P{ThT&pRp@$yY zTk839R~pZNjpxbabLJo-8_ww+1HBSFuKs5yB@4Hka+$6iLJ_spSp}QQ98F;icT*nV zl`t^IQ@A{k&)9NJA(Kx0?sl8-=)uFlV!&a{<7anFo-r+{rb^ZL;qlXla#rx*6l~UC z7keS-2dS0746~d6<a_3k9+<*+P}GHDO`7;5f#x#4A?7dKwJeEMtb*mi$sbuX?~QJy z=*5&9KXgyI8E@$XZ^<Q=(>wSB_WIK`2anr-Zmsh&Lr?zR7U{WiD~%goWO4HH>L$is z(qCl4o#s|-bvLAuVB707SE$s)cI_P616+bGTx)ad+~kJ<Sf4N+XxcgISz%T)TB?+L z#nk3|UK$Ue`tgB4v>8pG3GKs^lHX*d2$UhkKz1_c?36%`hOPoAc>}$-7XZOrQ=S8h z(u)!gnbTq+07s1XXB&(B@OH~F^~CBAAq8=t@pxoxS>e~3BQrCCSDx-6vd6w=S18}T z;}&|BJ6<)K7~=Bvb;WqbYj2lyOoMYaGO{<srAo}yG2@w<qJ~^SqqoI^=BuG+6AYJU z#V*fEMS3s1pcLXEEyS86SM4jm4u`@++VEby0ww_R0X?qW&~+;30)!}o(wQ+`Rj}!+ zKQFK*hB8A-Fe;;#DwvCD`l)mn2EhhJp401wi7#VU#}mhB$iJ;Pn$M<|esAMRWm)sf zq9h>=_*%kC!H4|0gX6g18AO7=3M%Cr)U~zf%C2_xeH#<QXp=ku@D&P~JxEMgWbtbo zkAAL&xiqzgCdXzxm}X>mU#ZUx)DJ=ku`n##ex232p7Bh0$S+qQ8&9;CwcA%{7EaQz z>j;TVw0-=9EAFx9{?$c$<^8T1vn4u=z{HGC&tK!~%h@^~+#VI5<eH^LA^Uap-{1Mt z?yvygjn*}^HAgV<l4x(Z6qIAlLM6kx&4vm~Byc@*hhG^N2q%6=tc#@p)rk4&ySJ+I zns1_B6+XNl(LC8_kI0;pn+O9a9s<&Z_r^)~_!yyt0oG#zzYdSzU`<g?unxUXN_ltq zb+wiUL+13LQjUfYOlF#Q2*SNsG7B8p?xPHsTTK-?U18^s<U;Lgy9x;I<GHr9l|pU9 z5RaVO6rC#Vn_Z(%9Syp#YQ82Y6QX5G68f%QeqEZFXz3n07tJ<2_Go72W8#H#+7rGb z;hAP>65}_WIM!)AfMUnL#C4t~AI82hL#(AnvxeslDNUZ>UXa7IzE05BvTC8up6d1} zQedl1Ol7pg7X0p>&!L*t)<nn^hF03wKtYa8NRn1SLsQH6^^V7{nm1oQf!`p~VmN{P zY#b=@fHvQiWe?8z&ROMoM6w<)hJoKD7{bf^pk&05)mRhFk4Dcj)@}URPKiwlO!i=3 z>er~j!$J=xEi68_+2cvO{qn?DJP7tXvfhQ-PhW1Baxn_AHGEw@O7*Nz(DAP>0V`aR zX>?Mv<z0T;;Klou3pZn<HMzdmEnK4*2N1g6xl3b@=v2VJPp;Uh{Xpxjn%s|;6?a+Q z7Eb}(e}~|G9$g834JLw}#P0%U!cL4SzZK&u39`t<TY@1NX$%t$kyvM$j;|RKHoXE7 zmd(1v{J=)ClF>6d?Y3FiWWg38X=7vUhLM+)az^R6X)-Vri+_gev^I}qn*g>t12rAQ zF$R7bs{bC^3^u|AI1EK2QsGS|nF1ByPdAx<g|%P@C7GXKEp&Nnbk&59^ZpDKdpiHq zj9m2+xnr;)?c=P2?kG*6potY(;Uuq?9Hq%zowzk=h@97+uL!VcT0oz1Q+s^*v*v66 zsMW4gHJ9V*5mtTsTyl?^PlP;v!dM=)t(ru?x=Kij8f$!{6K5~2Fnqq(Eog{YqE0Cm z)!B6Q1s^&Ca{(f7MX>KEKyMfah%9}nM{!79VXD8@B=B*$fWd%T0e27IsF6sLko)?x zTc8TfF7!51CXA}$OuCz0f}N+K&~xMAUpKCq=;`WSx%wSGUjWFME<_o@jO8(Ytw0Ex zp~APVV@(DBc(+h}{33htjW)f%1QvpK6d6N!23dL%<2n!92Z=FNi)2QLFZRAI4xwS| zuS8y*GLA8el<Kkx4K@or)2_4V<!Oz$^v_8_qvO;g_ml4lCXO_Zw2IUc2gjUe8YV8s z9u<lThBPNj5n}lZ<-@egjps)TwJS{a*bVkRTCvri)p~nN@bh8MP+E3+_eBnccvA7m zk%QZ<W4C?UZgcrO{9!5a4M=%fn7H1|0D<T#_bl%uzlL(%X*z*1yzUUn6#*POaxuG# z69K(tw=k0mnYjD%aK(0?=V&<N^H)6EzRUkW<NLrywroops#djr#N}>6M6d?s)%k2q z3nFI?eB-Iu5g71^$>$!$g44b+f333^J2Gd%uMkM_BG_3;S0^nGTd{^6S(H>u87Ic= ztcz@X8mdOGc!a#ar)2d`>6g-ouz>QFEcILzVVNU6Tz<59x@ke2FdMBCBqq0PT@dNA z^^z?ZQ78E}HEfNRuUJ1rj%P4pT{A+*Q<$3lWhi-P7ebC(*I12qR!*K*tCz@FYqtGd z(G{z+)BOaClV?zQF9~Pz*K}onNpk-Lr-i^AOU2dAWwB>Ceb5nB1ug`>A9EQ1YxqR^ zA7lxP@2h+UF%zqXv12>*CbGaYyr9tylbn_mU&>yZY-uIGuJB(@Dex=Gzb!S#U}g*s zfc6N>WjF(?&##Lb{($E+vTW9BN_gj>be2yyeZN2gbB><{4_>!l*Z^9O3xMeV5@~4( zMK1V&1Xhf?)sn!wOYjLT2rm|*2fZ4LK6N%q_qx=j^Vg!rL&mEdWe@W6&2HVY+yA{T zqHppgLH)(bNatJ6D#xGqm7M600(3*(zAtv#=Py<2NYwEi32~I};h!mVM5K;ZrSk_* zzFat|WqbG0iXFV6;6W)NnB%FGS5tloc_+Fuxs@^={=B}S**ZGYpSkkI>VfQWGtHxK zq~1gyt~a79Lnc&tfz&9^mN_i3q?DgUnq*P#RU6fy9LDs`1DNh3(I_dCWS)N5-CJ42 z(6dyUTuU6s$N4Dz+NfT*aqPu^26$Lp=LI;5KC^?f0=A1ab`HN^pp1Qo+iAD~Y1EP; z7xWYTg`lh|44Q33S=V4@?qCpmXO-Wyz?Ij?3iw1DYK+SEHn5X^Xj(rOXf{t?ZPqo{ zy2Co^i5SsXA7;soaWcS#?_!);GA1s;MaH98iPLP2`Jb6Lo?M<3?8k(Ia1UYbva*=+ z$*7+?%e0}~g&JY&SC{s^Z?`vU&$wLV9&YZar*O1U`6+XGNZ|t6P{%zUu@3VrRJ<BO zI^*#yv`<nc#4SYgt{w5iP!vIwJltw(meY5+;UAKCxG@<nwlV8MNbPd#dhc9#zC!El zs_Bz6jftNfWqju=t!s>bZ#%t2%%XrqvB|N7>w~4z8*uCTHK)TunXpXUAC^j>PV13t zK5doTk{(SFTi%2k<_6ui<)t!>Cgjw*a;+}9pVzkRB)T5BU-4J(!ZD*j#feV$3OkpC zFA2FPU)lCd`|I|+YSXDYb)>)ZYVm_khCvgP;iPC+=vCqK#QLSoXui&)@SMiaIiAy` za)71vNBzHn-J+nP!}d(Vh%s>cpqbtRK^DVF7=oXf;5mc7Bs%#bX*3G92Oarm-K&Y{ zju`I=RO$JsS@kgo((58-tlyC&<(P-l?zXUdIB`@vD1vtiK?^Sgvvk?-H3(}yzFmlx zN#RNN&1a>og@2zSvA$FwBT82&&B@)kJu3z;Q`v}KZ==C?mU~F8NhdKH^O?50(=0-} zz1z+%!J1<-q&b`XQZ;_EIb0_M)<9UBsDhmOa4nV<;Y9Uxb|iD)EU-D4pIQqKe{$c9 zn0$2o(@1+5TWMNoi9=zfp+nZHy(jC#@z-9c2G{pirlggpq{;4++3D`?oOp5DwmsWq zf3%+hR&0;29zpjAR&>LsW`0>5`SNrrD|h&tiSoRr-Dv2~%qXW|efR42>el*kXREj; zn}_~KnWczAILm$bmjN+#I}UFJ2j+i0?A-e0a*P>@);6Ebvmf={uVWL-;z4;6_iOfx zx7>F+dhv;x(@4msBu#j}FA<3EDEBDxmOG%~Tj&7hPbx)@XE{;JOmT=Vr7|<+zS*Vw zjwL@;%<`;jeHG6ca?3t>sI@N1>+^$ip1xI`<0anDuv)#gcbZ0xij5peO4-mA&6cHt z8$Y&mKo0)Fm6I?f(B+*Nw^YDf$({kuFMvYSHXwN)-ZXXpTKfb0U*>@&%j}*P81K=8 z(YE2_{pfF04u`D4;<K{e-i-^1JNYt{uO~3Y&Q}s#jIJm>q#^Faxvq`1hL$);yGRjE zp7H5puT4cwR{U^e{M(k2-30s!LtN)Ib4C8+3l{+xEHnrKsewo<al60^7lN7|7RVx> z9^5cah4;dA!UricuBdM&gxv3hRC8O#hA{fBFB_E}J$QtwBFE}|SiUMa!d9V_(~wFk z3B}8D+4Pmx-ug=Vz<#ChCt-m?-<-^c9$M```nA9EJevrqO(pyQymRtV=oi?sO6;V} zieR&tb*n%Ym(jA;#-Q)LG=L-qgXSZ>v%=i99Fp(d%FG?NHBQC>|3vvL09JI~C={xP z?r^)(W77X2BuN%AFIbtOg-?^#4Wu&jeXJ@>ma6mL{d{#N(RLyC8oseu(eHbEZuXBD z)F&);>QhoWxaCiin6q3ouG5xmh6#f2F%RGx31$(EHg90lTl6=^w5$}p_V61Zw?p)) zC_>4wjHW8O*l7Z?HE@=g#t;k^^tn@B1t`BYG!B3Gm5+KsF(D>UnUyX@hYKpp`}Xbe zWc33b2<AZW+h~{Klq%V2l3wzXVZCg*b#?G~RH5e?-;q)0JZ0IligwO<><D5~sm>4E zTEaIH$mly^@L*IX45;UlaCLp_JJ<;f$rMWiY#(0591hI+PC~<RFIOqDm;+2FA)XX_ z&kOptCKSyc?58wygA%<`wLYq!|Ie->&C%*l6V=!Oad{R!Bu8h4GKQ-gUP}NjxQ0-t ze)k5bUmN9lQl~hWF?0BQm=oZiQPvDtK|T2t8DP~p8(4j1yxl#TjqCLnXXQXUx<bo* zLOcc-+%TAT9>*dg^GiK!y2KWKwS0B5^|MP8A+vg=zw`#RC_2#i^KeQ<V$z;wb=!_m z&k{!!An4iSene@6pj6>-tN)qWS&5|27mMS62KQ=djy|tH(KAMM$)Y+YDov;_^%}Sk zMC=Ois-^P0Gq_F*;E0QWGzsgV^&<3wowb(An(4(g=>G+vIZ4_M&SQRsxe2h7#-#>8 zB>i=ZdD!;TTvzna2GHkW6z9`*4}a?-n@x~>GS%k+qu@(Tv6z_O?NHmOU*Wn3#~1lO zLG&!fzC*MO_c={FIRg`_E1KZaekKf)9{s;yd&z%b{3j_P2w5Hqv>h_djh?T9yp|`+ z(4i+D$7r(DUkBcekxMlu|E#J0Q#Poje)H-Rm3fU-lVo#UMwT!#{RL!0Es#u%y7sEI z-A{vBV2v+cPLH{J!q<ORH|YsV#y-M`+MfN&Vf0kqEw3D(i5{n&XB4kX>ppWoKh`x5 zL_~Q_Z0F9)9-_~*jfiz+rT$TxWhHjetB#vh-Xy!i=!g?V$n<v$+}2zJ(~nlWzt(Li zW^R49+4Zx^M08)3^Q?n&d`7_TUY*S`jwK7ulQjWNyJY5YB7Of2_+=6DNuD0$%5X}i zH{8dIntO7sLU1BK28MS+z=-;M8^4(3BoC=jy~e_>Pd3epe{4xw(}*VJtuxaH_x2!U zv0eBEPoCV}dHwZ9uC?GjV!tp?T6BYfTQ6n1GVoxJ1&5ttIA!5NI(+)bk_`}oRW!!H z=7zEY9ND<8I1|UBl`nMLv7lFew*==T4cGTqzA$zAe%k5|$sl*-i`TQsD%YL>WMy=Y z|L60q;}NnaU)0oueAk?)oOoTjGu%Cjcjx4FsU)9!{S~LFVtEe3d&Yh)k*u4Q@TzSW zsZ8yW7cWZ(M9AStZ=3lJ`WhhV?lj=%kxyW!r*JGs&^{FO!B~JT7!FV0YFU@8T!A;e z#LKN=)dcT;Z`<D<%b1l1O$6z7zj?SM8rJ1aPG{q-TFoSqNQC9qp2@%JL!5OJKQ?}d z&6ga<(kU!Qpn2sa0y6;VsqP{P2+ukNgg7nSeb6LMk!Q_<tv_J|ff@`(>+3EUrru`h zm9WM7RQ5x*^Mx)`okgaj(I=yIeW({{!6tXGg^{%v+7(=EaW^$<70yn-@4T4tY0j5r zcmLD)qGOSt$^}`4*A-T-#g5}mSr%SrvRVo+U14pOuG)3jGV9S5<k$s9W@2#L=0t3U zns}J)bcJ0h7HR)v;HPV$?R&RE?{;uxAXpBt4=_$C|1t0`of+di<-fSj&tB7o--r|D zAaYK#U~7ac3-mn@g3;W_I@?ub%&H|Zil3wVy7DT@kq$_vt}b)xNc)3DuePMk&4sfg zLL)Sd^)MMvckgf2b?m}NH=8Oh#`|eqj4!p8kGaK(DiIBF3BPsGr9x}`*0cJYM=O&= zm+;aM!QhZ1tu+eyKHXMLq1Mj%=hc$k>$`j9(gaapd7M7v2t=3J0F6+d$I}8B(R7}4 z4iofgxn^ESTJl;cOPN!~rmU-#AO)gOV_XTcqz4AWpct@3{BG0@o-_N_n44KoeiBj7 z=;@5E*oENQCa##p)w@qC`PKZ#pZOJEpwq=~xz{gYhqq$-ddH3EkZl+ofDlZ*MpnFq zX2{f8s>a!IOYLBe(l{+4_{>{kPfvA3>5N_8XNKGDt?g;O324mGV-c9|SB9pH8x|{^ z-71_rPF6aU%AfS(8Y$V0FOOxY9ICd~SanGwc;-EV9R?{fvNehk7Pd3iEyBaGX~hFM zUg6@R1H=_|3I9Q{9OButozlH6bQnKDr-Bm>Sup6yb2{Wa&8TT$Vl};Iktw4@=t^$Q z5ep-R!gpcfP&L7<6T2uXLQ_(IsOGf^uO$Yo0w)M_8x_2KN|Xi6FWq+1aCq;sbEsKh zUWLjN;!?iO-ICCPJBUA-171z-=kg}M9BZ8Ek3V0ye}@%3QucoBOw>~Uiwd!(N|zez z8j%X`k3;!Q57aY!K5q@FBor>l#fQ6e1=Lk+w)<+Ssje-UDHJHQ#!acNFc^-Njo}$9 znzzPpKEs*(t9Ai_J*G#WwgGD8f1f@np7B)sju&2nx9}E1*Dx)IY3^@2VQy76yNZ+K z?PK7SmkOG*<{y8DiLM|k(7pCVsSl)%4qic6PM=Jf-x)bKTg#Mxd-B&P!h%^`1H2DE zt<!g>V<Q=JicNVFGbe+RMmGOVr$av8X)9yitd_K}sMRL{S<d+mZ0r1zx%8lHqgQh3 z(nqiGdbYaKhB~y-&)W9odSn?(%6gMxU()iMh{;`yN+VN}reIb0Y)&pggkVb2i2DDf zvj2BWWq)A+TPofdbBVRF_HWu+2c6{_K%k%AtCIz(=ha&hDlzi>Izx)o2f+}%sjRdX zE%lSC)RKuo1ThZm=#H^eX6HR*>U#Q3njcekn9yC*Ge3R&tyZv$t;MiFq?5ZtFo|mv zUUvl{VjG`eX*R$;_~9e=!<-zyKot<^DA#phDgOIePE?#UZ!at60Y(SMs2UXRQIRH@ z2w>OPat{{#;@9<!(o;~x>+hQoW;%xaS1;N=r*-$&qWmVYQ>XF{M}K<c`lZL*eD#&L zmQ{bPyo86YhW4!z1=olY?MG!xy$Tq)9r`6Z#Z~;0s8_V8CB^r9%&)6FqxYzZjurlB zT7b`K*%d0f3Xp>Rg0~cwf1JEPRIySpOEyRlh{)*+4r0Wh0ImV=EQ^>#K7yV0^tcn# zMcl!2nG_saQBh?DmQ}EB2|s0uKZNo<l4Qu^y9&P0X8jKBvUOUv=_P&`W`g<<($-(I z<=$Aj^YgD7tSojAzk`AtPQspK39A0J3r;bs?O?wc?&1jK+E_)oU@XA*nf1f7-|ZXA zZDXsv9QJ0z8)s&vI^h>I%M(2vKG*KE{j9IJ7BwBPe07O=`1iIaMp->>X*dbHTH9%B zo$TgaiMVxGFrKG9yu+iT@O5S2kDJjq$NZjQi|-RoA<f(Z<Z3_BzfgFx0Da}6!1BoP zpGG+)wxUuo2J;)K6jAaqj2XX}EDELf741!C3RBC;8a%y@cD&ttff}R|y8x26Qxo4a zJ$*(=p=SmKS=Gw-b`JQ|G?7no#RYx%rYtHxLmG4bUm4zYNk|n1kiD0|MZ3m%MlZav zRN%FLDUJ>Ap3|CA>ZQA}E$BTPis%zvPV+Dk32nM-GeUAEcx$|+el-JSlp+{ZvgOKK z1)SOUoE@T5lxHyhXnA<9Y|&G4zuKkE4t;ZTt<gt*%Yv`6GO}Kk$+r(jc2mVf8&E3| z8CnC?$gcK!gEC@N$^LVT?xv-ND{j8LoJWYw%moT`WP>O3d)pa)A~^XWHTWA;8m?}H zpWYtIgs0s3`G%N^kp2X*QGicR^%rMP_1Iz!h@DyT&|T`BV#oV+IeJh2R%RU=Bj4Jz z;Iwl%skF5f0q%5WV`g#T2zC%r*VoOhvX5ZurvX?v)C7R1_lkCminLByY5_v?zIF?f zdsun2aF>VV-h5XwnH|P2`RANZSQPiv;F-0vLCDCXxVTBn(ticBo~?UItAo8QYsw)X zm#3FlJ%ZA*O9+3!paEkE4u~#zybrH^9&4Cs``rQI(HOf<JLx^;_Npc5*s!*-X|-Vl z@Nm)T885^bTgp^it+cz9r%uuwL&vtH<UHcHvxY8aA|p$^KU9L;q3E#tR>=e6{{FnM zmuz$hRPC<$)<3Nq6l2|PkCw}xRSZj87ux*|3=B_}j}@AQWS4$}uc>j(C~KKq1?&L6 zQLX25b4fXG5AbXSULUV3E_K(@La?f=#l-KcaOsfSBUSrPM=*Dem-#jtNXNf>XAAdo zwz^~b(fz{h6Y`nWB}yOqN!o*_Z}r%{eD<kCmephY%hT#wEZW@gnAZipJvp~Z?-N9w zZ#Cv*nwD#*XL}-brq$xI0SoZ;gTG#+X>R*}@`soN>{aO`OA1#HBbtUv+;YS-hiR@0 zsB}ZhJ$OUkJ)=WM3vej{V>E5R6>mRfZY%q04lfD36go7zi}cwSeq~rOHnxX|W|zKe zW(XFEof~548r%)sOeAx0{mhUNEFRJDj%>vL8y={Rq9d^f*`%bFAj$Ov_LS|>ngji_ z0Q;jBm{4ccIr67&koT-M`kW-Og<d$WwFw`^>g63k%2(rqb?Hm(G}A_L-<w|Eq`3xf ztC*Z=Eb;A$Dcp6$Un(m6CxSf?iV>b(x#F)qQL_jjG`2L;Bt>+939tr)oOw{CANna1 z)q_)-`+B{D?MPWTVDCPzXM5W|y7fW-2b-YKFmdcJE7`64heU5Cnp#=!wN-7l)lu!( z_p519JLl1&9&)gGW@WAh;D&DJovmcH{SW^9!ERaC;2!B?<9ht%fD2+4Q@x67A(%ky zX}Y^1i9WJ`OZx72s@u4AF@YrZusj4NYIy=uc0t#Wk75qJ9v?kbU*_wCj4P!-A8(Ry zwI#=X#{Y8z7q*cO-)RisI>4!FWIHjq$dvbV?0Znw6Yle@xaYC&$VcpP+DDkOBq%nl z9O5#JGB(tI>9)Hw&fjRF7a6m6@(gsp(_en~kSamiMU^c(dXwvO#xB`(<Pzl-E%a#T zX+}g@(K=4u0~*cQ?-6ON-_AB<)S>a)h7*XzL;zNtAc;9xuVLrl4YdVP<y88z;0QD| zj}Tq0Gd)qAmq0C2pVT!HAt}t!+zKXi3%<yU?XN%5suM5_j&VnRF2*r?8atE&xR;HX zivb=DAhzeQp7!G94WE3vHS!_^FaKif2JesgCi#QsHdh6@qc0w~1O&|i@t8C)W~CKQ z7RFT|m97`xeQEb?*(85bt|i9XchgW;l@{I2J~3@Ki26f!pUZ{aw!b`;qFzTh*?5b@ zy2kwSFR|SsY)L*9C%Ijx7h0hTE6M~8Si)!2+SvStBDgJ3CnBBF0G$qp%9IcYLuWbu z6IdCZ?FeQ5*`#+%a{HAMOeAYK-EeUVSXtiYKeyCfcCkgW35*H+gYPIAq|>~^E@hm= zGFc0{LG7LrXp_Qp)dYF_qzt_l+|OCLowE{6N!wmoSq=s(Z4=%mD!@{1JneAh++z22 z0<>c>zFNPQ1e><rxbz+#TnMuw6)ifmf3BTnA74D*0{tj^8xRP4S`(tOr)`9%0|Zxz z`0cpFspgxt(Z~b=7pX+fWlCt+Pc`Y<^<5e<>M$Gd93jOWUr}B+aUUh=1QVmy?VUQB zZ$H@Vy=z;XrxkGB_DHUuLd7of4vUhMlE-eGAAy$~w9dE)Jjc#ue~$66)ob>&Tve?( z)4WTix8UVG?U}nk*Jz^oS8m}9s&qC}N%KW}z|{Yjxy_b)z?RMhR5ES;Q`|LJs--$~ znMLWtr%b6RLp!*}$jxK3ce$rAx2JA^B|p}2Z2s3)cxFu~sAL|7B-~TUnqRlReeL{{ zDbRKgqQ}q4@}H4~d799^dAt$NfQ5g9Rbp8_iM{n4u?qWOc1<pZd}xAxS2;7%WnAj3 zKyW=M`|hA!?8}%~#jJenC~cZw0t%)wx`aShheW5sA&r)<$7FM;`F(B}+>KuWN7lf5 zD0{GyKC`FIa$k{Igg0T5OhL(}*0BM0+xgd0N9(PHC*6OV8{FrhLteeszq@xG#_E~o zZ*~Bpa#lLi+_S0MwX28QB>P#Qg?ki+VjlzQ&*kB#|FeszwW1W4RC6V+G$hZXHSbBq z^vC6`2ZeDc*@;|!;+lR~Y+!7tS%WWa0HvC$8eZzBs{JP3D%OT4b);viNCrC1w&%du zR{V)6fYe^rPyGqsi8-N8h*p~e=0ETJHOF2+$y_z`vq5$22olA@`82RF*~wS8S~_Qa zk6E8d_^`?(O0U*%4Z&l4juHr`?GeGgorlxP`WHF7q0f7R<odd|63|)9zO~GS(DK8g zs;%`<>J_$rC;ROoEG}4Kn`r$m7`z+pgu2IJdsZZ3=$zG2`_259PBF=Bf2#7@wcp!f zQaym#bY4BK;1|BF$=M2vjr;IF7!&^YV#ps)YeK!@N#Emqy8RDNGz|(<xAr{2)r1y& zA>l$xm$vk>N@)Xe6)Uu6Lfp}tNLnE2k@TSXH!~~oi|yHFFS6l^IK~33X=B|sNIwNP zkH>E(D=$-aEWXM$*W$VgJ`@nMfUXQ6K%Bk(0ZuYJSzyE{^3sawB*!yry?CvDif3C3 zX+`fo7jwr0U$H#3mu_+v9NHMXGn8M3?WNb>njH+C$IU2n@Q6J;bi6v~vQUDE_b4KQ zBiQ%&y_N_39QH|3UqS?bK&VFTS&2Wsb0`zo&vv;SYQJUib#!#HYm(O6OPlNO@1Ik) zKXvMSKr;!91kJqC(%jsV=zg%}fsEC;KV{vwmv8$6Ve|0Q_j-0kO1JUSSiW}JSb>Mb zEXTG+uBKyx^ozG@Ty-zkOd%@wQf-hrG4OV7-c*<c$Ioc&vlHZnU|qWXD+@plkovi6 z-#&wkLm4|E{VR4csPOkTi47Q-<I>(Vo3ZA*F3dX$EbLqO^7@dvjo?t$y7TXCBfk)r zMi90F;%CHnaqzI4;du|gfvkC&z!%czZHMtvH-rAmx7qxcuY!x&Y*BJ??@#>R_GSgQ zc}6fh2wyscn+H_WPfj^-#$~W?11U?dah=EEY;jyjmLRMDzkYRVC=EWrs}kJ93xq+{ znk~8haZC^|FZe@JaPHR$ft(~y&PR}*&yzj^947lX<-fNr@k;*dmx<L&C459bWeEho z`DpmBgZ~$ODoc18oOQ`U?_7I6Q-de1-?#6~$|iIhK27HkAW!vTrN4*6i%y_uRlR4= z$6R!~5T(r6hdS12OxGpn@C$JB#Ss`=s1wCKDj3D>Ch1S(IHltlcomxPW=;;@a}i?= znXtc6{gpH9nS(K$+t6@WG4e=nzPDAIRb&-}yEKA1#q2QdU$Hpju*dJHZMoh$2Hf#Z zO(?P)qH=7YLf|H5qEn2y$$!%HltROEqS#Hxj<ymKutvO{bsf+>#;Np^qB;C0RaP%U zR8%Iz{s|_2hs%d}NWc91V{6e-+@AS4m6xoCQK7Dmwvi{bF8L2XX(o;kN|r{Z<A>Qe zFM_yRuD83A!{*D0+1l4(EpC%DFiMnXAqEB|2C-2L{jQBPS{!BisHHm3jOEm=qwI8I zh^vbSeTOo1HjO^CEw=5@_`{gX;WN)7{nHVmaiF99pYl<EB>U-tBo#mX4GdMxdVrJX zSplwJR@d--tQcFq8=m5?Dfnk98axX&JQ%~__`;>BAGkFqab$k$gBnhp@hH=&@dM3w z*8cNayojncfk_9`fkdYO-Xh?alhwhKJlq9;P>_JR$Sz8cVM=aC>ob!(v$F8pjBqIh z4v8fzW{JbpXuy|lqZ;teWz#P_u&+vqvejRVACVZZFMV~~wN%lgO55=2%H;jn(%rR4 zP0~p%(<+i&#du7}B*EnF84q`*QU{Mpexa7}qtcf?jv}QRfl({24hS1l4YrQp!HAVi z(tlIec?vH89gMg@nGPx2L5Xa!PRm{60u88Zn6W{PJJmC%3ABB23%|FitjH-PRRFsF zF7J?^b#Iv{S%<P^@4{~Hxa`4bRkNtgYOBMyIzAgdP0#jB%gNk|R8usi|9TW8V$|xI ze#h1?(x!0ii|>lIZ_8AkqjN?L<!Gr$kHypb*^_!bGM?{nfQuHY5BkMB$fno<L?iHw zs4>3;e?SQxmIaDBz_84ew5=lp@*|KVzVW2&<5|z?jq8z8)JBiaP$%WLPtUisBg19R zElWLA)0vA}CO&8?*<IW4tguA(%UnvS+=7Sg-6rICl?ktD`HZ`^!|!dk_hyu8jHnM2 zG+$6mDM%+s$8${>s0$I}WFgRqQ6g;!@h-dvjVgMo%8}71`aM{aQ@4azSJ8I1M{&jS z9awt%m4)FXx(F$AN0*sUeyeeLgYU|n((2n)*EjtX4pW~*#usWk+PV=nlp<QcJbR>O zqM-X(ZMS2X4r#ZYk5$unpyrUBZf~}ugGMkxv-6}ycI2bWPe=NlAFOL^owB-x6=E;O zV>O`@RM<YOGP;e5wO~nRMmfN{i}tb!2|sG!-Tv}GP58qns&y>7P|SBUT&;P+aC-fd zO<M4^sOTFBeu1Gb&z2pdDuLy!!>j(|4tB>KhmJcA8cr14A(WiB&v4mS;b?cW!11_) z;h3j`QHeuH!N~I$ui(kQx5)szWUsm@5+aelw_Td8e1S#GFt-PT+h|$$+MQpZYx}5k z7Ob#eqddR)j_{4M9Yq%UK=$#njDB(K%eB_V$FZLKta+ZXu0g?VNYh^Xp%ZD3uC?_# zhd77Y$Q85(hdAe(9DVd!O|$J3?PAV-CfDwGdPdIVafefe@4+Mazi!e0<Pjm`jF9@b zy1?p#wDFIB)0-JE886{$U=y33ND0EH>9FSR#tOL-yZLv!Xvs-fMb+cs%4#^3Rm%fb z2b10<pa#=SJ{W)Gjq(5bp$~a)d9IsqFMm8f>%a}fP6B13Cfd{aZ{fCAfY&w2p>k0C zq7#}zxS)@`1CTz$@A{l!P^M%?VBpo!lW?48E>8?{0vW(x^qrRbz#N+w+UQV*FcrQ7 zbK-sT^Q?EHJV{33395(dKPA<pr^{Yu8w{N2f7e~Fx>bs;zyG}MgRkWQ+(5Y%+>L}Y zxs64nSF;rgKzo02?WuqM2qyP$z_ocF`9ICI!GOKKQx3Yz%t}Gj6MHF;=g3#6hBa(G zLJ5W?xY#Inj3X4`L6S1T_ybsZn;_K-sx3tey?Du{yT<5gpSR<IXCK{m<qRI$7Qb6) z?`ffHd(j6I#;-P3IT}^Cl>if0U&PBCr*DR(E^d`$Rj{;YkMb*HEXj~A+mi}AfZfB> zfM(~P3=St?4KPM5&=eN=C~qr1rv!pQrhZSs{>;NS*P;f!$Foh}8ZAfvatCMSt!L=F zoIfW7+n>xu^K;-xIN;L;j|x?r&mfY?yTA;@!kGD@XY4(?J;XP7d7eSbIYqy`%7M+} z_qM7KId1nWhN3>dd``nwOyeugguFcD0v8_*9jSW$B28QLe>Oe;&+msH0O7s_wiyfZ z8atqUfswKV4W3z5OQOGiZxe_b{NAPk>;#1XEZR`d_4>VSRSmsG{)^bb!2Oa3Eqx*& z@sA$F0A3Ac4EzO>rO`5QGRG-?16-5;e&`1RE|^4YhQgXC{HH*2i{N|uA0a;1S^SUb zU%=)lAD|XEXyEs@%GK^@S$MK5Pk&Pn&(;FsJZ#Vmt`5h0ys;Jp;Mr2{so&dHidk`S zJ_s!=sI@i&DD`-LLH^nXcyn^+j~`*!bJ)uN79haWhXs6+&g;0((m?wHsQAah|AhVC z=ANHY1-KRT6iNsJEDulz19sjZqIsA)9FJ?DCklixMvedWmV#h=F?ww9LCE@m!bg%% zvy{geWiiSj*B~ZwFWQEB5o12lM3g|Uz@^bS=Y0<N`wUv@M3dUiGxM166|n`A&W&U^ zuth_p!gg5dLoouWjYPb>xja)p8Nat(pHHk^g0cQa!+Y)o_GS{7VKZ2T@ih{dtg93@ z+l>pZZ)<K%e@7Q$|0A3H6#Wg@h8GIdJ_?591;?Q~(FXt%oPGTv7~e<ts%N7fU@U4e zrc>V$;)C-uw@FjxCd^QK%HE|h$e(GiYhyq0W!$jCdvAh!W4l4G2kZD&t(jM4I&Cx; z(^{ECUaJc|@wvNqvIC=XL@}}V3bJ|mszy?gN?VE2L!$TzQ60sEt;#e9Mx(Z#Q|!?o z&MkCIv}*a)C5C~*1^06u%_@axPb!dHw%!t)H>70XSZ<OtW__Fiei^_}P6<TSYA`q` zjO)safKXf`Kb``=mJ&!wgw?}33*j6rJ9jv#{RFFIL4S?HKEg>I`NC#~c(h*qh^9LO zt{lopx%|gc_;;-<&GAFF6i?nIVA{p`2o|mk%Hae;ZQRq?5qu-;FZlK04`=61`fBtU z8f$OaL|;g!0=>Er(@T^^kJ+$WGTQwBGPs@2uVTvI=cz?9{7%?6j_Q4R{l0+12x@C= zOJcgUaVg9eP?HKo7TCbTAj9$m=1+c6aXUM?yU_mpB=#thydQeX&Uyovz@7-9$qYo4 zt>Bc%;*c;??g`NA$alHF+eGJ_9jE<p99X#Z*m()xMBquMb4>X)=6--l!j$Z#+|O6S zXJO@`NLLmj0TVb6Pld^n#n44#ZwfjkDlw-fNc4DtDT_aqDmXC2$SBxD(EaiV^Xsps z?aX<5>Y2hDmWaPca&27q<#uXMhJ33Me=7SIy3+lKD2`l{RbyjNq3Du%eu_Fu+;89g zZK7f<l`qgRu5v6X6~9qw=8@KfbTXs{97(Pav2$=GB$WF6AuL*1P~u+$uc|hmP{Ivq ztbGPHh-3MbdOB~Pa(9tBztqwKB6Q=V;PXv*IqY7{6*fJEtO->*;J{qz8aNno8d-u= z5rPWKKOkmKGZDfjydzlUG>JWEPwV?jlO7!JsJ%4rvyxLdlGAz>_fNqfj+D%kKhfv^ z2k>UX_@TJ^;9)fa@``l0Fj+2CaB!;K24qZhspwEt=5Qs{MP;nfo{W7Tcw*V}0ug%L zr&EZQSe76)`&Kog*K-B;AzvT@oBt1+!Q&Wd?sn`9Vo&?Ng&O59m?Bx1=QwJZl`Jp- z$!jP>tPruVkww<PXio8VvUhZySW?<Zz+5U_Qc)dUkoWc*e8d?w8Z2-!s(QN^*TDiV zn|;8maL{a{n=Z{e2*JvN=MF<tU@571amy2-mg?-FZpqY`-`nKP4>2x*3820(=|Jcy zbMP{H)%T$}f=I<xhg@Csng|QO+Bl(4)IGY!Xt~rZM?NKDmjshz<p#U;P7%Dt@@{SY z6g#U~D$-?~sdy#gXRm3O`xW`v)7tevE1v~3zE$@fc)MQrW5xN!lF6W*A<Ne}{oTi9 zM_aDf)rPGRs!dU$iDs8**k~beVBrcaBUmnBM;1WSgB?np69Z36I+Vlc>uK|v;fzCx zTm+T|Zy@wZPA@Wv@$Ip!zDw-uPuTyCEF%hm$tnliA?I2zXbF8MPKw$I%#N=M6bpQG zt!Y=nucOTQWgy<uoESedz}yR()57rj4ld>rfV+UZ5pjhAQk-y5YG_QY%QE1ZxCTkU z(+KD)7)lI~r&4G~I6|YA`pXj42<RHJwUm(ciOj;Km$m0O=Pj)mbd-C+V=aYno$9#D zu<0tMXA<@(<^Z%0@U7kFb-<`ao<ZHDKnD}T{`r@s<Sf*6s@&W@h84iD>DEtaB_;~? z`pwelh?yp`^2qnM0VGP%kZ{^4@?Z!8RNC_o!4L+}KikJcuuXb#f9UBO!BYe}S$)7R zK)!_M1JM~y@i19laE_0k0~4PH24>zDs7|9Lfr=zD28l8*j;KN3`t`&1h181m*FP2R zM6XmV+<g4Z$dpkr>Q&@LicWH`H#Hkyc6KW|VEUerT&3h%@<VHEtLKI}Zd%9gS?ta6 zMSC&tB=xH9A;LuLX9D+ZvAwKlt~NIiF;}BKUg&kzvcr*BSK<;8z2sW?d)ra)p_}lH z!||577|=9{H^o?!fn@Mcph=XcpoZ^J#5cmp%GeQ#GFcu{r&E8@g|P;!oH=tSaGofF z-lS2%D1x@vyyrz4BTN!op`p!K*fsR_HIyB=SQ9j}V6%n<9GboSzbS!0e=Qi_`auAr zLoZG9q}ljx3lq-$tZoFUw`%@wjYcm{`Uh$o2CP1aovC{XNn}-YvU5Y$d4Mqobc;lm zEjJGnmLBA55#CVhDJ0lh1lL$F4aO_quoM@J!o;gFxOoZrZn6rSkaQy~dRPWJ;=@8^ zU^EsbCD2uGKwQJsBE5&>d8Y$q2gfhYB4aq&DSBU;uZPB#ra}$@0+Cs+3RJ?m`g>cF z8Jp5&OIF1WFSx#%dco@Jfu~SpK^|N-)!m}hG1*Q&IwsHsUs&%}R{D2}d}kHHZy>O8 zCiMM^60Rtq?saqWz6@<R$J}~PMqHtAq3gTcQ`G!@NiG8ieAS|sTAeu$y8^E0fAlCZ zWqvADQyfoAi_*4{b&_}8yW&ygQYVE0rCw0?*n{AT1x?#Y4oM09(Sqoa7=rU~$&9hI z)$htQVJ`AyN;v!ZkHHdF;iU8|<*}2NyCEGGg>#VvCOoS3aBl&`Oz7VVGLy2$(zp+~ zpsUQ>nVX-%6J{s(Zc$RP(leKQAOz4jQquK<d_oK5T^jDu{&}nL`77o6N$gHw^}<5o zGu8PuK+Qpmae=5zA`RB7a&!V0te@GM-|Hk9YN<O34{*SBn<ZzF_s^i?6<-3?>YN0! zIP3iTuQlI=>K_b*-D^!HstfL6!Gqgr0sM@ntE`~X7ntLa9ZG&7>0uZHNP{<evJL<g zhQHvHS|ryny7ApdQW!nZhdaQ#!n6qTVL6a<Dp%@!mKx{>w1p<Z+p7#ipS&y50RzQ$ z1ER6e7m&WFVzi;xEIbECc`rDG5u3nRRbmv_yBgV53qLb~O^XMW*|EbP4E17Ab|6(w zLZua6(A_r;O9J)DM`)~=`^e!}tv(;BrMATGJ|-2b*&k2Ri}tgNV&s*Y$ZF%)aw8R) zT5s)KT-*cFyUrNJGC$QR+{x9_{$4TRJ*4M#!Ckhb+ja2DT5X9v{y=;eqwa@Wu|hxG zv%-F#hucSOzXbzP*(0JPEy`FW&k{O6nGOn&U;H*D-b@YX2?6onHT8~A>fR=w)9iqS zC$CMcQGffS#B9(v*N48IK)f>gHNv&Vb!xybqiDkmMjL(qQf-BzQ3o&=C)$-6l;om= z<MYroNGUtXO6(voG7wA#txaI)D`8AoupSG2DzFmdd2S7rZF;ZJtUh@jUe8_*%Fy0p zBT{|aCp9gj_&$0Fcwg^#^vrsrUJ`1bhkoUF@mBn!BZg&##8R)`S~2CMTQ>`1?mzP0 zNP2hEYw~Gp#A1@tMHf5A<Bt$Jp70{#bPm(11;7>ag?SV)9)N~86)YkZMl%tJ!GJyd zNq#c#53X#er8p>f@LBLbY`TY4AgVnS#9Cc|XP94k?u8MDOd{@r@=+z^1XSWMWmvTu zcH;bI(`uQ?t<w=d0kfFy3w)BLIpEuiZ^d=O$;Y7<&MRQb08cQ$4w%rzcosk^TQXs( zCK#vhTE_)vC0ZVxYL6&cqLcEe_!L@M8^7Quc|X`;!pUGZ`h!G?UrDF_Q9s^`uSY0R zF5{*pIe$U=ApUYNP!`LcTE)UM<Z2ilIEHXC7Pexk3c>HPlKohT)f^SYN#Jo$V5pFa zSLl19l}{;GJwBXcLvB<TdvY)bbKJ}bIJpZ%et}7ey;yB1e*r&DVVt8&wqIfA+p#Qj z*@*6Z#9o0Gdesw3qU>=+WOd!?OGjBAhGsm&LVM~ZWkP#5e4gH}uQYy+a3gnIOCj7= zLf>)wcER{?{$%jgV~^_GN>yJIoFo(9vc6YMWqv1~41_8??4psx+J3iK4RiZig4SF2 zDuvCfp%X=V#yR$qsog;e)p6*tTzpf>eibc-7Cgwh`q`0nAA;zyQiuqA4?vCf)^gWK z6M~k$u#?z>qZM)ep(|jsp5T+)|0qMB_2O3p0<LZ(!?J@AOy6tdBsvoc-zV4`g4<il z+UUmZA0(_e#KzRF7DRK1cPnjBRlc(WI3W`emJ?4#7%*2E417EgWcTr8bbtkAJ-D<U z`vHM_3OljTH7CJ;9!;Ui11lIHWk{x)`0s=kfpdz#0VG`_=mv1a4lvWmAuuAQBG`4c zfO2ofd-8{EzH{6euQ1gP_*g!$rFsIZM}SPg2ZR>NUzW0=g5A&*b3-YFB8BmcW$0(n z_qU&9Eq2rQ8O>^cvzp{ZGhz3D3*%#qWDQjNwM($EF5lvRAP*U?5jq)gr*|IO=*B~b z;XG++=9hZ5k2`daYiLYDvWeZ^luYbl$arkQqz`b>F~=s%al|<v#41k8FzdcR0jdaO z5QTa6t?f-owiv4tq&K#)<4a(NM{C=O#}_UvFQ9$oQ?p1(Ht}y5E2Ce&xYXRw^H_Oh zXB+9-(<k=89j2?D$_!96YpT48bOWmmdB1|((|Z{PuS+q$ud@j!=j)BWX^&D3OHI3? zJoF_UbHYhl+AbAq%M*zrXMY%X#kAHpy#8A=n>k90*}ePE&93ZY7i|y481&jMbof+q z^6Ea&d@lK<?(8URH?9*2D*ECq(H<&u#{TIO75h}<!(U?$6b@aQew=;7<3vQ{)8scZ zqzP~qOw2Yd$E>`&`MyfrSVi_rrRV*F1l1Xrf;1Oo(^H4p*_XV3K^pb{u3Yz*<(~O> z-p|jql6h+=pKA@qk9&6M@9OyOZCzM_rg%OW{nlN;?>Eo$muH3F3vURqG+{^g_h?K< zWvYyFUE4H0Xp<g7o+4bNU}8>%|G;v`Do&0BlK`1ldtgPVKMaasc{27S#O%yZ#TxQ% zvH2Su4Jtgbb?0oii+Z7YNv&!2?`;>+rODPKO|wout(ohFGxrb$Fc2XFDSP4FyMb3| z7Hy-)4AqC9(ebG#rwgR&CB9*F_;|wu*8?Iny7Z?5I`0g$Vk}t}h4o28tKPwVsqL<6 zO|5wp%zD((usHJ;9pV$Why+el#Y{n%pU!vt>h{|F+f9_h$6wrTtwconk5+FC>}+fC z*wQ}*SN+jaJ%{Ub#9CmiPRDZfah;}?r_fYtqy#M{ixPrM@!!sWV6IbZrS8!2<q1RL zs-zsS2&=yq{H!qOjoCSdVw%rCEBgd+ej`~O*Xa!2ac`J)8ysA+z-8BhF&n@fUgNT1 z-Bk>ynbmHA7TaheULeYgfg;$EAUQq?tbdgtlvBP3TMA?Px<1UF>vHPU0h69|^5o_U zEm)mz`GVY#FL7CGv&Agu*^P}s*~+PKAD_4Mp~V3==Kg5MYbB1yawlH<4-W%1uTj9a zhGBuI$vk{VsFG(AC6t~(QKVdz@VwOCG8%=K!^Dh%X`2|~DWkTE(cOj}`kZvg9u&;` z(DpX3AK5E+N~Rh~?wRw|fAZP;#M$G7r6}pnXdx4iMytrMNSVl)kEahi2(d22pD#=D z`cPeOc%!=dWs>-ARi)vnC|&6uCA}I3MQVxWnbLlWxIEQr+4elNHLi^KV%6Ka6s1;H zQo2I(FBbnh#4L)FJs|VmLAmGXtp>A(dg0Av&ygRJkXmxecL256yOPJ@^G}Nm*q|iP zSfC`g3W${<IoK|N8ry@gCy_Cn0tg&NevY=ETKIYTnA5mYFLXNf#JlD>5xh$>;{D5( z_c&71D&opejYP-*ZDdtPP4m}V-Tim_wawdJkz)1LtP5iX>@0_f=VxCH^qWlBH-!7n zs2Aq&BSEM9<>G%v(*BSBZVsx-X)H#Idj+_+V<k1Y7DW|oNzPS%WmDDBo#zr3P;Vr_ z%&hvDup)0oo-0>%wefAdoA1YWA9fDmF6*$J=2aK%L4fT<k4_pJ=s&>CKUI-^)o%66 z>KjIzPvAMrJ^Lp4D|oVso#nc3@vDUQg}5z#HSyhvkdUIl%(Rin7QphmORw0nc84SL zuc-dK2(I?WPu#yw&_fpgXOh!)u<8HXh$284|7#qQJ175p+rv1-7GIx#SeVMz;PT;H zKVW?AD~J@j$|tCi8w%1W8%!8?FA@yd>qAXHorkThat$Hn1^ovb@argLV%KmwaC+6i zc(Jcn5YQg|i_Ea*htYAxXlZNKMsn?^?EcPmf5A`po*5^Uk}9sn2>QjIVZhd{K%Ap{ zPpuDt`42llsZqsB?t}ehDGw>K+Zb}GY@Ig<ajYujgn)nu!pEN_+>H_r!OM^e;Z{>k zVwrHCwy|i3lf?Df;@~UfmP^04-N~hAWz3lz&OP5Bd&58>KixU=Qj2izb%QRlowjSf zUf@yj9AxjaM<H)}&K6i+SBgHA^iEJ?I@V{GyVL6JSh`2w{G3L%yRO3EyoXls_@`&Z zjz82CD8--xahmSq+^dJ=**H!F|9@%kJ%E~8`+iaOwjd%R(xgU3KtPNNh!n}TAZ<fb znt%{H1jML-w2&wWNZ(SoAVg`QN4h{rq;ElblOTiyl^RJz!h#gfv%h=K%sJn8zdQFm z@0~f{nR%VT31zLUB+q*O?N_RB$5H%Xc6nDK8TlG}5UPF!O?H63^8!iOzh1eJkt0<N zRn8)#L-0}eY6x+SVT+4S^@@}y{z+lJ7~cT(fyQ)(&sSi)Gxh>Vp*^Fr?NLv;)4EU+ z6Tc06K=Ae+UyA*8n2Bm%d4`ZPND_za5aHCFYtMctA!M<?b2Q3JY>5@bl11rlncvTS zb#K~hBkch?%rNj@_)`eU4TZjNlDOeeSU%Ps+F(+ZpscRH%b%O1?a(m8IqEg!!sg5l zm6@urkIu~!xTh1JGYN9JZi|^wd89@ADc7t7i{-j;UtgI!lzrfwkEnbZPeumIgl7xP zQ^8Wo3IQ0Gl_(TYk})dAV#X-yNS?E>*X%?+q??a#EoOH;W%-xew1Sza?bqVjrkDWC zws+xfJ9NA8_wq~+>IC01tr1W+`_sA|N=QD*#s-Ua@<=Ir>C_%Er0lc89K}NIRINL3 zsmlhQI}Ox5$`%jbSQsG{+?G$jdDYuK&DbS%vGrJTEcXF1!LcCf^-<Tsw%-0-MfMK4 zbGDa>I|FkW$Z4F3aM(k?Qa}a9PZ5N@l<l}tlo(dMoPQTeU|=E$vZjEpkJTQgY{iC+ z>QbbFqCJ}9m~*WwyL{Gj)W-S8Z|9-c>y3S*NS=j<6l*Shk~YqbSgQwaPo5Bl>qdbr zW^CaMQoCdlLKO^3_WGFZgEZMN2AbVLjRUO5DE;RZ5Vc(~v0>I7|ND?BY$jXDDij@X z+U-(vtT?9rw_hn-t&Id;&{cekC1t<xBcW<MK?r&$EqZ)FFlP{!Q9$eowWkK5+yYqL zA9AGQW^sU5_B7`J6nc;584W~NvTSYM!DZ)S6d$>tr7<rk`dv`_^?a&7fqfDGadj3z zQ!0e5_&U36-YM{Tp*f;67SX1GMC?65%u%XWCA+`Fa8tCCusi6!&!*vH90F284gZMd ztpBYw>h{M<v5QyZvebKNk0WRIKwqHv*1|2)eI>(r%0n-2dK4&JZXbC!+VgOuOX`Tp zYmE}5$}W$+rHQ){j>LVV<rkzbX4_@{xc^hcJuko-$uqmBKkIZNPubDb$%qH{E`<kN z%HGM2=SXo^Qwnwtd`&G5LotA1QDBq-@+nwXaR6#B0p+p%Kx0jBrWx6VP{Dh`RlLo% zQ#g^7C{xK&{uy?9B?G<>(rE|99g`~I(<m2LR1lTt6LiC^+3c&>^Q|_ADDDhNNGC0- zv{d*$b36%u3l8#Wi+Auzi@!lLsBl<~b8Id0X~y)PF*7ynMfQ7+WQ4wibBn8F(l;PE zmo09eu2br+=P;@t3ahU&_mj<>=$Ru%Lwa1`>{QNE2O=iYE%-M(1gVr`xIr*u>59im zL85J{hXv(#AOQ!@{J_DQa?(>6US<b<LdN6P5aL)w4d@csY)?MGapZo4-wC^HYovDG z1SUhhiKHRh@s)s?lfu}Zz^1bZQixli|EDPwqXVTcp(^mbXnO~cQS8T%^9<_A*FmAS zH<dPy(@F$=nTe3NGTenYzv8QOPh7UdXS_iuv@>9F_E?gV%VpwPj^b-4sY?#GEsk~0 z7JdB~X@uh<99!*MncAz?$KUl8R7!C%^t7m+#p~Wh&JK&+Il-s8{o;xh7$=rB)ZoPK z%)(LZ-O5wmC+&kWAH{2G4?c_oTAhT+RW1WM01B5Kf<%f4Rzr|t>dRb-q)QuMlIoJt zj}TJ$JE1yBw2H|Vs;E9no<@BVfKd#hiT6LXXbtIQed&8;u%WM3re5my5$~mQzMOZQ zP3ln9<B;0u;t-K3-=>S28INDf5^Wq8?(-1>sz}@#L3Apv=}-ILM@(&IbiuA?F<K!} zbMg0EZ^&;Si@%2b>Ap|0R}wnh--4oD=5axq5{?7+S2d<(9`BXl%st1M>zpV1vF(Pt zrguX=0M7>L&GxsM;G@Td2iU|I%KpWZAtsB8$E$fZE>rIB*6%P+G4G10AS@{oP+OPq zG(bZGqWVS1rW7mA>8w<JeYBQMa_3;4n);OmT|84lcU^enZKlbVD+tc_HVjmBkTbkj z=j<%ymbH5K&U412<4^F*DFw!9IydgudY{cNJ#%KsXrw^<a=cb&t<hjHGV=lVIq~w| z-0Y)P_Q!j1<(kP3twr|n?33k+kI<lTnMn55>Gr<Z)+KqkJMU{MFpGZ4P~20nFuDpX zW<YyUqmpL>Y{{s6@OU(UX}tL>1urP0ejY=h@jOF-ouF=3kS$zCgepij^{*Ks-x|+E zz2MT9!Q4*6gr@+GEynfz6xl4o!aQKS`}h`Du<H|GY`dD1s9H~!+6;XQMDx|AyA0!` z=kqRdx;m!Zkj=@K{*!;1sX%9kRxb_LG!-+l_m3FtD61Je`8}QN5bK1zFNgprtS<ab z1kV6I0lHO=k`YUlg4c-K_=#$6B|8`xMZwuR3{)3#$Lkz7T{UK4xIav;A|ddB_#}nM z=;?Lswn^HxbWroU3YTC|`^n{lecrY4&lZcs??iT}a5o?Ng=Y?!YQKVzP1wReIApd} z_(9cxgLqa028{SxN!E3sIz!l<&*sJyLVE4AzKn-Csx>27=h&A&1w~s9`6jQ}bIcb% z7^K$AU35R;xd|qu(Utg?Bh&?T!W&VchQ;#IPmz8_D%=btPAzXhn;TfGac7_dpc?xy zPL(o~#kUr4%<3x^@gk;Yin};q<R4SEmb4v;YxP79edP@IFk!ntmE|1FWEGOfYMUN% zFLrE8tJ1nD_yylu2s!^0IX?`#K0+yq@=uX3z9QHsKSd;OfF<mLAF@GQljQ%NG%@~D zB*y^8=E5QM48fslt_aXE{Y#LI-<Wj<zhW0tWC5;D|8Kj|r<iEOhMNUn8xx8DVQU6H zUxpH+U+6R-`q%&C@t6LSCkq4T+L^^b38aw#`vF~m{C<k0)%@#%|Md?4Yf1d;ru*+( z(tp`q!g&I<o1%_2hFWGwlE7aOC}nNKUV;LcJyvkO?(9mmhxP#3cLva(+X5ckG*|&M zP(~@QROQYf9B7H5b&1dA+|X)ARQBy8{KwtZJtw<xfp$8Ll9i9<x_V@LMVYuG>}tb1 zyJ;s_2W&#!yv9ylY--w5vi(X<{(vZOCviCd6@!$?O^{>yOWBEDM$!b5dj`PnFDw{e z?vknO<)FAx5J<0f!dr0~BjPcpFZ@Z-cnP!P6Cbf}OzIO?Jd8Btz8^w4jQ1dFZ&&km zKzJy{^&_O#(!(cP@wLv&NdS^LL9`nA{duc1bN!v<GeejvUx|^`ys|IH7Rewb2~SSD zNdAbCB8vxATKw9p*Sltbi)_OZCZ2zR8s>V1I^`ghm_|KkUp><=$TJK5{@o&Dh&Cyl z{t8+#&9m)0XDJ6D@m8v==@!7QGBX}}HI9|#NVj?%WFz8g+|r!tN67}v=7Xze$MqK? zw3IZ4F%>s8MOE*F#4h{9LC*mmfGS2&4O(GlBnBKBB+IhtEz{~8dR#49M5_?;X>~RD z)?9&-y>n6Mly&_f<DsF9h;ruTFJCeY8t$%ym5&L}B^l}m<8(^hLVr|#2Q6un&KwO6 zb=io$8>Q&I3Pp4Xw+-^$*oY2e0d2>cL})vrn$(Vz=WiJ^Xm&cT^K|U!nq<_ty!>N5 z$=a*E5%JDbY3zEwv`tHnIvcrc!rlp2Sh(zE2Iq`LGm{ckwFd4PQ3g!U@y_#q<6yd) zZ=N0f)2$tLwQ9ud&?i^=vELNGk3HJ<WH;S^3<yQLN%nFco)wZy8!8S?PgCJ-{W>9j zCcdvgH@M;1|AZ+1AJN7C-yZ++-TCoVG(g=;KB@Zak@+kP(wJif=C|YLYS$wWXw=72 zrzk{Norl&))?{Eq7mEcN^S>Z!sWX+-tp`))qFaC_#R`q?7=xb!6RJc!IHKM<{4SS% zbB1Mmn!C1PiKutns0hWduDUmYZoX~1K=Kgi65WyG(NLRL%b_bkQ~o>B;s0G(K33!( z`8_`nEa7<k`#F$Q9Y!pd;l>b~?>)@^F9;iSxQm|?$lv>z3p6LIc~|1O?*++Z+;5vR zu)oF!WY@_6({gZ=y0(ehJTDo|M6Pe**A@|*4z00=NqJ`2e}5hS2`cW#HY`A|xjf+M z2!}R)isb#{a5-VCNYckpu0F8*TLw6~aLKJw?6{VVt&a@WZvT7G-2Vum`$ryQD|xqs z-MGDiN};GXeFy&>X>ochM<ubDL3&2rAv^+k54EmzBDM+jpe^=E9>Ap7rd6xxb)qfO zcI?1}pLefvNMTyhqVeeuap<B?p5NDJqFD%LxgJvnU**Zlc#Q@wWp{m*eJ;*(rP_)Y zsA5!a#IB%ww$Zbi%T3G#->qiw0g`^(HeoR%ppnon+>^}oIqf~X%syA_F8nbbgT)E5 zC|htN!e26gNR1RQ7)8mGcha(RntWad4PS`3tZ4YGX7q(W&|vetv~iEG#cVQfcq^A# znDqc%<SNKTR)6Yhu3q>8<~ntGsq!@ih**gUlK^CXMkqr6hv{;cZr-2vm$E*Wihu9D zM;fds?fK@lksQ16ZB8hL=EN~3<4_ZR!kwpqYvd_*Y;}hxP^6Dw*SBFNx3sFN4*yYg z1(?%wknJjP8_o|WZe7QLr;EU%%*xWffAvM6^IwQMYHL5F)wN7ut6~yp87#0S5JAMY zM#B!w#Ci)3UdzTWZvljxJ@eC&P#0&8(bYoO8#Ig@<-75&LX^%gKUCmd>x#oJz4;|m z6;E&YOx`+*TGD!O^jD(x{bI51?y<Ph1*!q%URRF;<CDKpX138;<7t)`xdF3&RiMq0 zd`3nvRK%}vEMnKDU~Q@jY;JaNnUC-wXE*V7?!2q&bK<*o!_C6V(nb3YOyR<So5i6N zq<)Nym6AzSkp@+3O~n&r@!<s!p)Eu*sz${bl!D>+A+-)ySUlfnmMSw(qeKSt9zgXi zn<Q@kQ`2AA^xsDF>eKNPH`>~1j}l6KrVlpP`=FF<3SH+zl=bFyOS}VBn~C7oJY}X7 z)a5=zcIUbB9iblHeJJ{TW9`TfcS&yK7*<!Z1tIApH!LU)(3J<OpUw<mgLII)aBuHu z`8R_p`Ku$kuATDrl*HdaZTXsGE+b{J>v-3$W1Y{<Wh^T@WG)q1&L--M9P)lF^7p@e zr0cEaf!>u4MQaHwTESN3GNBztes@2oG5Kl6{sBbI%k@^SgGnz#8Xl!w`Vvx9e}@3P z+pPNjk=yA%`&%rPhxjc31L>@^+Z3gC=YWDT{2t$|mK>mz-ieHZ)y%akEFXLky;<Lo zG&9QEzu5M>-=GgSm-;v2_oWk_oh}@wvmHqBmzbd{2qtNHl&7=cJXKf^$?{CS@bYU} zmWJ3?am+s9XpZKSx3qivU;b*Hf~u6=!1-Qs_)uA8w?s1l&~YC1^`@P(q-VwqbN&8h zKxt}T83Fs*Y9BlI|37Z~mhYv)oeeX1nU7c#NSt>y2-ZfU#??|kI?t9NRLIvEu-#Wa zEebU_7VnTUz#Sz5`1nQCvioB07SvMrpkWde%|$canyPrEtObffLFAvIO>b|e6tIyA zWHpXv3z+^snUUP<HGPb2k)W@u*FG)HagHX<jQl|C@lx!~ccyui7c9zE3{FiT(fW3Y z`Qfma%CUlDq%OoW0^3@}uf^lwIMxkdJiY@hvU5C%E@m(0A_oyi5k<1ATj`)Jc5CWp zdt>?PqHEVPQv<erdq`(OC_B(;1n$|CauKyHDCNvh#kZpptj_1UOUTn*)D%yto-}KN zS953czLI@=6<joyX}0=lYmIdorblMA(oeV6Din<uHu&lmDAhOG^`6|+Um4ik8@rK! zxSa&)lvzFp#%)5ee*c4x@$OfJ?ev;_XL%D5lvz+Y6(gnWXJ&Y!^VjV+_G}Zer!q1S zGZpk7{&cn-jDMCu!+_Y*M@bUg>Oo}UAu#yR14<Ci)vzzo!tK<*HoapvGK30X3j;fk zg)Ml|pc&AvB`5^l7}I$$CAkV#ViSIF7|clVoSk?yRhVRq82i0?1BA}Uh*kN`W`Kh_ zf?(^18JJ^xz=W;=WSU?Wng`Ah4*Ic78R0Nqpz4orJBn{{r_L_`$~6LtMAU$Gj72tV zNyCK?x=8O}8`1y8&xG0r#CA<@mM#K=fX{+toScnZ9?uH*Nsh7~P=j|tA^Un+({a~l zeN}zsnbc0!!)`e`8t5PbJgKj}2j?UyWa+bleyNI<IDK(UZgtbCfTw_Oap9w6LhOjV zU?H%@xvS_k4bZ4N@Ym=ut}F4vzJ0-Cjdv09_`A)X7ph;(eo}i&uq9u4S=wCNsud~h z5+qZvNj?Q=eo8YF5u1^7n6u9Mg=Y)#&esBB43dVQO)tn_qRRjy12^{sN2~-xNmmqj zxFVn8cjk@7xu0PqiK9b2((hQ_!wg@fL}HL~h?ryFUtdn&eIm^@rUI>Vn0{fs>CvPO z%5A}O@(E-4yw7lPZ0b$BEM%HVgF%U#i_S{vqu~#{E$=+U2hEfq`4;?czJc-p<oC`) znmrfJM@Y1XyD$e$b@L|XLmd3Ni*l{fTRTNQ$ey`ox#d(vRjdv(>COM@wDmt62LV+v zB%s;BYEaADc-Q#**#1bvL_UURhW)(`@>90<Kte7wizpJ-oIN=E%f1--u1t&QKV_Y6 zA69U9^DAXCH=#~q9ic(>1WFT)ur4r6MQx`TLAPgpx6DemuKdFPeaPwLuKiUUYLtGz zR2y3JP}hSUBfp=>!*bql$$Px`E`)`Z<%yy3?BZNOG5(g$Q#T92bA%5vj<TH{l>_b+ zHdYF14kQYPQ6lZ$N_R7TGuk8*T#r84`b<hZNHM5Rr*$T52~|3?B=8iECXJOHq?fvq zQW_EPj=><TZ3t|2(H3qqFZBKZw&evQkaE8I^`H65TrMj6*8MEjW$G_S$BQ<4$4@zm z?6sP?k&TkZx0st^m`U4lT_^>>zK$M;c|ye;KRbZXTApPGKmT(oDUMpx73U9sOqKXn zcI_PHE3+T?IQ{71p5OY+zxdWQH48Q4TD1#l*256m{Qai+$)6%>)(&5s%8f5UC06Dk z^*84+kebsAjbfhiB0|+OCKp2kD2jMN0p7b;-5vQtvyC1L@;NKf7M+aLS(XOoiK+^8 z^B7i}mF@V8xf7x^V2&zgrU1-!Nv(t^)ONlEnk%^ts}6<E0EHvT%GNEn8JJ%nKU;Qv zG;2qK<C!F@p>pXj7pa;eyLARizq|A~Y8od)g_F4QDMHb3IB|!4Qrp(#AJ#4AG7jng zS+M@E{&Sp$aCp;N*awi}q)3$9{V{?B$P!+`#D}?$IFawoZB2gvdw;*fgYV-bA+<oi z03F}(^$x_=T6#i=JtLzpYepF99doE!!>ptn0ck}qFEFXJo7_{txdag@bf<QaqFIcU zBz(0;(O^`HEzoM(x>Sdf#>q^^h;P|WD1oBeD!mk+E3E1#$t~_l3rKrT0cpCKF=Wrp zf<)PI+{bLoWY})3D;pL~_VgiMwX8zT%V!qEW6Fb^op<JSkNYP)Fwi|AJXkU?y`b0L zt*`_3plu%&9bFuevi@cz6ETxA{w-|<08dd;jIf-8Iodi?73FY_xf>^;TqqYB0GN3f zvHvuM-??LREEx~fnGREzsqF1Ewcr`#gq}r{ph<Y@LrJd}<@j^cOKd#Xf^G1bo$_UB zI;iI&n8Y87;a8E^a@}WLj>@8zB~VU7KO}VQ#{-(r_FE46yPK^$jjp+0VCtMk{66$! z35_w3IJ^1nkwa~D)R01cy2Z6;&-IRrNPC<E!8nMWPGnNU5qkmh9=>m3+{w%kgjGP- zZH87~yLmqva`Vnn5oG$GHl~9>v-}+Bgtu-u5oanQ@`G|6cKkUiJmUTRn68$=D&HcG zyAOU{-uq7?h8N)Sp8q2$&$GD=LSf<qK9hExpbE%8VmG`6ml5toz@(nf1MIZ_y=q4P z%=44xR$3_vC}61019F`@))Bu$lDwPLgr6dJ5fkvxU}ID??r>e6CxVs4mcHP0UF9uO z*2mSbJ?voXjzPJ8ChA%8wqpx>gYg6ESaFchC<~qkkZ5!+P7by!APmt$uNiNF$$|Xn z?__Z0S>xk?8g^0_Mx-rNBcgC=MQS@fTBtrq3h3Vn<xHesAr6o|vD4=>_DrUJ>#0nB z$6UCfR%Tz55p;Tdkvw#Y16j`Auf4xwWpjlx<VxH`Xmsm4_o1pXSn{uNBZRGd1CG7z zU@gF9C$U_q5jbt^iE(NmlM1F=3Z_y;y$otBLX3~{nkS)|Dv}*z#tC^D=iQ3b>Z$b- z=s=8`KnX=0eyQCmtJRY!yBA%a>AgV6?2{%M*}uH%xqTSvKbEjf=91I@iWJiSEqf8% z-vJ_~*)Nb3yB<VM1)!4?KHf^-4RUd$D~A|wutcs2!9~BCun7wqc=sZ7X+!RSY<by% zxyFVZ9UuQ>^vw&lMMN!=3)-yaWVg!v&m%V~regv<I}vq>gc9KhF!9;}>Op-|*|0EQ zIP2We;#oW`#bvOfux`Y38*dl(_=tZ7^#1Gggthu$EnIbpMu;*~UbtL!q7hoZ()c?- z8h)_f6d%%0&}rMW%Xl^K>YJF5>y;Uh{yYz@=w2pgd?&2{T{q~0rrpcFc{ffaJ6`UO zHksoWN}hM^%>|yzI0rD4!+Wkg3ki&-91?y4R;<w^vLVk_Nb6@L?d_k@_0sc(Ra2KI z8D48H2v5Sqd$kndnayw6uY#iO$N3&=iJv|Nb#;Z1{W_Pfd-ZG%qF&tjIG5DvP%6(W z3Vnl`a>ujgf&<5<BnPX=@+0^E$RA(vTmcDr@Q)v{RU3x?R)!(Z|NG|B|7qXz4+!c_ z_v5c2X(xDGh}{wS0aZm}iH9-ZTZME*xikfzzZ2R$tn{{4lo-`FZTPVchQT39=(QsD z7Sw<Lh`tD6+K0PNcqx`uh2`xU_|zubK_0y7_Q_m%0J$sXh~7DEv4fdfH`Q;&JWoqY z{o~xPV%sUJ9C#G}^elY$l5h{_UUWW`)taL{7=V<3vf4tp1fNnb+<vyO;ilP(viZ!u zj4<i8VzhHez~~2gxmC3eeLeC+9ko4p0zHcM2vj#)=b*_~!d9xxn(fVtz(z)GQ66;H zw=M(c7?Yk}J8@r)>7hBt5yTC|vuVwdL&5t2Kg&eit-(~(FHEmKmaOC>`&6%|9C{>F z5LQQ5qwb8Mo@F#>@(MQ%xu=!J%t8X3HtvE<(GRI|)-ct1Wv^R7&3GnCKD`+f`{v*S z=biifkLP@BJWY?A2zm10*i!_iMSD-PN6@|6Fum3saklf*ntG0AJ3f}Mhks7T6|l%N zT319TyHg-)S7Yt>Kg*Vxx6WVrN^Z^>M%FfwBW8QyGm<Pc&+PyQ$pRg}PDC}Lwf8a- zaZZp(y(0<x6y)4-X+F6%Mx~x5#d8QU4hmS=3nd15lHGt6hGCA<z_N{cJ-0u`2~qbx ze(7P)MC<Ny_vIR^(=W3IHHU*w`^tOzzP8CU8}f4~W(Ah|-pfrduV^w4@CL=#tDGZH z7MuRg4bL!wB4(1d-}?^A%~ls?vuIlrW$^2U621=303Dx$VT9@LsC$oIB3e;<0Zc%k zk?}6)G#iz0vz|Wpo&lGc#~rf7*2UafC0wBVX;DjjdUE;NhgSjyA(gKVK2U(K2@eZ8 z^8P`A5SZemmzI6fFArm`DD$=1gmx-yvE2B#l#{hUn(ky)$x`Fw30lkBsYSiLHzh`r zZ*YKOw4P9GZ8oruJVL!mod80Q<7KyE-QfWLgj1e}iJ7T+RsU8k9{d9qgWt*W$L#Ua z>-0>^3q5Tunm`Yc#TwXh+<3;gm*4pt2=3sdMEz-`houly<Gcs!y0`!+wDmi%A^{wu zw`M9X?=q8?5AejX_Cwe+oH;j7^bZ&f#cgNn*X8VIYc6jPE#$u$(4z>u4Px{$d!dgX zR^2>;O2*Hym(#0`W|yY1Q8Y>JH30=ETpi+!rSSHEsI!-%739<2xPQO^3Fyum#6kc- zSz`D9MsBom$9GVx2wjN1-o~e=NgU(x*6$uiQ}XyWqiYFfvzHTTtHQmy2CtK8JZ`B_ zj8q1|P|<*n=66mIJCkJz!DducXW#7!I(M~-U<swnz*T}Cegp0@49NuA?%Yy*F5fSr zW8FAksAFq|n+hylbXH7F{?rvFJJr?qVE1*8>>bx{UT_F&-uTSAlQ9vWQca~z(B6}R z31|M8IqXn=#G#~<R0*k#BEF`2j4d=$iszG>z-<HNEuZ~oN~QlQ$C7opDT0?~(m)|% zJN719bJp(3(dHS9g6pJw2C_NJf&-7_pUIy-QyS#f<sr&Z&pC?KoSl+xmkYrW^StZg zGfv8H`s;Pt4V8bZxm5QKx0Npo6>v~9$c{e|*YT4k&cXp`(37~>7`;5lJKIY<mm7rO zPC&MdzlN+#tqOKXPWVgGbj}Rr*CJJ+Gs@P<n*2kX%7LmX)a`BG7S~s3NnBURoQL9T zj8OLQZwv*9)pU;3tjGXHyl~&h7r_6(M<S%Kr`0||=a|G+f}+doMz(XiyV6C!(^0Q6 zf1K;}43?PA4h;5uu;DfX6s_=bh&%LWs`~^Ph<oBaa-5`RYliIc<#nDDdefa6E!-;{ zyWgTbj1D=Px2;JD^^{OEKXi>=8g%l@Y|Yr$y2epcDYF7%`8PstSf@>PXJeGKf|=`* zr7g?sJ#}uChhoy1tMM-eA9h?n5CV5;2}{y%?CMzt6)&b1ESt@UudV;UM=$vSs2^Fz zaf<~NW+!puxT8`_W?KX=u@7dc6NX%~p#t-ULH$y~js(4PK!N8Jx<{Mr7u4ngmEW#_ z4o(ZVJJk*dWv))PKJt+-L=AWIL7Oey{zTHp>Q-*miENY|G|RiSK+(P$p&c~T&+Udb z<l`yBiS=;irQ+@n$CATd*{I%lE+*Jlfz@u_835~Y*XkLwLbSN*R9X2OwlBEer|F{( zdR!Xj1g3nO3<EbrMT7_`vR3IYfY(J$4$ma9wlL0STC@@-<GMVg|2B;y?3#2th?WP? z;4n25qAIEDBQtSv2Mdb^HY3@ou-yT`NFpY{-H_?@M7ap{Tb0_kOh{6=^Q<f82|~Ok z`*?WJxQCztwEbL>ZJr5F<p+aUKSeO8iLrH?hG9&cCO6>2Njc-PA#2SV4cavGK;XSw ztD;#iFA-5kJBmq+R(oX4tA?~@U(AKU`7Z`Ui({>CO+u|YB?9AVcEYU9{7u+^I;i*l zB;@_m+9be|8zjR2`@e#tJ5k|a9#8ec0Gc#p#HoIl1B0Hk`#U!Sxt=j(HKyCS03!XI z@}T>~)MSrz{-#qky2y<-wXhlavTl7dU@fhAEwZ<Himaze9Q+OTR8^xAnglWJ+X$i( zXW(OY9NOURDY2yN5tOX$_=dINV7L#eW2$`_Fxoe|ZJkfjKCEGV?FR6(c75mCE$&W` z(q*}7VXqep;_`fq5zl@k=(Ve=j@C>Co*7%#HE>sQ8zhwn*s3`Xt<2bzO4$59Qg%BC zv#eQ{u5}<&Gw5@ox&NKZVgiNR&Q2DN{<ncA>xrs2Z8};r#aqKA%W*b9G4u&QNZL4! zQM5sxZ;IjT&d^y(Jo+Ap$Axp6Sz~bkwT~P*I+x75GHNEX=;|9eLa*xY(3$x1HtpLF zADw>D#r0}3Z9OtEFVOHAi$c_4DqOtSHcPskJm@fe$vvMk<StYf%zpJd5^nIabf(-c zZ?w&;rw?dTz#}&TxQgHL2|#d03fG5{tVPA*mB^v5fMnOv=eo7|w6O+#mb$PM$C689 z(09*s_t=Ipsj~E|TG7ZM{T`oQ<i)qQ>#J(szDU<wy+vnVT$$jtf^l7OG2b(z0|qF? z9k3@k;v;-tjvv<_YR*5$=uZUGLH(*R=4@+@V&bL`Dvq}ObxD?+t~4MxavNUx7={#N z*CrCxd*4L{Atpk(zMPD9l02GP*TpJ$)lw`RG}~`wfC;CH2E-^`VFxbTuxGD4IW%Ot zWoGK=;&C*h3;kn4mh3xw=5A)%)LJ`DHN$`uy__4hwB)2=kZM+ZYAir@gA~+h;%7~p z-aI@%Y3r`8L2+SxvFcyCl%_6v&&~C6SQ@|f;pm~^>PFiG0wRFlk`fm0fl><a11W9q zrhld06hw1E0U$TJ3@g<zdUx=Q@pKB$7}9JpJHYv4_{$+dI=#JrW^&^!CX+_fwAxMJ z!uhgAKSc`ht?$9U7H||10OZWp`C2fV3%`F82jFd8Ld9*^oVPqUS9<Oa?xW##gt=D< zS7OO{HRZdvR><K3Vrj~^2{W3G<;ADYnGZ3e@0-J9oS$!0{S>K5VokE0Z+Ed^Jo$ON z3}4FgV1g$zY1x3I<^+|JuDJB@YDU5J-=W5+?g3rQm~XFQ-4p!KjBI4b<{ycd$LazT z6r6d|GoGwp*rD81wn}|FAslyV*wl|<6(G)%<N92|{OI*;%Q-ldzh_BRhLaH)&|~6m z-{JwUdXWa#&vxzWzgT;e5>_R8cDe(VO|4m9CNUF~I$VjY7@ndl+GCfX0&4^;ePa1= z@uNxL8bFeUi`cbFZJ{U70?2PBJ)j&nJ$-QM)KUS7k!{XOtC$-=T^v3t?O4PcOlh}w z>nP~U2&&02$e2iem|=ywFkIkL^tMDgRG+ScpSEp6ZKPBQ-Mp#ii%3d>9H9{e<EBHp z?3GzUTQLSuBM;6?EpqD=n~UFYM57Jd+)A_6I#hRY?Ed`VX32?)W@ekX4%WtLs4v&r z?*2B~2e~#UH>w@(JD!Ttzm<Y)3@N;R`0e{n>!G-6_sl7<S^j?J-)jT?+xN0F1Z65Z zDS}?zQ5Z#)!dl%I>T=RPY+~B*ksiATYcVSPTM((DO_jp{=<S_kO>|&6H0yMz^N^g= z!!vi1!*Ye8Hjc^uFZyI=wMH_Zs+IcYXOxD>?y$>G)oST`ICv-vGpIHT-yRf2=c)^; z%nkripOHZ?d^L$R&QYCXuDnAiySM2QqsR(8Cm)(qLIMA}fPg(yTHQ=ZD~4$67IB@z zy*`E2wkVfTvU`DBSgzp6>$d_+QnD`M8a0+)FO1LwThVVnLi?ZMt524muE=|ZYs(Si z;4^0X<yB68AGc@s$Kbt53Da9f<!o#{h8=6FNAB|FXHHwI=-D<`)u*^{JYyg8wxz3* z*ht~dZ9Ex0bdLC7Z4#s<zuZf_x_$5#nL~VvRp15l_1K^8UqDnE{K9-&gBQh(j5(pV zhEHB3xUi#R3lne0en&{rbv%%(HGcf9$7PT4S=U#B93DKft^0(}N(WQ7va;+<Yb|O> zg+*eHYBuit5^X5ln{_BT<DL&%J@8RQmTv(!TXut|$(JeR2C*4Ffbt+EC_#8qrQUU5 zO!YWapkzY?;yHIqUgrdv+7kIXR5rKaMy$>0z8B>vw+oN0SC9Oj*jEp3+Z18{w00_* zunoMm_QFAmcG@`_gq*=INAfI#VwqR+U2wetpi<hyKRHA`&d5HAz9A;@t&)TuyR5J^ z?!o7>s;*_c*NjoHu*oJvNU`|dyz-^t3*K|Kp}~ykoW<YEXJK->=7CNuyKewDjjuOs zn@keZUV>E-_|jD7_R856l_0Lv19+6#5g{$F6?d2|A5Pf|SzLO0MD@V1P;s83%MpD> zzT~%c_{?o)4_$oWiqDjxQ`9(d0JLy`E6!hyNp{eCqBntUw=&VppBu;O&fsHmWH@wI zDd$eB^K)JP>0kh96ERo^kfl44k~Us{(IUH|4e)Y7B{y*Wg+4gNG)2x&5f5O7sR*HX zr}=O;ss$e|6d$ChV&PD5p2h2+bFGRpe&tbV*Q)yi@2@-vG_6<6DGn%_@^Gi4+<PB{ z6`@d{P^ha{-9Opt6X4mNsg*JX5iOJ;<ZO1PttRdYv4ZqyXxQBUTi8)#^2lqaQ)JgK zFU@v%unf5czWQ!QH=7J@4j<0V{QUIcsRLIczWs8La>37mVbts(eQ$C?hj6hL9jIcC zSBCAx`vM(Q^>My*(=356o4d;8LR**^5g-(RXUUoZ7nivX#7AC>8EnARodJLHx6I^> zA^S6`2GDd4EGX6{#evv<M$J+z*D$MOQLb)pT#{iwX&%l5?&0Hr|0U1v;ojz8I&}4z zWmIWE=Uv$fn&ZGC{3KQly2RNsOJr?hUVkt%rPi6FQ!|89z@A{Q2@Cvgze3Z<Iwka8 zz=LjzOem1Hk<mTnJyLl^dmqj3i(~POcjH4_$ESYUvd2&9<iFI^{^2#2p{q6GFGwA| zZTr^GNn>x@&82G^4Mgjp`B2LxyURU3!)>#7rwU{*6nJZO=W7+0RHhl~6MHtb%yAFg z@GXuY&5fgfqytoBFPceLfJMMW)_k4Y5K8{ONJdv2S&tLQKqi<X%I7ylpYg?KsZW9A zNSOg(Em2x|9z(QHlQ@6R`bL}5J8#1+^RY=!U~DH44_5*IjWt7`m~3-KDUroMSD-%9 zfTYadg---Hz|N#6LOH=Jz8a%x-<mrf1;Lnvm}!Y&U$X<~G1~d!Adk^5+L3j4m$)qd zVELfoV$<_){)ouY*ppzfl?3-Rwr@X+)@2yWEIK_+8<`)@wyfV_ZDZ7TEVoiRLtWH3 zbt5OnA@6dw_M1SOf5|A(uFO5#I3->qPviL}+8nnVY-48)@U(MWA3+U%2B_rj!Knx* zU^U`E7@~{{CPTY$zwplsG6Q<xPwEL<Ym%ZHqg#`deHv>so~qYCdKl)CQd6_%yGMc2 zHo?r&WwXkVKguo{kU`(25ML+BvUr8-M(jGssefiF#yba1!}ly2Qe=m}3SLq@`&EU5 z(;Do=0>kLqY*%_m&LOtvWBt+wK#&l3tw=kN{GFp#X()m5Ia0EIJgT+8+(st{t^RG8 z<Sd~b7@JvcBN@?tYSsEu0op?8!ts=%*L?-OhdcZXti3CIkK`J1E87M-LTwCf2iu)H z8O)*BG`G$xJBMC|Wp5}OIplwG9{u6m27J8e33UWz#C6of>TJS#*OU)8wwi-)>q*+P z2&>UdpzI5}Idig<v(rT4rfW5obEgvMo$k`_kFHzvUl{Z~@W`@!IoYKk*uzR`u0M*( zcBOwbVUpVM(K*}Lb1}H%3)6B?4`=r(L-XAVO}Z0SUEfLCj_W}w2{k9Cyq&aW@R9f$ zc#Nq$hC!0`33QE$D|cJ?0>4bZ{B~GT#VquNMy*>S?X7Fgs^Y{|=eo|pqJ{4l^u9iH zQ~|P4od(h`$JI|+OBEZ1#wxGJ_DIGS#PrPHN&TEIE@kT&zw!Do&k#9j<kxCnIJ?ui z*r@d(Q4u}j_xWnq5^xD-@IaTS#SUy^{S%?gwEh;|&~t)U!ee~bL9E1Wj!Bo<uYy0R zp71!3KzTH<n;3n?0eH_m(9O)?Yp;l-CLzi5&A{dZQKfP=h?r?2nth#Tl<2(KG$YzY ze~PMwwe*Vs890eqcOVzmij3J*#vNdz;%a)r&UQ=K+xf&S6Rm!*Jo=_rtwze%#BF+f znoN3vj+P%S`&$1t$|%oYxyH5qVsXqXL7i%<;3W|ja-4A58=Fur(_9FXvg>CJluO2@ zj)Z6%Tqry2i0iAhT)nV&$j-E8E<nEYYFaUpFN;Ppinj}*VsDi#$RsJ>LCYC>F?SW+ z4!G=eY^7E1+IHj?BU9(^qr59XW|-{^?$&CCfWXS)5V>W7mldYB%lKHf{1ZM1_gj<L z7A@3Yg3|jtr?){d99;i94<&YB+e*@Gd%Dn=<Jq}D77Ns99RI_uE<vS_5TGzf>G08U zWH34VmNm8+*j=I39KMiGa~Qp98+3i%e#ywXz&cjgC+)y|xn%s{{=<edWhd44W)enc zwN7cIe$pB)cFkYWRJ<Q)?Em?TaWzrtZKrOx6>OwKR@rXITqa5B{LqEOK{N6853l3z zIm}cgpXpMNRo5t^h2|Z8Z+HA9kQ5yqw(EDdul7jLXaohY<^O4OyDc1hp|~gzF93vY zDu3@B0~HI9<H$Wz5K*{OGB>D7*Qeb>X~@8hC4CkW<rxbH5H$f{tl|@xJ~ZWJXiNZV z5zx-*ZzlfV15Te*nlj!Pn#K}}mO2nKZ@7jYyPz|?Q^K)2kl>(}&3RY(deAIK<11~5 z`IXx>gG@99sRH*F^rb?+ru`2unq*slQYI<RK#!oajM!a>Zi!TFP99sNZPKQ>-AwT& zleY&NYno~^P2zk?eV&b7IjTDs6N}u^KL6LEb=8~OhxYh<JLNLGS*W2UTytnz>1H<I zWz}X+y+AIV3Pvy*AGtTA5)5aUNPrw1f$7U!vVG*>6nOi*`H^p*wM&ZBU;Oz}`?1&| z_VM3fFA0^W$sSt;x%ioU!dCz1pCLpmy;d|5cO0^@Yh?{70X-KL_6l%#vbpl7_6>1T z{0c|Uc}FGd0>^}ut+o*-KhMJ51BRP$d1hTT?*?wX&I11w;SHmCUQS`i4kS9pxj=B< zX-K+-UJa;(H*X$dc!Wu}=NtrZ7rZn&+1DY<hLe!?!7VS_N}yJOg|lyrw`m%}*bVPH zr>~Z~{RL$6?hx8jWvK11Wpy@IHSvEt4!nBzKaHeFMo=0jnxWZ}AVEfQ0J6P}6ltb( z<^X07){0XfMV83@WoJ}|F)?P%<cgW&ZQWJQThAgA(9?kqn`LoI&Jl5K2w=!A%X{G7 z*N;KZQloHUs3=19T|=*l6wpViS?{tT?S||ft1N$e4uY?ldV|ivBTmUp+mzQhL07dY z2g&H*lNAEFmCr8*&xyyhH(h+KO+lN$ovSvge;AWtvv}5Nwm9g14Ba!+<6mTthp+Ih zPF1s`pMFkXx(Y?l_L>uJpkcd(ib39EtL?BowT1W3PRuvnUK{p3Wd7K4l?CIT0ZDPD zaPUY1X9*A~tay@Ged+`C^)}Q;o=qb($~!ZKmcOP@<BQ}HpixE>Tr$YS!P3Fv(57fx zXv1tW+M~zif~6*kg1>|MD9ODnNX7&p_VN!v?%qK6m~3%xKZn#sde+$21c(|>IjhFS z4HyONT_~<-5m*V7A9`t;`wrDnFi9?}dvtQpHy+D{$IPGQv<wA&;SZ5-u49pN-Ju^n zlt?Bv@{X61zIP5~(AlJn<7{Nq59DMXw<(ATp=Zk7xVs^RM|<4)Ucg%Bw5FS&i0^V! z!v#nuERTi;m=V}zC)5u5AodpeT8~-bgW1f*W7UKK#6%WXMUX?iZM8<&i$$=fOPNV= zlw&pF!%R3!f^*ZI3>E-@0iONsy|!Bri5d7TaN3;l`7QFq@NYw%S&Z^ymA0A}-b71D zR?L&qI`8Zw`!`g|Ud^qG9nu-nL1s0rnq`SSZk@@yds<!L^yjsdH4G}bYB?M<F4f-r z_ob$l{Ois8+pR8$?9QLzAair`Wi9O_CTtDXOC?P7d9W6M!9zlaB;)k)kVaA@^=>F; z!ekZ}OZCM}5V&&yAz<NyuOf8t5Q&P@F+5AY0-MNEnROa5QwB}WDnh&X5=rt*xeu^L z*A2i+db=7vogG<o>pj@*r9nxCObLZ__9e_yyrQRo0`LGNNXt3;?`X13EQ0I9fiF83 z3Hyb{!#_oCpnmOuvu^Xuf&i4=s0J#1<vo2IQeYCcEH<vqMXc%Qn)~-lO)P9GEz)#X zA`klf6wyAMSDUpNn_Jy_DP3GF?fPK{ZzGceZA0fIL$2w=t0;X(!cn-`PQM3FUoThg zEV$AT>K*IGJT566*ZUygVBV3h28?G_SBHp4M(;pZIxk<gVjQ=75dZ0F$zZCM12z?X z_@{_6p*91sr6E1xj$m-nDbP40X^W7aBRPUV>m*48^|#aN`7%f0zs|$tC>q{ovSi4U zVS<Tq1(}uj0@s+>Hl@1QJ*I22oO5VczuAe25O;Hr+gJW2o;p&@1h@bM`(o0hj!;Yi zGXwj`<JLkd49rJ+7Y?SAL{poUWGS#i))|Nh{h~G46F<Ju0I<)7bFw?udZRl-r!ejE zgH{h7FydI}tQ37~<O>&>@;5QOC7uKB16>?(7Y}A@OwZ+BPF26lh?y+kZP1sDm^Fb( zE=fc9D~&^}+AtH>D|c(N9MDHpNcv3|`yz0?-33-a*YCH%yJW-HUKEw#^Lcgy`q_r& z!|mULtK)RF`;?R}={H$q!Jlsp&D!$d{kz_MA$y{3#&(Pr=BTFpMT{gWoT_y`6lJ)$ zS~T-2Z>OX8bsug05v|cQL+eXXvUZUkAxpGthp%f<Tx&fZRZ3UaSNECamUfDDU-m2B zdF9W>l+*&-ZUaQ{<*Nh2Tlkqbgj!@GNYU%L3T)J4>NSA6!gn;m0lNbNBil3l%TNM4 znyU-F;K(zO@i^EOtTLw~VU>S?GZ)o9mSZ%^7{TpdkgGmx`|{GKg?0An)*f$A>Nw$y zaP;&XpaO}N<Oty($AvqkW{-3FBv7rHfi052Bsef0>&~;po@yNNN6Cf}9H0)ipDh)D zj3j^PZpw8vWasbP?8}J^oO%T-@V75DcYa@KblSzpLf$oi=;&hA`0?&Zc_~>~q;}1l z)Zjq>{K9cVgXH|?s9+nDyl;K!syQn5A)Ozw7c^y~GES}+*vmhgb90P;MSyEI^V>jP z3M+(Ob25ZGI7RpteJqN10!IV5mrnXGVCK44xMz{R2akqN^mI|T{Ot)qWai_g{8bKO zqJUf3)%!?h)6#a1M8YV|9KCgHuqXxHeQ!_@FEqjaUM<K3dVP>4JPAMZ2M80Q?UD(E z>Zz`G)F{NhdsHuE#G@Q3!E>|S#CNBb^?hx)xois-j7i$%Bb0~YDqQQ5wl^`9P`kqN z+(s}fnI=|0Caxt12w%*TE>MmJ#rMGe91BnBExpD$Rr1K%;P}2>c81bXsp|Jaj!_U& z)-~w&4ynqT7wS}-PdO!NNbh)KbQJZRu~-|`t>yeWGR3MpP1#OfEV7JPRc@#(yYr7! z;6bC%d8ydCT&wRTHIq7t0VtZ<NFan_!X!Qs!v9uVj(NaS>t{zW9H^C3KzdlSm6itV zi{+UKYm{oyjv7H8?pV;#R{BLYB4+>i?V)o0P|O}E?gG1@?)k0Fg6Yn{282Dng-j5q z9RAyQFx*Di3q}k=eXJgk+lyeLBFI*}vot8{9YU6@TLIa0W|}NgYXSkHbHN_9FulJj z+-K6QS@O#8Q5GrJMHlmmY?I|u7-jCqb2q;pUTUg}ihXye#6m&);Y(W0r=rmghXT#g zuY{u~%C(czMqVR#7nd9_I6U&BGOZvk_4Me4m)h4$iVFM-qFaz#M=<P8qF%f8aVxU9 z^=eizYk%>%`<`;ES&lK^o}S3Mm7TpG-D;RzZR+*mxx&dP?fl19A~`zHhM=jI6i=ul zM)|Xp+URVn6`Cj4AH23tK^DyV$7D5j61R%2*?Rg8@+nNLCP@_QSW8J^dJuT{g-0}8 z0+R~%8_lftLEHSr6WuET%%<`q5Y9>=AHRGewlC>h<;Nqpn%BZhFMu<38+bV;T(Opw zZ{F=+aG{pef8k-gyAjRX(L&Mjz>c%_MFsM#{Ut*$)x|D2ln!JsIOk8s3$o8Mw9Z_} z&DL@t8mEf|t0bg`#1CHe7DM077~i9rv2{Ier|!r==WpMc=(ehgQ@ggg<;E7<WL{3O zGCH}OY=x?FRiDt6iae$5&#<-XwYN4*d6-gF`$e-XYpT99<P8vzxA_ks=)<{>|11CO z#9FWP1<uL1vLH;ms}rFD!nZ4Wyw3G$)F?-9e_i0Cc(NRA5JfETek5->=&`Ua=Pcc( ze?rINh~Ek0v>Q4btfpapD5%sjFR_Q2ETDY@qyn~n9^9544QlVi>FK2o<NoC=hGwv? zKRz7LRm<~iin`dP=Ms*9g|Q23l}HOS8@CSC)$6Q6S4~I+UKYlb7`={VYF*0#K5aqM z3a?+tSr*2cYaP1WITNFH>xP2m{_r#VD#&stDZf^gpLGh=UyIMULv_5<sF$J;@j5Yg zemr4pffNgG#25A#Z!|D4vJk3ERdt9AV99&ERFAO>o4xC4uo$v#8G3p@n_GQj`LB#j zr`r5ay}m`X42xeaZ~Lq75Zyl}xKA+#i}xJvuS`lqMG$r7In>6J52CjR_LeLGNsQ&& z9G`Z=R;*FU;`c*3X0_#5$$B2X4YSwBZ{I<mxG7{yM!oZqrV91c)vIv6F@1J_3Qr3$ zQ%0y~vwg5b3o8ux$L&bPnm<=t;+P78m~HJ+Ku@ylo`dTjz7ejTH#1-^HnI0BbSAHt zoDB;OzC~L6gkr1{;Pz1uv{8-iXD<rZ^--TvGngUVX41Te>|iZI3L>};VT~9pxjtWu zE912+>wb&DnryQPn@=e3kiysc_pi8gKF&yYk&@X}{R=8*Y%{4H4r%92@ec{I?qOYK z5FafZvn~z@&4iOD2YML`I77(#*b%H18^t}6zIvM;b7Pd>GX-~p2r6&c+?3lYd=Kfy zv9)TI&KF#21Y#x$U*%X=T@aKvD=g-8aTy%5n6T}|`B(oID3)6T#o_kh0JngY?$H{+ zQ0Cdy3<~#s^nQON{K~yoY==O(sy5~HWVfm<sA9SF^mMtpF*&(0uG2{;HqB*%c21?X zyr7`QN20m5maJX8jeI}%;6ag+BH7<WcCJ?MRx6DmYM*k&NQ%vWp87g|+gA-&!Qy)T zIHrt1N}d|)aB!U4o*C=0=u=9m`cdURjF2tE&%}4ld`%;w5tT`-4IUcT+bAEstgDaN z$%k_Yai$8qE7;xa=_kZ@>5Z6Zf5JB0pqcb&?G)@>U2W73UDl6&Idtpvj;Htk>>G3O z6l76nDyGH4P2A;Y<@wTc+-He{k1WDwk&)CI(dDL!#iEAx%<{RX<WP=rD~<Jb7q{{1 z{lwCH#xPp9PFLQGS@weVm4;58JnZ`rEqD0awbI)k6<LjMuUI1=W$554PGVLNVk9gI zB!$*&pB?1CIZ=IVQv0pQa3r305;sW~Biq$m=ZiK}4I{h=+xSO=M&heFj_}?o1y<kG zv*Fp|23%bwAuIe$2G9w)=)2e9(p;PbjBCRgIXr_E{!ZPE@Nsvq$osT=>`L3QmXBf` zxP?LY0p^jRlg(E~>Vqu6%+ZpWb0+=$lzr-HP59W{OhOl{xd{D2NEbv=X9^ULep=xn zmhCzAomt1nVeGP*anosFD6u}L>>1yM;rRjM$3OHXKV}Jcx|F3e%m#U)KuPTy{`dNJ z((d_iIM2&og@d9vI8m2Q-o!*@2m$eKLk+&>2d8$}15W`IKL&Ed@GkK&Bb`_Tc=;e? zhDR@*|0z-`uPP?UBFo(Bd+<|4U61=}fKT9|s}<&CWe`%}%BHp7Cd0Ra4ClxiRcz8E zt!4!8)jz2QJm15%bUvtJe(RqB30j9v)A_lt$OE61cI?1i#y=`5`_dkqpJJVV9XF69 zobtQE7P`p?+S#vV369_nyYP2`r+)`ETaE-!Ax+J9hbvMaj~&Mi;cLx^g@;PpeBZ_~ zbZfu-IMe9)*9lGMr1C5bbZEVk`s`y0HxC^83O6P+teeVUW0tA;SOY;Uf7?N;pCT~1 zA$ri3^JC-y-O^oKMZ0otXsPwyO+nR+I7{3eK3-97FMpD7e#3t4QkCcJBh~QS6{MJW zMY$h>;_Ug=8B|w*M8KCeoFT^ek^>p^2^ga*0e8sty~GG!#>}LLiK3ljx_Te0d{F8J z&CPyoVxG3J9pR5XZ?DUc`>LmN<e<KZ&X4X>v$6WIhb(hr<W9{dII1P;Uo3hSD{_9P xh{!IHV_H9)#eJLVAN74scCvmC=8l%X{sW+#fBo%0NfY|ld;M=TE6JY|{|!Qi<JtfK literal 57489 zcmeFZ1z1(xzAro%wdn3nLApCc1VltaI;B~_qB|9omJ|?FK#(r!?gka4rBfQ#qNKSC z-@Ui*_kQ>6ea^n$bD#U&<K&r>-x_nwIeN_BKgPJ8yj}qCA1Ej(03Z+uc!Kx?u9vWN zmE`11)HGBTlpe|>ZU6xG{f7_-M<jdzfI7IjXei#L*VWUf2Y&&O01N;P-~#|tb5|#o z#|oMN;!u*8qjyC_y1~DW<8c7-CIA@Wy01o0|111Ig$T@@T-*Qvq=twsU}5QMj=;|m z*xu94=_VY8z{F;DHy8=!2D>0Kh`{7G*!mC5`7_QR_|6TsfI3(p;@srf$pUI|gF6v8 z%){Lhfsr2~aEJ%Q(i4G45t!ND9ST9<69guPTAI2701Ea^xSOT94FdBbFt&@Ph8zM* z0suO;^`EfWpRk*y7b2ekAm`}h?E<m3aieE5zfI34CMHUM-_q0G(#?%a&D7k^)Ww2c z&JpTl>fj9kf0X&=DS+^%Zs`$~EX*e+EX>8vji~;A9RKv@KVtoJaMQMb#5hp@*=8Wp zPrt%`J^L%nF%tkpj}Ube`zy>W1psPZ008;auQ28e0Kj_=0M&zkDi7&Rf3b0Ma}wv_ z@$~fMhFF?&-&E*7j(-{9A2I)9@Tc^+Z_@im-O=B(v@&(KccZ_lRC7mrM|T%`S0__* zOM0%qcH)2g!9Qi|Px;`~u(Yytv2;KrrH$xi5C>~Sb30f-+#rq)^bm*tsD=N{$NrRu z8~8`RMgXD@R{*gIH-JA#3?T1(0#Jys0A#~7#2wH-+U+5RHgGfYbm-Rp==TVWxc~F| z|F|H<AbujbLaga;g5@+c>CN3;JZ>;zOx#>h0Zaf7AO<J^T7U^)1GoTwKm?Eg?f~+@ z13(qf1oQx7z#OmvpnwbD3HSlefDj-Ohy@aXR3IJr0OSKDKsitYGy-ivH!uJU0b{^4 zumG$ATL26=2F?)^85o2EA_7r>=s+wW4iG;`6eJCj2R#I7fb>95K-M4!kUPi^6by<4 z#e-5o??L&XGEg0;4b%tv3i<|G1Z{x!L1#$FNZ3fkNHj>dka&<pkz|pSku;HvkgSoM zk$jMzBSj;<M#@AgLaIh;LmEIDLs~%ELOMbQkg<?Skr|M=kVTQ@kX4cOk*$zjkOPn- zkdu%zkxP*4k$aFwkQb1*kxx<3P>4|IQMgbfQ0}8>p_rpMqXeKtp`@bZp;V)EqKu#{ zqQFqDP;pSHQ8`e>QI$}2QLRxuQA1FZP;*eLP`gmaP*+ip(a_LH(OA($(C(w@q1mGO zqD7&-Ln}pVMf-}jjCKSDgDJoqU@5RF*c9vr4gsfti@+`5Vel&W1RV>V4xJxe0bL*6 z0X+yk3B3Tl8GRUi4gDMgAA<!$977eu0>c|42IB)pJ;or$D#ke`0VXS^6s9JoE#@=K z*O(=kU6?bN2Uu8Gj96k=>R1r0r&zDCK4SG^En=Nw6JoPt%V8U0yJJUV=VG^Ff5Se& z!NIwOa|cHs#|<YMCm*K+XAb8Cmk5^|R|(ewHxTy?ZZ+;O?k*k%9t)l<o-v*eUJ_mf z-XPvKJ_i0Rd^vnm`~dtn_;vUb_(udp1bhUl1W<w~f&zj*f^|YL;VnXWLJPv@gjs|g zgv&(8L`+0-MCL@nL?4K{h*pWwh*^o1h;4}@iHnK95W`3aNd!r>NxVqjkTj7jkRp@b zB2^-_CygO3Cmkm}BcmacC9@<8Co3TvCi_87MSh3el01UElzfc*jDns*p2ChIj-rNQ zmJ)@Mol>3BlQNC6i*lQam`Z}moGOB<oN9_1iJF~SgW8umi~1Aw5e+SkB8?MG3QY&i zHZ3Xb9a>x3SG3Kv8+1f;(sU5IS9C3OoAe~~vh?=!ujxDKVGPs^_Zi$7-ZOk*IA>&K z)M5-`EM=T!!ekO<vSND0)XoHBrejuN_G2z&o?^jZ5oNJqNoMI``EiT&md>rvTQ#@V zS;<+ISbbOvS!dXA*kst8*)rKiZ=>B7y=`~<-R&WEBz7To8}>KsgB(a4!W<BecN||i zQ8~pp9XK;NC%CY=WVt-J3b_`!Nx2_#2XWVM@A5G7=<~$!^zdBq3h~<We&C(vBjkI) z7sOZ3x6jYcZ_fXQe^dZRKtUi-pjH4b$RTJcm?k(WL?oml6e`p%bSW$%>>^wuydlCY z@<imV$b=}dsG4Y$Xs;Nm*j=$eu|}~|aba;c@pAD!2`&jci2{jrNmfZq$q$mtQjAij zQtzb}r0J!fNM}ed$S}y5%4Etc-C??8aVO`_n(S>^TiGJn-MhSZUG7%h{UIkV=P%cK z5BZ+ry@-2X<O$`q<WuG66c`n(6$%w#ib9IMimmri?<?PryFZ~sqhzj>uLOG_^1%N= zr!uzkW93xkrHAYfT^`n}AgMf1Nl=-2#PZ1DQH?5~s-&8rI;+O2=B(DBj;5}r{zm<~ z2A_tXM$coS$0m;p9v^Go)r`@c)?(9g*J{_s(>B!3*Z!e%PbXeyL6=9@Uw1%{QV*h6 zs}I)K*3Z#DGLSb&G*~qhG7K>sH)1vNGU_v?G`2TxGQl%3HK}-l`b6hR!IKM9Rntt< zBQqtlw`MSNIrC)mEsHxAuPin!r7Ytu*Q}(h;;q)LrK}UIH*92Vl5Dmia*$LA-1fe0 zy6q1;Rl8ifYkM90QYbp~3AE0E(8123(~;KE%kirdhf|2tg0r}DqBG1z*(Jvn<Z9$v z>qd+qQ=i<~++VmadPsY`@i_6+_N?$C@N)3_<jw9K?!D%t;FIl(;%n~P?#JNw%x}?O z*8hC~D8MYBJ&-BzdEn|(#ix1CFrV2y`y9j@6d!aLtP@=SoaXt{=gTh?UlfMmhPZ@G zgi3^_hoOW)!oGwHgr|gGMOZ}iNAgA{MP5XiNA<ttd-?k1b+lFVml)xgv{<xQhuHBr znYg@of_R_!<ph<4>R0ryB3>OPnk4om2_~f{V<x*NFT8&Ex;BM5B{t<E)h2c9&D}Sp zZ>isgzy0yf;@xoCowSm4n)Jx@vkXYa<a@>U)tR?4ld{mVJhHxj(EiYqEt;L5Lzxqq zbD8UqyO5`m*O@PppI<;-5L1X$=w7%{WLPv*e6P5+gsbFzDQRisN8qFT$IY@QW#i?_ z<?R)s6(yA{m2azvsv@dE)!x;x8k?GhTD{t@br0&=>m}-|8n_#B8yOnkG?6sLG-EbD zZ@zBvX*p_jYTa(LZd-0QX`k-U?HKJ;@BGrG+|}Ey(B07^+tbo3)!W!7-dEQz+Fvsu zJW%~f=u_2aq0d!cguhe|iVW5ci4E0%mHgT~EHm6Ta&M%2RB3cz?9tfQajo&m3B!qn zNsGz#Z}#8dQyx?2(@$qmXCh|tXOrfr=Q8JS&zCL;FElOQTl}>2cxigse0gidb>(99 z`FEV}No#a#`Rn}ajT;IZLz@Pht6NT6=i4DW1Uqkc*>)@U?(BVn>B3gwF7WI9r~`_F z+(W^`_9L~Uxnt<@`HzSbij%xk(bJwYowK!b&kOLy>&x4h^;gPQGuP1T>+40T1bHur z6+%K#Vh7L>bO3RJP<|4L8Ds?f8!kjJ3rY~`&+sWkEULfZB?M*zA^!|VLSQ}+5`sbi zz$W5!4Fv=D5p0|g@hl|tk9RsC!lMu;5`dKPD;4Cw;hR)$4m1GyZ|_7y{Q7&k`Z&MR zHACP(rwc-e7&i~wBEpgWh8b{g-UZwUP(Sbe{lA(i#KG;cjiVF2nu{Ynp9nYajhOI1 zJATVph+6qAWBsLT{DXr7zh$g{DxCeP-hRtizh$i7GS+Vy>$i;cTgLh=WBrz~{yWK7 z_YpkkMmG8T|Le)?X+RD@LqSDDML|PDMFWG;&@u5bF)=VOiE;6;@yUqE$;pUGNhxVq z=qRa}s7XobIq8{hvEF9CO+m-S%f-ga!gibO<|QD6HW3pYlL!-&h>enzlI_2IT(<%Q z=m<q`1~P~NKq3Gk6M(Ke0Xjs6Q4va8#PN3y2SP$d<QI&Nfr*8PP=ya5fsm1rP>@kk zQ4m^WkU!!+fI@&u$iOFyMx<^EW^^Xze-`&1o#}2_8;QnO7_)$xOArPoDH%BhCCe>V zw%hE2Lc$`VV&Zc5<P{X}D?NCusim!hpw{LVmR8m_5L;I_cMnf5Z=c}jFG51Y!Xx4n zUL__azfMWb%=(a>lbe@cP+n14Rb5kCSKr>z+11_C+t)ulGCDRsG5Kw3acOyF_50fT z#wL9K;PB}9$I0p0O}Rh_!Se6NA0_)sxd;&DLP9}7MgiZH3xwo}IFSiZP#O5p2xZm5 zrp`o+{Lj#d@5a3^Yr|j?(14McxqQVWWfojyf!`GEXUYC^1q=GOO7@R}{VLZKfQ<}7 zG#)YmAOoBw<Gi(SJUafc?4UZc%?9kc{s^6u&W!G$bG!VyZUd~PtN&<PKO78f*tr9y zKErF;Q<U^czxiRFUM~$h67(cL@7Yb4y#uClkzR9F4*9|_a(1qU<XpqL>p|=(cHXf^ zr+x=)8dD#zekkZVn0G#R2srn=KW`rOS+1G;k%62hr8;|@RW>+F=jD?Mh;CHwT<rxG zQy-zA?asag=mtI>{>VE0A>BL*IInwc>>Oonbo*QbRfieK>FPbIj>XhIv_9vTY`~R& zOl}q3K<&|w49o)){Wf=;${%Wqp8OSERQHj^OGNO+V@0@~N1TpIj1SaW8Y@#uinvMy zzs07wr9j7Wy;3;^Iln(9aOvC+xGQaP2GK!xB@o+xEh3HPuX6N`n}<?G5YGzQo&EyD zCSJd|UM87{ex-+Lhhuf+UBnAzn)M5&z~)_vbo@EP$r;_tl=sO2*MO)BbEMdl1?+I) za`pmVEr1<(*8PC!arON7`Ax#eSJXT$rz~dy%oAyZzSCG?+(O0Lw{z1Vl;4N<z1}fd zz0OoTZ_mhkLqQ*07NA}wETIPbP(#Zm-c{XDWS{`=E`7pnIK8Ko87rzX&%i;TfV|4S z)2-~#$K`O$bs=o<gTu?9Eq*=8L%4kN2O0i!)2DKFCBXpK0r%slE|ymIbcr>|oDxrm zIbR6{0=6RJ(G~7FCmn_rM)H>5ux389>j~WLr6l-{Q|mDwT&KYM3|(7uq+I8!H1Gu3 z(PXPE@x33{IA`UXLK@ZXiL(g9s^$JPWm&4Gj*eS9`P^PMtD-2#YrHbI-2>ICpK)c} zH`ThoOl7X6R!A8g0gwZ@;6*Ko2U+>hYhd9TFrk0+Ro!Ib=q&zV!fEz$cm9Fsx-7>x z-v|`C!Mda4tWyu#v_rskQ*3MRFl+0Jcwog*26Bwtph;jh^;xvIufGj&Wz<vJWU=`} zgKlQ};ZS|K$QxQLfQp5;H40xiVFKGpwv7z^fS;9H)3O6wjf0cmgYB1L{0M;alir)? z)o|t?%&8XDAC+y#%jK!2%E_ll@2JN{k#Izr(-ao;@NO(dxM4`zF^1@XLY(^B5<_e0 z@qO`e>>y;1h&$XRK66O$#^1?^)s0POLw<TUdIc2!Na@Q5hY}+(y~(jFqtM(YL^ct< zl;iaVt`r`PRCA@j=%C4Q^D9J|L~r`^sV8moAqZ#o6RC%*;hYjR=|?7Ij{Ze$A@%s} zo~Hnq8UBb2e^;g@if+$=dPJ$|`IJFi(^009Bhl=g_($JG&G~^ZZF*tqGdtS$K1jY= z(Q7R&rByfvPLIyPxpbE=X`5P(ZRPMD1)ft%d0hkBdC07tgM1?|n(F&;M$l<(Ci6r1 zG0YR)ELmi=ELaj#3{a5u;xgqadngMSpB5DG0yF^0S9yplW~l*gW;GwCWhb9u8#5Wf zhmZZ)v?6N#n+oXkFj<;Biog!CtAMn)LH0WB+bGj*Wa?1ts1FSle$Q{4$yzDc1M()3 znj@>jA-N<wg&WQ4Z;gujO?6xQ{7<`Qu-8PFYCjZxscp6hFW%kRwIAM}`%Y2J67?XV zdof;<i6Jo*#k@A3VEmq<0&WyY`*AF|w29$ml1--9u+Ba2Hx{;^gE^j5lZ`IwE_i^4 z-&4L!J}y0WoaSn&d@f^iE|S`8+tuK3net_J#-UrF3rCl<f%-@1hDC7uS3&-IzA?cr z?kHLIi6IW2p&(hBHldW<lVrt~)Gt(aCs7lzT%Ri)Ur%{`^F+JTN=m-6EV>sl^(y}B z(KWzv6(VFd*A#i?QDa)axk+KX$#L75qU!^XQ@cDJKk6O5uwE24Q893=9Z~1h1!d&` z_q)841|_%3RAPKDJGCbj0`2fmnjKZu-VcT24mgYWL1G2XD---AJU+PH@77*IKh%a7 z<ApJVKS<JnHD_Ge%wXXi(Ff_!Q@<s8OZ{Y6m4pp`Zz{YAP6df^U@(%nB2baD<+YTy zLS#*%WgHhk;a`(LHA{V9Y^TaPGuG8?<l%E#((RF6G+<paJ$ns6y$Ya`4~Gr)lowTG zuZCJILrBziC?-+j1ZQv`KKDsl=WK*n;&h$Q)P!?0raH6K2e$JCq+7-e$n&kp9{6qU zBn%ElKh;QRuy@v0BBtA^AJF%T9zJ48-+ol>5o$J7=~CZ|#qVggN+`(BQSMy#_$|%U z=5ueYXg!IB7bF%hbVkhtHh!pcHIhAq^M`R$5e&pCOh_1D`p~_C`Ov<?jvoH*dmhGE zGs3N}nTl4c*YhfAGr*<mR1b5fn~CQj5hai1MGT=A-hDXf8}HoBm(D_o9fF0#QRQJd zrnffCxQqJdn-;8XUm0ciP1o7!4V+!Ilrm+$Tq?b^k!d%^DuX45Ovn#8I6{lZnJ6DT zkSr@xlbk8XH=(bgk9K}pP*80J%SrcM3$$Y!9rsoC6&pUDBU65c)lgXe94ND9j)H=z zw7kDOU}yh2az%R$q!dCMs$Jv#ZV#8Rz&eaKr@Dr{d>C+Exk$1TqiLf?C=}PCFRRIt z69oT=<l8t5#MR&M={hcm9}v3SX{u@(jgHH&<qgH+u$Lc5=Uq8GML4%~86*xHg|8ZZ zG3q{<e~_#TyGVj@q`a@KZqm)d)PIZu|3u))pU1VjjS`EtorY?ShpAigp7w*FFDQ^g zg8Mn&X~6>&QHQKmP{32%M75PUBlOgS6XjH^l*>e4@~RPS(Y1Fg?=lGTv@NapYkQgF z0cNDmkAoCUncEWQXXonLe*3kLW7t3Xz^{SHuN%|XfK}0F<cajF?wCh1=j_00qe-s* ztWYD_a4RSGKpRU|e9q=0W-YK&|BGm&z91T<!G3h^k^%>~SOEubW~Bfo8`6AeO`mrl z<D5J*#%e2c3~fCdXmGH<1_=A;2A;jFTl1t!YYaRV>U*pwP(p`!GjXv?`64E+D%6|# zERiP0F;LYOcr=fg%$hE52S(<o{_tl5{>bjk@$n&oSd64!^~V@r1Ls@-?c2a(O`iiq z?uSsJ3SW7OAL0&kKd9f1i%rtT{*Xu?7t3GQMggXeGrWK}#bwTQ#u=f~+A6FL35#KN z-Zo7OMV1cEc9cKlE2xjotl`D%9_OZ=(Eb>|d!I<2ueX9ihrvOl^9f7Y3p&m$=4Z7T zo7xGKSc?UN)+7_bEP&9~fHzk`8(VsRkmaOYd5E;VlftBhy>k<<o=L{0v2^X%tYm|) z+eJrs<tj{FASG}bttla!fY}|*yN`@H38l@O8cV$Q4<OTUaq|FG0sVGIqWcX;Soeq3 z3qQqmIhaQam7$K&YI8C1-3~oP;&X|(TXI~IK2me_S&umQrj??XTWU(1ka2yWR4onD zTZ%MF489_+cfbd`Qc~+zMlq#T0QDKJ__;sZ89Xh02k|vs9)(X#ZzxI!r=?v3`hZNb z>&A{|m5P`U`Ql?im=bXzyC5g%2_AcE5V~osEX_V|o37yY?(?y+l+M}7O%b2O?+Hak z1C}HoaFeTv`s`~QR`$AQBp<#co7K}Le&xU_O3GEmA$mk!X{R3O?2NCQl2x6y$Rr_) z#kr`bRh?+w6l|IpgN?EASo^!RL}UxQ5Oet$tDeRbTk(C5C{-8F63L>Xh%lE0F)jAb z6pOu*+&k|$RAN*No>myDaPC)@3l7$ncwuHdVb22qaK4z8m6&G!=S-pJV(ylEtCNT> zJ(#gvS~FZOh%2QkC3RKlFV<sn9Dw*7uZwN$-F%Lv(?)>?G!NJ4%9QrgcL>KCHx^$W z#^?+YkA)#nbk_OX2yyev=1Rai|7r~>%8hP*We7Axy4#2;9^3RL;SRo`{`Bl?tNNM{ z>zY7*CPuXVX^7E4TBzN}-tA7$T>Wp;Qwq=>(`fQY!EAH2ELT1&QJMilrClXK5Vupa zwQCm*hVeje267(GY^|Vq-qjl8Q|v>T`Gv0>Uw$$TZ*|r-E*%4W&p=Wv;*dA0{4x z3Xx}!L5#n}L!j#a|KedzR{AbpG3&dl!7T;9`Q%(5AmT&tUV1i9!qvovf@JBNQd=PD zcbwn#^Sf{S9vA<7PKZeEy9VqIvwmJP&<pNMpPp%)%IpNZGVedFKhD6g6}wtE8}^>x zzN)nTeCp%*JI?R=`Q0~ukBfgbC)ngQ?i~l}OWs_5lab$K?Ke^VP166r5Kw*-mfu9| z|2z_k$99=5Byq{QD0MlP;O}ngQG?GuXeS`g7)+0Xrh&(pG`|3M!n@!!30p`Uq+GiO zOto5Ud;2daM#CHeMxfJIsl{q=laOW}n>p&*Q9;$Pfr6R#H&;EEp-?RGXYm9dzuCKy zQyhIrRfqn##Git8anI-1zy=>r2nL4LUjxtFcZ`i(sFZd0wo7x72X!ZR_<9vJFsQGj zT4~6lu2@e5+I9L&mv926#t-9OlNjqfJe<L&X^G}YIH92pn<E)*H2JVeHZ@+ApKo{G zbmy69ywEl9xXSB}1Sk!Cjl2gHCSwABD~$tHR6KD=-JUA$oDk7(u+xS~O{b94^lsIH zM!?bh;$woxzS*UNjvb{2-ton7t~VpQ6Y!0-6@S#+ttsQ|%s_j{#E=o>4W(-3o#_eN zqm7v-@bNmkn5Y2YW3X!4US|@#Cy+dTp6`KfJ~Nsn)+-8d5>o9o5IRK{A^5GCVRlQJ z!UV@=DzRn?x~*|c^-AfvJu}B@nuwDsQdNspDIUaw4(%{Mt2#ta=>D8?wHr#mGVjz} z1}I9i*kE57-kctC4TNOOrW&`rXc@f1eX$o)e-Y=0KX#OkV_UAE_rw=r@(ub>_%N7# z-3R~v!gxuIXLf0rByMVQn2;UUBlp-T-Ra&Xs^k69mbXZ;(~3;a)M-Zx^$q@!3xQql zFlPvIAOZBe-*Q}-q6R4@rJk+9OYg&%z0*koaB$P%2(wfsX_)Qd=JH8lGV^f7#^)2= zJwNN^J**Q>`D*|}xXjUPKkCCh!Fb*y^rzKQ@Q7*HcukI1uu7!HSH~w!Wv^%|?tY_o z2M<Lr>~nl-CerUIxKKYyESk&nSD05*5hr!Rl1E|%c`t~V_-W7Pyj++^Z(B~F&6DnN zp*+wYS*}w*=KcDvOIuRXchtJ%5gJ=d!>4nTvr~7HQV#Cu`P}cy1sAp_p4poU-Fsc6 z={7i(r+j_<$m>SLc^6wX6{DO-M#6LFDs>^7U)=GChiWaHH_=PUVbv{5So4?$8_h^3 z))3Q0r6-Kz%X*_E=JCRG-ix5O!EaX*-X0asmnW^A$UvwF_Kgjn)nr^QF8I1?CT$#E zRkLiT?m4!fU&N?PxEmPZDKw4=p}AL9hUdg)lG%eMag|ExVKGq`>PvXh;+j71UVabH z7|bzLvCjQ+ZjmU*c3^eKiHhl|4_=9C-D}s;tHf3D$^90HD$b8<ARK4#Dk|TE^7QJV z%L(1UfnxE*+Y>sSdbmSN$w|!4WA0~m92=?7`Z|Jv)%3X;{%G<fD$TTKkHk|qB_Vk# z<_kL>yJHw>SQM*Kx7S>{JI&!FeHY-G2)dv&$_Ro=p}54DLOv_=(}V}d-;*s*=Pz0a zs>*i}Hom~r^AN)e>l1%y&eXihlCpbE9Va90!|X5P>`GC$7WmQo0BRlA#Y_DTgOSBi z!N^`m&K$!YK~4&Tx3vDQ-4Z(S0sZ<c&rLA3;ar*LDwktDQ_O4_u^fq*syX0CZ`1Xc z{UrRIGyDio>IE~0iqoomq4s5d%@6qGCHw)DH1-dtX7iST1FD^bjjwck0|HlkX|928 zp3q~;ufrvc>SoE$s)a=aZRcLnqk@;l-XLs9J>d&`AtUpU)Chyknuf<Japnj2qV99b zwwcuG26+{?AbZ)}t2nXCUU!BZmv1EKDXuA~k-RNO`ZV_;;qAops?OYF9eDNYVacTW zN4YGCg%u9BNa~{UzWCsD=M=1dkI}k<$b536lgv01ZlMWW9P~MHT`Csnx+19>74#kP zrDSgt%n_0MUQn`l0H_)_aVQ$X6(Sewijoamq6c<LzP1i?K&`YmU0O}=s6Ensrrjt) z>L-!yH^09@=6H<it-6DgYp_YV*I}UCNjR-i%GgJROX0+iSGJ+AHSA%eZ{mBu|IOvR zz)oQMT9%hiPZ|YW`;C8|jeeyLsW7L;_r<uS`@Ixm+(5Tj?^2oDH4u3P8asOSVdLsn zdHhMOyf$vw&Je1HT9K+fQR+$yWMKMrqtN1K{{EgFXWP`VrlzjUeg-ld{(CHgnB*@g zr1&%?%|)UcA1_R|;3~mf=CG`v_BiO6R5E^h+AhxhH7FMxByN^@-jcjffAN{C+|hMX zQnq9xfMx_?95ZaG{#fxim~)qyXcF)7wpfW{dmw2e&JxtfZI-H2c*NwsM-OpaA4cpM z_mNu9q*d)azR~oCG~J1><1!DPk=>QleYj<gyr|=RCa^Dtvl<J4opFL(BvMzd1dP3U zG^$~Vh@o-8lf5nsx2rBB<>?xm+gEC}P<!u$>;2{K7|WhP_RbkmpvNWDbsdlFq~i|E zJ!sDBjb_crg3Dd+6wVcQ%m^%z1srefisdaC!V8@fn>hkF^33`~po2jk^t+p)Wf%!z zx08!NEV=dDu~|07;(+0)i~6^WCfic%vw}wnaTE#I`)o2@@e1-9@Z*=Jj^~PgVq;vU z(Nxo`4az~mXsX;qRWbReVS_nRY`I05^9ENfD`LgW4Ln~3rtFg8j-4hLC+rq6ol~<- zuZFp`X^NSj`)uFQ#q%?(pJ%-+9d7hoK&=$1nG=BvZ8A@N?rw0k*{?-iijaEG1bz!h z;5o`F+3u_ccJYi-DVeOEUITHP##C_9xFsl`I}Ng0O-qG&&8HU?!Rv}@{?Rxt34TVf z+X<>qy@Hse4nLzd-o;R^2xXHss#CAU$7mH@78+QO>&OTgt}l*`Bb2o)w6P2Hbfm*r zD{8p+dL4sekUCz~C-m5G1gg`PI@{l#Zc{hzfL%_~_aq7_CS&DNDO0>^YU2D|knv`X zLH3WFL5mWmE7JLA8MRdrMga^$qdRukh$Jf`&e7S)=p9gy?tsgL=DZPly2PdVid%2& z&|WMx?6FWTtr^!vE3)%zp-%~{W3X<juoUeKf4;gM@Scn8%Y5)Pz%+CLfs?fdV0taX zg=-uw1%q8Wsy@WV7=>Y(&yyF^Ab0x6L<qhbm6XYw-Xyi3_Uko}cqGodnLGK^4nbQ& zXc^&VTWNd~f-#{ROA^yfW?q(~COSdimhZLdBGo5hX3-g*>lne-9>QJy490{$E4B?R z`qQa@#Bj&<i6Z0R>KC29MTEkqA-h~-4C$CtYA(12UbO3rI7H9CjDDkbQnz3vp-NWM zOdrICy>9@RCVR)>QlntiJ`Ic{|54J?kvf-B3W?kF5jtvGeID^(kSM6}Z{Ned7OOzK zNyTihH@dpZ(7S7T_0Lu<QImuM@M$}k-+F$wg5YGHSR?p_;5#L3X#xWS-+|W*RxdUC z$o7!7>4Uy?gJ=rOdE7-BUS0zz-HFrHebex=xp_b6iK{un-|OPBl{FZ`r^sA}s6``^ z2LV-qOWOGvpJBU3ifV9{<HXDdrp^WH2fVRJ%MLyA<{_eB0H)$ZX;rrbRc@1e2utr! z#^Grnw9%e3vQ|@jiD6ud8FUft%_{cAZg5T^a59a)^IZ)xh1l{=6UV)(8mktv>5OE> zxoaTthjjRaq~+os&Sy(#@n$%L<rASQ*2Ch^XkM%``zKC{C_%2>5J?#XkJyr?u7%@7 zmw}HtvHQ4YHisBLcFP_hr7drz9t4KhTUKYm-z<_;ZQBX5IAl)_#Og6A8pad^tNR7i zHY3_07<LWZFaJtp9Hi&KWAMQDd5v<NxonMVEysS)BRu1G8wQTT6Ei=~TNe9F;#FbO zds1ZWEhK(KOHB_qrtrd`4vyt_+xy`H%O#&TMFLx04UBo=!jX$A96MKxqtI<GPbl8) z)tJn@)tHZHjgOZMF7ClgL*x5pUO;kfg@=2jqBH7`ieey)ykhDvHbLaQTRR4iVwUKp z<Q`N_WM~_)cC<iip27N)@#uA&C*wwBwj&6?mz#_yUt&(1Z21PXq$qcp5D4xOdKE5y zO2dX@eYTV?8k$pta&$F$L~)kArFL&Mjo7|WE}ZpJT*8g?cXk&z2{dvaVU1AaOsbK5 zZE?p-?V=F$>|IWsW0Vh0lniMA`4*odr7e+dpjG179f%sB;O{)Rraxb~ZEVu^S$|cD zXj%m-$SKOcWYI>Kk{_wWdbty@eieO{S@CK{N4L;t&ZKm~XG~FMhEBiK+-*e9pnoIO zYb<Q=uDoix$m+&iexU@Crw(I2{rTF4@(RL*@=QI3<NMZBY~ODM+HXtd{~pDu=FvNa z5!qo!Ar%G7h%e)}l5`w|3!>6jho3UQb;1Q>Fl|O`o9UY)`n$QyrIlFA^Qg<DpAs1x zu*Ud#E^XHYT}aqvK~#PS;rM-wm%UthAx8U+nesENB-tH~rx(*>%=fJMQveiCDX~To zf(Oy_p`ZfRMrfz~rK?|XY<{xVm|_}iDQjly_3J(fw<L;=?RnqX8&LFU_x8pTUW75( zr}~ietdsIKG~I6wRSnZK&&3o(S3rFmMn@!B*@PrY)U1(v_RV5`LZfab=(AmRh0L(3 zuADqd3}{5H8O<H@Y2c}X*P#9}E0w9dlmd9e7%@|VRsG$`!X;T5H?DX;o@UgVBs#ou z4{`WZgeJCeSTes@8U?-ZUR3tiAXh2wuaNb$?cs71rxvFhiZSEBknfn?@V!z#6p4DZ zeNNpIDhbX%F<1&IF0^_44G(kc_V>BT``e`?gEEq~#QEUn`{zAFS~wD|b0RRUokJ6x zBO^=|++C|$BN@YJL%j;>kKvgsLB7<Wqdw`DKg|FY55{X$q~)a?zdQLTh6**&bx5-| z!4<<;$AygU&MCTS^x4^G3WFG;;)#9Ia-$}a{fNI_F-%{ueWhv_p9{TA_A9^tShFTB zyWDJ>yj^Xvjmd-BUFcALxW^qT)Ck`T(d4V&Wnx!A?>#P%rW!ij!&^_P7Y>IWtB8*h z5F48-RT3(#hO@@|q!Ny+)+_I}oM!k-2fmEqn~)|_xdt%pG-1&%{fW=2-}BbjwKTDy z9HeN?PTQFAlL1c->GWqOE{O6r(?nosWsX9b(_VpJ&NUi7dd^d3Od`d4)AW|!+Yg+& zWa~*haNkiazNDMY#7#Q7@RXb|hN*@hrju_P-#T&4+u)tq?2Qb}-Gui-SKP?;w6abU z%euvBZN6TSLt(iQTz*P<mkkDRF{@EWQ<aEZh^Fo+B5$}UTW31izl@gImJTM#10VN_ zuL1>n1OqYr9!B1a><~#~9-<#we6C;e9s<4HVsodPw`V|mx_kHj)_A?@vn%Wt7}*Qf zp(?CMyXVpS5Uvj|!f%n%mt5p^xc5@f9?kzUlNK~I`0EL4E0=LO)wev~D+%D*pk-Hr zhNMGs1}vE^-+boYiM`X!7vp!XW49JqarX7++@DybT61b}M5i2RFjMG`;;B7i@Tnm) z5!*pZmRx8}28q;&r(>!#m`921incy!(qW6GAs*Iu$le*)s^SxRW7pL~IL}oR*k$SC z<L&S;ZRc28X?)$Ct$HeIPtYyP#7f?m4ITMbdg8eCIn<G7>+Dcb1ZL1Kas`H==7bii z*a^)Ds4z)Y2r%F=m<ILdzCSEVNA(j+emLZ3_u(Y2*unlnwqVxiMYRXB92<bfqCo9y zfv101Dsh=U9(CHn^XPD{rp3;$@)}^6H#nU?J0GO_2utp~BzvHWYcqFih)(XMWS9_D z14QM5zztxNHXxx!va@x3;UTwhwDwGJI!9Ize93JZf8dDsotL%<Oq;l2ta@&eO}^jk z(3G#_dMjNmaure%Dh1o0=p2`(i5}@z&1yPCktc!#SSOijsb$w3w9v0nyA{QRHxsnL zg)0`O2QZ4wBNfI-j~5<(!lI`k!gL9!r9GPY)fEa7zT2wS2YP(S%ceDcL!I2|2-ZKD z@6dqgJRY!P@2w%7aP~j<PEK?_ybxuL8?U1~QM}^X6n>dC@Aamm+5<aCWXvA=oou_Q z7v;IObXKmqam}9wZ8l?xcgxEwI@<}60P=A^hQbfV^soyR6xBkqcihXCnC;ed!jA-M zt@F>KO?m=x8vU2;aHJ1RCJyh732x<;a0#BXeTeSmuhx4vDHzr1B-(noGEw_0^EqjV z5^pLi6?*F)1X6i%=+?sJb0hu8@}_jXBB)#;f;xGFcHX!ygeon(Z7T$yO?59M;qR`2 zE<CK0p3bT61m|066<l4}nSIo1g$!EwAt+@m;Z|l6=Gne~@*QH@t(;KZI-(jAnn`L* ze+(0exDZ*AGSE4J=5Ju2m5EI82sW9CVPlDSo7-GHQFR_U?8!g<hl4;?aYuHH6bu~1 z;|8_Ug0qHK364oB=NEJ_$0@FMI1(O4ut@F<y7sm1#oPOl!w_sc=t9UnXEQ;6K(c0J zmcu}Rf5!s_xlVSllw|fs!uiA6oMoMB7m#Ut{-b5laWttG@A({F3(P9GK=0;3uDL>I zpO9}UPnFPHC0@DqNHuNMAGCbCv>AM}f4Fz|>wN!w!X?_#Ra4-neS>*qufmQ1ma(e9 zkg2o;YVpcu|GDHMMTVy;bN7Ch=NcIAQCgtGJ&7ODZ5nShU&5;~ndxLy#^QocPUVT6 z62yG!Io}qTYH+*}G^4ytM2ttdKwJ3-j%_MDJp>zR%%hTc1vc$sM+13lW#bI7okc}5 zzEldH4)hDbJxEQM6N~&x3-iRfUZGlcm#&?~2=1+}8}zg_V4KlHb6a4r#$mZb94%on zQaKWeJGlMP&`@f$@O#5<;?;p=IFsnQXz|bCcf@{MR`ysKrgNcYbl~b;bF%Jrl>5Ay zWp2F%_HrwYX$V>=|J*?6INgz<Ih9^G)-#<8gPIW3WtH2m;&*{u3nQF^gsPEAEWGWb z59>L8#`DR;LYSWyM6qYQ_NP8wviswz(i1gP3854T5tSkG;!q!xW;!A2Bt6jQN_#=m zKa<sfLET5Lbu+P3uS8V9m0Q=kj#HssAsJ`dP1&l)FQqT>Ws)XW^?VDTSonIb`^x;9 zLf+IXSGT-ugZ$9l*OQchOLU@9@erOw$`89$#u@ARNmlfDIu*Fq_{OcS)tRgs7>mIa z-WS-_n}qwZ<jdupyis}iUa?M&nM^IMOgZKa#Trvw!aRx(_)lIh@WT1f0C70@Wh2v) zBe5MN)8`LHr}4UzMP+Toh8Eqrq@HGIo+JP$V*F$g1J2bG(ntWzO=WM+&P7>UzC{s8 z*hrepnro8zat&A?6`lLjsj4_^X-7|TXS)*0ac+O=>`s}=AbLT;ykO3nkZ^Z+HlgL{ zKp9R_<F8+{c4c3QSfj=ZM#}z+gC3f)`rr@nc*4)OW=T5^j5GoA@_!Qne%8JkiWW1S z-{oz;^y8KXe5-!5zG^5{)YpKMKjf!XUEupglf=;PNWZJ#U-p7tI9+(93EsEMER!+L z!%6Xw+E=$7Ex}3B9fF{24cQ-q!;R6)OWXB*e14L!?%Y!3)|Oh6vTv7iEnG1VG4rMZ zyJ#HW>2ysw1Y`NH3)=DUm2G@i?v-B3mu(7nQFfimO%-K?SjRDiE9lKW1$+~X>EIQe zXChI|u(cpTF0<~-DisZx9pcJ5J>ltl*wMw6FeT)xQ^7)5sBA-`HC$-7W=Z*}3zb$& z{2jdVd}P|0afOD(OS+D9vSzA4^9>Je()~aig?Z~2<^!k55HmF0{X2tA`X9!NVr<gF zj(WeE%ikYmX9H;|0JK3#5gZ@ZFxfJnVjifF>5WL!T{w=Ljydk!s7^gF{K^d<Foip0 zL8JH|fh<|VckPC^`Y#-Fb+_oKBpECNt)ZTn8x!#=B%R8P;%`xV>DO=uh2F`#n=Nb) zdX5}nNj;b`giLt+aH5TuW$ev`gGo4VMx<Srqcpyk(k{+;GSoBE__%hsByBcy2)Qyq zlPBbge!b)B_Ffp_{M{z5ssXKZ8NJX!-Fgj${vy5Y#G5hB(Zh6-yi|&F(JTpb8#dsX z8$Coje`xZU$aKgz;Y7z7u^KNdfC#Ri54);^xJ%%j99!*wr#t@qaXyEg(#~Z>)x0ck z?L2r(=@+?-?fvb`_luza@AcREt-ATGy7}j-8+9X9+;k|{*yz_TN6MslC342hXeDMC zDxhCt-6JhH<VW=+Ve1FHb(mVtp+#~e{a%#I418mdSqC-m;@*U$oikTx2f|SPNa40j z*dF68df!N|!L#mbz~-XSH&|Qp_5uyh-6nsH)m@=LeC1E|B^g)tpZv)r>9>)Nms5uY zXTn~MK0H0z3zHT+b+Z$MSw&p~`$v93cBf=xX6I$sKx-Y1yzWo{xtHU^`g8xI#?-lz z9=FtxqH6$>_-5k!g6|U@M|np|+gpThp8VuDwrrV2PfI0FrgLXSLOI<pxo;(11054( zn-3y?d<gf7lYa3?^}K4`EvjHstJRVTg&<b`R!NcIcE1mIu3a%i*eB^7Thj0*OA;&1 zXw<>pCi{*zGRs~&&ZF-(T;Cf{{M}ho`$&ydSSBwzdAy=sdsIv3j%dhqMydwT_bPoS zd7yN=16Uz+a_jyvf(gx0P_n$ttlBgV8|Jj{njz$)6@5&oTFCi`8dhgcT)hVHEUtm) zrAs1V7iPg8DJTg2RXJ^ZrtpTLFIhKUfIoXl!;<oRxQKw4V{3V%5BDCA=0x&yjPCUe zna&kz>x?t%iR^W8`J~{iDTt3I7HVPv`f`*9P2UmoO2SOS;|2~3lgtSAaLkoP1K3{s z6+0Jqes(91Yhq)~<>=;Y^wCP(UYrq>IJJ5C(KM|!3LT*&#+SH&G{LV#SB6Jq`+MsC zqz*CH0MZ4j2O_UmY&Nu<js6mDQ`#+bgd6%1rvmxcfCVizVu2Jh=2(eiw=u?0qp5w_ zmbPhb%xi~C3D)OFsMIcEYt%8e{hG;gx?}#TYCk^^#$@1dNeHa2ee3&j*BqX)^i1TN zEuo{>zyW!>xviPF)>32GL|A1hDFlC09OqCZNyCLXydwB=oJKQh9~Zrd(oVrX)t8dw zFT_-Xg6pS>TlE*O2aza&=Vcz4{1TBszvKLu>xbm0Sgv>@&D<f-jP*Y_%>Jb|yF)nY zdf9gL;g_WQJI?>z^}`W)lDIt0gRFNa5^Gcc2)<u=$&GhZgSWLE+@OZm676T1<;AKX z9^K%&T1XSyMm{UlaeRNIkmwY51#`dZs}N#JQ{PO-*H*^C&{eWttX*O7mz(+6eT<N7 zGmclN0&ro_?bHshik^g?ypsac^D1OsNZ*vGP6kth`;Se7mXa`y%}=b4Gns0FIBb}L zrkD{^E2ARi3oYafzzJ1Jo}?^HU7i^D`-zWE|5SDp$NRVtmL->TrXqKzZq!dGPlk&- zPV5CjlC7efWts$~9#Y;GOj}mT`W(234sZ91y-;byr+t6;n&$H3X>Ydb^W)U00g))7 zOUF(NUPs37%LCQDV%LD?Ya0KpVVms!&`-Ub_B%yfOjTtyAyZQfRg5o?7-Q~$k#aL5 zGVeyPWgz|iY+N>i(d<SR^ew4|c;5bCXR%vbCIep!0BgCe7y0t`7<cZ{#>gv2Bt?Cg zn#_VtFA2p&NnT3t&Z)3=j?0nNYxQ-8Uk%eO%Uv#R3XED;->ky(Hi>Q|v0HKv+MV+j z{2b7`TRz5Bn?!mjGbi^&^DHe<rXway#Ph7sORk4NKKG=0)|06ef0Hlf)DGjQmATZO zR+Paz7fh5%q#;H7*TqUfeZN=V{8E1YyDN`=v+@7VY+N@@0DkM;LJQ8^R%&Mhlq*3( zaV{=qR-R_^E$E6Eln$5Z@i0am8vkxHb)Z=v$fP>5^j$eWMjp8YO+nSs<&LCc$2EXH zr$29alGRO^y%%{zkYYid{nkGtatPGMRNC_R@#1lC-j;a7$zFHwqb9G~@(Ra>oN2oH z2njK8!bHMQQmH~ogj01<`-i<ugkA!%QOk{T;y-(SmkeCkk|Bq1pnN&5e_7!@E&_vd z6=lWtD~k3~&|HS0xE*TPp7G_Hr=6%SPk+|_zTDo-cdrT_omv-*OL)dT(zTVg;3oa{ z*@gguE0nUHAl&>;l%U0>ZVd^!Udbws5|2iSf~l?n=O*=fg2ClpDZKVlDmqG=1o^yH z&PMqhrDLeZ;Fb3S(HZa$DwN<s=Y*&U&c*&OuT@iMUjBvSjiJW>rX>GWI;Q@|ddr`t zj^C{LzlFrWRNo!7&#S4PWF-ptq{2k410$P-RcxK+VZk<!6Gs?KUyO3=1Uq%q^T-cg z1H>G@%OOpk1xlu{^NxM;BNaE&rgvh(N%TvK(mROVR1}?!q+jOf+f1qCyve=CP*D|b z77n&VrVBrk+R|5RXyDK1;9y)ORySf5{W~-Jul(0}dm%frcOg*H1_!r)OmW;!7#Oop zh^d(hB)h-?dZlM!phH_crqmS06um^U{#~e-qrpe`tzh#a2vPVp4-;YM@<=UH8JIr0 zlRzgIp`|otSm3CD%`eIKJYPHY6Ivqk{Y(?yK0OmSU=aAki(NW6Ud7kr=e`CjwqHJN zKH6OQt0CmKyh*43FJws)vYYJ>6mKrS<;4FRW#n<sup?z7qI*qaLKKn>5~8Wz246J^ zo%ebk)ZJJCCSv>?8Ul&8>cJfY3Z^s%FXgn^0#@Iuw5O}=0^m_ReTa#y2=qZ9_;9Om zT?mhAD^H<nN#vlt=AvOL=7^7{dWJ4Qptn>Zr#qb3j@mXa0;wRc6>vj(XB4NcEk@N) zi<m+>iPhzSf9G_WC-h@)=7v=stKWYzYHI3g>Viv&^?a3_b_V7jUOXQ>$K^=j#XX`{ zFH4$>VW)DJ`bm#7MMsc?bx6;X_7a^mn~H0o9SdHzD03@(nS6l9GWv>daKtCQE2hd^ zTS_Hg`Tfq(UrnVGVs*cDQvX69_rIWT;`r<bfwQZ~7$<ZXV~)twTs(+OnFtuLqxYfs zfLQrt;o>jbt<qsWvU#(i#6;9(=u-3h^XV2Eh`wCSEgBg`rLX?M4_xGh8fD;OA$VOT z7d6np?$U1qWwgUmgJwS+WR3X{RD7><@QgkE)L}aCCHsl*lI`1=h?Y}_scYaR?H8lq zDaE>-#Gfs1*Ddogfb|;qs&x$vcuZVCs1QV+==GC}zx}p3cG>=4cy%*tDbrGF8TF1M zo&kVZAs}o*d~qM<b*>mzba)Mr*n}UbNT(14?X&F8sY@8*$*;|*ZS=XJiZ<t#jo#?- zib^rJQcWT%C7>pEo0JQwo+Y}I)Bn(>k;7dWul~7(&u{7fU+ppQ|JIHuVPY@?V$2hf zer&EwQg&U&qRE`8Rg2XhS*t$KF%oRl2Hig*8NcuvXC1ntn$Jm78a+}}VgIa%GQc<8 zc7#<M<NWx<Kq0uJnR>)6(%7v083Rs~1HH?<hD#vjczi=$qK>7xMg&_)!dte?2;kp6 zIl@i`GCvo2;T|hvm*nS^Z)$6>Av{tZE!NCD?SL1~3BkJ)78y`P0{rK4-MdQOS<}67 z7CTlS@_mc>Ju7`FG%Wd32o6#ExN3iKt6f78Z3^-VwZ~R!is0exTUAmfZS~vItgxJf z;^`-f{`Lt_y4Fs=ffMSO_L?UMFLPt$MMosW4t?$7g8ClMmIvf3UcMV1He$fG(Tl_{ zuC2yk-ALoG<q0zuf;~Hs^j;i*Dn2l#dptz(rq5(g3F~5V_x^(Gc%310Jq^3FzpCv| zyMkUb9UM1hiR%`Qs$4_JakuHK)LXsM89z`w!ErjdX@WsYtZ@t>LfrcjKtYC!k`%Z} zNzV3WuNfV#YXA$}dJP<kdpTYMXikA}_o^&AaqUuML!0_{(b?z{%2SWx^JKJ`)zFIn znUNs~g}zLTE)n>{uclmPkz=T(EV}fvA}7!yWYRHUf-V+!r}jXgMr^s%m_y6(Q5J-p zi9Y!45Q_U>tQr5Yn+*r;R%aS*<>-V;Jsd3oEwjg_Dt=e!Rr;M56=&LC5TjOXkl6d# zpX?lkGbZ{yLQPGdKjEu5t<ozogR-AsuxDH@KYav#Vgow*PpAAp@dC1Ky2`f8)f@24 zQ^hGR;;2{3jb)@i;`Bn^Qmv_tIGYoLeo&&1yBE%EPhoU_AxM-yc>F2hUsa5MHdAUI z(HN+HJm=rk8Lu|6c_jy)x}|&Z4ey>|TwTl)u?&pOn+DUwk}5Q5ikNM6)Zh~^2&FZ9 z&y2?Lb|F^5$H}()Z(EN=ME>E`^z-_kGQH!<EVD-B!U8*Xlr=_1rE_l$7%(5DDPZQ# zQ*gx`+hHy4z2LfFSnRdWl6K!tZ*HzPPp*4c=V|6Gxkj<0C1V4T9Kb`mr?|TccSwtt zX_WEqS?4G#UN8RkQfCM;N@rUCI#dseZ{H|(qLTO~ueLhocn+9SXPjb97<YBlr@fYu z^VdF>LT}W&Udi3atMeBUDWb7n)oq|_z_KPthVuB+jr;3SpSr%HmWWr}Ukx*AdVlHy zjloOOg-g?Wk^0~ll6*@h+gQGzr490#&rwA5ELba|4#^iHbjG-^fz`kvVU?CGgMi%P zpD!%ET?g~*QkL-M=4@t}+SDP)x7p#YBxi`$ka_VA!*VdMBx_MI(xlrMy?nfc>qXVF zK$|PD9-l>G?`vV<4=I*i_xRhFklw5Gu6WI{+PK@`&co&HK)aYPgXbx;XWCi4+2nQT z5hA3lPf=-2SNMPS4l{X#CyKJUyDwSTp=U+alNzGpZ3R&!QP8vpaYBz!z?$Yrkt63$ ziQ^7qeN%6BkW!+tANIS5M+V3V5y0@z=|HoJ#!huadhjya+YWM^au5G9(=z6B?4kS@ z5x#A(;H9EkMOmfqy9b+d{o~TckQMaEBUV8DJ4)iscwkS8$%mwll}@=q9}KEhf0-8^ zwoDY*NMzQe><M#ILDEgo8+V(=xV0qMP=8<f5}6=X7}@scZg-|9xt-%aWG_mqQXK4u z$)7+N;~_ykOy(;_-`ZxJ(8Sjw+E1jGbuZi(3b8i!+)mwH=~8@Aiy~AS1cb*d`Z!vb zJy~hi_2+w4rse`CHa&h$3>?f}p|avF$7&O9EkA3|_NMmU)C43JO+HgDBi~EoQ+$09 zUhtFFyuL6vd4ZGM!9%cVj6LCYDg>1>JKeF0SZIHkm+NxT>U4iaC&MP0(8Q*i@XNO( zWv>;F*42ohN8s@n#q*A%Q2ja23$826h<Lo_fW(z_G9guBML1p?EhVfbEU=7B&!AL1 zuuR@Hn6mKp)bge%Q?ZInNC5l#kdm+Y@;Qyi5?|uSrmu`O(e{D3C(};1qTdcW?i0=R z6#vx9mFO(iykvlvzIZ=I_^p^bacXY&>-5ic4YV=@+ww;B@X(Nj(ZGt53)!sc`87K0 zq+^x@={L$&Z?ruKJ#~bBj(?qWKPPw{4E)T&or5D1xll~AGHA8iav;x2tCy=%_>m0d z)}fU7qtQyQDlQrIisnj=klLCQW65U+e*O2CTe4XS96!z_9SF1E)Ytf}cgu9Ro1WbR zDidyR2HM(!=8naf>0tOu#YI0I%>PY&*h(M?^VJk$F`+76>1Ujv!4kvVoUg0x^FO}X z?$`}Ba`7C`cbm}J@$~Ca`O{8cCd$v!Ul;}(2O9|{X1Z#5#;VJc<iGx}=t%w?SBC>m z*A)m0w?Xo?bM-Em?XOPt&yXCHfMu<Vr{_paXc?O^A-oSFmq(kO=B*d~#V{FGxIXtt zl#N{K_AIbl^eeP<v>?aeg}eJZIPUO}%;QYd?0P~6FUn(N+`#}SYIy|kI8;~Rz-V{@ zjtd@+xE;3NLZCYUt1(RIHYTe<EbYba#PNI($?zbdt5bQ1e=Sy&Ns3>^rze~{qZx)< z=4s9T8S0$<nN#C-ubWLciZ8yrPJIwZAq2W>u1BAU?ZSi|qQNVB%-FcL2<@y!ESG>T zHY8gobPUf!;7MX!7>MOGE1mwR&SuNDE_M|XyycC<{$Jn2I9lh3Mz09HtPa#+%`ZGX zg~}kT>8c@$u1$5EI=9LjWwULJDNYU+YW;8=q2_tUobb(YeV+pP<0mQF@j{D_Tx46n zNENa!%CtFxPSO{(!}B&%1w%EPs<>EYJQFz3W;&dVrulhgLTSg|!){He!bDyQIf86` zEMJz?hFO=7F~#8J8(2#wq`&qAmV<i+5?ro0_8X4&uj(mAt}wSU5gSQd)xX<|ya+hc zZfvH5&4qd{$q;zU*u)QGXpk>N{g3wEJF2N|-5ZV~qSBFGl_E+EU5YdtT|_{7iEL1s z5PA(nKzb7pP?RD?nt;^MLq|kFsZvAlC7}ig;ajlJxyN(7-+TADcf5Dp_n!Snma@il z;dnCFob!2}U)d7=as=YD_%uT>6UK41b?4b^xFebk*am~(_da>Qx8;qyqMY7ED$j?( ztx2@aM+to2f?iybW0mC`woDdgXQ>r7RG%$>LpNg7G&`w({{<C;#PmUb?=X%<uM`(O z>8;y>qFHq@R_N#!bEW)n+v@tl=X<`cvU(Kf_Z|ohmjvcQwh1T-luU$0x|K5l#h$|1 z$H53IlW}ThW(bnzaLcpt)ZQwuyQdq%sz$@=QLM5=FVl93_v-_4XCE@`3OgiBr+2M% zIiCgE^I3IFY$4Ec_Zr`qrl7k+LEB6-KBq^ADH-sZA!6Ru1e)_ExA7+W+yav?x0dgZ z6M7A1%I}lWkFzMjF0h~ZVEmcoFu&%I&n;B_BXBB<LGKfyoq8ORim=u;h6#vgTNAW+ zYALZG#I|5q-h;C&q6^J`XVCDr6rcw0-&^KI84h=|oqW_|YrD0aq-EHTa5Md25J!F) zM5D@12U=>wJ8bT$<C<6kyYZ+a(CR{=M&G%T!*o0k&m#Lh0((4Ic$88?h@YhHOV6OP z2QVThR))m~wXcoR%EPEPvqYFRYvG)l$2lr6^TFx<tIAf8Mk7U0$<~G02P`DnZ5+wM z+X5j~AJVgzbdpbFT;@Z}dhN*b`$tS#EI@HS8(gxg(P*N5W^_OPcIpUR;aln3h^t1p zz@=d;$Xd_2p?p(~O*==o+@c9v65f1!6UIKc6ZRo`sR|l;8B^y=H&gz}VB`l+OLJ%t zaT0n>Xon-nGi-cd<p?B&F!^nGcLQH|=)o4%RLI^meg1|aI6`<?X`nPm2RuR=!`P}v zlc{I_#ol9jqe13DiG@qH<#0f93f%diI+*=xK_g7Ch>?PUi`!)h+){UHxDAFin{p{N zC>zM1QMO_+8I$nfu`giZ21z`=)piS1=<Znlx#x2|XLym>bCMj<nb=V89=xxKuuX~d z@&laYgttfoYAEqTM3za_NVH2%_vx?@7A6mPQ@qx@WR}pN9T!w>pDgvFBeWY*AE}>; zOoY~QpXcJcxLn^hsn1+j=7IHFvL$U2xKZer_OdP4=A7%>Q5QdSzdBa#zL)p9LC=b< z^P$+tgWgnC8!5{HiL~2|eBq#LAI4_(k3hOdAc2&Zi3ak{1b}-bAZ)M+3rgU?B;XaW z;vrY0R<=S6@AWbiuQnv#Uh6Ei`0ikmXgV>8QrUnF#X8GTtWCcQtO?So37w&!d^2Z0 zzm>@mvd6oZxhK9G38n1xQ6pFpC$x+~_4>;?MVe&KRrMD-d*3lsO+qfx+R(ha{x(TL zKNvRlTxn!5aRiHKmRu>D$E*j`$hb_YQ{AQMc7dH)JR_Eic2*X_Fm4kuaaOJmZTl^y z(7d%A9lui9zB7$ibCjj}UZa03v=VKTh8u%Rx>mhRR(0)pUu~6UdXPmnLv?!Mu|s$5 z)#{BxFo6`merTns<LKjxhwdSaz115`qzR(~{LT=VV%0ZkMHp4m3ctDjndplVSf!eq z{taC33t;LvRJQeSU3EInrUB_$SC%>6VU1CAkCvakTpu^<av>mc;^Lg#wYdn6`l_<5 z7II2$ta|fXW2kl+^3CJkuD8S)C3n8ueatnJEg?DV>6#6|IfiU?4RHy~wBmVFQvQ5L zAo@n1JKn_{+qo)7ASZkeuoUja7W?FFAO=TWWiQ}DOsBvyW(NvA(|7x}pnK<kj8j*Q z+CpjsCtp5Sg26s`L}L}3t6Kc5>u7NfIo6&;v*|T&9|jI9KSI#lacpLHSk^NLtgGT} z<6y%D4#<`2FS0Y;^<cGN$s(_%O)*YABsRwEq)ZnwxjPOv-#h|EwT5MyjF2{&W&CJP zzAx`aiLTS5%1W-tvBS=7?AZ-Y+eBG(0MNH$L_lX=VX&uIHO5H`Yi{tcPZXDmkh@=d zq591OX7SIa`|N$4h$-)^7tT=PE>TJL5#ta|u#F;1s;p+QLMQ`yj5f&_QN9PvPfYpX ziu)I(%owGwgHKHE#+z2!>+L5lN^3B$;mB*wvcpM8h8a7hymp51DSJ28EIQ56b_jfe z*>~M3;_;|SnY%{V`Jktm^`SYc2FnCx56Jgtw&y>cs)!BuVx1rx;2CUzxT%D=P}1Y& z`bA+BB3B9D8KR9uZC2X87!&KH1rcixKI+DyE$T@QRiX0XDup2+hC^3IP&L19lx zE4G`CH@Jp$`t3U95<TG(uKM)CM*XsLFV1H`0-q*0<UMWKupF(jFX`&KF(ha5A--P` zwO3Mv?T)%$BVhA^)JuQcTV2OH(Iw&y@fwA&A@B3&RI09>9waRS4AzO|MHA%U<~qh| z%ua3{XK7`mc!lcgr@=y+I*+F9uUBfGVhJ)B?k)YX#SzXywrY1n$%ReA-BneJbONM2 ztI@%Ko<o?fjrT_WfxNO6-Fj2Dxr=!Il%Z}2BJ8o8(7t)$9N+X43DmleCUOz6G*hc( z0!>8WiM*z5+9#JG0&QY^Xq#_#v_{0sH{N}u(d#bMHeJxYu@ENeCKjzuN)oC~5iaE? zOzyzJ#v9>7<&NwGUHcTR;H^}D=wK*E5#xd(34CAkxl&E+hm!j43l;MFt}&}H0PQ-9 zE9!8WM>L}6_gG#%sypTmvL9bS@+X{zzewA%>vj&7{iQ7McXS&Mj#c6#5IZo<01pFL z?-A%#-0&dV!JW49k;%O-#8lzQtsjH(Y;OW=zX4~$zehI#XTDSOP$s$Q>B;rOh<&$B z&z*|nibg2hjHwdmE(3>VA;b%xoiJRD%6zy!BsBiY`FUz756F%P5NJ4XmXsF|jU0!u zhlPjH4kZU#UpK_oY=<@o&!#iIP{#;B6Q_KfjZW-zN|3ZXals*b@BQR)n<$~P?Tkrz zrAcjt4~105@ji%sRoXg|jU!M`bR!Pk0@NZ)Syts#u`=iv&N5QkuoC|2Col)T+m~6d zjibTrEeS9b2{Vd%1vRNj=#H`SVXCMQSyMH2MJ?;kA;y&i-G3uBBZ-upzOgtiiP??{ zuK@$1IT;?v>hNsS_J}sV=}}hZ%_O3RWG<{?Bk2){2)Dsy<6#XClTm)6d~YF0NHyS@ zpWrESKLx3I(v`BsP2G|Gae{n4cgj`@VtHG_mHnJ7<IOv87HmC&_i-(7GIsaQ@{t?x z7};IxmO(tYw#%@;8;obbG&Pz)!Zb?~^!wP+T86JIpC+H9xKkkkduJ=T9zN)1n*%o~ z?d?@WoL^+nZRdDZwZ4;<U(N2P5M*}*awGv96H4L66EX-LjH};jfxN@C8vP{faUV~B zS*U)X8yDZH7nfEiES{@-B5QYbq5dm(3=H#AX3>390Onae$b_w@j$v%6j;$Bs?3H&V zgkwc0SiV_}^P2MMX@r#C#n?EyO5kf$nQu4(?XJshsSGk4+*+!W8}MNSBs^d^pSGr9 z<B#9oMj)4+cO&Nb3hGmz-A_Ubb)T-~ok9d`*FM2>O$l63w3RKH+`Hdy<#46BBH|l3 z)y}tqkvCF}^j^k}<{`B@a#ug@;GQ?Ie4}<m4G=5+UvPua;=VcwYgZp`3@TV!E7MY^ z2vU&rLGQ4DJ5ipjr-SM7`G#SU=!X0#<^?%wILv$EV0j&A|L9cNl@Nceg1ccyRkA4z zEMjMkK!Vhc52hEunOH<Xjq-bUJPuv|ZP@yvu_5F`R;H2ajCZE)+D#=#x>nG9IK@_c zJj(3|l#wU5yhS>`vx1UAP%di)^uWx*rcwf3IEdiVSXf|cvX(_&#;{OXY5Z9Z@TWVE zXziSQw{!PP>Fda`-C?jg9vt(0vw=^hryazltXQ?rkIdBUF?y*nQi}a1btphfN5ka@ zYZT4g@~w7!XuswAGp*3fNsWZHa)Fpmuij(xF%q}KKdFMtV<M|RMsTPC6R1PJk3V9p zc+|S_{E8c8qIAbgGm^D{MTLRAy`A3egO*wXv~YXZRRk|&IuW++pUjQKB+c&(1eDLs zLJA)#U7z`($C-YWk2a#?<pYm1gKSpr$rOii5Em$=L6ceDlD)j`joA#olDb8ewd;cV ze(|8#m+Ewj?5A;FxduPnyADha*pl<7{90D8w_Dokn(>8WE9-vK^KJjgLF#_7nHkjx zMh6iDzsD-8D(QeB55Uby7B7{$iqDulI?dqUcd_SH)xIL8c5rcybSiAM6F1z24(Sxc z2w^I}_fE7$`tL~*@8Y@AR}Ha*k}fpL)nj?zLDPmgE4VD`MGB>~<dk#YGXa-cebKRd zqpjb%d8(wmC+zEs1KQWS^IglU=37S8Ziu>J;-g)BqW4q_k~DP2qHc5DM*4WY6gmW} zx73{-lvbW#yV5mp6p3J1)N0E%T++N#)H--ox#7{n(~nmdlL0p*@8h>*Q2jRdN_<!M z#`=;2w3M<5O7;VPwXd+0jlDy)ruRG3lhW{eJs-BoV(4ZYSn)$ML?%O%&^?e9VtbK+ z@EqJC%d?24gL_S=-Mxn(wc1yPUT-hajC_jXTXJocpe|6Dt}WS^s%^T5jA@hq6zbAk zG}6XWph!l0T3Gf~x|hnK!Qma~({V%=&|o6-Kr|Qb2>|SW#8{u)*26-R6L=;=OQ%2! z4g4-fF$4~oTwg8w9>3h>zglEy{y8^STyj78)S}K6EN_dk^|+xF$0@6Q;%0#4=AEzT z#Hvq#L>DO2_U_UcF7HJ`+3?X_aG3*AeK=IyGko79(j_mNXN7|LJqyX}&&^bnStR9> z1)ihs0uM2-hAm#yU9_(Hdir||#9&!H^v8X%BT)W<>SPJ}66Is?_d;13K0J_BFqvi3 zqiVE7k;Q(}2QiZ!vcmTIx7KcLf5dgSKn$B;lQ}h#nU+lD2dq^c)}%LSM@mZ{UK(YL zd>eea<dydzU2V*EarqIb7ow`ozlu_=PPWLGr_VWHvhobfq}A0O2<JprydhG)a9Hw` zI`~UOE<<GMDU&knU1f|;EN1h5+2H-e2QQEsDxzb2JmJuezI$fc9cQIq3iWbhjcOB` z3aQ6@zIIRK<teSYd^S#$7&m_sYulZO5{@}L*(R~uVuQ=Nhs_ETg_7Fjq#7(H^I%r7 zNzQNI&Lm2`-g&MR?W1iLDbky4<5~Pb6RFR;(Ys&tN=w<}3Q7uI(+$C)WaH#~Q#3Gx zjfZ!e)Sfty<>Ylgp-;x72!ih9jPG^HlqC0W2PU?xPCk`<8cySl0XO<wUEsLr!{dHs zs!Q#3QxmqaL$J3iH9=&&3Q0OgUX<n=KNF+G1|AbFx^7^bZq#a)(texU+)txCJlyYg z!?eF<9eoO-mjD`HEI{XY{-Z}m^iQQ+aF?tgE>Gk8WN(e+q|V)`B#%TEH@o=gQwJS@ zilnpi?)<yQ&sC=jL&f&_3*a)7`C<ra_tX#57SHMxpGh>SU!El4ima>nT<~NI9kNg0 z0f^*J-YY2LJr)LTV^j3G_e!E3h!~Uj;(4;%<rL{XJ>T%Y+)L1Pbb|C{<yE_I$GER1 zF4&}r#*EI#>3-XYkehs3nO?gx%+XA)IXt)FT2*0X%lYO>Tvhe8HG4jNyz+DkF_A!6 zX@@sLUcO_A`6gX5)y>kE$>-W6jX)$<YE?pe<434Fy}vM1lw>XMKVpuLa$y^LHGS>6 zMbvvXz*}ubdw)ZYR!^?+2t<Ww)P1j5(!>z+(Q`b7QR?#bTMR<k3ktU`x!@gTtgq2n z#^@!P02Xs!92wbPFkf6~C}_(5N3``{QP=;`pEZN^h-zF5R#;Lpt}BVNXxh<(3`8{Y z!D}#6HvyNcj=Sf>zc9|3Fw^5wjacz&7YT^8A9?S(e>l1W>g!17lhIfLFbEo#<!K+D zG9qbQX55BS<MolQ&c)TJ$bLUoD!5;PqDC^JwF*UR6Ys-<IV2Qi*Nqq|g-0y46-PPR zlT_xWoDv1s3e>B*<h9LO@ArSLs{J8zSH&^);KfLrS+t&RiP6XLh~7Cx4S<PBY2dEa zbkip&OIf-+USrRpW`*f<yYx%Y`T|(4Wt^y2T0alyT=loCMrLKG%@2<a-OTXsXL0<* z=M|0+Nv{6gZ{g%n`r8;<WVD|<fp5et28(J+y4W&Uz8}zmUL5W$eifwE%THeWMPS#B zB0aW`)qFZ9TbdU$qJSw+zmkh&{!o~&1hg;9Y|1MusDb$?R8O=(?3(O$AeYay=>uqW zLBv=Fo#rWnl8Jk3w5Pg*cOA!<s!XzAQ|G55k3cHbHXZ}Qj#^xoa-=8xP5LKb#0vzf zqRO%!&IIpXCjNq~mmM-qDN}CJv4bN()xYO&T*{2N^p2UAGkY1cb3=e69432<1E(V} z0WDif8KM+__jHqMbJwHJVtfe61SFza^M=V{7@er8PX3-j?Z_uFoz3wHt{bnjkoK}0 z5)1C^qZASaKOf7*b3av~_xq-5E&^&~Dww3Pe!RFra!#I6YNWBEcBME2nSYwyB~V>G z;<CO7ob6lYz`8)`p>DIMw0JR7!HRRIa{g)o>*3UwvmQH~wE{ndoATk088{1Oq=`T_ zxJ|Cr6s`7D;Bn}A2%Yb94{t^?yRkv;qAum-PFoyGKB(D;1@(Q+j^8%5HoS?u{44#f ztb(4~L|l<c%jze6t=~EO<i`4b-^bWxKHw=!dREd(df_h92U1FbTz+U>>NK+X0P5$G zHx_)(Vb<|nH_h~pK+kNL>&&fWzvmfyr<uNkSAUY#h$N}ZU;i6VYOTdw)dZS4Ebd3R zx_r~App&D824OiQE12V+7c#}$|C~bR7EEOe^*63nJn)4MZkOzlFMV_*P+|gt0<H|q z_I{{}F0+~M<J}xm(&xxFy4r|E^Nwe`S5KCQP1Wi^G?mLBV1rOO$PLJqg%P7?ghkVr zihazh_<xfd&@3OnTSu_qnJ5f05O}dZ-<O*3u&?ax3u@JzrGzVUvo8@8)Ag*bM5$Ya z95~nnM5@JLuo*mG*T0pdNl9aTCt+W0+E8sa=KLLn+eVw83)A7pW&kI$1-;7%u+MF8 z$ULGO-=*HfPW(+;{QbC<4*gSAuc=n-no$SOchf*&q1li#2Tn#^2^jUJ`0gFoz8Xb` z_x_d6%4OnW;4X*Ra2k=W^RnAZf0K>LUUqOUYO(@O0b6Wx4F-Uyoh>gaOljasj1{#p z(QOoyUfq3S7KKyfI@{_XD_#G_D~4g;0aQsxp8l%y;pC$W(%lKkw9X^OZhvzRx{2<^ zjI3Q$CddJC_K2M#XLtYzMLK0J>qL+JFybHLd};4}PUberOGVBKu)3cCkuocDV9}Jf zx>9gplyYr9yQ=)KkH#mE68nq9ZMnaF(Ga$_sv`es4r0q>{_pR5U2HTtkxya<R9%IX zpDphih;cYabqrUu%bs`lE)eY)0p_?te2%FH-Kg8{?l`=ygZtyX;t9pr7~0eBbG3>; zP0I5_V?_OWo0OqDt5cndiWr%418`-`fwpN6SmNy^j=NmX(-+3$7R$d}ypT#v1fpn6 z`bq91APu>FsK;I~Q~Jo{cr{@^zWjUX^S_g1d+4rS1TyNB<AmD&ID0M|)CCgi_NtLd zT&vt)6KbIgnNc9J#PRr$*V<!LP_%*7K&L~t@ogJBl&Aj8x%4G<y0g*wR-Rwgu0-YN zLtAS^47y@`xB1jO+?rOIyQl8$)(K;cNV6MwzJ9WD)#ZZIW4LU)<K3?}v&K-W3}P*s zz*;hqc@q-Pq~)eaCF5fBxHe4VrSXfLsZFP4YD)}0Y}vy0U-qaKWUtTmJ$v7G=~Mj1 zu+(B-jm|1x2)9OG?mRaIF0EUmxHfr?MXJLlBp=$5#2IIt23xBjnU!s=8EFBQwt$0) zh#zBzDQ@qayLt_iNwE}fz4G0=Bud|^fe-jMJF2Sgq>jGKcmus4c#9-Q&OjrnADuF> zqN?M%h1c#H58afg<6ge`?kT{Nl|9>6W(=Zl!B$0*#ND1v3#hYTPZBX;MmOh?LIa&} z9V+Es2kzxL<<a;A9pN+Pb}Tm6q{G>h_p-WTs+s+5x!qn3v2jKuD?E~Cy50r?SV=B` z=#SciUNto*0#$?>9#UrRt+-14(U%e%EQ^IYMM_Q;p)9?vccvSL$*L4mb~^bsmc}Pc zNaMmSE~iJC&a8DgeKSjB&EL%gRFt8D$Qkt5Aw23?S_xdzBOARiuzM4bU5xcPsOgX^ zy-qM;^xZ5ZxkS`f09MJ#qwzqua^MbdKyqjnn1mm%{jj4qiL`p^5-B>H)CFSC7!1P0 zlrx~GGE2wrtxk0iM;iO$%VnRkSWQf9hHj$6InK+1ZA8=@uGA6uO3oCIWTaStyDfz; z685+UA<*E(OiXgow0e}!SxomET2FuL-LSUUtW%e|Yu=pSqw&tG#d%NBVJ%*4b-F+z za?y9^f>qbAiiBGb$;f~=-Sm1~*yy)cZ-kNNv1BgOeN|(jtPkBknB;hYN4Q{mn+84Q znhRGt&fEG1b|0J_hw(W27_T^@we%A??mar@TNYjeQcSri#33CDU%raK<!WP~&5IP2 zEhxku{qfJBt^Je!?JOGOhrbVqej|@S@%_MsPHFrc?k(eI9CGyt#5q<7+_}YGbfN=O zlSQBc8g~E=MhBo`kCYvOrgC%}5EnBf<meBNKr@ENce07{vAZu;;IPTCBvS48ITqDG z1&i`0`ll4h1of4cNURrDzTL}Xp1<&3!8`9OVv-p5<_IJYI|4!SU<AJC8fvVVP&5`c zLXfY968IwH$-DJ_X9a5kD<Goz2(%AI?AZcPI_#so2~<cAD@T|Q@V|ka+&Q&)f>r@{ zp#FpBK9~9J<Xr8cj=u~b?v6kU`9P@-X#>ni%|br{uadnF<lqSO`Q*++lB^RrTp>RY zRE8;wlc46KaJdQ?#pWNaiY+MEUxK<F8i}ZoEt{3Y1mV^H64a9rs??F<k}fykkn4X7 zYRd1TCe;KgXh_OQNQZ$mX)1@Mf#aeX69@ev>aP(UIB7%=z5)PI99TI*D4JmCio$F2 z;RL3fgGi>(D16xQ&md3l6Ryj~I_|d56~uQ#090r_as+sKz|O$c!p-4^JAj0+-5_u$ zmGb{mRBIHQ6b}&7;*l<k_2#_>5Sni3*ZmC6sJHJ1vVIA*t_((jC65Kgwz!`^Ke8?1 zvX?xy(#&bq!3jSgxj=CqWG~`tpot64!6c{1)9Pw`ImYom8W273HdLwUEqo|b)(0UA zRn#{1$1jKAE_%zfHFem42heUA-C}RR7Jk1uGH_tn(lLd%3vb|P?^P*=+P#drvfS*} zdyi!7oc=A+)rK6ewD`76pl*6EfL7xc@jv@{;+}Qu6IRyWEyEhKSMGMLEa*;F_7j<4 z39dzhQcb2=SvIWgt}mbs-`%YLJm`vyl!@8F+YB~(9XZEaH)99cypXx_oN6@6q#vUz z$UbIW_FW^_?b}+G(fML1^I(9qT3Lae>E%siAM@rWCJiF2fFN;CFv4uas1K#3bd_@` zq2;>bvpq$6@}!`HqTQ5M#n5HlPx`|Ok}*21>ZxqXJFJ6baO&w0S!KeNaYho5VnzeY zbe$1v?D`|w(>Ml2EP3W`aLq#`^G=~{u92ak(UOzsBv-_9qsvlG@7_&E{>XW?=Sj!t z<N~owG<lD;F)Hcb&SQy?t1sujQc_&GXWlnUG8=zym2rg<YX3lH0Dy$NI;nA==;P(R zDIXj97Nhd{7o*3!zpBYT=Q(-Lf0BoJ{=a-s+V4F16SDkizT@Tir`TnmHaK2OO8=HE zEf9SM#4`kNyEp`}Y5An!GYYyM(cD`=yZCD11~3286ByWv>%HGt5W#MYIsK@&=7#sA zVc_|k1-Fvh*A)Y=cIk5y9b~XmmboCPYM?d~C`z$Rr;A2kdbkA*(oN@GFWExj<^(#D zFzHiixP*scn(xN-tg-yfeWJ^O$aexCCK~|ZMNg6@HU^dc)P9&<nXQ`^uzhH_d^CO= zA9VQrdHQmAXz#cc?=B!k{TYsQ#FGh7OQ~Yf^~K6eAtP#4Ps88E@u-t58s*A6h(%#r zr372eoUWYS<}x$)(JwUZlZHI;o+cM)j_Gg|C|xl3<JN&f;#w&t!($iyu1QNxs*;mv z4#I5BE8^9Dn_d+s6#?^=pKE-6mPBS#zsk0y%_c{nP8a(8nt}3;)nR*jXARNhI<vLO zh41{Oun;R^0EIn!Cy}l?_|`+|+L20dIO}GMse4-!7KfOyh)7qbF?9D+O_I0cO&sSA zmlukcS<q&BF3H3dZjtGE2H1*9Z##)t859OP+ep5yDQC7z{Wel5<M>*lJN1%IT_$Gp zb~T1InHm{rNPc-|#^4i;xx}-z$3xta%ALkal^^y|Qrq`)9Us2(x^f#!eFx7vsD0~# zs0WA$xt7;e1MG&}fV|cApxP1Wo6XR6KFx?OW^6pfaTAi+Sz=J#ddoi3HRk*o3-<ol zY#f79ADrBFQnkc+-@6oO_@vz`xfaD@B*-|{_CY3hKF`)I?BMNF2OKT*Y2CJ2Zd!@- zjSPgTC1!E@Vc!jEYO|fBJFuHi+j-}nkxfgm_h;Q#OoE#O?awHnelNa=?pz+8IELh2 z!-hYdpPbmn`WzpZN=}Lcj{UgEMCf^S<8P&fRQfDxP7g}^AW`^|$`Es+QH%Gxw23;g zX74Qrs-DK$K|b0BeooWj=G?G9&%K*h_it5zWsy67VDkTw7v%rVG1wHU`35KpgT}iv z-{Hn&uH6E+*;ekJqdaJ@-<I4ev^icMz@)9?biLAslhw5>24S7$6|BM-ti-JO-63C~ zo7KSi&N!*Lzdy88H({|3+=oBqMj2FhMeGts^(X9wJVyRLXPRVJ<;cp3iFZdJ>a7ls zsY>4~<#9ZLw=20L{K1qD6*Y_<V&dDbr%hM#(CP|ItFu{}P`x16T#?!G-=!C%6rOVQ zB);9v8~%>c)Aon5kwx?R#i$E7g{3tr^FE7~{1@Qe%nBc>Q)rUd6+T$tIIHYMjBIqe zl6Dp{NThg!?4zoj4B@Htpva^Eg*AZ_%PlK}{dyiIeL*2EgQ)V^BjBW>xbj3Ui>;n? z(K)_g+7k?H#gCMJhOGUwf7j(Y5Q+eR+mjVXpp>^Gl}^iVejrrPSYk}i7MlwGC2@Xf zYz$&Z6Wx<T=&thU@gaBD5hD>Ee(RsIZ@qd2fn91u-u0AGSNz9q>66(1@_p$Sq4z(y zHK(PW(27qkzwTbYzMMo~P?jmUt&{C6Fa@7kP*t%$%&(Tn@B;SkH&V)rKaB5>2b1Q= zj(#d=t-8AM6>R4hdNn`%@8mLo!Y!o~4nRqBASg}}deqy&a&<tovPN~P{Xn4>Gu0yb z2Cr`83|cO}^mWwxT;a15z16=oyML_@Jhl9_0saSj^}jys50)hUANq}&eb@6{SR`$4 zkXqX1KVdNsG^dD^IVa7$>ET1-Ze?czrC4IWgz1pHzMQ@Up^@Cx85LR)GRga?cJo~J zDW=dLwmpVusZu~r!x#j=mtA}oLh*if?^AK$og>h-b}q?QVo${vbsRX0#+cH5u{X>n za_H(e;U(gQUCq(SxcZOP?+-%`?kt^;C?|0Dm{qbzIM}%Y6=t6<o4SHTSL^v3s#%ZI zLcv;sX`xp^|Ah}x0B+XDdHn@}=M@cEJlcYBeNHfn1GnuN6tp#&;clkXdkdsL+ywI- z`Hh$WfT8II56p}(askgrZ^#(i@v5&IjzkIHLSWHfYD-S^Q=PPZu)WHbzfKkNKj$4_ z@%;abgZa<M&XR~vqsR?#O^wVS{NXB1Ui9@7=B*Qfig?CrcyPBYt9$X3vSFhSe@j+a zX+QPn7*^6g4p;Y(gNEnrZEy^D=%mU#3n1}jh+1_V6btqDiW{9^E*BZ+eXC|!9c7bb za^LgP-%tM<kX8Z1>BvHg566SBa@v8T_IcS>nDGkswPqUc#dgf*6c;O^aiHz6BOZ;{ zwq1>mDz<>hJ0LIxvYd$Ty1?e=c$}txKs6Nxhl&7u7F2GR-w=U)bXa<KNTmGu>AGlY zxrIE$Y_%;=qR@(+oNrc;0Lpv$Q3nYSpb{Z-JP?^yfx_u>91jGF8I>m^p_7Bn^2de1 z!jzvzmgfSvQ;UqnO|-SHkik%W^-R3Za4z^IFwP502J|%O^m3MwO#>6tKh~!57ZGPm z5Atv@X{-FDI=+ekWE=TrK-t};OC};fxkcqc@xl$|ZQ(jrLu2>NHt=-Eh==VKM3$lK z+i!1NYesM}GydNM_Y|}DomK?_QelSblL$Jl#<Xz0?0TYaT1CgA!8+t)V{dt${PvG2 z&mvjq|A`Frr(Y{*gEGqIFxFwBo1QYxIYQsG0<mT|O38l*!Gm9}$di@EG@F1iu0`*= zbZu9w6+Sk-@X1E|6UznILrKcoUC^M#Zm+n$k4<_eT;8*ZT%Xo<FP#Of8g;gycbW5Y zhJm_B<@0~`;Pv<8wFb(--W-axeT6KxWf=m}L5Fc?Qf6;;i8QGmq5$|2^6`2ZX8cur z*aAPV?Tr8OCn}qv$H)1G#evL10{!@D=79{MT=&o^HG}C$zq_yNcZN!RM;r>(e#fk7 zy1Ct)`uOqJ9rW@~ZSvn89R9T>2EJ>Ys8pR?{!}^qkCZ9?PbALY#q|H@#n3&R8qvsv zt=8+AVK+&bDkbz5%g`EMYF;E@84+J<vrd|Cfc)v?<VYcvyC~YS0}1^$%bc8CZ=Mhp zA2ljGV^-Pn2nw#qDEMfnq0de=m|vqlnIxrOBv(=MLK!LfV~k0?;<4hS!S;Y7ke=PY zeT-)Kr#^I1Q)p@{0^K+MGH(+o|1j}_>{hrZXXJ{y_1zo%c9+YLF<v=lf#+K1V-zmr z$|y^pGC{au!y0WI2lkDS`=g7ygQ_JSI|K<3;mm+Z?8KpZ+opU0TuC>y<<pJ{d!&Ni zdz1HBt>Se;ZFbY2wpmEQ#<Km8kC6A}$lC#-wCD1+h(ZmD7NVqqjjr}XBdN74Gf>Vr zH`C#gQJf83w^yf&nugLX%F*OItDkeftEoCT2&0BEU(YifRt({JdggQ%2t)xWPW)CN z1%Os5w$!OVBeSD2mmQwuZ>ti`bhEf}7l>1ut^0|nsHo{v@LwC<&T}_I`bj*Nj|5Z* zIm3w`3W73DFkgTHE!)y3o3@5z8L9#ltIv}-JmR{+yStK)g?ulqp)^V8mL!ic63fwf zY|5j5Zi7F0fBeh`{_7FoPs{CpJ<DxM%HI6$CRhlPoi%m_(;YnHHOA9DhOAtMpgzew z@+RR<+)DJHN(kV@qF(SjccLv&5dob6-z03jUd^(esJIkBCsRT*u>8T;C5^+k&CE`R zv0A03fuT<(-XUCyUV6H)<)ytUOc%^!&w|R+>a?EVxBi(G&&1G$5sPe9D6ME(sH2XQ z9OnAc;dS-P#pNOe#rN#nOWJcNVmUZLvIeo;>z;^x;Iomta^a&Cb@#mHcJVbAi+}4( zu|iu?0Fz;N{WZ}!A)7=k^1Ba`6u-tQZrZYUhMO&T>AGO%n;me-hGE%i=3MXcA#-P~ zmod&R><#{ujqeNOcAm*9`nrJSiM|vK6xx~*`IahIN5NhX$~-9%&)BDknhs2H@<5zT zm_V~yhdpuP$td|@Wg~6!8!Hu)`98b7y}U_}kHA7=V39?}Qk8mejii_R0YulZxARj{ zEnTsKN1Mc4O~=G;iyR4#N7^*Y{jJUc-Ao;4O%m6EXwQ@L4=FUxEoIfY=c@<TiS!Wo zhpl9paR~B(JVzHsex5$!k3~(7gB`n!`|W>BoEqe<JN1QmW4pF+vy3opwb(g}XoA)3 zR-H7g=S`%r?BxW(UO3&v5ajXzf4;%~5@AV%Q-36InDL5_8>F5j%A}G>)0!ZJb1Ja3 znkIMbZLXp(bS*!NeWugprH?)|(LS$p*N@Tx-!m|dz@U?#Z^`(bw1eH-g5kQ;HmC^Z zs29>AZnnhfRz*hV*B<NBbfhHT0ulefoQ`jk%maKR;kuJi!OYMXcv#NxvTM~>P1%&( z7HdcbO{?(tcIKptwmUMl?hoMuo2W^|21zC!RoJ)atj_ksM#tnzMej!!$+x#F6MNk0 z>8vbbgkm@!wz+zeO~e`t1Qi03v(D83LcG{`$UE&a@6bBe#$1?O<8o{uTZ_9}(?Lf* zxDn0l!X^v5veB*EcO_8meR);eOM6F3u8RC2!etv*jte!^E>AC^u&@vvQcGqRQpV^( z2sn4zcR{G{VN%al)cBW}67NERO7`yG`}%q6Rk)x@bvnzPeL`v-<ogZf`y!pTnR_%= z-Qjs00+-s7NyU#qg<37=rw`ql6PR6nT`VS(E{xN;^k@1>wHKJnXmjzO&R$R@oBv?7 zPviDGC%p|$I+a581FsWXPv9IZU8iZ|@vRNhDEGY-BS!Ae8vG)AdwaoauxNFoayj|x zLLx=EManXYK~zm#zq<FCXg8HjufA-@kf^szfmx~Dl)`6FB(6))CJ%!S-YTS95ZkZw zcSY)mTr8`<<R`&bzj00`!YN_X3QY#T*)fSCwcZakGHMqww8|Z%VbY_wVoNIVyT#PL zBnVD9e3$MAyRa<Y?)Gf=W_-Sbx04mfbcczB$v5u#<0jS_%4d<5fjL><eQqM4m`R8% zfPT^HJ7JdpE`w<HWv^=I4{1i6w?3wj_x#rdV*ki<{uGh@*AtPgNtkL2T6cyR%U;GA z5J6Vo`Ir=%r)(Tr-b5Zc-x8)w-Dv=L2gi6mzdF74y^_{zS+=+H-aYWo)3+Ix`H+*@ zRWf9(z#Y-%u;6`8@{H@HP?U+`6zO>aCC<+lsurm1!?C1gS6dcNfAi5-hq+XajnB!c z&UoLg5yN1+)!!=1;;tay5VdjJRvIgPz7&}ny7Xc%x<g{4*x=Y~;h(9z77F7mk3d5l zR5jj{L!SyEn1NIP7poVWJ$`FT2gKoI^47!)d-rZ%T3xTS*<|p&;qh>$C*=z;TXl5U zk|cONIg3tsO>@PQjxGcMKwn(+;r}+#?`-IMordy_I^{(-MYiB0&~hZ$?zLa&)eoG> zW!Vks3%gZu)5OTqfccm69DIK28y{1KHq|P62{el$CK1S+vF$lGeV<Fu^8-Dk=W%io z4#W6PtXA`MBQtm%)BmDftQH=z(Oy;zc^H`y-M$+2jKeK}>fXyH<`!COI&Fp1zV#qi zx=tNkS4H`&L&nKc2jXOlvNuSw@Z7tocCb{7iABSDuvjur?_I@-Bx}>Ng%*s5V;QSG zE*g#Vbu#DHgjNmg!m<{vZd%2=mfJ|@OQ(&~g)4KpxQ>Ju)wfIcmdS{9JS>XueN>99 zD@i!5>J=QXgeKwQtM~tWwz#&^U;&_BC?C#MkI5pXNv1i1@WN;&XIr<nfxzX4{SS%P zdsb9A<zbE=^_j-S&Ro-KeIUtOeit>#jAOHyGN$Nm@-(pVZ0Y>+T1%-fvuZ=^-Ci+= zly%y~#dj%YCc8Ue%9WO0p9Y)5cSXSgg|F%!-u~`t#>oCUcbGS5%a~Cqkv%HQ%6AV@ zG{P-aSADBf;-K0YV4oj>?(?ie)ER5BPv1<!hRw7IqZO*E;4$`dKK<949V(t>yDp?| zwogms?H6R3Vu_|L{5df#eFhN;LGmrDhqEQ;FW*_BqmtNDrMXGeD;rpKPMXPEu64{! zudj3=3$r~JLHF9@zFmwGD@~xV@`6;jZmN~RgOv9Mq_$UD2fp;NUoH>$?UuKF>V-s4 z1;`fAY*%qT0tv~?+LhyD5Hs!qF90mt9o*^>$W%rhz@{<hXNQOu2FjhS(ZhCJNqYa< z0GA4xy~}NS{{=-8N50qAO@(OkT0GUF7<Gfs*WKv+D&oqgjAoZwS)=>j&3zmm8oq7H z7LfUoen*2h{-r#lUcXSp`&;$weKBP<m52@KCck<j0gm1inj0Q$?(kZt#=-7k$minn z$IzpJwU#N}sy6aWZaj>{cDFzTV{pI#Lz4*)u4O7;d}<)KeAvbBmh)TE>bcS~_b;;< zB#wd44)!A*+G-XEEZ?`Zgua*7%BhR6^}IDS5A)czO|`&g48Wz*t>@?mN|U5Cg;?8m zs?8Hjw~gzkF}Xd90Fbih5c<WdaCsjw13v;y=;I+rAadRhZ4+VVAkL!S=a8XZ?kb9I zd`0|Unte7o+*}{HZZASrj4V1P%m=)z5)H?OjAh@L59+9B4wzj?K%RLVJi_!)k%TdC z?>L`bIJkx|Ea%TZ0vYn`v)fEYg)y_O=zeqmFxS+?+#k_2GHSKEoBL5{7XYoV>o8vO z)khIC9O$;m%3*bq!xPWuIh}FUq~qf!Q_@j>#4&l@b9b(mLl;vQ^0AY4xHb9%x`SaK zSyig6FB{<7V%0`7qL9a&EzhF;=<%u5d(MOT`Fn_$vjt%`qsjkBtO0Ox|KNZ3?<Jc5 zon!t?YX9|;+FwtM7mzxbu@O~U7aBE~Fyl<5JXt};V?WA=^6o~MaX3T&eaDaj0;_Sy z>kmvGq9Kfj18cu=W4tT|*S&U+8QBlVhek$+)oi`U+{Aj{N9sL!_+?4)l79p3)Tr^d zL^b_){iT>O!xov+98sEV=NK{I>E=|58k)NCOP=&dOz529I(duJM;!U<c85$qlW8@Q z7))%(rI|Pxsa@+W4oaIuSa9j<8#2smQq|W2b?(Vf+ABU(wD+LvKcB<e;~$bPPpkd% z-vn57Ofsj~7*rh-lMA!-E119zj?vM3(F$*Yn~&@hpB84BNHUSa9Ahr-T~wJa%8{ql zqWb%O!e5;`6rH5S$~V-VuG=@zBzSN?&dt3UqYip_CT@10P@C%x$a+*@5}xx24Y$H; zp(Z9~Wm*FgQCKAfMK(F%BK2%*KyIRtLmBf5$qe(44z*^U(2Q~>Im_~FM}`#ffMp0T zkZR?_@*Ps@eW6QAip6D5L$ZDCt7p7S`^*3xpR)oct3TPy1@ja_4p}2GFBRhukNoHY z<zIOW?7sJ80+71Y1k7TruXbAq-YcCj-wdT(D^xlHCFD~~+R>;3I<W%NCWAk9#f!s7 zT}2%<N3j}MQiG~JQFP$Aut(J?H_P(&>ccs+wpz+b^<Y_XD7b~;p}XQ8hmdEbLoaq? z{rj0=Rq3qoT940#)_)UqSL=(Vd7%+8nNM^d(-i^Bv~+bC)~gpRN?r1b@r?0`ZO;z| zQ~-WTX0osU0@3`Rw;cX=F1^40u76%I@Yh%QT<)<v;ZJe@pO-}bkwo*)F+V5tdF|3# zm^O``?XV~sIDb``THnm!?0BpIy<6KNp2Ay!<e-xK;3u=+i=S>dea16kPj?&F#wzyb zXMV%0!<zSv=f42!fv3p#!8%1dhe(^U>>9)bloGera(VD$NHV+a`Xw0u)aZVCR5uCN zaLuqhcD>0_;XN%u6w~Njg@oc>8wwm)0Xz!OHeiz0gN!hV5sc$+%hE8~eF3*fb+7u- zk-8)}dNaKJ)E-aN;f!=tfsirzv!t*2W?U^MoV{>Jnp*C%h=)$mV|cog6nkiTHHZ45 z3Bo2Fhn!+G?50K=yW2Dsvwd3>Q!8FFUwN94JAaTq_>MYJl*Iu{ImNS64gUb8!1*{? zWj3N6ZQ_nV`jTur$+U-*4{iFFd<5>)T@3Ca(7JP|g<m&eDSGE0vuka~6qUoTj?>76 zU&Y8~GU>y`%}()NUZou=PI>q+3R5s13eWpgnd0<?zch*gE_HUZDFJE1gITd~9#&h1 zRRu`BezNp~u!t9aVDn8HJOvSNxygOAAVU1D4Zq@7v0}lck5YnLgt$Tzje?P=AO!XG zvJpBVoLjR<n3^Wt!OKzPer5e?#f;E?a)u_(n*_^V_;<V2v2?hOPi4MU6}kWTviQJ` zPTqj>=vCfto++lC_O??9I`@?@L=dytkQJ;67xXAN!^`_|_~zv;ZnGky{QDW{l^S93 zZ5q?+ES5eLWy&|gJ{qYu)4z<^?D7N!OG;y7eb<e~6DA6&;7W-Q^(j`zMOhXy_Q9s; zN=kA2-TQ)w30AWa5}BaqvP_oAG*z~mPGTif${ufZP%z=&9J=b=Z;ueYvsW%S0{spy z>PnBUvC59SzWTVhEOfYwaiTonjv}oJordBq(8PtZzc;(<fAjt)3D6U_?ydi%mf@ez z`8%hB|AIpKpHPK=X_y8pG{T1-XV#J+XID`ct3L2(E$UHk%F6E5@#-KzqsN!Xh$MKG z!0k?Sw~ybzi;~`QU|>UeC(G2z>oi|%Wzf~hTiqN&EnX?d#9)1P!B5_>MuYJ;%zl87 zOnvIcdH11PhgLZ^KE2Z#-Hh_{bhJWUz>70qJ8Pp|<>*<Wde8`1mg0+yYaQf>Z=C7# zp?9~t_nQHWe3`vxp8P>EW!4x$dqmgBRq;B~6j&!|=$hfSy}g<^-6^F}8q+dyB}Hw^ z;Fr{kyNK(9FeACvRTUY5N%fGYapimE5zi_lcruzFBc{>u)=x_Kq|!4m`J2|sj6DLQ ze!DvMT}qPe=H!Pqnl<T-DEgXHAKalzpAgh{@~te$$cSs2h^YjKEXs^wk#dy2!->l( z;~`J%5j!fhaYzc?A6vQLuc%3<1qSjGr5?Ou9_mR^FI&hFsIEv$AQEm0mzz?<G?~B; zd$T*)66R5vyqe(bJ)R$1o|HsaD$W5#sW8uOVPqg&91u_T*>C~MkmP#men+4Y&9m6H zu<bDq&$3*J(lW$Hha=D(KI13fGL^Tl+n{NrucdX&7}-!6nZ%aaUN3PF5#M#aq9kXe z^NcyFS&ktlndKWNrYjk}Nj2z^!k%YW<4*we!)pba*hvA9T|nwajZDUn9@*nkA<&B4 zFro$>a-%v5Jhwbt>|(wn`Y1+vxd}?LCB?GU4_A}v4t4q7J-mIlmP>A{i=)E7VoA}= zBtr1CAgKQA&*B7tT-P6x7W;G5_V4-#|JUK&^71F1=f{_CXUa~TAFQPRO=sZS8SVdy z#QL94^8Qy20ZHHpqNNn~29aj67S<%oRUw}EXhqJAycd!GJb6eud!Hg>Ftfj#5$C3Y z4fJ(LUoIba$*6$EmDkwGE~mSiE+3w!o9S?y76_bPNxt-{8mk_dE)k?tWV~yiWh!lK z$bsev;hrkb(i|3Hezl$}BNjR(iV7u=^re$C1g|`PHjz2P;;OZ9*Ox$RokEAF{LpFu zrT)?7^gKQsINYb6qr~Zqc2p*SS6sK}blV8@y3KFdXgZwhm@qR_o7(Rv$GV+SOZg4! z{xMBi8hsZOs<Bbc?|0NG^~P|}CQVkgJScxhWo(5Xe_qOZ^4K|P-F$bcrklC}qNLs| zcaR&9%N<YNI(?pBmqNPAqI&5fR1&^3_h#AZyAKVQ*x6XBdcJP8?k~<|<qTk4ID3t7 zEWP+UH}&E#Xyb)nDJp-2#`5QwpGm6!y)ooF@d}DM?r;8YXAA#lSy}%`U;8so<NxUp zh`6YeE!?b5=yz7*-}~HP^Hyn=m#}{}y$I9gpF!shzSpy2^!RLfX(T5J{>%jc{4;cD zc|te))Aam*x9Ry`)AoM{Wgm0WX9V3nIg_WKyv4UJo`|&M@EwDLzMZ*o;wW<SL;&E_ ZjT7kkpa1C3K=A#`%m@HcOL8>y{{h6e?`!}7 From 17de6a67a2399226aaf84e5826af14ef6a7c507e Mon Sep 17 00:00:00 2001 From: tengu-alt <olexandr.luzhniy@gmail.com> Date: Wed, 27 Nov 2024 15:52:41 +0200 Subject: [PATCH 148/225] Autogenerate Native protocol specifications documentation Uses cqlprotodoc from cassandra-website to generate the versioned native protocol spec html pages. Uses process-native-protocol-specs-in-docker.sh as a wrapper around this to also generate the native protocol spec adoc page that includes all the versions and makes each header navigable. This is executed as part of the existing `ant gen-asciidoc` and doc/Makefile. patch by Olexandr Luzhniy; reviewed by Mick Semb Wever for CASSANDRA-19915 --- doc/Makefile | 2 + doc/modules/ROOT/pages/index.adoc | 3 +- doc/native_protocol_v3.spec | 2 +- doc/native_protocol_v5.spec | 7 +- ...process-native-protocol-specs-in-docker.sh | 126 ++++++++++++++++++ 5 files changed, 134 insertions(+), 6 deletions(-) create mode 100755 doc/scripts/process-native-protocol-specs-in-docker.sh diff --git a/doc/Makefile b/doc/Makefile index 2eee1146692e..dac8e11fb06f 100644 --- a/doc/Makefile +++ b/doc/Makefile @@ -13,6 +13,7 @@ GENERATE_NODETOOL_DOCS = ./scripts/gen-nodetool-docs.py MAKE_CASSANDRA_YAML = ./scripts/convert_yaml_to_adoc.py ../conf/cassandra.yaml ./modules/cassandra/pages/managing/configuration/cass_yaml_file.adoc +PROCESS_NATIVE_PROC_SPECS = ./scripts/process-native-protocol-specs-in-docker.sh .PHONY: html html: @@ -24,3 +25,4 @@ gen-asciidoc: @mkdir -p modules/cassandra/examples/TEXT/NODETOOL python3 $(GENERATE_NODETOOL_DOCS) python3 $(MAKE_CASSANDRA_YAML) + $(PROCESS_NATIVE_PROC_SPECS) diff --git a/doc/modules/ROOT/pages/index.adoc b/doc/modules/ROOT/pages/index.adoc index 02b3f949ea06..beeb121f220e 100644 --- a/doc/modules/ROOT/pages/index.adoc +++ b/doc/modules/ROOT/pages/index.adoc @@ -40,7 +40,8 @@ If you would like to contribute to this documentation, you are welcome to do so | xref:cassandra:integrating/plugins/index.adoc[Plug-ins] | Third-party plug-ins -| xref:master@_:ROOT:native_protocol.adoc[Native Protocols] | Native Cassandra protocol specifications +| link:cassandra/reference/native-protocol.html[Native Protocols] | Native Cassandra protocol specifications + |=== == Meta information diff --git a/doc/native_protocol_v3.spec b/doc/native_protocol_v3.spec index 8187ca0b2778..30881c949790 100644 --- a/doc/native_protocol_v3.spec +++ b/doc/native_protocol_v3.spec @@ -728,7 +728,7 @@ Table of Contents Clients are expected to answer the server challenge by an AUTH_RESPONSE message. -4.2.7. AUTH_SUCCESS +4.2.8. AUTH_SUCCESS Indicate the success of the authentication phase. See Section 4.2.3 for more details. diff --git a/doc/native_protocol_v5.spec b/doc/native_protocol_v5.spec index cbc7f9fe9caa..e080801978c5 100644 --- a/doc/native_protocol_v5.spec +++ b/doc/native_protocol_v5.spec @@ -29,7 +29,6 @@ Table of Contents 2.3.1 Initial Handshake 2.3.2 Compression 2.4. Frame Payload - 2.4.1 Frame Header 2.4.1.1. version 2.4.1.2. flags 2.4.1.3. stream @@ -232,7 +231,7 @@ Table of Contents All values in an envelope are big-endian (network byte order). Each envelope contains a fixed size header (9 bytes) followed by a variable size - body. The header is described in Section 2.4.1. The content of the body depends + body. The content of the body depends on the header opcode value (the body can in particular be empty for some opcode values). The list of allowed opcodes is defined in Section 2.4.1.4 and the details of each corresponding message are described Section 4. @@ -255,7 +254,7 @@ Table of Contents previously possible to enable compression for an individual envelope. This is no longer possible, as the framing format is responsible for compression, which is set for the lifetime of a connection and applies to all messages transmitted throughout it - (see Section 2.2.1 for caveats). The compression flag is therefore deprecated and + (see Section 2.2 for caveats). The compression flag is therefore deprecated and ignored in protocol v5. 2.4.1.1. version @@ -277,7 +276,7 @@ Table of Contents connection. This document describes version 5 of the protocol. For the changes made since - version 4, see Section 10. + version 4, see Section 9. 2.4.1.2. flags diff --git a/doc/scripts/process-native-protocol-specs-in-docker.sh b/doc/scripts/process-native-protocol-specs-in-docker.sh new file mode 100755 index 000000000000..3af83f4a310f --- /dev/null +++ b/doc/scripts/process-native-protocol-specs-in-docker.sh @@ -0,0 +1,126 @@ +#!/bin/sh +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +[ -f "../build.xml" ] || { echo "build.xml must exist (current directory needs to be doc/ in cassandra repo"; exit 1; } +[ -f "antora.yml" ] || { echo "antora.yml must exist (current directory needs to be doc/ in cassandra repo"; exit 1; } + +# Variables +GO_VERSION="1.23.1" +GO_TAR="go${GO_VERSION}.linux-amd64.tar.gz" +TMPDIR="${TMPDIR:-/tmp}" + +# Step 0: Download and install Go +echo "Downloading Go $GO_VERSION..." +wget -q "https://golang.org/dl/$GO_TAR" -O "$TMPDIR/$GO_TAR" + +echo "Installing Go..." +tar -C "$TMPDIR" -xzf "$TMPDIR/$GO_TAR" +rm "$TMPDIR/$GO_TAR" + +# Set Go environment variables +export PATH="$PATH:$TMPDIR/go/bin" +export GOPATH="$TMPDIR/go" + +# Step 1: Building the parser +echo "Building the cqlprotodoc..." +DIR="$(pwd)" +cd "${TMPDIR}" + +git clone -n --depth=1 --filter=tree:0 https://github.com/apache/cassandra-website + +cd "${TMPDIR}/cassandra-website" +git sparse-checkout set --no-cone /cqlprotodoc +git checkout +cd "${TMPDIR}/cassandra-website/cqlprotodoc" +go build -o "$TMPDIR"/cqlprotodoc + +# Step 2: Process the spec files using the parser +echo "Processing the .spec files..." +cd "${DIR}" +output_dir="modules/cassandra/attachments" +mkdir -p "${output_dir}" +"$TMPDIR"/cqlprotodoc . "${output_dir}" + +# Step 4: Generate summary file +summary_file="modules/cassandra/pages/reference/native-protocol.adoc" + +# Write the header +echo "= Native Protocol Versions" > "$summary_file" +echo ":page-layout: default" >> "$summary_file" +echo >> "$summary_file" + +# Loop through the files from step 2 in reverse version order +for file in $(ls ${output_dir}/native_protocol_v*.html | sort -r | awk -F/ '{print $NF}'); do + version=$(echo "$file" | sed -E 's/native_protocol_v([0-9]+)\.html/\1/') + echo "== Native Protocol Version $version" >> "$summary_file" + echo >> "$summary_file" + echo "[source, html]" >> "$summary_file" + echo "++++" >> "$summary_file" + echo "include::cassandra:attachment\$$file[Version $version]" >> "$summary_file" + echo "++++" >> "$summary_file" + echo >> "$summary_file" +done + +# Navigation setup +echo "[source, js]" >> "$summary_file" +echo "++++" >> "$summary_file" +echo "<script>" >> "$summary_file" +echo " function setNavigation() {" >> "$summary_file" +echo " var containers = document.querySelectorAll('.sect1');" >> "$summary_file" +echo >> "$summary_file" +echo " containers.forEach(function (container) {" >> "$summary_file" +echo " var preElements = container.querySelectorAll('pre');" >> "$summary_file" +echo " preElements.forEach(function(preElement) {" >> "$summary_file" +echo " if (!preElement.textContent.trim()) {" >> "$summary_file" +echo " preElement.remove();" >> "$summary_file" +echo " }" >> "$summary_file" +echo " });" >> "$summary_file" +echo " var h1Elements = container.querySelectorAll('h1');" >> "$summary_file" +echo " h1Elements.forEach(function(h1Element) {" >> "$summary_file" +echo " h1Element.remove();" >> "$summary_file" +echo " });" >> "$summary_file" +echo >> "$summary_file" +echo " var navLinks = container.querySelectorAll('nav a, pre a');" >> "$summary_file" +echo >> "$summary_file" +echo " navLinks.forEach(function (link) {" >> "$summary_file" +echo " link.addEventListener('click', function (event) {" >> "$summary_file" +echo >> "$summary_file" +echo " event.preventDefault();" >> "$summary_file" +echo " var section = link.getAttribute('href').replace(\"#\", '');" >> "$summary_file" +echo >> "$summary_file" +echo " var targetSection = container.querySelector('h2[id=\"' + section + '\"]') || container.querySelector('h3[id=\"' + section + '\"]') || container.querySelector('h4[id=\"' + section + '\"]') || container.querySelector('h5[id=\"' + section + '\"]');" >> "$summary_file" +echo >> "$summary_file" +echo " if (targetSection) {" >> "$summary_file" +echo " targetSection.scrollIntoView({ behavior: 'smooth' });" >> "$summary_file" +echo " }" >> "$summary_file" +echo " });" >> "$summary_file" +echo " });" >> "$summary_file" +echo " });" >> "$summary_file" +echo " }" >> "$summary_file" +echo >> "$summary_file" +echo " window.onload = function() {" >> "$summary_file" +echo " setNavigation()" >> "$summary_file" +echo " }" >> "$summary_file" +echo " </script>" >> "$summary_file" + + +# Step 3: Cleanup - Remove the Cassandra and parser directories +echo "Cleaning up..." +cd "${DIR}" +rm -rf "${TMPDIR}/go" "${TMPDIR}/cassandra-website" "${TMPDIR}/cqlprotodoc" 2>/dev/null + +echo "Script completed successfully." From 378c93de90e16ee90cc5282de130bc18cc48f97e Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Mon, 24 Feb 2025 17:06:02 +0100 Subject: [PATCH 149/225] Include materialized views to the output of DESCRIBE TABLE statements patch by Stefan Miklosovic; reviewed by Brandon Williams for CASSANDRA-20365 --- CHANGES.txt | 1 + .../cql3/statements/DescribeStatement.java | 44 +++++++++++++++++-- .../statements/DescribeStatementTest.java | 6 +-- 3 files changed, 45 insertions(+), 6 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 37e09acc988f..06c119f1fff3 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.4 + * Include materialized views to the output of DESCRIBE TABLE statements (CASSANDRA-20365) * Heap and GC jvm flags improvements (CASSANDRA-20296) * Fix unparseable YAML in default cassandra.yaml when uncommented for downstream tooling (CASSANDRA-20359) * Avoid fetching entire partitions on unresolved static rows in RFP when no static column predicates exist (CASSANDRA-20243) diff --git a/src/java/org/apache/cassandra/cql3/statements/DescribeStatement.java b/src/java/org/apache/cassandra/cql3/statements/DescribeStatement.java index 9d886455e530..1b3c5fbbb0d9 100644 --- a/src/java/org/apache/cassandra/cql3/statements/DescribeStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/DescribeStatement.java @@ -23,6 +23,7 @@ import java.util.function.BiFunction; import java.util.stream.Collectors; import java.util.stream.Stream; +import java.util.stream.StreamSupport; import com.google.common.collect.ImmutableList; @@ -474,9 +475,16 @@ public static DescribeStatement<SchemaElement> table(String keyspace, String nam TableMetadata table = checkNotNull(ks.getTableNullable(t), "Table '%s' not found in keyspace '%s'", t, ks.name); - return Stream.concat(Stream.of(table), table.indexes.stream() - .map(index -> toDescribable(table, index)) - .sorted(SchemaElement.NAME_COMPARATOR)); + + Stream<SchemaElement> withIndexes = Stream.concat(Stream.of(table), table.indexes.stream() + .map(index -> toDescribable(table, index)) + .sorted(SchemaElement.NAME_COMPARATOR)); + + Stream<SchemaElement> views = StreamSupport.stream(ks.views.forTable(table.id).spliterator(), false) + .map(viewMetadata -> toDescribable(table, viewMetadata)) + .sorted(SchemaElement.NAME_COMPARATOR); + + return Stream.concat(withIndexes, views); }); } @@ -582,6 +590,36 @@ public String toCqlString(boolean withInternals, boolean ifNotExists) }; } + private static SchemaElement toDescribable(TableMetadata table, ViewMetadata viewMetadata) + { + return new SchemaElement() + { + @Override + public SchemaElementType elementType() + { + return SchemaElementType.MATERIALIZED_VIEW; + } + + @Override + public String elementKeyspace() + { + return table.keyspace; + } + + @Override + public String elementName() + { + return viewMetadata.name(); + } + + @Override + public String toCqlString(boolean withInternals, boolean ifNotExists) + { + return viewMetadata.toCqlString(withInternals, ifNotExists); + } + }; + } + /** * Creates a {@link DescribeStatement} for the generic {@code DESCRIBE ...}. */ diff --git a/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java b/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java index 175a9f3809e1..572df6c03be3 100644 --- a/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java +++ b/test/unit/org/apache/cassandra/cql3/statements/DescribeStatementTest.java @@ -56,7 +56,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -public class DescribeStatementTest extends CQLTester +public class DescribeStatementTest extends CQLTester { @Test public void testSchemaChangeDuringPaging() @@ -555,9 +555,9 @@ public void testDescribeTableAndMaterializedViewWithClustringOrder() throws Thro try { - assertRowsNet(executeDescribeNet("DESCRIBE TABLE " + KEYSPACE_PER_TEST + "." + table), - row(KEYSPACE_PER_TEST, "table", table, tableCreateStatement)); + row(KEYSPACE_PER_TEST, "table", table, tableCreateStatement), + row(KEYSPACE_PER_TEST, "materialized_view", "mv", mvCreateStatement)); assertRowsNet(executeDescribeNet("DESCRIBE MATERIALIZED VIEW " + KEYSPACE_PER_TEST + ".mv"), row(KEYSPACE_PER_TEST, "materialized_view", "mv", mvCreateStatement)); From 989f0414b72db77e0897042a2e5b06a601032d8c Mon Sep 17 00:00:00 2001 From: Maxwell Guo <cclive1601@gmail.com> Date: Sat, 25 Jan 2025 00:27:51 +0800 Subject: [PATCH 150/225] Support CREATE TABLE LIKE with INDEXES patch by Maxwell Guo; reviewed by Stefan Miklosovic, Sam Tunnicliffe for CASSANDRA-19965 --- CHANGES.txt | 1 + doc/cql3/CQL.textile | 11 +- .../examples/BNF/create_table_like.bnf | 2 +- .../examples/CQL/create_table_like.cql | 3 + pylib/cqlshlib/cql3handling.py | 5 +- pylib/cqlshlib/test/test_cqlsh_completion.py | 28 +- src/antlr/Lexer.g | 1 + src/antlr/Parser.g | 12 +- .../statements/schema/CopyTableStatement.java | 69 +++- .../cassandra/schema/KeyspaceMetadata.java | 40 +- .../cql3/AlterSchemaStatementTest.java | 3 +- .../org/apache/cassandra/cql3/CQLTester.java | 32 +- .../schema/createlike/CreateLikeTest.java | 390 +++++++++++------- 13 files changed, 424 insertions(+), 173 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index b704cca5d454..269b8a8f45a5 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Support CREATE TABLE LIKE WITH INDEXES (CASSANDRA-19965) * Invalidate relevant prepared statements on every change to TableMetadata (CASSANDRA-20318) * Add per type max size guardrails (CASSANDRA-19677) * Make it possible to abort all kinds of multi step operations (CASSANDRA-20217) diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile index f819ec193d28..87812fc19b01 100644 --- a/doc/cql3/CQL.textile +++ b/doc/cql3/CQL.textile @@ -414,16 +414,19 @@ bc(syntax).. <copy-table-stmt> ::= CREATE ( TABLE | COLUMNFAMILY ) ( IF NOT EXISTS )? <newtablename> LIKE <oldtablename> ( WITH <option> ( AND <option>)* )? -<option> ::= <property> +<option> ::= <property> | INDEXES -p. +p. __Sample:__ bc(sample).. CREATE TABLE newtb1 LIKE oldtb; CREATE TABLE newtb2 LIKE oldtb WITH compaction = { 'class' : 'LeveledCompactionStrategy' }; -p. + +CREATE TABLE newtb4 LIKE oldtb WITH INDEXES AND compaction = { 'class' : 'LeveledCompactionStrategy' }; + +p. The @COPY TABLE@ statement creates a new table which is a clone of old table. The new table have the same column numbers, column names, column data types, column data mask with the old table. The new table is defined by a "name":#copyNewTableName, and the name of the old table being cloned is defined by a "name":#copyOldTableName . The table options of the new table can be defined by setting "copyoptions":#copyTableOptions. Note that the @CREATE COLUMNFAMILY LIKE@ syntax is supported as an alias for @CREATE TABLE like@ (for historical reasons). Attempting to create an already existing table will return an error unless the @IF NOT EXISTS@ option is used. If it is used, the statement will be a no-op if the table already exists. @@ -438,7 +441,7 @@ The old table name defines the already existed table. h4(#copyTableOptions). @<copyoptions>@ -The @COPY TABLE@ statement supports a number of options that controls the configuration of a new table. These options can be specified after the @WITH@ keyword, and all options are the same as those options when creating a table except for id . +The @COPY TABLE@ statement supports a number of options that controls the configuration of a new table. These options can be specified after the @WITH@ keyword, and all options are the same as those options when creating a table except for id. Besides the options can also be specified with keyword INDEXES which means copy source table's indexes. h3(#alterTableStmt). ALTER TABLE diff --git a/doc/modules/cassandra/examples/BNF/create_table_like.bnf b/doc/modules/cassandra/examples/BNF/create_table_like.bnf index 56d209c6ef28..ebbd9b46588e 100644 --- a/doc/modules/cassandra/examples/BNF/create_table_like.bnf +++ b/doc/modules/cassandra/examples/BNF/create_table_like.bnf @@ -1,3 +1,3 @@ create_table_statement::= CREATE TABLE [ IF NOT EXISTS ] new_table_name LIKE old_table_name [ WITH table_options ] -table_options::= options [ AND table_options ] \ No newline at end of file +table_options::= INDEXES | options [ AND table_options ] \ No newline at end of file diff --git a/doc/modules/cassandra/examples/CQL/create_table_like.cql b/doc/modules/cassandra/examples/CQL/create_table_like.cql index ef65b8b0509c..f47fb191f0d2 100644 --- a/doc/modules/cassandra/examples/CQL/create_table_like.cql +++ b/doc/modules/cassandra/examples/CQL/create_table_like.cql @@ -12,3 +12,6 @@ CREATE TABLE newtb3 LIKE oldtb WITH compaction = { 'class' : 'LeveledCompactionS AND compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 32 } AND cdc = true; +CREATE TABLE newtb4 LIKE oldtb WITH INDEXES; + +CREATE TABLE newtb6 LIKE oldtb WITH INDEXES AND compaction = { 'class' : 'LeveledCompactionStrategy' }; \ No newline at end of file diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py index 5c48f8e60011..e231d5c8a8ee 100644 --- a/pylib/cqlshlib/cql3handling.py +++ b/pylib/cqlshlib/cql3handling.py @@ -387,6 +387,9 @@ def dequote_value(cqlword): ( ender="," [propmapkey]=<term> ":" [propmapval]=<term> )* ender="}" ; +<propertyOrOption> ::= <property> + | "INDEXES" + ; ''' @@ -1314,7 +1317,7 @@ def create_cf_composite_primary_key_comma_completer(ctxt, cass): <copyTableStatement> ::= "CREATE" wat=("COLUMNFAMILY" | "TABLE" ) ("IF" "NOT" "EXISTS")? ( tks=<nonSystemKeyspaceName> dot="." )? tcf=<cfOrKsName> "LIKE" ( sks=<nonSystemKeyspaceName> dot="." )? scf=<cfOrKsName> - ( "WITH" <property> ( "AND" <property> )* )? + ( "WITH" <propertyOrOption> ( "AND" <propertyOrOption> )* )? ; ''' diff --git a/pylib/cqlshlib/test/test_cqlsh_completion.py b/pylib/cqlshlib/test/test_cqlsh_completion.py index da362c50f415..55f5511ef1a2 100644 --- a/pylib/cqlshlib/test/test_cqlsh_completion.py +++ b/pylib/cqlshlib/test/test_cqlsh_completion.py @@ -796,9 +796,9 @@ def test_complete_in_create_table_like(self): choices=['<new_table_name>']) self.trycompletions('CREATE TABLE ' + quoted_keyspace + '.new_table L', immediate='IKE ') - self.trycompletions('CREATE TABLE ' + 'new_table LIKE old_table W', + self.trycompletions('CREATE TABLE new_table LIKE old_table W', immediate='ITH ') - self.trycompletions('CREATE TABLE ' + 'new_table LIKE old_table WITH ', + self.trycompletions('CREATE TABLE new_table LIKE old_table WITH ', choices=['allow_auto_snapshot', 'bloom_filter_fp_chance', 'compaction', 'compression', @@ -808,23 +808,16 @@ def test_complete_in_create_table_like(self): 'memtable', 'memtable_flush_period_in_ms', 'caching', 'comment', - 'min_index_interval', 'speculative_retry', 'additional_write_policy', 'cdc', 'read_repair']) - self.trycompletions('CREATE TABLE ' + 'new_table LIKE old_table WITH ', - choices=['allow_auto_snapshot', - 'bloom_filter_fp_chance', 'compaction', - 'compression', - 'default_time_to_live', 'gc_grace_seconds', - 'incremental_backups', - 'max_index_interval', - 'memtable', - 'memtable_flush_period_in_ms', - 'caching', 'comment', - 'min_index_interval', 'speculative_retry', 'additional_write_policy', 'cdc', 'read_repair']) + 'min_index_interval', + 'speculative_retry', 'additional_write_policy', + 'cdc', 'read_repair', + 'INDEXES']) + self.trycompletions('CREATE TABLE new_table LIKE old_table WITH INDEXES ', + choices=[';' , '=', 'AND']) self.trycompletions('CREATE TABLE ' + 'new_table LIKE old_table WITH bloom_filter_fp_chance ', immediate='= ') self.trycompletions('CREATE TABLE ' + 'new_table LIKE old_table WITH bloom_filter_fp_chance = ', choices=['<float_between_0_and_1>']) - self.trycompletions('CREATE TABLE ' + 'new_table LIKE old_table WITH compaction ', immediate="= {'class': '") self.trycompletions('CREATE TABLE ' + "new_table LIKE old_table WITH compaction = " @@ -868,7 +861,10 @@ def test_complete_in_create_table_like(self): 'memtable', 'memtable_flush_period_in_ms', 'caching', 'comment', - 'min_index_interval', 'speculative_retry', 'additional_write_policy', 'cdc', 'read_repair']) + 'min_index_interval', + 'speculative_retry', 'additional_write_policy', + 'cdc', 'read_repair', + 'INDEXES']) self.trycompletions('CREATE TABLE ' + "new_table LIKE old_table WITH compaction = " + "{'class': 'TimeWindowCompactionStrategy', '", choices=['compaction_window_unit', 'compaction_window_size', diff --git a/src/antlr/Lexer.g b/src/antlr/Lexer.g index 96a51e23c80a..b192021fa318 100644 --- a/src/antlr/Lexer.g +++ b/src/antlr/Lexer.g @@ -98,6 +98,7 @@ K_TABLES: ( C O L U M N F A M I L I E S K_MATERIALIZED:M A T E R I A L I Z E D; K_VIEW: V I E W; K_INDEX: I N D E X; +K_INDEXES: I N D E X E S; K_CUSTOM: C U S T O M; K_ON: O N; K_TO: T O; diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g index 682ba873e46d..30e07fddf94d 100644 --- a/src/antlr/Parser.g +++ b/src/antlr/Parser.g @@ -836,7 +836,16 @@ copyTableStatement returns [CopyTableStatement.Raw stmt] : K_CREATE K_COLUMNFAMILY (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? newCf=columnFamilyName K_LIKE oldCf=columnFamilyName { $stmt = new CopyTableStatement.Raw(newCf, oldCf, ifNotExists); } - ( K_WITH property[stmt.attrs] ( K_AND property[stmt.attrs] )*)? + ( K_WITH propertyOrOption[stmt] ( K_AND propertyOrOption[stmt] )*)? + ; + +propertyOrOption[CopyTableStatement.Raw stmt] + : tableLikeSingleOption[stmt] + | property[stmt.attrs] + ; + +tableLikeSingleOption[CopyTableStatement.Raw stmt] + : K_INDEXES {$stmt.withLikeOption(CopyTableStatement.CreateLikeOption.INDEXES);} ; /** @@ -2084,5 +2093,6 @@ basic_unreserved_keyword returns [String str] | K_ANN | K_BETWEEN | K_CHECK + | K_INDEXES ) { $str = $k.text; } ; diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CopyTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CopyTableStatement.java index e8ee7e2cd308..3a79311dc771 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/CopyTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/CopyTableStatement.java @@ -19,6 +19,8 @@ package org.apache.cassandra.cql3.statements.schema; import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -35,6 +37,9 @@ import org.apache.cassandra.db.marshal.UserType; import org.apache.cassandra.db.marshal.VectorType; import org.apache.cassandra.exceptions.AlreadyExistsException; +import org.apache.cassandra.index.sai.StorageAttachedIndex; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.IndexMetadata; import org.apache.cassandra.schema.Indexes; import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.Keyspaces; @@ -46,6 +51,7 @@ import org.apache.cassandra.schema.Triggers; import org.apache.cassandra.schema.UserFunctions; import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.ClientWarn; import org.apache.cassandra.service.reads.repair.ReadRepairStrategy; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.transport.Event.SchemaChange; @@ -61,12 +67,14 @@ public final class CopyTableStatement extends AlterSchemaStatement private final String targetTableName; private final boolean ifNotExists; private final TableAttributes attrs; + private final CreateLikeOption createLikeOption; public CopyTableStatement(String sourceKeyspace, String targetKeyspace, String sourceTableName, String targetTableName, boolean ifNotExists, + CreateLikeOption createLikeOption, TableAttributes attrs) { super(targetKeyspace); @@ -75,6 +83,7 @@ public CopyTableStatement(String sourceKeyspace, this.sourceTableName = sourceTableName; this.targetTableName = targetTableName; this.ifNotExists = ifNotExists; + this.createLikeOption = createLikeOption; this.attrs = attrs; } @@ -196,6 +205,7 @@ public Keyspaces apply(ClusterMetadata metadata) TableParams originalParams = targetBuilder.build().params; TableParams newTableParams = attrs.asAlteredTableParams(originalParams); + maybeCopyIndexes(targetBuilder, sourceTableMeta, targetKeyspaceMeta); TableMetadata table = targetBuilder.params(newTableParams) .id(TableId.get(metadata)) @@ -229,12 +239,59 @@ public void validate(ClientState state) validateDefaultTimeToLive(attrs.asNewTableParams()); } + private void maybeCopyIndexes(TableMetadata.Builder builder, TableMetadata sourceTableMeta, KeyspaceMetadata targetKeyspaceMeta) + { + if (createLikeOption != CreateLikeOption.INDEXES || sourceTableMeta.indexes.isEmpty()) + return; + + Set<String> customIndexes = Sets.newTreeSet(); + List<IndexMetadata> indexesToCopy = new ArrayList<>(); + for (IndexMetadata indexMetadata : sourceTableMeta.indexes) + { + // only sai and legacy secondary index is supported + if (indexMetadata.isCustom() && !StorageAttachedIndex.class.getCanonicalName().equals(indexMetadata.getIndexClassName())) + { + customIndexes.add(indexMetadata.name); + continue; + } + + ColumnMetadata targetColumn = sourceTableMeta.getColumn(UTF8Type.instance.decompose(indexMetadata.options.get("target"))); + String indexName; + // The rules for generating the index names of the target table are: + // (1) If the source table's index names follow the pattern sourcetablename_columnname_idx_number, the index names are considered to be generated by the system, + // then we directly replace the name of source table with the name of target table, and increment the number after idx to avoid index name conflicts. + // (2) Index names that do not follow the above pattern are considered user-defined, so the index names are retained and increment the number after idx to avoid conflicts. + if (indexMetadata.name.startsWith(sourceTableName + "_" + targetColumn.name + "_idx")) + { + String baseName = IndexMetadata.generateDefaultIndexName(targetTableName, targetColumn.name); + indexName = targetKeyspaceMeta.findAvailableIndexName(baseName, indexesToCopy, targetKeyspaceMeta); + } + else + { + indexName = targetKeyspaceMeta.findAvailableIndexName(indexMetadata.name, indexesToCopy, targetKeyspaceMeta); + } + indexesToCopy.add(IndexMetadata.fromSchemaMetadata(indexName, indexMetadata.kind, indexMetadata.options)); + } + + if (!indexesToCopy.isEmpty()) + builder.indexes(Indexes.builder().add(indexesToCopy).build()); + + if (!customIndexes.isEmpty()) + ClientWarn.instance.warn(String.format("Source table %s.%s to copy indexes from to %s.%s has custom indexes. These indexes were not copied: %s", + sourceKeyspace, + sourceTableName, + targetKeyspace, + targetTableName, + customIndexes)); + } + public final static class Raw extends CQLStatement.Raw { private final QualifiedName oldName; private final QualifiedName newName; private final boolean ifNotExists; public final TableAttributes attrs = new TableAttributes(); + private CreateLikeOption createLikeOption = null; public Raw(QualifiedName newName, QualifiedName oldName, boolean ifNotExists) { @@ -248,7 +305,17 @@ public CQLStatement prepare(ClientState state) { String oldKeyspace = oldName.hasKeyspace() ? oldName.getKeyspace() : state.getKeyspace(); String newKeyspace = newName.hasKeyspace() ? newName.getKeyspace() : state.getKeyspace(); - return new CopyTableStatement(oldKeyspace, newKeyspace, oldName.getName(), newName.getName(), ifNotExists, attrs); + return new CopyTableStatement(oldKeyspace, newKeyspace, oldName.getName(), newName.getName(), ifNotExists, createLikeOption, attrs); + } + + public void withLikeOption(CreateLikeOption option) + { + this.createLikeOption = option; } } + + public enum CreateLikeOption + { + INDEXES; + } } diff --git a/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java b/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java index aec93c9f47b2..8065c5929007 100644 --- a/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java +++ b/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java @@ -18,6 +18,8 @@ package org.apache.cassandra.schema; import java.io.IOException; +import java.util.Collection; +import java.util.Collections; import java.util.HashSet; import java.util.Optional; import java.util.Set; @@ -207,19 +209,49 @@ public Stream<TableMetadata> tablesUsingFunction(Function function) public String findAvailableIndexName(String baseName) { - if (!hasIndex(baseName)) + return findAvailableIndexName(baseName, Collections.emptySet(), this); + } + + /** + * find an avaiable index name based on the indexes in target keyspace and indexes collections + * @param baseName the base name of index + * @param indexes find out whether there is any conflict with baseName in the indexes + * @param keyspaceMetadata find out whether there is any conflict with baseName in keyspaceMetadata + * */ + public String findAvailableIndexName(String baseName, Collection<IndexMetadata> indexes, KeyspaceMetadata keyspaceMetadata) + { + if (!hasIndex(baseName, indexes, keyspaceMetadata)) return baseName; - int i = 1; do { - String name = baseName + '_' + i++; - if (!hasIndex(name)) + String name = generateIndexName(baseName); + if (!hasIndex(name, indexes, keyspaceMetadata)) return name; + baseName = name; } while (true); } + private String generateIndexName(String baseName) + { + if (baseName.matches(".*_\\d+$")) + { + int lastUnderscoreIndex = baseName.lastIndexOf('_'); + String numberStr = baseName.substring(lastUnderscoreIndex + 1); + int number = Integer.parseInt(numberStr) + 1; + return baseName.substring(0, lastUnderscoreIndex + 1) + number; + } + + return baseName + "_1"; + } + + private boolean hasIndex(String baseName, Collection<IndexMetadata> indexes, KeyspaceMetadata keyspaceMetadata) + { + return any(indexes, t -> t.name.equals(baseName)) || + any(keyspaceMetadata.tables, t -> t.indexes.has(baseName)); + } + public Optional<TableMetadata> findIndexedTable(String indexName) { for (TableMetadata table : tablesAndViews()) diff --git a/test/unit/org/apache/cassandra/cql3/AlterSchemaStatementTest.java b/test/unit/org/apache/cassandra/cql3/AlterSchemaStatementTest.java index 57d7d2b5a5a8..6d58e7bcee4b 100644 --- a/test/unit/org/apache/cassandra/cql3/AlterSchemaStatementTest.java +++ b/test/unit/org/apache/cassandra/cql3/AlterSchemaStatementTest.java @@ -40,7 +40,8 @@ public class AlterSchemaStatementTest extends CQLTester "CREATE TABLE ks.t1 (k int PRIMARY KEY)", "ALTER MATERIALIZED VIEW ks.v1 WITH compaction = { 'class' : 'LeveledCompactionStrategy' }", "ALTER TABLE ks.t1 ADD v int", - "CREATE TABLE ks.tb like ks1.tb" + "CREATE TABLE ks.tb like ks1.tb", + "CREATE TABLE ks.tb like ks1.tb WITH indexes" }; private final ClientState clientState = ClientState.forExternalCalls(InetSocketAddress.createUnresolved("127.0.0.1", 1234)); diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index 59ab7be83f7e..22ab64c12fca 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -1095,7 +1095,7 @@ protected String createTableLike(String query, String sourceTable, String source } String currentTable = createTableName(targetTable); - String fullQuery = currentTable == null ? query : String.format(query, targetKeyspace + "." + currentTable, sourceKeyspace + "." + sourceTable);; + String fullQuery = currentTable == null ? query : String.format(query, targetKeyspace + "." + currentTable, sourceKeyspace + "." + sourceTable); logger.info(fullQuery); schemaChange(fullQuery); return currentTable; @@ -1972,21 +1972,41 @@ private static boolean isEmptyContainerNull(DataType type, * Determine whether the source and target TableMetadata is equal without compare the table name and dropped columns. * @param source the source TableMetadata * @param target the target TableMetadata - * @param compareParams wether compare table params + * @param compareParams wether compare table's params * @param compareIndexes wether compare table's indexes - * @param compareTrigger wether compare table's triggers + * @param compareIndexWithOutName wether ignore indexes' name when doing index comparison + * if true then compare the index without name * */ - protected boolean equalsWithoutTableNameAndDropCns(TableMetadata source, TableMetadata target, boolean compareParams, boolean compareIndexes, boolean compareTrigger) + protected boolean equalsWithoutTableNameAndDropCns(TableMetadata source, TableMetadata target, boolean compareParams, boolean compareIndexes, boolean compareIndexWithOutName) { return source.partitioner.equals(target.partitioner) && source.kind == target.kind && source.flags.equals(target.flags) && (!compareParams || source.params.equals(target.params)) - && (!compareIndexes || source.indexes.equals(target.indexes)) - && (!compareTrigger || source.triggers.equals(target.triggers)) + && (!compareIndexes || compareIndexes(source, target, compareIndexWithOutName)) && columnsEqualWitoutKsTb(source, target); } + private boolean compareIndexes(TableMetadata source, TableMetadata target, boolean compareIndexWithOutName) + { + if (compareIndexWithOutName) + { + if (source.indexes.size() != target.indexes.size()) + return false; + Iterator<IndexMetadata> leftIter = source.indexes.stream().sorted(Comparator.comparing(idx -> idx.name)).iterator(); + Iterator<IndexMetadata> rightIter = target.indexes.stream().sorted(Comparator.comparing(idx -> idx.name)).iterator(); + boolean result = true; + while (leftIter.hasNext() && rightIter.hasNext()) + { + IndexMetadata left = leftIter.next(); + IndexMetadata right = rightIter.next(); + result &= right.equalsWithoutName(left); + } + return result; + } + return source.indexes.equals(target.indexes); + } + // only compare columns private boolean columnsEqualWitoutKsTb(TableMetadata source, TableMetadata target) { diff --git a/test/unit/org/apache/cassandra/schema/createlike/CreateLikeTest.java b/test/unit/org/apache/cassandra/schema/createlike/CreateLikeTest.java index 1fe72121a6bd..291a05046407 100644 --- a/test/unit/org/apache/cassandra/schema/createlike/CreateLikeTest.java +++ b/test/unit/org/apache/cassandra/schema/createlike/CreateLikeTest.java @@ -20,11 +20,13 @@ import java.math.BigDecimal; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.Date; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.TreeSet; import java.util.UUID; import java.util.stream.Collectors; @@ -34,14 +36,16 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import com.datastax.driver.core.ResultSet; +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.cql3.Duration; import org.apache.cassandra.cql3.validation.operations.CreateTest; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.compaction.LeveledCompactionStrategy; import org.apache.cassandra.exceptions.AlreadyExistsException; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.index.StubIndex; import org.apache.cassandra.schema.CachingParams; import org.apache.cassandra.schema.CompactionParams; import org.apache.cassandra.schema.CompressionParams; @@ -54,6 +58,7 @@ import org.apache.cassandra.utils.TimeUUID; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; @@ -68,28 +73,27 @@ public class CreateLikeTest extends CQLTester public static Collection<Object[]> data() { List<Object[]> result = new ArrayList<>(); - result.add(new Object[]{false}); - result.add(new Object[]{true}); + result.add(new Object[]{ false }); + result.add(new Object[]{ true }); return result; } - private UUID uuid1 = UUID.fromString("62c3e96f-55cd-493b-8c8e-5a18883a1698"); - private UUID uuid2 = UUID.fromString("52c3e96f-55cd-493b-8c8e-5a18883a1698"); - private TimeUUID timeUuid1 = TimeUUID.fromString("00346642-2d2f-11ed-a261-0242ac120002"); - private TimeUUID timeUuid2 = TimeUUID.fromString("10346642-2d2f-11ed-a261-0242ac120002"); - private Duration duration1 = Duration.newInstance(1, 2, 3); - private Duration duration2 = Duration.newInstance(1, 2, 4); - private Date date1 = new Date(); - private Double d1 = Double.valueOf("1.1"); - private Double d2 = Double.valueOf("2.2"); - private Float f1 = Float.valueOf("3.33"); - private Float f2 = Float.valueOf("4.44"); - private BigDecimal decimal1 = BigDecimal.valueOf(1.1); - private BigDecimal decimal2 = BigDecimal.valueOf(2.2); - private Vector<Integer> vector1 = vector(1, 2); - private Vector<Integer> vector2 = vector(3, 4); - private String keyspace1 = "keyspace1"; - private String keyspace2 = "keyspace2"; + private final UUID uuid1 = UUID.fromString("62c3e96f-55cd-493b-8c8e-5a18883a1698"); + private final UUID uuid2 = UUID.fromString("52c3e96f-55cd-493b-8c8e-5a18883a1698"); + private final TimeUUID timeUuid1 = TimeUUID.fromString("00346642-2d2f-11ed-a261-0242ac120002"); + private final TimeUUID timeUuid2 = TimeUUID.fromString("10346642-2d2f-11ed-a261-0242ac120002"); + private final Duration duration1 = Duration.newInstance(1, 2, 3); + private final Duration duration2 = Duration.newInstance(1, 2, 4); + private final Double d1 = Double.valueOf("1.1"); + private final Double d2 = Double.valueOf("2.2"); + private final Float f1 = Float.valueOf("3.33"); + private final Float f2 = Float.valueOf("4.44"); + private final BigDecimal decimal1 = BigDecimal.valueOf(1.1); + private final BigDecimal decimal2 = BigDecimal.valueOf(2.2); + private final Vector<Integer> vector1 = vector(1, 2); + private final Vector<Integer> vector2 = vector(3, 4); + private final String keyspace1 = "keyspace1"; + private final String keyspace2 = "keyspace2"; private String sourceKs; private String targetKs; @@ -139,38 +143,38 @@ public void testTableSchemaCopy() targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", - 1, set(list("1", "2"), list("3", "4")), map("k", 1), (short)2, duration1, (byte)4); + 1, set(list("1", "2"), list("3", "4")), map("k", 1), (short) 2, duration1, (byte) 4); execute("INSERT INTO " + targetKs + "." + targetTb + " (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", - 2, set(list("5", "6"), list("7", "8")), map("nk", 2), (short)3, duration2, (byte)5); + 2, set(list("5", "6"), list("7", "8")), map("nk", 2), (short) 3, duration2, (byte) 5); assertRows(execute("SELECT * FROM " + sourceKs + "." + sourceTb), - row(1, set(list("1", "2"), list("3", "4")), map("k", 1), (short)2, duration1, (byte)4)); + row(1, set(list("1", "2"), list("3", "4")), map("k", 1), (short) 2, duration1, (byte) 4)); assertRows(execute("SELECT * FROM " + targetKs + "." + targetTb), - row(2, set(list("5", "6"), list("7", "8")), map("nk", 2), (short)3, duration2, (byte)5)); + row(2, set(list("5", "6"), list("7", "8")), map("nk", 2), (short) 3, duration2, (byte) 5)); sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int , b double, c tinyint, d float, e list<text>, f map<text, int>, g duration, PRIMARY KEY((a, b, c), d));"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a, b, c, d, e, f, g) VALUES (?, ?, ?, ?, ?, ?, ?) ", - 1, d1, (byte)4, f1, list("a", "b"), map("k", 1), duration1); + 1, d1, (byte) 4, f1, list("a", "b"), map("k", 1), duration1); execute("INSERT INTO " + targetKs + "." + targetTb + " (a, b, c, d, e, f, g) VALUES (?, ?, ?, ?, ?, ?, ?) ", - 2, d2, (byte)5, f2, list("c", "d"), map("nk", 2), duration2); + 2, d2, (byte) 5, f2, list("c", "d"), map("nk", 2), duration2); assertRows(execute("SELECT * FROM " + sourceKs + "." + sourceTb), - row(1, d1, (byte)4, f1, list("a", "b"), map("k", 1), duration1)); + row(1, d1, (byte) 4, f1, list("a", "b"), map("k", 1), duration1)); assertRows(execute("SELECT * FROM " + targetKs + "." + targetTb), - row(2, d2, (byte)5, f2, list("c", "d"), map("nk", 2), duration2)); + row(2, d2, (byte) 5, f2, list("c", "d"), map("nk", 2), duration2)); sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int , " + - "b text, " + - "c bigint, " + - "d decimal, " + - "e set<text>, " + - "f uuid, " + - "g vector<int, 2>, " + - "h list<float>, " + - "i timeuuid, " + - "j map<text, frozen<set<int>>>, " + - "PRIMARY KEY((a, b), c, d)) " + - "WITH CLUSTERING ORDER BY (c DESC, d ASC);"); + "b text, " + + "c bigint, " + + "d decimal, " + + "e set<text>, " + + "f uuid, " + + "g vector<int, 2>, " + + "h list<float>, " + + "i timeuuid, " + + "j map<text, frozen<set<int>>>, " + + "PRIMARY KEY((a, b), c, d)) " + + "WITH CLUSTERING ORDER BY (c DESC, d ASC);"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); execute("INSERT INTO " + sourceKs + "." + sourceTb + " (a, b, c, d, e, f, g, h, i, j) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?)", @@ -207,15 +211,15 @@ public void testCopyAfterAlterTable() String targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); - alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " DROP d"); + alterTable("ALTER TABLE " + sourceKs + "." + sourceTb + " DROP d"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); - alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " ADD e uuid"); + alterTable("ALTER TABLE " + sourceKs + "." + sourceTb + " ADD e uuid"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); - alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " ADD f float"); + alterTable("ALTER TABLE " + sourceKs + "." + sourceTb + " ADD f float"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); @@ -226,15 +230,15 @@ public void testCopyAfterAlterTable() assertRows(execute("SELECT * FROM " + targetKs + "." + targetTb), row(2, "2", duration2, uuid2, f2)); - alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " DROP f USING TIMESTAMP 20000"); + alterTable("ALTER TABLE " + sourceKs + "." + sourceTb + " DROP f USING TIMESTAMP 20000"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); - alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " RENAME b TO bb "); + alterTable("ALTER TABLE " + sourceKs + "." + sourceTb + " RENAME b TO bb "); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); - alterTable("ALTER TABLE " + sourceKs + " ." + sourceTb + " WITH compaction = {'class':'LeveledCompactionStrategy', 'sstable_size_in_mb':10, 'fanout_size':16} "); + alterTable("ALTER TABLE " + sourceKs + "." + sourceTb + " WITH compaction = {'class':'LeveledCompactionStrategy', 'sstable_size_in_mb' : 10, 'fanout_size' : 16} "); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); @@ -244,7 +248,6 @@ public void testCopyAfterAlterTable() row(1, "1", duration1, uuid1)); assertRows(execute("SELECT * FROM " + targetKs + "." + targetTb), row(2, "2", duration2, uuid2)); - } @Test @@ -252,52 +255,52 @@ public void testTableOptionsCopy() throws Throwable { // compression String tbCompressionDefault1 = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b))"); - String tbCompressionDefault2 =createTable(sourceKs,"CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + - " WITH compression = { 'enabled' : 'false'};"); + String tbCompressionDefault2 = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + + " WITH compression = { 'enabled' : 'false'};"); String tbCompressionSnappy1 = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + - " WITH compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 32 };"); - String tbCompressionSnappy2 =createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + - " WITH compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 32, 'enabled' : true };"); - String tbCompressionSnappy3 = createTable(sourceKs,"CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + - " WITH compression = { 'class' : 'SnappyCompressor', 'min_compress_ratio' : 2 };"); - String tbCompressionSnappy4 = createTable(sourceKs,"CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + - " WITH compression = { 'class' : 'SnappyCompressor', 'min_compress_ratio' : 1 };"); - String tbCompressionSnappy5 = createTable(sourceKs,"CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + - " WITH compression = { 'class' : 'SnappyCompressor', 'min_compress_ratio' : 0 };"); + " WITH compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 32 };"); + String tbCompressionSnappy2 = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + + " WITH compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 32, 'enabled' : true };"); + String tbCompressionSnappy3 = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + + " WITH compression = { 'class' : 'SnappyCompressor', 'min_compress_ratio' : 2 };"); + String tbCompressionSnappy4 = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + + " WITH compression = { 'class' : 'SnappyCompressor', 'min_compress_ratio' : 1 };"); + String tbCompressionSnappy5 = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + + " WITH compression = { 'class' : 'SnappyCompressor', 'min_compress_ratio' : 0 };"); // memtable - String tableMemtableSkipList = createTable(sourceKs,"CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + - " WITH memtable = 'skiplist';"); - String tableMemtableTrie = createTable(sourceKs,"CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + - " WITH memtable = 'trie';"); - String tableMemtableDefault = createTable(sourceKs,"CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + - " WITH memtable = 'default';"); + String tableMemtableSkipList = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + + " WITH memtable = 'skiplist';"); + String tableMemtableTrie = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + + " WITH memtable = 'trie';"); + String tableMemtableDefault = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b))" + + " WITH memtable = 'default';"); // compaction - String tableCompactionStcs = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b)) WITH compaction = {'class':'SizeTieredCompactionStrategy', 'min_threshold':2, 'enabled':false};"); - String tableCompactionLcs =createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b)) WITH compaction = {'class':'LeveledCompactionStrategy', 'sstable_size_in_mb':1, 'fanout_size':5};"); - String tableCompactionTwcs = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b)) WITH compaction = {'class':'TimeWindowCompactionStrategy', 'min_threshold':2};"); - String tableCompactionUcs =createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b)) WITH compaction = {'class':'UnifiedCompactionStrategy'};"); + String tableCompactionStcs = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b)) WITH compaction = {'class' : 'SizeTieredCompactionStrategy', 'min_threshold' : 2, 'enabled' : false};"); + String tableCompactionLcs = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b)) WITH compaction = {'class' : 'LeveledCompactionStrategy', 'sstable_size_in_mb' : 1, 'fanout_size' : 5};"); + String tableCompactionTwcs = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b)) WITH compaction = {'class' : 'TimeWindowCompactionStrategy', 'min_threshold' : 2};"); + String tableCompactionUcs = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b)) WITH compaction = {'class' : 'UnifiedCompactionStrategy'};"); // other options are all different from default String tableOtherOptions = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b)) WITH" + - " additional_write_policy = '95p' " + - " AND bloom_filter_fp_chance = 0.1 " + - " AND caching = {'keys': 'ALL', 'rows_per_partition': '100'}" + - " AND cdc = true " + - " AND comment = 'test for create like'" + - " AND crc_check_chance = 0.1" + - " AND default_time_to_live = 10" + - " AND compaction = {'class':'UnifiedCompactionStrategy'} " + - " AND compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 32 }" + - " AND gc_grace_seconds = 100" + - " AND incremental_backups = false" + - " AND max_index_interval = 1024" + - " AND min_index_interval = 64" + - " AND speculative_retry = '95p'" + - " AND read_repair = 'NONE'" + - " AND memtable_flush_period_in_ms = 360000" + - " AND memtable = 'default';" ); + " additional_write_policy = '95p' " + + " AND bloom_filter_fp_chance = 0.1 " + + " AND caching = {'keys' : 'ALL', 'rows_per_partition' : '100'}" + + " AND cdc = true " + + " AND comment = 'test for create like'" + + " AND crc_check_chance = 0.1" + + " AND default_time_to_live = 10" + + " AND compaction = {'class' : 'UnifiedCompactionStrategy'} " + + " AND compression = {'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 32 }" + + " AND gc_grace_seconds = 100" + + " AND incremental_backups = false" + + " AND max_index_interval = 1024" + + " AND min_index_interval = 64" + + " AND speculative_retry = '95p'" + + " AND read_repair = 'NONE'" + + " AND memtable_flush_period_in_ms = 360000" + + " AND memtable = 'default';"); String tbLikeCompressionDefault1 = createTableLike("CREATE TABLE %s LIKE %s", tbCompressionDefault1, sourceKs, targetKs); String tbLikeCompressionDefault2 = createTableLike("CREATE TABLE %s LIKE %s", tbCompressionDefault2, sourceKs, targetKs); @@ -313,7 +316,7 @@ public void testTableOptionsCopy() throws Throwable String tbLikeCompactionLcs = createTableLike("CREATE TABLE %s LIKE %s", tableCompactionLcs, sourceKs, targetKs); String tbLikeCompactionTwcs = createTableLike("CREATE TABLE %s LIKE %s", tableCompactionTwcs, sourceKs, targetKs); String tbLikeCompactionUcs = createTableLike("CREATE TABLE %s LIKE %s", tableCompactionUcs, sourceKs, targetKs); - String tbLikeCompactionOthers= createTableLike("CREATE TABLE %s LIKE %s", tableOtherOptions, sourceKs, targetKs); + String tbLikeCompactionOthers = createTableLike("CREATE TABLE %s LIKE %s", tableOtherOptions, sourceKs, targetKs); assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tbCompressionDefault1, tbLikeCompressionDefault1); assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tbCompressionDefault2, tbLikeCompressionDefault2); @@ -332,27 +335,28 @@ public void testTableOptionsCopy() throws Throwable assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tableOtherOptions, tbLikeCompactionOthers); // a copy of the table with the table parameters set - String tableCopyAndSetCompression = createTableLike("CREATE TABLE %s LIKE %s WITH compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 64 };", + String tableCopyAndSetCompression = createTableLike("CREATE TABLE %s LIKE %s WITH compression = {'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 64 };", tbCompressionSnappy1, sourceKs, targetKs); - String tableCopyAndSetLCSCompaction = createTableLike("CREATE TABLE %s LIKE %s WITH compaction = {'class':'LeveledCompactionStrategy', 'sstable_size_in_mb':10, 'fanout_size':16};", - tableCompactionLcs, sourceKs, targetKs); + String tableCopyAndSetLCSCompaction = createTableLike("CREATE TABLE %s LIKE %s WITH compaction = {'class' : 'LeveledCompactionStrategy', 'sstable_size_in_mb' : 10, 'fanout_size' : 16};", + tableCompactionLcs, sourceKs, targetKs); String tableCopyAndSetAllParams = createTableLike("CREATE TABLE %s (a text, b int, c int, primary key (a, b)) WITH" + - " bloom_filter_fp_chance = 0.75 " + - " AND caching = {'keys': 'NONE', 'rows_per_partition': '10'}" + - " AND cdc = true " + - " AND comment = 'test for create like and set params'" + - " AND crc_check_chance = 0.8" + - " AND default_time_to_live = 100" + - " AND compaction = {'class':'SizeTieredCompactionStrategy'} " + - " AND compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 64 }" + - " AND gc_grace_seconds = 1000" + - " AND incremental_backups = true" + - " AND max_index_interval = 128" + - " AND min_index_interval = 16" + - " AND speculative_retry = '96p'" + - " AND read_repair = 'NONE'" + - " AND memtable_flush_period_in_ms = 3600;", - tableOtherOptions, sourceKs, targetKs); + " bloom_filter_fp_chance = 0.75 " + + " AND caching = {'keys' : 'NONE', 'rows_per_partition' : '10'}" + + " AND cdc = true " + + " AND comment = 'test for create like and set params'" + + " AND crc_check_chance = 0.8" + + " AND default_time_to_live = 100" + + " AND compaction = {'class' : 'SizeTieredCompactionStrategy'} " + + " AND compression = {'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 64}" + + " AND gc_grace_seconds = 1000" + + " AND incremental_backups = true" + + " AND max_index_interval = 128" + + " AND min_index_interval = 16" + + " AND speculative_retry = '96p'" + + " AND read_repair = 'NONE'" + + " AND memtable_flush_period_in_ms = 3600;", + tableOtherOptions, sourceKs, targetKs); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tbCompressionDefault1, tableCopyAndSetCompression, false, false, false); assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tableCompactionLcs, tableCopyAndSetLCSCompaction, false, false, false); assertTableMetaEqualsWithoutKs(sourceKs, targetKs, tableOtherOptions, tableCopyAndSetAllParams, false, false, false); @@ -364,29 +368,29 @@ public void testTableOptionsCopy() throws Throwable assertEquals(paramsSetLCSCompaction, TableParams.builder().compaction(CompactionParams.create(LeveledCompactionStrategy.class, Map.of("sstable_size_in_mb", "10", "fanout_size", "16"))) - .build()); + .build()); assertEquals(paramsSetAllParams, TableParams.builder().bloomFilterFpChance(0.75) - .caching(new CachingParams(false, 10)) - .cdc(true) - .comment("test for create like and set params") - .crcCheckChance(0.8) - .defaultTimeToLive(100) - .compaction(CompactionParams.stcs(Collections.emptyMap())) - .compression(CompressionParams.snappy(64 * 1024, 0.0)) - .gcGraceSeconds(1000) - .incrementalBackups(true) - .maxIndexInterval(128) - .minIndexInterval(16) - .speculativeRetry(SpeculativeRetryPolicy.fromString("96PERCENTILE")) - .readRepair(ReadRepairStrategy.NONE) - .memtableFlushPeriodInMs(3600) - .build()); + .caching(new CachingParams(false, 10)) + .cdc(true) + .comment("test for create like and set params") + .crcCheckChance(0.8) + .defaultTimeToLive(100) + .compaction(CompactionParams.stcs(Collections.emptyMap())) + .compression(CompressionParams.snappy(64 * 1024, 0.0)) + .gcGraceSeconds(1000) + .incrementalBackups(true) + .maxIndexInterval(128) + .minIndexInterval(16) + .speculativeRetry(SpeculativeRetryPolicy.fromString("96PERCENTILE")) + .readRepair(ReadRepairStrategy.NONE) + .memtableFlushPeriodInMs(3600) + .build()); // table id TableId id = TableId.generate(); String tbNormal = createTable(sourceKs, "CREATE TABLE %s (a text, b int, c int, primary key (a, b))"); assertInvalidThrowMessage("Cannot alter table id.", ConfigurationException.class, - "CREATE TABLE " + targetKs + ".targetnormal LIKE " + sourceKs + "." + tbNormal + " WITH ID = " + id); + "CREATE TABLE " + targetKs + ".targetnormal LIKE " + sourceKs + "." + tbNormal + " WITH ID = " + id); } @Test @@ -401,7 +405,7 @@ public void testStaticColumnCopy() // add static column sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b))"); - alterTable("ALTER TABLE " + sourceKs + "." + sourceTb + " ADD d int static"); + alterTable("ALTER TABLE " + sourceKs + "." + sourceTb + " ADD d int static"); targetTb = createTableLike("CREATE TABLE %s LIKE %s", sourceTb, sourceKs, targetKs); assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb); } @@ -486,35 +490,35 @@ public void testUDTTableCopy() throws Throwable // source table's column's data type is udt, and its subtypes are native type and udt String sourceTbUdtFrozen = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b duration, c " + udtFrozen + ");"); // source table's column's data type is udt, and its subtypes are native type and more than one udt - String sourceTbUdtComb = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b duration, c " + udtFrozen + ", d " + udt+ ");"); + String sourceTbUdtComb = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b duration, c " + udtFrozen + ", d " + udt + ");"); // source table's column's data type is udt, and its subtypes are native type and more than one udt - String sourceTbUdtCombNotExist = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b duration, c " + udtFrozen + ", d " + udtFrozenNotExist+ ");"); + String sourceTbUdtCombNotExist = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b duration, c " + udtFrozen + ", d " + udtFrozenNotExist + ");"); if (differentKs) { - assertInvalidThrowMessage("UDTs " + udt + " do not exist in target keyspace '" + targetKs +"'.", + assertInvalidThrowMessage("UDTs " + udt + " do not exist in target keyspace '" + targetKs + "'.", InvalidRequestException.class, "CREATE TABLE " + targetKs + ".tbudt LIKE " + sourceKs + "." + sourceTbUdt); - assertInvalidThrowMessage("UDTs " + udtSet + " do not exist in target keyspace '" + targetKs +"'.", + assertInvalidThrowMessage("UDTs " + udtSet + " do not exist in target keyspace '" + targetKs + "'.", InvalidRequestException.class, "CREATE TABLE " + targetKs + ".tbdtset LIKE " + sourceKs + "." + sourceTbUdtSet); assertInvalidThrowMessage(String.format("UDTs %s do not exist in target keyspace '%s'.", Sets.newHashSet(udt, udtFrozen).stream().sorted().collect(Collectors.joining(", ")), targetKs), InvalidRequestException.class, - "CREATE TABLE " + targetKs + ".tbudtfrozen LIKE " + sourceKs + "." + sourceTbUdtFrozen); + "CREATE TABLE " + targetKs + ".tbudtfrozen LIKE " + sourceKs + "." + sourceTbUdtFrozen); assertInvalidThrowMessage(String.format("UDTs %s do not exist in target keyspace '%s'.", Sets.newHashSet(udt, udtFrozen).stream().sorted().collect(Collectors.joining(", ")), targetKs), InvalidRequestException.class, "CREATE TABLE " + targetKs + ".tbudtfrozen LIKE " + sourceKs + "." + sourceTbUdtFrozen); assertInvalidThrowMessage(String.format("UDTs %s do not exist in target keyspace '%s'.", Sets.newHashSet(udt, udtFrozen).stream().sorted().collect(Collectors.joining(", ")), targetKs), InvalidRequestException.class, "CREATE TABLE " + targetKs + ".tbudtcomb LIKE " + sourceKs + "." + sourceTbUdtComb); - assertInvalidThrowMessage(String.format("UDTs %s do not exist in target keyspace '%s'.", Sets.newHashSet(udtNew, udt,udtFrozenNotExist, udtFrozen).stream().sorted().collect(Collectors.joining(", ")), targetKs), - InvalidRequestException.class, - "CREATE TABLE " + targetKs + ".tbudtcomb LIKE " + sourceKs + "." + sourceTbUdtCombNotExist); + assertInvalidThrowMessage(String.format("UDTs %s do not exist in target keyspace '%s'.", Sets.newHashSet(udtNew, udt, udtFrozenNotExist, udtFrozen).stream().sorted().collect(Collectors.joining(", ")), targetKs), + InvalidRequestException.class, + "CREATE TABLE " + targetKs + ".tbudtcomb LIKE " + sourceKs + "." + sourceTbUdtCombNotExist); // different keyspaces with udts that have same udt name, different fields String udtWithDifferentField = createType(sourceKs, "CREATE TYPE %s (aa int, bb text)"); createType("CREATE TYPE IF NOT EXISTS " + targetKs + "." + udtWithDifferentField + " (aa int, cc text)"); String sourceTbDiffUdt = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b duration, c " + udtWithDifferentField + ");"); - assertInvalidThrowMessage("Target keyspace '" + targetKs + "' has same UDT name '"+ udtWithDifferentField +"' as source keyspace '" + sourceKs + "' but with different structure.", + assertInvalidThrowMessage("Target keyspace '" + targetKs + "' has same UDT name '" + udtWithDifferentField + "' as source keyspace '" + sourceKs + "' but with different structure.", InvalidRequestException.class, "CREATE TABLE " + targetKs + ".tbdiffudt LIKE " + sourceKs + "." + sourceTbDiffUdt); } @@ -576,28 +580,138 @@ public void testTriggerOperationOnCopiedTable() public void testUnSupportedSchema() throws Throwable { createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b int, c text)", "tb"); - String index = createIndex( "CREATE INDEX ON " + sourceKs + ".tb (c)"); + String index = createIndex("CREATE INDEX ON " + sourceKs + ".tb (c)"); assertInvalidThrowMessage("Souce Table '" + targetKs + "." + index + "' doesn't exist", InvalidRequestException.class, - "CREATE TABLE " + sourceKs + ".newtb LIKE " + targetKs + "." + index + ";"); - + "CREATE TABLE " + sourceKs + ".newtb LIKE " + targetKs + "." + index + ";"); assertInvalidThrowMessage("System keyspace 'system' is not user-modifiable", InvalidRequestException.class, "CREATE TABLE system.local_clone LIKE system.local ;"); assertInvalidThrowMessage("System keyspace 'system_views' is not user-modifiable", InvalidRequestException.class, "CREATE TABLE system_views.newtb LIKE system_views.snapshots ;"); } - private void assertTableMetaEqualsWithoutKs(String sourceKs, String targetKs, String sourceTb, String targetTb) + @Test + public void testTableCopyWithIndexes() + { + String sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b int, c text, d int, e text, f int, g text)", "sourcetb"); + createIndex(sourceKs, "CREATE INDEX ON %s (d)"); + createIndex(sourceKs, "CREATE INDEX ON %s (c)"); + createIndex(sourceKs, "CREATE INDEX ON %s (b) USING 'sai'"); + createIndex(sourceKs, "CREATE CUSTOM INDEX ON %s (e) USING 'storageattachedindex'"); + createIndex(sourceKs, "CREATE CUSTOM INDEX ON %s (f) USING 'org.apache.cassandra.index.sai.StorageAttachedIndex'"); + String targetTb = createTableLike("CREATE TABLE %s LIKE %s WITH INDEXES", sourceTb, sourceKs, targetKs); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb, true, true, true); + } + + @Test + public void testTableCopyWithMultiIndexOnSameColumn() + { + String sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b int, c text, d int, e text, f int, g text)", "sourcetb"); + createIndex(sourceKs, "CREATE INDEX " + sourceTb + "_b_idx1 ON %s (b) USING 'legacy_local_table'"); + createIndex(sourceKs, "CREATE INDEX " + sourceTb + "_b_idx2 ON %s (b) USING 'sai'"); + String targetTb = createTableLike("CREATE TABLE %s LIKE %s WITH INDEXES", sourceTb, sourceKs, targetKs); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb, true, true, true); + } + + @Test + public void testTableCopyWithOutIndexes() + { + String sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b int)", "sourcetb"); + // not copied + createIndex(sourceKs, "CREATE CUSTOM INDEX ON %s (b) USING 'org.apache.cassandra.index.sasi.SASIIndex'"); + createIndex(sourceKs, "CREATE CUSTOM INDEX testidx ON %s (b) USING '" + StubIndex.class.getName() + "'"); + String targetTb = createTableLike("CREATE TABLE %s LIKE %s WITH indexes", sourceTb, sourceKs, targetKs); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb, true, false, false); + assertEquals(2, getTableMetadata(sourceKs, sourceTb).indexes.size()); + assertEquals(0, getTableMetadata(targetKs, targetTb).indexes.size()); + } + + @Test + public void testManyTableCopyWithIndex() { + String sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b int, c int)", "sourcetb"); + createIndex(sourceKs, "CREATE INDEX myindex ON %s (b) USING 'legacy_local_table'"); + createIndex(sourceKs, "CREATE INDEX myindex_1 ON %s (b) USING 'sai'"); + createIndex(sourceKs, "CREATE INDEX myindex_1_1 ON %s (c) USING 'sai'"); + createIndex(sourceKs, "CREATE INDEX myindex__1 ON %s (c) USING 'legacy_local_table'"); + String targetTb = createTableLike("CREATE TABLE %s LIKE %s WITH indexes", sourceTb, sourceKs, targetKs); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb, true, false, false); + Set<String> resultIndexNames = getTableMetadata(targetKs, targetTb).indexes.stream().map(meta -> meta.name).collect(Collectors.toSet()); + Set<String> expectedIndexNames = differentKs ? Sets.newHashSet("myindex", "myindex_1", "myindex_1_1", "myindex__1") : + Sets.newHashSet("myindex_2", "myindex_3", "myindex_1_2", "myindex__2"); + assertEquals(0, Sets.difference(resultIndexNames, expectedIndexNames).size()); + } + + @Test + public void testTableCopyWithIndexesAndTableProperty() + { + String sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b int, c text, d int)", "sourcetb"); + createIndex(sourceKs, "CREATE INDEX ON %s (b)"); + + String targetTbWithAll = createTableLike("CREATE TABLE %s LIKE %s WITH INDEXES AND crc_check_chance = 0.8 AND cdc = true", sourceTb, sourceKs, targetKs); + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTbWithAll, false, true, true); + + TableMetadata source = getTableMetadata(sourceKs, sourceTb); + TableMetadata target = getTableMetadata(targetKs, targetTbWithAll); + assertNotEquals(source.params, target.params); + assertTrue(target.params.cdc); + assertFalse(source.params.cdc); + assertEquals(0.8, target.params.crcCheckChance, 0.0); + } + + @Test + public void testIndexesNameForCopiedTable() + { + String sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b int, c text, d int, e text)", "sourcetb"); + String idx1 = createIndex(sourceKs, "CREATE INDEX ON %s (d)"); + String idx2 = createIndex(sourceKs, "CREATE INDEX ON %s (c)"); + String idx3 = createIndex(sourceKs, "CREATE INDEX ON %s (b) USING 'sai'"); + String idx4 = createIndex(sourceKs, "CREATE INDEX idx_for_e_column ON %s (e) USING 'sai'"); + assertEquals(sourceTb + "_d_idx" , idx1); + assertEquals(sourceTb + "_c_idx" , idx2); + assertEquals(sourceTb + "_b_idx" , idx3); + assertEquals("idx_for_e_column", idx4); + String targetTb = createTableLike("CREATE TABLE %s LIKE %s WITH INDEXES", sourceTb, sourceKs, targetKs); assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb, true, true, true); + TableMetadata target = getTableMetadata(targetKs, targetTb); + assertTrue(target.indexes.has(targetTb + "_d_idx")); + assertTrue(target.indexes.has(targetTb + "_c_idx")); + assertTrue(target.indexes.has(targetTb + "_b_idx")); + + if (differentKs) + { + assertTrue(target.indexes.has("idx_for_e_column")); + } + else + { + // if within the same keyspace, the target table will be created with a new + // index name in the format oldindexname_number, where the number starts from 1. + assertTrue(target.indexes.has("idx_for_e_column_1")); + } + } + + @Test + public void testTableCopyWithCustomIndexes() + { + String sourceTb = createTable(sourceKs, "CREATE TABLE %s (a int PRIMARY KEY, b int, c text, d int)", "sourcetb"); + String idx = createIndex(sourceKs, "CREATE CUSTOM INDEX testidx ON %s (b) USING '" + StubIndex.class.getName() + "'"); + Set<String> customIndexes = new TreeSet<>(Arrays.asList(idx)); + ResultSet res = executeNet("CREATE TABLE " + targetKs + ".targettbb LIKE " + sourceKs + "." + sourceTb + " WITH INDEXES"); + assertWarningsContain(res.getExecutionInfo().getWarnings(), + "Source table " + sourceKs + "." + sourceTb + " to copy indexes from to " + targetKs + ".targettbb has custom indexes. These indexes were not copied: " + customIndexes); + } + + private void assertTableMetaEqualsWithoutKs(String sourceKs, String targetKs, String sourceTb, String targetTb) + { + assertTableMetaEqualsWithoutKs(sourceKs, targetKs, sourceTb, targetTb, true, true, false); } - private void assertTableMetaEqualsWithoutKs(String sourceKs, String targetKs, String sourceTb, String targetTb, boolean compareParams, boolean compareIndexes, boolean compareTrigger) + private void assertTableMetaEqualsWithoutKs(String sourceKs, String targetKs, String sourceTb, String targetTb, boolean compareParams, boolean compareIndexes, boolean compareIndexWithOutName) { TableMetadata left = getTableMetadata(sourceKs, sourceTb); TableMetadata right = getTableMetadata(targetKs, targetTb); assertNotNull(left); assertNotNull(right); - assertTrue(equalsWithoutTableNameAndDropCns(left, right, compareParams, compareIndexes, compareTrigger)); + assertTrue(equalsWithoutTableNameAndDropCns(left, right, compareParams, compareIndexes, compareIndexWithOutName)); assertNotEquals(left.id, right.id); assertNotEquals(left.name, right.name); } From 980047657dd724ead10d5b82609b7f3e8a0b8182 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Wed, 12 Feb 2025 14:03:26 +0100 Subject: [PATCH 151/225] Prevent invalid constraint combinations patch by Stefan Miklosovic; reviewed by Bernardo Botella for CASSANDRA-20330 --- CHANGES.txt | 1 + .../AbstractFunctionConstraint.java | 42 +++- ...AbstractFunctionSatisfiabilityChecker.java | 218 ++++++++++++++++++ .../cql3/constraints/ColumnConstraint.java | 52 ++++- .../cql3/constraints/ColumnConstraints.java | 61 ++++- .../cql3/constraints/ConstraintFunction.java | 20 ++ .../constraints/FunctionColumnConstraint.java | 55 +++-- .../cql3/constraints/LengthConstraint.java | 14 +- .../constraints/SatisfiabilityChecker.java | 45 ++++ .../constraints/ScalarColumnConstraint.java | 46 +++- .../UnaryFunctionColumnConstraint.java | 29 +-- .../schema/CreateTableStatement.java | 24 +- .../cassandra/schema/ColumnMetadata.java | 5 +- .../test/ColumnConstraintsTest.java | 5 +- .../ConstraintsSatisfiabilityTest.java | 180 +++++++++++++++ 15 files changed, 717 insertions(+), 80 deletions(-) create mode 100644 src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionSatisfiabilityChecker.java create mode 100644 src/java/org/apache/cassandra/cql3/constraints/SatisfiabilityChecker.java create mode 100644 test/unit/org/apache/cassandra/contraints/ConstraintsSatisfiabilityTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 269b8a8f45a5..5827b806256d 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Prevent invalid constraint combinations (CASSANDRA-20330) * Support CREATE TABLE LIKE WITH INDEXES (CASSANDRA-19965) * Invalidate relevant prepared statements on every change to TableMetadata (CASSANDRA-20318) * Add per type max size guardrails (CASSANDRA-19677) diff --git a/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionConstraint.java index 942d884a5e57..bbfe58fb26a7 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionConstraint.java @@ -18,14 +18,52 @@ package org.apache.cassandra.cql3.constraints; +import java.util.Set; + import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.CqlBuilder; +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.utils.LocalizeString; public abstract class AbstractFunctionConstraint<T> extends ColumnConstraint<T> { - public AbstractFunctionConstraint(ColumnIdentifier columnName) + protected final Operator relationType; + protected final String term; + + public AbstractFunctionConstraint(ColumnIdentifier columnName, Operator relationType, String term) { super(columnName); + this.relationType = relationType; + this.term = term; + } + + public Operator relationType() + { + return relationType; + } + + public String term() + { + return term; } - public abstract String name(); + public abstract Set<Operator> getSupportedOperators(); + + @Override + public void appendCqlTo(CqlBuilder builder) + { + builder.append(toString()); + } + + public static <T extends Enum<T>> T getEnum(Class<T> enumClass, String functionName) + { + try + { + return Enum.valueOf(enumClass, LocalizeString.toUpperCaseLocalized(functionName)); + } + catch (IllegalArgumentException e) + { + throw new InvalidConstraintDefinitionException("Unrecognized constraint function: " + functionName); + } + } } diff --git a/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionSatisfiabilityChecker.java b/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionSatisfiabilityChecker.java new file mode 100644 index 000000000000..33f261535c91 --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionSatisfiabilityChecker.java @@ -0,0 +1,218 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.constraints; + +import java.nio.ByteBuffer; +import java.util.LinkedList; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; + +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.utils.Pair; + +import static java.lang.String.format; +import static org.apache.cassandra.cql3.Operator.EQ; +import static org.apache.cassandra.cql3.Operator.GT; +import static org.apache.cassandra.cql3.Operator.GTE; +import static org.apache.cassandra.cql3.Operator.LT; +import static org.apache.cassandra.cql3.Operator.LTE; +import static org.apache.cassandra.cql3.Operator.NEQ; +import static org.apache.cassandra.cql3.constraints.ColumnConstraint.ConstraintType.FUNCTION; + +public abstract class AbstractFunctionSatisfiabilityChecker<CONSTRAINT_TYPE extends AbstractFunctionConstraint<CONSTRAINT_TYPE>> +{ + /** + * Performs check if constraints are satisfiable or not. + * + * @param functionName name of function + * @param constraints list of constraints to set + * @param columnMetadata metadata of a column. + */ + public void check(String functionName, List<ColumnConstraint<?>> constraints, ColumnMetadata columnMetadata) + { + Pair<List<CONSTRAINT_TYPE>, List<CONSTRAINT_TYPE>> filteredConstraints = categorizeConstraints(constraints, functionName); + + if (filteredConstraints.left.isEmpty()) + return; + + checkNumberOfConstraints(columnMetadata, filteredConstraints); + checkSupportedOperators(filteredConstraints.left, functionName); + ensureSatisfiability(columnMetadata, functionName, filteredConstraints.left); + } + + /** + * Categorizes given constraints into two lists. The first list, the left one in Pair, contains all + * constraints of implementation-specific {@link org.apache.cassandra.cql3.constraints.ColumnConstraint.ConstraintType}. + * The second list, the right one in Pair, contains all constraints of such constraint type which do have "not equal" operator. + * + * @param constraints constraints to categorize + * @param functionName name of function + * @return pair of categorized constraints + */ + abstract Pair<List<CONSTRAINT_TYPE>, List<CONSTRAINT_TYPE>> categorizeConstraints(List<ColumnConstraint<?>> constraints, String functionName); + + private void checkSupportedOperators(List<CONSTRAINT_TYPE> allConstraints, String functionName) + { + for (CONSTRAINT_TYPE constraint : allConstraints) + { + if (!constraint.getSupportedOperators().contains(constraint.relationType())) + throw new InvalidConstraintDefinitionException(format("%s constraint of relation '%s' is not supported. Only these are: %s", + functionName, + constraint.relationType(), + constraint.getSupportedOperators())); + } + } + + /** + * Checks if there are no duplicate constraints having same operator. + * + * @param columnMetadata medata of a column + * @param filteredConstraints pair of all constraints and all constraints having not-equal operator + */ + private void checkNumberOfConstraints(ColumnMetadata columnMetadata, Pair<List<CONSTRAINT_TYPE>, List<CONSTRAINT_TYPE>> filteredConstraints) + { + List<? extends AbstractFunctionConstraint<CONSTRAINT_TYPE>> allConstraints = filteredConstraints.left; + List<? extends AbstractFunctionConstraint<CONSTRAINT_TYPE>> notEqualConstraints = filteredConstraints.right; + + if ((allConstraints.size() - notEqualConstraints.size() > 2)) + { + throw new InvalidConstraintDefinitionException(format("There can not be more than 2 constraints (not including non-equal relations) on a column '%s' but you have specified %s", + columnMetadata.name, + allConstraints.size())); + } + + if (notEqualConstraints.size() > 1) + { + Set<String> uniqueTerms = new TreeSet<>(); + for (AbstractFunctionConstraint<CONSTRAINT_TYPE> notEqual : notEqualConstraints) + { + if (!uniqueTerms.add(notEqual.term())) + throw new InvalidConstraintDefinitionException(format("There are duplicate constraint definitions on column '%s': %s", + columnMetadata.name, + notEqual)); + } + } + } + + private void ensureSatisfiability(ColumnMetadata columnMetadata, + String constraintName, + List<CONSTRAINT_TYPE> allConstraints) + { + if (allConstraints.size() != 2) + return; + + Operator firstRelation = allConstraints.get(0).relationType(); + String firstTerm = allConstraints.get(0).term(); + Operator secondRelation = allConstraints.get(1).relationType(); + String secondTerm = allConstraints.get(1).term(); + + if ((firstRelation == GT && secondRelation == GTE) || + (firstRelation == GTE && secondRelation == GT) || + (firstRelation == LT && secondRelation == LTE) || + (firstRelation == LTE && secondRelation == LT) || + (firstRelation == EQ || secondRelation == EQ)) + { + throw new InvalidConstraintDefinitionException(format("Constraints combination of %s is not supported: %s %s %s, %s %s %s", + constraintName, + columnMetadata.name, + firstRelation, + firstTerm, + columnMetadata.name, + secondRelation, + secondTerm)); + } + else if (firstRelation == NEQ && secondRelation == NEQ) + { + if (firstTerm.equals(secondTerm)) + throw new InvalidConstraintDefinitionException(format("There are duplicate constraint definitions on column '%s'.", columnMetadata.name)); + } + else + { + ByteBuffer firstTermBuffer = columnMetadata.type.fromString(firstTerm); + ByteBuffer secondTermBuffer = columnMetadata.type.fromString(secondTerm); + + boolean firstSatisfaction = firstRelation.isSatisfiedBy(columnMetadata.type, secondTermBuffer, firstTermBuffer); + boolean secondSatisfaction = secondRelation.isSatisfiedBy(columnMetadata.type, firstTermBuffer, secondTermBuffer); + + if (!firstSatisfaction || !secondSatisfaction) + throw new InvalidConstraintDefinitionException(format("Constraints of %s are not satisfiable: %s %s %s, %s %s %s", + constraintName, + columnMetadata.name, + firstRelation, + firstTerm, + columnMetadata.name, + secondRelation, + secondTerm)); + } + } + + public static final AbstractFunctionSatisfiabilityChecker<ScalarColumnConstraint> SCALAR_SATISFIABILITY_CHECKER = new AbstractFunctionSatisfiabilityChecker<>() + { + @Override + public Pair<List<ScalarColumnConstraint>, List<ScalarColumnConstraint>> categorizeConstraints(List<ColumnConstraint<?>> constraints, String functionName) + { + List<ScalarColumnConstraint> scalars = new LinkedList<>(); + List<ScalarColumnConstraint> notEqualScalars = new LinkedList<>(); + + for (ColumnConstraint<?> columnConstraint : constraints) + { + if (columnConstraint.getConstraintType() == ColumnConstraint.ConstraintType.SCALAR) + { + ScalarColumnConstraint scalarColumnConstraint = (ScalarColumnConstraint) columnConstraint; + scalars.add(scalarColumnConstraint); + if (scalarColumnConstraint.relationType() == NEQ) + notEqualScalars.add(scalarColumnConstraint); + } + } + + return Pair.create(scalars, notEqualScalars); + } + }; + + public static final AbstractFunctionSatisfiabilityChecker<FunctionColumnConstraint> FUNCTION_SATISFIABILITY_CHECKER = new AbstractFunctionSatisfiabilityChecker<>() + { + @Override + public Pair<List<FunctionColumnConstraint>, List<FunctionColumnConstraint>> categorizeConstraints(List<ColumnConstraint<?>> constraints, String functionName) + { + List<FunctionColumnConstraint> funnctionColumnConstraints = new LinkedList<>(); + List<FunctionColumnConstraint> notEqualConstraints = new LinkedList<>(); + + for (ColumnConstraint<?> columnConstraint : constraints) + { + if (columnConstraint.getConstraintType() != FUNCTION) + continue; + + FunctionColumnConstraint functionColumnConstraint = (FunctionColumnConstraint) columnConstraint; + + ConstraintFunction function = functionColumnConstraint.function(); + + if (!function.name.equals(functionName)) + continue; + + funnctionColumnConstraints.add(functionColumnConstraint); + if (functionColumnConstraint.relationType() == NEQ) + notEqualConstraints.add(functionColumnConstraint); + } + + return Pair.create(funnctionColumnConstraints, notEqualConstraints); + } + }; +} diff --git a/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraint.java index bba70d1f8726..9733fccdccfc 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraint.java @@ -19,12 +19,17 @@ package org.apache.cassandra.cql3.constraints; import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.CqlBuilder; +import org.apache.cassandra.cql3.constraints.ColumnConstraints.DuplicatesChecker; +import org.apache.cassandra.cql3.constraints.ScalarColumnConstraint.ScalarColumnConstraintSatisfiabilityChecker; import org.apache.cassandra.db.marshal.AbstractType; -import org.apache.cassandra.tcm.serialization.MetadataSerializer; import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.tcm.serialization.MetadataSerializer; /** * Common class for the conditions that a CQL Constraint needs to implement to be integrated in the @@ -46,32 +51,65 @@ public enum ConstraintType // The order of that enum matters!! // We are serializing its enum position instead of its name. // Changing this enum would affect how that int is interpreted when deserializing. - COMPOSED(ColumnConstraints.serializer), - FUNCTION(FunctionColumnConstraint.serializer), - SCALAR(ScalarColumnConstraint.serializer), - UNARY_FUNCTION(UnaryFunctionColumnConstraint.serializer); + COMPOSED(ColumnConstraints.serializer, new DuplicatesChecker()), + FUNCTION(FunctionColumnConstraint.serializer, FunctionColumnConstraint.Functions.values()), + SCALAR(ScalarColumnConstraint.serializer, new ScalarColumnConstraintSatisfiabilityChecker()), + UNARY_FUNCTION(UnaryFunctionColumnConstraint.serializer, UnaryFunctionColumnConstraint.Functions.values()); private final MetadataSerializer<?> serializer; + private final SatisfiabilityChecker[] satisfiabilityCheckers; + + ConstraintType(MetadataSerializer<?> serializer, SatisfiabilityChecker satisfiabilityChecker) + { + this(serializer, new SatisfiabilityChecker[]{ satisfiabilityChecker }); + } - ConstraintType(MetadataSerializer<?> serializer) + ConstraintType(MetadataSerializer<?> serializer, SatisfiabilityChecker[] satisfiabilityCheckers) { this.serializer = serializer; + this.satisfiabilityCheckers = satisfiabilityCheckers; } public static MetadataSerializer<?> getSerializer(int i) { return ConstraintType.values()[i].serializer; } + + public static SatisfiabilityChecker[] getSatisfiabilityCheckers() + { + List<SatisfiabilityChecker> result = new ArrayList<>(); + for (ConstraintType constraintType : ConstraintType.values()) + result.addAll(Arrays.asList(constraintType.satisfiabilityCheckers)); + + return result.toArray(new SatisfiabilityChecker[0]); + } + } + + public abstract String name(); + + /** + * Typically includes name of a constraint as in {@link #name()}, + * plus an operator of a function, if constraint is a function. + * Full name serves as String which uniquely distinguishes two constraints even of same names for the purpose + * of checking if there is a specific constraint used twice. A duplicit usage of a constraint is illegal. + * + * @return full name of a constraint, with an operator. + */ + public String fullName() + { + return name(); } public abstract MetadataSerializer<T> serializer(); public abstract void appendCqlTo(CqlBuilder builder); + public abstract boolean enablesDuplicateDefinitions(String name); + /** * Method that evaluates the condition. It can either succeed or throw a {@link ConstraintViolationException}. * - * @param valueType value type of the column value under test + * @param valueType value type of the column value under test * @param columnValue Column value to be evaluated at write time */ public void evaluate(AbstractType<?> valueType, ByteBuffer columnValue) throws ConstraintViolationException diff --git a/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraints.java b/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraints.java index 7efe046b1615..02571976e12b 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraints.java +++ b/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraints.java @@ -24,8 +24,9 @@ import java.util.Collections; import java.util.List; import java.util.Objects; +import java.util.Set; +import java.util.TreeSet; -import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.CqlBuilder; import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.db.marshal.AbstractType; @@ -35,6 +36,8 @@ import org.apache.cassandra.tcm.serialization.MetadataSerializer; import org.apache.cassandra.tcm.serialization.Version; +import static java.lang.String.format; + // group of constraints for the column public class ColumnConstraints extends ColumnConstraint<ColumnConstraints> { @@ -49,6 +52,12 @@ public ColumnConstraints(List<ColumnConstraint<?>> constraints) this.constraints = constraints; } + @Override + public String name() + { + return getConstraintType().name(); + } + @Override public MetadataSerializer<ColumnConstraints> serializer() { @@ -62,6 +71,12 @@ public void appendCqlTo(CqlBuilder builder) constraint.appendCqlTo(builder); } + @Override + public boolean enablesDuplicateDefinitions(String name) + { + return false; + } + @Override public void evaluate(AbstractType<?> valueType, ByteBuffer columnValue) throws ConstraintViolationException { @@ -101,11 +116,6 @@ public boolean hasRelevantConstraints() return false; } - public void checkInvalidConstraintsCombinations(ColumnIdentifier columnName) - { - // TODO check duplicities etc CASSANDRA-20330 - } - @Override public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException { @@ -114,6 +124,12 @@ public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefi + columnMetadata.name + " of type " + columnMetadata.type.asCQL3Type() + " for the table " + columnMetadata.ksName + "." + columnMetadata.cfName); + // this will look at constraints as a whole, + // checking if combinations of a particular constraint make sense (duplicities, satisfiability etc.). + for (SatisfiabilityChecker satisfiabilityChecker : ConstraintType.getSatisfiabilityCheckers()) + satisfiabilityChecker.checkSatisfiability(constraints, columnMetadata); + + // this validation will check whether it makes sense to execute such constraint on a given column for (ColumnConstraint<?> constraint : constraints) constraint.validate(columnMetadata); } @@ -124,6 +140,33 @@ public ConstraintType getConstraintType() return ConstraintType.COMPOSED; } + public static class DuplicatesChecker implements SatisfiabilityChecker + { + @Override + public void checkSatisfiability(List<ColumnConstraint<?>> constraints, ColumnMetadata columnMetadata) + { + Set<String> constraintNames = new TreeSet<>(); + List<String> duplicateConstraints = new ArrayList<>(); + + for (ColumnConstraint<?> constraint : constraints) + { + String constraintFullName = constraint.fullName(); + String constraintName = constraint.name(); + + if (!constraintNames.add(constraintFullName)) + { + if (!constraint.enablesDuplicateDefinitions(constraintName)) + duplicateConstraints.add(constraintFullName); + } + } + + if (!duplicateConstraints.isEmpty()) + throw new InvalidConstraintDefinitionException(format("There are duplicate constraint definitions on column '%s': %s", + columnMetadata.name, + duplicateConstraints)); + } + } + private static class Noop extends ColumnConstraints { private Noop() @@ -136,6 +179,12 @@ public void validate(ColumnMetadata columnMetadata) { // Do nothing. It is always valid } + + @Override + public String name() + { + return "NO_OP"; + } } public final static class Raw diff --git a/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java b/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java index cdc22f2bd486..bdd36d11818f 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java +++ b/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java @@ -19,17 +19,27 @@ package org.apache.cassandra.cql3.constraints; import java.nio.ByteBuffer; +import java.util.Set; import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.Operator; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.schema.ColumnMetadata; +import static org.apache.cassandra.cql3.Operator.EQ; +import static org.apache.cassandra.cql3.Operator.GT; +import static org.apache.cassandra.cql3.Operator.GTE; +import static org.apache.cassandra.cql3.Operator.LT; +import static org.apache.cassandra.cql3.Operator.LTE; +import static org.apache.cassandra.cql3.Operator.NEQ; + /** * Interface to be implemented by functions that are executed as part of CQL constraints. */ public abstract class ConstraintFunction { + public static final Set<Operator> DEFAULT_FUNCTION_OPERATORS = Set.of(EQ, NEQ, GTE, GT, LTE, LT); + protected final ColumnIdentifier columnName; protected final String name; @@ -70,4 +80,14 @@ public void evaluate(AbstractType<?> valueType, ByteBuffer columnValue) throws C * if the CQL statement is valid or needs to be rejected as invalid throwing a {@link InvalidConstraintDefinitionException} */ public abstract void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException; + + /** + * Return operators this function supports. By default, it returns an empty set, modelling unary function. + * + * @return set of operators this function is allowed to have. + */ + public Set<Operator> getSupportedOperators() + { + return Set.of(); + } } diff --git a/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java index a1a3339c434b..02f49986b552 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java @@ -20,10 +20,11 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.List; +import java.util.Set; import java.util.function.Function; import org.apache.cassandra.cql3.ColumnIdentifier; -import org.apache.cassandra.cql3.CqlBuilder; import org.apache.cassandra.cql3.Operator; import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.db.marshal.AbstractType; @@ -32,15 +33,15 @@ import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.tcm.serialization.MetadataSerializer; import org.apache.cassandra.tcm.serialization.Version; -import org.apache.cassandra.utils.LocalizeString; + +import static org.apache.cassandra.cql3.Operator.NEQ; +import static org.apache.cassandra.cql3.constraints.AbstractFunctionSatisfiabilityChecker.FUNCTION_SATISFIABILITY_CHECKER; public class FunctionColumnConstraint extends AbstractFunctionConstraint<FunctionColumnConstraint> { public static final Serializer serializer = new Serializer(); private final ConstraintFunction function; - private final Operator relationType; - private final String term; public final static class Raw { @@ -63,9 +64,16 @@ public FunctionColumnConstraint prepare() } } - private enum Functions + public enum Functions implements SatisfiabilityChecker { - LENGTH(LengthConstraint::new); + LENGTH(LengthConstraint::new) + { + @Override + public void checkSatisfiability(List<ColumnConstraint<?>> constraints, ColumnMetadata columnMetadata) + { + FUNCTION_SATISFIABILITY_CHECKER.check(name(), constraints, columnMetadata); + } + }; private final Function<ColumnIdentifier, ConstraintFunction> functionCreator; @@ -77,33 +85,42 @@ private enum Functions private static ConstraintFunction createConstraintFunction(String functionName, ColumnIdentifier columnName) { - try - { - return Functions.valueOf(LocalizeString.toUpperCaseLocalized(functionName)).functionCreator.apply(columnName); - } - catch (IllegalArgumentException ex) - { - throw new InvalidConstraintDefinitionException("Unrecognized constraint function: " + functionName); - } + return getEnum(Functions.class, functionName).functionCreator.apply(columnName); } private FunctionColumnConstraint(ConstraintFunction function, ColumnIdentifier columnName, Operator relationType, String term) { - super(columnName); + super(columnName, relationType, term); this.function = function; - this.relationType = relationType; - this.term = term; } + public ConstraintFunction function() + { + return function; + } + + @Override + public Set<Operator> getSupportedOperators() + { + return function.getSupportedOperators(); + } + + @Override public String name() { return function.name; } @Override - public void appendCqlTo(CqlBuilder builder) + public String fullName() + { + return function.name + ' ' + relationType; + } + + @Override + public boolean enablesDuplicateDefinitions(String name) { - builder.append(toString()); + return relationType == NEQ; } @Override diff --git a/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java index 8aa9aaf99d11..708028a6eadf 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java @@ -19,6 +19,7 @@ package org.apache.cassandra.cql3.constraints; import java.nio.ByteBuffer; +import java.util.Set; import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.Operator; @@ -32,11 +33,12 @@ public class LengthConstraint extends ConstraintFunction { - private static final AbstractType<?>[] SUPPORTED_TYPES = new AbstractType[] { BytesType.instance, UTF8Type.instance, AsciiType.instance }; + private static final String NAME = "LENGTH"; + private static final AbstractType<?>[] SUPPORTED_TYPES = new AbstractType[]{ BytesType.instance, UTF8Type.instance, AsciiType.instance }; public LengthConstraint(ColumnIdentifier columnName) { - super(columnName, "LENGTH"); + super(columnName, NAME); } @Override @@ -50,7 +52,7 @@ public void internalEvaluate(AbstractType<?> valueType, Operator relationType, S if (!relationType.isSatisfiedBy(Int32Type.instance, leftOperand, rightOperand)) throw new ConstraintViolationException("Column value does not satisfy value constraint for column '" + columnName + "'. " - + "It has a length of " + valueLength + " and it should be should be " + + "It has a length of " + valueLength + " and it should be " + relationType + ' ' + term); } @@ -72,6 +74,12 @@ public void validate(ColumnMetadata columnMetadata) throw invalidConstraintDefinitionException(columnMetadata.type); } + @Override + public Set<Operator> getSupportedOperators() + { + return DEFAULT_FUNCTION_OPERATORS; + } + private int getValueLength(ByteBuffer value, AbstractType<?> valueType) { if (valueType.getClass() == BytesType.class) diff --git a/src/java/org/apache/cassandra/cql3/constraints/SatisfiabilityChecker.java b/src/java/org/apache/cassandra/cql3/constraints/SatisfiabilityChecker.java new file mode 100644 index 000000000000..cc35a6090236 --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/constraints/SatisfiabilityChecker.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.constraints; + +import java.util.List; + +import org.apache.cassandra.schema.ColumnMetadata; + +public interface SatisfiabilityChecker +{ + void checkSatisfiability(List<ColumnConstraint<?>> constraints, ColumnMetadata columnMetadata); + + default boolean enableDuplicateDefinitions() + { + return true; + } + + interface UnaryFunctionSatisfiabilityChecker extends SatisfiabilityChecker + { + default void checkSatisfiability(List<ColumnConstraint<?>> constraints, ColumnMetadata columnMetadata) + { + } + + default boolean enableDuplicateDefinitions() + { + return false; + } + } +} diff --git a/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java index 71c2a77bf9d6..cd3558945471 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java @@ -20,9 +20,10 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.List; +import java.util.Set; import org.apache.cassandra.cql3.ColumnIdentifier; -import org.apache.cassandra.cql3.CqlBuilder; import org.apache.cassandra.cql3.Operator; import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.db.marshal.AbstractType; @@ -32,12 +33,19 @@ import org.apache.cassandra.tcm.serialization.MetadataSerializer; import org.apache.cassandra.tcm.serialization.Version; -public class ScalarColumnConstraint extends ColumnConstraint<ScalarColumnConstraint> +import static org.apache.cassandra.cql3.Operator.EQ; +import static org.apache.cassandra.cql3.Operator.GT; +import static org.apache.cassandra.cql3.Operator.GTE; +import static org.apache.cassandra.cql3.Operator.LT; +import static org.apache.cassandra.cql3.Operator.LTE; +import static org.apache.cassandra.cql3.Operator.NEQ; +import static org.apache.cassandra.cql3.constraints.AbstractFunctionSatisfiabilityChecker.SCALAR_SATISFIABILITY_CHECKER; + +public class ScalarColumnConstraint extends AbstractFunctionConstraint<ScalarColumnConstraint> { - public final static Serializer serializer = new Serializer(); + public static final Set<Operator> SUPPORTED_OPERATORS = Set.of(EQ, NEQ, GTE, GT, LTE, LT); - private final Operator relationType; - private final String term; + public static final Serializer serializer = new Serializer(); public final static class Raw { @@ -58,13 +66,25 @@ public ScalarColumnConstraint prepare() } } + public static class ScalarColumnConstraintSatisfiabilityChecker implements SatisfiabilityChecker + { + @Override + public void checkSatisfiability(List<ColumnConstraint<?>> constraints, ColumnMetadata columnMetadata) + { + SCALAR_SATISFIABILITY_CHECKER.check("scalar", constraints, columnMetadata); + } + } + private ScalarColumnConstraint(ColumnIdentifier param, Operator relationType, String term) { - super(param); - this.relationType = relationType; - this.term = term; + super(param, relationType, term); } + @Override + public Set<Operator> getSupportedOperators() + { + return SUPPORTED_OPERATORS; + } @Override protected void internalEvaluate(AbstractType<?> valueType, ByteBuffer columnValue) @@ -103,6 +123,12 @@ public String toString() return columnName + " " + relationType + " " + term; } + @Override + public String name() + { + return columnName + " " + relationType; + } + @Override public MetadataSerializer<ScalarColumnConstraint> serializer() { @@ -110,9 +136,9 @@ public MetadataSerializer<ScalarColumnConstraint> serializer() } @Override - public void appendCqlTo(CqlBuilder builder) + public boolean enablesDuplicateDefinitions(String name) { - builder.append(toString()); + return relationType == NEQ; } private static class Serializer implements MetadataSerializer<ScalarColumnConstraint> diff --git a/src/java/org/apache/cassandra/cql3/constraints/UnaryFunctionColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/UnaryFunctionColumnConstraint.java index 4ca3fb35cee6..8905bafd071e 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/UnaryFunctionColumnConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/UnaryFunctionColumnConstraint.java @@ -20,10 +20,12 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Set; import java.util.function.Function; import org.apache.cassandra.cql3.ColumnIdentifier; -import org.apache.cassandra.cql3.CqlBuilder; +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.cql3.constraints.SatisfiabilityChecker.UnaryFunctionSatisfiabilityChecker; import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.io.util.DataInputPlus; @@ -31,7 +33,6 @@ import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.tcm.serialization.MetadataSerializer; import org.apache.cassandra.tcm.serialization.Version; -import org.apache.cassandra.utils.LocalizeString; import static org.apache.cassandra.cql3.constraints.ColumnConstraint.ConstraintType.UNARY_FUNCTION; @@ -58,7 +59,7 @@ public UnaryFunctionColumnConstraint prepare() } } - private enum Functions + public enum Functions implements UnaryFunctionSatisfiabilityChecker { NOT_NULL(NotNullConstraint::new); @@ -72,22 +73,16 @@ private enum Functions private static ConstraintFunction createConstraintFunction(String functionName, ColumnIdentifier columnName) { - try - { - return Functions.valueOf(LocalizeString.toUpperCaseLocalized(functionName)).functionCreator.apply(columnName); - } - catch (IllegalArgumentException ex) - { - throw new InvalidConstraintDefinitionException("Unrecognized constraint function: " + functionName); - } + return getEnum(Functions.class, functionName).functionCreator.apply(columnName); } private UnaryFunctionColumnConstraint(ConstraintFunction function, ColumnIdentifier columnName) { - super(columnName); + super(columnName, null, null); this.function = function; } + @Override public String name() { return function.name; @@ -100,9 +95,15 @@ public MetadataSerializer<UnaryFunctionColumnConstraint> serializer() } @Override - public void appendCqlTo(CqlBuilder builder) + public Set<Operator> getSupportedOperators() + { + return Set.of(); + } + + @Override + public boolean enablesDuplicateDefinitions(String name) { - builder.append(toString()); + return Functions.valueOf(name).enableDuplicateDefinitions(); } @Override diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java index 518ae571a653..a527a1c2c5bd 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java @@ -346,16 +346,18 @@ public TableMetadata.Builder builder(Types types, UserFunctions functions) for (int i = 0; i < partitionKeyColumns.size(); i++) { - ColumnConstraints constraints = validateConstraints(partitionKeyColumns.get(i)); ColumnProperties properties = partitionKeyColumnProperties.get(i); - builder.addPartitionKeyColumn(partitionKeyColumns.get(i), properties.type, properties.mask, constraints); + ColumnIdentifier columnIdentifier = partitionKeyColumns.get(i); + builder.addPartitionKeyColumn(columnIdentifier, properties.type, properties.mask); + builder.getColumn(columnIdentifier).setColumnConstraints(columnConstraints.get(columnIdentifier)); } for (int i = 0; i < clusteringColumns.size(); i++) { - ColumnConstraints constraints = validateConstraints(clusteringColumns.get(i)); ColumnProperties properties = clusteringColumnProperties.get(i); - builder.addClusteringColumn(clusteringColumns.get(i), properties.type, properties.mask, constraints); + ColumnIdentifier columnIdentifier = clusteringColumns.get(i); + builder.addClusteringColumn(columnIdentifier, properties.type, properties.mask); + builder.getColumn(columnIdentifier).setColumnConstraints(columnConstraints.get(columnIdentifier)); } if (useCompactStorage) @@ -365,22 +367,16 @@ public TableMetadata.Builder builder(Types types, UserFunctions functions) else { columns.forEach((column, properties) -> { - ColumnConstraints constraints = validateConstraints(column); if (staticColumns.contains(column)) - builder.addStaticColumn(column, properties.type, properties.mask, constraints); + builder.addStaticColumn(column, properties.type, properties.mask); else - builder.addRegularColumn(column, properties.type, properties.mask, constraints); + builder.addRegularColumn(column, properties.type, properties.mask); }); } - return builder; - } + columns.keySet().forEach(id -> builder.getColumn(id).setColumnConstraints(columnConstraints.get(id))); - private ColumnConstraints validateConstraints(ColumnIdentifier columnIdentifier) - { - ColumnConstraints constraints = columnConstraints.get(columnIdentifier); - constraints.checkInvalidConstraintsCombinations(columnIdentifier); - return constraints; + return builder; } private void validateCompactTable(List<ColumnProperties> clusteringColumnProperties, diff --git a/src/java/org/apache/cassandra/schema/ColumnMetadata.java b/src/java/org/apache/cassandra/schema/ColumnMetadata.java index ad44732e2b89..cdaec3f80b78 100644 --- a/src/java/org/apache/cassandra/schema/ColumnMetadata.java +++ b/src/java/org/apache/cassandra/schema/ColumnMetadata.java @@ -31,7 +31,6 @@ import com.google.common.collect.Lists; import org.apache.cassandra.cql3.*; -import org.apache.cassandra.cql3.constraints.AbstractFunctionConstraint; import org.apache.cassandra.cql3.constraints.ColumnConstraint; import org.apache.cassandra.cql3.constraints.ColumnConstraints; import org.apache.cassandra.cql3.functions.masking.ColumnMask; @@ -327,7 +326,7 @@ public static boolean hasFunctionConstraint(ColumnConstraints columnConstraints, if (constraint.getConstraintType() == ColumnConstraint.ConstraintType.UNARY_FUNCTION || constraint.getConstraintType() == ColumnConstraint.ConstraintType.FUNCTION) { - if (((AbstractFunctionConstraint<?>) constraint).name().equals(name)) + if (constraint.name().equals(name)) return true; } } @@ -366,7 +365,7 @@ public ColumnConstraints getColumnConstraints() public void setColumnConstraints(ColumnConstraints constraints) { - constraints.checkInvalidConstraintsCombinations(name); + constraints.validate(this); this.columnConstraints = constraints; } diff --git a/test/distributed/org/apache/cassandra/distributed/test/ColumnConstraintsTest.java b/test/distributed/org/apache/cassandra/distributed/test/ColumnConstraintsTest.java index cfd3ecb16eae..16fc38c55c6c 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/ColumnConstraintsTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/ColumnConstraintsTest.java @@ -23,6 +23,7 @@ import java.util.Set; import org.apache.cassandra.cql3.constraints.ConstraintViolationException; +import org.apache.cassandra.cql3.constraints.InvalidConstraintDefinitionException; import org.apache.cassandra.exceptions.InvalidRequestException; import org.junit.Test; @@ -51,7 +52,7 @@ public void testInvalidConstraintsExceptions() throws IOException { assertThrowsInvalidConstraintException(cluster, String.format("CREATE TABLE %s (pk int, ck1 text CHECK ck1 < 100, ck2 int, v int, " + "PRIMARY KEY ((pk), ck1, ck2));", tableName), - "ck1 is not a number"); + "Column 'ck1' is not a number type."); assertThrowsInvalidConstraintException(cluster, String.format("CREATE TABLE %s (pk int, ck1 int CHECK LENGTH(ck1) < 100, ck2 int, v int, " + "PRIMARY KEY ((pk), ck1, ck2));", tableName), @@ -319,6 +320,6 @@ private void assertThrowsInvalidConstraintException(Cluster cluster, String stat assertThatThrownBy(() -> cluster.schemaChange(statement)) .describedAs(description) .has(new Condition<Throwable>(t -> t.getClass().getCanonicalName() - .equals(InvalidRequestException.class.getCanonicalName()), description)); + .equals(InvalidConstraintDefinitionException.class.getCanonicalName()), description)); } } diff --git a/test/unit/org/apache/cassandra/contraints/ConstraintsSatisfiabilityTest.java b/test/unit/org/apache/cassandra/contraints/ConstraintsSatisfiabilityTest.java new file mode 100644 index 000000000000..a0a87b7d7b0d --- /dev/null +++ b/test/unit/org/apache/cassandra/contraints/ConstraintsSatisfiabilityTest.java @@ -0,0 +1,180 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.contraints; + +import org.junit.Test; + +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.cql3.constraints.ColumnConstraints; +import org.apache.cassandra.cql3.constraints.FunctionColumnConstraint; +import org.apache.cassandra.cql3.constraints.ScalarColumnConstraint; +import org.apache.cassandra.cql3.constraints.ScalarColumnConstraint.Raw; +import org.apache.cassandra.db.marshal.IntegerType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.schema.ColumnMetadata; +import org.assertj.core.api.ThrowableAssert; + +import static java.util.List.of; +import static org.apache.cassandra.cql3.Operator.EQ; +import static org.apache.cassandra.cql3.Operator.GT; +import static org.apache.cassandra.cql3.Operator.GTE; +import static org.apache.cassandra.cql3.Operator.LT; +import static org.apache.cassandra.cql3.Operator.LTE; +import static org.apache.cassandra.cql3.Operator.NEQ; +import static org.apache.cassandra.cql3.constraints.ScalarColumnConstraint.SUPPORTED_OPERATORS; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public class ConstraintsSatisfiabilityTest +{ + private static final ColumnIdentifier columnIdentifier = new ColumnIdentifier("a_column", false); + private static final ColumnIdentifier lengthFunctionIdentifier = new ColumnIdentifier("LENGTH", false); + private static final ColumnMetadata regularIntColumn = new ColumnMetadata("a", "b", columnIdentifier, IntegerType.instance, -1, ColumnMetadata.Kind.REGULAR, null); + private static final ColumnMetadata regularStringColumn = new ColumnMetadata("a", "b", columnIdentifier, UTF8Type.instance, -1, ColumnMetadata.Kind.REGULAR, null); + + @Test + public void testScalarSatisfiability() throws Throwable + { + run(this::scalar, regularIntColumn); + } + + @Test + public void testLengthSatisfiability() throws Throwable + { + run(this::length, regularStringColumn); + } + + private <T> void run(QuadFunction<T> quadFunction, ColumnMetadata columnMetadata) throws Throwable + { + for (Operator op1 : SUPPORTED_OPERATORS) + { + for (Operator op2 : SUPPORTED_OPERATORS) + { + if (op1 == op2) + { + if (op1 == NEQ) + { + // a_column != 0 and a_column != 10 -> valid + check(op1, 0, op2, 100, quadFunction, null, columnMetadata); + // does not make sense to check twice + // check a_column != 0 and a_column != 0 + check(op1, 0, op2, 0, quadFunction, "There are duplicate constraint definitions on column", columnMetadata); + } + else + check(op1, 0, op2, 100, quadFunction, "There are duplicate constraint definitions on column", columnMetadata); + } + else if ((op1 == GT && op2 == GTE) || + (op1 == GTE && op2 == GT) || + (op1 == LT && op2 == LTE) || + (op1 == LTE && op2 == LT) || + (op1 == EQ || op2 == EQ)) + { + check(op1, 0, op2, 100, quadFunction, "not supported", columnMetadata); + } + else if ((op1 == LTE && op2 == GT) || + (op1 == LT && op2 == GT) || + (op1 == LTE && op2 == GTE) || + (op1 == LT && op2 == GTE)) + { + check(op1, 0, op2, 100, quadFunction, "are not satisfiable", columnMetadata); + } + else if (!(op1 == NEQ || op2 == NEQ)) + { + check(op1, 0, op2, 100, quadFunction, null, columnMetadata); + } + else + { + // this is valid + // a_column < 0, a_column != 10 + } + } + } + } + + @Test + public void testNumberOfScalarConstraints() + { + // one + new ColumnConstraints(of(scalar(LT, 5))).validate(regularIntColumn); + + // two + new ColumnConstraints(of(scalar(LT, 5), scalar(GT, 0))).validate(regularIntColumn); + + // three - invalid + + assertThatThrownBy(() -> new ColumnConstraints(of(scalar(LT, 5), scalar(GT, 0), scalar(GTE, 0))).validate(regularIntColumn)) + .hasMessage("There can not be more than 2 constraints (not including non-equal relations) on a column 'a_column' but you have specified 3"); + + // valid + new ColumnConstraints(of(scalar(LT, 5), scalar(GT, 0), scalar(NEQ, 3))).validate(regularIntColumn); + + // valid, because NEQs have different terms + new ColumnConstraints(of(scalar(LT, 5), scalar(GT, 0), scalar(NEQ, 3), scalar(NEQ, 4))).validate(regularIntColumn); + + // this has duplicate a_column != 3 + assertThatThrownBy(() -> new ColumnConstraints(of(scalar(LT, 5), scalar(GT, 0), scalar(NEQ, 3), scalar(NEQ, 3))).validate(regularIntColumn)) + .hasMessage("There are duplicate constraint definitions on column 'a_column': a_column != 3"); + } + + private interface QuadFunction<T> + { + ColumnConstraints f(Operator op1, Integer term1, Operator o2, Integer term2); + } + + private <T> void check(Operator operator, + Integer term, + Operator operator2, + Integer term2, + QuadFunction<T> quadFunction, + String exceptionMessage, + ColumnMetadata columnMetadata) throws Throwable + { + ThrowableAssert.ThrowingCallable callable = () -> quadFunction.f(operator, term, operator2, term2).validate(columnMetadata); + + if (exceptionMessage != null) + assertThatThrownBy(callable).hasMessageContaining(exceptionMessage); + else + callable.call(); + } + + private ColumnConstraints scalar(Operator operator, Integer term, Operator operator2, Integer term2) + { + return new ColumnConstraints(of(scalar(operator, term), + scalar(operator2, term2))); + } + + private ScalarColumnConstraint scalar(Operator operator, Integer term) + { + return new Raw(columnIdentifier, operator, term.toString()).prepare(); + } + + private FunctionColumnConstraint length(Operator operator, Integer term) + { + return new FunctionColumnConstraint.Raw(lengthFunctionIdentifier, + columnIdentifier, + operator, + term.toString()).prepare(); + } + + private ColumnConstraints length(Operator operator, Integer term, Operator operator2, Integer term2) + { + return new ColumnConstraints(of(length(operator, term), + length(operator2, term2))); + } +} From a51344c6115ca432a5e52a4f8aac69dbfa813c8c Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Thu, 20 Feb 2025 13:53:07 +0100 Subject: [PATCH 152/225] Add JSON constraint patch by Stefan Miklosovic; reviewed by Bernardo Botella for CASSANDRA-20273 --- CHANGES.txt | 1 + .../pages/developing/cql/constraints.adoc | 27 ++++++ .../cql3/constraints/JsonConstraint.java | 81 +++++++++++++++++ .../UnaryFunctionColumnConstraint.java | 3 +- .../cassandra/db/marshal/AbstractType.java | 5 ++ .../contraints/JsonConstraintTest.java | 89 +++++++++++++++++++ 6 files changed, 205 insertions(+), 1 deletion(-) create mode 100644 src/java/org/apache/cassandra/cql3/constraints/JsonConstraint.java create mode 100644 test/unit/org/apache/cassandra/contraints/JsonConstraintTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 5827b806256d..22149cd8de9c 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Add JSON constraint (CASSANDRA-20273) * Prevent invalid constraint combinations (CASSANDRA-20330) * Support CREATE TABLE LIKE WITH INDEXES (CASSANDRA-19965) * Invalidate relevant prepared statements on every change to TableMetadata (CASSANDRA-20318) diff --git a/doc/modules/cassandra/pages/developing/cql/constraints.adoc b/doc/modules/cassandra/pages/developing/cql/constraints.adoc index 82f454618a35..0bd3120edc33 100644 --- a/doc/modules/cassandra/pages/developing/cql/constraints.adoc +++ b/doc/modules/cassandra/pages/developing/cql/constraints.adoc @@ -139,3 +139,30 @@ DELETE col2 FROM ks.tb WHERE id = 1 AND cl = 2; Additionally, `NOT_NULL` can not be specified on any column of a primary key, being it a partition key or a clustering column. + +=== JSON constraint + +Defines a constraint which checks if a column contains a string which is a valid JSON. + +`JSON` constraint can be used only for columns of `text`, `varchar` or `ascii` types. + +---- +CREATE TABLE ks.tb ( + id int primary key, + val text CHECK JSON(val) +); + +-- valid JSON string + +INSERT INTO ks.tb (id, val) VALUES (1, '{"a": 5}'); +INSERT INTO ks.tb (id, val) VALUES (1, '{}'); + +-- invalid JSON string + +INSERT INTO ks.tb (id, val) VALUES (1, '{"a": 5'); +INSERT INTO ks.tb (id, val) VALUES (1, 'abc'); + +... [Invalid query] message="Value for column 'val' violated JSON +constraint as it is not a valid JSON." + +---- \ No newline at end of file diff --git a/src/java/org/apache/cassandra/cql3/constraints/JsonConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/JsonConstraint.java new file mode 100644 index 000000000000..0abd1d511e13 --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/constraints/JsonConstraint.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.constraints; + +import java.nio.ByteBuffer; + +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.serializers.MarshalException; +import org.apache.cassandra.utils.JsonUtils; + +import static java.lang.String.format; + +public class JsonConstraint extends ConstraintFunction +{ + public static final String FUNCTION_NAME = "JSON"; + + public JsonConstraint(ColumnIdentifier columnName) + { + this(columnName, FUNCTION_NAME); + } + + public JsonConstraint(ColumnIdentifier columnName, String name) + { + super(columnName, name); + } + + @Override + public void internalEvaluate(AbstractType<?> valueType, Operator relationType, String term, ByteBuffer columnValue) + { + try + { + JsonUtils.decodeJson(valueType.getString(columnValue)); + } + catch (MarshalException ex) + { + throw new ConstraintViolationException(format("Value for column '%s' violated %s constraint as it is not a valid JSON.", + columnName.toCQLString(), + name)); + } + } + + @Override + public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException + { + if (!columnMetadata.type.unwrap().isString()) + throw new InvalidConstraintDefinitionException(name + " can be used only for columns of 'text', 'varchar' or 'ascii' types."); + } + + @Override + public boolean equals(Object o) + { + if (this == o) + return true; + + if (!(o instanceof JsonConstraint)) + return false; + + JsonConstraint other = (JsonConstraint) o; + + return columnName.equals(other.columnName) && name.equals(other.name); + } +} diff --git a/src/java/org/apache/cassandra/cql3/constraints/UnaryFunctionColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/UnaryFunctionColumnConstraint.java index 8905bafd071e..bb41f57c239a 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/UnaryFunctionColumnConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/UnaryFunctionColumnConstraint.java @@ -61,7 +61,8 @@ public UnaryFunctionColumnConstraint prepare() public enum Functions implements UnaryFunctionSatisfiabilityChecker { - NOT_NULL(NotNullConstraint::new); + NOT_NULL(NotNullConstraint::new), + JSON(JsonConstraint::new); private final Function<ColumnIdentifier, ConstraintFunction> functionCreator; diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractType.java b/src/java/org/apache/cassandra/db/marshal/AbstractType.java index 29f446838949..c624ce505be0 100644 --- a/src/java/org/apache/cassandra/db/marshal/AbstractType.java +++ b/src/java/org/apache/cassandra/db/marshal/AbstractType.java @@ -552,6 +552,11 @@ public boolean isNumber() return unwrap() instanceof org.apache.cassandra.db.marshal.NumberType; } + public boolean isString() + { + return unwrap() instanceof org.apache.cassandra.db.marshal.StringType; + } + // This assumes that no empty values are passed public void writeValue(ByteBuffer value, DataOutputPlus out) throws IOException { diff --git a/test/unit/org/apache/cassandra/contraints/JsonConstraintTest.java b/test/unit/org/apache/cassandra/contraints/JsonConstraintTest.java new file mode 100644 index 000000000000..ba9281223060 --- /dev/null +++ b/test/unit/org/apache/cassandra/contraints/JsonConstraintTest.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.contraints; + +import org.junit.Test; + +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.constraints.ColumnConstraints; +import org.apache.cassandra.cql3.constraints.UnaryFunctionColumnConstraint.Raw; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.AsciiType; +import org.apache.cassandra.db.marshal.IntegerType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.schema.ColumnMetadata; +import org.assertj.core.api.ThrowableAssert.ThrowingCallable; + +import static java.util.List.of; +import static org.apache.cassandra.cql3.constraints.JsonConstraint.FUNCTION_NAME; +import static org.apache.cassandra.schema.ColumnMetadata.Kind.REGULAR; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public class JsonConstraintTest +{ + private static final ColumnIdentifier columnIdentifier = new ColumnIdentifier("a_column", false); + private static final ColumnIdentifier jsonFunctionIdentifier = new ColumnIdentifier(FUNCTION_NAME, false); + private static final ColumnMetadata regularStringColumn = getColumnOfType(UTF8Type.instance); + private static final ColumnMetadata regularAsciiColumn = getColumnOfType(AsciiType.instance); + + private static final ColumnConstraints json = new ColumnConstraints(of(new Raw(jsonFunctionIdentifier, columnIdentifier).prepare())); + + @Test + public void testJsonConstraint() throws Throwable + { + run("{}"); + run("{\"a\": 5, \"b\": \"1\", \"c\": [1,2,3]}"); + run("nonsense", "Value for column 'a_column' violated JSON constraint as it is not a valid JSON."); + run("", "Column value does not satisfy value constraint for column 'a_column' as it is null."); + } + + @Test + public void testInvalidTypes() + { + assertThatThrownBy(() -> json.validate(getColumnOfType(IntegerType.instance))) + .hasMessageContaining("JSON can be used only for columns of 'text', 'varchar' or 'ascii' types."); + } + + private void run(String jsonToCheck) throws Throwable + { + run(jsonToCheck, null); + } + + private void run(String jsonToCheck, String exceptionMessage) throws Throwable + { + ThrowingCallable callable = () -> + { + json.validate(regularStringColumn); + json.evaluate(regularStringColumn.type, regularAsciiColumn.type.fromString(jsonToCheck)); + + json.validate(regularAsciiColumn); + json.evaluate(regularAsciiColumn.type, regularAsciiColumn.type.fromString(jsonToCheck)); + }; + + if (exceptionMessage == null) + callable.call(); + else + assertThatThrownBy(callable).hasMessageContaining(exceptionMessage); + } + + private static ColumnMetadata getColumnOfType(AbstractType<?> type) + { + return new ColumnMetadata("a", "b", columnIdentifier, type, -1, REGULAR, null); + } +} From 548316593898c3b2169986d27eae7a6624bd873d Mon Sep 17 00:00:00 2001 From: Caleb Rackliffe <calebrackliffe@gmail.com> Date: Thu, 20 Feb 2025 17:54:29 -0600 Subject: [PATCH 153/225] Avoid limit on RFP fetch in the case of an unresolved static row patch by Caleb Rackliffe; reviewed by David Capwell and Zhao Yang for CASSANDRA-20323 --- CHANGES.txt | 1 + .../reads/ReplicaFilteringProtection.java | 15 +- .../test/ReplicaFilteringProtectionTest.java | 19 --- .../sai/ReplicaFilteringWithStaticsTest.java | 131 ++++++++++++++++++ .../test/sai/StrictFilteringTest.java | 20 --- 5 files changed, 142 insertions(+), 44 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/test/sai/ReplicaFilteringWithStaticsTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 06c119f1fff3..b01f18babcdc 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.4 + * Avoid limit on RFP fetch in the case of an unresolved static row (CASSANDRA-20323) * Include materialized views to the output of DESCRIBE TABLE statements (CASSANDRA-20365) * Heap and GC jvm flags improvements (CASSANDRA-20296) * Fix unparseable YAML in default cassandra.yaml when uncommented for downstream tooling (CASSANDRA-20359) diff --git a/src/java/org/apache/cassandra/service/reads/ReplicaFilteringProtection.java b/src/java/org/apache/cassandra/service/reads/ReplicaFilteringProtection.java index 7c57245bb178..c66c2007d6df 100644 --- a/src/java/org/apache/cassandra/service/reads/ReplicaFilteringProtection.java +++ b/src/java/org/apache/cassandra/service/reads/ReplicaFilteringProtection.java @@ -230,7 +230,7 @@ public void onMergedRows(Row merged, Row[] versions) // Even if there are no completely missing rows, replicas may still be silent about individual // columns, so we need to check for divergence at the column level: - for (ColumnMetadata column : columns) + for (ColumnMetadata column : merged.isStatic() ? columns.statics : columns.regulars) { Arrays.fill(silentColumnAt, false); boolean allSilent = true; @@ -538,14 +538,19 @@ private UnfilteredPartitionIterator fetchFromSource() Tracing.trace("Requesting {} rows in partition {} from {} for replica filtering protection", clusterings.size(), key, source); - // build the read command taking into account that we could be requesting only in the static row - DataLimits limits = clusterings.isEmpty() ? DataLimits.cqlLimits(1) : DataLimits.NONE; - ClusteringIndexFilter filter = unresolvedStatic ? command.clusteringIndexFilter(key) : new ClusteringIndexNamesFilter(clusterings, command.isReversed()); + // If there is an unresolved static column, we must fetch the entire partition, as static column predicates + // may produce row matches across the entire partition. If there are only non-static rows to complete, we + // query the partition specifically for the corresponding cluterings by name. In either case, we do not + // provide a limit. (In the unresolved static case, we have no way of knowing how many stale rows we might + // read on a silent replica before finding a live one.) + ClusteringIndexFilter filter = unresolvedStatic ? command.clusteringIndexFilter(key) + : new ClusteringIndexNamesFilter(clusterings, command.isReversed()); + SinglePartitionReadCommand cmd = SinglePartitionReadCommand.create(command.metadata(), command.nowInSec(), command.columnFilter(), RowFilter.none(), - limits, + DataLimits.NONE, key, filter); diff --git a/test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java b/test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java index 1cff326f41f4..fd8110cba72a 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/ReplicaFilteringProtectionTest.java @@ -27,7 +27,6 @@ import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.distributed.Cluster; -import org.apache.cassandra.distributed.api.ConsistencyLevel; import org.apache.cassandra.distributed.api.IInvokableInstance; import org.apache.cassandra.distributed.api.SimpleQueryResult; import org.apache.cassandra.exceptions.OverloadedException; @@ -72,24 +71,6 @@ public static void teardown() cluster.close(); } - @Test - public void testMissingStaticRowWithNonStaticExpression() - { - cluster.schemaChange(withKeyspace("CREATE TABLE %s.single_predicate (pk0 int, ck0 int, ck1 int, s0 int static, s1 int static, v0 int, PRIMARY KEY (pk0, ck0, ck1)) " + - "WITH CLUSTERING ORDER BY (ck0 ASC, ck1 DESC) AND read_repair = 'NONE'")); - - cluster.get(1).executeInternal(withKeyspace("INSERT INTO %s.single_predicate (pk0, ck0, ck1, s0, s1, v0) " + - "VALUES (0, 1, 2, 3, 4, 5) USING TIMESTAMP 1")); - cluster.get(2).executeInternal(withKeyspace("UPDATE %s.single_predicate USING TIMESTAMP 2 SET s0 = 6, s1 = 7, v0 = 8 " + - "WHERE pk0 = 0 AND ck0 = 9 AND ck1 = 10")); - - // Node 2 will not produce a match for the static row. Make sure that replica filtering protection does not - // fetch the entire partition, which could let non-matching rows slip through combined with the fact that we - // don't post-filter at the coordinator with no regular column predicates in the query. - String select = withKeyspace("SELECT pk0, ck0, ck1, s0, s1 FROM %s.single_predicate WHERE ck1 = 2 ALLOW FILTERING"); - assertRows(cluster.coordinator(1).execute(select, ConsistencyLevel.ALL), row(0, 1, 2, 6, 7)); - } - @Test public void testMissedUpdatesBelowCachingWarnThreshold() { diff --git a/test/distributed/org/apache/cassandra/distributed/test/sai/ReplicaFilteringWithStaticsTest.java b/test/distributed/org/apache/cassandra/distributed/test/sai/ReplicaFilteringWithStaticsTest.java new file mode 100644 index 000000000000..59cc1cc6a51f --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/sai/ReplicaFilteringWithStaticsTest.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.sai; + +import java.io.IOException; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.test.TestBaseImpl; + +import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL; +import static org.apache.cassandra.distributed.api.Feature.GOSSIP; +import static org.apache.cassandra.distributed.api.Feature.NETWORK; +import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows; +import static org.apache.cassandra.distributed.shared.AssertUtils.row; + +public class ReplicaFilteringWithStaticsTest extends TestBaseImpl +{ + private static Cluster CLUSTER; + + @BeforeClass + public static void setUpCluster() throws IOException + { + CLUSTER = init(Cluster.build(3).withConfig(config -> config.set("hinted_handoff_enabled", false).with(GOSSIP).with(NETWORK)).start()); + } + + @Test + public void testStaticMatchWithPartitionDelete() + { + testStaticMatchWithPartitionDelete(false); + } + + @Test + public void testStaticMatchWithPartitionDeleteSAI() + { + testStaticMatchWithPartitionDelete(true); + } + + public void testStaticMatchWithPartitionDelete(boolean sai) + { + String table = "static_and_delete" + (sai ? "_sai" : ""); + CLUSTER.schemaChange(withKeyspace("CREATE TABLE %s." + table + " (pk0 boolean, ck0 ascii, s1 tinyint static, v0 boolean, PRIMARY KEY (pk0, ck0)) " + + "WITH CLUSTERING ORDER BY (ck0 ASC) AND read_repair = 'NONE'")); + + if (sai) + { + CLUSTER.schemaChange(withKeyspace("CREATE INDEX ON %s." + table + "(s1) USING 'sai'")); + SAIUtil.waitForIndexQueryable(CLUSTER, KEYSPACE); + } + + CLUSTER.get(3).executeInternal(withKeyspace("UPDATE %s." + table + " USING TIMESTAMP 1 SET v0 = false WHERE pk0 = true AND ck0 = 'D'")); + CLUSTER.get(1).executeInternal(withKeyspace("DELETE FROM %s." + table + " USING TIMESTAMP 2 WHERE pk0 = true")); + + CLUSTER.get(1).executeInternal(withKeyspace("INSERT INTO %s." + table + " (pk0, ck0, s1, v0) VALUES (true, 'G', -114, true) USING TIMESTAMP 3")); + CLUSTER.get(3).executeInternal(withKeyspace("INSERT INTO %s." + table + " (pk0, ck0) VALUES (true, 'F') USING TIMESTAMP 4")); + CLUSTER.get(1).executeInternal(withKeyspace("INSERT INTO %s." + table + " (pk0, ck0, s1, v0) VALUES (true, 'C', 17, true) USING TIMESTAMP 5")); + + // This update to the static column creates matches across all previously written live rows in the partition. + // When RFP sees the unresolved static row, it must read enough data from the silent replicas at nodes 1 and 3 + // to find all potential matches. With a page size of 1, reading only 1 row from node 3 will return the row at + // ck = 'D', as the partition delete never made it to node 3. This means we'll ignore the live result node 3 has + // at ck = 'F', because node 1 will produce a result at ck = 'G', and that determines the next paging cursor. + CLUSTER.get(2).executeInternal(withKeyspace("UPDATE %s." + table + " USING TIMESTAMP 6 SET s1 = 1, v0 = false WHERE pk0 = true AND ck0 = 'A'")); + + String select = withKeyspace("SELECT ck0 FROM %s." + table + " WHERE s1 = 1" + (sai ? "" : " ALLOW FILTERING" )); + assertRows(CLUSTER.coordinator(1).executeWithPaging(select, ALL, 1), row("A"), row("C"), row("F"), row("G")); + } + + @Test + public void testMissingStaticRowWithNonStaticExpression() + { + testMissingStaticRowWithNonStaticExpression(false); + } + + @Test + public void testMissingStaticRowWithNonStaticExpressionSAI() + { + testMissingStaticRowWithNonStaticExpression(true); + } + + public void testMissingStaticRowWithNonStaticExpression(boolean sai) + { + String table = "single_predicate" + (sai ? "_sai" : ""); + CLUSTER.schemaChange(withKeyspace("CREATE TABLE %s." + table + " (pk0 int, ck0 int, ck1 int, s0 int static, s1 int static, v0 int, PRIMARY KEY (pk0, ck0, ck1)) " + + "WITH CLUSTERING ORDER BY (ck0 ASC, ck1 DESC) AND read_repair = 'NONE'")); + + if (sai) + { + CLUSTER.schemaChange(withKeyspace("CREATE INDEX ON %s." + table + "(ck1) USING 'sai'")); + SAIUtil.waitForIndexQueryable(CLUSTER, KEYSPACE); + } + + CLUSTER.get(1).executeInternal(withKeyspace("INSERT INTO %s." + table + " (pk0, ck0, ck1, s0, s1, v0) " + + "VALUES (0, 1, 2, 3, 4, 5) USING TIMESTAMP 1")); + CLUSTER.get(2).executeInternal(withKeyspace("UPDATE %s." + table + " USING TIMESTAMP 2 SET s0 = 6, s1 = 7, v0 = 8 " + + "WHERE pk0 = 0 AND ck0 = 9 AND ck1 = 10")); + + // Node 2 will not produce a match for the static row. Make sure that replica filtering protection does not + // fetch the entire partition, which could let non-matching rows slip through combined with the fact that we + // don't post-filter at the coordinator with no regular column predicates in the query. + String select = withKeyspace("SELECT pk0, ck0, ck1, s0, s1 FROM %s." + table + " WHERE ck1 = 2" + (sai ? "" : " ALLOW FILTERING")); + assertRows(CLUSTER.coordinator(1).execute(select, ConsistencyLevel.ALL), row(0, 1, 2, 6, 7)); + } + + @AfterClass + public static void shutDownCluster() + { + if (CLUSTER != null) + CLUSTER.close(); + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/sai/StrictFilteringTest.java b/test/distributed/org/apache/cassandra/distributed/test/sai/StrictFilteringTest.java index 5ef92bb9ef80..6ec80fd0ae19 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/sai/StrictFilteringTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/sai/StrictFilteringTest.java @@ -53,26 +53,6 @@ public static void setUpCluster() throws IOException CLUSTER = init(Cluster.build(2).withConfig(config -> config.set("hinted_handoff_enabled", false).with(GOSSIP).with(NETWORK)).start()); } - @Test - public void testMissingStaticRowWithNonStaticExpression() - { - CLUSTER.schemaChange(withKeyspace("CREATE TABLE %s.single_predicate (pk0 int, ck0 int, ck1 int, s0 int static, s1 int static, v0 int, PRIMARY KEY (pk0, ck0, ck1)) " + - "WITH CLUSTERING ORDER BY (ck0 ASC, ck1 DESC) AND read_repair = 'NONE'")); - CLUSTER.schemaChange(withKeyspace("CREATE INDEX ON %s.single_predicate(ck1) USING 'sai'")); - SAIUtil.waitForIndexQueryable(CLUSTER, KEYSPACE); - - CLUSTER.get(1).executeInternal(withKeyspace("INSERT INTO %s.single_predicate (pk0, ck0, ck1, s0, s1, v0) " + - "VALUES (0, 1, 2, 3, 4, 5) USING TIMESTAMP 1")); - CLUSTER.get(2).executeInternal(withKeyspace("UPDATE %s.single_predicate USING TIMESTAMP 2 SET s0 = 6, s1 = 7, v0 = 8 " + - "WHERE pk0 = 0 AND ck0 = 9 AND ck1 = 10")); - - // Node 2 will not produce a match for the static row. Make sure that replica filtering protection does not - // fetch the entire partition, which could let non-matching rows slip through combined with the fact that we - // don't post-filter at the coordinator with no regular column predicates in the query. - String select = withKeyspace("SELECT pk0, ck0, ck1, s0, s1 FROM %s.single_predicate WHERE ck1 = 2 ALLOW FILTERING"); - assertRows(CLUSTER.coordinator(1).execute(select, ConsistencyLevel.ALL), row(0, 1, 2, 6, 7)); - } - @Test public void shouldDegradeToUnionOnSingleStatic() { From f1bec5d0c5dccce4128b3ff9bc087cdf0577f2b0 Mon Sep 17 00:00:00 2001 From: Abe Ratnofsky <abe@aber.io> Date: Sun, 23 Feb 2025 11:32:51 -0500 Subject: [PATCH 154/225] Improve performance of DistributedSchema.validate for large schemas patch by Abe Ratnofsky; reviewed by Caleb Rackliffe, Benedict Elliott Smith, Matt Byrd, Sam Tunnicliffe for CASSANDRA-20360 --- CHANGES.txt | 1 + .../utils/btree/AbstractBTreeMap.java | 28 +++++++++++++++---- 2 files changed, 23 insertions(+), 6 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 3e63e11f67bc..afda96a8f7d5 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Improve performance of DistributedSchema.validate for large schemas (CASSANDRA-20360) * Add JSON constraint (CASSANDRA-20273) * Prevent invalid constraint combinations (CASSANDRA-20330) * Support CREATE TABLE LIKE WITH INDEXES (CASSANDRA-19965) diff --git a/src/java/org/apache/cassandra/utils/btree/AbstractBTreeMap.java b/src/java/org/apache/cassandra/utils/btree/AbstractBTreeMap.java index 0db33a52cc16..3d0baf08f178 100644 --- a/src/java/org/apache/cassandra/utils/btree/AbstractBTreeMap.java +++ b/src/java/org/apache/cassandra/utils/btree/AbstractBTreeMap.java @@ -18,13 +18,15 @@ package org.apache.cassandra.utils.btree; +import java.util.AbstractCollection; import java.util.AbstractMap; +import java.util.Collection; import java.util.Comparator; import java.util.Iterator; import java.util.Map; import java.util.Set; -import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterators; public abstract class AbstractBTreeMap<K, V> extends AbstractMap<K, V> { @@ -105,13 +107,27 @@ public Set<K> keySet() return keySet; } + /** + * This method, according to the contract of {@link Map#values()}, returns a collection backed by the map. It also + * closely mirrors {@link AbstractMap#values()}, which returns an {@link AbstractCollection}. + */ @Override - public Set<V> values() + public Collection<V> values() { - ImmutableSet.Builder<V> b = ImmutableSet.builder(); - for (Map.Entry<K, V> e : entrySet()) - b.add(e.getValue()); - return b.build(); + return new AbstractCollection<>() + { + @Override + public Iterator<V> iterator() + { + return Iterators.transform(BTree.<Entry<K, V>>iterator(tree), Entry::getValue); + } + + @Override + public int size() + { + return AbstractBTreeMap.this.size(); + } + }; } @Override From a80663de61cb27781910b2ff64e99a239bbe5944 Mon Sep 17 00:00:00 2001 From: Abe Ratnofsky <abe@aber.io> Date: Wed, 12 Feb 2025 11:08:40 -0500 Subject: [PATCH 155/225] Fix NoSuchElementException on dropped keyspace during Paxos cleanup patch by Abe Ratnofsky; reviewed by Francisco Guerrero, Sam Tunnicliffe for CASSANDRA-20320 --- .../cassandra/service/paxos/PaxosRepair.java | 7 ++- .../cleanup/PaxosCleanupLocalCoordinator.java | 10 +++- .../distributed/test/PaxosRepair2Test.java | 49 +++++++++++++++++++ 3 files changed, 61 insertions(+), 5 deletions(-) diff --git a/src/java/org/apache/cassandra/service/paxos/PaxosRepair.java b/src/java/org/apache/cassandra/service/paxos/PaxosRepair.java index 016086cbb18d..e36e1d352250 100644 --- a/src/java/org/apache/cassandra/service/paxos/PaxosRepair.java +++ b/src/java/org/apache/cassandra/service/paxos/PaxosRepair.java @@ -677,16 +677,15 @@ static boolean validatePeerCompatibility(ClusterMetadata metadata, Replica peer) return result; } - static boolean validatePeerCompatibility(SharedContext ctx, TableMetadata table, Range<Token> range) + static boolean validatePeerCompatibility(SharedContext ctx, ClusterMetadata metadata, TableMetadata table, Range<Token> range) { - ClusterMetadata metadata = ClusterMetadata.current(); Participants participants = Participants.get(metadata, table, range.right, ConsistencyLevel.SERIAL, r -> ctx.failureDetector().isAlive(r.endpoint())); return Iterables.all(participants.all, (participant) -> validatePeerCompatibility(metadata, participant)); } - public static boolean validatePeerCompatibility(SharedContext ctx, TableMetadata table, Collection<Range<Token>> ranges) + public static boolean validatePeerCompatibility(SharedContext ctx, ClusterMetadata metadata, TableMetadata table, Collection<Range<Token>> ranges) { - return Iterables.all(ranges, range -> validatePeerCompatibility(ctx, table, range)); + return Iterables.all(ranges, range -> validatePeerCompatibility(ctx, metadata, table, range)); } public static void shutdownAndWait(long timeout, TimeUnit units) throws InterruptedException, TimeoutException diff --git a/src/java/org/apache/cassandra/service/paxos/cleanup/PaxosCleanupLocalCoordinator.java b/src/java/org/apache/cassandra/service/paxos/cleanup/PaxosCleanupLocalCoordinator.java index f91846bc9f2d..a53fec3e6081 100644 --- a/src/java/org/apache/cassandra/service/paxos/cleanup/PaxosCleanupLocalCoordinator.java +++ b/src/java/org/apache/cassandra/service/paxos/cleanup/PaxosCleanupLocalCoordinator.java @@ -40,6 +40,7 @@ import org.apache.cassandra.service.paxos.PaxosRepair; import org.apache.cassandra.service.paxos.PaxosState; import org.apache.cassandra.service.paxos.uncommitted.UncommittedPaxosKey; +import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.utils.CloseableIterator; import org.apache.cassandra.utils.concurrent.AsyncFuture; @@ -84,7 +85,14 @@ public synchronized void start() return; } - if (!PaxosRepair.validatePeerCompatibility(ctx, table, ranges)) + ClusterMetadata metadata = ClusterMetadata.current(); + if (metadata.schema.getKeyspace(table.keyspace) == null) + { + fail("Unknown keyspace: " + table.keyspace); + return; + } + + if (!PaxosRepair.validatePeerCompatibility(ctx, metadata, table, ranges)) { fail("Unsupported peer versions for " + tableId + ' ' + ranges.toString()); return; diff --git a/test/distributed/org/apache/cassandra/distributed/test/PaxosRepair2Test.java b/test/distributed/org/apache/cassandra/distributed/test/PaxosRepair2Test.java index 8b043ab6aa83..175f70c7973f 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/PaxosRepair2Test.java +++ b/test/distributed/org/apache/cassandra/distributed/test/PaxosRepair2Test.java @@ -23,12 +23,17 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import com.google.common.collect.Iterators; import com.google.common.collect.Sets; import org.apache.cassandra.distributed.shared.WithProperties; +import org.apache.cassandra.repair.SharedContext; +import org.apache.cassandra.service.paxos.cleanup.PaxosCleanupLocalCoordinator; +import org.apache.cassandra.service.paxos.cleanup.PaxosCleanupResponse; +import org.apache.cassandra.utils.Shared; import org.awaitility.Awaitility; import org.junit.Assert; import org.junit.Test; @@ -456,6 +461,13 @@ public long currentTimeMillis() } } + @Shared(scope = Shared.Scope.ANY) + public static class Conditions + { + public java.util.concurrent.CountDownLatch beforeKeyspaceDrop = new CountDownLatch(1); + public java.util.concurrent.CountDownLatch afterKeyspaceDrop = new CountDownLatch(1); + } + @Test public void legacyPurgeRepairLoop() throws Exception { @@ -573,6 +585,43 @@ public void legacyPurgeRepairLoop() throws Exception assertUncommitted(cluster.get(1), KEYSPACE, TABLE, 0); assertUncommitted(cluster.get(2), KEYSPACE, TABLE, 0); assertUncommitted(cluster.get(3), KEYSPACE, TABLE, 0); + + Conditions conditions = new Conditions(); + + Thread keyspaceDropTask = new Thread(() -> { + try + { + conditions.beforeKeyspaceDrop.await(); + cluster.schemaChange("DROP KEYSPACE " + KEYSPACE, 3); + conditions.afterKeyspaceDrop.countDown(); + } + catch (InterruptedException e) + { + throw new RuntimeException(e); + } + }); + + keyspaceDropTask.start(); + + boolean failedAsExpected = cluster.get(3).applyOnInstance(conds -> { + try + { + TableId tableId = Schema.instance.getTableMetadata(KEYSPACE, TABLE).id; + Token token = DatabaseDescriptor.getPartitioner().getMinimumToken(); + Collection<Range<Token>> ranges = Collections.singleton(new Range<>(token, token)); + PaxosCleanupLocalCoordinator repair = PaxosCleanupLocalCoordinator.createForAutoRepair(SharedContext.Global.instance, tableId, ranges); + conds.beforeKeyspaceDrop.countDown(); + conds.afterKeyspaceDrop.await(); + repair.start(); + PaxosCleanupResponse result = repair.get(); + return !result.wasSuccessful && result.message.contains("Unknown keyspace: " + KEYSPACE); + } + catch (Exception e) + { + throw new RuntimeException(e); + } + }, conditions); + Assert.assertTrue(failedAsExpected); } } finally From 773b3a391155d39bc6f65422d4990b1179d3108d Mon Sep 17 00:00:00 2001 From: Francisco Guerrero <frankgh@apache.org> Date: Thu, 27 Feb 2025 14:43:44 -0800 Subject: [PATCH 156/225] ninja-fix CHANGES.txt add missing entry for CASSANDRA-20320 --- CHANGES.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGES.txt b/CHANGES.txt index afda96a8f7d5..630a13c373b5 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Paxos Repair: NoSuchElementException on DistributedSchema.getKeyspaceMetadata (CASSANDRA-20320) * Improve performance of DistributedSchema.validate for large schemas (CASSANDRA-20360) * Add JSON constraint (CASSANDRA-20273) * Prevent invalid constraint combinations (CASSANDRA-20330) From d276dec0ade704ed01e82185b151077feeb48802 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Wed, 26 Feb 2025 21:14:17 +0100 Subject: [PATCH 157/225] Deduplicate constraint validation logic for supported types patch by Stefan Miklosovic; reviewed by Bernardo Botella for CASSANDRA-20371 --- .../AbstractFunctionConstraint.java | 4 +- .../cql3/constraints/ColumnConstraint.java | 25 +++++++++++- .../cql3/constraints/ColumnConstraints.java | 6 +++ .../cql3/constraints/ConstraintFunction.java | 20 +++++++--- .../constraints/FunctionColumnConstraint.java | 31 ++++++++++----- .../cql3/constraints/JsonConstraint.java | 13 ++++++- .../cql3/constraints/LengthConstraint.java | 39 ++++++------------- .../cql3/constraints/NotNullConstraint.java | 7 ++++ .../constraints/ScalarColumnConstraint.java | 31 ++++++++++++--- .../UnaryFunctionColumnConstraint.java | 13 +++++-- ...WithColumnCqlConstraintValidationTest.java | 2 +- .../contraints/JsonConstraintTest.java | 4 +- 12 files changed, 136 insertions(+), 59 deletions(-) diff --git a/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionConstraint.java index bbfe58fb26a7..6204b96a212c 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionConstraint.java @@ -18,7 +18,7 @@ package org.apache.cassandra.cql3.constraints; -import java.util.Set; +import java.util.List; import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.CqlBuilder; @@ -47,7 +47,7 @@ public String term() return term; } - public abstract Set<Operator> getSupportedOperators(); + public abstract List<Operator> getSupportedOperators(); @Override public void appendCqlTo(CqlBuilder builder) diff --git a/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraint.java index 9733fccdccfc..ddcca653ea30 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraint.java @@ -31,6 +31,8 @@ import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.tcm.serialization.MetadataSerializer; +import static java.lang.String.format; + /** * Common class for the conditions that a CQL Constraint needs to implement to be integrated in the * CQL Constraints framework, with T as a constraint serializer. @@ -52,7 +54,7 @@ public enum ConstraintType // We are serializing its enum position instead of its name. // Changing this enum would affect how that int is interpreted when deserializing. COMPOSED(ColumnConstraints.serializer, new DuplicatesChecker()), - FUNCTION(FunctionColumnConstraint.serializer, FunctionColumnConstraint.Functions.values()), + FUNCTION(FunctionColumnConstraint.serializer, FunctionColumnConstraint.getSatisfiabilityCheckers()), SCALAR(ScalarColumnConstraint.serializer, new ScalarColumnConstraintSatisfiabilityChecker()), UNARY_FUNCTION(UnaryFunctionColumnConstraint.serializer, UnaryFunctionColumnConstraint.Functions.values()); @@ -140,4 +142,25 @@ public void evaluate(AbstractType<?> valueType, ByteBuffer columnValue) throws C * @return the Constraint type serializer */ public abstract ConstraintType getConstraintType(); + + + /** + * Tells what types of columns are supported by this constraint. + * Returning empty list or null means that all types are supported. + * + * @return supported types for given constraint + */ + public abstract List<AbstractType<?>> getSupportedTypes(); + + protected void validateTypes(ColumnMetadata columnMetadata) + { + if (getSupportedTypes() == null || getSupportedTypes().isEmpty()) + return; + + if (!getSupportedTypes().contains(columnMetadata.type.unwrap())) + throw new InvalidConstraintDefinitionException(format("Constraint '%s' can be used only for columns of type %s but it was %s", + name(), + getSupportedTypes(), + columnMetadata.type.getClass())); + } } diff --git a/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraints.java b/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraints.java index 02571976e12b..0acace098f46 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraints.java +++ b/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraints.java @@ -140,6 +140,12 @@ public ConstraintType getConstraintType() return ConstraintType.COMPOSED; } + @Override + public List<AbstractType<?>> getSupportedTypes() + { + return null; + } + public static class DuplicatesChecker implements SatisfiabilityChecker { @Override diff --git a/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java b/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java index bdd36d11818f..a95a4f782dd2 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java +++ b/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java @@ -19,7 +19,7 @@ package org.apache.cassandra.cql3.constraints; import java.nio.ByteBuffer; -import java.util.Set; +import java.util.List; import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.Operator; @@ -38,7 +38,7 @@ */ public abstract class ConstraintFunction { - public static final Set<Operator> DEFAULT_FUNCTION_OPERATORS = Set.of(EQ, NEQ, GTE, GT, LTE, LT); + public static final List<Operator> DEFAULT_FUNCTION_OPERATORS = List.of(EQ, NEQ, GTE, GT, LTE, LT); protected final ColumnIdentifier columnName; protected final String name; @@ -82,12 +82,20 @@ public void evaluate(AbstractType<?> valueType, ByteBuffer columnValue) throws C public abstract void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException; /** - * Return operators this function supports. By default, it returns an empty set, modelling unary function. + * Return operators this function supports. By default, it returns an empty list, modelling unary function. * - * @return set of operators this function is allowed to have. + * @return list of operators this function is allowed to have. */ - public Set<Operator> getSupportedOperators() + public List<Operator> getSupportedOperators() { - return Set.of(); + return List.of(); } + + /** + * Tells what types of columns are supported by this constraint. + * Returning null or empty list means that all types are supported. + * + * @return supported types for given constraint + */ + public abstract List<AbstractType<?>> getSupportedTypes(); } diff --git a/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java index 02f49986b552..2383f4ee9c4d 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.List; -import java.util.Set; import java.util.function.Function; import org.apache.cassandra.cql3.ColumnIdentifier; @@ -64,16 +63,21 @@ public FunctionColumnConstraint prepare() } } - public enum Functions implements SatisfiabilityChecker + public static SatisfiabilityChecker[] getSatisfiabilityCheckers() { - LENGTH(LengthConstraint::new) + SatisfiabilityChecker[] satisfiabilityCheckers = new SatisfiabilityChecker[Functions.values().length]; + for (int i = 0; i < Functions.values().length; i++) { - @Override - public void checkSatisfiability(List<ColumnConstraint<?>> constraints, ColumnMetadata columnMetadata) - { - FUNCTION_SATISFIABILITY_CHECKER.check(name(), constraints, columnMetadata); - } - }; + String name = Functions.values()[i].name(); + satisfiabilityCheckers[i] = (constraints, columnMetadata) -> FUNCTION_SATISFIABILITY_CHECKER.check(name, constraints, columnMetadata); + } + + return satisfiabilityCheckers; + } + + public enum Functions + { + LENGTH(LengthConstraint::new); private final Function<ColumnIdentifier, ConstraintFunction> functionCreator; @@ -100,11 +104,17 @@ public ConstraintFunction function() } @Override - public Set<Operator> getSupportedOperators() + public List<Operator> getSupportedOperators() { return function.getSupportedOperators(); } + @Override + public List<AbstractType<?>> getSupportedTypes() + { + return function.getSupportedTypes(); + } + @Override public String name() { @@ -145,6 +155,7 @@ protected void internalEvaluate(AbstractType<?> valueType, ByteBuffer columnValu public void validate(ColumnMetadata columnMetadata) { validateArgs(columnMetadata); + validateTypes(columnMetadata); function.validate(columnMetadata); } diff --git a/src/java/org/apache/cassandra/cql3/constraints/JsonConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/JsonConstraint.java index 0abd1d511e13..62c961743778 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/JsonConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/JsonConstraint.java @@ -19,10 +19,13 @@ package org.apache.cassandra.cql3.constraints; import java.nio.ByteBuffer; +import java.util.List; import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.Operator; import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.AsciiType; +import org.apache.cassandra.db.marshal.UTF8Type; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.utils.JsonUtils; @@ -31,6 +34,8 @@ public class JsonConstraint extends ConstraintFunction { + private static final List<AbstractType<?>> SUPPORTED_TYPES = List.of(UTF8Type.instance, AsciiType.instance); + public static final String FUNCTION_NAME = "JSON"; public JsonConstraint(ColumnIdentifier columnName) @@ -61,8 +66,12 @@ public void internalEvaluate(AbstractType<?> valueType, Operator relationType, S @Override public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException { - if (!columnMetadata.type.unwrap().isString()) - throw new InvalidConstraintDefinitionException(name + " can be used only for columns of 'text', 'varchar' or 'ascii' types."); + } + + @Override + public List<AbstractType<?>> getSupportedTypes() + { + return SUPPORTED_TYPES; } @Override diff --git a/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java index 708028a6eadf..5b5d8c97dd8d 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java @@ -19,7 +19,7 @@ package org.apache.cassandra.cql3.constraints; import java.nio.ByteBuffer; -import java.util.Set; +import java.util.List; import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.Operator; @@ -34,7 +34,7 @@ public class LengthConstraint extends ConstraintFunction { private static final String NAME = "LENGTH"; - private static final AbstractType<?>[] SUPPORTED_TYPES = new AbstractType[]{ BytesType.instance, UTF8Type.instance, AsciiType.instance }; + private static final List<AbstractType<?>> SUPPORTED_TYPES = List.of(BytesType.instance, UTF8Type.instance, AsciiType.instance); public LengthConstraint(ColumnIdentifier columnName) { @@ -57,45 +57,28 @@ public void internalEvaluate(AbstractType<?> valueType, Operator relationType, S } @Override - public void validate(ColumnMetadata columnMetadata) + public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException { - boolean supported = false; - AbstractType<?> unwrapped = columnMetadata.type.unwrap(); - for (AbstractType<?> supportedType : SUPPORTED_TYPES) - { - if (supportedType == unwrapped) - { - supported = true; - break; - } - } - - if (!supported) - throw invalidConstraintDefinitionException(columnMetadata.type); } @Override - public Set<Operator> getSupportedOperators() + public List<Operator> getSupportedOperators() { return DEFAULT_FUNCTION_OPERATORS; } + @Override + public List<AbstractType<?>> getSupportedTypes() + { + return SUPPORTED_TYPES; + } + private int getValueLength(ByteBuffer value, AbstractType<?> valueType) { if (valueType.getClass() == BytesType.class) - { return value.remaining(); - } - - if (valueType.getClass() == AsciiType.class || valueType.getClass() == UTF8Type.class) + else return ((String) valueType.compose(value)).length(); - - throw invalidConstraintDefinitionException(valueType); - } - - private InvalidConstraintDefinitionException invalidConstraintDefinitionException(AbstractType<?> valueType) - { - throw new InvalidConstraintDefinitionException("Column type " + valueType.getClass() + " is not supported."); } @Override diff --git a/src/java/org/apache/cassandra/cql3/constraints/NotNullConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/NotNullConstraint.java index 0fb164562b34..465db80933bd 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/NotNullConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/NotNullConstraint.java @@ -19,6 +19,7 @@ package org.apache.cassandra.cql3.constraints; import java.nio.ByteBuffer; +import java.util.List; import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.Operator; @@ -57,6 +58,12 @@ public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefi columnMetadata.name)); } + @Override + public List<AbstractType<?>> getSupportedTypes() + { + return null; + } + @Override public boolean equals(Object o) { diff --git a/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java index cd3558945471..70b33a7569a7 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java @@ -21,12 +21,22 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.List; -import java.util.Set; + +import com.google.common.annotations.VisibleForTesting; import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.Operator; import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.ByteType; +import org.apache.cassandra.db.marshal.CounterColumnType; +import org.apache.cassandra.db.marshal.DecimalType; +import org.apache.cassandra.db.marshal.DoubleType; +import org.apache.cassandra.db.marshal.FloatType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.IntegerType; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.db.marshal.ShortType; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.schema.ColumnMetadata; @@ -43,7 +53,13 @@ public class ScalarColumnConstraint extends AbstractFunctionConstraint<ScalarColumnConstraint> { - public static final Set<Operator> SUPPORTED_OPERATORS = Set.of(EQ, NEQ, GTE, GT, LTE, LT); + private static final List<AbstractType<?>> SUPPORTED_TYPES = + List.of(ByteType.instance, CounterColumnType.instance, DecimalType.instance, DoubleType.instance, + FloatType.instance, Int32Type.instance, IntegerType.instance, LongType.instance, + ShortType.instance); + + @VisibleForTesting + public static final List<Operator> SUPPORTED_OPERATORS = List.of(EQ, NEQ, GTE, GT, LTE, LT); public static final Serializer serializer = new Serializer(); @@ -81,11 +97,17 @@ private ScalarColumnConstraint(ColumnIdentifier param, Operator relationType, St } @Override - public Set<Operator> getSupportedOperators() + public List<Operator> getSupportedOperators() { return SUPPORTED_OPERATORS; } + @Override + public List<AbstractType<?>> getSupportedTypes() + { + return SUPPORTED_TYPES; + } + @Override protected void internalEvaluate(AbstractType<?> valueType, ByteBuffer columnValue) { @@ -107,8 +129,7 @@ protected void internalEvaluate(AbstractType<?> valueType, ByteBuffer columnValu @Override public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException { - if (!columnMetadata.type.isNumber()) - throw new InvalidConstraintDefinitionException("Column '" + columnName + "' is not a number type."); + validateTypes(columnMetadata); } @Override diff --git a/src/java/org/apache/cassandra/cql3/constraints/UnaryFunctionColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/UnaryFunctionColumnConstraint.java index bb41f57c239a..d39f7f865d97 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/UnaryFunctionColumnConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/UnaryFunctionColumnConstraint.java @@ -20,7 +20,7 @@ import java.io.IOException; import java.nio.ByteBuffer; -import java.util.Set; +import java.util.List; import java.util.function.Function; import org.apache.cassandra.cql3.ColumnIdentifier; @@ -96,9 +96,15 @@ public MetadataSerializer<UnaryFunctionColumnConstraint> serializer() } @Override - public Set<Operator> getSupportedOperators() + public List<Operator> getSupportedOperators() { - return Set.of(); + return List.of(); + } + + @Override + public List<AbstractType<?>> getSupportedTypes() + { + return function.getSupportedTypes(); } @Override @@ -117,6 +123,7 @@ public void internalEvaluate(AbstractType<?> valueType, ByteBuffer columnValue) public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException { validateArgs(columnMetadata); + validateTypes(columnMetadata); function.validate(columnMetadata); } diff --git a/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnCqlConstraintValidationTest.java b/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnCqlConstraintValidationTest.java index b92f7b1c4111..17c093c9f480 100644 --- a/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnCqlConstraintValidationTest.java +++ b/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnCqlConstraintValidationTest.java @@ -1372,7 +1372,7 @@ public void testCreateTableWithColumnWithClusteringColumnInvalidScalarTypeConstr catch (InvalidRequestException e) { assertTrue(e.getCause() instanceof InvalidRequestException); - assertTrue(e.getCause().getMessage().equals("Column 'pk' is not a number type.")); + assertTrue(e.getCause().getMessage().contains("can be used only for columns of type")); assertTrue(e.getMessage().contains("Error setting schema for test")); } } diff --git a/test/unit/org/apache/cassandra/contraints/JsonConstraintTest.java b/test/unit/org/apache/cassandra/contraints/JsonConstraintTest.java index ba9281223060..5a436892af18 100644 --- a/test/unit/org/apache/cassandra/contraints/JsonConstraintTest.java +++ b/test/unit/org/apache/cassandra/contraints/JsonConstraintTest.java @@ -57,7 +57,9 @@ public void testJsonConstraint() throws Throwable public void testInvalidTypes() { assertThatThrownBy(() -> json.validate(getColumnOfType(IntegerType.instance))) - .hasMessageContaining("JSON can be used only for columns of 'text', 'varchar' or 'ascii' types."); + .hasMessage("Constraint 'JSON' can be used only for columns of type " + + "[org.apache.cassandra.db.marshal.UTF8Type, org.apache.cassandra.db.marshal.AsciiType] " + + "but it was class org.apache.cassandra.db.marshal.IntegerType"); } private void run(String jsonToCheck) throws Throwable From 415eaffb9cc50df5a12a330e14027ef60c07ee02 Mon Sep 17 00:00:00 2001 From: Marcus Eriksson <marcuse@apache.org> Date: Fri, 31 Jan 2025 10:02:49 +0100 Subject: [PATCH 158/225] Reduce heap pressure when initializing CMS Patch by marcuse; reviewed by Sam Tunnicliffe for CASSANDRA-20267 --- CHANGES.txt | 1 + NEWS.txt | 3 +- src/java/org/apache/cassandra/net/Verb.java | 8 +- .../cassandra/schema/SchemaKeyspace.java | 28 +++ .../cassandra/tcm/ClusterMetadataService.java | 2 +- .../org/apache/cassandra/tcm/Startup.java | 7 +- .../migration/CMSInitializationRequest.java | 169 ++++++++++++++++++ .../migration/CMSInitializationResponse.java | 74 ++++++++ .../tcm/migration/ClusterMetadataHolder.java | 95 ---------- .../cassandra/tcm/migration/Election.java | 160 ++++++----------- .../tcm/serialization/MessageSerializers.java | 8 +- ...ClusterMetadataUpgradeIgnoreHostsTest.java | 50 ------ .../upgrade/ClusterMetadataUpgradeTest.java | 36 ++++ 13 files changed, 379 insertions(+), 262 deletions(-) create mode 100644 src/java/org/apache/cassandra/tcm/migration/CMSInitializationRequest.java create mode 100644 src/java/org/apache/cassandra/tcm/migration/CMSInitializationResponse.java delete mode 100644 src/java/org/apache/cassandra/tcm/migration/ClusterMetadataHolder.java delete mode 100644 test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeIgnoreHostsTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 630a13c373b5..685d86119b61 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Reduce heap pressure when initializing CMS (CASSANDRA-20267) * Paxos Repair: NoSuchElementException on DistributedSchema.getKeyspaceMetadata (CASSANDRA-20320) * Improve performance of DistributedSchema.validate for large schemas (CASSANDRA-20360) * Add JSON constraint (CASSANDRA-20273) diff --git a/NEWS.txt b/NEWS.txt index bb8898d53b25..b35a3c02745b 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -149,7 +149,8 @@ Upgrading However, nodes still UP and running the old version will. This will eventually cause the migration to fail, as the cluster will not be in agreement. - > nodetool cms initialize - Got mismatching cluster metadatas from [/x.x.x.x:7000] aborting migration + Got mismatching cluster metadatas. Check logs on peers ([/x.x.x.x:7000]) for details of mismatches. + Aborting migration. See 'nodetool help' or 'nodetool help <command>'. If the cms initialize command fails, it will indicate which nodes’ current metadata does not agree with the node where the command was executed. To mitigate this situation, bring any mismatching nodes DOWN and rerun the diff --git a/src/java/org/apache/cassandra/net/Verb.java b/src/java/org/apache/cassandra/net/Verb.java index 17c4550fc09a..c2cce663efd6 100644 --- a/src/java/org/apache/cassandra/net/Verb.java +++ b/src/java/org/apache/cassandra/net/Verb.java @@ -94,7 +94,9 @@ import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.FetchCMSLog; import org.apache.cassandra.tcm.FetchPeerLog; +import org.apache.cassandra.tcm.migration.CMSInitializationResponse; import org.apache.cassandra.tcm.migration.Election; +import org.apache.cassandra.tcm.migration.CMSInitializationRequest; import org.apache.cassandra.tcm.sequences.DataMovements; import org.apache.cassandra.tcm.serialization.MessageSerializers; import org.apache.cassandra.utils.BooleanSerializer; @@ -232,9 +234,9 @@ public enum Verb TCM_NOTIFY_RSP (806, P0, rpcTimeout, INTERNAL_METADATA, () -> Epoch.messageSerializer, () -> ResponseVerbHandler.instance ), TCM_NOTIFY_REQ (807, P0, rpcTimeout, INTERNAL_METADATA, MessageSerializers::logStateSerializer, () -> logNotifyHandler(), TCM_NOTIFY_RSP ), TCM_CURRENT_EPOCH_REQ (808, P0, rpcTimeout, INTERNAL_METADATA, () -> Epoch.messageSerializer, () -> currentEpochRequestHandler(), TCM_NOTIFY_RSP ), - TCM_INIT_MIG_RSP (809, P0, rpcTimeout, INTERNAL_METADATA, MessageSerializers::metadataHolderSerializer, () -> ResponseVerbHandler.instance ), - TCM_INIT_MIG_REQ (810, P0, rpcTimeout, INTERNAL_METADATA, () -> Election.Initiator.serializer, () -> Election.instance.prepareHandler, TCM_INIT_MIG_RSP ), - TCM_ABORT_MIG (811, P0, rpcTimeout, INTERNAL_METADATA, () -> Election.Initiator.serializer, () -> Election.instance.abortHandler, TCM_INIT_MIG_RSP ), + TCM_INIT_MIG_RSP (809, P0, rpcTimeout, INTERNAL_METADATA, () -> CMSInitializationResponse.serializer, () -> ResponseVerbHandler.instance ), + TCM_INIT_MIG_REQ (810, P0, rpcTimeout, INTERNAL_METADATA, MessageSerializers::initRequestSerializer, () -> Election.instance.prepareHandler, TCM_INIT_MIG_RSP ), + TCM_ABORT_MIG (811, P0, rpcTimeout, INTERNAL_METADATA, () -> CMSInitializationRequest.Initiator.serializer,() -> Election.instance.abortHandler, TCM_INIT_MIG_RSP ), TCM_DISCOVER_RSP (812, P0, rpcTimeout, INTERNAL_METADATA, () -> Discovery.serializer, () -> ResponseVerbHandler.instance ), TCM_DISCOVER_REQ (813, P0, rpcTimeout, INTERNAL_METADATA, () -> NoPayload.serializer, () -> Discovery.instance.requestHandler, TCM_DISCOVER_RSP ), TCM_FETCH_PEER_LOG_RSP (818, P0, rpcTimeout, FETCH_LOG, MessageSerializers::logStateSerializer, () -> ResponseVerbHandler.instance ), diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java index a6877cc6318c..b3ed1c2702e1 100644 --- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java @@ -364,6 +364,34 @@ private static void flush() ALL.forEach(table -> FBUtilities.waitOnFuture(getSchemaCFS(table).forceFlush(ColumnFamilyStore.FlushReason.INTERNALLY_FORCED))); } + /** + * Read schema from system keyspace and calculate MD5 digest of every row, resulting digest + * will be converted into UUID which would act as content-based version of the schema. + * + * Only used when initializing the CMS + */ + public static UUID calculateSchemaDigest() + { + Digest digest = Digest.forSchema(); + for (String table : ALL) + { + ReadCommand cmd = getReadCommandForTableSchema(table); + try (ReadExecutionController executionController = cmd.executionController(); + PartitionIterator schema = cmd.executeInternal(executionController)) + { + while (schema.hasNext()) + { + try (RowIterator partition = schema.next()) + { + if (!isSystemKeyspaceSchemaPartition(partition.partitionKey())) + RowIterators.digest(partition, digest); + } + } + } + } + return UUID.nameUUIDFromBytes(digest.digest()); + } + /** * @param schemaTableName The name of the table responsible for part of the schema * @return CFS responsible to hold low-level serialized schema diff --git a/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java b/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java index 555cdb7f9383..8adfb81051db 100644 --- a/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java +++ b/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java @@ -362,7 +362,7 @@ public void upgradeFromGossip(List<String> ignoredEndpoints) !ignored.contains(ep)) .collect(toImmutableSet()); - Election.instance.nominateSelf(candidates, ignored, metadata::equals, metadata); + Election.instance.nominateSelf(candidates, ignored, metadata, true); ClusterMetadataService.instance().triggerSnapshot(); } else diff --git a/src/java/org/apache/cassandra/tcm/Startup.java b/src/java/org/apache/cassandra/tcm/Startup.java index b184866f5725..151d24ab90c6 100644 --- a/src/java/org/apache/cassandra/tcm/Startup.java +++ b/src/java/org/apache/cassandra/tcm/Startup.java @@ -59,6 +59,7 @@ import org.apache.cassandra.tcm.membership.NodeId; import org.apache.cassandra.tcm.membership.NodeState; import org.apache.cassandra.tcm.migration.Election; +import org.apache.cassandra.tcm.migration.CMSInitializationRequest; import org.apache.cassandra.tcm.ownership.UniformRangePlacement; import org.apache.cassandra.tcm.sequences.InProgressSequences; import org.apache.cassandra.tcm.sequences.ReconfigureCMS; @@ -229,8 +230,8 @@ public static void initializeForDiscovery(Runnable initMessaging) { Election.instance.nominateSelf(candidates.nodes(), Collections.singleton(FBUtilities.getBroadcastAddressAndPort()), - (cm) -> true, - null); + ClusterMetadata.current(), + false); } } @@ -243,7 +244,7 @@ public static void initializeForDiscovery(Runnable initMessaging) } else { - Election.Initiator initiator = Election.instance.initiator(); + CMSInitializationRequest.Initiator initiator = Election.instance.initiator(); candidates = Discovery.instance.discoverOnce(initiator == null ? null : initiator.initiator); } Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS); diff --git a/src/java/org/apache/cassandra/tcm/migration/CMSInitializationRequest.java b/src/java/org/apache/cassandra/tcm/migration/CMSInitializationRequest.java new file mode 100644 index 000000000000..dac50e5edbdc --- /dev/null +++ b/src/java/org/apache/cassandra/tcm/migration/CMSInitializationRequest.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.tcm.migration; + +import java.io.IOException; +import java.util.Objects; +import java.util.UUID; + +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.schema.SchemaKeyspace; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.membership.Directory; +import org.apache.cassandra.tcm.membership.NodeVersion; +import org.apache.cassandra.tcm.ownership.TokenMap; +import org.apache.cassandra.tcm.serialization.Version; +import org.apache.cassandra.utils.UUIDSerializer; + +public class CMSInitializationRequest +{ + public static final IVersionedSerializer<CMSInitializationRequest> defaultMessageSerializer = new Serializer(NodeVersion.CURRENT.serializationVersion()); + + private static volatile Serializer serializerCache; + + public static IVersionedSerializer<CMSInitializationRequest> messageSerializer(Version version) + { + Serializer cached = serializerCache; + if (cached != null && cached.serializationVersion.equals(version)) + return cached; + cached = new Serializer(version); + serializerCache = cached; + return cached; + } + + public final Initiator initiator; + public final Directory directory; + public final TokenMap tokenMap; + public final UUID schemaVersion; + + public CMSInitializationRequest(InetAddressAndPort initiator, UUID initToken, ClusterMetadata metadata) + { + this(new Initiator(initiator, initToken), metadata.directory, metadata.tokenMap, SchemaKeyspace.calculateSchemaDigest()); + } + + public CMSInitializationRequest(Initiator initiator, Directory directory, TokenMap tokenMap, UUID schemaVersion) + { + this.initiator = initiator; + this.directory = directory; + this.tokenMap = tokenMap; + this.schemaVersion = schemaVersion; + } + + public static class Serializer implements IVersionedSerializer<CMSInitializationRequest> + { + private final Version serializationVersion; + + public Serializer(Version serializationVersion) + { + this.serializationVersion = serializationVersion; + } + + @Override + public void serialize(CMSInitializationRequest t, DataOutputPlus out, int version) throws IOException + { + Initiator.serializer.serialize(t.initiator, out, version); + Directory.serializer.serialize(t.directory, out, serializationVersion); + TokenMap.serializer.serialize(t.tokenMap, out, serializationVersion); + UUIDSerializer.serializer.serialize(t.schemaVersion, out, version); + } + + @Override + public CMSInitializationRequest deserialize(DataInputPlus in, int version) throws IOException + { + Initiator initiator = Initiator.serializer.deserialize(in, version); + Directory directory = Directory.serializer.deserialize(in, serializationVersion); + TokenMap tokenMap = TokenMap.serializer.deserialize(in, serializationVersion); + UUID schemaVersion = UUIDSerializer.serializer.deserialize(in, version); + return new CMSInitializationRequest(initiator, directory, tokenMap, schemaVersion); + } + + @Override + public long serializedSize(CMSInitializationRequest t, int version) + { + return Initiator.serializer.serializedSize(t.initiator, version) + + Directory.serializer.serializedSize(t.directory, serializationVersion) + + TokenMap.serializer.serializedSize(t.tokenMap, serializationVersion) + + UUIDSerializer.serializer.serializedSize(t.schemaVersion, version); + } + } + + public static class Initiator + { + public static final Serializer serializer = new Serializer(); + public final InetAddressAndPort initiator; + public final UUID initToken; + + public Initiator(InetAddressAndPort initiator, UUID initToken) + { + this.initiator = initiator; + this.initToken = initToken; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (!(o instanceof Initiator)) return false; + Initiator other = (Initiator) o; + return Objects.equals(initiator, other.initiator) && Objects.equals(initToken, other.initToken); + } + + @Override + public int hashCode() + { + return Objects.hash(initiator, initToken); + } + + @Override + public String toString() + { + return "Initiator{" + + "initiator=" + initiator + + ", initToken=" + initToken + + '}'; + } + + public static class Serializer implements IVersionedSerializer<Initiator> + { + @Override + public void serialize(Initiator t, DataOutputPlus out, int version) throws IOException + { + InetAddressAndPort.Serializer.inetAddressAndPortSerializer.serialize(t.initiator, out, version); + UUIDSerializer.serializer.serialize(t.initToken, out, version); + } + + @Override + public Initiator deserialize(DataInputPlus in, int version) throws IOException + { + return new Initiator(InetAddressAndPort.Serializer.inetAddressAndPortSerializer.deserialize(in, version), + UUIDSerializer.serializer.deserialize(in, version)); + } + + @Override + public long serializedSize(Initiator t, int version) + { + return InetAddressAndPort.Serializer.inetAddressAndPortSerializer.serializedSize(t.initiator, version) + + UUIDSerializer.serializer.serializedSize(t.initToken, version); + } + } + } +} diff --git a/src/java/org/apache/cassandra/tcm/migration/CMSInitializationResponse.java b/src/java/org/apache/cassandra/tcm/migration/CMSInitializationResponse.java new file mode 100644 index 000000000000..73502fdb630b --- /dev/null +++ b/src/java/org/apache/cassandra/tcm/migration/CMSInitializationResponse.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.tcm.migration; + +import java.io.IOException; + +import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; + +public class CMSInitializationResponse +{ + public static final IVersionedSerializer<CMSInitializationResponse> serializer = new Serializer(); + + public final CMSInitializationRequest.Initiator initiator; + public final boolean metadataMatches; + + public CMSInitializationResponse(CMSInitializationRequest.Initiator initiator, boolean metadataMatches) + { + this.initiator = initiator; + this.metadataMatches = metadataMatches; + } + + @Override + public String toString() + { + return "CMSInitializationResponse{" + + "initiator=" + initiator + + ", metadataMatches=" + metadataMatches + + '}'; + } + + private static class Serializer implements IVersionedSerializer<CMSInitializationResponse> + { + @Override + public void serialize(CMSInitializationResponse t, DataOutputPlus out, int version) throws IOException + { + CMSInitializationRequest.Initiator.serializer.serialize(t.initiator, out, version); + out.writeBoolean(t.metadataMatches); + } + + @Override + public CMSInitializationResponse deserialize(DataInputPlus in, int version) throws IOException + { + CMSInitializationRequest.Initiator coordinator = CMSInitializationRequest.Initiator.serializer.deserialize(in, version); + boolean metadataMatches = in.readBoolean(); + return new CMSInitializationResponse(coordinator, metadataMatches); + } + + @Override + public long serializedSize(CMSInitializationResponse t, int version) + { + return CMSInitializationRequest.Initiator.serializer.serializedSize(t.initiator, version) + + TypeSizes.sizeof(t.metadataMatches); + } + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/tcm/migration/ClusterMetadataHolder.java b/src/java/org/apache/cassandra/tcm/migration/ClusterMetadataHolder.java deleted file mode 100644 index fbb0bfa70159..000000000000 --- a/src/java/org/apache/cassandra/tcm/migration/ClusterMetadataHolder.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.tcm.migration; - -import java.io.IOException; - -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.tcm.ClusterMetadata; -import org.apache.cassandra.tcm.membership.NodeVersion; -import org.apache.cassandra.tcm.serialization.VerboseMetadataSerializer; -import org.apache.cassandra.tcm.serialization.Version; - -public class ClusterMetadataHolder -{ - public static final IVersionedSerializer<ClusterMetadataHolder> defaultMessageSerializer = new ClusterMetadataHolder.Serializer(NodeVersion.CURRENT.serializationVersion()); - - private static volatile Serializer serializerCache; - public static IVersionedSerializer<ClusterMetadataHolder> messageSerializer(Version version) - { - Serializer cached = serializerCache; - if (cached != null && cached.serializationVersion.equals(version)) - return cached; - cached = new Serializer(version); - serializerCache = cached; - return cached; - } - - public final Election.Initiator coordinator; - public final ClusterMetadata metadata; - - public ClusterMetadataHolder(Election.Initiator coordinator, ClusterMetadata metadata) - { - this.coordinator = coordinator; - this.metadata = metadata; - } - - @Override - public String toString() - { - return "ClusterMetadataHolder{" + - "coordinator=" + coordinator + - ", epoch=" + metadata.epoch + - '}'; - } - - private static class Serializer implements IVersionedSerializer<ClusterMetadataHolder> - { - private final Version serializationVersion; - - public Serializer(Version serializationVersion) - { - this.serializationVersion = serializationVersion; - } - - @Override - public void serialize(ClusterMetadataHolder t, DataOutputPlus out, int version) throws IOException - { - Election.Initiator.serializer.serialize(t.coordinator, out, version); - VerboseMetadataSerializer.serialize(ClusterMetadata.serializer, t.metadata, out, serializationVersion); - } - - @Override - public ClusterMetadataHolder deserialize(DataInputPlus in, int version) throws IOException - { - Election.Initiator coordinator = Election.Initiator.serializer.deserialize(in, version); - ClusterMetadata metadata = VerboseMetadataSerializer.deserialize(ClusterMetadata.serializer, in); - return new ClusterMetadataHolder(coordinator, metadata); - } - - @Override - public long serializedSize(ClusterMetadataHolder t, int version) - { - return Election.Initiator.serializer.serializedSize(t.coordinator, version) + - VerboseMetadataSerializer.serializedSize(ClusterMetadata.serializer, t.metadata, serializationVersion); - } - } -} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/tcm/migration/Election.java b/src/java/org/apache/cassandra/tcm/migration/Election.java index 2bd2f7a39248..04e1a8fa4c8f 100644 --- a/src/java/org/apache/cassandra/tcm/migration/Election.java +++ b/src/java/org/apache/cassandra/tcm/migration/Election.java @@ -20,14 +20,11 @@ import java.io.IOException; import java.util.Collection; -import java.util.HashMap; import java.util.HashSet; -import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Function; import java.util.stream.Collectors; import com.google.common.collect.Sets; @@ -36,12 +33,12 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.db.SystemKeyspace; -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.schema.SchemaKeyspace; import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.Startup; +import org.apache.cassandra.tcm.membership.Directory; +import org.apache.cassandra.tcm.ownership.TokenMap; import org.apache.cassandra.tcm.transformations.Register; import org.apache.cassandra.net.MessageDelivery; import org.apache.cassandra.net.IVerbHandler; @@ -52,7 +49,6 @@ import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.Pair; -import org.apache.cassandra.utils.UUIDSerializer; /** * Election process establishes initial CMS leader, from which you can further evolve cluster metadata. @@ -60,9 +56,9 @@ public class Election { private static final Logger logger = LoggerFactory.getLogger(Election.class); - private static final Initiator MIGRATED = new Initiator(null, null); + private static final CMSInitializationRequest.Initiator MIGRATED = new CMSInitializationRequest.Initiator(null, null); - private final AtomicReference<Initiator> initiator = new AtomicReference<>(); + private final AtomicReference<CMSInitializationRequest.Initiator> initiator = new AtomicReference<>(); public static Election instance = new Election(); @@ -83,7 +79,7 @@ private Election(MessageDelivery messaging) this.abortHandler = new AbortHandler(); } - public void nominateSelf(Set<InetAddressAndPort> candidates, Set<InetAddressAndPort> ignoredEndpoints, Function<ClusterMetadata, Boolean> isMatch, ClusterMetadata metadata) + public void nominateSelf(Set<InetAddressAndPort> candidates, Set<InetAddressAndPort> ignoredEndpoints, ClusterMetadata metadata, boolean verifyAllPeersMetadata) { Set<InetAddressAndPort> sendTo = new HashSet<>(candidates); sendTo.removeAll(ignoredEndpoints); @@ -91,7 +87,7 @@ public void nominateSelf(Set<InetAddressAndPort> candidates, Set<InetAddressAndP try { - initiate(sendTo, isMatch, metadata); + initiate(sendTo, metadata, verifyAllPeersMetadata); finish(sendTo); } catch (Exception e) @@ -101,13 +97,14 @@ public void nominateSelf(Set<InetAddressAndPort> candidates, Set<InetAddressAndP } } - private void initiate(Set<InetAddressAndPort> sendTo, Function<ClusterMetadata, Boolean> isMatch, ClusterMetadata metadata) + private void initiate(Set<InetAddressAndPort> sendTo, ClusterMetadata metadata, boolean verifyAllPeersMetadata) { - if (!updateInitiator(null, new Initiator(FBUtilities.getBroadcastAddressAndPort(), UUID.randomUUID()))) + CMSInitializationRequest initializationRequest = new CMSInitializationRequest(FBUtilities.getBroadcastAddressAndPort(), UUID.randomUUID(), metadata); + if (!updateInitiator(null, initializationRequest.initiator)) throw new IllegalStateException("Migration already initiated by " + initiator.get()); logger.info("No previous migration detected, initiating"); - Collection<Pair<InetAddressAndPort, ClusterMetadataHolder>> metadatas = MessageDelivery.fanoutAndWait(messaging, sendTo, Verb.TCM_INIT_MIG_REQ, initiator.get()); + Collection<Pair<InetAddressAndPort, CMSInitializationResponse>> metadatas = MessageDelivery.fanoutAndWait(messaging, sendTo, Verb.TCM_INIT_MIG_REQ, initializationRequest); if (metadatas.size() != sendTo.size()) { Set<InetAddressAndPort> responded = metadatas.stream().map(p -> p.left).collect(Collectors.toSet()); @@ -116,45 +113,38 @@ private void initiate(Set<InetAddressAndPort> sendTo, Function<ClusterMetadata, throw new IllegalStateException(msg); } - Set<InetAddressAndPort> mismatching = metadatas.stream().filter(p -> !isMatch.apply(p.right.metadata)).map(p -> p.left).collect(Collectors.toSet()); - if (!mismatching.isEmpty()) + if (verifyAllPeersMetadata) { - String msg = String.format("Got mismatching cluster metadatas from %s aborting migration", mismatching); - Map<InetAddressAndPort, ClusterMetadataHolder> metadataMap = new HashMap<>(); - metadatas.forEach(pair -> metadataMap.put(pair.left, pair.right)); - if (metadata != null) + Set<InetAddressAndPort> mismatching = metadatas.stream().filter(p -> !p.right.metadataMatches).map(p -> p.left).collect(Collectors.toSet()); + if (!mismatching.isEmpty()) { - for (InetAddressAndPort e : mismatching) - { - logger.warn("Diff with {}", e); - metadata.dumpDiff(metadataMap.get(e).metadata); - } + String msg = String.format("Got mismatching cluster metadatas. Check logs on peers (%s) for details of mismatches. Aborting migration.", mismatching); + throw new IllegalStateException(msg); } - throw new IllegalStateException(msg); } } private void finish(Set<InetAddressAndPort> sendTo) { - Initiator currentCoordinator = initiator.get(); - assert currentCoordinator.initiator.equals(FBUtilities.getBroadcastAddressAndPort()); + CMSInitializationRequest.Initiator currentInitiator = initiator.get(); + assert currentInitiator.initiator.equals(FBUtilities.getBroadcastAddressAndPort()); Startup.initializeAsFirstCMSNode(); Register.maybeRegister(); SystemKeyspace.setLocalHostId(ClusterMetadata.current().myNodeId().toUUID()); - updateInitiator(currentCoordinator, MIGRATED); + updateInitiator(currentInitiator, MIGRATED); MessageDelivery.fanoutAndWait(messaging, sendTo, Verb.TCM_NOTIFY_REQ, DistributedMetadataLogKeyspace.getLogState(Epoch.EMPTY, false)); } private void abort(Set<InetAddressAndPort> sendTo) { - Initiator init = initiator.getAndSet(null); + CMSInitializationRequest.Initiator init = initiator.getAndSet(null); for (InetAddressAndPort ep : sendTo) messaging.send(Message.out(Verb.TCM_ABORT_MIG, init), ep); } - public Initiator initiator() + public CMSInitializationRequest.Initiator initiator() { return initiator.get(); } @@ -164,103 +154,63 @@ public void migrated() initiator.set(MIGRATED); } - private boolean updateInitiator(Initiator expected, Initiator newCoordinator) + private boolean updateInitiator(CMSInitializationRequest.Initiator expected, CMSInitializationRequest.Initiator newInitiator) { - Initiator current = initiator.get(); - return Objects.equals(current, expected) && initiator.compareAndSet(current, newCoordinator); + CMSInitializationRequest.Initiator current = initiator.get(); + return Objects.equals(current, expected) && initiator.compareAndSet(current, newInitiator); } public boolean isMigrating() { - Initiator coordinator = initiator(); - return coordinator != null && coordinator != MIGRATED; + CMSInitializationRequest.Initiator initiator = initiator(); + return initiator != null && initiator != MIGRATED; } - public class PrepareHandler implements IVerbHandler<Initiator> + public class PrepareHandler implements IVerbHandler<CMSInitializationRequest> { @Override - public void doVerb(Message<Initiator> message) throws IOException + public void doVerb(Message<CMSInitializationRequest> message) throws IOException { logger.info("Received election initiation message {} from {}", message.payload, message.from()); - if (!updateInitiator(null, message.payload)) + if (!updateInitiator(null, message.payload.initiator)) throw new IllegalStateException(String.format("Got duplicate initiate migration message from %s, migration is already started by %s", message.from(), initiator())); - // todo; disallow ANY changes to state managed in ClusterMetadata logger.info("Sending initiation response"); - messaging.send(message.responseWith(new ClusterMetadataHolder(message.payload, ClusterMetadata.current())), message.from()); + Directory initiatorDirectory = message.payload.directory; + TokenMap initiatorTokenMap = message.payload.tokenMap; + UUID initiatorSchemaVersion = message.payload.schemaVersion; + ClusterMetadata metadata = ClusterMetadata.current(); + boolean match = true; + if (!initiatorDirectory.equals(metadata.directory)) + { + match = false; + logger.warn("Initiator directory different from our"); + initiatorDirectory.dumpDiff(metadata.directory); + } + if (!initiatorTokenMap.equals(metadata.tokenMap)) + { + match = false; + logger.warn("Initiator tokenmap different from ours"); + initiatorTokenMap.dumpDiff(metadata.tokenMap); + } + UUID schemaDigest = SchemaKeyspace.calculateSchemaDigest(); + if (!initiatorSchemaVersion.equals(schemaDigest)) + { + match = false; + logger.warn("Initiator schema different from our: {} != {}", initiatorSchemaVersion, schemaDigest); + } + messaging.send(message.responseWith(new CMSInitializationResponse(message.payload.initiator, match)), message.from()); } } - public class AbortHandler implements IVerbHandler<Initiator> + public class AbortHandler implements IVerbHandler<CMSInitializationRequest.Initiator> { @Override - public void doVerb(Message<Initiator> message) throws IOException + public void doVerb(Message<CMSInitializationRequest.Initiator> message) throws IOException { logger.info("Received election abort message {} from {}", message.payload, message.from()); if (!message.from().equals(initiator().initiator) || !updateInitiator(message.payload, null)) logger.error("Could not clear initiator - initiator is set to {}, abort message received from {}", initiator(), message.payload); } } - - public static class Initiator - { - public static final Serializer serializer = new Serializer(); - - public final InetAddressAndPort initiator; - public final UUID initToken; - - public Initiator(InetAddressAndPort initiator, UUID initToken) - { - this.initiator = initiator; - this.initToken = initToken; - } - - @Override - public boolean equals(Object o) - { - if (this == o) return true; - if (!(o instanceof Initiator)) return false; - Initiator other = (Initiator) o; - return Objects.equals(initiator, other.initiator) && Objects.equals(initToken, other.initToken); - } - - @Override - public int hashCode() - { - return Objects.hash(initiator, initToken); - } - - @Override - public String toString() - { - return "Initiator{" + - "initiator=" + initiator + - ", initToken=" + initToken + - '}'; - } - - public static class Serializer implements IVersionedSerializer<Initiator> - { - @Override - public void serialize(Initiator t, DataOutputPlus out, int version) throws IOException - { - InetAddressAndPort.Serializer.inetAddressAndPortSerializer.serialize(t.initiator, out, version); - UUIDSerializer.serializer.serialize(t.initToken, out, version); - } - - @Override - public Initiator deserialize(DataInputPlus in, int version) throws IOException - { - return new Initiator(InetAddressAndPort.Serializer.inetAddressAndPortSerializer.deserialize(in, version), - UUIDSerializer.serializer.deserialize(in, version)); - } - - @Override - public long serializedSize(Initiator t, int version) - { - return InetAddressAndPort.Serializer.inetAddressAndPortSerializer.serializedSize(t.initiator, version) + - UUIDSerializer.serializer.serializedSize(t.initToken, version); - } - } - } } diff --git a/src/java/org/apache/cassandra/tcm/serialization/MessageSerializers.java b/src/java/org/apache/cassandra/tcm/serialization/MessageSerializers.java index cfced3426bfe..deef0a4b1bd4 100644 --- a/src/java/org/apache/cassandra/tcm/serialization/MessageSerializers.java +++ b/src/java/org/apache/cassandra/tcm/serialization/MessageSerializers.java @@ -23,7 +23,7 @@ import org.apache.cassandra.tcm.Commit; import org.apache.cassandra.tcm.log.LogState; import org.apache.cassandra.tcm.membership.NodeVersion; -import org.apache.cassandra.tcm.migration.ClusterMetadataHolder; +import org.apache.cassandra.tcm.migration.CMSInitializationRequest; /** * Provides IVersionedSerializers for internode messages where the payload includes @@ -69,13 +69,13 @@ public static IVersionedSerializer<Commit> commitSerializer() return Commit.messageSerializer(metadata.directory.clusterMinVersion.serializationVersion()); } - public static IVersionedSerializer<ClusterMetadataHolder> metadataHolderSerializer() + public static IVersionedSerializer<CMSInitializationRequest> initRequestSerializer() { ClusterMetadata metadata = ClusterMetadata.currentNullable(); if (metadata == null || metadata.directory.clusterMinVersion.serializationVersion == NodeVersion.CURRENT.serializationVersion) - return ClusterMetadataHolder.defaultMessageSerializer; + return CMSInitializationRequest.defaultMessageSerializer; assert !metadata.directory.clusterMinVersion.serializationVersion().equals(NodeVersion.CURRENT.serializationVersion()); - return ClusterMetadataHolder.messageSerializer(metadata.directory.clusterMinVersion.serializationVersion()); + return CMSInitializationRequest.messageSerializer(metadata.directory.clusterMinVersion.serializationVersion()); } } diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeIgnoreHostsTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeIgnoreHostsTest.java deleted file mode 100644 index d89b902d7c7a..000000000000 --- a/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeIgnoreHostsTest.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.distributed.upgrade; - -import org.junit.Test; - -import org.apache.cassandra.distributed.Constants; -import org.apache.cassandra.distributed.api.Feature; - -public class ClusterMetadataUpgradeIgnoreHostsTest extends UpgradeTestBase -{ - @Test - public void upgradeIgnoreHostsTest() throws Throwable - { - new TestCase() - .nodes(3) - .nodesToUpgrade(1, 2, 3) - .withConfig((cfg) -> cfg.with(Feature.NETWORK, Feature.GOSSIP) - .set(Constants.KEY_DTEST_FULL_STARTUP, true)) - .upgradesToCurrentFrom(v41) - .setup((cluster) -> { - cluster.schemaChange(withKeyspace("ALTER KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor':2}")); - cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))"); - }) - .runAfterClusterUpgrade((cluster) -> { - // todo; isolate node 3 - actually shutting it down makes us throw exceptions when test finishes - cluster.filters().allVerbs().to(3).drop(); - cluster.filters().allVerbs().from(3).drop(); - cluster.get(1).nodetoolResult("cms", "initialize").asserts().failure(); // node3 unreachable - cluster.get(1).nodetoolResult("cms", "initialize", "--ignore", "127.0.0.1").asserts().failure(); // can't ignore localhost - cluster.get(1).nodetoolResult("cms", "initialize", "--ignore", "127.0.0.3").asserts().success(); - }).run(); - } -} diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeTest.java index 87399d8a44de..930933425c25 100644 --- a/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeTest.java +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeTest.java @@ -27,6 +27,12 @@ import org.apache.cassandra.distributed.api.Feature; import org.apache.cassandra.distributed.api.IInvokableInstance; import org.apache.cassandra.distributed.shared.ClusterUtils; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.ClusterMetadataService; +import org.apache.cassandra.tcm.membership.Directory; +import org.apache.cassandra.tcm.membership.Location; +import org.apache.cassandra.tcm.membership.NodeAddresses; import org.apache.cassandra.tcm.membership.NodeId; import static org.junit.Assert.assertFalse; @@ -79,4 +85,34 @@ public void upgradeSystemKeyspaces() throws Throwable assertTrue(Arrays.toString(desc[0]).contains("NetworkTopologyStrategy")); }).run(); } + + + @Test + public void upgradeMismatchTest() throws Throwable + { + new TestCase() + .nodes(3) + .nodesToUpgrade(1, 2, 3) + .withConfig((cfg) -> cfg.with(Feature.NETWORK, Feature.GOSSIP) + .set(Constants.KEY_DTEST_FULL_STARTUP, true)) + .upgradesToCurrentFrom(v50) + .setup((cluster) -> { + cluster.schemaChange(withKeyspace("ALTER KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor':2}")); + cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))"); + }) + .runAfterClusterUpgrade((cluster) -> { + IInvokableInstance n3 = ((IInvokableInstance) cluster.get(3)); + n3.runOnInstance(() -> { + ClusterMetadata metadata = ClusterMetadata.current(); + Directory diffingDirectory = metadata.directory.with(new NodeAddresses(InetAddressAndPort.getByNameUnchecked("127.0.0.99")), new Location("aaa", "bbb")); + ClusterMetadata diffing = ClusterMetadata.current().transformer().with(diffingDirectory).buildForGossipMode(); + ClusterMetadataService.instance().setFromGossip(diffing); + }); + cluster.get(1).nodetoolResult("cms", "initialize").asserts().failure(); + cluster.get(3).logs().watchFor("Initiator directory different from our"); + cluster.get(1).nodetoolResult("cms", "initialize", "--ignore", "127.0.0.3").asserts().success(); + cluster.schemaChange(withKeyspace("create table %s.tbl2 (id int primary key)")); + }).run(); + } + } From 2bc24da8417c944bc1470ea4e480c5ca6bb14852 Mon Sep 17 00:00:00 2001 From: Marcus Eriksson <marcuse@apache.org> Date: Mon, 3 Mar 2025 08:38:33 +0100 Subject: [PATCH 159/225] Allow empty placements when deserializing cluster metadata Patch by marcuse; reviewed by Sam Tunnicliffe for CASSANDRA-20343 --- CHANGES.txt | 1 + .../tcm/ownership/ReplicaGroups.java | 3 +- ...terMetadataSerializationRoundTripTest.java | 90 +++++++++++++++++++ 3 files changed, 92 insertions(+), 2 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/test/log/ClusterMetadataSerializationRoundTripTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 685d86119b61..d0778def1844 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Allow empty placements when deserializing cluster metadata (CASSANDRA-20343) * Reduce heap pressure when initializing CMS (CASSANDRA-20267) * Paxos Repair: NoSuchElementException on DistributedSchema.getKeyspaceMetadata (CASSANDRA-20320) * Improve performance of DistributedSchema.validate for large schemas (CASSANDRA-20360) diff --git a/src/java/org/apache/cassandra/tcm/ownership/ReplicaGroups.java b/src/java/org/apache/cassandra/tcm/ownership/ReplicaGroups.java index fbf84b57ae94..adc26ff820c9 100644 --- a/src/java/org/apache/cassandra/tcm/ownership/ReplicaGroups.java +++ b/src/java/org/apache/cassandra/tcm/ownership/ReplicaGroups.java @@ -487,9 +487,8 @@ public ReplicaGroups deserialize(DataInputPlus in, IPartitioner partitioner, Ver InetAddressAndPort replicaAddress = InetAddressAndPort.MetadataSerializer.serializer.deserialize(in, version); boolean isFull = in.readBoolean(); replicas.add(new Replica(replicaAddress, replicaRange, isFull)); - } - EndpointsForRange efr = EndpointsForRange.copyOf(replicas); + EndpointsForRange efr = replicas.isEmpty() ? EndpointsForRange.builder(range).build() : EndpointsForRange.copyOf(replicas); result.put(range, VersionedEndpoints.forRange(lastModified, efr)); } return new ReplicaGroups(result); diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/ClusterMetadataSerializationRoundTripTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/ClusterMetadataSerializationRoundTripTest.java new file mode 100644 index 000000000000..9ac01cb5510d --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/log/ClusterMetadataSerializationRoundTripTest.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.log; + +import java.io.IOException; + +import org.junit.Test; + +import net.bytebuddy.ByteBuddy; +import net.bytebuddy.dynamic.loading.ClassLoadingStrategy; +import net.bytebuddy.implementation.MethodDelegation; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.test.TestBaseImpl; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.locator.NetworkTopologyStrategy; +import org.apache.cassandra.schema.ReplicationParams; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.ClusterMetadataService; +import org.apache.cassandra.tcm.Epoch; +import org.apache.cassandra.tcm.membership.NodeVersion; + +import static net.bytebuddy.matcher.ElementMatchers.named; +import static net.bytebuddy.matcher.ElementMatchers.takesArguments; + +public class ClusterMetadataSerializationRoundTripTest extends TestBaseImpl +{ + @Test + public void testEmptyPlacements() throws IOException + { + try (Cluster cluster = init(builder().withNodes(1) + .withConfig(c -> c.with(Feature.NETWORK, Feature.GOSSIP)) + .withInstanceInitializer(BBHelper::install) + .start())) + { + cluster.schemaChange("create keyspace x with replication = { 'class': 'org.apache.cassandra.locator.NetworkTopologyStrategy', 'dc1': '3'}"); + cluster.get(1).runOnInstance(() -> { + Epoch epoch = ClusterMetadata.current().epoch; + try + { + String dump = ClusterMetadataService.instance().dumpClusterMetadata(epoch, epoch, NodeVersion.CURRENT_METADATA_VERSION); + ClusterMetadataService.instance().loadClusterMetadata(dump); + } + catch (IOException e) + { + throw new RuntimeException(e); + } + }); + } + } + + // shame, allow us to create a keyspace with a bad replication strategy + public static class BBHelper + { + public static void install(ClassLoader cl, int i) + { + new ByteBuddy().rebase(ReplicationParams.class) + .method(named("validate").and(takesArguments(3))) + .intercept(MethodDelegation.to(BBHelper.class)) + .make() + .load(cl, ClassLoadingStrategy.Default.INJECTION); + new ByteBuddy().rebase(NetworkTopologyStrategy.class) + .method(named("validateExpectedOptions").and(takesArguments(1))) + .intercept(MethodDelegation.to(BBHelper.class)) + .make() + .load(cl, ClassLoadingStrategy.Default.INJECTION); + } + + public static void validate(String name, ClientState state, ClusterMetadata metadata) {} + + public static void validateExpectedOptions(ClusterMetadata metadata) throws ConfigurationException {} + } +} From 417bb21d2eea9081bbdafd11c1ca6769b8ca9acf Mon Sep 17 00:00:00 2001 From: Marcus Eriksson <marcuse@apache.org> Date: Mon, 3 Mar 2025 08:40:21 +0100 Subject: [PATCH 160/225] Avoid adding LEFT nodes to tokenMap on upgrade from gossip Patch by marcuse; reviewed by Sam Tunnicliffe for CASSANDRA-20344 --- CHANGES.txt | 1 + .../cassandra/tcm/ClusterMetadataService.java | 6 +- .../org/apache/cassandra/tcm/Startup.java | 14 +++- .../tcm/compatibility/GossipHelper.java | 19 ++++-- .../cassandra/tcm/membership/Directory.java | 5 +- ...ClusterMetadataUpgradeAssassinateTest.java | 67 +++++++++++++++++++ 6 files changed, 101 insertions(+), 11 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeAssassinateTest.java diff --git a/CHANGES.txt b/CHANGES.txt index d0778def1844..76e6fc487297 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Avoid adding LEFT nodes to tokenMap on upgrade from gossip (CASSANDRA-20344) * Allow empty placements when deserializing cluster metadata (CASSANDRA-20343) * Reduce heap pressure when initializing CMS (CASSANDRA-20267) * Paxos Repair: NoSuchElementException on DistributedSchema.getKeyspaceMetadata (CASSANDRA-20320) diff --git a/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java b/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java index 8adfb81051db..4c6eed11b4a1 100644 --- a/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java +++ b/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java @@ -55,6 +55,7 @@ import org.apache.cassandra.tcm.log.LocalLog; import org.apache.cassandra.tcm.log.LogState; import org.apache.cassandra.tcm.membership.NodeId; +import org.apache.cassandra.tcm.membership.NodeState; import org.apache.cassandra.tcm.membership.NodeVersion; import org.apache.cassandra.tcm.migration.Election; import org.apache.cassandra.tcm.migration.GossipProcessor; @@ -343,6 +344,9 @@ public void upgradeFromGossip(List<String> ignoredEndpoints) continue; } + if (metadata.directory.peerState(entry.getKey()) == NodeState.LEFT) + continue; + if (!version.isUpgraded()) { String msg = String.format("All nodes are not yet upgraded - %s is running %s", metadata.directory.endpoint(entry.getKey()), version); @@ -356,7 +360,7 @@ public void upgradeFromGossip(List<String> ignoredEndpoints) logger.info("First CMS node"); Set<InetAddressAndPort> candidates = metadata .directory - .allAddresses() + .allJoinedEndpoints() .stream() .filter(ep -> !FBUtilities.getBroadcastAddressAndPort().equals(ep) && !ignored.contains(ep)) diff --git a/src/java/org/apache/cassandra/tcm/Startup.java b/src/java/org/apache/cassandra/tcm/Startup.java index 151d24ab90c6..edd8734fca27 100644 --- a/src/java/org/apache/cassandra/tcm/Startup.java +++ b/src/java/org/apache/cassandra/tcm/Startup.java @@ -321,11 +321,21 @@ public static void initializeFromGossip(Function<Processor, Processor> wrapProce } Gossiper.instance.maybeInitializeLocalState(SystemKeyspace.incrementAndGetGeneration()); for (Map.Entry<NodeId, NodeState> entry : initial.directory.states.entrySet()) - Gossiper.instance.mergeNodeToGossip(entry.getKey(), initial); + { + InetAddressAndPort ep = initial.directory.addresses.get(entry.getKey()).broadcastAddress; + if (entry.getValue() != NodeState.LEFT) + Gossiper.instance.mergeNodeToGossip(entry.getKey(), initial); + else + Gossiper.runInGossipStageBlocking(() -> Gossiper.instance.endpointStateMap.put(ep, epStates.get(ep))); + } // double check that everything was added, can remove once we are confident ClusterMetadata cmGossip = fromEndpointStates(emptyFromSystemTables.schema, Gossiper.instance.getEndpointStates()); - assert cmGossip.equals(initial) : cmGossip + " != " + initial; + if (!cmGossip.equals(initial)) + { + cmGossip.dumpDiff(initial); + throw new AssertionError("Issue when populating gossip from cluster metadata"); + } } public static void reinitializeWithClusterMetadata(String fileName, Function<Processor, Processor> wrapProcessor, Runnable initMessaging) throws IOException, StartupException diff --git a/src/java/org/apache/cassandra/tcm/compatibility/GossipHelper.java b/src/java/org/apache/cassandra/tcm/compatibility/GossipHelper.java index 31a20d0bd0d7..0e1acbc4eac3 100644 --- a/src/java/org/apache/cassandra/tcm/compatibility/GossipHelper.java +++ b/src/java/org/apache/cassandra/tcm/compatibility/GossipHelper.java @@ -107,10 +107,10 @@ public static void evictFromMembership(InetAddressAndPort endpoint) } public static VersionedValue nodeStateToStatus(NodeId nodeId, - ClusterMetadata metadata, - Collection<Token> tokens, - VersionedValue.VersionedValueFactory valueFactory, - VersionedValue oldValue) + ClusterMetadata metadata, + Collection<Token> tokens, + VersionedValue.VersionedValueFactory valueFactory, + VersionedValue oldValue) { NodeState nodeState = metadata.directory.peerState(nodeId); if ((tokens == null || tokens.isEmpty()) && !NodeState.isBootstrap(nodeState)) @@ -344,13 +344,18 @@ public static ClusterMetadata fromEndpointStates(Map<InetAddressAndPort, Endpoin NodeAddresses nodeAddresses = getAddressesFromEndpointState(endpoint, epState); NodeVersion nodeVersion = getVersionFromEndpointState(endpoint, epState); assert hostIdString != null; + NodeState nodeState = toNodeState(endpoint, epState); + directory = directory.withNonUpgradedNode(nodeAddresses, new Location(dc, rack), nodeVersion, - toNodeState(endpoint, epState), + nodeState, UUID.fromString(hostIdString)); - NodeId nodeId = directory.peerId(endpoint); - tokenMap = tokenMap.assignTokens(nodeId, getTokensIn(partitioner, epState)); + if (nodeState != NodeState.LEFT) + { + NodeId nodeId = directory.peerId(endpoint); + tokenMap = tokenMap.assignTokens(nodeId, getTokensIn(partitioner, epState)); + } } ClusterMetadata forPlacementCalculation = new ClusterMetadata(Epoch.UPGRADE_GOSSIP, diff --git a/src/java/org/apache/cassandra/tcm/membership/Directory.java b/src/java/org/apache/cassandra/tcm/membership/Directory.java index 87a6bde05320..aab40989d0c7 100644 --- a/src/java/org/apache/cassandra/tcm/membership/Directory.java +++ b/src/java/org/apache/cassandra/tcm/membership/Directory.java @@ -156,7 +156,10 @@ public Directory withNonUpgradedNode(NodeAddresses addresses, UUID hostId) { NodeId id = new NodeId(nextId); - return with(addresses, id, hostId, location, version).withNodeState(id, state).withRackAndDC(id); + Directory updated = with(addresses, id, hostId, location, version).withNodeState(id, state); + if (state != NodeState.LEFT) + updated = updated.withRackAndDC(id); + return updated; } @VisibleForTesting diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeAssassinateTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeAssassinateTest.java new file mode 100644 index 000000000000..9e5be9639a14 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeAssassinateTest.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.upgrade; + +import com.google.common.collect.Streams; +import org.junit.Test; + +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.api.IUpgradeableInstance; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.tcm.ClusterMetadata; + +public class ClusterMetadataUpgradeAssassinateTest extends UpgradeTestBase +{ + @Test + public void simpleUpgradeTest() throws Throwable + { + new TestCase() + .nodes(3) + .nodesToUpgrade(1, 2) + .withConfig((cfg) -> cfg.with(Feature.NETWORK, Feature.GOSSIP)) + .upgradesToCurrentFrom(v50) + .setup((cluster) -> { + cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))"); + cluster.get(3).shutdown().get(); + cluster.get(1).nodetoolResult("assassinate", "127.0.0.3").asserts().success(); + }) + .runAfterClusterUpgrade((cluster) -> { + checkPlacements(cluster.get(1)); + checkPlacements(cluster.get(2)); + cluster.get(1).nodetoolResult("cms", "initialize").asserts().success(); + checkPlacements(cluster.get(1)); + checkPlacements(cluster.get(2)); + }).run(); + } + + private void checkPlacements(IUpgradeableInstance i) + { + ((IInvokableInstance) i).runOnInstance(() -> { + ClusterMetadata metadata = ClusterMetadata.current(); + InetAddressAndPort ep = InetAddressAndPort.getByNameUnchecked("127.0.0.3"); + metadata.placements.asMap().forEach((key, value) -> { + if (Streams.concat(value.reads.endpoints.stream(), + value.writes.endpoints.stream()) + .anyMatch(fr -> fr.endpoints().contains(ep))) + throw new IllegalStateException(ep + " should not be in placements " + metadata.placements); + }); + }); + } +} From 369daf56bca1cf56ceddfc5808b8e91a67d4a7ef Mon Sep 17 00:00:00 2001 From: Marcus Eriksson <marcuse@apache.org> Date: Mon, 3 Mar 2025 08:41:34 +0100 Subject: [PATCH 161/225] Fix PartitionUpdate.isEmpty deserialization issue to avoid potential EOFException Patch by marcuse; reviewed by Sam Tunnicliffe for CASSANDRA-20345 --- CHANGES.txt | 1 + .../db/partitions/PartitionUpdate.java | 2 +- .../distributed/test/PaxosRepairTest.java | 33 +++++++++++++++++++ .../paxos/uncommitted/PaxosRowsTest.java | 5 +-- 4 files changed, 38 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 76e6fc487297..f6cbace4f0b9 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Fix PartitionUpdate.isEmpty deserialization issue to avoid potential EOFException (CASSANDRA-20345) * Avoid adding LEFT nodes to tokenMap on upgrade from gossip (CASSANDRA-20344) * Allow empty placements when deserializing cluster metadata (CASSANDRA-20343) * Reduce heap pressure when initializing CMS (CASSANDRA-20267) diff --git a/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java b/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java index c20a8490a4bd..ff5d0f9035ce 100644 --- a/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java +++ b/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java @@ -800,7 +800,7 @@ public static boolean isEmpty(ByteBuffer in, DeserializationHelper.Flag flag, De if (version >= MessagingService.VERSION_51) { long epoch = VIntCoding.getUnsignedVInt(in, position); - position += VIntCoding.computeVIntSize(epoch); + position += VIntCoding.computeUnsignedVIntSize(epoch); } // DecoratedKey key = metadata.decorateKey(ByteBufferUtil.readWithVIntLength(in)); diff --git a/test/distributed/org/apache/cassandra/distributed/test/PaxosRepairTest.java b/test/distributed/org/apache/cassandra/distributed/test/PaxosRepairTest.java index e22a6be3ea1c..0197106bc631 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/PaxosRepairTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/PaxosRepairTest.java @@ -74,6 +74,7 @@ import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.membership.Directory; import org.apache.cassandra.tcm.membership.NodeVersion; +import org.apache.cassandra.tcm.transformations.CustomTransformation; import org.apache.cassandra.tcm.transformations.ForceSnapshot; import org.apache.cassandra.utils.*; @@ -271,6 +272,38 @@ public void paxosRepairTest() throws Throwable } } + @Test + public void epochBadDeserializationTest() throws Throwable + { + try (Cluster cluster = init(Cluster.build(3).withConfig(WITH_NETWORK).withoutVNodes().start())) + { + cluster.schemaChange("CREATE TABLE " + KEYSPACE + '.' + TABLE + " (pk text primary key, v int)"); + cluster.get(1).runOnInstance(() -> { + // just execute transformations to get epoch bumped enough for the bug to occur + for (int i = 0; i < 75; i++) + ClusterMetadataService.instance().commit(CustomTransformation.make("x"+i)); + }); + // and bump the epoch in the tablemetadata: + cluster.schemaChange("ALTER TABLE " + KEYSPACE + '.' + TABLE + " WITH comment='abc'"); + + cluster.verbs(PAXOS_COMMIT_REQ).drop(); + try + { + cluster.coordinator(1).execute("INSERT INTO " + KEYSPACE + '.' + TABLE + " (pk, v) VALUES ('xyzxyzxyzxyzxyzxyzxyzxyz', 1) IF NOT EXISTS", ConsistencyLevel.QUORUM); + Assert.fail("expected write timeout"); + } + catch (RuntimeException e) + { + // exception expected + } + + cluster.filters().reset(); + cluster.get(1).shutdown().get(); + cluster.get(1).startup(); + } + } + + @Ignore @Test public void topologyChangePaxosTest() throws Throwable diff --git a/test/unit/org/apache/cassandra/service/paxos/uncommitted/PaxosRowsTest.java b/test/unit/org/apache/cassandra/service/paxos/uncommitted/PaxosRowsTest.java index 911ac72d82de..529a2cf0a84c 100644 --- a/test/unit/org/apache/cassandra/service/paxos/uncommitted/PaxosRowsTest.java +++ b/test/unit/org/apache/cassandra/service/paxos/uncommitted/PaxosRowsTest.java @@ -43,6 +43,7 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.paxos.Ballot; import org.apache.cassandra.service.paxos.Commit; +import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.CloseableIterator; import org.apache.cassandra.utils.FBUtilities; @@ -96,7 +97,7 @@ public static void setUpClass() throws Exception SchemaLoader.prepareServer(); ks = "coordinatorsessiontest"; - metadata = CreateTableStatement.parse("CREATE TABLE tbl (k INT PRIMARY KEY, v INT)", ks).build(); + metadata = CreateTableStatement.parse("CREATE TABLE tbl (k INT PRIMARY KEY, v INT)", ks).epoch(Epoch.create(100)).build(); tableId = metadata.id; } @@ -109,7 +110,7 @@ public void setUp() throws Exception @Test public void testRowInterpretation() { - DecoratedKey key = dk(5); + DecoratedKey key = dk(Integer.MAX_VALUE); Ballot[] ballots = createBallots(3); SystemKeyspace.savePaxosWritePromise(key, metadata, ballots[0]); From ca227d82ba1618f9f624a77a6bfa134b0d846784 Mon Sep 17 00:00:00 2001 From: Marcus Eriksson <marcuse@apache.org> Date: Mon, 3 Mar 2025 08:46:51 +0100 Subject: [PATCH 162/225] Unregistering a node should also remove it from tokenMap if it is there and recalculate the placements Patch by marcuse; reviewed by Sam Tunnicliffe for CASSANDRA-20346 --- CHANGES.txt | 1 + .../cassandra/service/StorageService.java | 2 +- .../apache/cassandra/tcm/CMSOperations.java | 2 +- .../apache/cassandra/tcm/ClusterMetadata.java | 4 +- .../cassandra/tcm/membership/Directory.java | 4 + .../tcm/transformations/Register.java | 3 +- .../tcm/transformations/Unregister.java | 19 ++- .../distributed/test/log/RegisterTest.java | 8 +- .../apache/cassandra/tcm/UnregisterTest.java | 148 ++++++++++++++++++ 9 files changed, 180 insertions(+), 11 deletions(-) create mode 100644 test/unit/org/apache/cassandra/tcm/UnregisterTest.java diff --git a/CHANGES.txt b/CHANGES.txt index f6cbace4f0b9..34c11549502b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Unregistering a node should also remove it from tokenMap if it is there and recalculate the placements (CASSANDRA-20346) * Fix PartitionUpdate.isEmpty deserialization issue to avoid potential EOFException (CASSANDRA-20345) * Avoid adding LEFT nodes to tokenMap on upgrade from gossip (CASSANDRA-20344) * Allow empty placements when deserializing cluster metadata (CASSANDRA-20343) diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 252f451d85ea..ab0cb41ee3f4 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -1630,7 +1630,7 @@ public void abortBootstrap(String nodeStr, String endpointStr) throw new RuntimeException("Can't abort bootstrap for " + nodeId + " since it is not bootstrapping"); ClusterMetadataService.instance().commit(new CancelInProgressSequence(nodeId)); } - ClusterMetadataService.instance().commit(new Unregister(nodeId, EnumSet.of(REGISTERED, BOOTSTRAPPING, BOOT_REPLACING))); + ClusterMetadataService.instance().commit(new Unregister(nodeId, EnumSet.of(REGISTERED, BOOTSTRAPPING, BOOT_REPLACING), ClusterMetadataService.instance().placementProvider())); break; default: throw new RuntimeException("Can't abort bootstrap for node " + nodeId + " since the state is " + nodeState); diff --git a/src/java/org/apache/cassandra/tcm/CMSOperations.java b/src/java/org/apache/cassandra/tcm/CMSOperations.java index 87e6d0e37214..f4a608b31e25 100644 --- a/src/java/org/apache/cassandra/tcm/CMSOperations.java +++ b/src/java/org/apache/cassandra/tcm/CMSOperations.java @@ -252,7 +252,7 @@ public void unregisterLeftNodes(List<String> nodeIdStrings) for (NodeId nodeId : nodeIds) { logger.info("Unregistering " + nodeId); - cms.commit(new Unregister(nodeId, EnumSet.of(NodeState.LEFT))); + cms.commit(new Unregister(nodeId, EnumSet.of(NodeState.LEFT), ClusterMetadataService.instance().placementProvider())); } } } diff --git a/src/java/org/apache/cassandra/tcm/ClusterMetadata.java b/src/java/org/apache/cassandra/tcm/ClusterMetadata.java index e2b50483ce92..64eadc76dddb 100644 --- a/src/java/org/apache/cassandra/tcm/ClusterMetadata.java +++ b/src/java/org/apache/cassandra/tcm/ClusterMetadata.java @@ -411,7 +411,9 @@ public Transformer register(NodeAddresses addresses, Location location, NodeVers public Transformer unregister(NodeId nodeId) { - directory = directory.without(nodeId); + directory = directory.withoutRackAndDC(nodeId).without(nodeId); + if (!tokenMap.tokens(nodeId).isEmpty()) + tokenMap = tokenMap.unassignTokens(nodeId); return this; } diff --git a/src/java/org/apache/cassandra/tcm/membership/Directory.java b/src/java/org/apache/cassandra/tcm/membership/Directory.java index aab40989d0c7..90af4ff79f51 100644 --- a/src/java/org/apache/cassandra/tcm/membership/Directory.java +++ b/src/java/org/apache/cassandra/tcm/membership/Directory.java @@ -252,7 +252,11 @@ public Directory withoutRackAndDC(NodeId id) { InetAddressAndPort endpoint = peers.get(id); Location location = locations.get(id); + if (location == null) + return this; BTreeMultimap<String, InetAddressAndPort> rackEP = (BTreeMultimap<String, InetAddressAndPort>) racksByDC.get(location.datacenter); + if (rackEP == null) + return this; rackEP = rackEP.without(location.rack, endpoint); BTreeMap<String, Multimap<String, InetAddressAndPort>> newRacksByDC; if (rackEP.isEmpty()) diff --git a/src/java/org/apache/cassandra/tcm/transformations/Register.java b/src/java/org/apache/cassandra/tcm/transformations/Register.java index 389f62c69801..7cf45fab8b92 100644 --- a/src/java/org/apache/cassandra/tcm/transformations/Register.java +++ b/src/java/org/apache/cassandra/tcm/transformations/Register.java @@ -115,8 +115,7 @@ private static NodeId register(NodeAddresses nodeAddresses, Location location, N if (nodeId == null || metadata.directory.peerState(nodeId) == NodeState.LEFT) { if (nodeId != null) - ClusterMetadataService.instance().commit(new Unregister(nodeId, EnumSet.of(NodeState.LEFT))); - + ClusterMetadataService.instance().commit(new Unregister(nodeId, EnumSet.of(NodeState.LEFT), ClusterMetadataService.instance().placementProvider())); Register registration = new Register(nodeAddresses, location, nodeVersion); nodeId = ClusterMetadataService.instance().commit(registration).directory.peerId(nodeAddresses.broadcastAddress); } diff --git a/src/java/org/apache/cassandra/tcm/transformations/Unregister.java b/src/java/org/apache/cassandra/tcm/transformations/Unregister.java index 6d6fa0499ae9..a4e473272f86 100644 --- a/src/java/org/apache/cassandra/tcm/transformations/Unregister.java +++ b/src/java/org/apache/cassandra/tcm/transformations/Unregister.java @@ -31,6 +31,8 @@ import org.apache.cassandra.tcm.Transformation; import org.apache.cassandra.tcm.membership.NodeId; import org.apache.cassandra.tcm.membership.NodeState; +import org.apache.cassandra.tcm.ownership.DataPlacements; +import org.apache.cassandra.tcm.ownership.PlacementProvider; import org.apache.cassandra.tcm.sequences.LockedRanges; import org.apache.cassandra.tcm.serialization.AsymmetricMetadataSerializer; import org.apache.cassandra.tcm.serialization.Version; @@ -43,11 +45,13 @@ public class Unregister implements Transformation private final NodeId nodeId; private final EnumSet<NodeState> allowedNodeStartStates; + private final PlacementProvider placementProvider; - public Unregister(NodeId nodeId, EnumSet<NodeState> allowedNodeStartStates) + public Unregister(NodeId nodeId, EnumSet<NodeState> allowedNodeStartStates, PlacementProvider placementProvider) { this.nodeId = nodeId; this.allowedNodeStartStates = allowedNodeStartStates; + this.placementProvider = placementProvider; } @Override @@ -67,6 +71,15 @@ public Result execute(ClusterMetadata prev) return new Transformation.Rejected(INVALID, "Can't unregister " + nodeId + " - node state is " + startState + " not " + allowedNodeStartStates); ClusterMetadata.Transformer next = prev.transformer().unregister(nodeId); + if (!prev.tokenMap.tokens(nodeId).isEmpty()) + { + ClusterMetadata nextMetadata = next.build().metadata; + DataPlacements placements = placementProvider.calculatePlacements(nextMetadata.epoch, + nextMetadata.tokenMap.toRanges(), + nextMetadata, + nextMetadata.schema.getKeyspaces()); + next = next.with(placements); + } return Transformation.success(next, LockedRanges.AffectedRanges.EMPTY); } @@ -78,7 +91,7 @@ public Result execute(ClusterMetadata prev) public static void unregister(NodeId nodeId) { ClusterMetadataService.instance() - .commit(new Unregister(nodeId, EnumSet.allOf(NodeState.class))); + .commit(new Unregister(nodeId, EnumSet.allOf(NodeState.class), ClusterMetadataService.instance().placementProvider())); } public String toString() @@ -113,7 +126,7 @@ public Unregister deserialize(DataInputPlus in, Version version) throws IOExcept states.add(NodeState.valueOf(in.readUTF())); } NodeId nodeId = NodeId.serializer.deserialize(in, version); - return new Unregister(nodeId, version.isAtLeast(Version.V2) ? states : EnumSet.allOf(NodeState.class)); + return new Unregister(nodeId, version.isAtLeast(Version.V2) ? states : EnumSet.allOf(NodeState.class), ClusterMetadataService.instance().placementProvider()); } public long serializedSize(Transformation t, Version version) diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/RegisterTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/RegisterTest.java index 1a75b6203e76..786dc7938896 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/RegisterTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/RegisterTest.java @@ -44,6 +44,7 @@ import org.apache.cassandra.tcm.membership.NodeId; import org.apache.cassandra.tcm.membership.NodeState; import org.apache.cassandra.tcm.membership.NodeVersion; +import org.apache.cassandra.tcm.ownership.PlacementProvider; import org.apache.cassandra.tcm.sequences.LeaveStreams; import org.apache.cassandra.tcm.sequences.UnbootstrapAndLeave; import org.apache.cassandra.tcm.serialization.Version; @@ -75,15 +76,16 @@ public void testRegistrationIdempotence() throws Throwable for (int i : new int[]{ 3, 2 }) { cluster.get(i).runOnInstance(() -> { + PlacementProvider pp = ClusterMetadataService.instance().placementProvider(); ClusterMetadataService.instance().commit(new PrepareLeave(ClusterMetadata.current().myNodeId(), true, - ClusterMetadataService.instance().placementProvider(), + pp, LeaveStreams.Kind.UNBOOTSTRAP)); UnbootstrapAndLeave unbootstrapAndLeave = (UnbootstrapAndLeave) ClusterMetadata.current().inProgressSequences.get(ClusterMetadata.current().myNodeId()); ClusterMetadataService.instance().commit(unbootstrapAndLeave.startLeave); ClusterMetadataService.instance().commit(unbootstrapAndLeave.midLeave); ClusterMetadataService.instance().commit(unbootstrapAndLeave.finishLeave); - ClusterMetadataService.instance().commit(new Unregister(ClusterMetadata.current().myNodeId(), EnumSet.of(NodeState.LEFT))); + ClusterMetadataService.instance().commit(new Unregister(ClusterMetadata.current().myNodeId(), EnumSet.of(NodeState.LEFT), pp)); }); cluster.get(1).runOnInstance(() -> { @@ -142,7 +144,7 @@ public void serializationVersionCeilingTest() throws Throwable } // If we unregister oldNode, then the ceiling for serialization version will rise - ClusterMetadataService.instance().commit(new Unregister(oldNode, EnumSet.allOf(NodeState.class))); + ClusterMetadataService.instance().commit(new Unregister(oldNode, EnumSet.allOf(NodeState.class), ClusterMetadataService.instance().placementProvider())); assertEquals(ClusterMetadata.current().directory.clusterMinVersion.serializationVersion, NodeVersion.CURRENT_METADATA_VERSION.asInt()); bytes = t.kind().toVersionedBytes(t); diff --git a/test/unit/org/apache/cassandra/tcm/UnregisterTest.java b/test/unit/org/apache/cassandra/tcm/UnregisterTest.java new file mode 100644 index 000000000000..59403e40807f --- /dev/null +++ b/test/unit/org/apache/cassandra/tcm/UnregisterTest.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.tcm; + +import java.util.EnumSet; +import java.util.function.BiFunction; + +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; +import com.google.common.collect.Streams; +import org.junit.Test; + +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.schema.DistributedMetadataLogKeyspace; +import org.apache.cassandra.schema.DistributedSchema; +import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.schema.Keyspaces; +import org.apache.cassandra.tcm.membership.Directory; +import org.apache.cassandra.tcm.membership.Location; +import org.apache.cassandra.tcm.membership.NodeAddresses; +import org.apache.cassandra.tcm.membership.NodeId; +import org.apache.cassandra.tcm.membership.NodeState; +import org.apache.cassandra.tcm.membership.NodeVersion; +import org.apache.cassandra.tcm.ownership.UniformRangePlacement; +import org.apache.cassandra.tcm.transformations.Assassinate; +import org.apache.cassandra.tcm.transformations.Register; +import org.apache.cassandra.tcm.transformations.Unregister; +import org.apache.cassandra.tcm.transformations.UnsafeJoin; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; + +public class UnregisterTest +{ + private final KeyspaceMetadata ksm = KeyspaceMetadata.create("ks_nts", KeyspaceParams.nts("dc1", 3, "dc2", 3)); + private final Keyspaces kss = Keyspaces.of(DistributedMetadataLogKeyspace.initialMetadata(Sets.newHashSet("dc1", "dc2")), ksm); + private final DistributedSchema initialSchema = new DistributedSchema(kss); + + @Test + public void testBasicUnregister() + { + unregisterHelper((toUnregister, metadata) -> { + metadata = assassinate(toUnregister, metadata); + metadata = unregister(toUnregister, metadata); + return metadata; + }); + } + + @Test + public void badStateUnregister() + { + unregisterHelper((toUnregister, metadata) -> { + metadata = left(toUnregister, metadata); + metadata = unregister(toUnregister, metadata); + return metadata; + }); + } + + private void unregisterHelper(BiFunction<Integer, ClusterMetadata, ClusterMetadata> f) + { + ClusterMetadata metadata = new ClusterMetadata(Murmur3Partitioner.instance, Directory.EMPTY, initialSchema); + + int toUnregister = 5; + for (int i = 0; i < 10; i++) + { + // node we're unregistering is only node in dc2 - make sure the dc is gone after unregistration + String dc = "dc"+(i == toUnregister ? "2" :"1"); + metadata = unsafejoin(i, register(i, dc, metadata)); + } + Token t = token(toUnregister, metadata); + InetAddressAndPort ep = ep(toUnregister); + NodeId nodeId = metadata.directory.peerId(ep(toUnregister)); + metadata = f.apply(toUnregister, metadata); + assertNoTrace(ep, nodeId, t, metadata); + } + + private ClusterMetadata left(int i, ClusterMetadata metadata) + { + NodeId nodeId = metadata.directory.peerId(ep(i)); + ClusterMetadata.Transformer t = metadata.transformer().withNodeState(nodeId, NodeState.LEFT); + return t.build().metadata; + } + + private static void assertNoTrace(InetAddressAndPort ep, NodeId nodeId, Token t, ClusterMetadata metadata) + { + assertNull(metadata.tokenMap.owner(t)); + assertFalse(metadata.directory.states.containsKey(nodeId)); + assertFalse(metadata.directory.peerIds().contains(nodeId)); + assertFalse(metadata.directory.allAddresses().contains(ep)); + assertFalse(metadata.directory.allJoinedEndpoints().contains(ep)); + assertFalse(metadata.directory.allDatacenterRacks().containsKey("dc2")); + assertFalse(metadata.directory.knownDatacenters().contains("dc2")); + metadata.placements.asMap().forEach((params, placement) -> { + assertFalse(Streams.concat(placement.writes.endpoints.stream(), placement.reads.endpoints.stream()).anyMatch((fr) -> fr.endpoints().contains(ep))); + }); + } + + private Token token(int i, ClusterMetadata metadata) + { + NodeId nodeId = metadata.directory.peerId(ep(i)); + return metadata.tokenMap.tokens(nodeId).iterator().next(); + } + + private ClusterMetadata assassinate(int i, ClusterMetadata metadata) + { + return new Assassinate(metadata.directory.peerId(ep(i)), new UniformRangePlacement()).execute(metadata).success().metadata; + } + + private ClusterMetadata unregister(int i, ClusterMetadata metadata ) + { + return new Unregister(metadata.directory.peerId(ep(i)), EnumSet.of(NodeState.LEFT), new UniformRangePlacement()).execute(metadata).success().metadata; + } + + private ClusterMetadata register(int i, String dc, ClusterMetadata metadata) + { + return new Register(new NodeAddresses(ep(i)), new Location(dc, "rack1"), NodeVersion.CURRENT).execute(metadata).success().metadata; + } + + private ClusterMetadata unsafejoin(int i, ClusterMetadata metadata) + { + NodeId nodeId = metadata.directory.peerId(ep(i)); + return new UnsafeJoin(nodeId, ImmutableSet.of(Murmur3Partitioner.instance.getRandomToken()), new UniformRangePlacement()).execute(metadata).success().metadata; + } + + private InetAddressAndPort ep(int i) + { + return InetAddressAndPort.getByNameUnchecked("127.0.0."+i); + } +} From 37fe4b679c28234de3ee2ac0694118d5be570300 Mon Sep 17 00:00:00 2001 From: maulin-vasavada <maulin.vasavada@gmail.com> Date: Mon, 17 Feb 2025 20:06:11 -0800 Subject: [PATCH 163/225] Provide keystore_password_file and truststore_password_file options to read credentials from a file patch by Maulin Vasavada; reviewed by Stefan Miklosovic, Maxwell Guo for CASSANDRA-13428 --- CHANGES.txt | 1 + conf/cassandra.yaml | 29 ++ conf/cassandra_latest.yaml | 29 ++ .../pages/managing/operating/security.adoc | 52 ++- .../KubernetesSecretsSslContextFactory.java | 12 +- .../cassandra/config/EncryptionOptions.java | 305 ++++++++++++------ .../cassandra/config/JMXServerOptions.java | 6 + .../security/FileBasedSslContextFactory.java | 60 +++- ...andra-jmx-sslconfig-with-passwordfile.yaml | 74 +++++ ...m-sslcontextfactory-with-passwordfile.yaml | 152 +++++++++ ...ssandra_ssl_test_keystore_passwordfile.txt | 1 + ...ra_ssl_test_outbound_keystore_password.txt | 1 + ...andra_ssl_test_truststore_passwordfile.txt | 1 + .../config/EncryptionOptionsEqualityTest.java | 48 +++ .../config/EncryptionOptionsTest.java | 13 +- .../FileBasedSslContextFactoryTest.java | 45 ++- .../security/FileBasedStoreContextTest.java | 80 +++++ ...textFactoryConfigWithPasswordFileTest.java | 83 +++++ .../cassandra/transport/TlsTestUtils.java | 3 + ...ithYamlFileOptionsAndPasswordFileTest.java | 88 +++++ 20 files changed, 960 insertions(+), 123 deletions(-) create mode 100644 test/conf/cassandra-jmx-sslconfig-with-passwordfile.yaml create mode 100644 test/conf/cassandra-pem-sslcontextfactory-with-passwordfile.yaml create mode 100644 test/conf/cassandra_ssl_test_keystore_passwordfile.txt create mode 100644 test/conf/cassandra_ssl_test_outbound_keystore_password.txt create mode 100644 test/conf/cassandra_ssl_test_truststore_passwordfile.txt create mode 100644 test/unit/org/apache/cassandra/security/FileBasedStoreContextTest.java create mode 100644 test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryConfigWithPasswordFileTest.java create mode 100644 test/unit/org/apache/cassandra/utils/jmx/JMXSslConfiguredWithYamlFileOptionsAndPasswordFileTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 34c11549502b..9b84f503c45f 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Provide keystore_password_file and truststore_password_file options to read credentials from a file (CASSANDRA-13428) * Unregistering a node should also remove it from tokenMap if it is there and recalculate the placements (CASSANDRA-20346) * Fix PartitionUpdate.isEmpty deserialization issue to avoid potential EOFException (CASSANDRA-20345) * Avoid adding LEFT nodes to tokenMap on upgrade from gossip (CASSANDRA-20344) diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index 9f8f24b58445..6eb150f5dfac 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -1723,6 +1723,10 @@ server_encryption_options: # Set to a valid keystore if internode_encryption is dc, rack or all keystore: conf/.keystore #keystore_password: cassandra + # Optional configuration to specify password for keystore in a separate file + # When keystore_password and keystore_password_file both are specified, the keystore_password will take precedence + # The password in the file should be on the first line + #keystore_password_file: conf/keystore_passwordfile.txt # Configure the way Cassandra creates SSL contexts. # To use PEM-based key material, see org.apache.cassandra.security.PEMBasedSslContextFactory # ssl_context_factory: @@ -1734,11 +1738,20 @@ server_encryption_options: # to create SSLContext. By default, outbound_keystore is the same as keystore indicating mTLS is not enabled. # outbound_keystore: conf/.keystore # outbound_keystore_password: cassandra + # Optional configuration to specify password for keystore in a separate file + # When outbound_keystore_password and outbound_keystore_password_file both are specified, + # the outbound_keystore_password will take precedence + # The password in the file should be on the first line + #outbound_keystore_password_file: conf/outbound_keystore_passwordfile.txt # Verify peer server certificates require_client_auth: false # Set to a valid trustore if require_client_auth is true truststore: conf/.truststore #truststore_password: cassandra + # Optional configuration to specify password for truststore in a separate file + # When truststore_password and truststore_password_file both are specified, the truststore_password will take precedence + # The password in the file should be on the first line + #truststore_password_file: conf/truststore_passwordfile.txt # Verify that the host name in the certificate matches the connected host require_endpoint_verification: false # More advanced defaults: @@ -1780,6 +1793,10 @@ client_encryption_options: # Set keystore and keystore_password to valid keystores if enabled is true keystore: conf/.keystore #keystore_password: cassandra + # Optional configuration to specify password for keystore in a separate file + # When keystore_password and keystore_password_file both are specified, the keystore_password will take precedence + # The password in the file should be on the first line + #keystore_password_file: conf/keystore_passwordfile.txt # Configure the way Cassandra creates SSL contexts. # To use PEM-based key material, see org.apache.cassandra.security.PEMBasedSslContextFactory # ssl_context_factory: @@ -1794,6 +1811,10 @@ client_encryption_options: # Set trustore and truststore_password if require_client_auth is true # truststore: conf/.truststore # truststore_password: cassandra + # Optional configuration to specify password for truststore in a separate file + # When truststore_password and truststore_password_file both are specified, the truststore_password will take precedence + # The password in the file should be on the first line + #truststore_password_file: conf/truststore_passwordfile.txt # More advanced defaults: # protocol: TLS # store_type: JKS @@ -1844,8 +1865,16 @@ client_encryption_options: # accepted_protocols: [TLSv1.2,TLSv1.3,TLSv1.1] # keystore: conf/cassandra_ssl.keystore # keystore_password: cassandra + # Optional configuration to specify password for keystore in a separate file + # When keystore_password and keystore_password_file both are specified, the keystore_password will take precedence + # The password in the file should be on the first line + # keystore_password_file: conf/keystore_passwordfile.txt # truststore: conf/cassandra_ssl.truststore # truststore_password: cassandra + # Optional configuration to specify password for truststore in a separate file + # When truststore_password and truststore_password_file both are specified, the truststore_password will take precedence + # The password in the file should be on the first line + # truststore_password_file: conf/truststore_passwordfile.txt # # jmx authentication and authorization options. # authenticate: false diff --git a/conf/cassandra_latest.yaml b/conf/cassandra_latest.yaml index f8d791b26f64..9c86beeea829 100644 --- a/conf/cassandra_latest.yaml +++ b/conf/cassandra_latest.yaml @@ -1613,6 +1613,10 @@ server_encryption_options: # Set to a valid keystore if internode_encryption is dc, rack or all keystore: conf/.keystore #keystore_password: cassandra + # Optional configuration to specify password for keystore in a separate file + # When keystore_password and keystore_password_file both are specified, the keystore_password will take precedence + # The password in the file should be on the first line + #keystore_password_file: conf/keystore_passwordfile.txt # Configure the way Cassandra creates SSL contexts. # To use PEM-based key material, see org.apache.cassandra.security.PEMBasedSslContextFactory # ssl_context_factory: @@ -1624,11 +1628,20 @@ server_encryption_options: # to create SSLContext. By default, outbound_keystore is the same as keystore indicating mTLS is not enabled. # outbound_keystore: conf/.keystore # outbound_keystore_password: cassandra + # Optional configuration to specify password for keystore in a separate file + # When outbound_keystore_password and outbound_keystore_password_file both are specified, + # the outbound_keystore_password will take precedence + # The password in the file should be on the first line + #outbound_keystore_password_file: conf/outbound_keystore_passwordfile.txt # Verify peer server certificates require_client_auth: false # Set to a valid trustore if require_client_auth is true truststore: conf/.truststore #truststore_password: cassandra + # Optional configuration to specify password for truststore in a separate file + # When truststore_password and truststore_password_file both are specified, the truststore_password will take precedence + # The password in the file should be on the first line + #truststore_password_file: conf/truststore_passwordfile.txt # Verify that the host name in the certificate matches the connected host require_endpoint_verification: false # More advanced defaults: @@ -1663,6 +1676,10 @@ client_encryption_options: # Set keystore and keystore_password to valid keystores if enabled is true keystore: conf/.keystore #keystore_password: cassandra + # Optional configuration to specify password for keystore in a separate file + # When keystore_password and keystore_password_file both are specified, the keystore_password will take precedence + # The password in the file should be on the first line + #keystore_password_file: conf/keystore_passwordfile.txt # Configure the way Cassandra creates SSL contexts. # To use PEM-based key material, see org.apache.cassandra.security.PEMBasedSslContextFactory # ssl_context_factory: @@ -1677,6 +1694,10 @@ client_encryption_options: # Set trustore and truststore_password if require_client_auth is true # truststore: conf/.truststore # truststore_password: cassandra + # Optional configuration to specify password for truststore in a separate file + # When truststore_password and truststore_password_file both are specified, the truststore_password will take precedence + # The password in the file should be on the first line + #truststore_password_file: conf/truststore_passwordfile.txt # More advanced defaults: # protocol: TLS # store_type: JKS @@ -1718,8 +1739,16 @@ client_encryption_options: # accepted_protocols: [TLSv1.2,TLSv1.3,TLSv1.1] # keystore: conf/cassandra_ssl.keystore # keystore_password: cassandra + # Optional configuration to specify password for keystore in a separate file + # When keystore_password and keystore_password_file both are specified, the keystore_password will take precedence + # The password in the file should be on the first line + # keystore_password_file: conf/keystore_passwordfile.txt # truststore: conf/cassandra_ssl.truststore # truststore_password: cassandra + # Optional configuration to specify password for truststore in a separate file + # When truststore_password and truststore_password_file both are specified, the truststore_password will take precedence + # The password in the file should be on the first line + # truststore_password_file: conf/truststore_passwordfile.txt # # jmx authentication and authorization options. # authenticate: false diff --git a/doc/modules/cassandra/pages/managing/operating/security.adoc b/doc/modules/cassandra/pages/managing/operating/security.adoc index e7748e93fb61..cdc76a625dbb 100644 --- a/doc/modules/cassandra/pages/managing/operating/security.adoc +++ b/doc/modules/cassandra/pages/managing/operating/security.adoc @@ -100,6 +100,28 @@ YAML!) -----END CERTIFICATE----- ---- +* Configuration: PEM private key's password specified via a file + +[source,yaml] +---- + client/server_encryption_options: + ssl_context_factory: + class_name: org.apache.cassandra.security.PEMBasedSslContextFactory + parameters: + private_key: | + -----BEGIN ENCRYPTED PRIVATE KEY----- OR -----BEGIN PRIVATE KEY----- + <your base64 encoded private key> + -----END ENCRYPTED PRIVATE KEY----- OR -----END PRIVATE KEY----- + -----BEGIN CERTIFICATE----- + <your base64 encoded certificate chain> + -----END CERTIFICATE----- + trusted_certificates: | + -----BEGIN CERTIFICATE----- + <your base64 encoded certificate> + -----END CERTIFICATE----- + keystore_password_file: "<file having your password for the encrypted private key>" +---- + * Configuration: PEM keys/certs defined in files [source,yaml] @@ -112,6 +134,18 @@ YAML!) truststore: <file path to the truststore file in the PEM format> ---- +* Configuration: PEM private key's password specified via a file + +[source,yaml] +---- + client/server_encryption_options: + ssl_context_factory: + class_name: org.apache.cassandra.security.PEMBasedSslContextFactory + keystore: <file path to the keystore file in the PEM format with the private key and the certificate chain> + keystore_password_file: "<file having your password for the encrypted private key>" + truststore: <file path to the truststore file in the PEM format> +---- + == SSL Certificate Hot Reloading Beginning with Cassandra 4, Cassandra supports hot reloading of SSL @@ -590,11 +624,27 @@ jmx_encryption_options: truststore_password: cassandra ---- +Below is an example of configuring JMX SSL in `cassandra.yaml` with password files for keystore and truststore. +[source,yaml] +---- +jmx_encryption_options: + enabled: true + cipher_suites: [TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256] + accepted_protocols: [TLSv1.2,TLSv1.3,TLSv1.1] + keystore: test/conf/cassandra_ssl_test.keystore + keystore_password_file: test/conf/keystore_passwordfile.txt + truststore: test/conf/cassandra_ssl_test.truststore + truststore_password: cassandra + truststore_password_file: test/conf/truststore_passwordfile.txt +---- + Similar to `client/server_encryption_options`, you can specify PEM-based key material or customize the SSL configuration using `ssl_context_factory` in `jmx_encryption_options`. -Below is an example of configuring PEM based key material, +Below is an example of configuring PEM based key material. You can use `keystore_password_file` +configuration with in-line PEM as documented priorly in case you have stored the password for the +keystore in a file. [source,yaml] ---- jmx_encryption_options: diff --git a/examples/ssl-factory/src/org/apache/cassandra/security/KubernetesSecretsSslContextFactory.java b/examples/ssl-factory/src/org/apache/cassandra/security/KubernetesSecretsSslContextFactory.java index ebeac6c4e832..12622d4458e2 100644 --- a/examples/ssl-factory/src/org/apache/cassandra/security/KubernetesSecretsSslContextFactory.java +++ b/examples/ssl-factory/src/org/apache/cassandra/security/KubernetesSecretsSslContextFactory.java @@ -150,10 +150,12 @@ public interface ConfigKeys { public KubernetesSecretsSslContextFactory() { keystoreContext = new FileBasedStoreContext(getString(EncryptionOptions.ConfigKey.KEYSTORE.toString(), KEYSTORE_PATH_VALUE), - getValueFromEnv(KEYSTORE_PASSWORD_ENV_VAR_NAME, DEFAULT_KEYSTORE_PASSWORD)); + getValueFromEnv(KEYSTORE_PASSWORD_ENV_VAR_NAME, DEFAULT_KEYSTORE_PASSWORD), + null); trustStoreContext = new FileBasedStoreContext(getString(EncryptionOptions.ConfigKey.TRUSTSTORE.toString(), TRUSTSTORE_PATH_VALUE), - getValueFromEnv(TRUSTSTORE_PASSWORD_ENV_VAR_NAME, DEFAULT_TRUSTSTORE_PASSWORD)); + getValueFromEnv(TRUSTSTORE_PASSWORD_ENV_VAR_NAME, DEFAULT_TRUSTSTORE_PASSWORD), + null); keystoreLastUpdatedTime = System.nanoTime(); keystoreUpdatedTimeSecretKeyPath = getString(ConfigKeys.KEYSTORE_UPDATED_TIMESTAMP_PATH, @@ -168,11 +170,13 @@ public KubernetesSecretsSslContextFactory(Map<String, Object> parameters) super(parameters); keystoreContext = new FileBasedStoreContext(getString(EncryptionOptions.ConfigKey.KEYSTORE.toString(), KEYSTORE_PATH_VALUE), getValueFromEnv(getString(ConfigKeys.KEYSTORE_PASSWORD_ENV_VAR, - KEYSTORE_PASSWORD_ENV_VAR_NAME), DEFAULT_KEYSTORE_PASSWORD)); + KEYSTORE_PASSWORD_ENV_VAR_NAME), DEFAULT_KEYSTORE_PASSWORD), + null); trustStoreContext = new FileBasedStoreContext(getString(EncryptionOptions.ConfigKey.TRUSTSTORE.toString(), TRUSTSTORE_PATH_VALUE), getValueFromEnv(getString(ConfigKeys.TRUSTSTORE_PASSWORD_ENV_VAR, - TRUSTSTORE_PASSWORD_ENV_VAR_NAME), DEFAULT_TRUSTSTORE_PASSWORD)); + TRUSTSTORE_PASSWORD_ENV_VAR_NAME), DEFAULT_TRUSTSTORE_PASSWORD), + null); keystoreLastUpdatedTime = System.nanoTime(); keystoreUpdatedTimeSecretKeyPath = getString(ConfigKeys.KEYSTORE_UPDATED_TIMESTAMP_PATH, KEYSTORE_UPDATED_TIMESTAMP_PATH_VALUE); diff --git a/src/java/org/apache/cassandra/config/EncryptionOptions.java b/src/java/org/apache/cassandra/config/EncryptionOptions.java index 1cd3b7adddaa..07f78b9a5eea 100644 --- a/src/java/org/apache/cassandra/config/EncryptionOptions.java +++ b/src/java/org/apache/cassandra/config/EncryptionOptions.java @@ -114,9 +114,13 @@ public String value() public final String keystore; @Nullable public final String keystore_password; + @Nullable + public final String keystore_password_file; public final String truststore; @Nullable public final String truststore_password; + @Nullable + public final String truststore_password_file; public final List<String> cipher_suites; protected String protocol; protected List<String> accepted_protocols; @@ -153,10 +157,13 @@ public enum ConfigKey { KEYSTORE("keystore"), KEYSTORE_PASSWORD("keystore_password"), + KEYSTORE_PASSWORD_FILE("keystore_password_file"), OUTBOUND_KEYSTORE("outbound_keystore"), OUTBOUND_KEYSTORE_PASSWORD("outbound_keystore_password"), + OUTBOUND_KEYSTORE_PASSWORD_FILE("outbound_keystore_password_file"), TRUSTSTORE("truststore"), TRUSTSTORE_PASSWORD("truststore_password"), + TRUSTSTORE_PASSWORD_FILE("truststore_password_file"), CIPHER_SUITES("cipher_suites"), PROTOCOL("protocol"), ACCEPTED_PROTOCOLS("accepted_protocols"), @@ -199,8 +206,10 @@ public EncryptionOptions() new HashMap<>()); keystore = "conf/.keystore"; keystore_password = null; + keystore_password_file = null; truststore = "conf/.truststore"; truststore_password = null; + truststore_password_file = null; cipher_suites = null; protocol = null; accepted_protocols = null; @@ -214,18 +223,22 @@ public EncryptionOptions() certificate_validity_warn_threshold = null; } - public EncryptionOptions(ParameterizedClass ssl_context_factory, String keystore, String keystore_password, - String truststore, String truststore_password, List<String> cipher_suites, - String protocol, List<String> accepted_protocols, String algorithm, String store_type, - String require_client_auth, boolean require_endpoint_verification, Boolean enabled, - Boolean optional, DurationSpec.IntMinutesBound max_certificate_validity_period, + public EncryptionOptions(ParameterizedClass ssl_context_factory, + String keystore, String keystore_password, String keystore_password_file, + String truststore, String truststore_password, String truststore_password_file, + List<String> cipher_suites, String protocol, List<String> accepted_protocols, + String algorithm, String store_type, String require_client_auth, + boolean require_endpoint_verification, Boolean enabled, Boolean optional, + DurationSpec.IntMinutesBound max_certificate_validity_period, DurationSpec.IntMinutesBound certificate_validity_warn_threshold) { this.ssl_context_factory = ssl_context_factory; this.keystore = keystore; this.keystore_password = keystore_password; + this.keystore_password_file = keystore_password_file; this.truststore = truststore; this.truststore_password = truststore_password; + this.truststore_password_file = truststore_password_file; this.cipher_suites = cipher_suites; this.protocol = protocol; this.accepted_protocols = accepted_protocols; @@ -244,8 +257,10 @@ public EncryptionOptions(EncryptionOptions options) ssl_context_factory = options.ssl_context_factory; keystore = options.keystore; keystore_password = options.keystore_password; + keystore_password_file = options.keystore_password_file; truststore = options.truststore; truststore_password = options.truststore_password; + truststore_password_file = options.truststore_password_file; cipher_suites = options.cipher_suites; protocol = options.protocol; accepted_protocols = options.accepted_protocols; @@ -324,8 +339,10 @@ protected void fillSslContextParams(Map<String, Object> sslContextFactoryParamet */ putSslContextFactoryParameter(sslContextFactoryParameters, ConfigKey.KEYSTORE, this.keystore); putSslContextFactoryParameter(sslContextFactoryParameters, ConfigKey.KEYSTORE_PASSWORD, this.keystore_password); + putSslContextFactoryParameter(sslContextFactoryParameters, ConfigKey.KEYSTORE_PASSWORD_FILE, this.keystore_password_file); putSslContextFactoryParameter(sslContextFactoryParameters, ConfigKey.TRUSTSTORE, this.truststore); putSslContextFactoryParameter(sslContextFactoryParameters, ConfigKey.TRUSTSTORE_PASSWORD, this.truststore_password); + putSslContextFactoryParameter(sslContextFactoryParameters, ConfigKey.TRUSTSTORE_PASSWORD_FILE, this.truststore_password_file); putSslContextFactoryParameter(sslContextFactoryParameters, ConfigKey.CIPHER_SUITES, this.cipher_suites); putSslContextFactoryParameter(sslContextFactoryParameters, ConfigKey.PROTOCOL, this.protocol); putSslContextFactoryParameter(sslContextFactoryParameters, ConfigKey.ACCEPTED_PROTOCOLS, this.accepted_protocols); @@ -502,56 +519,72 @@ else if (getEnabled()) public EncryptionOptions withSslContextFactory(ParameterizedClass sslContextFactoryClass) { - return new EncryptionOptions(sslContextFactoryClass, keystore, keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, algorithm, + return new EncryptionOptions(sslContextFactoryClass, keystore, keystore_password, keystore_password_file, truststore, + truststore_password, truststore_password_file, cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, enabled, optional, max_certificate_validity_period, max_certificate_validity_period).applyConfig(); } public EncryptionOptions withKeyStore(String keystore) { - return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, algorithm, + return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, truststore, + truststore_password, truststore_password_file, cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, enabled, optional, max_certificate_validity_period, max_certificate_validity_period).applyConfig(); } public EncryptionOptions withKeyStorePassword(String keystore_password) { - return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, algorithm, + return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, truststore, + truststore_password, truststore_password_file, cipher_suites, protocol, accepted_protocols, algorithm, + store_type, require_client_auth, require_endpoint_verification, enabled, + optional, max_certificate_validity_period, max_certificate_validity_period).applyConfig(); + } + + public EncryptionOptions withKeyStorePasswordFile(String keystore_password_file) + { + return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, truststore, + truststore_password, truststore_password_file, cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, enabled, optional, max_certificate_validity_period, max_certificate_validity_period).applyConfig(); } public EncryptionOptions withTrustStore(String truststore) { - return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, algorithm, + return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, truststore, + truststore_password, truststore_password_file, cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, enabled, optional, max_certificate_validity_period, max_certificate_validity_period).applyConfig(); } public EncryptionOptions withTrustStorePassword(String truststore_password) { - return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, algorithm, + return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, truststore, + truststore_password, truststore_password_file, cipher_suites, protocol, accepted_protocols, algorithm, + store_type, require_client_auth, require_endpoint_verification, enabled, + optional, max_certificate_validity_period, max_certificate_validity_period).applyConfig(); + } + + public EncryptionOptions withTrustStorePasswordFile(String truststore_password_file) + { + return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, truststore, + truststore_password, truststore_password_file, cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, enabled, optional, max_certificate_validity_period, max_certificate_validity_period).applyConfig(); } public EncryptionOptions withCipherSuites(List<String> cipher_suites) { - return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, algorithm, + return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, truststore, + truststore_password, truststore_password_file, cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, enabled, optional, max_certificate_validity_period, max_certificate_validity_period).applyConfig(); } public EncryptionOptions withCipherSuites(String... cipher_suites) { - return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, truststore, - truststore_password, ImmutableList.copyOf(cipher_suites), protocol, + return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, truststore, + truststore_password, truststore_password_file, ImmutableList.copyOf(cipher_suites), protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, enabled, optional, max_certificate_validity_period, max_certificate_validity_period).applyConfig(); @@ -559,8 +592,8 @@ public EncryptionOptions withCipherSuites(String... cipher_suites) public EncryptionOptions withProtocol(String protocol) { - return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, algorithm, + return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, truststore, + truststore_password, truststore_password_file, cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, enabled, optional, max_certificate_validity_period, max_certificate_validity_period).applyConfig(); } @@ -568,8 +601,8 @@ public EncryptionOptions withProtocol(String protocol) public EncryptionOptions withAcceptedProtocols(List<String> accepted_protocols) { - return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols == null ? null : + return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, truststore, + truststore_password, truststore_password_file, cipher_suites, protocol, accepted_protocols == null ? null : ImmutableList.copyOf(accepted_protocols), algorithm, store_type, require_client_auth, require_endpoint_verification, enabled, optional, max_certificate_validity_period, max_certificate_validity_period).applyConfig(); @@ -578,64 +611,64 @@ public EncryptionOptions withAcceptedProtocols(List<String> accepted_protocols) public EncryptionOptions withAlgorithm(String algorithm) { - return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, algorithm, + return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, truststore, + truststore_password, truststore_password_file, cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, enabled, optional, max_certificate_validity_period, max_certificate_validity_period).applyConfig(); } public EncryptionOptions withStoreType(String store_type) { - return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, algorithm, + return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, truststore, + truststore_password, truststore_password_file, cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, enabled, optional, max_certificate_validity_period, max_certificate_validity_period).applyConfig(); } public EncryptionOptions withRequireClientAuth(ClientAuth require_client_auth) { - return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, algorithm, + return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, truststore, + truststore_password, truststore_password_file, cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth.value, require_endpoint_verification, enabled, optional, max_certificate_validity_period, max_certificate_validity_period).applyConfig(); } public EncryptionOptions withRequireEndpointVerification(boolean require_endpoint_verification) { - return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, algorithm, + return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, truststore, + truststore_password, truststore_password_file, cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, enabled, optional, max_certificate_validity_period, max_certificate_validity_period).applyConfig(); } public EncryptionOptions withEnabled(boolean enabled) { - return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, algorithm, + return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, truststore, + truststore_password, truststore_password_file, cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, enabled, optional, max_certificate_validity_period, max_certificate_validity_period).applyConfig(); } public EncryptionOptions withOptional(Boolean optional) { - return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, algorithm, + return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, truststore, + truststore_password, truststore_password_file, cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, enabled, optional, max_certificate_validity_period, max_certificate_validity_period).applyConfig(); } public EncryptionOptions withMaxCertificateValidityPeriod(DurationSpec.IntMinutesBound maxCertificateValidityPeriod) { - return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, algorithm, + return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, truststore, + truststore_password, truststore_password_file, cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, enabled, optional, maxCertificateValidityPeriod, certificate_validity_warn_threshold).applyConfig(); } public EncryptionOptions withCertificateValidityWarnThreshold(DurationSpec.IntMinutesBound certificateValidityWarnThreshold) { - return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, algorithm, + return new EncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, truststore, + truststore_password, truststore_password_file, cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, enabled, optional, max_certificate_validity_period, certificateValidityWarnThreshold).applyConfig(); } @@ -659,8 +692,10 @@ public boolean equals(Object o) require_endpoint_verification == opt.require_endpoint_verification && Objects.equals(keystore, opt.keystore) && Objects.equals(keystore_password, opt.keystore_password) && + Objects.equals(keystore_password_file, opt.keystore_password_file) && Objects.equals(truststore, opt.truststore) && Objects.equals(truststore_password, opt.truststore_password) && + Objects.equals(truststore_password_file, opt.truststore_password_file) && Objects.equals(protocol, opt.protocol) && Objects.equals(accepted_protocols, opt.accepted_protocols) && Objects.equals(algorithm, opt.algorithm) && @@ -679,8 +714,10 @@ public int hashCode() int result = 0; result += 31 * (keystore == null ? 0 : keystore.hashCode()); result += 31 * (keystore_password == null ? 0 : keystore_password.hashCode()); + result += 31 * (keystore_password_file == null ? 0 : keystore_password_file.hashCode()); result += 31 * (truststore == null ? 0 : truststore.hashCode()); result += 31 * (truststore_password == null ? 0 : truststore_password.hashCode()); + result += 31 * (truststore_password_file == null ? 0 : truststore_password_file.hashCode()); result += 31 * (protocol == null ? 0 : protocol.hashCode()); result += 31 * (accepted_protocols == null ? 0 : accepted_protocols.hashCode()); result += 31 * (algorithm == null ? 0 : algorithm.hashCode()); @@ -707,6 +744,8 @@ public enum InternodeEncryption public final String outbound_keystore; @Nullable public final String outbound_keystore_password; + @Nullable + public final String outbound_keystore_password_file; public ServerEncryptionOptions() { @@ -714,11 +753,13 @@ public ServerEncryptionOptions() this.legacy_ssl_storage_port_enabled = false; this.outbound_keystore = null; this.outbound_keystore_password = null; + this.outbound_keystore_password_file = null; } public ServerEncryptionOptions(ParameterizedClass sslContextFactoryClass, String keystore, - String keystore_password,String outbound_keystore, - String outbound_keystore_password, String truststore, String truststore_password, + String keystore_password, String keystore_password_file, String outbound_keystore, + String outbound_keystore_password, String outbound_keystore_password_file, + String truststore, String truststore_password, String truststore_password_file, List<String> cipher_suites, String protocol, List<String> accepted_protocols, String algorithm, String store_type, String require_client_auth, boolean require_endpoint_verification, Boolean optional, @@ -726,13 +767,15 @@ public ServerEncryptionOptions(ParameterizedClass sslContextFactoryClass, String DurationSpec.IntMinutesBound maxCertificateAgeMinutes, DurationSpec.IntMinutesBound certificateValidityWarnThreshold) { - super(sslContextFactoryClass, keystore, keystore_password, truststore, truststore_password, cipher_suites, + super(sslContextFactoryClass, keystore, keystore_password, keystore_password_file, + truststore, truststore_password, truststore_password_file, cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, null, optional, maxCertificateAgeMinutes, certificateValidityWarnThreshold); this.internode_encryption = internode_encryption; this.legacy_ssl_storage_port_enabled = legacy_ssl_storage_port_enabled; this.outbound_keystore = outbound_keystore; this.outbound_keystore_password = outbound_keystore_password; + this.outbound_keystore_password_file = outbound_keystore_password_file; } public ServerEncryptionOptions(ServerEncryptionOptions options) @@ -742,6 +785,7 @@ public ServerEncryptionOptions(ServerEncryptionOptions options) this.legacy_ssl_storage_port_enabled = options.legacy_ssl_storage_port_enabled; this.outbound_keystore = options.outbound_keystore; this.outbound_keystore_password = options.outbound_keystore_password; + this.outbound_keystore_password_file = options.outbound_keystore_password_file; } @Override @@ -750,6 +794,7 @@ protected void fillSslContextParams(Map<String, Object> sslContextFactoryParamet super.fillSslContextParams(sslContextFactoryParameters); putSslContextFactoryParameter(sslContextFactoryParameters, ConfigKey.OUTBOUND_KEYSTORE, this.outbound_keystore); putSslContextFactoryParameter(sslContextFactoryParameters, ConfigKey.OUTBOUND_KEYSTORE_PASSWORD, this.outbound_keystore_password); + putSslContextFactoryParameter(sslContextFactoryParameters, ConfigKey.OUTBOUND_KEYSTORE_PASSWORD_FILE, this.outbound_keystore_password_file); } @Override @@ -845,7 +890,8 @@ public boolean equals(Object o) return internode_encryption == opt.internode_encryption && legacy_ssl_storage_port_enabled == opt.legacy_ssl_storage_port_enabled && Objects.equals(outbound_keystore, opt.outbound_keystore) && - Objects.equals(outbound_keystore_password, opt.outbound_keystore_password); + Objects.equals(outbound_keystore_password, opt.outbound_keystore_password) && + Objects.equals(outbound_keystore_password_file, opt.outbound_keystore_password_file); } /** @@ -861,15 +907,17 @@ public int hashCode() result += 31 * Boolean.hashCode(legacy_ssl_storage_port_enabled); result += 31 * (outbound_keystore == null ? 0 : outbound_keystore.hashCode()); result += 31 * (outbound_keystore_password == null ? 0 : outbound_keystore_password.hashCode()); + result += 31 * (outbound_keystore_password_file == null ? 0 : outbound_keystore_password_file.hashCode()); return result; } @Override public ServerEncryptionOptions withSslContextFactory(ParameterizedClass sslContextFactoryClass) { - return new ServerEncryptionOptions(sslContextFactoryClass, keystore, keystore_password, - outbound_keystore, outbound_keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, + return new ServerEncryptionOptions(sslContextFactoryClass, keystore, keystore_password, keystore_password_file, + outbound_keystore, outbound_keystore_password, outbound_keystore_password_file, + truststore, truststore_password, truststore_password_file, + cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, optional, internode_encryption, legacy_ssl_storage_port_enabled, max_certificate_validity_period, @@ -879,9 +927,10 @@ public ServerEncryptionOptions withSslContextFactory(ParameterizedClass sslConte @Override public ServerEncryptionOptions withKeyStore(String keystore) { - return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, - outbound_keystore, outbound_keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, + return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, + outbound_keystore, outbound_keystore_password, outbound_keystore_password_file, + truststore, truststore_password, truststore_password_file, + cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, optional, internode_encryption, legacy_ssl_storage_port_enabled, max_certificate_validity_period, @@ -891,9 +940,23 @@ public ServerEncryptionOptions withKeyStore(String keystore) @Override public ServerEncryptionOptions withKeyStorePassword(String keystore_password) { - return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, - outbound_keystore, outbound_keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, + return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, + outbound_keystore, outbound_keystore_password, outbound_keystore_password_file, + truststore, truststore_password, truststore_password_file, + cipher_suites, protocol, accepted_protocols, + algorithm, store_type, require_client_auth, + require_endpoint_verification, optional, internode_encryption, + legacy_ssl_storage_port_enabled, max_certificate_validity_period, + max_certificate_validity_period).applyConfigInternal(); + } + + @Override + public ServerEncryptionOptions withKeyStorePasswordFile(String keystore_password_file) + { + return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, + outbound_keystore, outbound_keystore_password, outbound_keystore_password_file, + truststore, truststore_password, truststore_password_file, + cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, optional, internode_encryption, legacy_ssl_storage_port_enabled, max_certificate_validity_period, @@ -903,9 +966,10 @@ public ServerEncryptionOptions withKeyStorePassword(String keystore_password) @Override public ServerEncryptionOptions withTrustStore(String truststore) { - return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, - outbound_keystore, outbound_keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, + return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, + outbound_keystore, outbound_keystore_password, outbound_keystore_password_file, + truststore, truststore_password, truststore_password_file, + cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, optional, internode_encryption, legacy_ssl_storage_port_enabled, max_certificate_validity_period, @@ -915,9 +979,23 @@ public ServerEncryptionOptions withTrustStore(String truststore) @Override public ServerEncryptionOptions withTrustStorePassword(String truststore_password) { - return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, - outbound_keystore, outbound_keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, + return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, + outbound_keystore, outbound_keystore_password, outbound_keystore_password_file, + truststore, truststore_password, truststore_password_file, + cipher_suites, protocol, accepted_protocols, + algorithm, store_type, require_client_auth, + require_endpoint_verification, optional, internode_encryption, + legacy_ssl_storage_port_enabled, max_certificate_validity_period, + max_certificate_validity_period).applyConfigInternal(); + } + + @Override + public ServerEncryptionOptions withTrustStorePasswordFile(String truststore_password_file) + { + return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, + outbound_keystore, outbound_keystore_password, outbound_keystore_password_file, + truststore, truststore_password, truststore_password_file, + cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, optional, internode_encryption, legacy_ssl_storage_port_enabled, max_certificate_validity_period, @@ -927,9 +1005,10 @@ public ServerEncryptionOptions withTrustStorePassword(String truststore_password @Override public ServerEncryptionOptions withCipherSuites(List<String> cipher_suites) { - return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, - outbound_keystore, outbound_keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, + return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, + outbound_keystore, outbound_keystore_password, outbound_keystore_password_file, + truststore, truststore_password, truststore_password_file, + cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, optional, internode_encryption, legacy_ssl_storage_port_enabled, max_certificate_validity_period, @@ -939,10 +1018,11 @@ public ServerEncryptionOptions withCipherSuites(List<String> cipher_suites) @Override public ServerEncryptionOptions withCipherSuites(String... cipher_suites) { - return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, - outbound_keystore, outbound_keystore_password, truststore, - truststore_password, Arrays.asList(cipher_suites), protocol, - accepted_protocols, algorithm, store_type, require_client_auth, + return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, + outbound_keystore, outbound_keystore_password, outbound_keystore_password_file, + truststore, truststore_password, truststore_password_file, + Arrays.asList(cipher_suites), protocol, accepted_protocols, + algorithm, store_type, require_client_auth, require_endpoint_verification, optional, internode_encryption, legacy_ssl_storage_port_enabled, max_certificate_validity_period, max_certificate_validity_period).applyConfigInternal(); @@ -951,9 +1031,10 @@ public ServerEncryptionOptions withCipherSuites(String... cipher_suites) @Override public ServerEncryptionOptions withProtocol(String protocol) { - return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, - outbound_keystore, outbound_keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, + return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, + outbound_keystore, outbound_keystore_password, outbound_keystore_password_file, + truststore, truststore_password, truststore_password_file, + cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, optional, internode_encryption, legacy_ssl_storage_port_enabled, max_certificate_validity_period, @@ -963,9 +1044,10 @@ public ServerEncryptionOptions withProtocol(String protocol) @Override public ServerEncryptionOptions withAcceptedProtocols(List<String> accepted_protocols) { - return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, - outbound_keystore, outbound_keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, + return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, + outbound_keystore, outbound_keystore_password, outbound_keystore_password_file, + truststore, truststore_password, truststore_password_file, + cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, optional, internode_encryption, legacy_ssl_storage_port_enabled, max_certificate_validity_period, @@ -975,9 +1057,10 @@ public ServerEncryptionOptions withAcceptedProtocols(List<String> accepted_proto @Override public ServerEncryptionOptions withAlgorithm(String algorithm) { - return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, - outbound_keystore, outbound_keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, + return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, + outbound_keystore, outbound_keystore_password, outbound_keystore_password_file, + truststore, truststore_password, truststore_password_file, + cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, optional, internode_encryption, legacy_ssl_storage_port_enabled, max_certificate_validity_period, @@ -987,9 +1070,10 @@ public ServerEncryptionOptions withAlgorithm(String algorithm) @Override public ServerEncryptionOptions withStoreType(String store_type) { - return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, - outbound_keystore, outbound_keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, + return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, + outbound_keystore, outbound_keystore_password, outbound_keystore_password_file, + truststore, truststore_password, truststore_password_file, + cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, optional, internode_encryption, legacy_ssl_storage_port_enabled, max_certificate_validity_period, @@ -999,9 +1083,10 @@ public ServerEncryptionOptions withStoreType(String store_type) @Override public ServerEncryptionOptions withRequireClientAuth(ClientAuth require_client_auth) { - return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, - outbound_keystore, outbound_keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, + return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, + outbound_keystore, outbound_keystore_password, outbound_keystore_password_file, + truststore, truststore_password, truststore_password_file, + cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth.value, require_endpoint_verification, optional, internode_encryption, legacy_ssl_storage_port_enabled, max_certificate_validity_period, @@ -1011,9 +1096,10 @@ public ServerEncryptionOptions withRequireClientAuth(ClientAuth require_client_a @Override public ServerEncryptionOptions withRequireEndpointVerification(boolean require_endpoint_verification) { - return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, - outbound_keystore, outbound_keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, + return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, + outbound_keystore, outbound_keystore_password, outbound_keystore_password_file, + truststore, truststore_password, truststore_password_file, + cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, optional, internode_encryption, legacy_ssl_storage_port_enabled, max_certificate_validity_period, @@ -1022,9 +1108,10 @@ public ServerEncryptionOptions withRequireEndpointVerification(boolean require_e public ServerEncryptionOptions withOptional(boolean optional) { - return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, - outbound_keystore, outbound_keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, + return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, + outbound_keystore, outbound_keystore_password, outbound_keystore_password_file, + truststore, truststore_password, truststore_password_file, + cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, optional, internode_encryption, legacy_ssl_storage_port_enabled, max_certificate_validity_period, @@ -1033,9 +1120,10 @@ public ServerEncryptionOptions withOptional(boolean optional) public ServerEncryptionOptions withInternodeEncryption(InternodeEncryption internode_encryption) { - return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, - outbound_keystore, outbound_keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, + return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, + outbound_keystore, outbound_keystore_password, outbound_keystore_password_file, + truststore, truststore_password, truststore_password_file, + cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, optional, internode_encryption, legacy_ssl_storage_port_enabled, max_certificate_validity_period, @@ -1044,9 +1132,10 @@ public ServerEncryptionOptions withInternodeEncryption(InternodeEncryption inter public ServerEncryptionOptions withLegacySslStoragePort(boolean enable_legacy_ssl_storage_port) { - return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, - outbound_keystore, outbound_keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, + return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, + outbound_keystore, outbound_keystore_password, outbound_keystore_password_file, + truststore, truststore_password, truststore_password_file, + cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, optional, internode_encryption, enable_legacy_ssl_storage_port, max_certificate_validity_period, @@ -1055,9 +1144,10 @@ public ServerEncryptionOptions withLegacySslStoragePort(boolean enable_legacy_ss public ServerEncryptionOptions withOutboundKeystore(String outboundKeystore) { - return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, - outboundKeystore, outbound_keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, + return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, + outboundKeystore, outbound_keystore_password, outbound_keystore_password_file, + truststore, truststore_password, truststore_password_file, + cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, optional, internode_encryption, legacy_ssl_storage_port_enabled, max_certificate_validity_period, @@ -1066,21 +1156,34 @@ public ServerEncryptionOptions withOutboundKeystore(String outboundKeystore) public ServerEncryptionOptions withOutboundKeystorePassword(String outboundKeystorePassword) { - return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, - outbound_keystore, outboundKeystorePassword, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, + return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, + outbound_keystore, outboundKeystorePassword, outbound_keystore_password_file, + truststore, truststore_password, truststore_password_file, + cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, optional, internode_encryption, legacy_ssl_storage_port_enabled, max_certificate_validity_period, max_certificate_validity_period).applyConfigInternal(); } + public ServerEncryptionOptions withOutboundKeystorePasswordFile(String outboundKeystorePasswordFile) + { + return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, + outbound_keystore, outbound_keystore_password, outboundKeystorePasswordFile, + truststore, truststore_password, truststore_password_file, + cipher_suites, protocol, accepted_protocols, + algorithm, store_type, require_client_auth, + require_endpoint_verification, optional, internode_encryption, + legacy_ssl_storage_port_enabled, max_certificate_validity_period, + max_certificate_validity_period).applyConfigInternal(); + } @Override public ServerEncryptionOptions withMaxCertificateValidityPeriod(DurationSpec.IntMinutesBound maxCertificateValidityPeriod) { - return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, - outbound_keystore, outbound_keystore_password, truststore, - truststore_password, cipher_suites, protocol, accepted_protocols, + return new ServerEncryptionOptions(ssl_context_factory, keystore, keystore_password, keystore_password_file, + outbound_keystore, outbound_keystore_password, outbound_keystore_password_file, + truststore, truststore_password, truststore_password_file, + cipher_suites, protocol, accepted_protocols, algorithm, store_type, require_client_auth, require_endpoint_verification, optional, internode_encryption, legacy_ssl_storage_port_enabled, maxCertificateValidityPeriod, diff --git a/src/java/org/apache/cassandra/config/JMXServerOptions.java b/src/java/org/apache/cassandra/config/JMXServerOptions.java index 8b51d3b75688..705ab02bb898 100644 --- a/src/java/org/apache/cassandra/config/JMXServerOptions.java +++ b/src/java/org/apache/cassandra/config/JMXServerOptions.java @@ -194,11 +194,17 @@ public static JMXServerOptions createParsingSystemProperties() boolean sslEnabled = COM_SUN_MANAGEMENT_JMXREMOTE_SSL.getBoolean(); + // CASSANDRA-13428: Support for specifying password file for keystores is only added to the `encryption_options` + // in the `cassandra.yaml`. Since the JMX SSL Config can also leverage it as per CASSANDRA-18508, password file + // support is not added to the JMX SSL configuration via the system properties. Hence, `null` is used as + // the password file arguments for the keystore and the truststore while constructing the encryption options here. EncryptionOptions encryptionOptions = new EncryptionOptions(new ParameterizedClass("org.apache.cassandra.security.DefaultSslContextFactory", new HashMap<>()), keystore, keystorePassword, + null, truststore, truststorePassword, + null, cipherSuites, null, // protocol acceptedProtocols, diff --git a/src/java/org/apache/cassandra/security/FileBasedSslContextFactory.java b/src/java/org/apache/cassandra/security/FileBasedSslContextFactory.java index 39e584d25763..e0867493d7b7 100644 --- a/src/java/org/apache/cassandra/security/FileBasedSslContextFactory.java +++ b/src/java/org/apache/cassandra/security/FileBasedSslContextFactory.java @@ -36,9 +36,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.io.util.File; +import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.utils.Clock; +import static java.lang.String.format; + /** * Abstract implementation for {@link ISslContextFactory} using file based, standard keystore format with the ability * to hot-reload the files upon file changes (detected by the {@code last modified timestamp}). @@ -60,18 +64,19 @@ public abstract class FileBasedSslContextFactory extends AbstractSslContextFacto public FileBasedSslContextFactory() { - keystoreContext = new FileBasedStoreContext("conf/.keystore", "cassandra"); - outboundKeystoreContext = new FileBasedStoreContext("conf/.keystore", "cassandra"); - trustStoreContext = new FileBasedStoreContext("conf/.truststore", "cassandra"); + keystoreContext = new FileBasedStoreContext("conf/.keystore", "cassandra", null); + outboundKeystoreContext = new FileBasedStoreContext("conf/.keystore", "cassandra", null); + trustStoreContext = new FileBasedStoreContext("conf/.truststore", "cassandra", null); } public FileBasedSslContextFactory(Map<String, Object> parameters) { super(parameters); - keystoreContext = new FileBasedStoreContext(getString("keystore"), getString("keystore_password")); + keystoreContext = new FileBasedStoreContext(getString("keystore"), getString("keystore_password"), getString("keystore_password_file")); outboundKeystoreContext = new FileBasedStoreContext(StringUtils.defaultString(getString("outbound_keystore"), keystoreContext.filePath), - StringUtils.defaultString(getString("outbound_keystore_password"), keystoreContext.password)); - trustStoreContext = new FileBasedStoreContext(getString("truststore"), getString("truststore_password")); + StringUtils.defaultString(getString("outbound_keystore_password"), keystoreContext.password), + StringUtils.defaultString(getString("outbound_keystore_password_file"), keystoreContext.passwordFilePath)); + trustStoreContext = new FileBasedStoreContext(getString("truststore"), getString("truststore_password"), getString("truststore_password_file")); } @Override @@ -135,7 +140,7 @@ protected void validatePassword(boolean isOutboundKeystore, String password) if (password == null) { String keyName = isOutboundKeystore ? "outbound_" : ""; - final String msg = String.format("'%skeystore_password' must be specified", keyName); + final String msg = format("'%skeystore_password' must be specified", keyName); throw new IllegalArgumentException(msg); } } @@ -278,11 +283,13 @@ protected static class FileBasedStoreContext public volatile boolean checkedExpiry = false; public String filePath; public String password; + public String passwordFilePath; - public FileBasedStoreContext(String keystore, String keystorePassword) + public FileBasedStoreContext(String keystoreFilePath, String keystorePassword, String keystorePasswordFilePath) { - this.filePath = keystore; - this.password = keystorePassword; + this.filePath = keystoreFilePath; + this.passwordFilePath = keystorePasswordFilePath; + this.password = resolvePassword(keystoreFilePath, keystorePassword, keystorePasswordFilePath); } protected boolean hasKeystore() @@ -294,5 +301,38 @@ protected boolean passwordMatchesIfPresent(String keyPassword) { return StringUtils.isEmpty(password) || keyPassword.equals(password); } + + private static String resolvePassword(String keystoreFilePath, String password, String passwordFilePath) + { + if (password != null) + return password; + + if (StringUtils.isEmpty(passwordFilePath)) + return null; + + File keystorePasswordFile = new File(passwordFilePath); + + if (!keystorePasswordFile.exists()) + { + final String msg = format("keystore password file %s does not exist", keystorePasswordFile.path()); + throw new ConfigurationException(msg); + } + + try + { + // we expect a password to be on the first line + List<String> lines = FileUtils.readLines(keystorePasswordFile); + if (lines.isEmpty()) + return ""; + + return lines.get(0); + } + catch (RuntimeException e) + { + throw new ConfigurationException(format("'Failed to read keystore password from the %s for %s", + keystorePasswordFile, keystoreFilePath), + e); + } + } } } diff --git a/test/conf/cassandra-jmx-sslconfig-with-passwordfile.yaml b/test/conf/cassandra-jmx-sslconfig-with-passwordfile.yaml new file mode 100644 index 000000000000..0b495485d128 --- /dev/null +++ b/test/conf/cassandra-jmx-sslconfig-with-passwordfile.yaml @@ -0,0 +1,74 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# +# Testing for pluggable ssl_context_factory option for client and server encryption options with a valid and a missing +# implementation classes. +# +cluster_name: Test Cluster +# memtable_allocation_type: heap_buffers +memtable_allocation_type: offheap_objects +commitlog_sync: batch +commitlog_segment_size: 5MiB +commitlog_directory: build/test/cassandra/commitlog +# commitlog_compression: +# - class_name: LZ4Compressor +cdc_raw_directory: build/test/cassandra/cdc_raw +cdc_enabled: false +hints_directory: build/test/cassandra/hints +partitioner: org.apache.cassandra.dht.ByteOrderedPartitioner +listen_address: 127.0.0.1 +storage_port: 7012 +ssl_storage_port: 17012 +start_native_transport: true +native_transport_port: 9042 +column_index_size: 4KiB +saved_caches_directory: build/test/cassandra/saved_caches +data_file_directories: + - build/test/cassandra/data +disk_access_mode: mmap_index_only +seed_provider: + - class_name: org.apache.cassandra.locator.SimpleSeedProvider + parameters: + - seeds: "127.0.0.1:7012" +endpoint_snitch: org.apache.cassandra.locator.SimpleSnitch +dynamic_snitch: true +jmx_server_options: + enabled: true + jmx_encryption_options: + enabled: true + cipher_suites: [TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256] + accepted_protocols: [TLSv1.2,TLSv1.3,TLSv1.1] + keystore: test/conf/cassandra_ssl_test.keystore + keystore_password_file: test/conf/cassandra_ssl_test_keystore_passwordfile.txt + truststore: test/conf/cassandra_ssl_test.truststore + truststore_password_file: test/conf/cassandra_ssl_test_truststore_passwordfile.txt +incremental_backups: true +concurrent_compactors: 4 +compaction_throughput: 0MiB/s +row_cache_class_name: org.apache.cassandra.cache.OHCProvider +row_cache_size: 16MiB +user_defined_functions_enabled: true +scripted_user_defined_functions_enabled: false +prepared_statements_cache_size: 1MiB +corrupted_tombstone_strategy: exception +stream_entire_sstables: true +stream_throughput_outbound: 24MiB/s +sasi_indexes_enabled: true +materialized_views_enabled: true +file_cache_enabled: true diff --git a/test/conf/cassandra-pem-sslcontextfactory-with-passwordfile.yaml b/test/conf/cassandra-pem-sslcontextfactory-with-passwordfile.yaml new file mode 100644 index 000000000000..19ef0f3cd756 --- /dev/null +++ b/test/conf/cassandra-pem-sslcontextfactory-with-passwordfile.yaml @@ -0,0 +1,152 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# +# Testing for pluggable ssl_context_factory option for client and server encryption options with a valid and a missing +# implementation classes. +# +cluster_name: Test Cluster +# memtable_allocation_type: heap_buffers +memtable_allocation_type: offheap_objects +commitlog_sync: periodic +commitlog_sync_period: 10s +commitlog_segment_size: 5MiB +commitlog_directory: build/test/cassandra/commitlog +# commitlog_compression: +# - class_name: LZ4Compressor +cdc_raw_directory: build/test/cassandra/cdc_raw +cdc_enabled: false +hints_directory: build/test/cassandra/hints +partitioner: org.apache.cassandra.dht.ByteOrderedPartitioner +listen_address: 127.0.0.1 +storage_port: 7012 +ssl_storage_port: 17012 +start_native_transport: true +native_transport_port: 9042 +column_index_size: 4KiB +saved_caches_directory: build/test/cassandra/saved_caches +data_file_directories: + - build/test/cassandra/data +disk_access_mode: mmap_index_only +seed_provider: + - class_name: org.apache.cassandra.locator.SimpleSeedProvider + parameters: + - seeds: "127.0.0.1:7012" +endpoint_snitch: org.apache.cassandra.locator.SimpleSnitch +dynamic_snitch: true +client_encryption_options: + ssl_context_factory: + class_name: org.apache.cassandra.security.PEMBasedSslContextFactory + parameters: + private_key: | + -----BEGIN ENCRYPTED PRIVATE KEY----- + MIIE6jAcBgoqhkiG9w0BDAEDMA4ECOWqSzq5PBIdAgIFxQSCBMjXsCK30J0aT3J/ + g5kcbmevTOY1pIhJGbf5QYYrMUPiuDK2ydxIbiPzoTE4/S+OkCeHhlqwn/YydpBl + xgjZZ1Z5rLJHO27d2biuESqanDiBVXYuVmHmaifRnFy0uUTFkStB5mjVZEiJgO29 + L83hL60uWru71EVuVriC2WCfmZ/EXp6wyYszOqCFQ8Quk/rDO6XuaBl467MJbx5V + sucGT6E9XKNd9hB14/Izb2jtVM5kqKxoiHpz1na6yhEYJiE5D1uOonznWjBnjwB/ + f0x+acpDfVDoJKTlRdz+DEcbOF7mb9lBVVjP6P/AAsmQzz6JKwHjvCrjYfQmyyN8 + RI4KRQnWgm4L3dtByLqY8HFU4ogisCMCgI+hZQ+OKMz/hoRO540YGiPcTRY3EOUR + 0bd5JxU6tCJDMTqKP9aSL2KmLoiLowdMkSPz7TCzLsZ2bGJemuCfpAs4XT1vXCHs + evrUbOnh8et1IA8mZ9auThfqsZtNagJLEXA6hWIKp1FfVL3Q49wvMKZt4eTn/zwU + tLL0m5yPo6/HAaOA3hbm/oghZS0dseshXl7PZrmZQtvYnIvjyoxEL7ducYDQCDP6 + wZ7Nzyh1QZAauSS15hl3vLFRZCA9hWAVgwQAviTvhB342O0i9qI7TQkcHk+qcTPN + K+iGNbFZ8ma1izXNKSJ2PgI/QqFNIeJWvZrb9PhJRmaZVsTJ9fERm1ewpebZqkVv + zMqMhlKgx9ggAaSKgnGZkwXwB6GrSbbzUrwRCKm3FieD1QE4VVYevaadVUU75GG5 + mrFKorJEH7kFZlic8OTjDksYnHbcgU36XZrGEXa2+ldVeGKL3CsXWciaQRcJg8yo + WQDjZpcutGI0eMJWCqUkv8pYZC2/wZU4htCve5nVJUU4t9uuo9ex7lnwlLWPvheQ + jUBMgzSRsZ+zwaIusvufAAxiKK/cJm4ubZSZPIjBbfd4U7VPxtirP4Accydu7EK6 + eG/MZwtAMFNJxfxUR+/aYzJU/q1ePw7fWVHrpt58t/22CX2SJBEiUGmSmuyER4Ny + DPw6d6mhvPUS1jRhIZ9A81ht8MOX7VL5uVp307rt7o5vRpV1mo0iPiRHzGscMpJn + AP36klEAUNTf0uLTKZa7KHiwhn5iPmsCrENHkOKJjxhRrqHjD2wy3YHs3ow2voyY + Ua4Cids+c1hvRkNEDGNHm4+rKGFOGOsG/ZU7uj/6gflO4JXxNGiyTLflqMdWBvow + Zd7hk1zCaGAAn8nZ0hPweGxQ4Q30I9IBZrimGxB0vjiUqNio9+qMf33dCHFJEuut + ZGJMaUGVaPhXQcTy4uD5hzsPZV5xcsU4H3vBYyBcZgrusJ6OOgkuZQaU7p8rWQWr + bUEVbXuZdwEmxsCe7H/vEVv5+aA4sF4kWnMMFL7/LIYaiEzkTqdJlRv/KyJJgcAH + hg2BvR3XTAq8wiX0C98CdmTbsx2eyQdj5tCU606rEohFLKUxWkJYAKxCiUbxGGpI + RheVmxkef9ErxJiq7hsAsGrSJvMtJuDKIasnD14SOEwD/7jRAq6WdL9VLpxtzlOw + pWnIl8kUCO3WoaG9Jf+ZTIv2hnxJhaSzYrdXzGPNnaWKhBlwnXJRvQEdrIxZOimP + FujZhqbKUDbYAcqTkoQ= + -----END ENCRYPTED PRIVATE KEY----- + -----BEGIN CERTIFICATE----- + MIIDkTCCAnmgAwIBAgIETxH5JDANBgkqhkiG9w0BAQsFADB5MRAwDgYDVQQGEwdV + bmtub3duMRAwDgYDVQQIEwdVbmtub3duMRAwDgYDVQQHEwdVbmtub3duMRAwDgYD + VQQKEwdVbmtub3duMRQwEgYDVQQLDAtzc2xfdGVzdGluZzEZMBcGA1UEAxMQQXBh + Y2hlIENhc3NhbmRyYTAeFw0xNjAzMTgyMTI4MDJaFw0xNjA2MTYyMTI4MDJaMHkx + EDAOBgNVBAYTB1Vua25vd24xEDAOBgNVBAgTB1Vua25vd24xEDAOBgNVBAcTB1Vu + a25vd24xEDAOBgNVBAoTB1Vua25vd24xFDASBgNVBAsMC3NzbF90ZXN0aW5nMRkw + FwYDVQQDExBBcGFjaGUgQ2Fzc2FuZHJhMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8A + MIIBCgKCAQEAjkmVX/HS49cS8Hn6o26IGwMIcEV3d7ZhH0GNcx8rnSRd10dU9F6d + ugSjbwGFMcWUQzYNejN6az0Wb8JIQyXRPTWjfgaWTyVGr0bGTnxg6vwhzfI/9jzy + q59xv29OuSY1dxmY31f0pZ9OOw3mabWksjoO2TexfKoxqsRHJ8PrM1f8E84Z4xo2 + TJXGzpuIxRkAJ+sVDqKEAhrKAfRYMSgdJ7zRt8VXv9ngjX20uA2m092NcH0Kmeto + TmuWUtK8E/qcN7ULN8xRWNUn4hu6mG6mayk4XliGRqI1VZupqh+MgNqHznuTd0bA + YrQsFPw9HaZ2hvVnJffJ5l7njAekZNOL+wIDAQABoyEwHzAdBgNVHQ4EFgQUcdiD + N6aylI91kAd34Hl2AzWY51QwDQYJKoZIhvcNAQELBQADggEBAG9q29ilUgCWQP5v + iHkZHj10gXGEoMkdfrPBf8grC7dpUcaw1Qfku/DJ7kPvMALeEsmFDk/t78roeNbh + IYBLJlzI1HZN6VPtpWQGsqxltAy5XN9Xw9mQM/tu70ShgsodGmE1UoW6eE5+/GMv + 6Fg+zLuICPvs2cFNmWUvukN5LW146tJSYCv0Q/rCPB3m9dNQ9pBxrzPUHXw4glwG + qGnGddXmOC+tSW5lDLLG1BRbKv4zxv3UlrtIjqlJtZb/sQMT6WtG2ihAz7SKOBHa + HOWUwuPTetWIuJCKP7P4mWWtmSmjLy+BFX5seNEngn3RzJ2L8uuTJQ/88OsqgGru + n3MVF9w= + -----END CERTIFICATE----- + trusted_certificates: | + -----BEGIN CERTIFICATE----- + MIIDkTCCAnmgAwIBAgIETxH5JDANBgkqhkiG9w0BAQsFADB5MRAwDgYDVQQGEwdV + bmtub3duMRAwDgYDVQQIEwdVbmtub3duMRAwDgYDVQQHEwdVbmtub3duMRAwDgYD + VQQKEwdVbmtub3duMRQwEgYDVQQLDAtzc2xfdGVzdGluZzEZMBcGA1UEAxMQQXBh + Y2hlIENhc3NhbmRyYTAeFw0xNjAzMTgyMTI4MDJaFw0xNjA2MTYyMTI4MDJaMHkx + EDAOBgNVBAYTB1Vua25vd24xEDAOBgNVBAgTB1Vua25vd24xEDAOBgNVBAcTB1Vu + a25vd24xEDAOBgNVBAoTB1Vua25vd24xFDASBgNVBAsMC3NzbF90ZXN0aW5nMRkw + FwYDVQQDExBBcGFjaGUgQ2Fzc2FuZHJhMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8A + MIIBCgKCAQEAjkmVX/HS49cS8Hn6o26IGwMIcEV3d7ZhH0GNcx8rnSRd10dU9F6d + ugSjbwGFMcWUQzYNejN6az0Wb8JIQyXRPTWjfgaWTyVGr0bGTnxg6vwhzfI/9jzy + q59xv29OuSY1dxmY31f0pZ9OOw3mabWksjoO2TexfKoxqsRHJ8PrM1f8E84Z4xo2 + TJXGzpuIxRkAJ+sVDqKEAhrKAfRYMSgdJ7zRt8VXv9ngjX20uA2m092NcH0Kmeto + TmuWUtK8E/qcN7ULN8xRWNUn4hu6mG6mayk4XliGRqI1VZupqh+MgNqHznuTd0bA + YrQsFPw9HaZ2hvVnJffJ5l7njAekZNOL+wIDAQABoyEwHzAdBgNVHQ4EFgQUcdiD + N6aylI91kAd34Hl2AzWY51QwDQYJKoZIhvcNAQELBQADggEBAG9q29ilUgCWQP5v + iHkZHj10gXGEoMkdfrPBf8grC7dpUcaw1Qfku/DJ7kPvMALeEsmFDk/t78roeNbh + IYBLJlzI1HZN6VPtpWQGsqxltAy5XN9Xw9mQM/tu70ShgsodGmE1UoW6eE5+/GMv + 6Fg+zLuICPvs2cFNmWUvukN5LW146tJSYCv0Q/rCPB3m9dNQ9pBxrzPUHXw4glwG + qGnGddXmOC+tSW5lDLLG1BRbKv4zxv3UlrtIjqlJtZb/sQMT6WtG2ihAz7SKOBHa + HOWUwuPTetWIuJCKP7P4mWWtmSmjLy+BFX5seNEngn3RzJ2L8uuTJQ/88OsqgGru + n3MVF9w= + -----END CERTIFICATE----- + keystore_password_file: test/conf/cassandra_ssl_test_keystore_passwordfile.txt +server_encryption_options: + ssl_context_factory: + class_name: org.apache.cassandra.security.PEMBasedSslContextFactory + internode_encryption: none + keystore: test/conf/cassandra_ssl_test.keystore.pem + keystore_password_file: test/conf/cassandra_ssl_test_keystore_passwordfile.txt + truststore: test/conf/cassandra_ssl_test.truststore.pem +incremental_backups: true +concurrent_compactors: 4 +compaction_throughput: 0MiB/s +row_cache_class_name: org.apache.cassandra.cache.OHCProvider +row_cache_size: 16MiB +user_defined_functions_enabled: true +scripted_user_defined_functions_enabled: false +prepared_statements_cache_size: 1MiB +corrupted_tombstone_strategy: exception +stream_entire_sstables: true +stream_throughput_outbound: 24MiB/s +sasi_indexes_enabled: true +materialized_views_enabled: true +file_cache_enabled: true diff --git a/test/conf/cassandra_ssl_test_keystore_passwordfile.txt b/test/conf/cassandra_ssl_test_keystore_passwordfile.txt new file mode 100644 index 000000000000..03ef327b7ac6 --- /dev/null +++ b/test/conf/cassandra_ssl_test_keystore_passwordfile.txt @@ -0,0 +1 @@ +cassandra \ No newline at end of file diff --git a/test/conf/cassandra_ssl_test_outbound_keystore_password.txt b/test/conf/cassandra_ssl_test_outbound_keystore_password.txt new file mode 100644 index 000000000000..03ef327b7ac6 --- /dev/null +++ b/test/conf/cassandra_ssl_test_outbound_keystore_password.txt @@ -0,0 +1 @@ +cassandra \ No newline at end of file diff --git a/test/conf/cassandra_ssl_test_truststore_passwordfile.txt b/test/conf/cassandra_ssl_test_truststore_passwordfile.txt new file mode 100644 index 000000000000..03ef327b7ac6 --- /dev/null +++ b/test/conf/cassandra_ssl_test_truststore_passwordfile.txt @@ -0,0 +1 @@ +cassandra \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/config/EncryptionOptionsEqualityTest.java b/test/unit/org/apache/cassandra/config/EncryptionOptionsEqualityTest.java index 317c8ec6260b..5e6d26b8173e 100644 --- a/test/unit/org/apache/cassandra/config/EncryptionOptionsEqualityTest.java +++ b/test/unit/org/apache/cassandra/config/EncryptionOptionsEqualityTest.java @@ -81,6 +81,54 @@ public void testKeystoreOptions() { assertEquals(encryptionOptions1.hashCode(), encryptionOptions2.hashCode()); } + @Test + public void testKeystoreOptionsWithPasswordFile() { + EncryptionOptions encryptionOptions1 = + new EncryptionOptions() + .withStoreType("JKS") + .withKeyStore(TlsTestUtils.SERVER_KEYSTORE_PATH) + .withKeyStorePasswordFile(TlsTestUtils.SERVER_KEYSTORE_PASSWORD_FILE) + .withTrustStore(TlsTestUtils.SERVER_TRUSTSTORE_PATH) + .withTrustStorePasswordFile(TlsTestUtils.SERVER_TRUSTSTORE_PASSWORD_FILE) + .withProtocol("TLSv1.1") + .withRequireClientAuth(REQUIRED) + .withRequireEndpointVerification(false); + + EncryptionOptions encryptionOptions2 = + new EncryptionOptions() + .withStoreType("JKS") + .withKeyStore(TlsTestUtils.SERVER_KEYSTORE_PATH) + .withKeyStorePasswordFile(TlsTestUtils.SERVER_KEYSTORE_PASSWORD_FILE) + .withTrustStore(TlsTestUtils.SERVER_TRUSTSTORE_PATH) + .withTrustStorePasswordFile(TlsTestUtils.SERVER_TRUSTSTORE_PASSWORD_FILE) + .withProtocol("TLSv1.1") + .withRequireClientAuth(REQUIRED) + .withRequireEndpointVerification(false); + + assertEquals(encryptionOptions1, encryptionOptions2); + assertEquals(encryptionOptions1.hashCode(), encryptionOptions2.hashCode()); + } + + @Test + public void testMismatchForKeystoreOptionsWithPasswordFile() + { + EncryptionOptions.ServerEncryptionOptions encryptionOptions1 = createServerEncryptionOptions(); + EncryptionOptions.ServerEncryptionOptions encryptionOptions2 = createServerEncryptionOptions(); + + encryptionOptions1 = encryptionOptions1 + .withKeyStore(TlsTestUtils.SERVER_KEYSTORE_PATH) + .withKeyStorePassword(null) + .withKeyStorePasswordFile(TlsTestUtils.SERVER_KEYSTORE_PASSWORD_FILE); + + encryptionOptions2 = encryptionOptions2 + .withKeyStore(TlsTestUtils.SERVER_KEYSTORE_PATH) + .withKeyStorePassword(null) + .withKeyStorePasswordFile(TlsTestUtils.SERVER_TRUSTSTORE_PASSWORD_FILE); + + assertNotEquals(encryptionOptions1, encryptionOptions2); + assertNotEquals(encryptionOptions1.hashCode(), encryptionOptions2.hashCode()); + } + @Test public void testSameCustomSslContextFactoryImplementation() { diff --git a/test/unit/org/apache/cassandra/config/EncryptionOptionsTest.java b/test/unit/org/apache/cassandra/config/EncryptionOptionsTest.java index a295095dbbd1..5ef08eb060a0 100644 --- a/test/unit/org/apache/cassandra/config/EncryptionOptionsTest.java +++ b/test/unit/org/apache/cassandra/config/EncryptionOptionsTest.java @@ -61,8 +61,8 @@ public static EncryptionOptionsTestCase of(Boolean optional, String keystorePath { return new EncryptionOptionsTestCase(new EncryptionOptions(new ParameterizedClass("org.apache.cassandra.security.DefaultSslContextFactory", new HashMap<>()), - keystorePath, "dummypass", - "dummytruststore", "dummypass", + keystorePath, "dummypass", null, + "dummytruststore", "dummypass", null, Collections.emptyList(), null, null, null, "JKS", "false", false, enabled, optional, null, null) .applyConfig(), expected, @@ -75,8 +75,8 @@ public static EncryptionOptionsTestCase of(Boolean optional, String keystorePath { return new EncryptionOptionsTestCase(new EncryptionOptions(new ParameterizedClass("org.apache.cassandra.security.DefaultSslContextFactory", customSslContextFactoryParams), - keystorePath, "dummypass", - "dummytruststore", "dummypass", + keystorePath, "dummypass", null, + "dummytruststore", "dummypass", null, Collections.emptyList(), null, null, null, "JKS", "false", false, enabled, optional, null, null) .applyConfig(), expected, @@ -127,7 +127,10 @@ public static ServerEncryptionOptionsTestCase of(Boolean optional, String keysto EncryptionOptions.TlsEncryptionPolicy expected) { return new ServerEncryptionOptionsTestCase(new EncryptionOptions.ServerEncryptionOptions(new ParameterizedClass("org.apache.cassandra.security.DefaultSslContextFactory", - new HashMap<>()), keystorePath, "dummypass", keystorePath, "dummypass", "dummytruststore", "dummypass", + new HashMap<>()), + keystorePath, "dummypass", null, + keystorePath, "dummypass", null, + "dummytruststore", "dummypass", null, Collections.emptyList(), null, null, null, "JKS", "false", false, optional, internodeEncryption, false, null, null) .applyConfig(), expected, diff --git a/test/unit/org/apache/cassandra/security/FileBasedSslContextFactoryTest.java b/test/unit/org/apache/cassandra/security/FileBasedSslContextFactoryTest.java index 2916f0d46032..d6d936ba0e6c 100644 --- a/test/unit/org/apache/cassandra/security/FileBasedSslContextFactoryTest.java +++ b/test/unit/org/apache/cassandra/security/FileBasedSslContextFactoryTest.java @@ -31,6 +31,7 @@ import org.apache.cassandra.config.EncryptionOptions; import org.apache.cassandra.config.ParameterizedClass; import org.apache.cassandra.distributed.shared.WithProperties; +import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.transport.TlsTestUtils; import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_CONFIG; @@ -89,7 +90,7 @@ public void testHappyPath() throws SSLException } /** - * Tests that empty {@code keystore_password} and {@code outbound_keystore_password} is allowed. + * Tests that empty {@code keystore_password} and {@code outbound_keystore_password} are allowed. */ @Test public void testEmptyKeystorePasswords() throws SSLException @@ -103,7 +104,7 @@ public void testEmptyKeystorePasswords() throws SSLException Assert.assertEquals("org.apache.cassandra.security.FileBasedSslContextFactoryTest$TestFileBasedSSLContextFactory", localEncryptionOptions.ssl_context_factory.class_name); Assert.assertEquals("keystore_password must be empty", "", localEncryptionOptions.keystore_password); - Assert.assertEquals("outbound_keystore_password must empty", "", localEncryptionOptions.outbound_keystore_password); + Assert.assertEquals("outbound_keystore_password must be empty", "", localEncryptionOptions.outbound_keystore_password); TestFileBasedSSLContextFactory sslContextFactory = (TestFileBasedSSLContextFactory) localEncryptionOptions.sslContextFactoryInstance; @@ -112,6 +113,46 @@ public void testEmptyKeystorePasswords() throws SSLException sslContextFactory.buildTrustManagerFactory(); } + @Test + public void testKeystorePasswordFile() throws SSLException + { + // Here we only override password configuration and specify password_file configuration since keystore paths + // are already loaded in the `encryptionOptions` + EncryptionOptions.ServerEncryptionOptions localEncryptionOptions = encryptionOptions + .withKeyStorePassword(null) + .withKeyStorePasswordFile(TlsTestUtils.SERVER_KEYSTORE_PASSWORD_FILE) + .withOutboundKeystorePassword(null) + .withOutboundKeystorePasswordFile(TlsTestUtils.SERVER_OUTBOUND_KEYSTORE_PASSWORD_FILE) + .withTrustStorePassword(null) + .withTrustStorePasswordFile(TlsTestUtils.SERVER_TRUSTSTORE_PASSWORD_FILE); + + Assert.assertEquals("org.apache.cassandra.security.FileBasedSslContextFactoryTest$TestFileBasedSSLContextFactory", + localEncryptionOptions.ssl_context_factory.class_name); + TestFileBasedSSLContextFactory sslContextFactory = + (TestFileBasedSSLContextFactory) localEncryptionOptions.sslContextFactoryInstance; + + sslContextFactory.buildKeyManagerFactory(); + sslContextFactory.buildTrustManagerFactory(); + } + + /** + * Tests for missing password configuration and non-existance file specified in the password_file configuration. + * @throws SSLException + */ + @Test(expected = ConfigurationException.class) + public void testBadKeystorePasswordFile() throws SSLException + { + // Here we only override password configuration and specify password_file configuration since keystore paths + // are already loaded in the `encryptionOptions` + encryptionOptions + .withKeyStorePassword(null) + .withKeyStorePasswordFile("/path/to/non-existance-password-file") + .withOutboundKeystorePassword(null) + .withOutboundKeystorePasswordFile("/path/to/non-existance-password-file") + .withTrustStorePassword(null) + .withTrustStorePasswordFile("/path/to/non-existance-password-file"); + } + /** * Tests that an absent keystore_password for the {@code keystore} is disallowed. */ diff --git a/test/unit/org/apache/cassandra/security/FileBasedStoreContextTest.java b/test/unit/org/apache/cassandra/security/FileBasedStoreContextTest.java new file mode 100644 index 000000000000..cd1e52c61f70 --- /dev/null +++ b/test/unit/org/apache/cassandra/security/FileBasedStoreContextTest.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.security; + +import org.junit.Test; + +import org.apache.cassandra.exceptions.ConfigurationException; + +import static org.junit.Assert.assertEquals; + +public class FileBasedStoreContextTest +{ + /** + * Tests default behavior without keystore password file specified. + */ + @Test + public void testPasswordConfiguration() + { + FileBasedSslContextFactory.FileBasedStoreContext wrapper = new FileBasedSslContextFactory.FileBasedStoreContext("test/conf/cassandra_ssl_test.keystore", "cassandra", null); + assertEquals("Password must be loaded from the direct configuration", "cassandra", wrapper.password); + } + + /** + * Tests behavior when password for keystore is specified via a password file. + */ + @Test + public void testPasswordFileConfiguration() + { + FileBasedSslContextFactory.FileBasedStoreContext wrapper = new FileBasedSslContextFactory.FileBasedStoreContext("test/conf/cassandra_ssl_test.keystore", null, "test/conf/cassandra_ssl_test_keystore_passwordfile.txt"); + assertEquals("Password must be loaded from the password file", "cassandra", wrapper.password); + } + + /** + * Tests when password for keystore is specified via password configuration and a password file both. + */ + @Test + public void testPasswordAndPasswordFileConfiguration() + { + String expectedPassword = "cassandra123"; + FileBasedSslContextFactory.FileBasedStoreContext wrapper = new FileBasedSslContextFactory.FileBasedStoreContext("test/conf/cassandra_ssl_test.keystore", expectedPassword, "test/conf/cassandra_ssl_test_keystore_passwordfile.txt"); + assertEquals("Password configuration must take precedence", expectedPassword, wrapper.password); + } + + /** + * Tests behavior when a non-existing password file is specified for keystore's password and password configuration + * is {@code null}. + */ + @Test(expected = ConfigurationException.class) + public void testMissingPasswordFile() + { + new FileBasedSslContextFactory.FileBasedStoreContext("test/conf/cassandra_ssl_test.keystore", null, "passwordfile-that-doesnotexist"); + } + + /** + * Tests behavior when non-null empty password is specified in the password configuration. + * The empty password via the configuration must take precedence in this case. + */ + @Test + public void testBlankPasswordConfiguration() + { + FileBasedSslContextFactory.FileBasedStoreContext wrapper = new FileBasedSslContextFactory.FileBasedStoreContext("test/conf/cassandra_ssl_test.keystore", "", "test/conf/cassandra_ssl_test_keystore_passwordfile.txt"); + assertEquals("Password must be loaded from the direct configuration", "", wrapper.password); + } +} diff --git a/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryConfigWithPasswordFileTest.java b/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryConfigWithPasswordFileTest.java new file mode 100644 index 000000000000..fc837a372970 --- /dev/null +++ b/test/unit/org/apache/cassandra/security/PEMBasedSslContextFactoryConfigWithPasswordFileTest.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.security; + +import javax.net.ssl.SSLException; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.Config; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.distributed.shared.WithProperties; + +import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_CONFIG; +import static org.junit.Assert.assertEquals; + +public class PEMBasedSslContextFactoryConfigWithPasswordFileTest +{ + static WithProperties properties; + + @BeforeClass + public static void setupDatabaseDescriptor() + { + properties = new WithProperties().set(CASSANDRA_CONFIG, "cassandra-pem-sslcontextfactory-with-passwordfile.yaml"); + } + + @AfterClass + public static void tearDownDatabaseDescriptor() + { + properties.close(); + } + + @Test + public void testHappyPathInlinePEM() throws SSLException + { + + Config config = DatabaseDescriptor.loadConfig(); + config.client_encryption_options.applyConfig(); + + assertEquals("org.apache.cassandra.security.PEMBasedSslContextFactory", + config.client_encryption_options.ssl_context_factory.class_name); + assertEquals(config.client_encryption_options.ssl_context_factory.class_name, + config.client_encryption_options.sslContextFactoryInstance.getClass().getName()); + PEMBasedSslContextFactory sslContextFactory = + (PEMBasedSslContextFactory) config.client_encryption_options.sslContextFactoryInstance; + sslContextFactory.buildKeyManagerFactory(); + sslContextFactory.buildTrustManagerFactory(); + } + + @Test + public void testHappyPathFileBasedPEM() throws SSLException + { + + Config config = DatabaseDescriptor.loadConfig(); + config.server_encryption_options.applyConfig(); + + assertEquals("org.apache.cassandra.security.PEMBasedSslContextFactory", + config.server_encryption_options.ssl_context_factory.class_name); + assertEquals(config.server_encryption_options.ssl_context_factory.class_name, + config.server_encryption_options.sslContextFactoryInstance.getClass().getName()); + PEMBasedSslContextFactory sslContextFactory = + (PEMBasedSslContextFactory) config.server_encryption_options.sslContextFactoryInstance; + sslContextFactory.buildKeyManagerFactory(); + sslContextFactory.buildTrustManagerFactory(); + } +} diff --git a/test/unit/org/apache/cassandra/transport/TlsTestUtils.java b/test/unit/org/apache/cassandra/transport/TlsTestUtils.java index f3994e76136f..30a6054127ca 100644 --- a/test/unit/org/apache/cassandra/transport/TlsTestUtils.java +++ b/test/unit/org/apache/cassandra/transport/TlsTestUtils.java @@ -58,16 +58,19 @@ public class TlsTestUtils public static String SERVER_KEYSTORE_PATH_PEM = "test/conf/cassandra_ssl_test.keystore.pem"; public static String SERVER_KEYSTORE_PATH_UNENCRYPTED_PEM = "test/conf/cassandra_ssl_test.unencrypted_keystore.pem"; public static String SERVER_KEYSTORE_PASSWORD = "cassandra"; + public static String SERVER_KEYSTORE_PASSWORD_FILE = "test/conf/cassandra_ssl_test_keystore_passwordfile.txt"; public static String SERVER_KEYSTORE_ENDPOINT_VERIFY_PATH = "test/conf/cassandra_ssl_test_endpoint_verify.keystore"; public static String SERVER_KEYSTORE_ENDPOINT_VERIFY_PASSWORD = "cassandra"; public static String SERVER_OUTBOUND_KEYSTORE_PATH = "test/conf/cassandra_ssl_test_outbound.keystore"; public static String SERVER_OUTBOUND_KEYSTORE_PASSWORD = "cassandra"; + public static String SERVER_OUTBOUND_KEYSTORE_PASSWORD_FILE = "test/conf/cassandra_ssl_test_outbound_keystore_password.txt"; public static String SERVER_TRUSTSTORE_PATH = "test/conf/cassandra_ssl_test.truststore"; public static String SERVER_TRUSTSTORE_PEM_PATH = "test/conf/cassandra_ssl_test.truststore.pem"; public static String SERVER_TRUSTSTORE_PASSWORD = "cassandra"; + public static String SERVER_TRUSTSTORE_PASSWORD_FILE = "test/conf/cassandra_ssl_test_truststore_passwordfile.txt"; // To regenerate: // 1. generate keystore diff --git a/test/unit/org/apache/cassandra/utils/jmx/JMXSslConfiguredWithYamlFileOptionsAndPasswordFileTest.java b/test/unit/org/apache/cassandra/utils/jmx/JMXSslConfiguredWithYamlFileOptionsAndPasswordFileTest.java new file mode 100644 index 000000000000..ae2b7e6128dc --- /dev/null +++ b/test/unit/org/apache/cassandra/utils/jmx/JMXSslConfiguredWithYamlFileOptionsAndPasswordFileTest.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils.jmx; + +import java.net.InetAddress; +import java.util.Map; +import javax.management.remote.rmi.RMIConnectorServer; +import javax.net.ssl.SSLException; +import javax.rmi.ssl.SslRMIServerSocketFactory; + +import org.apache.commons.lang3.StringUtils; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.JMXServerOptions; +import org.apache.cassandra.distributed.shared.WithProperties; +import org.apache.cassandra.utils.JMXServerUtils; + +import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_CONFIG; +import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_SSL; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * Tests for JMX SSL configuration specified in the cassandra.yaml using jmx_encryption_options with password files for + * keystore and truststores as applicable. + * @see JMXSslConfiguredWithYamlFileOptionsTest + */ +public class JMXSslConfiguredWithYamlFileOptionsAndPasswordFileTest +{ + static WithProperties properties; + + @BeforeClass + public static void setupDatabaseDescriptor() + { + properties = new WithProperties().set(CASSANDRA_CONFIG, "cassandra-jmx-sslconfig-with-passwordfile.yaml"); + DatabaseDescriptor.daemonInitialization(); + } + + @AfterClass + public static void tearDownDatabaseDescriptor() + { + properties.close(); + } + + @Test + public void testYamlFileJmxEncryptionOptions() throws SSLException + { + JMXServerOptions serverOptions = DatabaseDescriptor.getJmxServerOptions(); + String expectedProtocols = StringUtils.join(serverOptions.jmx_encryption_options.getAcceptedProtocols(), ","); + String expectedCipherSuites = StringUtils.join(serverOptions.jmx_encryption_options.cipherSuitesArray(), ","); + + InetAddress serverAddress = InetAddress.getLoopbackAddress(); + + try (WithProperties ignored = JMXSslPropertiesUtil.use(false)) + { + Map<String, Object> env = JMXServerUtils.configureJmxSocketFactories(serverAddress, serverOptions); + assertTrue("com.sun.management.jmxremote.ssl must be true", COM_SUN_MANAGEMENT_JMXREMOTE_SSL.getBoolean()); + assertNotNull("ServerSocketFactory must not be null", env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE)); + assertTrue("RMI_SERVER_SOCKET_FACTORY must be of JMXSslRMIServerSocketFactory type", env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE) instanceof SslRMIServerSocketFactory); + assertNotNull("ClientSocketFactory must not be null", env.get(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE)); + assertNotNull("com.sun.jndi.rmi.factory.socket must be set in the env", env.get("com.sun.jndi.rmi.factory.socket")); + assertEquals("javax.rmi.ssl.client.enabledProtocols must match", expectedProtocols, JAVAX_RMI_SSL_CLIENT_ENABLED_PROTOCOLS.getString()); + assertEquals("javax.rmi.ssl.client.enabledCipherSuites must match", expectedCipherSuites, JAVAX_RMI_SSL_CLIENT_ENABLED_CIPHER_SUITES.getString()); + } + } +} From eed4fbc8f7789424d670c42a60f12150c48cd492 Mon Sep 17 00:00:00 2001 From: Dmitry Konstantinov <konstantinov@netcracker.com> Date: Tue, 4 Mar 2025 22:33:14 +0300 Subject: [PATCH 164/225] Reduce heap memory allocations in different places along the hot write path Avoid iterator allocations if possible Handle typical cases (such as a single row, single table writes) more efficiently Add fast paths for typical scenarios (like absense of views and triggers) Memorize things which can be computed once Patch by Dmitry Konstantinov; reviewed by Chris Lohfink, Michael Semb Wever, Vladimir Sitnikov for CASSANDRA-20167 --- CHANGES.txt | 1 + .../cassandra/concurrent/SEPWorker.java | 4 +- .../org/apache/cassandra/cql3/Attributes.java | 4 +- .../cassandra/cql3/BatchQueryOptions.java | 6 +- .../apache/cassandra/cql3/CQLStatement.java | 6 +- .../apache/cassandra/cql3/QueryOptions.java | 139 ++++++++++-------- .../cassandra/cql3/UpdateParameters.java | 4 +- .../cql3/VariableSpecifications.java | 15 ++ .../cql3/statements/BatchStatement.java | 11 +- .../cql3/statements/DescribeStatement.java | 4 +- .../statements/ModificationStatement.java | 47 ++++-- .../cql3/statements/SelectStatement.java | 5 +- ...eTableSinglePartitionUpdatesCollector.java | 105 +++++++++++++ .../SingleTableUpdatesCollector.java | 34 +++-- .../org/apache/cassandra/db/Keyspace.java | 9 +- .../org/apache/cassandra/db/Mutation.java | 11 +- .../cassandra/db/RegularAndStaticColumns.java | 16 +- .../cassandra/db/filter/ColumnFilter.java | 9 ++ .../db/partitions/PartitionUpdate.java | 55 +++++-- .../apache/cassandra/db/rows/BTreeRow.java | 19 +++ .../cassandra/db/rows/EncodingStats.java | 7 + .../org/apache/cassandra/db/view/View.java | 2 + .../apache/cassandra/db/view/ViewManager.java | 17 ++- .../index/SecondaryIndexManager.java | 2 + .../io/sstable/format/SortedTableWriter.java | 16 +- .../cassandra/io/util/DataOutputBuffer.java | 12 ++ .../cassandra/locator/ReplicaLayout.java | 2 + .../service/AbstractWriteResponseHandler.java | 21 ++- .../apache/cassandra/service/ClientWarn.java | 10 +- .../cassandra/service/StorageProxy.java | 29 +++- .../transport/CQLMessageHandler.java | 9 +- .../apache/cassandra/transport/Envelope.java | 77 +++++----- .../apache/cassandra/transport/Flusher.java | 37 +++-- .../apache/cassandra/transport/Message.java | 22 +-- .../cassandra/cql3/QueryOptionsFlagsTest.java | 70 +++++++++ .../transport/CQLConnectionTest.java | 2 +- .../transport/EnvelopeHeaderFlagsTest.java | 62 ++++++++ 37 files changed, 697 insertions(+), 204 deletions(-) create mode 100644 src/java/org/apache/cassandra/cql3/statements/SingleTableSinglePartitionUpdatesCollector.java create mode 100644 test/unit/org/apache/cassandra/cql3/QueryOptionsFlagsTest.java create mode 100644 test/unit/org/apache/cassandra/transport/EnvelopeHeaderFlagsTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 9b84f503c45f..f790960a2166 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Reduce memory allocations in miscellaneous places along the hot write path (CASSANDRA-20167) * Provide keystore_password_file and truststore_password_file options to read credentials from a file (CASSANDRA-13428) * Unregistering a node should also remove it from tokenMap if it is there and recalculate the placements (CASSANDRA-20346) * Fix PartitionUpdate.isEmpty deserialization issue to avoid potential EOFException (CASSANDRA-20345) diff --git a/src/java/org/apache/cassandra/concurrent/SEPWorker.java b/src/java/org/apache/cassandra/concurrent/SEPWorker.java index 93c01fa7a3c7..add2ef8b061c 100644 --- a/src/java/org/apache/cassandra/concurrent/SEPWorker.java +++ b/src/java/org/apache/cassandra/concurrent/SEPWorker.java @@ -39,6 +39,7 @@ final class SEPWorker extends AtomicReference<SEPWorker.Work> implements Runnabl private static final boolean SET_THREAD_NAME = SET_SEP_THREAD_NAME.getBoolean(); final Long workerId; + final String workerIdThreadSuffix; final Thread thread; final SharedExecutorPool pool; @@ -55,6 +56,7 @@ final class SEPWorker extends AtomicReference<SEPWorker.Work> implements Runnabl { this.pool = pool; this.workerId = workerId; + this.workerIdThreadSuffix = '-' + workerId.toString(); thread = new FastThreadLocalThread(threadGroup, this, threadGroup.getName() + "-Worker-" + workerId); thread.setDaemon(true); set(initialState); @@ -122,7 +124,7 @@ public void run() if (assigned == null) continue; if (SET_THREAD_NAME) - Thread.currentThread().setName(assigned.name + '-' + workerId); + Thread.currentThread().setName(assigned.name + workerIdThreadSuffix); task = assigned.tasks.poll(); currentTask.lazySet(task); diff --git a/src/java/org/apache/cassandra/cql3/Attributes.java b/src/java/org/apache/cassandra/cql3/Attributes.java index 492c23672aa4..493193b40012 100644 --- a/src/java/org/apache/cassandra/cql3/Attributes.java +++ b/src/java/org/apache/cassandra/cql3/Attributes.java @@ -47,12 +47,14 @@ public class Attributes */ public static final int MAX_TTL = 20 * 365 * 24 * 60 * 60; // 20 years in seconds + private static final Attributes NONE = new Attributes(null, null); + private final Term timestamp; private final Term timeToLive; public static Attributes none() { - return new Attributes(null, null); + return NONE; } private Attributes(Term timestamp, Term timeToLive) diff --git a/src/java/org/apache/cassandra/cql3/BatchQueryOptions.java b/src/java/org/apache/cassandra/cql3/BatchQueryOptions.java index 63be061e5ef3..dc38c4d0ce05 100644 --- a/src/java/org/apache/cassandra/cql3/BatchQueryOptions.java +++ b/src/java/org/apache/cassandra/cql3/BatchQueryOptions.java @@ -22,6 +22,8 @@ import java.util.Collections; import java.util.List; +import com.google.common.collect.ImmutableList; + import org.apache.cassandra.utils.MD5Digest; import org.apache.cassandra.db.ConsistencyLevel; @@ -54,7 +56,7 @@ public static BatchQueryOptions withPerStatementVariables(QueryOptions options, public abstract QueryOptions forStatement(int i); - public void prepareStatement(int i, List<ColumnSpecification> boundNames) + public void prepareStatement(int i, ImmutableList<ColumnSpecification> boundNames) { forStatement(i).prepare(boundNames); } @@ -128,7 +130,7 @@ public QueryOptions forStatement(int i) } @Override - public void prepareStatement(int i, List<ColumnSpecification> boundNames) + public void prepareStatement(int i, ImmutableList<ColumnSpecification> boundNames) { if (isPreparedStatement(i)) { diff --git a/src/java/org/apache/cassandra/cql3/CQLStatement.java b/src/java/org/apache/cassandra/cql3/CQLStatement.java index badf9c342879..349e79b30ff4 100644 --- a/src/java/org/apache/cassandra/cql3/CQLStatement.java +++ b/src/java/org/apache/cassandra/cql3/CQLStatement.java @@ -20,6 +20,8 @@ import java.util.Collections; import java.util.List; +import com.google.common.collect.ImmutableList; + import org.apache.cassandra.audit.AuditLogContext; import org.apache.cassandra.cql3.functions.Function; import org.apache.cassandra.service.ClientState; @@ -32,9 +34,9 @@ public interface CQLStatement /** * Returns all bind variables for the statement */ - default List<ColumnSpecification> getBindVariables() + default ImmutableList<ColumnSpecification> getBindVariables() { - return Collections.emptyList(); + return ImmutableList.of(); } /** diff --git a/src/java/org/apache/cassandra/cql3/QueryOptions.java b/src/java/org/apache/cassandra/cql3/QueryOptions.java index d3093cbb8eac..c4e6e33b25b5 100644 --- a/src/java/org/apache/cassandra/cql3/QueryOptions.java +++ b/src/java/org/apache/cassandra/cql3/QueryOptions.java @@ -110,7 +110,7 @@ public static QueryOptions create(ConsistencyLevel consistency, version); } - public static QueryOptions addColumnSpecifications(QueryOptions options, List<ColumnSpecification> columnSpecs) + public static QueryOptions addColumnSpecifications(QueryOptions options, ImmutableList<ColumnSpecification> columnSpecs) { return new OptionsWithColumnSpecifications(options, columnSpecs); } @@ -476,10 +476,10 @@ static class OptionsWithColumnSpecifications extends QueryOptionsWrapper { private final ImmutableList<ColumnSpecification> columnSpecs; - OptionsWithColumnSpecifications(QueryOptions wrapped, List<ColumnSpecification> columnSpecs) + OptionsWithColumnSpecifications(QueryOptions wrapped, ImmutableList<ColumnSpecification> columnSpecs) { super(wrapped); - this.columnSpecs = ImmutableList.copyOf(columnSpecs); + this.columnSpecs = columnSpecs; } @Override @@ -568,9 +568,9 @@ public SpecificOptions withNowInSec(long nowInSec) } } - private static class Codec implements CBCodec<QueryOptions> + static class Codec implements CBCodec<QueryOptions> { - private enum Flag + enum Flag { // The order of that enum matters!! VALUES, @@ -583,40 +583,53 @@ private enum Flag KEYSPACE, NOW_IN_SECONDS; - private static final Flag[] ALL_VALUES = values(); + private final int mask; - public static EnumSet<Flag> deserialize(int flags) + Flag() { - EnumSet<Flag> set = EnumSet.noneOf(Flag.class); - for (int n = 0; n < ALL_VALUES.length; n++) - { - if ((flags & (1 << n)) != 0) - set.add(ALL_VALUES[n]); - } - return set; + this.mask = 1 << this.ordinal(); + } + + public static int none() + { + return 0; + } + + public static boolean isEmpty(int flags) + { + return flags == 0; + } + + public static int add(int flags, Flag flagToAdd) + { + flags |= flagToAdd.mask; + return flags; + } + + public static int remove(int flags, Flag flagToRemove) + { + flags &= ~ flagToRemove.mask; + return flags; } - public static int serialize(EnumSet<Flag> flags) + public static boolean contains(long flags, Flag flag) { - int i = 0; - for (Flag flag : flags) - i |= 1 << flag.ordinal(); - return i; + return (flags & flag.mask) != 0; } } public QueryOptions decode(ByteBuf body, ProtocolVersion version) { ConsistencyLevel consistency = CBUtil.readConsistencyLevel(body); - EnumSet<Flag> flags = Flag.deserialize(version.isGreaterOrEqualTo(ProtocolVersion.V5) - ? (int)body.readUnsignedInt() - : (int)body.readUnsignedByte()); + int flags = version.isGreaterOrEqualTo(ProtocolVersion.V5) + ? (int)body.readUnsignedInt() + : (int)body.readUnsignedByte(); List<ByteBuffer> values = Collections.<ByteBuffer>emptyList(); List<String> names = null; - if (flags.contains(Flag.VALUES)) + if (Flag.contains(flags, Flag.VALUES)) { - if (flags.contains(Flag.NAMES_FOR_VALUES)) + if (Flag.contains(flags, Flag.NAMES_FOR_VALUES)) { Pair<List<String>, List<ByteBuffer>> namesAndValues = CBUtil.readNameAndValueList(body, version); names = namesAndValues.left; @@ -628,27 +641,27 @@ public QueryOptions decode(ByteBuf body, ProtocolVersion version) } } - boolean skipMetadata = flags.contains(Flag.SKIP_METADATA); - flags.remove(Flag.VALUES); - flags.remove(Flag.SKIP_METADATA); + boolean skipMetadata = Flag.contains(flags, Flag.SKIP_METADATA); + flags = Flag.remove(flags, Flag.VALUES); + flags = Flag.remove(flags, Flag.SKIP_METADATA); SpecificOptions options = SpecificOptions.DEFAULT; - if (!flags.isEmpty()) + if (!Flag.isEmpty(flags)) { - int pageSize = flags.contains(Flag.PAGE_SIZE) ? body.readInt() : -1; - PagingState pagingState = flags.contains(Flag.PAGING_STATE) ? PagingState.deserialize(CBUtil.readValueNoCopy(body), version) : null; - ConsistencyLevel serialConsistency = flags.contains(Flag.SERIAL_CONSISTENCY) ? CBUtil.readConsistencyLevel(body) : ConsistencyLevel.SERIAL; + int pageSize = Flag.contains(flags, Flag.PAGE_SIZE) ? body.readInt() : -1; + PagingState pagingState = Flag.contains(flags, Flag.PAGING_STATE) ? PagingState.deserialize(CBUtil.readValueNoCopy(body), version) : null; + ConsistencyLevel serialConsistency = Flag.contains(flags, Flag.SERIAL_CONSISTENCY) ? CBUtil.readConsistencyLevel(body) : ConsistencyLevel.SERIAL; long timestamp = Long.MIN_VALUE; - if (flags.contains(Flag.TIMESTAMP)) + if (Flag.contains(flags, Flag.TIMESTAMP)) { long ts = body.readLong(); if (ts == Long.MIN_VALUE) throw new ProtocolException(String.format("Out of bound timestamp, must be in [%d, %d] (got %d)", Long.MIN_VALUE + 1, Long.MAX_VALUE, ts)); timestamp = ts; } - String keyspace = flags.contains(Flag.KEYSPACE) ? CBUtil.readString(body) : null; - long nowInSeconds = flags.contains(Flag.NOW_IN_SECONDS) ? CassandraUInt.toLong(body.readInt()) - : UNSET_NOWINSEC; + String keyspace = Flag.contains(flags, Flag.KEYSPACE) ? CBUtil.readString(body) : null; + long nowInSeconds = Flag.contains(flags, Flag.NOW_IN_SECONDS) ? CassandraUInt.toLong(body.readInt()) + : UNSET_NOWINSEC; options = new SpecificOptions(pageSize, pagingState, serialConsistency, timestamp, keyspace, nowInSeconds); } @@ -660,25 +673,25 @@ public void encode(QueryOptions options, ByteBuf dest, ProtocolVersion version) { CBUtil.writeConsistencyLevel(options.getConsistency(), dest); - EnumSet<Flag> flags = gatherFlags(options, version); + int flags = gatherFlags(options, version); if (version.isGreaterOrEqualTo(ProtocolVersion.V5)) - dest.writeInt(Flag.serialize(flags)); + dest.writeInt(flags); else - dest.writeByte((byte)Flag.serialize(flags)); + dest.writeByte((byte) flags); - if (flags.contains(Flag.VALUES)) + if (Flag.contains(flags, Flag.VALUES)) CBUtil.writeValueList(options.getValues(), dest); - if (flags.contains(Flag.PAGE_SIZE)) + if (Flag.contains(flags, Flag.PAGE_SIZE)) dest.writeInt(options.getPageSize()); - if (flags.contains(Flag.PAGING_STATE)) + if (Flag.contains(flags, Flag.PAGING_STATE)) CBUtil.writeValue(options.getPagingState().serialize(version), dest); - if (flags.contains(Flag.SERIAL_CONSISTENCY)) + if (Flag.contains(flags, Flag.SERIAL_CONSISTENCY)) CBUtil.writeConsistencyLevel(options.getSerialConsistency(), dest); - if (flags.contains(Flag.TIMESTAMP)) + if (Flag.contains(flags, Flag.TIMESTAMP)) dest.writeLong(options.getSpecificOptions().timestamp); - if (flags.contains(Flag.KEYSPACE)) + if (Flag.contains(flags, Flag.KEYSPACE)) CBUtil.writeAsciiString(options.getSpecificOptions().keyspace, dest); - if (flags.contains(Flag.NOW_IN_SECONDS)) + if (Flag.contains(flags, Flag.NOW_IN_SECONDS)) dest.writeInt(CassandraUInt.fromLong(options.getSpecificOptions().nowInSeconds)); // Note that we don't really have to bother with NAMES_FOR_VALUES server side, @@ -692,49 +705,49 @@ public int encodedSize(QueryOptions options, ProtocolVersion version) size += CBUtil.sizeOfConsistencyLevel(options.getConsistency()); - EnumSet<Flag> flags = gatherFlags(options, version); + int flags = gatherFlags(options, version); size += (version.isGreaterOrEqualTo(ProtocolVersion.V5) ? 4 : 1); - if (flags.contains(Flag.VALUES)) + if (Flag.contains(flags, Flag.VALUES)) size += CBUtil.sizeOfValueList(options.getValues()); - if (flags.contains(Flag.PAGE_SIZE)) + if (Flag.contains(flags, Flag.PAGE_SIZE)) size += 4; - if (flags.contains(Flag.PAGING_STATE)) + if (Flag.contains(flags, Flag.PAGING_STATE)) size += CBUtil.sizeOfValue(options.getPagingState().serializedSize(version)); - if (flags.contains(Flag.SERIAL_CONSISTENCY)) + if (Flag.contains(flags, Flag.SERIAL_CONSISTENCY)) size += CBUtil.sizeOfConsistencyLevel(options.getSerialConsistency()); - if (flags.contains(Flag.TIMESTAMP)) + if (Flag.contains(flags, Flag.TIMESTAMP)) size += 8; - if (flags.contains(Flag.KEYSPACE)) + if (Flag.contains(flags, Flag.KEYSPACE)) size += CBUtil.sizeOfAsciiString(options.getSpecificOptions().keyspace); - if (flags.contains(Flag.NOW_IN_SECONDS)) + if (Flag.contains(flags, Flag.NOW_IN_SECONDS)) size += 4; return size; } - private EnumSet<Flag> gatherFlags(QueryOptions options, ProtocolVersion version) + private int gatherFlags(QueryOptions options, ProtocolVersion version) { - EnumSet<Flag> flags = EnumSet.noneOf(Flag.class); + int flags = Flag.none(); if (options.getValues().size() > 0) - flags.add(Flag.VALUES); + flags = Flag.add(flags, Flag.VALUES); if (options.skipMetadata()) - flags.add(Flag.SKIP_METADATA); + flags = Flag.add(flags, Flag.SKIP_METADATA); if (options.getPageSize() >= 0) - flags.add(Flag.PAGE_SIZE); + flags = Flag.add(flags, Flag.PAGE_SIZE); if (options.getPagingState() != null) - flags.add(Flag.PAGING_STATE); + flags = Flag.add(flags, Flag.PAGING_STATE); if (options.getSerialConsistency() != ConsistencyLevel.SERIAL) - flags.add(Flag.SERIAL_CONSISTENCY); + flags = Flag.add(flags, Flag.SERIAL_CONSISTENCY); if (options.getSpecificOptions().timestamp != Long.MIN_VALUE) - flags.add(Flag.TIMESTAMP); + flags = Flag.add(flags, Flag.TIMESTAMP); if (version.isGreaterOrEqualTo(ProtocolVersion.V5)) { if (options.getSpecificOptions().keyspace != null) - flags.add(Flag.KEYSPACE); + flags = Flag.add(flags, Flag.KEYSPACE); if (options.getSpecificOptions().nowInSeconds != UNSET_NOWINSEC) - flags.add(Flag.NOW_IN_SECONDS); + flags = Flag.add(flags, Flag.NOW_IN_SECONDS); } return flags; diff --git a/src/java/org/apache/cassandra/cql3/UpdateParameters.java b/src/java/org/apache/cassandra/cql3/UpdateParameters.java index 51df331b8dc7..d13d0b49a76f 100644 --- a/src/java/org/apache/cassandra/cql3/UpdateParameters.java +++ b/src/java/org/apache/cassandra/cql3/UpdateParameters.java @@ -103,13 +103,13 @@ public <V> void newRow(Clustering<V> clustering) throws InvalidRequestException if (clustering == Clustering.STATIC_CLUSTERING) { if (staticBuilder == null) - staticBuilder = BTreeRow.unsortedBuilder(); + staticBuilder = BTreeRow.pooledUnsortedBuilder(); builder = staticBuilder; } else { if (regularBuilder == null) - regularBuilder = BTreeRow.unsortedBuilder(); + regularBuilder = BTreeRow.pooledUnsortedBuilder(); builder = regularBuilder; } diff --git a/src/java/org/apache/cassandra/cql3/VariableSpecifications.java b/src/java/org/apache/cassandra/cql3/VariableSpecifications.java index e58290eba98b..504859cac447 100644 --- a/src/java/org/apache/cassandra/cql3/VariableSpecifications.java +++ b/src/java/org/apache/cassandra/cql3/VariableSpecifications.java @@ -21,6 +21,8 @@ import java.util.Collections; import java.util.List; +import com.google.common.collect.ImmutableList; + import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.TableMetadata; @@ -28,6 +30,7 @@ public class VariableSpecifications { private final List<ColumnIdentifier> variableNames; private final List<ColumnSpecification> specs; + private volatile ImmutableList<ColumnSpecification> immutableSpecs; private final ColumnMetadata[] targetColumns; public VariableSpecifications(List<ColumnIdentifier> variableNames) @@ -56,6 +59,17 @@ public List<ColumnSpecification> getBindVariables() return specs; } + public ImmutableList<ColumnSpecification> getImmutableBindVariables() + { + ImmutableList<ColumnSpecification> result = immutableSpecs; + if (result == null) // strong syncrhronization is not needed, it is ok if sometimes we create several immutable lists + { + result = ImmutableList.copyOf(specs); + immutableSpecs = result; + } + return result; + } + /** * Returns an array with the same length as the number of partition key columns for the table corresponding * to table. Each short in the array represents the bind index of the marker that holds the value for that @@ -87,6 +101,7 @@ public short[] getPartitionKeyBindVariableIndexes(TableMetadata metadata) public void add(int bindIndex, ColumnSpecification spec) { + assert immutableSpecs == null : "bind variable specs cannot be modified once we started to use them"; if (spec instanceof ColumnMetadata) targetColumns[bindIndex] = (ColumnMetadata) spec; diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java index e5104376cef4..bfec675464db 100644 --- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java @@ -23,6 +23,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.HashMultiset; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import org.slf4j.Logger; @@ -137,9 +138,9 @@ public BatchStatement(Type type, VariableSpecifications bindVariables, List<Modi } @Override - public List<ColumnSpecification> getBindVariables() + public ImmutableList<ColumnSpecification> getBindVariables() { - return bindVariables.getBindVariables(); + return bindVariables.getImmutableBindVariables(); } @Override @@ -419,8 +420,10 @@ public ResultMessage execute(QueryState queryState, BatchQueryOptions options, D throw new InvalidRequestException("Invalid empty serial consistency level"); ClientState clientState = queryState.getClientState(); - Guardrails.writeConsistencyLevels.guard(EnumSet.of(options.getConsistency(), options.getSerialConsistency()), - clientState); + if (Guardrails.writeConsistencyLevels.enabled(clientState)) // to avoid EnumSet allocation + Guardrails.writeConsistencyLevels.guard(EnumSet.of(options.getConsistency(), + options.getSerialConsistency()), + clientState); for (int i = 0; i < statements.size(); i++ ) statements.get(i).validateDiskUsage(options.forStatement(i), clientState); diff --git a/src/java/org/apache/cassandra/cql3/statements/DescribeStatement.java b/src/java/org/apache/cassandra/cql3/statements/DescribeStatement.java index 360d34b293e9..64b7862d6d3c 100644 --- a/src/java/org/apache/cassandra/cql3/statements/DescribeStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/DescribeStatement.java @@ -107,9 +107,9 @@ public final CQLStatement prepare(ClientState clientState) throws RequestValidat return this; } - public final List<ColumnSpecification> getBindVariables() + public final ImmutableList<ColumnSpecification> getBindVariables() { - return Collections.emptyList(); + return ImmutableList.of(); } @Override diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java index 7698c0b59c07..ce9da9a538ee 100644 --- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java @@ -21,6 +21,7 @@ import java.util.*; import com.google.common.collect.HashMultiset; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -103,6 +104,8 @@ public abstract class ModificationStatement implements CQLStatement.SingleKeyspa private final RegularAndStaticColumns requiresRead; + private final List<Function> functions; + public ModificationStatement(StatementType type, VariableSpecifications bindVariables, TableMetadata metadata, @@ -179,12 +182,13 @@ else if (this.type.isDelete()) this.conditionColumns = conditionColumnsBuilder.build(); this.requiresRead = requiresReadBuilder.build(); + this.functions = findAllFunctions(); } @Override - public List<ColumnSpecification> getBindVariables() + public ImmutableList<ColumnSpecification> getBindVariables() { - return bindVariables.getBindVariables(); + return bindVariables.getImmutableBindVariables(); } @Override @@ -195,9 +199,19 @@ public short[] getPartitionKeyBindVariableIndexes() @Override public Iterable<Function> getFunctions() + { + return functions; + } + + private List<Function> findAllFunctions() { List<Function> functions = new ArrayList<>(); addFunctionsTo(functions); + if (functions.isEmpty()) + { + functions = Collections.emptyList(); // to avoid a new Iterator object creation during each authorization + } + return functions; } @@ -518,8 +532,9 @@ public ResultMessage execute(QueryState queryState, QueryOptions options, Dispat if (options.getConsistency() == null) throw new InvalidRequestException("Invalid empty consistency level"); - Guardrails.writeConsistencyLevels.guard(EnumSet.of(options.getConsistency(), options.getSerialConsistency()), - queryState.getClientState()); + if (Guardrails.writeConsistencyLevels.enabled(queryState.getClientState())) // to avoid EnumSet allocation + Guardrails.writeConsistencyLevels.guard(EnumSet.of(options.getConsistency(), options.getSerialConsistency()), + queryState.getClientState()); return hasConditions() ? executeWithCondition(queryState, options, requestTime) @@ -768,10 +783,18 @@ private List<? extends IMutation> getMutations(ClientState state, Dispatcher.RequestTime requestTime) { List<ByteBuffer> keys = buildPartitionKeyNames(options, state); - HashMultiset<ByteBuffer> perPartitionKeyCounts = HashMultiset.create(keys); - SingleTableUpdatesCollector collector = new SingleTableUpdatesCollector(metadata, updatedColumns, perPartitionKeyCounts); - addUpdates(collector, keys, state, options, local, timestamp, nowInSeconds, requestTime); - return collector.toMutations(state); + if(keys.size() == 1) + { + SingleTableSinglePartitionUpdatesCollector collector = new SingleTableSinglePartitionUpdatesCollector(metadata, updatedColumns); + addUpdates(collector, keys, state, options, local, timestamp, nowInSeconds, requestTime); + return collector.toMutations(state); + } else + { + HashMultiset<ByteBuffer> perPartitionKeyCounts = HashMultiset.create(keys); + SingleTableUpdatesCollector collector = new SingleTableUpdatesCollector(metadata, updatedColumns, perPartitionKeyCounts); + addUpdates(collector, keys, state, options, local, timestamp, nowInSeconds, requestTime); + return collector.toMutations(state); + } } final void addUpdates(UpdatesCollector collector, @@ -807,8 +830,12 @@ final void addUpdates(UpdatesCollector collector, PartitionUpdate.Builder updateBuilder = collector.getPartitionUpdateBuilder(metadata(), dk, options.getConsistency()); - for (Slice slice : slices) - addUpdateForKey(updateBuilder, slice, params); + if (slices == Slices.ALL) // to avoid Slices iterator allocation for a common case + addUpdateForKey(updateBuilder, Slice.ALL, params); + else + for (Slice slice : slices) + addUpdateForKey(updateBuilder, slice, params); + } } else diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java index 1762b776a786..209ba88f5260 100644 --- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java @@ -27,6 +27,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; @@ -176,9 +177,9 @@ public SelectStatement(TableMetadata table, } @Override - public List<ColumnSpecification> getBindVariables() + public ImmutableList<ColumnSpecification> getBindVariables() { - return bindVariables.getBindVariables(); + return bindVariables.getImmutableBindVariables(); } @Override diff --git a/src/java/org/apache/cassandra/cql3/statements/SingleTableSinglePartitionUpdatesCollector.java b/src/java/org/apache/cassandra/cql3/statements/SingleTableSinglePartitionUpdatesCollector.java new file mode 100644 index 000000000000..c650ef0370ea --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/statements/SingleTableSinglePartitionUpdatesCollector.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.cql3.statements; + +import java.util.Collections; +import java.util.List; + +import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.db.CounterMutation; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.IMutation; +import org.apache.cassandra.db.Mutation; +import org.apache.cassandra.db.RegularAndStaticColumns; +import org.apache.cassandra.db.commitlog.CommitLogSegment; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.virtual.VirtualMutation; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.ClientState; + +/** + * Utility class to collect updates. + */ +final class SingleTableSinglePartitionUpdatesCollector implements UpdatesCollector +{ + /** + * the table to be updated + */ + private final TableMetadata metadata; + + /** + * the columns to update + */ + private final RegularAndStaticColumns updatedColumns; + /** + * the partition update builders per key + */ + private PartitionUpdate.Builder builder; + + /** + * if it is a counter table, we will set this + */ + private ConsistencyLevel counterConsistencyLevel = null; + + SingleTableSinglePartitionUpdatesCollector(TableMetadata metadata, RegularAndStaticColumns updatedColumns) + { + this.metadata = metadata; + this.updatedColumns = updatedColumns; + } + + public PartitionUpdate.Builder getPartitionUpdateBuilder(TableMetadata metadata, DecoratedKey dk, ConsistencyLevel consistency) + { + if (metadata.isCounter()) + counterConsistencyLevel = consistency; + if (builder == null) + { + builder = new PartitionUpdate.Builder(metadata, dk, updatedColumns, 1); + } + return builder; + } + + /** + * Returns a collection containing all the mutations. + */ + @Override + public List<IMutation> toMutations(ClientState state) + { + // it is possible that a modification statement does not create any mutations + // for example: DELETE FROM some_table WHERE part_key = 1 AND clust_key < 3 AND clust_key > 5 + if (builder == null) + return Collections.emptyList(); + return Collections.singletonList(createMutation(state, builder)); + } + + private IMutation createMutation(ClientState state, PartitionUpdate.Builder builder) + { + IMutation mutation; + + if (metadata.isVirtual()) + mutation = new VirtualMutation(builder.build()); + else if (metadata.isCounter()) + mutation = new CounterMutation(new Mutation(builder.build()), counterConsistencyLevel); + else + mutation = new Mutation(builder.build()); + + mutation.validateIndexedColumns(state); + mutation.validateSize(MessagingService.current_version, CommitLogSegment.ENTRY_OVERHEAD_SIZE); + return mutation; + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/cql3/statements/SingleTableUpdatesCollector.java b/src/java/org/apache/cassandra/cql3/statements/SingleTableUpdatesCollector.java index 5ff299eb88d4..2da6b8918080 100644 --- a/src/java/org/apache/cassandra/cql3/statements/SingleTableUpdatesCollector.java +++ b/src/java/org/apache/cassandra/cql3/statements/SingleTableUpdatesCollector.java @@ -19,6 +19,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -96,23 +97,34 @@ public PartitionUpdate.Builder getPartitionUpdateBuilder(TableMetadata metadata, @Override public List<IMutation> toMutations(ClientState state) { + if (puBuilders.size() == 1) + { + PartitionUpdate.Builder builder = puBuilders.values().iterator().next(); + return Collections.singletonList(createMutation(state, builder)); + } List<IMutation> ms = new ArrayList<>(puBuilders.size()); for (PartitionUpdate.Builder builder : puBuilders.values()) { - IMutation mutation; - - if (metadata.isVirtual()) - mutation = new VirtualMutation(builder.build()); - else if (metadata.isCounter()) - mutation = new CounterMutation(new Mutation(builder.build()), counterConsistencyLevel); - else - mutation = new Mutation(builder.build()); - - mutation.validateIndexedColumns(state); - mutation.validateSize(MessagingService.current_version, CommitLogSegment.ENTRY_OVERHEAD_SIZE); + IMutation mutation = createMutation(state, builder); ms.add(mutation); } return ms; } + + private IMutation createMutation(ClientState state, PartitionUpdate.Builder builder) + { + IMutation mutation; + + if (metadata.isVirtual()) + mutation = new VirtualMutation(builder.build()); + else if (metadata.isCounter()) + mutation = new CounterMutation(new Mutation(builder.build()), counterConsistencyLevel); + else + mutation = new Mutation(builder.build()); + + mutation.validateIndexedColumns(state); + mutation.validateSize(MessagingService.current_version, CommitLogSegment.ENTRY_OVERHEAD_SIZE); + return mutation; + } } diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java index af651570bb8f..f731139e442f 100644 --- a/src/java/org/apache/cassandra/db/Keyspace.java +++ b/src/java/org/apache/cassandra/db/Keyspace.java @@ -453,11 +453,11 @@ private Future<?> applyInternal(final Mutation mutation, Lock[] locks = null; - boolean requiresViewUpdate = updateIndexes && viewManager.updatesAffectView(Collections.singleton(mutation), false); + boolean requiresViewUpdate = updateIndexes && viewManager.updatesAffectView(mutation, false); if (requiresViewUpdate) { - mutation.viewLockAcquireStart.compareAndSet(0L, currentTimeMillis()); + Mutation.viewLockAcquireStartUpdater.compareAndSet(mutation, 0L, currentTimeMillis()); // the order of lock acquisition doesn't matter (from a deadlock perspective) because we only use tryLock() Collection<TableId> tableIds = mutation.getTableIds(); @@ -534,7 +534,7 @@ else if (isDeferrable) } } - long acquireTime = currentTimeMillis() - mutation.viewLockAcquireStart.get(); + long acquireTime = currentTimeMillis() - Mutation.viewLockAcquireStartUpdater.get(mutation); // Metrics are only collected for droppable write operations // Bulk non-droppable operations (e.g. commitlog replay, hint delivery) are not measured if (isDroppable) @@ -553,10 +553,11 @@ else if (isDeferrable) logger.error("Attempting to mutate non-existant table {} ({}.{})", upd.metadata().id, upd.metadata().keyspace, upd.metadata().name); continue; } - AtomicLong baseComplete = new AtomicLong(Long.MAX_VALUE); + AtomicLong baseComplete = null; if (requiresViewUpdate) { + baseComplete = new AtomicLong(Long.MAX_VALUE); try { Tracing.trace("Creating materialized view mutations from base table replica"); diff --git a/src/java/org/apache/cassandra/db/Mutation.java b/src/java/org/apache/cassandra/db/Mutation.java index 384f2b497286..0861bb64c41f 100644 --- a/src/java/org/apache/cassandra/db/Mutation.java +++ b/src/java/org/apache/cassandra/db/Mutation.java @@ -20,7 +20,7 @@ import java.io.IOException; import java.util.*; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicLongFieldUpdater; import java.util.function.Supplier; import com.google.common.base.Preconditions; @@ -68,7 +68,10 @@ public class Mutation implements IMutation, Supplier<Mutation> // Time at which this mutation or the builder that built it was instantiated final long approxCreatedAtNanos; // keep track of when mutation has started waiting for a MV partition lock - final AtomicLong viewLockAcquireStart = new AtomicLong(0); + + final static AtomicLongFieldUpdater<Mutation> viewLockAcquireStartUpdater = + AtomicLongFieldUpdater.newUpdater(Mutation.class, "viewLockAcquireStart"); + volatile long viewLockAcquireStart; private final boolean cdcEnabled; @@ -458,7 +461,7 @@ private Serialization serialization(Mutation mutation, int version) try (DataOutputBuffer dob = DataOutputBuffer.scratchBuffer.get()) { serializeInternal(PartitionUpdate.serializer, mutation, dob, version); - serialization = new CachedSerialization(dob.toByteArray()); + serialization = new CachedSerialization(dob.unsafeToByteArray()); } catch (IOException e) { @@ -521,7 +524,7 @@ public Mutation deserialize(DataInputPlus in, int version, DeserializationHelper //Only cache serializations that don't hit the limit if (!teeIn.isLimitReached()) - m.cachedSerializations[MessagingService.getVersionOrdinal(version)] = new CachedSerialization(dob.toByteArray()); + m.cachedSerializations[MessagingService.getVersionOrdinal(version)] = new CachedSerialization(dob.unsafeToByteArray()); return m; } diff --git a/src/java/org/apache/cassandra/db/RegularAndStaticColumns.java b/src/java/org/apache/cassandra/db/RegularAndStaticColumns.java index b6da183d013f..4daea2f4f7b4 100644 --- a/src/java/org/apache/cassandra/db/RegularAndStaticColumns.java +++ b/src/java/org/apache/cassandra/db/RegularAndStaticColumns.java @@ -153,6 +153,8 @@ public static class Builder private BTree.Builder<ColumnMetadata> regularColumns; private BTree.Builder<ColumnMetadata> staticColumns; + private RegularAndStaticColumns lastAddedColumns; + public Builder add(ColumnMetadata c) { if (c.isStatic()) @@ -180,18 +182,24 @@ public Builder addAll(Iterable<ColumnMetadata> columns) public Builder addAll(RegularAndStaticColumns columns) { + // for batch statements it is a frequent case when we have the same columns in each inner prepared statement + // we use == instead of uquals to make the optimization check cheap + if (lastAddedColumns != null && lastAddedColumns == columns) { + return this; + } if (regularColumns == null && !columns.regulars.isEmpty()) regularColumns = BTree.builder(naturalOrder()); - for (ColumnMetadata c : columns.regulars) - regularColumns.add(c); + if (!columns.regulars.isEmpty()) + regularColumns.addAll(columns.regulars); if (staticColumns == null && !columns.statics.isEmpty()) staticColumns = BTree.builder(naturalOrder()); - for (ColumnMetadata c : columns.statics) - staticColumns.add(c); + if (!columns.statics.isEmpty()) + staticColumns.addAll(columns.statics); + lastAddedColumns = columns; return this; } diff --git a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java index 90fc9f3a1126..ae043039e25e 100644 --- a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java +++ b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java @@ -180,6 +180,15 @@ public static ColumnFilter all(TableMetadata metadata) return new WildCardColumnFilter(metadata.regularAndStaticColumns()); } + /** + * A filter for a PartitionUpdate entity + * which we've just constructed and there no a real need to filter it + */ + public static ColumnFilter all(RegularAndStaticColumns columns) + { + return new WildCardColumnFilter(columns); + } + /** * A filter that only fetches/queries the provided columns. * <p> diff --git a/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java b/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java index ff5d0f9035ce..00f26451c1a3 100644 --- a/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java +++ b/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java @@ -223,6 +223,18 @@ public static PartitionUpdate fromIterator(RowIterator iterator, ColumnFilter fi return new PartitionUpdate(iterator.metadata(), iterator.metadata().epoch, iterator.partitionKey(), holder, deletionInfo, false); } + /** + * An override of default AbstractBTreePartition iterator + * It is added as a performance optimization to avoid full-functional filtering + * using org.apache.cassandra.db.Columns.inOrderInclusionTester() predicate + * when we iterate over row within a PartitionUpdate + */ + @Override + public UnfilteredRowIterator unfilteredIterator() + { + return unfilteredIterator(ColumnFilter.SelectionColumnFilter.all(columns()), Slices.ALL, false); + } + public PartitionUpdate withOnlyPresentColumns() { @@ -884,7 +896,11 @@ public static class Builder private final MutableDeletionInfo deletionInfo; private final boolean canHaveShadowedData; private Object[] tree = BTree.empty(); - private final BTree.Builder<Row> rowBuilder; + + private Row firstRow; + private BTree.Builder<Row> rowBuilder; + + private final int initialRowCapacity; private Row staticRow = Rows.EMPTY_STATIC_ROW; private final RegularAndStaticColumns columns; private boolean isBuilt = false; @@ -920,7 +936,7 @@ private Builder(TableMetadata metadata, this.metadata = metadata; this.key = key; this.columns = columns; - this.rowBuilder = rowBuilder(initialRowCapacity); + this.initialRowCapacity = initialRowCapacity; this.canHaveShadowedData = canHaveShadowedData; this.deletionInfo = deletionInfo.mutableCopy(); this.staticRow = staticRow; @@ -963,19 +979,25 @@ public void add(Row row) if (row.isStatic()) { - // this assert is expensive, and possibly of limited value; we should consider removing it - // or introducing a new class of assertions for test purposes - assert columns().statics.containsAll(row.columns()) : columns().statics + " is not superset of " + row.columns(); staticRow = staticRow.isEmpty() ? row : Rows.merge(staticRow, row); } else { - // this assert is expensive, and possibly of limited value; we should consider removing it - // or introducing a new class of assertions for test purposes - assert columns().regulars.containsAll(row.columns()) : columns().regulars + " is not superset of " + row.columns(); - rowBuilder.add(row); + if (firstRow == null) + { + firstRow = row; + } + else + { + if (rowBuilder == null) + { + rowBuilder = rowBuilder(initialRowCapacity); + rowBuilder.add(firstRow); + } + rowBuilder.add(row); + } } } @@ -999,13 +1021,22 @@ public TableMetadata metadata() return metadata; } + private static final UpdateFunction<Row, Row> ROWS_MERGE_FUNCTION = UpdateFunction.Simple.of(Rows::merge); + public PartitionUpdate build() { // assert that we are not calling build() several times assert !isBuilt : "A PartitionUpdate.Builder should only get built once"; - Object[] add = rowBuilder.build(); - Object[] merged = BTree.<Row, Row, Row>update(tree, add, metadata.comparator, - UpdateFunction.Simple.of(Rows::merge)); + Object[] add; + if (rowBuilder == null) + { + add = firstRow != null ? BTree.singleton(firstRow) : BTree.empty(); + } + else + { + add = rowBuilder.build(); + } + Object[] merged = BTree.<Row, Row, Row>update(tree, add, metadata.comparator, ROWS_MERGE_FUNCTION); EncodingStats newStats = EncodingStats.Collector.collect(staticRow, BTree.iterator(merged), deletionInfo); diff --git a/src/java/org/apache/cassandra/db/rows/BTreeRow.java b/src/java/org/apache/cassandra/db/rows/BTreeRow.java index 075a4f67fe6d..fe44fbde536b 100644 --- a/src/java/org/apache/cassandra/db/rows/BTreeRow.java +++ b/src/java/org/apache/cassandra/db/rows/BTreeRow.java @@ -58,6 +58,7 @@ import org.apache.cassandra.utils.btree.BTree; import org.apache.cassandra.utils.btree.BTreeSearchIterator; import org.apache.cassandra.utils.btree.UpdateFunction; +import org.apache.cassandra.utils.caching.TinyThreadLocalPool; import org.apache.cassandra.utils.memory.Cloner; /** @@ -560,6 +561,17 @@ public static Row.Builder unsortedBuilder() return new Builder(false); } + private static final TinyThreadLocalPool<Builder> POOL = new TinyThreadLocalPool<>(); + + public static Row.Builder pooledUnsortedBuilder() { + TinyThreadLocalPool.TinyPool<Builder> pool = POOL.get(); + Builder builder = pool.poll(); + if (builder == null) + builder = new Builder(false); + builder.pool = pool; + return builder; + } + // This is only used by PartitionUpdate.CounterMark but other uses should be avoided as much as possible as it breaks our general // assumption that Row objects are immutable. This method should go away post-#6506 in particular. // This method is in particular not exposed by the Row API on purpose. @@ -818,6 +830,8 @@ public ColumnData resolve(Object[] cells, int lb, int ub) // For complex column at index i of 'columns', we store at complexDeletions[i] its complex deletion. + private TinyThreadLocalPool.TinyPool<Builder> pool; + protected Builder(boolean isSorted) { cells_ = null; @@ -873,6 +887,11 @@ protected void reset() this.deletion = Deletion.LIVE; this.cells_.reuse(); this.hasComplex = false; + if (pool != null) + { + pool.offer(this); + pool = null; + } } public void addPrimaryKeyLivenessInfo(LivenessInfo info) diff --git a/src/java/org/apache/cassandra/db/rows/EncodingStats.java b/src/java/org/apache/cassandra/db/rows/EncodingStats.java index d0f788ae5ae5..0acc67815b04 100644 --- a/src/java/org/apache/cassandra/db/rows/EncodingStats.java +++ b/src/java/org/apache/cassandra/db/rows/EncodingStats.java @@ -109,6 +109,13 @@ public EncodingStats mergeWith(EncodingStats that) ? that.minTTL : (that.minTTL == TTL_EPOCH ? this.minTTL : Math.min(this.minTTL, that.minTTL)); + // EncodingStats is immutable, so if the result feilds are the same as in the current object we can avoid new object creation + // usually we merge an older object with a newer one and timestamp usually grows, so chances to reuse the object are high + if (this.minTimestamp == minTimestamp + && this.minLocalDeletionTime == minDelTime + && this.minTTL == minTTL) { + return this; + } return new EncodingStats(minTimestamp, minDelTime, minTTL); } diff --git a/src/java/org/apache/cassandra/db/view/View.java b/src/java/org/apache/cassandra/db/view/View.java index 30bad17b3460..e926edb3a970 100644 --- a/src/java/org/apache/cassandra/db/view/View.java +++ b/src/java/org/apache/cassandra/db/view/View.java @@ -238,6 +238,8 @@ public static TableMetadataRef findBaseTable(String keyspace, String viewName) public static Iterable<ViewMetadata> findAll(String keyspace, String baseTable) { KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace); + if (ksm.views.isEmpty()) // memory optimization, to avoid a capturing lambda allocation + return Collections.emptyList(); return Iterables.filter(ksm.views, view -> view.baseTableName.equals(baseTable)); } diff --git a/src/java/org/apache/cassandra/db/view/ViewManager.java b/src/java/org/apache/cassandra/db/view/ViewManager.java index fe0f8236f455..06fdcb4db53b 100644 --- a/src/java/org/apache/cassandra/db/view/ViewManager.java +++ b/src/java/org/apache/cassandra/db/view/ViewManager.java @@ -68,11 +68,25 @@ public ViewManager(Keyspace keyspace) this.keyspace = keyspace; } + public boolean updatesAffectView(IMutation mutation, boolean coordinatorBatchlog) + { + if (!enableCoordinatorBatchlog && coordinatorBatchlog) + return false; + + if (viewsByName.isEmpty()) + return false; + + return updatesAffectView(Collections.singleton(mutation), coordinatorBatchlog); + } + public boolean updatesAffectView(Collection<? extends IMutation> mutations, boolean coordinatorBatchlog) { if (!enableCoordinatorBatchlog && coordinatorBatchlog) return false; + if (viewsByName.isEmpty()) + return false; + ClusterMetadata metadata = ClusterMetadata.currentNullable(); for (IMutation mutation : mutations) { @@ -83,7 +97,8 @@ public boolean updatesAffectView(Collection<? extends IMutation> mutations, bool if (coordinatorBatchlog && keyspace.getReplicationStrategy().getReplicationFactor().allReplicas == 1) continue; - if (!forTable(update.metadata()).updatedViews(update, metadata).isEmpty()) + TableViews tableViews = forTable(update.metadata()); + if (tableViews.hasViews() && !tableViews.updatedViews(update, metadata).isEmpty()) return true; } } diff --git a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java index c0123933b8fb..791293fbb951 100644 --- a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java +++ b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java @@ -1326,6 +1326,8 @@ public <T extends Index> Optional<T> getBestIndexFor(RowFilter.Expression expres @Override public void validate(PartitionUpdate update, ClientState state) throws InvalidRequestException { + if (indexes.isEmpty()) + return; for (Index index : indexes.values()) index.validate(update, state); } diff --git a/src/java/org/apache/cassandra/io/sstable/format/SortedTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/SortedTableWriter.java index 4fca9ca1816a..6fba07ba1e3d 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SortedTableWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SortedTableWriter.java @@ -259,7 +259,8 @@ private AbstractRowIndexEntry endPartition(DecoratedKey key, DeletionTime partit protected void onStartPartition(DecoratedKey key) { - notifyObservers(o -> o.startPartition(key, partitionWriter.getInitialPosition(), partitionWriter.getInitialPosition())); + if (hasObservers()) + notifyObservers(o -> o.startPartition(key, partitionWriter.getInitialPosition(), partitionWriter.getInitialPosition())); } protected void onStaticRow(Row row) @@ -269,22 +270,29 @@ protected void onStaticRow(Row row) protected void onRow(Row row) { - notifyObservers(o -> o.nextUnfilteredCluster(row)); + if (hasObservers()) + notifyObservers(o -> o.nextUnfilteredCluster(row)); } protected void onRangeTombstoneMarker(RangeTombstoneMarker marker) { - notifyObservers(o -> o.nextUnfilteredCluster(marker)); + if (hasObservers()) + notifyObservers(o -> o.nextUnfilteredCluster(marker)); } protected abstract AbstractRowIndexEntry createRowIndexEntry(DecoratedKey key, DeletionTime partitionLevelDeletion, long finishResult) throws IOException; protected final void notifyObservers(Consumer<SSTableFlushObserver> action) { - if (observers != null && !observers.isEmpty()) + if (hasObservers()) observers.forEach(action); } + private boolean hasObservers() + { + return observers != null && !observers.isEmpty(); + } + @Override public void mark() { diff --git a/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java b/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java index 3e157b6b9a01..3cb5db0f00c6 100644 --- a/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java +++ b/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java @@ -290,6 +290,18 @@ public byte[] toByteArray() return result; } + /** + * If the calling logic knows that no new calls to this object will happen after calling this + * method, then this method can avoid the ByteBuffer copying done in {@link #buffer()}. + */ + public byte[] unsafeToByteArray() + { + ByteBuffer buffer = unsafeGetBufferAndFlip(); + byte[] result = new byte[buffer.remaining()]; + buffer.get(result); + return result; + } + public String asString() { try diff --git a/src/java/org/apache/cassandra/locator/ReplicaLayout.java b/src/java/org/apache/cassandra/locator/ReplicaLayout.java index 751737fafb13..f0069f2555cc 100644 --- a/src/java/org/apache/cassandra/locator/ReplicaLayout.java +++ b/src/java/org/apache/cassandra/locator/ReplicaLayout.java @@ -301,6 +301,8 @@ public static ReplicaLayout.ForTokenWrite forTokenWrite(AbstractReplicationStrat */ static <E extends Endpoints<E>> boolean haveWriteConflicts(E natural, E pending) { + if (pending.isEmpty()) + return false; Set<InetAddressAndPort> naturalEndpoints = natural.endpoints(); for (InetAddressAndPort pendingEndpoint : pending.endpoints()) { diff --git a/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java b/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java index 2b7342d5715c..343bac1c4f80 100644 --- a/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java +++ b/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java @@ -17,6 +17,7 @@ */ package org.apache.cassandra.service; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -77,7 +78,7 @@ public abstract class AbstractWriteResponseHandler<T> implements RequestCallback private static final AtomicIntegerFieldUpdater<AbstractWriteResponseHandler> failuresUpdater = AtomicIntegerFieldUpdater.newUpdater(AbstractWriteResponseHandler.class, "failures"); private volatile int failures = 0; - private final Map<InetAddressAndPort, RequestFailureReason> failureReasonByEndpoint; + private volatile Map<InetAddressAndPort, RequestFailureReason> failureReasonByEndpoint; private final Dispatcher.RequestTime requestTime; private @Nullable final Supplier<Mutation> hintOnFailure; @@ -106,7 +107,6 @@ protected AbstractWriteResponseHandler(ForWrite replicaPlan, Runnable callback, this.callback = callback; this.writeType = writeType; this.hintOnFailure = hintOnFailure; - this.failureReasonByEndpoint = new ConcurrentHashMap<>(); this.requestTime = requestTime; } @@ -129,12 +129,12 @@ public void get() throws WriteTimeoutException, WriteFailureException if (blockFor() + failures > candidateReplicaCount()) { - if (RequestCallback.isTimeout(this.failureReasonByEndpoint.keySet().stream() + if (RequestCallback.isTimeout(this.getFailureReasonByEndpointMap().keySet().stream() .filter(this::waitingFor) // DatacenterWriteResponseHandler filters errors from remote DCs - .collect(Collectors.toMap(Function.identity(), this.failureReasonByEndpoint::get)))) + .collect(Collectors.toMap(Function.identity(), this.getFailureReasonByEndpointMap()::get)))) throwTimeout(); - throw new WriteFailureException(replicaPlan.consistencyLevel(), ackCount(), blockFor(), writeType, this.failureReasonByEndpoint); + throw new WriteFailureException(replicaPlan.consistencyLevel(), ackCount(), blockFor(), writeType, this.getFailureReasonByEndpointMap()); } if (replicaPlan.stillAppliesTo(ClusterMetadata.current())) @@ -303,6 +303,12 @@ public void onFailure(InetAddressAndPort from, RequestFailureReason failureReaso ? failuresUpdater.incrementAndGet(this) : failures; + if (failureReasonByEndpoint == null) + synchronized (this) + { + if (failureReasonByEndpoint == null) + failureReasonByEndpoint = new ConcurrentHashMap<>(); + } failureReasonByEndpoint.put(from, failureReason); logFailureOrTimeoutToIdealCLDelegate(); @@ -377,4 +383,9 @@ public void maybeTryAdditionalReplicas(IMutation mutation, WritePerformer writeP throw new UncheckedInterruptedException(e); } } + + private Map<InetAddressAndPort, RequestFailureReason> getFailureReasonByEndpointMap() + { + return failureReasonByEndpoint != null ? failureReasonByEndpoint : Collections.emptyMap(); + } } diff --git a/src/java/org/apache/cassandra/service/ClientWarn.java b/src/java/org/apache/cassandra/service/ClientWarn.java index 13cb21d6b270..de8e84da1379 100644 --- a/src/java/org/apache/cassandra/service/ClientWarn.java +++ b/src/java/org/apache/cassandra/service/ClientWarn.java @@ -77,7 +77,7 @@ public void resumeCapture() public List<String> getWarnings() { State state = get(); - if (state == null || state.warnings.isEmpty()) + if (state == null || state.warnings == null || state.warnings.isEmpty()) return null; return state.warnings; } @@ -92,10 +92,16 @@ public static class State private boolean collecting = true; // This must be a thread-safe list. Even though it's wrapped in a ThreadLocal, it's propagated to each thread // from shared state, so multiple threads can reference the same State. - private final List<String> warnings = new CopyOnWriteArrayList<>(); + private volatile List<String> warnings; private void add(String warning) { + if (warnings == null) + synchronized (this) { + if (warnings == null) { + warnings = new CopyOnWriteArrayList<>(); + } + } if (collecting && warnings.size() < FBUtilities.MAX_UNSIGNED_SHORT) warnings.add(maybeTruncate(warning)); } diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java index 31e60d68e60a..f7219a4b580d 100644 --- a/src/java/org/apache/cassandra/service/StorageProxy.java +++ b/src/java/org/apache/cassandra/service/StorageProxy.java @@ -880,24 +880,25 @@ public static void mutate(List<? extends IMutation> mutations, ConsistencyLevel Tracing.trace("Determining replicas for mutation"); final String localDataCenter = DatabaseDescriptor.getLocator().local().datacenter; - List<AbstractWriteResponseHandler<IMutation>> responseHandlers = new ArrayList<>(mutations.size()); + AbstractWriteResponseHandler<IMutation>[] responseHandlers = new AbstractWriteResponseHandler[mutations.size()]; WriteType plainWriteType = mutations.size() <= 1 ? WriteType.SIMPLE : WriteType.UNLOGGED_BATCH; try { + int j = 0; for (IMutation mutation : mutations) { if (mutation instanceof CounterMutation) - responseHandlers.add(mutateCounter((CounterMutation)mutation, localDataCenter, requestTime)); + responseHandlers[j++] = mutateCounter((CounterMutation)mutation, localDataCenter, requestTime); else - responseHandlers.add(performWrite(mutation, consistencyLevel, localDataCenter, standardWritePerformer, null, plainWriteType, requestTime)); + responseHandlers[j++] = performWrite(mutation, consistencyLevel, localDataCenter, standardWritePerformer, null, plainWriteType, requestTime); } // upgrade to full quorum any failed cheap quorums for (int i = 0 ; i < mutations.size() ; ++i) { if (!(mutations.get(i) instanceof CounterMutation)) // at the moment, only non-counter writes support cheap quorums - responseHandlers.get(i).maybeTryAdditionalReplicas(mutations.get(i), standardWritePerformer, localDataCenter); + responseHandlers[i].maybeTryAdditionalReplicas(mutations.get(i), standardWritePerformer, localDataCenter); } // wait for writes. throws TimeoutException if necessary @@ -1275,14 +1276,28 @@ private static void updateCoordinatorWriteLatencyTableMetric(Collection<? extend { //We could potentially pass a callback into performWrite. And add callback provision for mutateCounter or mutateAtomically (sendToHintedEndPoints) //However, Trade off between write metric per CF accuracy vs performance hit due to callbacks. Similar issue exists with CoordinatorReadLatency metric. - Set<ColumnFamilyStore> uniqueColumnFamilyStores = new HashSet<>(); + Set<ColumnFamilyStore> uniqueColumnFamilyStores = null; + // very frequently we update just a single table + // so an allocation of uniqueColumnFamilyStores set can be avoided + ColumnFamilyStore firstColumnFamilyStore = null; for (IMutation mutation : mutations) { + Keyspace keyspace = Keyspace.open(mutation.getKeyspaceName()); for (TableId tableId : mutation.getTableIds()) { - ColumnFamilyStore store = Keyspace.open(mutation.getKeyspaceName()).getColumnFamilyStore(tableId); - if (uniqueColumnFamilyStores.add(store)) + ColumnFamilyStore store = keyspace.getColumnFamilyStore(tableId); + if (firstColumnFamilyStore == null) + { store.metric.coordinatorWriteLatency.update(latency, NANOSECONDS); + firstColumnFamilyStore = store; + } + else if (!firstColumnFamilyStore.equals(store)) + { + if (uniqueColumnFamilyStores == null) + uniqueColumnFamilyStores = new HashSet<>(); + if (uniqueColumnFamilyStores.add(store)) + store.metric.coordinatorWriteLatency.update(latency, NANOSECONDS); + } } } } diff --git a/src/java/org/apache/cassandra/transport/CQLMessageHandler.java b/src/java/org/apache/cassandra/transport/CQLMessageHandler.java index 87d809c6c0a4..65e67925adcf 100644 --- a/src/java/org/apache/cassandra/transport/CQLMessageHandler.java +++ b/src/java/org/apache/cassandra/transport/CQLMessageHandler.java @@ -75,7 +75,7 @@ * has exceeded the maximum number of allowed permits. The choices are to either pause reads from the incoming socket * and allow TCP backpressure to do the work, or to throw an explict exception and rely on the client to back off. */ -public class CQLMessageHandler<M extends Message> extends AbstractMessageHandler +public class CQLMessageHandler<M extends Message> extends AbstractMessageHandler implements Flusher.OnFlushCleanup<Envelope> { private static final Logger logger = LoggerFactory.getLogger(CQLMessageHandler.class); private static final NoSpamLogger noSpamLogger = NoSpamLogger.getLogger(logger, 1L, TimeUnit.SECONDS); @@ -374,7 +374,7 @@ private Envelope composeRequest(Envelope.Header header, ShareableBytes bytes) ByteBuffer buf = bytes.get(); int idx = buf.position() + Envelope.Header.LENGTH; final int end = idx + Ints.checkedCast(header.bodySizeInBytes); - ByteBuf body = Unpooled.wrappedBuffer(buf.slice()); + ByteBuf body = Unpooled.wrappedBuffer(buf); // buf.slice() is not needed: Unpooled.wrappedBuffer does ByteBuffer.slice inside body.readerIndex(Envelope.Header.LENGTH); body.retain(); buf.position(end); @@ -492,10 +492,11 @@ private Framed toFlushItem(Channel channel, Message.Request request, Message.Res responseFrame, request.getSource(), payloadAllocator, - this::release); + this); } - private void release(Flusher.FlushItem<Envelope> flushItem) + @Override + public void cleanup(Flusher.FlushItem<Envelope> flushItem) { release(flushItem.request.header); flushItem.request.release(); diff --git a/src/java/org/apache/cassandra/transport/Envelope.java b/src/java/org/apache/cassandra/transport/Envelope.java index 99c6e135afe6..b425abc63ed2 100644 --- a/src/java/org/apache/cassandra/transport/Envelope.java +++ b/src/java/org/apache/cassandra/transport/Envelope.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.nio.ByteBuffer; -import java.util.EnumSet; import java.util.List; import com.google.common.annotations.VisibleForTesting; @@ -79,7 +78,7 @@ public Envelope clone() return new Envelope(header, Unpooled.wrappedBuffer(ByteBufferUtil.clone(body.nioBuffer()))); } - public static Envelope create(Message.Type type, int streamId, ProtocolVersion version, EnumSet<Header.Flag> flags, ByteBuf body) + public static Envelope create(Message.Type type, int streamId, ProtocolVersion version, int flags, ByteBuf body) { Header header = new Header(version, flags, streamId, type, body.readableBytes()); return new Envelope(header, body); @@ -92,7 +91,7 @@ public ByteBuf encodeHeader() Message.Type type = header.type; buf.writeByte(type.direction.addToVersion(header.version.asInt())); - buf.writeByte(Header.Flag.serialize(header.flags)); + buf.writeByte(header.flags); // Continue to support writing pre-v3 headers so that we can give proper error messages to drivers that // connect with the v1/v2 protocol. See CASSANDRA-11464. @@ -110,7 +109,7 @@ public ByteBuf encodeHeader() public void encodeHeaderInto(ByteBuffer buf) { buf.put((byte) header.type.direction.addToVersion(header.version.asInt())); - buf.put((byte) Envelope.Header.Flag.serialize(header.flags)); + buf.put((byte) header.flags); if (header.version.isGreaterOrEqualTo(ProtocolVersion.V3)) buf.putShort((short) header.streamId); @@ -125,7 +124,11 @@ public void encodeHeaderInto(ByteBuffer buf) public void encodeInto(ByteBuffer buf) { encodeHeaderInto(buf); - buf.put(body.nioBuffer()); + // an alternative logic for : buf.put(body.nioBuffer()) without ByteBuffer slicing + int originalLimit = buf.limit(); + buf.limit(buf.position() + body.readableBytes()); + body.readBytes(buf); + buf.limit(originalLimit); } public static class Header @@ -136,12 +139,12 @@ public static class Header public static final int BODY_LENGTH_SIZE = 4; public final ProtocolVersion version; - public final EnumSet<Flag> flags; + public int flags; public final int streamId; public final Message.Type type; public final long bodySizeInBytes; - private Header(ProtocolVersion version, EnumSet<Flag> flags, int streamId, Message.Type type, long bodySizeInBytes) + private Header(ProtocolVersion version, int flags, int streamId, Message.Type type, long bodySizeInBytes) { this.version = version; this.flags = flags; @@ -150,6 +153,16 @@ private Header(ProtocolVersion version, EnumSet<Flag> flags, int streamId, Messa this.bodySizeInBytes = bodySizeInBytes; } + public void addFlag(Flag flag) + { + this.flags = Flag.add(this.flags, flag); + } + + public boolean hasFlag(Flag flag) + { + return Flag.contains(this.flags, flag); + } + public enum Flag { // The order of that enum matters!! @@ -159,25 +172,27 @@ public enum Flag WARNING, USE_BETA; - private static final Flag[] ALL_VALUES = values(); + private final int mask; - public static EnumSet<Flag> deserialize(int flags) + Flag() { - EnumSet<Flag> set = EnumSet.noneOf(Flag.class); - for (int n = 0; n < ALL_VALUES.length; n++) - { - if ((flags & (1 << n)) != 0) - set.add(ALL_VALUES[n]); - } - return set; + this.mask = 1 << this.ordinal(); } - public static int serialize(EnumSet<Flag> flags) + public static int none() { - int i = 0; - for (Flag flag : flags) - i |= 1 << flag.ordinal(); - return i; + return 0; + } + + public static int add(int flags, Flag flagToAdd) + { + flags |= flagToAdd.mask; + return flags; + } + + public static boolean contains(long flags, Flag flag) + { + return (flags & flag.mask) != 0; } } } @@ -236,15 +251,14 @@ HeaderExtractionResult extractHeader(ByteBuffer buffer) Message.Direction direction = Message.Direction.extractFromVersion(firstByte); Message.Type type; ProtocolVersion version; - EnumSet<Header.Flag> decodedFlags; try { // This throws a protocol exception if the version number is unsupported, // the opcode is unknown or invalid flags are set for the version version = ProtocolVersion.decode(versionNum, DatabaseDescriptor.getNativeTransportAllowOlderProtocols()); - decodedFlags = decodeFlags(version, flags); + validateFlags(version, flags); type = Message.Type.fromOpcode(opcode, direction); - return new HeaderExtractionResult.Success(new Header(version, decodedFlags, streamId, type, bodyLength)); + return new HeaderExtractionResult.Success(new Header(version, flags, streamId, type, bodyLength)); } catch (ProtocolException e) { @@ -372,7 +386,7 @@ Envelope decode(ByteBuf buffer) return null; int flags = buffer.getByte(idx++); - EnumSet<Header.Flag> decodedFlags = decodeFlags(version, flags); + validateFlags(version, flags); int streamId = buffer.getShort(idx); idx += 2; @@ -417,17 +431,14 @@ Envelope decode(ByteBuf buffer) idx += bodyLength; buffer.readerIndex(idx); - return new Envelope(new Header(version, decodedFlags, streamId, type, bodyLength), body); + return new Envelope(new Header(version, flags, streamId, type, bodyLength), body); } - private EnumSet<Header.Flag> decodeFlags(ProtocolVersion version, int flags) + private void validateFlags(ProtocolVersion version, int flags) { - EnumSet<Header.Flag> decodedFlags = Header.Flag.deserialize(flags); - - if (version.isBeta() && !decodedFlags.contains(Header.Flag.USE_BETA)) + if (version.isBeta() && !Header.Flag.contains(flags, Header.Flag.USE_BETA)) throw new ProtocolException(String.format("Beta version of the protocol used (%s), but USE_BETA flag is unset", version), version); - return decodedFlags; } @Override @@ -488,7 +499,7 @@ public void decode(ChannelHandlerContext ctx, Envelope source, List<Object> resu { Connection connection = ctx.channel().attr(Connection.attributeKey).get(); - if (!source.header.flags.contains(Header.Flag.COMPRESSED) || connection == null) + if (!source.header.hasFlag(Header.Flag.COMPRESSED) || connection == null) { results.add(source); return; @@ -529,7 +540,7 @@ public void encode(ChannelHandlerContext ctx, Envelope source, List<Object> resu results.add(source); return; } - source.header.flags.add(Header.Flag.COMPRESSED); + source.header.addFlag(Header.Flag.COMPRESSED); results.add(compressor.compress(source)); } } diff --git a/src/java/org/apache/cassandra/transport/Flusher.java b/src/java/org/apache/cassandra/transport/Flusher.java index 50261de0368a..ebf708158abc 100644 --- a/src/java/org/apache/cassandra/transport/Flusher.java +++ b/src/java/org/apache/cassandra/transport/Flusher.java @@ -23,7 +23,6 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Consumer; import com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; @@ -49,6 +48,9 @@ abstract class Flusher implements Runnable Math.min(BufferPool.NORMAL_CHUNK_SIZE, FrameEncoder.Payload.MAX_SIZE - Math.max(FrameEncoderCrc.HEADER_AND_TRAILER_LENGTH, FrameEncoderLZ4.HEADER_AND_TRAILER_LENGTH)); + interface OnFlushCleanup<T> { + void cleanup(FlushItem<T> item); + } static class FlushItem<T> { enum Kind {FRAMED, UNFRAMED} @@ -57,9 +59,9 @@ enum Kind {FRAMED, UNFRAMED} final Channel channel; final T response; final Envelope request; - final Consumer<FlushItem<T>> tidy; + final OnFlushCleanup<T> tidy; - FlushItem(Kind kind, Channel channel, T response, Envelope request, Consumer<FlushItem<T>> tidy) + FlushItem(Kind kind, Channel channel, T response, Envelope request, OnFlushCleanup<T> tidy) { this.kind = kind; this.channel = channel; @@ -70,7 +72,7 @@ enum Kind {FRAMED, UNFRAMED} void release() { - tidy.accept(this); + tidy.cleanup(this); } static class Framed extends FlushItem<Envelope> @@ -80,7 +82,7 @@ static class Framed extends FlushItem<Envelope> Envelope response, Envelope request, FrameEncoder.PayloadAllocator allocator, - Consumer<FlushItem<Envelope>> tidy) + OnFlushCleanup<Envelope> tidy) { super(Kind.FRAMED, channel, response, request, tidy); this.allocator = allocator; @@ -89,7 +91,7 @@ static class Framed extends FlushItem<Envelope> static class Unframed extends FlushItem<Response> { - Unframed(Channel channel, Response response, Envelope request, Consumer<FlushItem<Response>> tidy) + Unframed(Channel channel, Response response, Envelope request, OnFlushCleanup<Response> tidy) { super(Kind.UNFRAMED, channel, response, request, tidy); } @@ -156,8 +158,14 @@ private void processFramedResponse(FlushItem.Framed flush) } else { - payloads.computeIfAbsent(flush.channel, channel -> new FlushBuffer(channel, flush.allocator, 5)) - .add(flush.response); + FlushBuffer flushBuffer = payloads.get(flush.channel); + if (flushBuffer == null) + { + flushBuffer = new FlushBuffer(flush.channel, flush.allocator, 5); + payloads.put(flushBuffer.channel, flushBuffer); + } + + flushBuffer.add(flush.response); } } @@ -226,8 +234,9 @@ protected boolean processQueue() protected void flushWrittenChannels() { // flush the channels pre-V5 to which messages were written in writeSingleResponse - for (Channel channel : channels) - channel.flush(); + if (!channels.isEmpty()) + for (Channel channel : channels) + channel.flush(); // Framed messages (V5) are grouped by channel, now encode them into payloads, write and flush for (FlushBuffer buffer : payloads.values()) @@ -247,8 +256,11 @@ protected void flushWrittenChannels() // collated into frames, and so their buffers can be released immediately after flushing. // In V4 however, the buffers containing each CQL envelope are emitted from Envelope.Encoder // and so releasing them is handled by Netty internally. - for (FlushItem<?> item : processed) + for (int i = 0; i < processed.size(); i++) + { + FlushItem<?> item = processed.get(i); item.release(); + } payloads.clear(); channels.clear(); @@ -298,8 +310,9 @@ public void finish() int writtenBytes = 0; int messagesToWrite = this.size(); FrameEncoder.Payload sending = allocate(sizeInBytes, messagesToWrite); - for (Envelope f : this) + for (int i = 0; i < this.size(); i++) { + Envelope f = this.get(i); messageSize = envelopeSize(f.header); if (sending.remaining() < messageSize) { diff --git a/src/java/org/apache/cassandra/transport/Message.java b/src/java/org/apache/cassandra/transport/Message.java index ed853c0cbd7c..47d4289b0780 100644 --- a/src/java/org/apache/cassandra/transport/Message.java +++ b/src/java/org/apache/cassandra/transport/Message.java @@ -20,7 +20,6 @@ import java.lang.reflect.Field; import java.lang.reflect.Modifier; import java.nio.ByteBuffer; -import java.util.EnumSet; import java.util.List; import java.util.Map; @@ -35,6 +34,7 @@ import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.transport.messages.*; +import org.apache.cassandra.transport.Envelope.Header.Flag; import org.apache.cassandra.service.QueryState; import org.apache.cassandra.utils.MonotonicClock; import org.apache.cassandra.utils.ReflectionUtils; @@ -328,7 +328,7 @@ public List<String> getWarnings() public Envelope encode(ProtocolVersion version) { - EnumSet<Envelope.Header.Flag> flags = EnumSet.noneOf(Envelope.Header.Flag.class); + int flags = Flag.none(); @SuppressWarnings("unchecked") Codec<Message> codec = (Codec<Message>)this.type.codec; try @@ -366,24 +366,24 @@ public Envelope encode(ProtocolVersion version) if (tracingId != null) { CBUtil.writeUUID(tracingId, body); - flags.add(Envelope.Header.Flag.TRACING); + flags = Flag.add(flags, Flag.TRACING); } if (warnings != null) { CBUtil.writeStringList(warnings, body); - flags.add(Envelope.Header.Flag.WARNING); + flags = Flag.add(flags, Flag.WARNING); } if (customPayload != null) { CBUtil.writeBytesMap(customPayload, body); - flags.add(Envelope.Header.Flag.CUSTOM_PAYLOAD); + flags = Flag.add(flags, Flag.CUSTOM_PAYLOAD); } } else { assert this instanceof Request; if (((Request)this).isTracingRequested()) - flags.add(Envelope.Header.Flag.TRACING); + flags = Flag.add(flags, Flag.TRACING); Map<String, ByteBuffer> payload = getCustomPayload(); if (payload != null) messageSize += CBUtil.sizeOfBytesMap(payload); @@ -391,7 +391,7 @@ public Envelope encode(ProtocolVersion version) if (payload != null) { CBUtil.writeBytesMap(payload, body); - flags.add(Envelope.Header.Flag.CUSTOM_PAYLOAD); + flags = Flag.add(flags, Flag.CUSTOM_PAYLOAD); } } @@ -412,7 +412,7 @@ public Envelope encode(ProtocolVersion version) : forcedProtocolVersion; if (responseVersion.isBeta()) - flags.add(Envelope.Header.Flag.USE_BETA); + flags = Flag.add(flags, Flag.USE_BETA); return Envelope.create(type, getStreamId(), responseVersion, flags, body); } @@ -427,9 +427,9 @@ abstract static class Decoder<M extends Message> static Message decodeMessage(Channel channel, Envelope inbound) { boolean isRequest = inbound.header.type.direction == Direction.REQUEST; - boolean isTracing = inbound.header.flags.contains(Envelope.Header.Flag.TRACING); - boolean isCustomPayload = inbound.header.flags.contains(Envelope.Header.Flag.CUSTOM_PAYLOAD); - boolean hasWarning = inbound.header.flags.contains(Envelope.Header.Flag.WARNING); + boolean isTracing = inbound.header.hasFlag(Flag.TRACING); + boolean isCustomPayload = inbound.header.hasFlag(Flag.CUSTOM_PAYLOAD); + boolean hasWarning = inbound.header.hasFlag(Flag.WARNING); TimeUUID tracingId = isRequest || !isTracing ? null : CBUtil.readTimeUUID(inbound.body); List<String> warnings = isRequest || !hasWarning ? null : CBUtil.readStringList(inbound.body); diff --git a/test/unit/org/apache/cassandra/cql3/QueryOptionsFlagsTest.java b/test/unit/org/apache/cassandra/cql3/QueryOptionsFlagsTest.java new file mode 100644 index 000000000000..58b6f103f7a1 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/QueryOptionsFlagsTest.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3; + +import org.junit.Assert; +import org.junit.Test; + +import org.apache.cassandra.cql3.QueryOptions.Codec.Flag; + +public class QueryOptionsFlagsTest +{ + @Test + public void checkFlagOperations() + { + int flags = Flag.none(); + for (Flag flag : Flag.values()) + { + flags = Flag.add(flags, flag); + Assert.assertTrue(Flag.contains(flags, flag)); + for (int i = flag.ordinal() + 1; i < Flag.values().length; i++) + Assert.assertFalse(Flag.contains(flags, Flag.values()[i])); + } + for (Flag flag : Flag.values()) + { + flags = Flag.remove(flags, flag); + Assert.assertFalse(Flag.contains(flags, flag)); + for (int i = flag.ordinal() + 1; i < Flag.values().length; i++) + Assert.assertTrue(Flag.contains(flags, Flag.values()[i])); + } + + } + + @Test + public void checkFlagEncoding() + { + int flags = Flag.none(); + flags = Flag.add(flags, Flag.VALUES); + flags = Flag.add(flags, Flag.PAGING_STATE); + flags = Flag.add(flags, Flag.TIMESTAMP); + + Assert.assertEquals(flags, 0x0001 | 0x0008 | 0x0020); + } + + @Test + public void checkFlagDecoding() + { + int flags = 0x0001 | 0x0040 | 0x0004 | 0x0100; + Assert.assertTrue(Flag.contains(flags, Flag.VALUES)); + Assert.assertTrue(Flag.contains(flags, Flag.NAMES_FOR_VALUES)); + Assert.assertTrue(Flag.contains(flags, Flag.PAGE_SIZE)); + Assert.assertFalse(Flag.contains(flags, Flag.SKIP_METADATA)); + Assert.assertTrue(Flag.contains(flags, Flag.NOW_IN_SECONDS)); + } +} diff --git a/test/unit/org/apache/cassandra/transport/CQLConnectionTest.java b/test/unit/org/apache/cassandra/transport/CQLConnectionTest.java index e01128a6d7b7..a3c9bfe40743 100644 --- a/test/unit/org/apache/cassandra/transport/CQLConnectionTest.java +++ b/test/unit/org/apache/cassandra/transport/CQLConnectionTest.java @@ -524,7 +524,7 @@ private Envelope randomEnvelope(int streamId, Message.Type type, int minSize, in return Envelope.create(type, streamId, ProtocolVersion.V5, - EnumSet.of(Envelope.Header.Flag.USE_BETA), + Envelope.Header.Flag.add(Envelope.Header.Flag.none(), Envelope.Header.Flag.USE_BETA), Unpooled.wrappedBuffer(bytes)); } diff --git a/test/unit/org/apache/cassandra/transport/EnvelopeHeaderFlagsTest.java b/test/unit/org/apache/cassandra/transport/EnvelopeHeaderFlagsTest.java new file mode 100644 index 000000000000..de31d1bedb40 --- /dev/null +++ b/test/unit/org/apache/cassandra/transport/EnvelopeHeaderFlagsTest.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.transport; + +import org.junit.Assert; +import org.junit.Test; + +import org.apache.cassandra.transport.Envelope.Header.Flag; + +public class EnvelopeHeaderFlagsTest +{ + @Test + public void checkFlagOperations() + { + int flags = Flag.none(); + for (Flag flag : Flag.values()) + { + flags = Flag.add(flags, flag); + Assert.assertTrue(Flag.contains(flags, flag)); + for (int i = flag.ordinal() + 1; i < Flag.values().length; i++) + Assert.assertFalse(Flag.contains(flags, Flag.values()[i])); + } + } + + @Test + public void checkFlagEncoding() + { + int flags = Flag.none(); + flags = Flag.add(flags, Flag.COMPRESSED); + flags = Flag.add(flags, Flag.TRACING); + flags = Flag.add(flags, Flag.USE_BETA); + + Assert.assertEquals(flags, 0x01 | 0x02 | 0x10); + } + + @Test + public void checkFlagDecoding() + { + int flags = 0x02 | 0x08 | 0x10; + Assert.assertFalse(Flag.contains(flags, Flag.COMPRESSED)); + Assert.assertTrue(Flag.contains(flags, Flag.TRACING)); + Assert.assertFalse(Flag.contains(flags, Flag.CUSTOM_PAYLOAD)); + Assert.assertTrue(Flag.contains(flags, Flag.WARNING)); + Assert.assertTrue(Flag.contains(flags, Flag.USE_BETA)); + } +} From 20562d10a0efe96b7afba7815033cdb5195823ae Mon Sep 17 00:00:00 2001 From: Ariel Weisberg <aweisberg@apple.com> Date: Tue, 4 Mar 2025 14:57:23 -0500 Subject: [PATCH 165/225] Improve IntervalTree build throughput patch by Ariel Weisberg; reviewed by Marcus Eriksson for CASSANDRA-19596 --- CHANGES.txt | 1 + .../config/CassandraRelevantProperties.java | 5 +- .../cassandra/db/SizeEstimatesRecorder.java | 8 +- .../db/compaction/CompactionManager.java | 5 +- .../db/lifecycle/LifecycleTransaction.java | 2 +- .../db/lifecycle/SSTableIntervalTree.java | 41 +- .../cassandra/db/lifecycle/Tracker.java | 66 ++- .../apache/cassandra/db/lifecycle/View.java | 9 +- .../db/streaming/CassandraStreamManager.java | 18 +- .../index/sai/disk/SSTableIndex.java | 11 +- .../cassandra/index/sasi/SSTableIndex.java | 9 +- .../io/sstable/format/SSTableReader.java | 36 +- .../org/apache/cassandra/utils/Interval.java | 36 +- .../apache/cassandra/utils/IntervalTree.java | 399 ++++++++----- .../org/apache/cassandra/utils/TimeUUID.java | 10 + .../LeveledCompactionStrategyTest.java | 3 +- .../UnifiedCompactionStrategyTest.java | 5 +- .../cassandra/db/lifecycle/TrackerTest.java | 29 +- .../cassandra/db/lifecycle/ViewTest.java | 5 +- .../cassandra/utils/IntervalTreeTest.java | 539 +++++++++++++----- .../cassandra/utils/OverlapIteratorTest.java | 3 +- 21 files changed, 868 insertions(+), 372 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index b01f18babcdc..6995e07fd21b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.4 + * Improve IntervalTree build throughput (CASSANDRA-19596) * Avoid limit on RFP fetch in the case of an unresolved static row (CASSANDRA-20323) * Include materialized views to the output of DESCRIBE TABLE statements (CASSANDRA-20365) * Heap and GC jvm flags improvements (CASSANDRA-20296) diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java index db45f94b9745..c25b41e5fae2 100644 --- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java +++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java @@ -21,7 +21,6 @@ import java.util.HashSet; import java.util.Set; import java.util.concurrent.TimeUnit; - import javax.annotation.Nullable; import com.google.common.primitives.Ints; @@ -540,7 +539,9 @@ public enum CassandraRelevantProperties * faster. Note that this is disabled for unit tests but if an individual test requires schema to be flushed, it * can be also done manually for that particular case: {@code flush(SchemaConstants.SCHEMA_KEYSPACE_NAME);}. */ TEST_FLUSH_LOCAL_SCHEMA_CHANGES("cassandra.test.flush_local_schema_changes", "true"), + TEST_HARRY_SWITCH_AFTER("cassandra.test.harry.progression.switch-after", "1"), TEST_IGNORE_SIGAR("cassandra.test.ignore_sigar"), + TEST_INTERVAL_TREE_EXPENSIVE_CHECKS("cassandra.test.interval_tree_expensive_checks"), TEST_INVALID_LEGACY_SSTABLE_ROOT("invalid-legacy-sstable-root"), TEST_JVM_DTEST_DISABLE_SSL("cassandra.test.disable_ssl"), TEST_LEGACY_SSTABLE_ROOT("legacy-sstable-root"), @@ -616,7 +617,7 @@ public enum CassandraRelevantProperties prev = next; } } - + CassandraRelevantProperties(String key, String defaultVal) { this.key = key; diff --git a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java index f9233bf904d1..7384a809e2a3 100644 --- a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java +++ b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java @@ -17,9 +17,13 @@ */ package org.apache.cassandra.db; -import java.util.*; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.TimeUnit; +import com.google.common.collect.ImmutableList; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -133,7 +137,7 @@ private static Map<Range<Token>, Pair<Long, Long>> computeSizeEstimates(ColumnFa while (refs == null) { Iterable<SSTableReader> sstables = table.getTracker().getView().select(SSTableSet.CANONICAL); - SSTableIntervalTree tree = SSTableIntervalTree.build(sstables); + SSTableIntervalTree tree = SSTableIntervalTree.buildSSTableIntervalTree(ImmutableList.copyOf(sstables)); Range<PartitionPosition> r = Range.makeRowRange(unwrappedRange); Iterable<SSTableReader> canonicalSSTables = View.sstablesInBounds(r.left, r.right, tree); refs = Refs.tryRef(canonicalSSTables); diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java index 791a8e3c67ca..16079ff8883d 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java @@ -47,6 +47,7 @@ import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Collections2; import com.google.common.collect.ConcurrentHashMultiset; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -122,6 +123,7 @@ import static org.apache.cassandra.concurrent.FutureTask.callable; import static org.apache.cassandra.config.DatabaseDescriptor.getConcurrentCompactors; import static org.apache.cassandra.db.compaction.CompactionManager.CompactionExecutor.compactionThreadGroup; +import static org.apache.cassandra.db.lifecycle.SSTableIntervalTree.buildSSTableIntervalTree; import static org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR; import static org.apache.cassandra.service.ActiveRepairService.UNREPAIRED_SSTABLE; import static org.apache.cassandra.utils.Clock.Global.nanoTime; @@ -1090,8 +1092,7 @@ private static Collection<SSTableReader> sstablesInBounds(ColumnFamilyStore cfs, { final Set<SSTableReader> sstables = new HashSet<>(); Iterable<SSTableReader> liveTables = cfs.getTracker().getView().select(SSTableSet.LIVE); - SSTableIntervalTree tree = SSTableIntervalTree.build(liveTables); - + SSTableIntervalTree tree = buildSSTableIntervalTree(ImmutableList.copyOf(liveTables)); for (Range<Token> tokenRange : tokenRangeCollection) { if (!AbstractBounds.strictlyWrapsAround(tokenRange.left, tokenRange.right)) diff --git a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java index 20e1d92eafd4..383d9b08f26e 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java +++ b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java @@ -161,7 +161,7 @@ public static LifecycleTransaction offline(OperationType operationType, SSTableR /** * construct a Transaction for use in an offline operation */ - public static LifecycleTransaction offline(OperationType operationType, Iterable<SSTableReader> readers) + public static LifecycleTransaction offline(OperationType operationType, Collection<SSTableReader> readers) { // if offline, for simplicity we just use a dummy tracker Tracker dummy = Tracker.newDummyTracker(); diff --git a/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java b/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java index 91005d39dd91..9f28fc5995b9 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java +++ b/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java @@ -20,12 +20,11 @@ */ package org.apache.cassandra.db.lifecycle; -import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.List; -import com.google.common.collect.Iterables; - import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.utils.Interval; @@ -40,21 +39,49 @@ public class SSTableIntervalTree extends IntervalTree<PartitionPosition, SSTable super(intervals); } + private SSTableIntervalTree(Interval<PartitionPosition, SSTableReader>[] minOrder, Interval<PartitionPosition, SSTableReader>[] maxOrder) + { + super(minOrder, maxOrder); + } + + @Override + protected SSTableIntervalTree create(Interval<PartitionPosition, SSTableReader>[] minOrder, Interval<PartitionPosition, SSTableReader>[] maxOrder) + { + return new SSTableIntervalTree(minOrder, maxOrder); + } + public static SSTableIntervalTree empty() { return EMPTY; } - public static SSTableIntervalTree build(Iterable<SSTableReader> sstables) + public static SSTableIntervalTree buildSSTableIntervalTree(Collection<SSTableReader> sstables) { + if (sstables.isEmpty()) + return EMPTY; return new SSTableIntervalTree(buildIntervals(sstables)); } - public static List<Interval<PartitionPosition, SSTableReader>> buildIntervals(Iterable<SSTableReader> sstables) + public static List<Interval<PartitionPosition, SSTableReader>> buildIntervals(Collection<SSTableReader> sstables) { - List<Interval<PartitionPosition, SSTableReader>> intervals = new ArrayList<>(Iterables.size(sstables)); + if (sstables == null || sstables.isEmpty()) + return Collections.emptyList(); + return Arrays.asList(buildIntervalsArray(sstables)); + } + + public static Interval<PartitionPosition, SSTableReader>[] buildIntervalsArray(Collection<SSTableReader> sstables) + { + if (sstables == null || sstables.isEmpty()) + return IntervalTree.EMPTY_ARRAY; + Interval<PartitionPosition, SSTableReader>[] intervals = new Interval[sstables.size()]; + int i = 0; for (SSTableReader sstable : sstables) - intervals.add(Interval.<PartitionPosition, SSTableReader>create(sstable.getFirst(), sstable.getLast(), sstable)); + intervals[i++] = sstable.getInterval(); return intervals; } + + public static SSTableIntervalTree update(SSTableIntervalTree tree, Collection<SSTableReader> removals, Collection<SSTableReader> additions) + { + return (SSTableIntervalTree) tree.update(buildIntervalsArray(removals), buildIntervalsArray(additions)); + } } diff --git a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java index f5cedf38e877..c268b872adaa 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java +++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java @@ -23,7 +23,7 @@ import java.util.List; import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReentrantLock; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; @@ -89,7 +89,10 @@ public class Tracker private final List<INotificationConsumer> subscribers = new CopyOnWriteArrayList<>(); public final ColumnFamilyStore cfstore; - final AtomicReference<View> view; + + // Constructing views update can be quite slow so locking generates less CPU/garbage compared to CAS + final ReentrantLock viewUpdateLock = new ReentrantLock(true); + volatile View view = null; public final boolean loadsstables; /** @@ -100,7 +103,6 @@ public class Tracker public Tracker(ColumnFamilyStore columnFamilyStore, Memtable memtable, boolean loadsstables) { this.cfstore = columnFamilyStore; - this.view = new AtomicReference<>(); this.loadsstables = loadsstables; this.reset(memtable); } @@ -154,15 +156,22 @@ Throwable apply(Function<View, View> function, Throwable accumulate) */ Pair<View, View> apply(Predicate<View> permit, Function<View, View> function) { - while (true) + View updated; + View cur; + viewUpdateLock.lock(); + try { - View cur = view.get(); + cur = view; if (!permit.apply(cur)) return null; - View updated = function.apply(cur); - if (view.compareAndSet(cur, updated)) - return Pair.create(cur, updated); + updated = function.apply(cur); + view = updated; } + finally + { + viewUpdateLock.unlock(); + } + return Pair.create(cur, updated); } Throwable updateSizeTracking(Iterable<SSTableReader> oldSSTables, Iterable<SSTableReader> newSSTables, Throwable accumulate) @@ -225,12 +234,12 @@ public void updateLiveDiskSpaceUsed(long adjustment) // SETUP / CLEANUP - public void addInitialSSTables(Iterable<SSTableReader> sstables) + public void addInitialSSTables(Collection<SSTableReader> sstables) { addSSTablesInternal(sstables, true, false, true); } - public void addInitialSSTablesWithoutUpdatingSize(Iterable<SSTableReader> sstables) + public void addInitialSSTablesWithoutUpdatingSize(Collection<SSTableReader> sstables) { addSSTablesInternal(sstables, true, false, false); } @@ -240,12 +249,12 @@ public void updateInitialSSTableSize(Iterable<SSTableReader> sstables) maybeFail(updateSizeTracking(emptySet(), sstables, null)); } - public void addSSTables(Iterable<SSTableReader> sstables) + public void addSSTables(Collection<SSTableReader> sstables) { addSSTablesInternal(sstables, false, true, true); } - private void addSSTablesInternal(Iterable<SSTableReader> sstables, + private void addSSTablesInternal(Collection<SSTableReader> sstables, boolean isInitialSSTables, boolean maybeIncrementallyBackup, boolean updateSize) @@ -264,11 +273,19 @@ private void addSSTablesInternal(Iterable<SSTableReader> sstables, @VisibleForTesting public void reset(Memtable memtable) { - view.set(new View(memtable != null ? singletonList(memtable) : Collections.emptyList(), - Collections.emptyList(), - Collections.emptyMap(), - Collections.emptyMap(), - SSTableIntervalTree.empty())); + viewUpdateLock.lock(); + try + { + view = new View(memtable != null ? singletonList(memtable) : Collections.emptyList(), + Collections.emptyList(), + Collections.emptyMap(), + Collections.emptyMap(), + SSTableIntervalTree.empty()); + } + finally + { + viewUpdateLock.unlock(); + } } public Throwable dropSSTablesIfInvalid(Throwable accumulate) @@ -359,12 +376,13 @@ public Memtable getMemtableFor(OpOrder.Group opGroup, CommitLogPosition commitLo // there may be multiple memtables in the list that would 'accept' us, however we only ever choose // the oldest such memtable, as accepts() only prevents us falling behind (i.e. ensures we don't // assign operations to a memtable that was retired/queued before we started) - for (Memtable memtable : view.get().liveMemtables) + View view = this.view; + for (Memtable memtable : view.liveMemtables) { if (memtable.accepts(opGroup, commitLogPosition)) return memtable; } - throw new AssertionError(view.get().liveMemtables.toString()); + throw new AssertionError(view.liveMemtables.toString()); } /** @@ -391,7 +409,7 @@ public void markFlushing(Memtable memtable) apply(View.markFlushing(memtable)); } - public void replaceFlushed(Memtable memtable, Iterable<SSTableReader> sstables) + public void replaceFlushed(Memtable memtable, Collection<SSTableReader> sstables) { assert !isDummy(); if (Iterables.isEmpty(sstables)) @@ -429,17 +447,17 @@ public void replaceFlushed(Memtable memtable, Iterable<SSTableReader> sstables) public Set<SSTableReader> getCompacting() { - return view.get().compacting; + return view.compacting; } public Iterable<SSTableReader> getUncompacting() { - return view.get().select(SSTableSet.NONCOMPACTING); + return view.select(SSTableSet.NONCOMPACTING); } public Iterable<SSTableReader> getUncompacting(Iterable<SSTableReader> candidates) { - return view.get().getUncompacting(candidates); + return view.getUncompacting(candidates); } public void maybeIncrementallyBackup(final Iterable<SSTableReader> sstables) @@ -580,7 +598,7 @@ private static Set<SSTableReader> emptySet() public View getView() { - return view.get(); + return view; } @VisibleForTesting diff --git a/src/java/org/apache/cassandra/db/lifecycle/View.java b/src/java/org/apache/cassandra/db/lifecycle/View.java index b238d24d582d..ba200d5d0bc1 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/View.java +++ b/src/java/org/apache/cassandra/db/lifecycle/View.java @@ -17,6 +17,7 @@ */ package org.apache.cassandra.db.lifecycle; +import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.List; @@ -296,7 +297,7 @@ public boolean apply(View view) } // construct a function to change the liveset in a Snapshot - static Function<View, View> updateLiveSet(final Set<SSTableReader> remove, final Iterable<SSTableReader> add) + static Function<View, View> updateLiveSet(final Set<SSTableReader> remove, final Collection<SSTableReader> add) { if (remove.isEmpty() && Iterables.isEmpty(add)) return Functions.identity(); @@ -306,7 +307,7 @@ public View apply(View view) { Map<SSTableReader, SSTableReader> sstableMap = replace(view.sstablesMap, remove, add); return new View(view.liveMemtables, view.flushingMemtables, sstableMap, view.compactingMap, - SSTableIntervalTree.build(sstableMap.keySet())); + SSTableIntervalTree.update(view.intervalTree, remove, add)); } }; } @@ -345,7 +346,7 @@ public View apply(View view) } // called after flush: removes memtable from flushingMemtables, and inserts flushed into the live sstable set - static Function<View, View> replaceFlushed(final Memtable memtable, final Iterable<SSTableReader> flushed) + static Function<View, View> replaceFlushed(final Memtable memtable, final Collection<SSTableReader> flushed) { return new Function<View, View>() { @@ -360,7 +361,7 @@ public View apply(View view) Map<SSTableReader, SSTableReader> sstableMap = replace(view.sstablesMap, emptySet(), flushed); return new View(view.liveMemtables, flushingMemtables, sstableMap, view.compactingMap, - SSTableIntervalTree.build(sstableMap.keySet())); + SSTableIntervalTree.update(view.intervalTree, null, flushed)); } }; } diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java index d122c9c78852..6940f11b57fc 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java @@ -18,10 +18,19 @@ package org.apache.cassandra.db.streaming; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Set; + import com.google.common.base.Predicate; import com.google.common.base.Predicates; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.db.lifecycle.SSTableIntervalTree; @@ -44,13 +53,8 @@ import org.apache.cassandra.utils.TimeUUID; import org.apache.cassandra.utils.concurrent.Ref; import org.apache.cassandra.utils.concurrent.Refs; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Set; +import static org.apache.cassandra.db.lifecycle.SSTableIntervalTree.buildSSTableIntervalTree; /** * Implements the streaming interface for the native cassandra storage engine. @@ -93,7 +97,7 @@ public Collection<OutgoingStream> createOutgoingStreams(StreamSession session, R keyRanges.add(Range.makeRowRange(replica.range())); refs.addAll(cfs.selectAndReference(view -> { Set<SSTableReader> sstables = Sets.newHashSet(); - SSTableIntervalTree intervalTree = SSTableIntervalTree.build(view.select(SSTableSet.CANONICAL)); + SSTableIntervalTree intervalTree = buildSSTableIntervalTree(ImmutableList.copyOf(view.select(SSTableSet.CANONICAL))); Predicate<SSTableReader> predicate; if (previewKind.isPreview()) { diff --git a/src/java/org/apache/cassandra/index/sai/disk/SSTableIndex.java b/src/java/org/apache/cassandra/index/sai/disk/SSTableIndex.java index aca92e6dd22a..03eb6163ad84 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/SSTableIndex.java +++ b/src/java/org/apache/cassandra/index/sai/disk/SSTableIndex.java @@ -36,11 +36,11 @@ import org.apache.cassandra.index.sai.QueryContext; import org.apache.cassandra.index.sai.SSTableContext; import org.apache.cassandra.index.sai.StorageAttachedIndex; -import org.apache.cassandra.index.sai.utils.IndexIdentifier; import org.apache.cassandra.index.sai.disk.format.Version; import org.apache.cassandra.index.sai.disk.v1.segment.SegmentOrdering; import org.apache.cassandra.index.sai.iterators.KeyRangeIterator; import org.apache.cassandra.index.sai.plan.Expression; +import org.apache.cassandra.index.sai.utils.IndexIdentifier; import org.apache.cassandra.index.sai.utils.IndexTermType; import org.apache.cassandra.io.sstable.SSTableIdFactory; import org.apache.cassandra.io.sstable.format.SSTableReader; @@ -53,7 +53,7 @@ * <li>Exposes the index metadata for the column index</li> * </ul> */ -public abstract class SSTableIndex implements SegmentOrdering +public abstract class SSTableIndex implements SegmentOrdering, Comparable<SSTableIndex> { private static final Logger logger = LoggerFactory.getLogger(SSTableIndex.class); @@ -267,4 +267,11 @@ public String toString() .add("totalRows", sstableContext.sstable.getTotalRows()) .toString(); } + + @Override + public int compareTo(SSTableIndex index) + { + // SSTableReader is truly unique for comparison which is relied on in IntervalTree + return getSSTable().compareTo(index.getSSTable()); + } } diff --git a/src/java/org/apache/cassandra/index/sasi/SSTableIndex.java b/src/java/org/apache/cassandra/index/sasi/SSTableIndex.java index de9c0c2b4ee6..b5e790564013 100644 --- a/src/java/org/apache/cassandra/index/sasi/SSTableIndex.java +++ b/src/java/org/apache/cassandra/index/sasi/SSTableIndex.java @@ -39,7 +39,7 @@ import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.utils.concurrent.Ref; -public class SSTableIndex +public class SSTableIndex implements Comparable<SSTableIndex> { private final ColumnIndex columnIndex; private final Ref<SSTableReader> sstableRef; @@ -162,6 +162,13 @@ public String toString() return String.format("SSTableIndex(column: %s, SSTable: %s)", columnIndex.getColumnName(), sstable.descriptor); } + @Override + public int compareTo(SSTableIndex o) + { + // Relied on in IntervalTree to be unique + return sstable.compareTo(o.sstable); + } + private static class DecoratedKeyFetcher implements Function<Long, DecoratedKey> { private final SSTableReader sstable; diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java index 4f622be5a51e..ececa03dfa78 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java @@ -100,6 +100,7 @@ import org.apache.cassandra.utils.EstimatedHistogram; import org.apache.cassandra.utils.ExecutorUtils; import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.Interval; import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.NativeLibrary; import org.apache.cassandra.utils.OutputHandler; @@ -112,6 +113,7 @@ import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException; import static org.apache.cassandra.concurrent.ExecutorFactory.Global.executorFactory; +import static org.apache.cassandra.utils.TimeUUID.unixMicrosToRawTimestamp; import static org.apache.cassandra.utils.concurrent.BlockingQueues.newBlockingQueue; import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull; @@ -149,7 +151,7 @@ * <p> * TODO: fill in details about Tracker and lifecycle interactions for tools, and for compaction strategies */ -public abstract class SSTableReader extends SSTable implements UnfilteredSource, SelfRefCounted<SSTableReader> +public abstract class SSTableReader extends SSTable implements UnfilteredSource, SelfRefCounted<SSTableReader>, Comparable<SSTableReader> { private static final Logger logger = LoggerFactory.getLogger(SSTableReader.class); @@ -177,11 +179,25 @@ private static ScheduledExecutorPlus initSyncExecutor() public static final Comparator<SSTableReader> maxTimestampAscending = Comparator.comparingLong(SSTableReader::getMaxTimestamp); public static final Comparator<SSTableReader> maxTimestampDescending = maxTimestampAscending.reversed(); + private static final TimeUUID.Generator.Factory<UniqueIdentifier> UNIQUE_IDENTIFIER_FACTORY = new TimeUUID.Generator.Factory<UniqueIdentifier>() + { + @Override + public UniqueIdentifier atUnixMicrosWithLsb(long unixMicros, long clockSeqAndNode) + { + return new UniqueIdentifier(unixMicrosToRawTimestamp(unixMicros), clockSeqAndNode); + } + }; + // it's just an object, which we use regular Object equality on; we introduce a special class just for easy recognition - public static final class UniqueIdentifier + // Also includes a TimeUUID to make these sortable + public static final class UniqueIdentifier extends TimeUUID { + private UniqueIdentifier(long unixMicros, long clockSeqAndNode) + { + super(unixMicros, clockSeqAndNode); + } } - public final UniqueIdentifier instanceId = new UniqueIdentifier(); + public final UniqueIdentifier instanceId = TimeUUID.Generator.nextTimeUUID(UNIQUE_IDENTIFIER_FACTORY); public static final Comparator<SSTableReader> firstKeyComparator = (o1, o2) -> o1.getFirst().compareTo(o2.getFirst()); public static final Ordering<SSTableReader> firstKeyOrdering = Ordering.from(firstKeyComparator); @@ -271,6 +287,7 @@ public enum OpenReason protected final DecoratedKey first; protected final DecoratedKey last; public final AbstractBounds<Token> bounds; + private final Interval<PartitionPosition, SSTableReader> interval; /** * Calculate approximate key count. @@ -458,6 +475,7 @@ protected SSTableReader(Builder<?, ?> builder, Owner owner) this.openReason = builder.getOpenReason(); this.first = builder.getFirst(); this.last = builder.getLast(); + this.interval = Interval.create(first, last, this); this.bounds = first == null || last == null || AbstractBounds.strictlyWrapsAround(first.getToken(), last.getToken()) ? null // this will cause the validation to fail, but the reader is opened with no validation, // e.g. for scrubbing, we should accept screwed bounds @@ -479,6 +497,11 @@ public DecoratedKey getLast() return last; } + public Interval<PartitionPosition, SSTableReader> getInterval() + { + return interval; + } + @Override public AbstractBounds<Token> getBounds() { @@ -1741,6 +1764,13 @@ public abstract IVerifier getVerifier(ColumnFamilyStore cfs, boolean isOffline, IVerifier.Options options); + @Override + public int compareTo(SSTableReader other) + { + // Used in IntervalTree with the expecation that compareTo uniquely identifies an SSTableReader + return instanceId.compareTo(other.instanceId); + } + /** * A method to be called by {@link #getPosition(PartitionPosition, Operator, boolean, SSTableReadsListener)} * and {@link #getRowIndexEntry(PartitionPosition, Operator, boolean, SSTableReadsListener)} methods when diff --git a/src/java/org/apache/cassandra/utils/Interval.java b/src/java/org/apache/cassandra/utils/Interval.java index 93981449d588..6e7b8c33a3fb 100644 --- a/src/java/org/apache/cassandra/utils/Interval.java +++ b/src/java/org/apache/cassandra/utils/Interval.java @@ -65,35 +65,49 @@ public final boolean equals(Object o) return Objects.equal(min, that.min) && Objects.equal(max, that.max) && Objects.equal(data, that.data); } - private static final AsymmetricOrdering<Interval<Comparable, Object>, Comparable> minOrdering - = new AsymmetricOrdering<Interval<Comparable, Object>, Comparable>() + private static final AsymmetricOrdering<Interval<Comparable, Comparable>, Comparable> minOrdering + = new AsymmetricOrdering<Interval<Comparable, Comparable>, Comparable>() { - public int compareAsymmetric(Interval<Comparable, Object> left, Comparable right) + public int compareAsymmetric(Interval<Comparable, Comparable> left, Comparable right) { return left.min.compareTo(right); } - public int compare(Interval<Comparable, Object> i1, Interval<Comparable, Object> i2) + public int compare(Interval<Comparable, Comparable> i1, Interval<Comparable, Comparable> i2) { - return i1.min.compareTo(i2.min); + int cmpMin = i1.min.compareTo(i2.min); + if (cmpMin != 0) + return cmpMin; + int cmpMax = i1.max.compareTo(i2.max); + if (cmpMax != 0) + return cmpMax; + // Null is allowed if all data values are null otherwise NPE + return i1.data == i2.data ? 0 : i1.data.compareTo(i2.data); } }; - private static final AsymmetricOrdering<Interval<Comparable, Object>, Comparable> maxOrdering - = new AsymmetricOrdering<Interval<Comparable, Object>, Comparable>() + private static final AsymmetricOrdering<Interval<Comparable, Comparable>, Comparable> maxOrdering + = new AsymmetricOrdering<Interval<Comparable, Comparable>, Comparable>() { - public int compareAsymmetric(Interval<Comparable, Object> left, Comparable right) + public int compareAsymmetric(Interval<Comparable, Comparable> left, Comparable right) { return left.max.compareTo(right); } - public int compare(Interval<Comparable, Object> i1, Interval<Comparable, Object> i2) + public int compare(Interval<Comparable, Comparable> i1, Interval<Comparable, Comparable> i2) { - return i1.max.compareTo(i2.max); + int cmpMax = i1.max.compareTo(i2.max); + if (cmpMax != 0) + return cmpMax; + int cmpMin = i1.min.compareTo(i2.min); + if (cmpMin != 0) + return cmpMin; + // Null is allowed if all data values are null otherwise NPE + return i1.data == i2.data ? 0 : i1.data.compareTo(i2.data); } }; - private static final AsymmetricOrdering<Interval<Comparable, Object>, Comparable> reverseMaxOrdering = maxOrdering.reverse(); + private static final AsymmetricOrdering<Interval<Comparable, Comparable>, Comparable> reverseMaxOrdering = maxOrdering.reverse(); public static <C extends Comparable<? super C>, V> AsymmetricOrdering<Interval<C, V>, C> minOrdering() { diff --git a/src/java/org/apache/cassandra/utils/IntervalTree.java b/src/java/org/apache/cassandra/utils/IntervalTree.java index 35ec614a818e..bde80c5dfe0f 100644 --- a/src/java/org/apache/cassandra/utils/IntervalTree.java +++ b/src/java/org/apache/cassandra/utils/IntervalTree.java @@ -17,61 +17,110 @@ */ package org.apache.cassandra.utils; -import java.io.IOException; -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; -import java.util.*; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Deque; +import java.util.Iterator; +import java.util.List; import com.google.common.base.Joiner; +import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.db.TypeSizes; -import org.apache.cassandra.io.ISerializer; -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputPlus; + import org.apache.cassandra.utils.AsymmetricOrdering.Op; -public class IntervalTree<C extends Comparable<? super C>, D, I extends Interval<C, D>> implements Iterable<I> +import static com.google.common.base.Preconditions.checkState; +import static org.apache.cassandra.config.CassandraRelevantProperties.TEST_INTERVAL_TREE_EXPENSIVE_CHECKS; + +public class IntervalTree<C extends Comparable<? super C>, D extends Comparable<? super D>, I extends Interval<C, D>> implements Iterable<I> { + public static final boolean EXPENSIVE_CHECKS = TEST_INTERVAL_TREE_EXPENSIVE_CHECKS.getBoolean(); + private static final Logger logger = LoggerFactory.getLogger(IntervalTree.class); + public static final Interval[] EMPTY_ARRAY = new Interval[0]; + @SuppressWarnings("unchecked") private static final IntervalTree EMPTY_TREE = new IntervalTree(null); private final IntervalNode head; - private final int count; + private final I[] intervalsByMinOrder; + private final I[] intervalsByMaxOrder; protected IntervalTree(Collection<I> intervals) { - this.head = intervals == null || intervals.isEmpty() ? null : new IntervalNode(intervals); - this.count = intervals == null ? 0 : intervals.size(); + if (intervals == null || intervals.isEmpty()) + { + this.head = null; + intervalsByMinOrder = intervalsByMaxOrder = (I[])EMPTY_ARRAY; + } + else if (intervals.size() == 1) + { + intervalsByMinOrder = intervalsByMaxOrder = (I[])new Interval[] { intervals.iterator().next() }; + this.head = new IntervalNode(intervals); + } + else + { + intervalsByMinOrder = intervals.toArray((I[])EMPTY_ARRAY); + Arrays.sort(intervalsByMinOrder, Interval.minOrdering()); + intervalsByMaxOrder = intervals.toArray((I[])EMPTY_ARRAY); + Arrays.sort(intervalsByMaxOrder, Interval.maxOrdering()); + this.head = new IntervalNode(Arrays.asList(intervalsByMinOrder), Arrays.asList(intervalsByMaxOrder)); + } } - public static <C extends Comparable<? super C>, D, I extends Interval<C, D>> IntervalTree<C, D, I> build(Collection<I> intervals) + /** + * This constructor will not modify minSortedIntervals and maxSortedIntervals, but it also won't + * make defensive copies and will keep the originals. + */ + protected IntervalTree(I[] minSortedIntervals, I[] maxSortedIntervals) { - if (intervals == null || intervals.isEmpty()) - return emptyTree(); + if (minSortedIntervals == null || minSortedIntervals.length == 0) + { + this.head = null; + intervalsByMinOrder = intervalsByMaxOrder = (I[])EMPTY_ARRAY; + } + else if (minSortedIntervals.length == 1) + { + intervalsByMinOrder = intervalsByMaxOrder = minSortedIntervals; + List<I> intervals = Collections.singletonList(minSortedIntervals[0]); + this.head = new IntervalNode(intervals, intervals); + } + else + { + intervalsByMinOrder = minSortedIntervals; + intervalsByMaxOrder = maxSortedIntervals; + this.head = new IntervalNode(Arrays.asList(minSortedIntervals), Arrays.asList(maxSortedIntervals)); + } + } - return new IntervalTree<C, D, I>(intervals); + protected IntervalTree<C, D, I> create(I[] minOrder, I[] maxOrder) + { + return new IntervalTree(minOrder, maxOrder); } - public static <C extends Comparable<? super C>, D, I extends Interval<C, D>> Serializer<C, D, I> serializer(ISerializer<C> pointSerializer, ISerializer<D> dataSerializer, Constructor<I> constructor) + public static <C extends Comparable<? super C>, D extends Comparable<? super D>, I extends Interval<C, D>> IntervalTree<C, D, I> build(Collection<I> intervals) { - return new Serializer<>(pointSerializer, dataSerializer, constructor); + if (intervals == null || intervals.isEmpty()) + return emptyTree(); + + return new IntervalTree<>(intervals); } @SuppressWarnings("unchecked") - public static <C extends Comparable<? super C>, D, I extends Interval<C, D>> IntervalTree<C, D, I> emptyTree() + public static <C extends Comparable<? super C>, D extends Comparable<? super D>, I extends Interval<C, D>> IntervalTree<C, D, I> emptyTree() { return EMPTY_TREE; } public int intervalCount() { - return count; + return intervalsByMinOrder.length; } public boolean isEmpty() @@ -110,6 +159,123 @@ public List<D> search(C point) return search(Interval.<C, D>create(point, point, null)); } + /** + * The input arrays aren't defensively copied and will be sorted. The update method doesn't allow duplicates or elements to be removed + * to be missing and this differs from the constructor which does not duplicate checking at all. + * + * It made more sense for update to be stricter because it is tracking removals and additions explicitly instead of building + * a list from scratch and in the targeted use case of a list of SSTables there are no duplicates. At a given point in time + * an sstable represents exactly one interval (although it may switch via removal and addition as in early open). + */ + public IntervalTree<C, D, I> update(I[] removals, I[] additions) + { + if (removals == null) + removals = (I[])EMPTY_ARRAY; + if (additions == null) + additions = (I[])EMPTY_ARRAY; + + if (removals.length == 0 && additions.length == 0) + { + return this; + } + + Arrays.sort(removals, Interval.<C, D>minOrdering()); + Arrays.sort(additions, Interval.<C, D>minOrdering()); + + for (int i = 1; i < additions.length; i++) + checkState( Interval.<C, D>minOrdering().compare(additions[i], additions[i-1]) != 0, "Duplicate interval in additions %s", additions[i]); + + I[] newByMin = buildUpdatedArray( + intervalsByMinOrder, + removals, + additions, + Interval.<C, D>minOrdering() + ); + + Arrays.sort(removals, Interval.<C, D>maxOrdering()); + Arrays.sort(additions, Interval.<C, D>maxOrdering()); + + I[] newByMax = buildUpdatedArray( + intervalsByMaxOrder, + removals, + additions, + Interval.<C, D>maxOrdering() + ); + + return create(newByMin, newByMax); + } + + @SuppressWarnings("unchecked") + private I[] buildUpdatedArray(I[] existingSorted, + I[] removalsSorted, + I[] additionsSorted, + AsymmetricOrdering<Interval<C, D>, C> cmp) + { + int finalSize = existingSorted.length + additionsSorted.length - removalsSorted.length; + I[] result = (I[]) new Interval[finalSize]; + + int existingIndex = 0; + int removalsIndex = 0; + int additionsIndex = 0; + int resultIndex = 0; + + while (existingIndex < existingSorted.length) + { + I currentExisting = existingSorted[existingIndex]; + + int c; + while (removalsIndex < removalsSorted.length + && (c = cmp.compare(removalsSorted[removalsIndex], currentExisting)) <= 0) + { + if (c < 0) + { + throw new IllegalStateException("Removal interval not found in the existing tree: " + removalsSorted[removalsIndex]); + } + else + { + existingIndex++; + removalsIndex++; + + if (existingIndex >= existingSorted.length) + break; + currentExisting = existingSorted[existingIndex]; + } + } + + if (existingIndex >= existingSorted.length ) + break; + + while (additionsIndex < additionsSorted.length) + { + int additionCmp = cmp.compare(additionsSorted[additionsIndex], currentExisting); + if (additionCmp == 0) + throw new IllegalStateException("Attempting to add duplicate interval: " + additionsSorted[additionsIndex]); + else if (additionCmp < 0) + result[resultIndex++] = additionsSorted[additionsIndex++]; + else + break; + } + + result[resultIndex++] = currentExisting; + existingIndex++; + } + + if (removalsIndex < removalsSorted.length) + throw new IllegalStateException("Removal interval not found in the existing tree: " + removalsSorted[removalsIndex]); + + while (additionsIndex < additionsSorted.length) + result[resultIndex++] = additionsSorted[additionsIndex++]; + + if (EXPENSIVE_CHECKS) + { + if (result.length > 1) + for (int i = 1; i < result.length; i++) + checkState(cmp.compare(result[i - 1], result[i]) < 0, "%s and %s out of order", result[i-1], result[i]); + } + + return result; + } + public Iterator<I> iterator() { if (head == null) @@ -121,7 +287,7 @@ public Iterator<I> iterator() @Override public String toString() { - return "<" + Joiner.on(", ").join(this) + ">"; + return "<" + Joiner.on(", ").join(Iterables.limit(this, 100)) + ">"; } @Override @@ -142,7 +308,7 @@ public final int hashCode() return result; } - private class IntervalNode + protected class IntervalNode { final C center; final C low; @@ -156,14 +322,28 @@ private class IntervalNode public IntervalNode(Collection<I> toBisect) { - assert !toBisect.isEmpty(); - logger.trace("Creating IntervalNode from {}", toBisect); + assert toBisect.size() == 1; + I interval = toBisect.iterator().next(); + low = interval.min; + center = interval.max; + high = interval.max; + List<I> l = Collections.singletonList(interval); + intersectsLeft = l; + intersectsRight = l; + left = null; + right = null; + } + + public IntervalNode(List<I> minOrder, List<I> maxOrder) + { + assert !minOrder.isEmpty(); + logger.trace("Creating IntervalNode from {}", minOrder); // Building IntervalTree with one interval will be a reasonably // common case for range tombstones, so it's worth optimizing - if (toBisect.size() == 1) + if (minOrder.size() == 1) { - I interval = toBisect.iterator().next(); + I interval = minOrder.iterator().next(); low = interval.min; center = interval.max; high = interval.max; @@ -172,51 +352,83 @@ public IntervalNode(Collection<I> toBisect) intersectsRight = l; left = null; right = null; + return; + } + + low = minOrder.get(0).min; + high = maxOrder.get(maxOrder.size() - 1).max; + + int i = 0, j = 0, count = 0; + while (count < minOrder.size()) + { + if (i < minOrder.size() && (j >= maxOrder.size() || minOrder.get(i).min.compareTo(maxOrder.get(j).max) <= 0)) + i++; + else + j++; + count++; } + + if (i < minOrder.size() && (j >= maxOrder.size() || minOrder.get(i).min.compareTo(maxOrder.get(j).max) < 0)) + center = minOrder.get(i).min; else + center = maxOrder.get(j).max; + + if (EXPENSIVE_CHECKS) { - // Find min, median and max - List<C> allEndpoints = new ArrayList<C>(toBisect.size() * 2); - for (I interval : toBisect) + List<C> allEndpoints = new ArrayList<C>(minOrder.size() * 2); + for (I interval : minOrder) { allEndpoints.add(interval.min); allEndpoints.add(interval.max); } Collections.sort(allEndpoints); + C expectedCenter = allEndpoints.get(minOrder.size()); + checkState(expectedCenter.equals(center)); + } - low = allEndpoints.get(0); - center = allEndpoints.get(toBisect.size()); - high = allEndpoints.get(allEndpoints.size() - 1); - - // Separate interval in intersecting center, left of center and right of center - List<I> intersects = new ArrayList<I>(); - List<I> leftSegment = new ArrayList<I>(); - List<I> rightSegment = new ArrayList<I>(); + // Separate interval in intersecting center, left of center and right of center + int initialIntersectionSize = i - j + 1; + intersectsLeft = new ArrayList<I>(initialIntersectionSize); + intersectsRight = new ArrayList<I>(initialIntersectionSize); + int initialChildSize = Math.min(i, j); + List<I> leftSegmentMinOrder = new ArrayList<I>(initialChildSize); + List<I> leftSegmentMaxOrder = new ArrayList<>(initialChildSize); + List<I> rightSegmentMinOrder = new ArrayList<I>(initialChildSize); + List<I> rightSegmentMaxOrder = new ArrayList<>(initialChildSize); + + for (I candidate : minOrder) + { + if (candidate.max.compareTo(center) < 0) + leftSegmentMinOrder.add(candidate); + else if (candidate.min.compareTo(center) > 0) + rightSegmentMinOrder.add(candidate); + else + intersectsLeft.add(candidate); + } - for (I candidate : toBisect) - { - if (candidate.max.compareTo(center) < 0) - leftSegment.add(candidate); - else if (candidate.min.compareTo(center) > 0) - rightSegment.add(candidate); - else - intersects.add(candidate); - } + for (I candidate : maxOrder) + { + if (candidate.max.compareTo(center) < 0) + leftSegmentMaxOrder.add(candidate); + else if (candidate.min.compareTo(center) > 0) + rightSegmentMaxOrder.add(candidate); + else + intersectsRight.add(candidate); + } - intersectsLeft = Interval.<C, D>minOrdering().sortedCopy(intersects); - intersectsRight = Interval.<C, D>maxOrdering().sortedCopy(intersects); - left = leftSegment.isEmpty() ? null : new IntervalNode(leftSegment); - right = rightSegment.isEmpty() ? null : new IntervalNode(rightSegment); + left = leftSegmentMinOrder.isEmpty() ? null : new IntervalNode(leftSegmentMinOrder, leftSegmentMaxOrder); + right = rightSegmentMinOrder.isEmpty() ? null : new IntervalNode(rightSegmentMinOrder, rightSegmentMaxOrder); - assert (intersects.size() + leftSegment.size() + rightSegment.size()) == toBisect.size() : - "intersects (" + String.valueOf(intersects.size()) + - ") + leftSegment (" + String.valueOf(leftSegment.size()) + - ") + rightSegment (" + String.valueOf(rightSegment.size()) + - ") != toBisect (" + String.valueOf(toBisect.size()) + ")"; - } + assert (intersectsLeft.size() == intersectsRight.size()); + assert (intersectsLeft.size() + leftSegmentMinOrder.size() + rightSegmentMinOrder.size()) == minOrder.size() : + "intersects (" + String.valueOf(intersectsLeft.size()) + + ") + leftSegment (" + String.valueOf(leftSegmentMinOrder.size()) + + ") + rightSegment (" + String.valueOf(rightSegmentMinOrder.size()) + + ") != toBisect (" + String.valueOf(minOrder.size()) + ")"; } + void searchInternal(Interval<C, D> searchInterval, List<D> results) { if (center.compareTo(searchInterval.min) < 0) @@ -298,73 +510,4 @@ private void gotoMinOf(IntervalNode node) } } - - public static class Serializer<C extends Comparable<? super C>, D, I extends Interval<C, D>> implements IVersionedSerializer<IntervalTree<C, D, I>> - { - private final ISerializer<C> pointSerializer; - private final ISerializer<D> dataSerializer; - private final Constructor<I> constructor; - - private Serializer(ISerializer<C> pointSerializer, ISerializer<D> dataSerializer, Constructor<I> constructor) - { - this.pointSerializer = pointSerializer; - this.dataSerializer = dataSerializer; - this.constructor = constructor; - } - - public void serialize(IntervalTree<C, D, I> it, DataOutputPlus out, int version) throws IOException - { - out.writeInt(it.count); - for (Interval<C, D> interval : it) - { - pointSerializer.serialize(interval.min, out); - pointSerializer.serialize(interval.max, out); - dataSerializer.serialize(interval.data, out); - } - } - - /** - * Deserialize an IntervalTree whose keys use the natural ordering. - * Use deserialize(DataInput, int, Comparator) instead if the interval - * tree is to use a custom comparator, as the comparator is *not* - * serialized. - */ - public IntervalTree<C, D, I> deserialize(DataInputPlus in, int version) throws IOException - { - return deserialize(in, version, null); - } - - public IntervalTree<C, D, I> deserialize(DataInputPlus in, int version, Comparator<C> comparator) throws IOException - { - try - { - int count = in.readInt(); - List<I> intervals = new ArrayList<I>(count); - for (int i = 0; i < count; i++) - { - C min = pointSerializer.deserialize(in); - C max = pointSerializer.deserialize(in); - D data = dataSerializer.deserialize(in); - intervals.add(constructor.newInstance(min, max, data)); - } - return new IntervalTree<C, D, I>(intervals); - } - catch (InstantiationException | InvocationTargetException | IllegalAccessException e) - { - throw new RuntimeException(e); - } - } - - public long serializedSize(IntervalTree<C, D, I> it, int version) - { - long size = TypeSizes.sizeof(0); - for (Interval<C, D> interval : it) - { - size += pointSerializer.serializedSize(interval.min); - size += pointSerializer.serializedSize(interval.max); - size += dataSerializer.serializedSize(interval.data); - } - return size; - } - } -} +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/utils/TimeUUID.java b/src/java/org/apache/cassandra/utils/TimeUUID.java index d993f171af32..90ae00e9cb0b 100644 --- a/src/java/org/apache/cassandra/utils/TimeUUID.java +++ b/src/java/org/apache/cassandra/utils/TimeUUID.java @@ -371,6 +371,16 @@ public static class Generator private static final AtomicLong lastMicros = new AtomicLong(); + public interface Factory<T extends TimeUUID> + { + T atUnixMicrosWithLsb(long unixMicros, long clockSeqAndNode); + } + + public static <T extends TimeUUID> T nextTimeUUID(Factory<T> factory) + { + return factory.atUnixMicrosWithLsb(nextUnixMicros(), clockSeqAndNode); + } + public static TimeUUID nextTimeUUID() { return atUnixMicrosWithLsb(nextUnixMicros(), clockSeqAndNode); diff --git a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java index d56003ae2a4a..ec8c8ad97509 100644 --- a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java @@ -32,6 +32,7 @@ import java.util.Set; import java.util.stream.Collectors; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; import org.junit.After; @@ -920,7 +921,7 @@ public void testReduceScopeL0L1() throws IOException List<SSTableReader> l0sstables = new ArrayList<>(); for (int i = 10; i < 20; i++) l0sstables.add(MockSchema.sstable(i, (i + 1) * 1024 * 1024, cfs)); - try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.COMPACTION, Iterables.concat(l0sstables, l1sstables))) + try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.COMPACTION, ImmutableList.copyOf(Iterables.concat(l0sstables, l1sstables)))) { Set<SSTableReader> nonExpired = Sets.difference(txn.originals(), Collections.emptySet()); CompactionTask task = new LeveledCompactionTask(cfs, txn, 1, 0, 1024*1024, false); diff --git a/test/unit/org/apache/cassandra/db/compaction/UnifiedCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/UnifiedCompactionStrategyTest.java index a673b27b7ceb..82fee2144c91 100644 --- a/test/unit/org/apache/cassandra/db/compaction/UnifiedCompactionStrategyTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/UnifiedCompactionStrategyTest.java @@ -36,6 +36,7 @@ import org.junit.BeforeClass; import org.junit.Test; +import com.google.monitoring.runtime.instrumentation.common.collect.ImmutableList; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.BufferDecoratedKey; import org.apache.cassandra.db.ColumnFamilyStore; @@ -54,6 +55,7 @@ import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.Interval; import org.apache.cassandra.utils.Overlaps; import org.apache.cassandra.utils.Pair; import org.mockito.Answers; @@ -596,7 +598,7 @@ private void testDropExpiredAndCompactNonExpired() List<SSTableReader> nonExpiredSSTables = createSStables(cfs.getPartitioner(), 0); strategy.addSSTables(expiredSSTables); strategy.addSSTables(nonExpiredSSTables.subList(0, 3)); - dataTracker.addInitialSSTables(Iterables.concat(expiredSSTables, nonExpiredSSTables)); + dataTracker.addInitialSSTables(ImmutableList.copyOf(Iterables.concat(expiredSSTables, nonExpiredSSTables))); long timestamp = expiredSSTables.get(expiredSSTables.size() - 1).getMaxLocalDeletionTime(); long expirationPoint = timestamp + 1; @@ -848,6 +850,7 @@ SSTableReader mockSSTable(int level, when(ret.getMinTimestamp()).thenReturn(timestamp); when(ret.getFirst()).thenReturn(first); when(ret.getLast()).thenReturn(last); + when(ret.getInterval()).thenReturn(new Interval<>(first, last, ret)); when(ret.isMarkedSuspect()).thenReturn(false); when(ret.isRepaired()).thenReturn(false); when(ret.getRepairedAt()).thenReturn(repairedAt); diff --git a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java index 7192d50031e8..e9e82e796095 100644 --- a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java +++ b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java @@ -24,10 +24,9 @@ import java.util.Optional; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; -import javax.annotation.Nullable; -import com.google.common.base.Function; import com.google.common.base.Predicate; +import com.google.common.base.Predicates; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import org.junit.Assert; @@ -121,27 +120,7 @@ public void testApply() final Tracker tracker = Tracker.newDummyTracker(); final View resultView = ViewTest.fakeView(0, 0, cfs); final AtomicInteger count = new AtomicInteger(); - tracker.apply(new Predicate<View>() - { - public boolean apply(View view) - { - // confound the CAS by swapping the view, and check we retry - if (count.incrementAndGet() < 3) - tracker.view.set(ViewTest.fakeView(0, 0, cfs)); - return true; - } - }, new Function<View, View>() - { - @Nullable - public View apply(View view) - { - return resultView; - } - }); - Assert.assertEquals(3, count.get()); - Assert.assertEquals(resultView, tracker.getView()); - - count.set(0); + tracker.apply(Predicates.alwaysTrue(), view -> resultView); // check that if the predicate returns false, we stop immediately and return null Assert.assertNull(tracker.apply(new Predicate<View>() { @@ -167,7 +146,7 @@ public void testAddInitialSSTables() MockSchema.sstable(2, 9, cfs)); tracker.addInitialSSTables(copyOf(readers)); - Assert.assertEquals(3, tracker.view.get().sstables.size()); + Assert.assertEquals(3, tracker.view.sstables.size()); Assert.assertEquals(1, listener.senders.size()); Assert.assertEquals(1, listener.received.size()); Assert.assertTrue(listener.received.get(0) instanceof InitialSSTableAddedNotification); @@ -193,7 +172,7 @@ public void testAddSSTables() MockSchema.sstable(2, 9, cfs)); tracker.addSSTables(copyOf(readers)); - Assert.assertEquals(3, tracker.view.get().sstables.size()); + Assert.assertEquals(3, tracker.view.sstables.size()); for (SSTableReader reader : readers) { diff --git a/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java b/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java index eb162d59b9f7..5d2c00634db4 100644 --- a/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java +++ b/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java @@ -28,10 +28,10 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; +import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; -import org.junit.Assert; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.PartitionPosition; @@ -46,6 +46,7 @@ import static com.google.common.collect.Iterables.concat; import static java.util.Collections.singleton; import static org.apache.cassandra.db.lifecycle.Helpers.emptySet; +import static org.apache.cassandra.db.lifecycle.SSTableIntervalTree.buildSSTableIntervalTree; public class ViewTest { @@ -225,6 +226,6 @@ static View fakeView(int memtableCount, int sstableCount, ColumnFamilyStore cfs, for (int i = 0 ; i < sstableCount ; i++) sstables.add(MockSchema.sstable(i, keepRef, cfs)); return new View(ImmutableList.copyOf(memtables), Collections.<Memtable>emptyList(), Helpers.identityMap(sstables), - Collections.<SSTableReader, SSTableReader>emptyMap(), SSTableIntervalTree.build(sstables)); + Collections.<SSTableReader, SSTableReader>emptyMap(), buildSSTableIntervalTree(sstables)); } } diff --git a/test/unit/org/apache/cassandra/utils/IntervalTreeTest.java b/test/unit/org/apache/cassandra/utils/IntervalTreeTest.java index 7e720986d9ee..7160e81d3f70 100644 --- a/test/unit/org/apache/cassandra/utils/IntervalTreeTest.java +++ b/test/unit/org/apache/cassandra/utils/IntervalTreeTest.java @@ -1,198 +1,443 @@ package org.apache.cassandra.utils; -/* - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - * - */ - - -import java.io.IOException; -import java.lang.reflect.Constructor; + import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.junit.BeforeClass; import org.junit.Test; -import org.apache.cassandra.io.ISerializer; -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.io.util.DataInputBuffer; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputBuffer; -import org.apache.cassandra.io.util.DataOutputPlus; +import org.quicktheories.WithQuickTheories; +import org.quicktheories.core.Gen; +import org.quicktheories.generators.SourceDSL; + +import static java.util.concurrent.TimeUnit.SECONDS; +import static java.util.function.Predicate.not; +import static org.apache.cassandra.config.CassandraRelevantProperties.TEST_INTERVAL_TREE_EXPENSIVE_CHECKS; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; -public class IntervalTreeTest +public class IntervalTreeTest implements WithQuickTheories { - @Test - public void testSearch() throws Exception + @BeforeClass + public static void enableExpensiveRangeChecks() { - List<Interval<Integer, Void>> intervals = new ArrayList<Interval<Integer, Void>>(); - - intervals.add(Interval.<Integer, Void>create(-300, -200)); - intervals.add(Interval.<Integer, Void>create(-3, -2)); - intervals.add(Interval.<Integer, Void>create(1, 2)); - intervals.add(Interval.<Integer, Void>create(3, 6)); - intervals.add(Interval.<Integer, Void>create(2, 4)); - intervals.add(Interval.<Integer, Void>create(5, 7)); - intervals.add(Interval.<Integer, Void>create(1, 3)); - intervals.add(Interval.<Integer, Void>create(4, 6)); - intervals.add(Interval.<Integer, Void>create(8, 9)); - intervals.add(Interval.<Integer, Void>create(15, 20)); - intervals.add(Interval.<Integer, Void>create(40, 50)); - intervals.add(Interval.<Integer, Void>create(49, 60)); - - - IntervalTree<Integer, Void, Interval<Integer, Void>> it = IntervalTree.build(intervals); - - assertEquals(3, it.search(Interval.<Integer, Void>create(4, 4)).size()); - assertEquals(4, it.search(Interval.<Integer, Void>create(4, 5)).size()); - assertEquals(7, it.search(Interval.<Integer, Void>create(-1, 10)).size()); - assertEquals(0, it.search(Interval.<Integer, Void>create(-1, -1)).size()); - assertEquals(5, it.search(Interval.<Integer, Void>create(1, 4)).size()); - assertEquals(2, it.search(Interval.<Integer, Void>create(0, 1)).size()); - assertEquals(0, it.search(Interval.<Integer, Void>create(10, 12)).size()); + assertFalse(TEST_INTERVAL_TREE_EXPENSIVE_CHECKS.getBoolean()); // Expect off by default + TEST_INTERVAL_TREE_EXPENSIVE_CHECKS.setBoolean(true); + assertTrue(TEST_INTERVAL_TREE_EXPENSIVE_CHECKS.getBoolean()); + assertTrue(IntervalTree.EXPENSIVE_CHECKS); + } - List<Interval<Integer, Void>> intervals2 = new ArrayList<Interval<Integer, Void>>(); + @Test + public void testSearch() + { + List<Interval<Integer, Integer>> intervals = new ArrayList<>(); + + intervals.add(Interval.create(-300, -200)); + intervals.add(Interval.create(-3, -2)); + intervals.add(Interval.create(1, 2)); + intervals.add(Interval.create(3, 6)); + intervals.add(Interval.create(2, 4)); + intervals.add(Interval.create(5, 7)); + intervals.add(Interval.create(1, 3)); + intervals.add(Interval.create(4, 6)); + intervals.add(Interval.create(8, 9)); + intervals.add(Interval.create(15, 20)); + intervals.add(Interval.create(40, 50)); + intervals.add(Interval.create(49, 60)); + + IntervalTree<Integer, Integer, Interval<Integer, Integer>> it = IntervalTree.build(intervals); + + assertEquals(3, it.search(Interval.create(4, 4)).size()); + assertEquals(4, it.search(Interval.create(4, 5)).size()); + assertEquals(7, it.search(Interval.create(-1, 10)).size()); + assertEquals(0, it.search(Interval.create(-1, -1)).size()); + assertEquals(5, it.search(Interval.create(1, 4)).size()); + assertEquals(2, it.search(Interval.create(0, 1)).size()); + assertEquals(0, it.search(Interval.create(10, 12)).size()); + + List<Interval<Integer, Integer>> intervals2 = new ArrayList<>(); //stravinsky 1880-1971 - intervals2.add(Interval.<Integer, Void>create(1880, 1971)); + intervals2.add(Interval.create(1880, 1971)); //Schoenberg - intervals2.add(Interval.<Integer, Void>create(1874, 1951)); + intervals2.add(Interval.create(1874, 1951)); //Grieg - intervals2.add(Interval.<Integer, Void>create(1843, 1907)); + intervals2.add(Interval.create(1843, 1907)); //Schubert - intervals2.add(Interval.<Integer, Void>create(1779, 1828)); + intervals2.add(Interval.create(1779, 1828)); //Mozart - intervals2.add(Interval.<Integer, Void>create(1756, 1828)); + intervals2.add(Interval.create(1756, 1828)); //Schuetz - intervals2.add(Interval.<Integer, Void>create(1585, 1672)); + intervals2.add(Interval.create(1585, 1672)); - IntervalTree<Integer, Void, Interval<Integer, Void>> it2 = IntervalTree.build(intervals2); + IntervalTree<Integer, Integer, Interval<Integer, Integer>> it2 = IntervalTree.build(intervals2); - assertEquals(0, it2.search(Interval.<Integer, Void>create(1829, 1842)).size()); + assertEquals(0, it2.search(Interval.create(1829, 1842)).size()); - List<Void> intersection1 = it2.search(Interval.<Integer, Void>create(1907, 1907)); + List<Integer> intersection1 = it2.search(Interval.create(1907, 1907)); assertEquals(3, intersection1.size()); - intersection1 = it2.search(Interval.<Integer, Void>create(1780, 1790)); + intersection1 = it2.search(Interval.create(1780, 1790)); assertEquals(2, intersection1.size()); - } @Test public void testIteration() { - List<Interval<Integer, Void>> intervals = new ArrayList<Interval<Integer, Void>>(); + List<Interval<Integer, Integer>> intervals = new ArrayList<>(); - intervals.add(Interval.<Integer, Void>create(-300, -200)); - intervals.add(Interval.<Integer, Void>create(-3, -2)); - intervals.add(Interval.<Integer, Void>create(1, 2)); - intervals.add(Interval.<Integer, Void>create(3, 6)); - intervals.add(Interval.<Integer, Void>create(2, 4)); - intervals.add(Interval.<Integer, Void>create(5, 7)); - intervals.add(Interval.<Integer, Void>create(1, 3)); - intervals.add(Interval.<Integer, Void>create(4, 6)); - intervals.add(Interval.<Integer, Void>create(8, 9)); - intervals.add(Interval.<Integer, Void>create(15, 20)); - intervals.add(Interval.<Integer, Void>create(40, 50)); - intervals.add(Interval.<Integer, Void>create(49, 60)); + intervals.add(Interval.create(-300, -200)); + intervals.add(Interval.create(-3, -2)); + intervals.add(Interval.create(1, 2)); + intervals.add(Interval.create(3, 6)); + intervals.add(Interval.create(2, 4)); + intervals.add(Interval.create(5, 7)); + intervals.add(Interval.create(1, 3)); + intervals.add(Interval.create(4, 6)); + intervals.add(Interval.create(8, 9)); + intervals.add(Interval.create(15, 20)); + intervals.add(Interval.create(40, 50)); + intervals.add(Interval.create(49, 60)); - IntervalTree<Integer, Void, Interval<Integer, Void>> it = IntervalTree.build(intervals); + IntervalTree<Integer, Integer, Interval<Integer, Integer>> it = IntervalTree.build(intervals); - Collections.sort(intervals, Interval.<Integer, Void>minOrdering()); + Collections.sort(intervals, Interval.minOrdering()); - List<Interval<Integer, Void>> l = new ArrayList<Interval<Integer, Void>>(); - for (Interval<Integer, Void> i : it) - l.add(i); + List<Interval<Integer, Integer>> l = ImmutableList.copyOf(it); assertEquals(intervals, l); } @Test - public void testSerialization() throws Exception - { - List<Interval<Integer, String>> intervals = new ArrayList<Interval<Integer, String>>(); - - intervals.add(Interval.<Integer, String>create(-300, -200, "a")); - intervals.add(Interval.<Integer, String>create(-3, -2, "b")); - intervals.add(Interval.<Integer, String>create(1, 2, "c")); - intervals.add(Interval.<Integer, String>create(1, 3, "d")); - intervals.add(Interval.<Integer, String>create(2, 4, "e")); - intervals.add(Interval.<Integer, String>create(3, 6, "f")); - intervals.add(Interval.<Integer, String>create(4, 6, "g")); - intervals.add(Interval.<Integer, String>create(5, 7, "h")); - intervals.add(Interval.<Integer, String>create(8, 9, "i")); - intervals.add(Interval.<Integer, String>create(15, 20, "j")); - intervals.add(Interval.<Integer, String>create(40, 50, "k")); - intervals.add(Interval.<Integer, String>create(49, 60, "l")); - - IntervalTree<Integer, String, Interval<Integer, String>> it = IntervalTree.build(intervals); - - IVersionedSerializer<IntervalTree<Integer, String, Interval<Integer, String>>> serializer = IntervalTree.serializer( - new ISerializer<Integer>() + public void testEmptyTree() + { + IntervalTree<Integer, String, Interval<Integer, String>> emptyTree = IntervalTree.emptyTree(); + + assertTrue("Tree should be empty", emptyTree.isEmpty()); + assertEquals("Interval count should be 0", 0, emptyTree.intervalCount()); + + try + { + emptyTree.min(); + fail("Expected IllegalStateException when calling min() on empty tree"); + } + catch (IllegalStateException e) { /* expected */ } + + try + { + emptyTree.max(); + fail("Expected IllegalStateException when calling max() on empty tree"); + } + catch (IllegalStateException e) { /* expected */ } + + assertTrue("Search should yield empty list for empty tree", + emptyTree.search(Interval.create(1, 5, "data")).isEmpty()); + assertTrue("Search by point should yield empty list for empty tree", + emptyTree.search(5).isEmpty()); + + assertFalse("Iterator should have no elements", emptyTree.iterator().hasNext()); + } + + @Test + public void testSingleInterval() + { + Interval<Integer, String> interval = Interval.create(10, 20, "single"); + List<Interval<Integer, String>> list = new ArrayList<>(); + list.add(interval); + + IntervalTree<Integer, String, Interval<Integer, String>> tree = IntervalTree.build(list); + + assertFalse("Tree should not be empty", tree.isEmpty()); + assertEquals("Interval count should be 1", 1, tree.intervalCount()); + assertEquals("min() should match the only interval's min", Integer.valueOf(10), tree.min()); + assertEquals("max() should match the only interval's max", Integer.valueOf(20), tree.max()); + + List<String> result = tree.search(Interval.create(10, 20)); + assertEquals("Should find exactly 1 match", 1, result.size()); + assertEquals("Data should match 'single'", "single", result.get(0)); + + result = tree.search(Interval.create(15, 25)); + assertEquals("Should find overlap", 1, result.size()); + + result = tree.search(Interval.create(1, 9)); + assertTrue("Should find no intervals that do not overlap", result.isEmpty()); + + List<Interval<Integer, String>> iterationList = ImmutableList.copyOf(tree); + + assertEquals("Iteration should produce exactly our single interval", + list, iterationList); + } + + @Test + public void testMultipleIntervals() + { + List<Interval<Integer, String>> intervals = new ArrayList<>(); + intervals.add(Interval.create(1, 3, "A")); + intervals.add(Interval.create(2, 4, "B")); + intervals.add(Interval.create(5, 7, "C")); + intervals.add(Interval.create(6, 6, "D")); // single-point overlap within (5,7) + intervals.add(Interval.create(8, 10, "E")); + intervals.add(Interval.create(10, 12, "F")); // boundary adjacency with (8,10) + + IntervalTree<Integer, String, Interval<Integer, String>> tree = IntervalTree.build(intervals); + + assertFalse("Tree should not be empty", tree.isEmpty()); + assertEquals("Interval count", intervals.size(), tree.intervalCount()); + + assertEquals("min()", Integer.valueOf(1), tree.min()); + assertEquals("max()", Integer.valueOf(12), tree.max()); + + List<String> result = tree.search(Interval.create(2, 2)); // point in [1,3] and also in [2,4] + assertTrue(result.contains("A")); + assertTrue(result.contains("B")); + assertEquals("Should find 2 intervals for point=2", 2, result.size()); + + result = tree.search(Interval.create(4, 6)); + assertTrue(result.contains("B")); + assertTrue(result.contains("C")); + assertTrue(result.contains("D")); + assertEquals("Should have 3 overlaps for [4,6]", 3, result.size()); + + result = tree.search(Interval.create(13, 14)); + assertTrue("Should be no matches for [13,14]", result.isEmpty()); + + result = tree.search(Interval.create(10, 10)); + assertTrue(result.contains("E")); + assertTrue(result.contains("F")); + assertEquals("Should find 2 intervals at boundary=10", 2, result.size()); + + Collections.sort(intervals, Interval.minOrdering()); + List<Interval<Integer, String>> iterated = ImmutableList.copyOf(tree); + assertEquals("Iterated intervals should be in ascending min-order", intervals, iterated); + } + + @Test + public void testDuplicateAndSameBoundaryIntervals() + { + List<Interval<Integer, String>> intervals = new ArrayList<>(); + intervals.add(Interval.create(5, 5, "X")); // single point + intervals.add(Interval.create(5, 5, "Y")); // same single point, different data + intervals.add(Interval.create(5, 5, "X")); // same data and boundary as first + intervals.add(Interval.create(5, 6, "Z")); // partial overlap + intervals.add(Interval.create(4, 5, "W")); // partial overlap at boundary + + IntervalTree<Integer, String, Interval<Integer, String>> tree = IntervalTree.build(intervals); + + assertEquals("min()", Integer.valueOf(4), tree.min()); + assertEquals("max()", Integer.valueOf(6), tree.max()); + + List<String> result = tree.search(5); + assertEquals("Should have 5 matching intervals that contain point=5", 5, result.size()); + + result = tree.search(Interval.create(5, 5)); + assertEquals("Should have 5 matching intervals that contain [5,5]", 5, result.size()); + + result = tree.search(Interval.create(6, 6)); + assertEquals("Should match only [5,6] for point=6", 1, result.size()); + assertTrue(result.contains("Z")); + + result = tree.search(7); + assertTrue("No intervals should contain point=7", result.isEmpty()); + } + + @Test + public void testEqualsAndHashCode() + { + List<Interval<Integer, String>> intervals1 = new ArrayList<>(); + intervals1.add(Interval.create(1, 2, "A")); + intervals1.add(Interval.create(3, 5, "B")); + intervals1.add(Interval.create(4, 6, "C")); + + List<Interval<Integer, String>> intervals2 = new ArrayList<>(); + // same intervals but in different order + intervals2.add(Interval.create(4, 6, "C")); + intervals2.add(Interval.create(1, 2, "A")); + intervals2.add(Interval.create(3, 5, "B")); + + IntervalTree<Integer, String, Interval<Integer, String>> tree1 = IntervalTree.build(intervals1); + IntervalTree<Integer, String, Interval<Integer, String>> tree2 = IntervalTree.build(intervals2); + + assertTrue("tree1 should be equal to tree2 despite different input order", tree1.equals(tree2)); + assertEquals("tree1.hashCode() should match tree2.hashCode()", tree1.hashCode(), tree2.hashCode()); + + // Create a slightly different set + List<Interval<Integer, String>> intervals3 = new ArrayList<>(intervals1); + intervals3.add(Interval.create(10, 11, "X")); // extra interval + + IntervalTree<Integer, String, Interval<Integer, String>> tree3 = IntervalTree.build(intervals3); + assertFalse("tree1 should not equal tree3 which has an extra interval", tree1.equals(tree3)); + assertNotEquals("hashCode should differ if intervals differ", tree1.hashCode(), tree3.hashCode()); + } + + @Test + public void testPointSearchEquivalence() + { + List<Interval<Integer, String>> intervals = new ArrayList<>(); + intervals.add(Interval.create(1, 3, "A")); + intervals.add(Interval.create(2, 5, "B")); + intervals.add(Interval.create(10, 20, "C")); + + IntervalTree<Integer, String, Interval<Integer, String>> tree = IntervalTree.build(intervals); + + List<String> resultPoint = tree.search(2); + List<String> resultInterval = tree.search(Interval.create(2, 2)); + + assertEquals("Results of point search and interval-based search should match", + resultPoint, resultInterval); + } + + private Gen<Interval<Integer, String>> intervalGen() + { + AtomicInteger id = new AtomicInteger(); + return SourceDSL.integers().between(-5, 5) + .flatMap(start -> + SourceDSL.integers().between(-5, 5) + .map(end -> { + int lo = Math.min(start, end); + int hi = Math.max(start, end); + String data = "(" + lo + "," + hi + "," + id.getAndIncrement() + ")"; + return Interval.create(lo, hi, data); + })); + } + + private Gen<List<Interval<Integer, String>>> intervalsListGen() + { + return lists().of(intervalGen()) + .ofSizeBetween(0, 7); + } + + private Gen<Interval<Integer, String>> queryGen() + { + return SourceDSL.booleans().all() + .flatMap(isPoint -> { + if (isPoint) + { + return SourceDSL.integers().between(-5, 5) + .map(x -> Interval.create(x, x, "queryPoint(" + x + ")")); + } + else + { + return intervalGen().map(i -> Interval.create(i.min, i.max, "query[" + i.min + "," + i.max + "]")); + } + }); + } + + private boolean overlaps(Interval<Integer, ?> a, Interval<Integer, ?> b) + { + return a.min <= b.max && a.max >= b.min; + } + + private <D> List<D> search(Collection<Interval<Integer, D>> intervals, Interval<Integer, ?> query) + { + List<D> results = new ArrayList<>(); + for (Interval<Integer, D> candidate : intervals) + { + if (overlaps(candidate, query)) + results.add(candidate.data); + } + return results; + } + + @Test + public void qtIntervalTreeTest() + { + qt().forAll(intervalsListGen(), queryGen()) + .check((intervals, query) -> { + IntervalTree<Integer, String, Interval<Integer, String>> tree = IntervalTree.build(intervals); + + List<String> expected = search(intervals, query); + List<String> actual = tree.search(query); + + Set<String> setExpected = new HashSet<>(expected); + Set<String> setActual = new HashSet<>(actual); + + assertEquals(setExpected, setActual); + + if (query.min.equals(query.max)) { - public void serialize(Integer i, DataOutputPlus out) throws IOException - { - out.writeInt(i); - } + List<String> actualPoint = tree.search(query.min); + assertEquals(setExpected, new HashSet<>(actualPoint)); + } - public Integer deserialize(DataInputPlus in) throws IOException - { - return in.readInt(); - } + Set<Interval<Integer, String>> fromTree = ImmutableSet.copyOf(tree); - public long serializedSize(Integer i) - { - return 4; - } - }, - new ISerializer<String>() + assertEquals(intervals.size(), fromTree.size()); + Set<Interval<Integer, String>> original = ImmutableSet.copyOf(intervals); + + assertEquals(original, fromTree); + + IntervalTree<Integer, String, Interval<Integer, String>> tree2 = IntervalTree.build(intervals); + assertEquals(tree, tree2); + assertEquals(tree.hashCode(), tree2.hashCode()); + + return true; + }); + } + + @Test + public void qtUpdateFunctionTest() + { + qt().withExamples(-1).withTestingTime(30, SECONDS).forAll(intervalsListGen(), + intervalsListGen(), + SourceDSL.lists().of(queryGen()).ofSizeBetween(1, 4), + SourceDSL.integers().all()) + .check((original, toAdd, queries, seed) -> { + IntervalTree<Integer, String, Interval<Integer, String>> originalTree = IntervalTree.build(original); + + java.util.Random rng = new java.util.Random(seed); + + List<Interval<Integer, String>> removals = new ArrayList<>(); + for (Interval<Integer, String> candidate : original) { - public void serialize(String v, DataOutputPlus out) throws IOException - { - out.writeUTF(v); - } + if (rng.nextBoolean()) + removals.add(candidate); + } - public String deserialize(DataInputPlus in) throws IOException - { - return in.readUTF(); - } + toAdd.removeAll(original.stream().filter(not(removals::contains)).collect(Collectors.toList())); - public long serializedSize(String v) - { - return v.length(); - } - }, - (Constructor<Interval<Integer, String>>) (Object) Interval.class.getConstructor(Object.class, Object.class, Object.class) - ); + IntervalTree<Integer, String, Interval<Integer, String>> updatedTree = originalTree.update(removals.toArray(new Interval[0]), toAdd.toArray(new Interval[0])); + + Set<Interval<Integer, String>> naiveFinal = new HashSet<>(original); + naiveFinal.removeAll(removals); + naiveFinal.addAll(toAdd); - DataOutputBuffer out = new DataOutputBuffer(); + Set<Interval<Integer, String>> iteratedTree = ImmutableSet.copyOf(updatedTree); + if (!naiveFinal.equals(iteratedTree)) + originalTree.update(removals.toArray(new Interval[0]), toAdd.toArray(new Interval[0])); + assertEquals(naiveFinal, iteratedTree); - serializer.serialize(it, out, 0); + for (Interval<Integer, String> query : queries) + { + Set<String> actualResults = ImmutableSet.copyOf(updatedTree.search(query)); + Set<String> expectedResults = ImmutableSet.copyOf(search(naiveFinal, query)); + + if (!expectedResults.equals(actualResults)) + { + originalTree.update(removals.toArray(new Interval[0]), toAdd.toArray(new Interval[0])); + updatedTree.search(query); + } - DataInputPlus in = new DataInputBuffer(out.toByteArray()); + assertEquals(expectedResults, actualResults); - IntervalTree<Integer, String, Interval<Integer, String>> it2 = serializer.deserialize(in, 0); - List<Interval<Integer, String>> intervals2 = new ArrayList<Interval<Integer, String>>(); - for (Interval<Integer, String> i : it2) - intervals2.add(i); + if (query.min.equals(query.max)) + { + List<String> pointResults = updatedTree.search(query.min); + assertEquals(new HashSet<>(expectedResults), new HashSet<>(pointResults)); + } + } - assertEquals(intervals, intervals2); + return true; + }); } -} +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/utils/OverlapIteratorTest.java b/test/unit/org/apache/cassandra/utils/OverlapIteratorTest.java index 6f9960338de9..11f38c09d817 100644 --- a/test/unit/org/apache/cassandra/utils/OverlapIteratorTest.java +++ b/test/unit/org/apache/cassandra/utils/OverlapIteratorTest.java @@ -76,7 +76,7 @@ private void test(int range, int increment, int count) compare(randomIntervals(range, increment, count), random(range, increment, count), 3); } - private <I extends Comparable<I>, V> void compare(List<Interval<I, V>> intervals, List<I> points, int initCount) + private <I extends Comparable<I>, V extends Comparable<V>> void compare(List<Interval<I, V>> intervals, List<I> points, int initCount) { Collections.sort(points); IntervalTree<I, V, Interval<I, V>> tree = IntervalTree.build(intervals); @@ -98,5 +98,4 @@ private <I extends Comparable<I>, V> void compare(List<Interval<I, V>> intervals assertTrue(missing.isEmpty()); } } - } From 4edf79a26349d8b4510d76a187c81344f25d64ff Mon Sep 17 00:00:00 2001 From: Ariel Weisberg <aweisberg@apple.com> Date: Tue, 4 Mar 2025 18:57:33 -0500 Subject: [PATCH 166/225] Ninja add missing license to IntervalTreeTest --- .../cassandra/utils/IntervalTreeTest.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/test/unit/org/apache/cassandra/utils/IntervalTreeTest.java b/test/unit/org/apache/cassandra/utils/IntervalTreeTest.java index 7160e81d3f70..14e70c0b001e 100644 --- a/test/unit/org/apache/cassandra/utils/IntervalTreeTest.java +++ b/test/unit/org/apache/cassandra/utils/IntervalTreeTest.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.cassandra.utils; import java.util.ArrayList; From 58b7ebfa4b18092f52ba6ef21efe73084e0ee1f1 Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi <contacto@bernardobotella.com> Date: Mon, 17 Feb 2025 15:44:03 -0800 Subject: [PATCH 167/225] Add OCTET_LENGTH constraint patch by Bernardo Botella; reviewed by Stefan Miklosovic for CASSANDRA-20340 --- CHANGES.txt | 1 + .../pages/developing/cql/constraints.adoc | 26 + .../cql3/constraints/ConstraintFunction.java | 4 +- .../constraints/FunctionColumnConstraint.java | 3 +- .../cql3/constraints/JsonConstraint.java | 6 - .../cql3/constraints/LengthConstraint.java | 6 - .../constraints/OctetLengthConstraint.java | 82 +++ ...mnOctetLengthConstraintValidationTest.java | 566 ++++++++++++++++++ 8 files changed, 680 insertions(+), 14 deletions(-) create mode 100644 src/java/org/apache/cassandra/cql3/constraints/OctetLengthConstraint.java create mode 100644 test/unit/org/apache/cassandra/contraints/CreateTableWithColumnOctetLengthConstraintValidationTest.java diff --git a/CHANGES.txt b/CHANGES.txt index b91aca2cb7bf..58d702a7b841 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Add OCTET_LENGTH constraint (CASSANDRA-20340) * Reduce memory allocations in miscellaneous places along the hot write path (CASSANDRA-20167) * Provide keystore_password_file and truststore_password_file options to read credentials from a file (CASSANDRA-13428) * Unregistering a node should also remove it from tokenMap if it is there and recalculate the placements (CASSANDRA-20346) diff --git a/doc/modules/cassandra/pages/developing/cql/constraints.adoc b/doc/modules/cassandra/pages/developing/cql/constraints.adoc index 0bd3120edc33..f768d4fd8ff7 100644 --- a/doc/modules/cassandra/pages/developing/cql/constraints.adoc +++ b/doc/modules/cassandra/pages/developing/cql/constraints.adoc @@ -92,6 +92,32 @@ Finally, the constraint can be removed: ALTER TABLE keyspace.table ALTER name DROP CHECK; ---- +=== OCTET_LENGTH CONSTRAINT + +Defines a condition that checks the size in bytes of text or binary type. + +For example, we can create a constraint that checks that name can't be bigger than 256 characters: + +---- +CREATE TABLE keyspace.table ( + name text CHECK OCTET_LENGTH(name) < 2 + ..., +); +---- + +Inserting a valid row: +---- +INSERT INTO keyspace.table (name) VALUES ("f") +---- + +Inserting an invalid row: +---- +INSERT INTO keyspace.table (name) VALUES ("fooooooo") + +ERROR: Column value does not satisfy value constraint for column 'name'. It has a length of 8 and +and it should be should be < 2 +---- + === NOT_NULL constraint Defines a constraint that checks if a column is not null in every modification statement. diff --git a/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java b/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java index a95a4f782dd2..1dda89093f4c 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java +++ b/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java @@ -79,7 +79,9 @@ public void evaluate(AbstractType<?> valueType, ByteBuffer columnValue) throws C * Method that validates that a condition is valid. This method is called when the CQL constraint is created to determine * if the CQL statement is valid or needs to be rejected as invalid throwing a {@link InvalidConstraintDefinitionException} */ - public abstract void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException; + public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException + { + } /** * Return operators this function supports. By default, it returns an empty list, modelling unary function. diff --git a/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java index 2383f4ee9c4d..dac62ddfc69a 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java @@ -77,7 +77,8 @@ public static SatisfiabilityChecker[] getSatisfiabilityCheckers() public enum Functions { - LENGTH(LengthConstraint::new); + LENGTH(LengthConstraint::new), + OCTET_LENGTH(OctetLengthConstraint::new); private final Function<ColumnIdentifier, ConstraintFunction> functionCreator; diff --git a/src/java/org/apache/cassandra/cql3/constraints/JsonConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/JsonConstraint.java index 62c961743778..99aeb6734e63 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/JsonConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/JsonConstraint.java @@ -26,7 +26,6 @@ import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.AsciiType; import org.apache.cassandra.db.marshal.UTF8Type; -import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.utils.JsonUtils; @@ -63,11 +62,6 @@ public void internalEvaluate(AbstractType<?> valueType, Operator relationType, S } } - @Override - public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException - { - } - @Override public List<AbstractType<?>> getSupportedTypes() { diff --git a/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java index 5b5d8c97dd8d..49954c28fb93 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/LengthConstraint.java @@ -28,7 +28,6 @@ import org.apache.cassandra.db.marshal.BytesType; import org.apache.cassandra.db.marshal.Int32Type; import org.apache.cassandra.db.marshal.UTF8Type; -import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.utils.ByteBufferUtil; public class LengthConstraint extends ConstraintFunction @@ -56,11 +55,6 @@ public void internalEvaluate(AbstractType<?> valueType, Operator relationType, S + relationType + ' ' + term); } - @Override - public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException - { - } - @Override public List<Operator> getSupportedOperators() { diff --git a/src/java/org/apache/cassandra/cql3/constraints/OctetLengthConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/OctetLengthConstraint.java new file mode 100644 index 000000000000..8147d37d62d2 --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/constraints/OctetLengthConstraint.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.constraints; + +import java.nio.ByteBuffer; +import java.util.List; + +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.AsciiType; +import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.utils.ByteBufferUtil; + +public class OctetLengthConstraint extends ConstraintFunction +{ + private static final List<AbstractType<?>> SUPPORTED_TYPES = List.of(BytesType.instance, UTF8Type.instance, AsciiType.instance); + + public OctetLengthConstraint(ColumnIdentifier columnName) + { + super(columnName, "OCTET_LENGTH"); + } + + @Override + public void internalEvaluate(AbstractType<?> valueType, Operator relationType, String term, ByteBuffer columnValue) + { + int valueLength = columnValue.remaining(); + int sizeConstraint = Integer.parseInt(term); + + ByteBuffer leftOperand = ByteBufferUtil.bytes(valueLength); + ByteBuffer rightOperand = ByteBufferUtil.bytes(sizeConstraint); + + if (!relationType.isSatisfiedBy(Int32Type.instance, leftOperand, rightOperand)) + throw new ConstraintViolationException("Column value does not satisfy value constraint for column '" + columnName + "'. " + + "It has a length of " + valueLength + " and it should be should be " + + relationType + ' ' + term); + } + + @Override + public List<Operator> getSupportedOperators() + { + return DEFAULT_FUNCTION_OPERATORS; + } + + @Override + public List<AbstractType<?>> getSupportedTypes() + { + return SUPPORTED_TYPES; + } + + @Override + public boolean equals(Object o) + { + if (this == o) + return true; + + if (!(o instanceof OctetLengthConstraint)) + return false; + + OctetLengthConstraint other = (OctetLengthConstraint) o; + + return columnName.equals(other.columnName); + } +} diff --git a/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnOctetLengthConstraintValidationTest.java b/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnOctetLengthConstraintValidationTest.java new file mode 100644 index 000000000000..5c79c957a513 --- /dev/null +++ b/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnOctetLengthConstraintValidationTest.java @@ -0,0 +1,566 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.contraints; + + +import java.util.Arrays; +import java.util.Collection; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.utils.Generators; + +import static accord.utils.Property.qt; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.quicktheories.generators.SourceDSL.integers; + +@RunWith(Parameterized.class) +public class CreateTableWithColumnOctetLengthConstraintValidationTest extends CqlConstraintValidationTester +{ + + @Parameterized.Parameter + public String order; + + @Parameterized.Parameters() + public static Collection<Object[]> generateData() + { + return Arrays.asList(new Object[][]{ + { "ASC" }, + { "DESC" } + }); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnSerializedSizeEqualToConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 text CHECK OCTET_LENGTH(ck1) = 4, ck2 int, v int, PRIMARY KEY ((pk), ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fooo', 2, 3)"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fño', 2, 3)"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foo', 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fooñ', 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'foooo', 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnSerializedSizeDifferentThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 text CHECK OCTET_LENGTH(ck1) != 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fñ', 2, 3)"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fñoo', 2, 3)"); + + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fño', 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnSerializedSizeBiggerThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 text CHECK OCTET_LENGTH(ck1) > 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fñoo', 2, 3)"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fñ', 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fño', 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnSerializedSizeBiggerOrEqualThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 text CHECK OCTET_LENGTH(ck1) >= 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fñoo', 2, 3)"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fño', 2, 3)"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fñ', 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnSerializedSizeSmallerThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 text CHECK OCTET_LENGTH(ck1) < 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fñ', 2, 3)"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fño', 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fñoo', 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnSerializedSizeSmallerOrEqualThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 text CHECK OCTET_LENGTH(ck1) <= 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fñ', 2, 3)"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fño', 2, 3)"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 'fñoo', 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringBlobColumnSerializedSizeEqualToConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 blob CHECK OCTET_LENGTH(ck1) = 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fño'), 2, 3)"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fñ'), 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fñoo'), 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringBlobColumnSerializedSizeDifferentThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 blob CHECK OCTET_LENGTH(ck1) != 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fñ'), 2, 3)"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fñoo'), 2, 3)"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fño'), 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringBlobColumnSerializedSizeBiggerThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 blob CHECK OCTET_LENGTH(ck1) > 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fñoo'), 2, 3)"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fñ'), 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fño'), 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringBlobColumnSerializedSizeBiggerOrEqualThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 blob CHECK OCTET_LENGTH(ck1) >= 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fñoo'), 2, 3)"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fño'), 2, 3)"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fñ'), 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringBlobColumnSerializedSizeSmallerThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 blob CHECK OCTET_LENGTH(ck1) < 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fñ'), 2, 3)"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fño'), 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fñoo'), 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithClusteringBlobColumnSerializedSizeSmallerOrEqualThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 blob CHECK OCTET_LENGTH(ck1) <= 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fñ'), 2, 3)"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fño'), 2, 3)"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'ck1'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, textAsBlob('fñoo'), 2, 3)"); + } + + + @Test + public void testCreateTableWithColumnWithPkColumnSerializedSizeEqualToConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk text CHECK OCTET_LENGTH(pk) = 4, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fño', 1, 2, 3)"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'pk'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fñ', 1, 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fñoo', 1, 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithPkColumnSerializedSizeDifferentThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk text CHECK OCTET_LENGTH(pk) != 4, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fñ', 1, 2, 3)"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fñoo', 1, 2, 3)"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'pk'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fño', 1, 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithPkColumnSerializedSizeBiggerThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk text CHECK OCTET_LENGTH(pk) > 4, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fñoo', 1, 2, 3)"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'pk'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fñ', 1, 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fño', 1, 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithPkColumnSerializedSizeBiggerOrEqualThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk text CHECK OCTET_LENGTH(pk) >= 4, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fñoo', 1, 2, 3)"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fño', 1, 2, 3)"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'pk'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fñ', 1, 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithPkColumnSerializedSizeSmallerThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk text CHECK OCTET_LENGTH(pk) < 4, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fñ', 1, 2, 3)"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'pk'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fño', 1, 2, 3)"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fñoo', 1, 2, 3)"); + } + + @Test + public void testCreateTableWithColumnWithPkColumnSerializedSizeSmallerOrEqualThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk text CHECK OCTET_LENGTH(pk) <= 4, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fñ', 1, 2, 3)"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fño', 1, 2, 3)"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'pk'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fñoo', 1, 2, 3)"); + } + + + @Test + public void testCreateTableWithColumnWithRegularColumnSerializedSizeEqualToConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v text CHECK OCTET_LENGTH(v) = 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fño')"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fñ')"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fñoo')"); + } + + @Test + public void testCreateTableWithColumnWithRegularColumnSerializedSizeDifferentThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v text CHECK OCTET_LENGTH(v) != 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fñ')"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fñoo')"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fño')"); + } + + @Test + public void testCreateTableWithColumnWithRegularColumnSerializedSizeBiggerThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v text CHECK OCTET_LENGTH(v) > 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fñoo')"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fñ')"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fño')"); + } + + @Test + public void testCreateTableWithColumnWithRegularColumnSerializedSizeBiggerOrEqualThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v text CHECK OCTET_LENGTH(v) >= 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fñoo')"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fño')"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fñ')"); + } + + @Test + public void testCreateTableWithColumnWithRegularColumnSerializedSizeSmallerThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v text CHECK OCTET_LENGTH(v) < 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fñ')"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fño')"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fñoo')"); + } + + @Test + public void testCreateTableWithColumnWithRegularColumnSerializedSizeSmallerOrEqualThanConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v text CHECK OCTET_LENGTH(v) <= 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fñ')"); + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fño')"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, 'fñoo')"); + } + + @Test + public void testCreateTableWithColumnWithRegularColumnSerializedSizeCheckNullTextConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v text CHECK OCTET_LENGTH(v) <= 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v' as it is null."; + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, null)"); + } + + @Test + public void testCreateTableWithColumnWithRegularColumnSerializedSizeCheckNullVarcharConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v varchar CHECK OCTET_LENGTH(v) <= 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v' as it is null."; + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, null)"); + } + + @Test + public void testCreateTableWithColumnWithRegularColumnSerializedSizeCheckNullAsciiConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v ascii CHECK OCTET_LENGTH(v) <= 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v' as it is null."; + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, null)"); + } + + @Test + public void testCreateTableWithColumnWithRegularColumnSerializedSizeCheckNullBlobConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v blob CHECK OCTET_LENGTH(v) <= 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'v' as it is null."; + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES (1, 2, 3, null)"); + } + + @Test + public void testCreateTableWithColumnMixedColumnsSerializedSizeConstraint() throws Throwable + { + createTable("CREATE TABLE %s (pk text CHECK OCTET_LENGTH(pk) = 4, ck1 int, ck2 int, v text CHECK OCTET_LENGTH(v) = 4, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + + // Valid + execute("INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fño', 2, 3, 'fño')"); + + final String expectedErrorMessage = "Column value does not satisfy value constraint for column 'pk'. It has a length of"; + final String expectedErrorMessage2 = "Column value does not satisfy value constraint for column 'v'. It has a length of"; + // Invalid + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fñ', 2, 3, 'fñ')"); + assertInvalidThrowMessage(expectedErrorMessage2, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fño', 2, 3, 'fñ')"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fñ', 2, 3, 'fño')"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fñoo', 2, 3, 'fño')"); + assertInvalidThrowMessage(expectedErrorMessage2, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fño', 2, 3, 'fñoo')"); + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "INSERT INTO %s (pk, ck1, ck2, v) VALUES ('fñoo', 2, 3, 'fñoo')"); + } + + @Test + public void testCreateTableWithWrongColumnConstraint() throws Throwable + { + try + { + createTable("CREATE TABLE %s (pk text, ck1 int CHECK OCTET_LENGTH(pk) = 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + fail(); + } + catch (InvalidRequestException e) + { + assertTrue(e.getCause() instanceof InvalidRequestException); + assertTrue(e.getMessage().contains("Error setting schema for test")); + } + } + + @Test + public void testCreateTableWithWrongColumnMultipleConstraint() throws Throwable + { + try + { + createTable("CREATE TABLE %s (pk text, ck1 int CHECK OCTET_LENGTH(pk) = 4 AND ck1 < 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + fail(); + } + catch (InvalidRequestException e) + { + assertTrue(e.getCause() instanceof InvalidRequestException); + assertTrue(e.getMessage().contains("Error setting schema for test")); + } + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnInvalidTypeConstraint() throws Throwable + { + try + { + createTable("CREATE TABLE %s (pk int, ck1 int CHECK OCTET_LENGTH(ck1) = 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + fail(); + } + catch (InvalidRequestException e) + { + assertTrue(e.getCause() instanceof InvalidRequestException); + assertTrue(e.getMessage().contains("Error setting schema for test")); + } + } + + @Test + public void testCreateTableWithColumnWithClusteringColumnInvalidScalarTypeConstraint() throws Throwable + { + try + { + createTable("CREATE TABLE %s (pk text CHECK pk = 4, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + fail(); + } + catch (InvalidRequestException e) + { + assertTrue(e.getCause() instanceof InvalidRequestException); + assertTrue(e.getCause().getMessage().contains("Constraint 'pk =' can be used only for columns of type")); + assertTrue(e.getMessage().contains("Error setting schema for test")); + } + } + + @Test + public void testCreateTableInvalidFunction() throws Throwable + { + try + { + createTable("CREATE TABLE %s (pk text CHECK not_a_function(pk) = 4, ck1 int, ck2 int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");"); + fail(); + } + catch (InvalidRequestException e) + { + assertTrue(e.getCause() instanceof InvalidRequestException); + assertTrue(e.getMessage().contains("Error setting schema for test")); + } + } + + @Test + public void testCreateTableWithPKConstraintsAndCDCEnabled() throws Throwable + { + // It works + createTable("CREATE TABLE %s (pk text CHECK length(pk) = 4, ck1 int, ck2 int, PRIMARY KEY ((pk), ck1, ck2)) WITH cdc = true;"); + } + + @Test + public void testCreateTableWithClusteringConstraintsAndCDCEnabled() throws Throwable + { + // It works + createTable("CREATE TABLE %s (pk text, ck1 int CHECK ck1 < 100, ck2 int, PRIMARY KEY ((pk), ck1, ck2)) WITH cdc = true;"); + } + + @Test + public void testCreateTableWithRegularConstraintsAndCDCEnabled() throws Throwable + { + // It works + createTable("CREATE TABLE %s (pk text, ck1 int CHECK ck1 < 100, ck2 int, PRIMARY KEY (pk)) WITH cdc = true;"); + } + + // Copy table with like + @Test + public void testCreateTableWithColumnWithClusteringColumnLessThanScalarConstraintIntegerOnLikeTable() throws Throwable + { + createTable(KEYSPACE, "CREATE TABLE %s (pk int, ck1 int CHECK ck1 < 4, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 " + order + ");", "liketabletame"); + + execute("create table " + KEYSPACE + ".tb_copy like %s"); + + // Valid + qt().forAll(Generators.toGen(integers().between(0, 3))) + .check(d -> execute("INSERT INTO " + KEYSPACE + ".tb_copy (pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)")); + + // Invalid + qt().forAll(Generators.toGen(integers().between(4, 100))) + .check(d -> { + try + { + assertInvalidThrow(InvalidRequestException.class, "INSERT INTO " + KEYSPACE + ".tb_copy(pk, ck1, ck2, v) VALUES (1, " + d + ", 3, 4)"); + } + catch (Throwable e) + { + throw new RuntimeException(e); + } + }); + } +} From a09d5240abc4ed2b6a6a3f77ad8a87d55219f1bd Mon Sep 17 00:00:00 2001 From: Jordan West <jordan@moonwest.dev> Date: Wed, 17 Apr 2024 10:27:02 -0700 Subject: [PATCH 168/225] Add read ahead buffer for scans of compressed data files Patch by Jordan West, Jon Haddad; Reviewed by David Capwell, Caleb Rackliffe, Dmitry Konstantinov for CASSANDRA-15452 --- .../apache/cassandra/cache/ChunkCache.java | 2 +- .../org/apache/cassandra/config/Config.java | 2 + .../cassandra/config/DatabaseDescriptor.java | 21 ++ .../io/sstable/format/SSTableReader.java | 5 + .../io/sstable/format/SSTableScanner.java | 2 +- .../io/util/BufferManagingRebufferer.java | 1 + .../apache/cassandra/io/util/ChunkReader.java | 2 + .../io/util/CompressedChunkReader.java | 202 ++++++++++++++--- .../cassandra/io/util/EmptyRebufferer.java | 2 +- .../apache/cassandra/io/util/FileHandle.java | 19 +- .../cassandra/io/util/MmapRebufferer.java | 2 +- .../cassandra/io/util/RandomAccessReader.java | 2 +- .../cassandra/io/util/RebuffererFactory.java | 2 +- .../cassandra/io/util/SimpleChunkReader.java | 2 +- .../io/util/ThreadLocalReadAheadBuffer.java | 154 +++++++++++++ .../cassandra/service/StorageService.java | 14 ++ .../service/StorageServiceMBean.java | 3 + .../io/util/CompressedChunkReaderTest.java | 141 ++++++++++++ .../util/ThreadLocalReadAheadBufferTest.java | 204 ++++++++++++++++++ 19 files changed, 744 insertions(+), 38 deletions(-) create mode 100644 src/java/org/apache/cassandra/io/util/ThreadLocalReadAheadBuffer.java create mode 100644 test/unit/org/apache/cassandra/io/util/CompressedChunkReaderTest.java create mode 100644 test/unit/org/apache/cassandra/io/util/ThreadLocalReadAheadBufferTest.java diff --git a/src/java/org/apache/cassandra/cache/ChunkCache.java b/src/java/org/apache/cassandra/cache/ChunkCache.java index 8f22ee8b7b15..e7e50296accf 100644 --- a/src/java/org/apache/cassandra/cache/ChunkCache.java +++ b/src/java/org/apache/cassandra/cache/ChunkCache.java @@ -251,7 +251,7 @@ public void invalidate(long position) } @Override - public Rebufferer instantiateRebufferer() + public Rebufferer instantiateRebufferer(boolean isScan) { return this; } diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 779178795c6d..ca62c51bc8d6 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -337,6 +337,8 @@ public MemtableOptions() @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"); + public DataStorageSpec.IntKibibytesBound compressed_read_ahead_buffer_size = new DataStorageSpec.IntKibibytesBound("256KiB"); + // fraction of free disk space available for compaction after min free space is subtracted public volatile Double max_space_usable_for_compactions_in_percentage = .95; diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 6de2c7473d8d..bc58fe151ba3 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -954,6 +954,9 @@ else if (conf.max_value_size.toMebibytes() >= 2048) break; } + if (conf.compressed_read_ahead_buffer_size.toKibibytes() > 0 && conf.compressed_read_ahead_buffer_size.toKibibytes() < 256) + throw new ConfigurationException("compressed_read_ahead_buffer_size must be at least 256KiB (set to 0 to disable), but was " + conf.compressed_read_ahead_buffer_size, false); + if (conf.server_encryption_options != null) { conf.server_encryption_options.applyConfig(); @@ -2534,6 +2537,24 @@ public static void setConcurrentViewBuilders(int value) conf.concurrent_materialized_view_builders = value; } + public static int getCompressedReadAheadBufferSize() + { + return conf.compressed_read_ahead_buffer_size.toBytes(); + } + + public static int getCompressedReadAheadBufferSizeInKB() + { + return conf.compressed_read_ahead_buffer_size.toKibibytes(); + } + + public static void setCompressedReadAheadBufferSizeInKb(int sizeInKb) + { + if (sizeInKb < 256) + throw new IllegalArgumentException("compressed_read_ahead_buffer_size_in_kb must be at least 256KiB"); + + conf.compressed_read_ahead_buffer_size = createIntKibibyteBoundAndEnsureItIsValidForByteConversion(sizeInKb, "compressed_read_ahead_buffer_size"); + } + public static long getMinFreeSpacePerDriveInMebibytes() { return conf.min_free_space_per_drive.toMebibytes(); diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java index ececa03dfa78..1824412d9bbe 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java @@ -1266,6 +1266,11 @@ public RandomAccessReader openDataReader() return dfile.createReader(); } + public RandomAccessReader openDataReaderForScan() + { + return dfile.createReaderForScan(); + } + public void trySkipFileCacheBefore(DecoratedKey key) { long position = getPosition(key, SSTableReader.Operator.GE); diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableScanner.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableScanner.java index 217c17720639..5136a06bca46 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableScanner.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableScanner.java @@ -78,7 +78,7 @@ protected SSTableScanner(S sstable, { assert sstable != null; - this.dfile = sstable.openDataReader(); + this.dfile = sstable.openDataReaderForScan(); this.sstable = sstable; this.columns = columns; this.dataRange = dataRange; diff --git a/src/java/org/apache/cassandra/io/util/BufferManagingRebufferer.java b/src/java/org/apache/cassandra/io/util/BufferManagingRebufferer.java index 3a297ee0e260..13b7c9d44141 100644 --- a/src/java/org/apache/cassandra/io/util/BufferManagingRebufferer.java +++ b/src/java/org/apache/cassandra/io/util/BufferManagingRebufferer.java @@ -50,6 +50,7 @@ protected BufferManagingRebufferer(ChunkReader wrapped) public void closeReader() { BufferPools.forChunkCache().put(buffer); + source.releaseUnderlyingResources(); offset = -1; } diff --git a/src/java/org/apache/cassandra/io/util/ChunkReader.java b/src/java/org/apache/cassandra/io/util/ChunkReader.java index 33bf7921edd6..779e7c35f94a 100644 --- a/src/java/org/apache/cassandra/io/util/ChunkReader.java +++ b/src/java/org/apache/cassandra/io/util/ChunkReader.java @@ -48,4 +48,6 @@ public interface ChunkReader extends RebuffererFactory * This is not guaranteed to be fulfilled. */ BufferType preferredBufferType(); + + default void releaseUnderlyingResources() {} } diff --git a/src/java/org/apache/cassandra/io/util/CompressedChunkReader.java b/src/java/org/apache/cassandra/io/util/CompressedChunkReader.java index b0aa24bd8f6f..b6b3c9a6a267 100644 --- a/src/java/org/apache/cassandra/io/util/CompressedChunkReader.java +++ b/src/java/org/apache/cassandra/io/util/CompressedChunkReader.java @@ -26,11 +26,13 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.primitives.Ints; +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.io.compress.BufferType; import org.apache.cassandra.io.compress.CompressionMetadata; import org.apache.cassandra.io.compress.CorruptBlockException; import org.apache.cassandra.io.sstable.CorruptSSTableException; import org.apache.cassandra.utils.ChecksumType; +import org.apache.cassandra.utils.Closeable; public abstract class CompressedChunkReader extends AbstractReaderFileProxy implements ChunkReader { @@ -47,6 +49,11 @@ protected CompressedChunkReader(ChannelProxy channel, CompressionMetadata metada assert Integer.bitCount(metadata.chunkLength()) == 1; //must be a power of two } + protected CompressedChunkReader forScan() + { + return this; + } + @VisibleForTesting public double getCrcCheckChance() { @@ -83,20 +90,167 @@ public BufferType preferredBufferType() } @Override - public Rebufferer instantiateRebufferer() + public Rebufferer instantiateRebufferer(boolean isScan) { - return new BufferManagingRebufferer.Aligned(this); + return new BufferManagingRebufferer.Aligned(isScan ? forScan() : this); } - public static class Standard extends CompressedChunkReader + protected interface CompressedReader extends Closeable { - // we read the raw compressed bytes into this buffer, then uncompressed them into the provided one. + default void allocateResources() + { + } + + default void deallocateResources() + { + } + + default boolean allocated() + { + return false; + } + + default void close() + { + + } + + + ByteBuffer read(CompressionMetadata.Chunk chunk, boolean shouldCheckCrc) throws CorruptBlockException; + } + + private static class RandomAccessCompressedReader implements CompressedReader + { + private final ChannelProxy channel; private final ThreadLocalByteBufferHolder bufferHolder; + private RandomAccessCompressedReader(ChannelProxy channel, CompressionMetadata metadata) + { + this.channel = channel; + this.bufferHolder = new ThreadLocalByteBufferHolder(metadata.compressor().preferredBufferType()); + } + + @Override + public ByteBuffer read(CompressionMetadata.Chunk chunk, boolean shouldCheckCrc) throws CorruptBlockException + { + int length = shouldCheckCrc ? chunk.length + Integer.BYTES // compressed length + checksum length + : chunk.length; + ByteBuffer compressed = bufferHolder.getBuffer(length); + if (channel.read(compressed, chunk.offset) != length) + throw new CorruptBlockException(channel.filePath(), chunk); + compressed.flip(); + compressed.limit(chunk.length); + + if (shouldCheckCrc) + { + int checksum = (int) ChecksumType.CRC32.of(compressed); + compressed.limit(length); + if (compressed.getInt() != checksum) + throw new CorruptBlockException(channel.filePath(), chunk); + compressed.position(0).limit(chunk.length); + } + return compressed; + } + } + + private static class ScanCompressedReader implements CompressedReader + { + private final ChannelProxy channel; + private final ThreadLocalByteBufferHolder bufferHolder; + private final ThreadLocalReadAheadBuffer readAheadBuffer; + + private ScanCompressedReader(ChannelProxy channel, CompressionMetadata metadata, int readAheadBufferSize) + { + this.channel = channel; + this.bufferHolder = new ThreadLocalByteBufferHolder(metadata.compressor().preferredBufferType()); + this.readAheadBuffer = new ThreadLocalReadAheadBuffer(channel, readAheadBufferSize, metadata.compressor().preferredBufferType()); + } + + @Override + public ByteBuffer read(CompressionMetadata.Chunk chunk, boolean shouldCheckCrc) throws CorruptBlockException + { + int length = shouldCheckCrc ? chunk.length + Integer.BYTES // compressed length + checksum length + : chunk.length; + ByteBuffer compressed = bufferHolder.getBuffer(length); + + int copied = 0; + while (copied < length) + { + readAheadBuffer.fill(chunk.offset + copied); + int leftToRead = length - copied; + if (readAheadBuffer.remaining() >= leftToRead) + copied += readAheadBuffer.read(compressed, leftToRead); + else + copied += readAheadBuffer.read(compressed, readAheadBuffer.remaining()); + } + + compressed.flip(); + compressed.limit(chunk.length); + + if (shouldCheckCrc) + { + int checksum = (int) ChecksumType.CRC32.of(compressed); + compressed.limit(length); + if (compressed.getInt() != checksum) + throw new CorruptBlockException(channel.filePath(), chunk); + compressed.position(0).limit(chunk.length); + } + return compressed; + } + + @Override + public void allocateResources() + { + readAheadBuffer.allocateBuffer(); + } + + @Override + public void deallocateResources() + { + readAheadBuffer.clear(true); + } + + @Override + public boolean allocated() + { + return readAheadBuffer.hasBuffer(); + } + + public void close() + { + readAheadBuffer.close(); + } + } + + public static class Standard extends CompressedChunkReader + { + + private final CompressedReader reader; + private final CompressedReader scanReader; + public Standard(ChannelProxy channel, CompressionMetadata metadata, Supplier<Double> crcCheckChanceSupplier) { super(channel, metadata, crcCheckChanceSupplier); - bufferHolder = new ThreadLocalByteBufferHolder(metadata.compressor().preferredBufferType()); + reader = new RandomAccessCompressedReader(channel, metadata); + + int readAheadBufferSize = DatabaseDescriptor.getCompressedReadAheadBufferSize(); + scanReader = (readAheadBufferSize > 0 && readAheadBufferSize > metadata.chunkLength()) + ? new ScanCompressedReader(channel, metadata, readAheadBufferSize) : null; + } + + protected CompressedChunkReader forScan() + { + if (scanReader != null) + scanReader.allocateResources(); + + return this; + } + + @Override + public void releaseUnderlyingResources() + { + if (scanReader != null) + scanReader.deallocateResources(); } @Override @@ -110,31 +264,13 @@ public void readChunk(long position, ByteBuffer uncompressed) CompressionMetadata.Chunk chunk = metadata.chunkFor(position); boolean shouldCheckCrc = shouldCheckCrc(); - int length = shouldCheckCrc ? chunk.length + Integer.BYTES // compressed length + checksum length - : chunk.length; + CompressedReader readFrom = (scanReader != null && scanReader.allocated()) ? scanReader : reader; if (chunk.length < maxCompressedLength) { - ByteBuffer compressed = bufferHolder.getBuffer(length); - - if (channel.read(compressed, chunk.offset) != length) - throw new CorruptBlockException(channel.filePath(), chunk); - - compressed.flip(); - compressed.limit(chunk.length); + ByteBuffer compressed = readFrom.read(chunk, shouldCheckCrc); uncompressed.clear(); - if (shouldCheckCrc) - { - int checksum = (int) ChecksumType.CRC32.of(compressed); - - compressed.limit(length); - if (compressed.getInt() != checksum) - throw new CorruptBlockException(channel.filePath(), chunk); - - compressed.position(0).limit(chunk.length); - } - try { metadata.compressor().uncompress(compressed, uncompressed); @@ -155,10 +291,9 @@ public void readChunk(long position, ByteBuffer uncompressed) uncompressed.flip(); int checksum = (int) ChecksumType.CRC32.of(uncompressed); - ByteBuffer scratch = bufferHolder.getBuffer(Integer.BYTES); - + ByteBuffer scratch = ByteBuffer.allocate(Integer.BYTES); if (channel.read(scratch, chunk.offset + chunk.length) != Integer.BYTES - || scratch.getInt(0) != checksum) + || scratch.getInt(0) != checksum) throw new CorruptBlockException(channel.filePath(), chunk); } } @@ -171,6 +306,16 @@ public void readChunk(long position, ByteBuffer uncompressed) throw new CorruptSSTableException(e, channel.filePath()); } } + + @Override + public void close() + { + reader.close(); + if (scanReader != null) + scanReader.close(); + + super.close(); + } } public static class Mmap extends CompressedChunkReader @@ -233,7 +378,6 @@ public void readChunk(long position, ByteBuffer uncompressed) uncompressed.position(0).limit(0); throw new CorruptSSTableException(e, channel.filePath()); } - } public void close() diff --git a/src/java/org/apache/cassandra/io/util/EmptyRebufferer.java b/src/java/org/apache/cassandra/io/util/EmptyRebufferer.java index aa8e7e046f39..7f54a6b180f2 100644 --- a/src/java/org/apache/cassandra/io/util/EmptyRebufferer.java +++ b/src/java/org/apache/cassandra/io/util/EmptyRebufferer.java @@ -64,7 +64,7 @@ public void closeReader() } @Override - public Rebufferer instantiateRebufferer() + public Rebufferer instantiateRebufferer(boolean isScan) { return this; } diff --git a/src/java/org/apache/cassandra/io/util/FileHandle.java b/src/java/org/apache/cassandra/io/util/FileHandle.java index 943355d01dd7..67bfd239d61e 100644 --- a/src/java/org/apache/cassandra/io/util/FileHandle.java +++ b/src/java/org/apache/cassandra/io/util/FileHandle.java @@ -134,6 +134,11 @@ public RandomAccessReader createReader() return createReader(null); } + public RandomAccessReader createReaderForScan() + { + return createReader(null, true); + } + /** * Create {@link RandomAccessReader} with configured method of reading content of the file. * Reading from file will be rate limited by given {@link RateLimiter}. @@ -143,7 +148,12 @@ public RandomAccessReader createReader() */ public RandomAccessReader createReader(RateLimiter limiter) { - return new RandomAccessReader(instantiateRebufferer(limiter)); + return createReader(limiter, false); + } + + public RandomAccessReader createReader(RateLimiter limiter, boolean forScan) + { + return new RandomAccessReader(instantiateRebufferer(limiter, forScan)); } public FileDataInput createReader(long position) @@ -186,7 +196,12 @@ public void dropPageCache(long before) public Rebufferer instantiateRebufferer(RateLimiter limiter) { - Rebufferer rebufferer = rebuffererFactory.instantiateRebufferer(); + return instantiateRebufferer(limiter, false); + } + + public Rebufferer instantiateRebufferer(RateLimiter limiter, boolean forScan) + { + Rebufferer rebufferer = rebuffererFactory.instantiateRebufferer(forScan); if (limiter != null) rebufferer = new LimitingRebufferer(rebufferer, limiter, DiskOptimizationStrategy.MAX_BUFFER_SIZE); diff --git a/src/java/org/apache/cassandra/io/util/MmapRebufferer.java b/src/java/org/apache/cassandra/io/util/MmapRebufferer.java index 8df6370c5e69..884bc9718642 100644 --- a/src/java/org/apache/cassandra/io/util/MmapRebufferer.java +++ b/src/java/org/apache/cassandra/io/util/MmapRebufferer.java @@ -41,7 +41,7 @@ public BufferHolder rebuffer(long position) } @Override - public Rebufferer instantiateRebufferer() + public Rebufferer instantiateRebufferer(boolean isScan) { return this; } diff --git a/src/java/org/apache/cassandra/io/util/RandomAccessReader.java b/src/java/org/apache/cassandra/io/util/RandomAccessReader.java index 3ce1a2eb0862..b89e59eb5291 100644 --- a/src/java/org/apache/cassandra/io/util/RandomAccessReader.java +++ b/src/java/org/apache/cassandra/io/util/RandomAccessReader.java @@ -332,7 +332,7 @@ public static RandomAccessReader open(File file) try { ChunkReader reader = new SimpleChunkReader(channel, -1, BufferType.OFF_HEAP, DEFAULT_BUFFER_SIZE); - Rebufferer rebufferer = reader.instantiateRebufferer(); + Rebufferer rebufferer = reader.instantiateRebufferer(false); return new RandomAccessReaderWithOwnChannel(rebufferer); } catch (Throwable t) diff --git a/src/java/org/apache/cassandra/io/util/RebuffererFactory.java b/src/java/org/apache/cassandra/io/util/RebuffererFactory.java index ec35f0ba530b..192fb8ea0cd8 100644 --- a/src/java/org/apache/cassandra/io/util/RebuffererFactory.java +++ b/src/java/org/apache/cassandra/io/util/RebuffererFactory.java @@ -28,5 +28,5 @@ */ public interface RebuffererFactory extends ReaderFileProxy { - Rebufferer instantiateRebufferer(); + Rebufferer instantiateRebufferer(boolean isScan); } diff --git a/src/java/org/apache/cassandra/io/util/SimpleChunkReader.java b/src/java/org/apache/cassandra/io/util/SimpleChunkReader.java index 8d00ce5d4000..fec1216bf4e4 100644 --- a/src/java/org/apache/cassandra/io/util/SimpleChunkReader.java +++ b/src/java/org/apache/cassandra/io/util/SimpleChunkReader.java @@ -55,7 +55,7 @@ public BufferType preferredBufferType() } @Override - public Rebufferer instantiateRebufferer() + public Rebufferer instantiateRebufferer(boolean forScan) { if (Integer.bitCount(bufferSize) == 1) return new BufferManagingRebufferer.Aligned(this); diff --git a/src/java/org/apache/cassandra/io/util/ThreadLocalReadAheadBuffer.java b/src/java/org/apache/cassandra/io/util/ThreadLocalReadAheadBuffer.java new file mode 100644 index 000000000000..824acaa8d88f --- /dev/null +++ b/src/java/org/apache/cassandra/io/util/ThreadLocalReadAheadBuffer.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.io.util; + +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; + +import io.netty.util.concurrent.FastThreadLocal; +import org.apache.cassandra.io.compress.BufferType; +import org.apache.cassandra.io.sstable.CorruptSSTableException; + +public final class ThreadLocalReadAheadBuffer +{ + private static class Block + { + ByteBuffer buffer = null; + int index = -1; + } + + private final ChannelProxy channel; + + private final BufferType bufferType; + + private static final FastThreadLocal<Map<String, Block>> blockMap = new FastThreadLocal<>() + { + @Override + protected Map<String, Block> initialValue() + { + return new HashMap<>(); + } + }; + + private final int bufferSize; + private final long channelSize; + + public ThreadLocalReadAheadBuffer(ChannelProxy channel, int bufferSize, BufferType bufferType) + { + this.channel = channel; + this.channelSize = channel.size(); + this.bufferSize = bufferSize; + this.bufferType = bufferType; + } + + public boolean hasBuffer() + { + return block().buffer != null; + } + + /** + * Safe to call only if {@link #hasBuffer()} is true + */ + public int remaining() + { + return getBlock().buffer.remaining(); + } + + public void allocateBuffer() + { + getBlock(); + } + + private Block getBlock() + { + Block block = block(); + if (block.buffer == null) + { + block.buffer = bufferType.allocate(bufferSize); + block.buffer.clear(); + } + return block; + } + + private Block block() + { + return blockMap.get().computeIfAbsent(channel.filePath(), k -> new Block()); + } + + public void fill(long position) + { + Block block = getBlock(); + ByteBuffer blockBuffer = block.buffer; + long realPosition = Math.min(channelSize, position); + int blockNo = (int) (realPosition / bufferSize); + long blockPosition = blockNo * (long) bufferSize; + + long remaining = channelSize - blockPosition; + int sizeToRead = (int) Math.min(remaining, bufferSize); + if (block.index != blockNo) + { + blockBuffer.flip(); + blockBuffer.limit(sizeToRead); + if (channel.read(blockBuffer, blockPosition) != sizeToRead) + throw new CorruptSSTableException(null, channel.filePath()); + + block.index = blockNo; + } + + blockBuffer.flip(); + blockBuffer.limit(sizeToRead); + blockBuffer.position((int) (realPosition - blockPosition)); + } + + public int read(ByteBuffer dest, int length) + { + Block block = getBlock(); + ByteBuffer blockBuffer = block.buffer; + ByteBuffer tmp = blockBuffer.duplicate(); + tmp.limit(tmp.position() + length); + dest.put(tmp); + blockBuffer.position(blockBuffer.position() + length); + + return length; + } + + public void clear(boolean deallocate) + { + Block block = getBlock(); + block.index = -1; + + ByteBuffer blockBuffer = block.buffer; + if (blockBuffer != null) + { + blockBuffer.clear(); + if (deallocate) + { + FileUtils.clean(blockBuffer); + block.buffer = null; + } + } + } + + public void close() + { + clear(true); + blockMap.get().remove(channel.filePath()); + } +} diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 8f2760145e52..533ca08e6d5e 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -1893,6 +1893,20 @@ public void setCompactionThroughputMbPerSec(int value) value, oldValue); } + @Override + public int getCompressedReadAheadBufferInKB() + { + return DatabaseDescriptor.getCompressedReadAheadBufferSizeInKB(); + } + + @Override + public void setCompressedReadAheadBufferInKB(int sizeInKb) + { + DatabaseDescriptor.setCompressedReadAheadBufferSizeInKb(sizeInKb); + logger.info("set compressed read ahead buffer size to {}KiB", sizeInKb); + } + + public int getBatchlogReplayThrottleInKB() { return DatabaseDescriptor.getBatchlogReplayThrottleInKiB(); diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java index 8418f8bd156b..0bfcfd500b30 100644 --- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java +++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java @@ -812,6 +812,9 @@ default int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, public int getCompactionThroughputMbPerSec(); public void setCompactionThroughputMbPerSec(int value); + public int getCompressedReadAheadBufferInKB(); + public void setCompressedReadAheadBufferInKB(int sizeInKb); + public int getBatchlogReplayThrottleInKB(); public void setBatchlogReplayThrottleInKB(int value); diff --git a/test/unit/org/apache/cassandra/io/util/CompressedChunkReaderTest.java b/test/unit/org/apache/cassandra/io/util/CompressedChunkReaderTest.java new file mode 100644 index 000000000000..af4b458fecc4 --- /dev/null +++ b/test/unit/org/apache/cassandra/io/util/CompressedChunkReaderTest.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.io.util; + +import accord.utils.Gen; +import accord.utils.Gens; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ClusteringComparator; +import org.apache.cassandra.io.compress.CompressedSequentialWriter; +import org.apache.cassandra.io.compress.CompressionMetadata; +import org.apache.cassandra.io.filesystem.ListenableFileSystem; +import org.apache.cassandra.io.sstable.metadata.MetadataCollector; +import org.apache.cassandra.schema.CompressionParams; +import org.assertj.core.api.Assertions; + +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.nio.file.Files; +import java.util.concurrent.atomic.AtomicInteger; + +import static accord.utils.Property.qt; + +public class CompressedChunkReaderTest +{ + static + { + DatabaseDescriptor.clientInitialization(); + } + + @Test + public void scanReaderReadsLessThanRAReader() + { + var optionGen = options(); + var paramsGen = params(); + var lengthGen = Gens.longs().between(1, 1 << 16); + qt().withSeed(-1871070464864118891L).forAll(Gens.random(), optionGen, paramsGen).check((rs, option, params) -> { + ListenableFileSystem fs = FileSystems.newGlobalInMemoryFileSystem(); + + File f = new File("/file.db"); + AtomicInteger reads = new AtomicInteger(); + fs.onPostRead(f.path::equals, (p, c, pos, dst, r) -> { + reads.incrementAndGet(); + }); + long length = lengthGen.nextLong(rs); + CompressionMetadata metadata1, metadata2; + try (CompressedSequentialWriter writer = new CompressedSequentialWriter(f, new File("/file.offset"), new File("/file.digest"), option, params, new MetadataCollector(new ClusteringComparator()))) + { + for (long i = 0; i < length; i++) + writer.writeLong(i); + + writer.sync(); + metadata1 = writer.open(0); + metadata2 = writer.open(0); + } + + doReads(f, metadata1, length, true); + int scanReads = reads.getAndSet(0); + + doReads(f, metadata2, length, false); + int raReads = reads.getAndSet(0); + + if (Files.size(f.toPath()) > DatabaseDescriptor.getCompressedReadAheadBufferSize()) + Assert.assertTrue(scanReads < raReads); + }); + } + + private void doReads(File f, CompressionMetadata metadata, long length, boolean useReadAhead) + { + ByteBuffer buffer = ByteBuffer.allocateDirect(metadata.chunkLength()); + + try (ChannelProxy channel = new ChannelProxy(f); + CompressedChunkReader reader = new CompressedChunkReader.Standard(channel, metadata, () -> 1.1); + metadata) + { + if (useReadAhead) + reader.forScan(); + + long offset = 0; + long maxOffset = length * Long.BYTES; + do + { + reader.readChunk(offset, buffer); + for (long expected = offset / Long.BYTES; buffer.hasRemaining(); expected++) + Assertions.assertThat(buffer.getLong()).isEqualTo(expected); + + offset += metadata.chunkLength(); + } + while (offset < maxOffset); + } + finally + { + FileUtils.clean(buffer); + }} + + private static Gen<SequentialWriterOption> options() + { + Gen<Integer> bufferSizes = Gens.constant(1 << 10); //.pickInt(1 << 4, 1 << 10, 1 << 15); + return rs -> SequentialWriterOption.newBuilder() + .finishOnClose(false) + .bufferSize(bufferSizes.next(rs)) + .build(); + } + + private enum CompressionKind { Noop, Snappy, Deflate, Lz4, Zstd } + + private static Gen<CompressionParams> params() + { + Gen<Integer> chunkLengths = Gens.constant(CompressionParams.DEFAULT_CHUNK_LENGTH); + Gen<Double> compressionRatio = Gens.pick(1.1D); + return rs -> { + CompressionKind kind = rs.pick(CompressionKind.values()); + switch (kind) + { + case Noop: return CompressionParams.noop(); + case Snappy: return CompressionParams.snappy(chunkLengths.next(rs), compressionRatio.next(rs)); + case Deflate: return CompressionParams.deflate(chunkLengths.next(rs)); + case Lz4: return CompressionParams.lz4(chunkLengths.next(rs)); + case Zstd: return CompressionParams.zstd(chunkLengths.next(rs)); + default: throw new UnsupportedOperationException(kind.name()); + } + }; + } +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/io/util/ThreadLocalReadAheadBufferTest.java b/test/unit/org/apache/cassandra/io/util/ThreadLocalReadAheadBufferTest.java new file mode 100644 index 000000000000..4d43017b2a5a --- /dev/null +++ b/test/unit/org/apache/cassandra/io/util/ThreadLocalReadAheadBufferTest.java @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.io.util; + + +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.file.Files; +import java.util.List; +import java.util.Random; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.DataStorageSpec; +import org.apache.cassandra.io.compress.BufferType; +import org.apache.cassandra.io.sstable.CorruptSSTableException; +import org.apache.cassandra.utils.Pair; +import org.quicktheories.WithQuickTheories; +import org.quicktheories.core.Gen; + +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_IO_TMPDIR; + +public class ThreadLocalReadAheadBufferTest implements WithQuickTheories +{ + private static final int numFiles = 5; + private static final File[] files = new File[numFiles]; + private static final Logger logger = LoggerFactory.getLogger(ThreadLocalReadAheadBufferTest.class); + + @BeforeClass + public static void setup() + { + int seed = new Random().nextInt(); + logger.info("Seed: {}", seed); + + for (int i = 0; i < numFiles; i++) + { + int size = new Random().nextInt((Integer.MAX_VALUE - 1) / 8); + files[i] = writeFile(seed, size); + } + } + + @AfterClass + public static void cleanup() + { + for (File f : files) + { + try + { + f.delete(); + } + catch (Exception e) + { + // ignore + } + } + } + + @Test + public void testLastBlockReads() + { + qt().forAll(lastBlockReads()) + .checkAssert(this::testReads); + } + + @Test + public void testReadsLikeChannelProxy() + { + + qt().forAll(randomReads()) + .checkAssert(this::testReads); + } + + private void testReads(InputData propertyInputs) + { + try (ChannelProxy channel = new ChannelProxy(propertyInputs.file)) + { + ThreadLocalReadAheadBuffer trlab = new ThreadLocalReadAheadBuffer(channel, new DataStorageSpec.IntKibibytesBound("256KiB").toBytes(), BufferType.OFF_HEAP); + for (Pair<Long, Integer> read : propertyInputs.positionsAndLengths) + { + int readSize = Math.min(read.right,(int) (channel.size() - read.left)); + ByteBuffer buf1 = ByteBuffer.allocate(readSize); + channel.read(buf1, read.left); + + ByteBuffer buf2 = ByteBuffer.allocate(readSize); + try + { + int copied = 0; + while (copied < readSize) { + trlab.fill(read.left + copied); + int leftToRead = readSize - copied; + if (trlab.remaining() >= leftToRead) + copied += trlab.read(buf2, leftToRead); + else + copied += trlab.read(buf2, trlab.remaining()); + } + } + catch (CorruptSSTableException e) + { + throw new RuntimeException(e); + } + + Assert.assertEquals(buf1, buf2); + } + } + } + + private Gen<InputData> lastBlockReads() + { + return arbitrary().pick(List.of(files)) + .flatMap((file) -> + lists().of(longs().between(0, fileSize(file)).zip(integers().between(1, 100), Pair::create)) + .ofSizeBetween(5, 10) + .map(positionsAndLengths -> new InputData(file, positionsAndLengths))); + + } + + private Gen<InputData> randomReads() + { + int blockSize = new DataStorageSpec.IntKibibytesBound("256KiB").toBytes(); + return arbitrary().pick(List.of(files)) + .flatMap((file) -> + lists().of(longs().between(fileSize(file) - blockSize, fileSize(file)).zip(integers().between(1, 100), Pair::create)) + .ofSizeBetween(5, 10) + .map(positionsAndLengths -> new InputData(file, positionsAndLengths))); + + } + + // need this becasue generators don't handle the IOException + private long fileSize(File file) + { + try + { + return Files.size(file.toPath()); + } catch (IOException e) + { + throw new RuntimeException(e); + } + } + + private static class InputData + { + + private final File file; + private final List<Pair<Long, Integer>> positionsAndLengths; + + public InputData(File file, List<Pair<Long, Integer>> positionsAndLengths) + { + this.file = file; + this.positionsAndLengths = positionsAndLengths; + } + } + + private static File writeFile(int seed, int length) + { + String fileName = JAVA_IO_TMPDIR.getString() + "data+" + length + ".bin"; + + byte[] dataChunk = new byte[4096 * 8]; + java.util.Random random = new Random(seed); + int writtenData = 0; + + File file = new File(fileName); + try (FileOutputStream fos = new FileOutputStream(file.toJavaIOFile())) + { + while (writtenData < length) + { + random.nextBytes(dataChunk); + int toWrite = Math.min((length - writtenData), dataChunk.length); + fos.write(dataChunk, 0, toWrite); + writtenData += toWrite; + } + fos.flush(); + } + catch (IOException e) + { + throw new RuntimeException(e); + } + + return file; + } + +} From ef354d0d5ea513a0dffce2f60fb2bd8e33829fc0 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Fri, 28 Feb 2025 14:13:27 +0100 Subject: [PATCH 169/225] Update Zstd library to 1.5.7-1 patch by Stefan Miklosovic; reviewed by Yifan Cai for CASSANDRA-20367 --- CHANGES.txt | 2 + build.xml | 2 +- ide/nbproject/project.xml | 2 +- .../upgrade/CompressionUpgradeTest.java | 93 +++++++++++++++++++ 4 files changed, 97 insertions(+), 2 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/upgrade/CompressionUpgradeTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 00fc0747e876..fff1039d33ba 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,6 @@ 4.0.18 + * Update Zstd library to 1.5.7-1 (CASSANDRA-20367) + * Fix premature auto-failing of long-running repairs (CASSANDRA-20312) 4.0.17 diff --git a/build.xml b/build.xml index 367996d0fb27..41e59c9f1b99 100644 --- a/build.xml +++ b/build.xml @@ -547,7 +547,7 @@ <dependencyManagement> <dependency groupId="org.xerial.snappy" artifactId="snappy-java" version="1.1.10.4"/> <dependency groupId="org.lz4" artifactId="lz4-java" version="1.8.0"/> - <dependency groupId="com.github.luben" artifactId="zstd-jni" version="1.5.5-1"/> + <dependency groupId="com.github.luben" artifactId="zstd-jni" version="1.5.7-1"/> <dependency groupId="com.google.guava" artifactId="guava" version="27.0-jre"> <exclusion groupId="com.google.code.findbugs" artifactId="jsr305" /> <exclusion groupId="org.codehaus.mojo" artifactId="animal-sniffer-annotations" /> diff --git a/ide/nbproject/project.xml b/ide/nbproject/project.xml index 32e993ab40b4..49efd63261ca 100644 --- a/ide/nbproject/project.xml +++ b/ide/nbproject/project.xml @@ -7,7 +7,7 @@ <properties> <property name="project.dir">..</property> <!-- the compile classpaths should be distinct per compilation unit… but it is kept simple and the build will catch errors --> - <property name="cassandra.classpath.jars">${project.dir}/build/lib/jars/HdrHistogram-2.1.12.jar:${project.dir}/build/lib/jars/ST4-4.0.8.jar:${project.dir}/build/lib/jars/airline-0.8.jar:${project.dir}/build/lib/jars/antlr-3.5.2.jar:${project.dir}/build/lib/jars/antlr-runtime-3.5.2.jar:${project.dir}/build/lib/jars/asm-7.1.jar:${project.dir}/build/lib/jars/byteman-4.0.6.jar:${project.dir}/build/lib/jars/byteman-bmunit-4.0.6.jar:${project.dir}/build/lib/jars/byteman-install-4.0.6.jar:${project.dir}/build/lib/jars/byteman-submit-4.0.6.jar:${project.dir}/build/lib/jars/caffeine-2.5.6.jar:${project.dir}/build/lib/jars/cassandra-driver-core-3.11.0-shaded.jar:${project.dir}/build/lib/jars/chronicle-bytes-2.20.111.jar:${project.dir}/build/lib/jars/chronicle-core-2.20.126.jar:${project.dir}/build/lib/jars/chronicle-queue-5.20.123.jar:${project.dir}/build/lib/jars/chronicle-threads-2.20.111.jar:${project.dir}/build/lib/jars/chronicle-wire-2.20.117.jar:${project.dir}/build/lib/jars/commons-beanutils-1.7.0.jar:${project.dir}/build/lib/jars/commons-beanutils-core-1.8.0.jar:${project.dir}/build/lib/jars/commons-cli-1.1.jar:${project.dir}/build/lib/jars/commons-codec-1.9.jar:${project.dir}/build/lib/jars/commons-collections-3.2.1.jar:${project.dir}/build/lib/jars/commons-configuration-1.6.jar:${project.dir}/build/lib/jars/commons-digester-1.8.jar:${project.dir}/build/lib/jars/commons-el-1.0.jar:${project.dir}/build/lib/jars/commons-httpclient-3.0.1.jar:${project.dir}/build/lib/jars/commons-lang-2.4.jar:${project.dir}/build/lib/jars/commons-lang3-3.11.jar:${project.dir}/build/lib/jars/commons-math-2.1.jar:${project.dir}/build/lib/jars/commons-math3-3.2.jar:${project.dir}/build/lib/jars/commons-net-1.4.1.jar:${project.dir}/build/lib/jars/compile-command-annotations-1.2.0.jar:${project.dir}/build/lib/jars/concurrent-trees-2.4.0.jar:${project.dir}/build/lib/jars/ecj-4.6.1.jar:${project.dir}/build/lib/jars/ftplet-api-1.0.0.jar:${project.dir}/build/lib/jars/ftpserver-core-1.0.0.jar:${project.dir}/build/lib/jars/ftpserver-deprecated-1.0.0-M2.jar:${project.dir}/build/lib/jars/guava-27.0-jre.jar:${project.dir}/build/lib/jars/hadoop-core-1.0.3.jar:${project.dir}/build/lib/jars/hadoop-minicluster-1.0.3.jar:${project.dir}/build/lib/jars/hadoop-test-1.0.3.jar:${project.dir}/build/lib/jars/high-scale-lib-1.0.6.jar:${project.dir}/build/lib/jars/hppc-0.8.1.jar:${project.dir}/build/lib/jars/hsqldb-1.8.0.10.jar:${project.dir}/build/lib/jars/j2objc-annotations-1.3.jar:${project.dir}/build/lib/jars/jackson-annotations-2.13.2.jar:${project.dir}/build/lib/jars/jackson-core-2.13.2.jar:${project.dir}/build/lib/jars/jackson-databind-2.13.2.2.jar:${project.dir}/build/lib/jars/jacocoagent.jar:${project.dir}/build/lib/jars/jamm-0.3.2.jar:${project.dir}/build/lib/jars/jasper-compiler-5.5.12.jar:${project.dir}/build/lib/jars/jasper-runtime-5.5.12.jar:${project.dir}/build/lib/jars/javax.inject-1.jar:${project.dir}/build/lib/jars/jbcrypt-0.4.jar:${project.dir}/build/lib/jars/jcl-over-slf4j-1.7.25.jar:${project.dir}/build/lib/jars/jcommander-1.30.jar:${project.dir}/build/lib/jars/jctools-core-3.1.0.jar:${project.dir}/build/lib/jars/jersey-core-1.0.jar:${project.dir}/build/lib/jars/jersey-server-1.0.jar:${project.dir}/build/lib/jars/jets3t-0.7.1.jar:${project.dir}/build/lib/jars/jetty-6.1.26.jar:${project.dir}/build/lib/jars/jetty-util-6.1.26.jar:${project.dir}/build/lib/jars/jna-5.6.0.jar:${project.dir}/build/lib/jars/json-simple-1.1.jar:${project.dir}/build/lib/jars/jsp-2.1-6.1.14.jar:${project.dir}/build/lib/jars/jsp-api-2.1-6.1.14.jar:${project.dir}/build/lib/jars/jsr305-2.0.2.jar:${project.dir}/build/lib/jars/jsr311-api-1.0.jar:${project.dir}/build/lib/jars/jvm-attach-api-1.5.jar:${project.dir}/build/lib/jars/kfs-0.3.jar:${project.dir}/build/lib/jars/log4j-over-slf4j-1.7.25.jar:${project.dir}/build/lib/jars/logback-classic-1.2.9.jar:${project.dir}/build/lib/jars/logback-core-1.2.9.jar:${project.dir}/build/lib/jars/lz4-java-1.8.0.jar:${project.dir}/build/lib/jars/metrics-core-3.1.5.jar:${project.dir}/build/lib/jars/metrics-jvm-3.1.5.jar:${project.dir}/build/lib/jars/metrics-logback-3.1.5.jar:${project.dir}/build/lib/jars/mina-core-2.0.0-M5.jar:${project.dir}/build/lib/jars/mxdump-0.14.jar:${project.dir}/build/lib/jars/netty-all-4.1.58.Final.jar:${project.dir}/build/lib/jars/netty-tcnative-boringssl-static-2.0.36.Final.jar:${project.dir}/build/lib/jars/ohc-core-0.5.1.jar:${project.dir}/build/lib/jars/ohc-core-j8-0.5.1.jar:${project.dir}/build/lib/jars/oro-2.0.8.jar:${project.dir}/build/lib/jars/psjava-0.1.19.jar:${project.dir}/build/lib/jars/reporter-config-base-3.0.3.jar:${project.dir}/build/lib/jars/reporter-config3-3.0.3.jar:${project.dir}/build/lib/jars/servlet-api-2.5-6.1.14.jar:${project.dir}/build/lib/jars/sigar-1.6.4.jar:${project.dir}/build/lib/jars/sjk-cli-0.14.jar:${project.dir}/build/lib/jars/sjk-core-0.14.jar:${project.dir}/build/lib/jars/sjk-json-0.14.jar:${project.dir}/build/lib/jars/sjk-stacktrace-0.14.jar:${project.dir}/build/lib/jars/slf4j-api-1.7.25.jar:${project.dir}/build/lib/jars/snakeyaml-1.26.jar:${project.dir}/build/lib/jars/snappy-java-1.1.10.4.jar:${project.dir}/build/lib/jars/snowball-stemmer-1.3.0.581.1.jar:${project.dir}/build/lib/jars/stream-2.5.2.jar:${project.dir}/build/lib/jars/xmlenc-0.52.jar:${project.dir}/build/lib/jars/zstd-jni-1.5.5-1.jar:${project.dir}/build/test/lib/jars/ant-1.10.12.jar:${project.dir}/build/test/lib/jars/ant-junit-1.10.12.jar:${project.dir}/build/test/lib/jars/ant-launcher-1.10.12.jar:${project.dir}/build/test/lib/jars/asm-6.0.jar:${project.dir}/build/test/lib/jars/asm-analysis-6.0.jar:${project.dir}/build/test/lib/jars/asm-commons-6.0.jar:${project.dir}/build/test/lib/jars/asm-tree-6.0.jar:${project.dir}/build/test/lib/jars/asm-util-6.0.jar:${project.dir}/build/test/lib/jars/asm-xml-6.0.jar:${project.dir}/build/test/lib/jars/assertj-core-3.25.3.jar:${project.dir}/build/test/lib/jars/awaitility-4.0.3.jar:${project.dir}/build/test/lib/jars/byte-buddy-1.14.11.jar:${project.dir}/build/test/lib/jars/byte-buddy-agent-1.10.5.jar:${project.dir}/build/test/lib/jars/commons-io-2.6.jar:${project.dir}/build/test/lib/jars/commons-math3-3.6.1.jar:${project.dir}/build/test/lib/jars/dtest-api-0.0.16.jar:${project.dir}/build/test/lib/jars/hamcrest-2.2.jar:${project.dir}/build/test/lib/jars/java-allocation-instrumenter-3.1.0.jar:${project.dir}/build/test/lib/jars/javassist-3.28.0-GA.jar:${project.dir}/build/test/lib/jars/jmh-core-1.37.jar:${project.dir}/build/test/lib/jars/jmh-generator-annprocess-1.37.jar:${project.dir}/build/test/lib/jars/jopt-simple-5.0.4.jar:${project.dir}/build/test/lib/jars/jsr305-3.0.2.jar:${project.dir}/build/test/lib/jars/junit-4.12.jar:${project.dir}/build/test/lib/jars/mockito-core-3.2.4.jar:${project.dir}/build/test/lib/jars/objenesis-2.6.jar:${project.dir}/build/test/lib/jars/org.jacoco.agent-0.8.6.jar:${project.dir}/build/test/lib/jars/org.jacoco.ant-0.8.6.jar:${project.dir}/build/test/lib/jars/org.jacoco.core-0.8.6.jar:${project.dir}/build/test/lib/jars/org.jacoco.report-0.8.6.jar:${project.dir}/build/test/lib/jars/quicktheories-0.26.jar:${project.dir}/build/test/lib/jars/reflections-0.10.2.jar:${project.dir}/build/test/lib/jars/semver4j-3.1.0.jar:${project.dir}/build/test/lib/jars/slf4j-api-1.7.32.jar:</property> + <property name="cassandra.classpath.jars">${project.dir}/build/lib/jars/HdrHistogram-2.1.12.jar:${project.dir}/build/lib/jars/ST4-4.0.8.jar:${project.dir}/build/lib/jars/airline-0.8.jar:${project.dir}/build/lib/jars/antlr-3.5.2.jar:${project.dir}/build/lib/jars/antlr-runtime-3.5.2.jar:${project.dir}/build/lib/jars/asm-7.1.jar:${project.dir}/build/lib/jars/byteman-4.0.6.jar:${project.dir}/build/lib/jars/byteman-bmunit-4.0.6.jar:${project.dir}/build/lib/jars/byteman-install-4.0.6.jar:${project.dir}/build/lib/jars/byteman-submit-4.0.6.jar:${project.dir}/build/lib/jars/caffeine-2.5.6.jar:${project.dir}/build/lib/jars/cassandra-driver-core-3.11.0-shaded.jar:${project.dir}/build/lib/jars/chronicle-bytes-2.20.111.jar:${project.dir}/build/lib/jars/chronicle-core-2.20.126.jar:${project.dir}/build/lib/jars/chronicle-queue-5.20.123.jar:${project.dir}/build/lib/jars/chronicle-threads-2.20.111.jar:${project.dir}/build/lib/jars/chronicle-wire-2.20.117.jar:${project.dir}/build/lib/jars/commons-beanutils-1.7.0.jar:${project.dir}/build/lib/jars/commons-beanutils-core-1.8.0.jar:${project.dir}/build/lib/jars/commons-cli-1.1.jar:${project.dir}/build/lib/jars/commons-codec-1.9.jar:${project.dir}/build/lib/jars/commons-collections-3.2.1.jar:${project.dir}/build/lib/jars/commons-configuration-1.6.jar:${project.dir}/build/lib/jars/commons-digester-1.8.jar:${project.dir}/build/lib/jars/commons-el-1.0.jar:${project.dir}/build/lib/jars/commons-httpclient-3.0.1.jar:${project.dir}/build/lib/jars/commons-lang-2.4.jar:${project.dir}/build/lib/jars/commons-lang3-3.11.jar:${project.dir}/build/lib/jars/commons-math-2.1.jar:${project.dir}/build/lib/jars/commons-math3-3.2.jar:${project.dir}/build/lib/jars/commons-net-1.4.1.jar:${project.dir}/build/lib/jars/compile-command-annotations-1.2.0.jar:${project.dir}/build/lib/jars/concurrent-trees-2.4.0.jar:${project.dir}/build/lib/jars/ecj-4.6.1.jar:${project.dir}/build/lib/jars/ftplet-api-1.0.0.jar:${project.dir}/build/lib/jars/ftpserver-core-1.0.0.jar:${project.dir}/build/lib/jars/ftpserver-deprecated-1.0.0-M2.jar:${project.dir}/build/lib/jars/guava-27.0-jre.jar:${project.dir}/build/lib/jars/hadoop-core-1.0.3.jar:${project.dir}/build/lib/jars/hadoop-minicluster-1.0.3.jar:${project.dir}/build/lib/jars/hadoop-test-1.0.3.jar:${project.dir}/build/lib/jars/high-scale-lib-1.0.6.jar:${project.dir}/build/lib/jars/hppc-0.8.1.jar:${project.dir}/build/lib/jars/hsqldb-1.8.0.10.jar:${project.dir}/build/lib/jars/j2objc-annotations-1.3.jar:${project.dir}/build/lib/jars/jackson-annotations-2.13.2.jar:${project.dir}/build/lib/jars/jackson-core-2.13.2.jar:${project.dir}/build/lib/jars/jackson-databind-2.13.2.2.jar:${project.dir}/build/lib/jars/jacocoagent.jar:${project.dir}/build/lib/jars/jamm-0.3.2.jar:${project.dir}/build/lib/jars/jasper-compiler-5.5.12.jar:${project.dir}/build/lib/jars/jasper-runtime-5.5.12.jar:${project.dir}/build/lib/jars/javax.inject-1.jar:${project.dir}/build/lib/jars/jbcrypt-0.4.jar:${project.dir}/build/lib/jars/jcl-over-slf4j-1.7.25.jar:${project.dir}/build/lib/jars/jcommander-1.30.jar:${project.dir}/build/lib/jars/jctools-core-3.1.0.jar:${project.dir}/build/lib/jars/jersey-core-1.0.jar:${project.dir}/build/lib/jars/jersey-server-1.0.jar:${project.dir}/build/lib/jars/jets3t-0.7.1.jar:${project.dir}/build/lib/jars/jetty-6.1.26.jar:${project.dir}/build/lib/jars/jetty-util-6.1.26.jar:${project.dir}/build/lib/jars/jna-5.6.0.jar:${project.dir}/build/lib/jars/json-simple-1.1.jar:${project.dir}/build/lib/jars/jsp-2.1-6.1.14.jar:${project.dir}/build/lib/jars/jsp-api-2.1-6.1.14.jar:${project.dir}/build/lib/jars/jsr305-2.0.2.jar:${project.dir}/build/lib/jars/jsr311-api-1.0.jar:${project.dir}/build/lib/jars/jvm-attach-api-1.5.jar:${project.dir}/build/lib/jars/kfs-0.3.jar:${project.dir}/build/lib/jars/log4j-over-slf4j-1.7.25.jar:${project.dir}/build/lib/jars/logback-classic-1.2.9.jar:${project.dir}/build/lib/jars/logback-core-1.2.9.jar:${project.dir}/build/lib/jars/lz4-java-1.8.0.jar:${project.dir}/build/lib/jars/metrics-core-3.1.5.jar:${project.dir}/build/lib/jars/metrics-jvm-3.1.5.jar:${project.dir}/build/lib/jars/metrics-logback-3.1.5.jar:${project.dir}/build/lib/jars/mina-core-2.0.0-M5.jar:${project.dir}/build/lib/jars/mxdump-0.14.jar:${project.dir}/build/lib/jars/netty-all-4.1.58.Final.jar:${project.dir}/build/lib/jars/netty-tcnative-boringssl-static-2.0.36.Final.jar:${project.dir}/build/lib/jars/ohc-core-0.5.1.jar:${project.dir}/build/lib/jars/ohc-core-j8-0.5.1.jar:${project.dir}/build/lib/jars/oro-2.0.8.jar:${project.dir}/build/lib/jars/psjava-0.1.19.jar:${project.dir}/build/lib/jars/reporter-config-base-3.0.3.jar:${project.dir}/build/lib/jars/reporter-config3-3.0.3.jar:${project.dir}/build/lib/jars/servlet-api-2.5-6.1.14.jar:${project.dir}/build/lib/jars/sigar-1.6.4.jar:${project.dir}/build/lib/jars/sjk-cli-0.14.jar:${project.dir}/build/lib/jars/sjk-core-0.14.jar:${project.dir}/build/lib/jars/sjk-json-0.14.jar:${project.dir}/build/lib/jars/sjk-stacktrace-0.14.jar:${project.dir}/build/lib/jars/slf4j-api-1.7.25.jar:${project.dir}/build/lib/jars/snakeyaml-1.26.jar:${project.dir}/build/lib/jars/snappy-java-1.1.10.4.jar:${project.dir}/build/lib/jars/snowball-stemmer-1.3.0.581.1.jar:${project.dir}/build/lib/jars/stream-2.5.2.jar:${project.dir}/build/lib/jars/xmlenc-0.52.jar:${project.dir}/build/lib/jars/zstd-jni-1.5.7-1.jar:${project.dir}/build/test/lib/jars/ant-1.10.12.jar:${project.dir}/build/test/lib/jars/ant-junit-1.10.12.jar:${project.dir}/build/test/lib/jars/ant-launcher-1.10.12.jar:${project.dir}/build/test/lib/jars/asm-6.0.jar:${project.dir}/build/test/lib/jars/asm-analysis-6.0.jar:${project.dir}/build/test/lib/jars/asm-commons-6.0.jar:${project.dir}/build/test/lib/jars/asm-tree-6.0.jar:${project.dir}/build/test/lib/jars/asm-util-6.0.jar:${project.dir}/build/test/lib/jars/asm-xml-6.0.jar:${project.dir}/build/test/lib/jars/assertj-core-3.25.3.jar:${project.dir}/build/test/lib/jars/awaitility-4.0.3.jar:${project.dir}/build/test/lib/jars/byte-buddy-1.14.11.jar:${project.dir}/build/test/lib/jars/byte-buddy-agent-1.10.5.jar:${project.dir}/build/test/lib/jars/commons-io-2.6.jar:${project.dir}/build/test/lib/jars/commons-math3-3.6.1.jar:${project.dir}/build/test/lib/jars/dtest-api-0.0.16.jar:${project.dir}/build/test/lib/jars/hamcrest-2.2.jar:${project.dir}/build/test/lib/jars/java-allocation-instrumenter-3.1.0.jar:${project.dir}/build/test/lib/jars/javassist-3.28.0-GA.jar:${project.dir}/build/test/lib/jars/jmh-core-1.37.jar:${project.dir}/build/test/lib/jars/jmh-generator-annprocess-1.37.jar:${project.dir}/build/test/lib/jars/jopt-simple-5.0.4.jar:${project.dir}/build/test/lib/jars/jsr305-3.0.2.jar:${project.dir}/build/test/lib/jars/junit-4.12.jar:${project.dir}/build/test/lib/jars/mockito-core-3.2.4.jar:${project.dir}/build/test/lib/jars/objenesis-2.6.jar:${project.dir}/build/test/lib/jars/org.jacoco.agent-0.8.6.jar:${project.dir}/build/test/lib/jars/org.jacoco.ant-0.8.6.jar:${project.dir}/build/test/lib/jars/org.jacoco.core-0.8.6.jar:${project.dir}/build/test/lib/jars/org.jacoco.report-0.8.6.jar:${project.dir}/build/test/lib/jars/quicktheories-0.26.jar:${project.dir}/build/test/lib/jars/reflections-0.10.2.jar:${project.dir}/build/test/lib/jars/semver4j-3.1.0.jar:${project.dir}/build/test/lib/jars/slf4j-api-1.7.32.jar:</property> </properties> <folders> <source-folder> diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/CompressionUpgradeTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/CompressionUpgradeTest.java new file mode 100644 index 000000000000..33ca4272f772 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/CompressionUpgradeTest.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.upgrade; + +import org.junit.Test; + +import org.apache.cassandra.io.compress.DeflateCompressor; +import org.apache.cassandra.io.compress.LZ4Compressor; +import org.apache.cassandra.io.compress.SnappyCompressor; + +import static java.lang.String.format; +import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL; +import static org.apache.cassandra.distributed.api.Feature.GOSSIP; +import static org.apache.cassandra.distributed.api.Feature.NETWORK; +import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows; +import static org.apache.cassandra.distributed.shared.AssertUtils.fail; +import static org.apache.cassandra.distributed.shared.AssertUtils.row; + +public class CompressionUpgradeTest extends UpgradeTestBase +{ + @Test + public void testCompressors() throws Throwable + { + run(new String[][]{ + forCompressor(SnappyCompressor.class), + forCompressor(DeflateCompressor.class), + forCompressor(LZ4Compressor.class) + }); + } + + private String[] forCompressor(Class<?> compressorClass) + { + String compressorName = compressorClass.getSimpleName(); + return new String[]{ + "CREATE TABLE " + KEYSPACE + ".tbl_" + compressorName + " (pk int, ck int, v int, PRIMARY KEY (pk, ck)) WITH compression = {'class': '" + compressorName + "'}", + "INSERT INTO " + KEYSPACE + ".tbl_" + compressorName + " (pk, ck, v) VALUES (1, 1, 1)", + "SELECT * FROM " + KEYSPACE + ".tbl_" + compressorName + " WHERE pk = 1" + }; + } + + private void run(String[][] queries) throws Throwable + { + new TestCase() + .nodes(2) + .nodesToUpgrade(1) + .withConfig((cfg) -> cfg.with(NETWORK, GOSSIP)) + .upgradesFrom(v3X) + .setup((cluster) -> { + for (int i = 0; i < queries.length; i++) + { + try + { + cluster.schemaChange(queries[i][0]); + cluster.coordinator(1).execute(queries[i][1], ALL); + } + catch (Throwable t) + { + fail(format("Detected error against table %s", queries[i][0])); + } + } + }) + .runAfterNodeUpgrade((cluster, node) -> { + for (int i : new int[]{ 1, 2 }) + for (int j = 0; j < queries.length; j++) + { + try + { + assertRows(cluster.coordinator(i).execute(queries[j][2], ALL), row(1, 1, 1)); + } + catch (AssertionError e) + { + fail(format("Detected failed response from coordinator %s against table %s", i, queries[j][0])); + } + } + }).run(); + } +} From f375a3914f9e322e11ac30b2ea2999ff14bb65d4 Mon Sep 17 00:00:00 2001 From: Caleb Rackliffe <calebrackliffe@gmail.com> Date: Tue, 4 Mar 2025 18:04:08 -0600 Subject: [PATCH 170/225] Support null column value tombstones in FQL batch statements patch by Caleb Rackliffe; reviewed by Abe Ratnofsky for CASSANDRA-20397 --- CHANGES.txt | 1 + .../apache/cassandra/fql/FullQueryLogger.java | 4 +- .../{ => fql}/FqlReplayDDLExclusionTest.java | 3 +- .../test/fql/FqlTombstoneHandlingTest.java | 115 ++++++++++++++++++ .../cassandra/fqltool/FQLQueryReader.java | 5 +- .../cassandra/fqltool/commands/Dump.java | 9 +- 6 files changed, 129 insertions(+), 8 deletions(-) rename test/distributed/org/apache/cassandra/distributed/test/{ => fql}/FqlReplayDDLExclusionTest.java (97%) create mode 100644 test/distributed/org/apache/cassandra/distributed/test/fql/FqlTombstoneHandlingTest.java diff --git a/CHANGES.txt b/CHANGES.txt index fff1039d33ba..30d253d29b56 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0.18 + * Support null column value tombstones in FQL batch statements (CASSANDRA-20397) * Update Zstd library to 1.5.7-1 (CASSANDRA-20367) * Fix premature auto-failing of long-running repairs (CASSANDRA-20312) diff --git a/src/java/org/apache/cassandra/fql/FullQueryLogger.java b/src/java/org/apache/cassandra/fql/FullQueryLogger.java index 0604df67fb5c..34c9641e7769 100644 --- a/src/java/org/apache/cassandra/fql/FullQueryLogger.java +++ b/src/java/org/apache/cassandra/fql/FullQueryLogger.java @@ -424,9 +424,7 @@ public void writeMarshallablePayload(WireOut wire) { valueOut.int32(subValues.size()); for (ByteBuffer value : subValues) - { - valueOut.bytes(BytesStore.wrap(value)); - } + valueOut.bytes(value == null ? null : BytesStore.wrap(value)); } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/FqlReplayDDLExclusionTest.java b/test/distributed/org/apache/cassandra/distributed/test/fql/FqlReplayDDLExclusionTest.java similarity index 97% rename from test/distributed/org/apache/cassandra/distributed/test/FqlReplayDDLExclusionTest.java rename to test/distributed/org/apache/cassandra/distributed/test/fql/FqlReplayDDLExclusionTest.java index 3dbef35c78c2..daa1ae25d617 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/FqlReplayDDLExclusionTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/fql/FqlReplayDDLExclusionTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.distributed.test; +package org.apache.cassandra.distributed.test.fql; import org.junit.Ignore; import org.junit.Rule; @@ -27,6 +27,7 @@ import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.api.IInvokableInstance; import org.apache.cassandra.distributed.api.QueryResults; +import org.apache.cassandra.distributed.test.TestBaseImpl; import org.apache.cassandra.tools.ToolRunner; import org.apache.cassandra.tools.ToolRunner.ToolResult; diff --git a/test/distributed/org/apache/cassandra/distributed/test/fql/FqlTombstoneHandlingTest.java b/test/distributed/org/apache/cassandra/distributed/test/fql/FqlTombstoneHandlingTest.java new file mode 100644 index 000000000000..45e10ab1b08b --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/fql/FqlTombstoneHandlingTest.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.fql; + +import com.google.common.collect.Sets; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import com.datastax.driver.core.BatchStatement; +import com.datastax.driver.core.PreparedStatement; +import com.datastax.driver.core.Session; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.test.TestBaseImpl; +import org.apache.cassandra.tools.ToolRunner; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.apache.cassandra.distributed.api.Feature.GOSSIP; +import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL; +import static org.apache.cassandra.distributed.api.Feature.NETWORK; +import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows; +import static org.apache.cassandra.distributed.shared.AssertUtils.row; + +public class FqlTombstoneHandlingTest extends TestBaseImpl +{ + private static Cluster CLUSTER; + + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @BeforeClass + public static void beforeClass() throws Throwable + { + CLUSTER = init(Cluster.build(1).withConfig(updater -> updater.with(NETWORK, GOSSIP, NATIVE_PROTOCOL)).start()); + } + + @Test + public void testNullCellBindingInBatch() + { + String tableName = "null_as_tombstone_in_batch"; + CLUSTER.schemaChange(withKeyspace("CREATE TABLE %s." + tableName + " (k int, c int, s set<int>, primary key (k, c))")); + CLUSTER.get(1).nodetool("enablefullquerylog", "--path", temporaryFolder.getRoot().getAbsolutePath()); + String insertTemplate = withKeyspace("INSERT INTO %s." + tableName + " (k, c, s) VALUES ( ?, ?, ?) USING TIMESTAMP 2"); + String select = withKeyspace("SELECT * FROM %s." + tableName + " WHERE k = 0 AND c = 0"); + + com.datastax.driver.core.Cluster.Builder builder1 =com.datastax.driver.core.Cluster.builder().addContactPoint("127.0.0.1"); + + // Use the driver to write this initial row, since otherwise we won't hit the dispatcher + try (com.datastax.driver.core.Cluster cluster1 = builder1.build(); Session session1 = cluster1.connect()) + { + BatchStatement batch = new BatchStatement(BatchStatement.Type.UNLOGGED); + PreparedStatement preparedWrite = session1.prepare(insertTemplate); + batch.add(preparedWrite.bind(0, 0, null)); + session1.execute(batch); + } + + CLUSTER.get(1).nodetool("disablefullquerylog"); + + // The dump should contain a null entry for our tombstone + ToolRunner.ToolResult runner = ToolRunner.invokeClass("org.apache.cassandra.fqltool.FullQueryLogTool", + "dump", + "--", + temporaryFolder.getRoot().getAbsolutePath()); + assertTrue(runner.getStdout().contains(insertTemplate)); + assertEquals(0, runner.getExitCode()); + + Object[][] preReplayResult = CLUSTER.get(1).executeInternal(select); + assertRows(preReplayResult, row(0, 0, null)); + + // Make sure the row no longer exists after truncate... + CLUSTER.get(1).executeInternal(withKeyspace("TRUNCATE %s." + tableName)); + assertRows(CLUSTER.get(1).executeInternal(select)); + + // ...insert a new row with an actual value for the set at an earlier timestamp... + CLUSTER.get(1).executeInternal(withKeyspace("INSERT INTO %s." + tableName + " (k, c, s) VALUES ( ?, ?, ?) USING TIMESTAMP 1"), 0, 0, Sets.newHashSet(1)); + assertRows(CLUSTER.get(1).executeInternal(select), row(0, 0, Sets.newHashSet(1))); + + runner = ToolRunner.invokeClass("org.apache.cassandra.fqltool.FullQueryLogTool", + "replay", + "--keyspace", KEYSPACE, + "--target", "127.0.0.1", + "--", temporaryFolder.getRoot().getAbsolutePath()); + assertEquals(0, runner.getExitCode()); + + // ...then ensure the replayed row deletes the one we wrote before replay. + Object[][] postReplayResult = CLUSTER.get(1).executeInternal(select); + assertRows(postReplayResult, preReplayResult); + } + + @AfterClass + public static void afterClass() + { + if (CLUSTER != null) + CLUSTER.close(); + } +} diff --git a/tools/fqltool/src/org/apache/cassandra/fqltool/FQLQueryReader.java b/tools/fqltool/src/org/apache/cassandra/fqltool/FQLQueryReader.java index 20f362b2a689..c717a6e5cefa 100644 --- a/tools/fqltool/src/org/apache/cassandra/fqltool/FQLQueryReader.java +++ b/tools/fqltool/src/org/apache/cassandra/fqltool/FQLQueryReader.java @@ -94,7 +94,10 @@ public void readMarshallable(WireIn wireIn) throws IORuntimeException values.add(subValues); int numSubValues = in.int32(); for (int zz = 0; zz < numSubValues; zz++) - subValues.add(ByteBuffer.wrap(in.bytes())); + { + byte[] valueBytes = in.bytes(); + subValues.add(valueBytes == null ? null : ByteBuffer.wrap(valueBytes)); + } } query = new FQLQuery.Batch(keyspace, protocolVersion, diff --git a/tools/fqltool/src/org/apache/cassandra/fqltool/commands/Dump.java b/tools/fqltool/src/org/apache/cassandra/fqltool/commands/Dump.java index e954f8193312..e383dbc7ba54 100644 --- a/tools/fqltool/src/org/apache/cassandra/fqltool/commands/Dump.java +++ b/tools/fqltool/src/org/apache/cassandra/fqltool/commands/Dump.java @@ -126,7 +126,7 @@ public static void dump(List<String> arguments, String rollCycle, boolean follow break; case (FullQueryLogger.BATCH): - dumpBatch(options, wireIn, sb); + dumpBatch(wireIn, sb); break; default: @@ -183,7 +183,7 @@ static void dumpQuery(QueryOptions options, WireIn wireIn, StringBuilder sb) sb.append(System.lineSeparator()); } - private static void dumpBatch(QueryOptions options, WireIn wireIn, StringBuilder sb) + private static void dumpBatch(WireIn wireIn, StringBuilder sb) { sb.append("Batch type: ") .append(wireIn.read(FullQueryLogger.BATCH_TYPE).text()) @@ -203,7 +203,10 @@ private static void dumpBatch(QueryOptions options, WireIn wireIn, StringBuilder int numSubValues = in.int32(); List<ByteBuffer> subValues = new ArrayList<>(numSubValues); for (int j = 0; j < numSubValues; j++) - subValues.add(ByteBuffer.wrap(in.bytes())); + { + byte[] valueBytes = in.bytes(); + subValues.add(valueBytes == null ? null : ByteBuffer.wrap(valueBytes)); + } sb.append("Query: ") .append(queries.get(i)) From 3fb88e0f9d5a41d30e6d9116b972d28d296a35ad Mon Sep 17 00:00:00 2001 From: Maxwell Guo <cclive1601@gmail.com> Date: Thu, 6 Mar 2025 15:22:19 +0800 Subject: [PATCH 171/225] Add missed documentation for CREATE TABLE LIKE patch by Maxwell guo; reviewed by Brad Schoening and Michael Semb Wever for CASSANDRA-20491 --- CHANGES.txt | 1 + .../examples/BNF/create_table_like.bnf | 4 +- .../pages/developing/cql/cql_singlefile.adoc | 49 +++++++++++++++++++ 3 files changed, 52 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 6b4bc0d7c1dc..d326cc95c855 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Add missed documentation for CREATE TABLE LIKE (CASSANDRA-20401) * Add OCTET_LENGTH constraint (CASSANDRA-20340) * Reduce memory allocations in miscellaneous places along the hot write path (CASSANDRA-20167) * Provide keystore_password_file and truststore_password_file options to read credentials from a file (CASSANDRA-13428) diff --git a/doc/modules/cassandra/examples/BNF/create_table_like.bnf b/doc/modules/cassandra/examples/BNF/create_table_like.bnf index ebbd9b46588e..2a8914c75125 100644 --- a/doc/modules/cassandra/examples/BNF/create_table_like.bnf +++ b/doc/modules/cassandra/examples/BNF/create_table_like.bnf @@ -1,3 +1,3 @@ create_table_statement::= CREATE TABLE [ IF NOT EXISTS ] new_table_name LIKE old_table_name - [ WITH table_options ] -table_options::= INDEXES | options [ AND table_options ] \ No newline at end of file + [ WITH like_options ] +like_options::= INDEXES | options [ AND like_options ] \ No newline at end of file diff --git a/doc/modules/cassandra/pages/developing/cql/cql_singlefile.adoc b/doc/modules/cassandra/pages/developing/cql/cql_singlefile.adoc index 5b39524b380a..f2bf21590682 100644 --- a/doc/modules/cassandra/pages/developing/cql/cql_singlefile.adoc +++ b/doc/modules/cassandra/pages/developing/cql/cql_singlefile.adoc @@ -708,6 +708,55 @@ new columns (see `ALTER TABLE`) is a constant time operation. There is thus no need to try to anticipate future usage (or to cry when you haven’t) when creating a table. +[[copyTableStmt]] +=== CREATE TABLE LIKE + +_Syntax:_ + +[source,bnf] +include::cassandra:example$BNF/create_table_like.bnf[] + +_Sample:_ + +[source,sql] +---- +include::cassandra:example$CQL/create_table_like.cql[] +---- + +The `CREATE TABLE LIKE` statement creates a new empty table based on +the definition of an already existing table. By default, the new table +will have the same schema as the existing table. + +The statement allows changing the new table's options using the `WITH` +keyword. Additionally, the new table can create indexes with the same +schema as the existing table's indexes if the `INDEXES` keyword is specified. +However, only SAI and legacy secondary indexes are supported. TRIGGER +and MATERIALIZED VIEW are not supported at this time. + +Attempting to create a table that already exists will return an error +unless the IF NOT EXISTS option is used. If this option is used, the +statement will have no effect if the table already exists. + +[[copyTableStmt]] +==== `<tablename>` + +Valid the newly created table names are the same as valid +link:#createTableStmt[table names] (up to 32 characters long +alphanumerical identifiers). If the table name is provided alone, the +table is created within the current keyspace (see `USE`), but if it is +prefixed by an existing keyspace name (see +link:#copyTableStmt[`<tablename>`] grammar), it is created in the specified +keyspace (but does *not* change the current keyspace). + +[[createTableLikeOptions]] +==== `<like option>` + +The table options specified by `CREATE TABLE LIKE` statement are the same +as the supported link:#createTableOptions[supported `<option>`] in the +`CREATE TABLE` statement. And both are set throught keyword `WITH`. + +Indexs will be created together with table if `INDEXES` keyword is used. + [[alterTableStmt]] ==== ALTER TABLE From db38529ab801bfc801b07c7fd53db126dfc49a17 Mon Sep 17 00:00:00 2001 From: Dmitry Konstantinov <netudima@gmail.com> Date: Fri, 7 Mar 2025 23:24:22 +0300 Subject: [PATCH 172/225] Fix marking an SSTable as suspected and BufferPool leakage in case of a corrupted SSTable read during a compaction Patch by Dmitry Konstantinov; reviewed by Branimir Lambov for CASSANDRA-20396 --- CHANGES.txt | 1 + .../apache/cassandra/cache/ChunkCache.java | 12 +++++- .../io/sstable/SSTableIdentityIterator.java | 30 +++++++++++++ .../sstable/format/SSTableSimpleScanner.java | 42 ++++++++++++++----- 4 files changed, 73 insertions(+), 12 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index d326cc95c855..736400a3d31e 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Fix marking an SSTable as suspected and BufferPool leakage in case of a corrupted SSTable read during a compaction (CASSANDRA-20396) * Add missed documentation for CREATE TABLE LIKE (CASSANDRA-20401) * Add OCTET_LENGTH constraint (CASSANDRA-20340) * Reduce memory allocations in miscellaneous places along the hot write path (CASSANDRA-20167) diff --git a/src/java/org/apache/cassandra/cache/ChunkCache.java b/src/java/org/apache/cassandra/cache/ChunkCache.java index d62ccbacb822..afd5804089bc 100644 --- a/src/java/org/apache/cassandra/cache/ChunkCache.java +++ b/src/java/org/apache/cassandra/cache/ChunkCache.java @@ -161,8 +161,16 @@ public Buffer load(Key key) { ByteBuffer buffer = bufferPool.get(key.file.chunkSize(), key.file.preferredBufferType()); assert buffer != null; - key.file.readChunk(key.position, buffer); - return new Buffer(buffer, key.position); + try + { + key.file.readChunk(key.position, buffer); + return new Buffer(buffer, key.position); + } + catch (Throwable t) + { + bufferPool.put(buffer); + throw t; + } } @Override diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java b/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java index 072a364af32c..d5a1ae8bccc5 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java @@ -76,6 +76,11 @@ public static SSTableIdentityIterator create(SSTableReader sstable, RandomAccess sstable.markSuspect(); throw new CorruptSSTableException(e, file.getPath()); } + catch (CorruptSSTableException e) // to ensure that we marked the sstable as suspected if CorruptSSTableException is thrown from lower levels + { + sstable.markSuspect(); + throw e; + } } public static SSTableIdentityIterator create(SSTableReader sstable, FileDataInput dfile, long dataPosition, DecoratedKey key, boolean tombstoneOnly) @@ -99,6 +104,11 @@ public static SSTableIdentityIterator create(SSTableReader sstable, FileDataInpu sstable.markSuspect(); throw new CorruptSSTableException(e, dfile.getPath()); } + catch (CorruptSSTableException e) // to ensure that we marked the sstable as suspected if CorruptSSTableException is thrown from lower levels + { + sstable.markSuspect(); + throw e; + } } public static SSTableIdentityIterator create(SSTableReader sstable, FileDataInput dfile, boolean tombstoneOnly) @@ -121,6 +131,11 @@ public static SSTableIdentityIterator create(SSTableReader sstable, FileDataInpu sstable.markSuspect(); throw new CorruptSSTableException(e, dfile.getPath()); } + catch (CorruptSSTableException e) // to ensure that we marked the sstable as suspected if CorruptSSTableException is thrown from lower levels + { + sstable.markSuspect(); + throw e; + } } public TableMetadata metadata() @@ -164,6 +179,11 @@ public boolean hasNext() sstable.markSuspect(); throw new CorruptSSTableException(e, filename); } + catch (CorruptSSTableException e) // to ensure that we marked the sstable as suspected if CorruptSSTableException is thrown from lower levels + { + sstable.markSuspect(); + throw e; + } catch (IOError e) { if (e.getCause() instanceof IOException) @@ -192,6 +212,11 @@ public Unfiltered next() sstable.markSuspect(); throw new CorruptSSTableException(e, filename); } + catch (CorruptSSTableException e) // to ensure that we marked the sstable as suspected if CorruptSSTableException is thrown from lower levels + { + sstable.markSuspect(); + throw e; + } catch (IOError e) { if (e.getCause() instanceof IOException) @@ -240,6 +265,11 @@ public void exhaust() sstable.markSuspect(); throw new CorruptSSTableException(e, filename); } + catch (CorruptSSTableException e) // to ensure that we marked the sstable as suspected if CorruptSSTableException is thrown from lower levels + { + sstable.markSuspect(); + throw e; + } catch (IOError e) { if (e.getCause() instanceof IOException) diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableSimpleScanner.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableSimpleScanner.java index 6015265ba045..190ec42fa939 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableSimpleScanner.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableSimpleScanner.java @@ -17,6 +17,8 @@ */ package org.apache.cassandra.io.sstable.format; +import java.io.IOError; +import java.io.IOException; import java.util.Collection; import java.util.Iterator; import java.util.NoSuchElementException; @@ -148,19 +150,39 @@ public boolean hasNext() boolean advanceRange() { - if (!rangeIterator.hasNext()) - return false; + try + { + if (!rangeIterator.hasNext()) + return false; - bytesScannedInPreviousRanges += currentEndPosition - currentStartPosition; + bytesScannedInPreviousRanges += currentEndPosition - currentStartPosition; - PartitionPositionBounds nextRange = rangeIterator.next(); - if (currentEndPosition > nextRange.lowerPosition) - throw new IllegalArgumentException("Ranges supplied to SSTableSimpleScanner must be non-overlapping and in ascending order."); + PartitionPositionBounds nextRange = rangeIterator.next(); + if (currentEndPosition > nextRange.lowerPosition) + throw new IllegalArgumentException("Ranges supplied to SSTableSimpleScanner must be non-overlapping and in ascending order."); - currentEndPosition = nextRange.upperPosition; - currentStartPosition = nextRange.lowerPosition; - dfile.seek(currentStartPosition); - return true; + currentEndPosition = nextRange.upperPosition; + currentStartPosition = nextRange.lowerPosition; + dfile.seek(currentStartPosition); + return true; + } + catch (CorruptSSTableException e) + { + sstable.markSuspect(); + throw e; + } + catch (IOError e) + { + if (e.getCause() instanceof IOException) + { + sstable.markSuspect(); + throw new CorruptSSTableException((Exception)e.getCause(), sstable.getFilename()); + } + else + { + throw e; + } + } } public UnfilteredRowIterator next() From ef13e9f9bf0eed0531b0eee9b426754257af2c12 Mon Sep 17 00:00:00 2001 From: Ariel Weisberg <aweisberg@apple.com> Date: Fri, 7 Mar 2025 15:35:33 -0500 Subject: [PATCH 173/225] Ensure only offline tools can build IntervalTrees without first/last key fields patch by Ariel Weisberg; reviewed by Caleb Rackliffe for CASSANDRA-20407 --- CHANGES.txt | 1 + .../db/lifecycle/SSTableIntervalTree.java | 25 ++++++++++++++++++- .../io/sstable/format/SSTableReader.java | 2 +- .../org/apache/cassandra/utils/Interval.java | 4 +++ 4 files changed, 30 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 46d57bf4a0bc..a0726e8a367f 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.4 + * Ensure only offline tools can build IntervalTrees without first/last key fields (CASSANDRA-20407) * Improve IntervalTree build throughput (CASSANDRA-19596) * Avoid limit on RFP fetch in the case of an unresolved static row (CASSANDRA-20323) * Include materialized views to the output of DESCRIBE TABLE statements (CASSANDRA-20365) diff --git a/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java b/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java index 9f28fc5995b9..4d5a87f3991d 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java +++ b/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java @@ -25,11 +25,14 @@ import java.util.Collections; import java.util.List; +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.utils.Interval; import org.apache.cassandra.utils.IntervalTree; +import static com.google.common.base.Preconditions.checkState; + public class SSTableIntervalTree extends IntervalTree<PartitionPosition, SSTableReader, Interval<PartitionPosition, SSTableReader>> { private static final SSTableIntervalTree EMPTY = new SSTableIntervalTree(null); @@ -75,8 +78,28 @@ public static Interval<PartitionPosition, SSTableReader>[] buildIntervalsArray(C return IntervalTree.EMPTY_ARRAY; Interval<PartitionPosition, SSTableReader>[] intervals = new Interval[sstables.size()]; int i = 0; + int missingIntervals = 0; for (SSTableReader sstable : sstables) - intervals[i++] = sstable.getInterval(); + { + Interval<PartitionPosition, SSTableReader> interval = sstable.getInterval(); + if (interval == null) + { + missingIntervals++; + continue; + } + intervals[i++] = interval; + } + + // Offline (scrub) tools create SSTableReader without a first and last key and the old interval tree + // built a corrupt tree that couldn't be searched so continue to do that rather than complicate Tracker/View + if (missingIntervals > 0) + { + checkState(DatabaseDescriptor.isToolInitialized(), "Can only safely build an interval tree on sstables with missing first and last for offline tools"); + Interval<PartitionPosition, SSTableReader>[] replacementIntervals = new Interval[intervals.length - missingIntervals]; + System.arraycopy(intervals, 0, replacementIntervals, 0, replacementIntervals.length); + return replacementIntervals; + } + return intervals; } diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java index 1824412d9bbe..75272a0b5f04 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java @@ -475,7 +475,7 @@ protected SSTableReader(Builder<?, ?> builder, Owner owner) this.openReason = builder.getOpenReason(); this.first = builder.getFirst(); this.last = builder.getLast(); - this.interval = Interval.create(first, last, this); + this.interval = first == null || last == null ? null : Interval.create(first, last, this); this.bounds = first == null || last == null || AbstractBounds.strictlyWrapsAround(first.getToken(), last.getToken()) ? null // this will cause the validation to fail, but the reader is opened with no validation, // e.g. for scrubbing, we should accept screwed bounds diff --git a/src/java/org/apache/cassandra/utils/Interval.java b/src/java/org/apache/cassandra/utils/Interval.java index 6e7b8c33a3fb..b84e0a9d8388 100644 --- a/src/java/org/apache/cassandra/utils/Interval.java +++ b/src/java/org/apache/cassandra/utils/Interval.java @@ -19,6 +19,8 @@ import com.google.common.base.Objects; +import static com.google.common.base.Preconditions.checkNotNull; + public class Interval<C, D> { public final C min; @@ -27,6 +29,8 @@ public class Interval<C, D> public Interval(C min, C max, D data) { + checkNotNull(min, "min is null"); + checkNotNull(max, "max is null"); this.min = min; this.max = max; this.data = data; From 31ee3a795b023c3a6611621105040ab3c89c16d2 Mon Sep 17 00:00:00 2001 From: maoling <maoling@apache.org> Date: Fri, 31 May 2024 23:00:00 +0800 Subject: [PATCH 174/225] Make more unit tests generate test-related files in the temporary directory patch by Ling Mao; reviewed by Isaac Reath, Paulo Motta, Stefan Miklosovic for CASSANDRA-19672 --- .../service/StorageServiceServerTest.java | 23 ++++++++ .../cassandra/tools/CompactionStressTest.java | 59 +++++++++++-------- .../cassandra/tools/cqlsh/CqlshTest.java | 7 ++- .../tools/nodetool/GetAuditLogTest.java | 22 +++++++ 4 files changed, 85 insertions(+), 26 deletions(-) diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java index 83e060fe3246..7814b82ba08a 100644 --- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java +++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java @@ -21,6 +21,8 @@ import java.io.IOException; import java.net.UnknownHostException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -36,6 +38,7 @@ import org.apache.cassandra.ServerTestUtils; import org.apache.cassandra.audit.AuditLogManager; +import org.apache.cassandra.audit.AuditLogOptions; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.dht.Murmur3Partitioner.LongToken; @@ -562,6 +565,9 @@ public void testGetNativeAddressIPV6() throws Exception @Test public void testAuditLogEnableLoggerNotFound() throws Exception { + AuditLogOptions options = getBaseAuditLogOptions(); + DatabaseDescriptor.setAuditLoggingOptions(options); + StorageService.instance.enableAuditLog(null, null, null, null, null, null, null, null); assertTrue(AuditLogManager.instance.isEnabled()); try @@ -578,6 +584,9 @@ public void testAuditLogEnableLoggerNotFound() throws Exception @Test public void testAuditLogEnableLoggerTransitions() throws Exception { + AuditLogOptions options = getBaseAuditLogOptions(); + DatabaseDescriptor.setAuditLoggingOptions(options); + StorageService.instance.enableAuditLog(null, null, null, null, null, null, null, null); assertTrue(AuditLogManager.instance.isEnabled()); @@ -594,4 +603,18 @@ public void testAuditLogEnableLoggerTransitions() throws Exception assertTrue(AuditLogManager.instance.isEnabled()); StorageService.instance.disableAuditLog(); } + + /** + Create a new AuditLogOptions instance with the log dir set appropriately to a temp dir for unit testing. + */ + private static AuditLogOptions getBaseAuditLogOptions() throws IOException + { + AuditLogOptions options = new AuditLogOptions(); + + // Ensure that we create a new audit log directory to separate outputs + Path tmpDir = Files.createTempDirectory("StorageServiceServerTestForAuditLog"); + options.audit_logs_dir = tmpDir.toString(); + + return options; + } } diff --git a/test/unit/org/apache/cassandra/tools/CompactionStressTest.java b/test/unit/org/apache/cassandra/tools/CompactionStressTest.java index 148856ed90d3..07afeb424301 100644 --- a/test/unit/org/apache/cassandra/tools/CompactionStressTest.java +++ b/test/unit/org/apache/cassandra/tools/CompactionStressTest.java @@ -18,12 +18,15 @@ package org.apache.cassandra.tools; - -import org.apache.cassandra.io.util.File; import org.junit.Test; +import org.apache.cassandra.distributed.shared.WithProperties; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.tools.ToolRunner.ToolResult; +import static org.apache.cassandra.config.CassandraRelevantProperties.LOG_DIR; + public class CompactionStressTest extends OfflineToolUtils { @Test @@ -36,30 +39,36 @@ public void testNoArgs() @Test public void testWriteAndCompact() { - ClassLoader classLoader = getClass().getClassLoader(); - File file = new File(classLoader.getResource("blogpost.yaml").getFile()); - String profileFile = file.absolutePath(); + File tmpDir = FileUtils.getTempDir(); + tmpDir.deleteRecursiveOnExit(); - ToolResult tool = ToolRunner.invokeClass("org.apache.cassandra.stress.CompactionStress", - "write", - "-d", - "build/test/cassandra", - "-g", - "0", - "-p", - profileFile, - "-t", - "8"); - tool.assertOnCleanExit(); + // For the implementation of CompactionLogger, set the LOG_DIR to a tmp + // directory, the generated compaction.log file will be thrown in. + try (WithProperties ignore = new WithProperties().set(LOG_DIR, tmpDir.toString())) + { + ClassLoader classLoader = getClass().getClassLoader(); + File file = new File(classLoader.getResource("blogpost.yaml").getFile()); + String profileFile = file.absolutePath(); + + ToolRunner.invokeClass("org.apache.cassandra.stress.CompactionStress", + "write", + "-d", + "build/test/cassandra", + "-g", + "0", + "-p", + profileFile, + "-t", + "8").assertOnCleanExit(); - tool = ToolRunner.invokeClass("org.apache.cassandra.stress.CompactionStress", - "compact", - "-d", - "build/test/cassandra", - "-p", - profileFile, - "-t", - "8"); - tool.assertOnCleanExit(); + ToolRunner.invokeClass("org.apache.cassandra.stress.CompactionStress", + "compact", + "-d", + "build/test/cassandra", + "-p", + profileFile, + "-t", + "8").assertOnCleanExit(); + } } } diff --git a/test/unit/org/apache/cassandra/tools/cqlsh/CqlshTest.java b/test/unit/org/apache/cassandra/tools/cqlsh/CqlshTest.java index 356769b84064..9b7868c11867 100644 --- a/test/unit/org/apache/cassandra/tools/cqlsh/CqlshTest.java +++ b/test/unit/org/apache/cassandra/tools/cqlsh/CqlshTest.java @@ -29,6 +29,8 @@ import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.cql3.UntypedResultSet; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.tools.ToolRunner; import org.apache.cassandra.tools.ToolRunner.ToolResult; @@ -133,7 +135,10 @@ public void testCopyOnlyThoseRowsThatMatchVectorTypeSize() throws IOException Path csv = prepareCSVFile(rows); // when running COPY via cqlsh - ToolRunner.ToolResult result = ToolRunner.invokeCqlsh(format("COPY %s.%s FROM '%s'", KEYSPACE, currentTable(), csv.toAbsolutePath())); + Path tmpDir = Files.createTempDirectory("CqlshTest"); + File tempFile = FileUtils.createTempFile("testCopyOnlyThoseRowsThatMatchVectorTypeSize", "", new File(tmpDir)); + // Since this test has failure, with ERRFILE option of COPY command, we can put the err file to tmp directory + ToolRunner.ToolResult result = ToolRunner.invokeCqlsh(format("COPY %s.%s FROM '%s' WITH ERRFILE = '%s'", KEYSPACE, currentTable(), csv.toAbsolutePath(), tempFile)); // then only rows that match type size should be imported result.asserts().failure(); diff --git a/test/unit/org/apache/cassandra/tools/nodetool/GetAuditLogTest.java b/test/unit/org/apache/cassandra/tools/nodetool/GetAuditLogTest.java index b96187fb0cc7..e23c7f8c3d49 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/GetAuditLogTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/GetAuditLogTest.java @@ -18,10 +18,16 @@ package org.apache.cassandra.tools.nodetool; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; + import org.junit.After; import org.junit.BeforeClass; import org.junit.Test; +import org.apache.cassandra.audit.AuditLogOptions; +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.tools.ToolRunner; @@ -32,6 +38,8 @@ public class GetAuditLogTest extends CQLTester @BeforeClass public static void setup() throws Exception { + AuditLogOptions options = getBaseAuditLogOptions(); + DatabaseDescriptor.setAuditLoggingOptions(options); requireNetwork(); startJMXServer(); } @@ -150,4 +158,18 @@ private void testDefaultOutput(final String getAuditLogOutput) assertThat(output).contains("included_users \n"); assertThat(output).endsWith("excluded_users"); } + + /** + Create a new AuditLogOptions instance with the log dir set appropriately to a temp dir for unit testing. + */ + private static AuditLogOptions getBaseAuditLogOptions() throws IOException + { + AuditLogOptions options = new AuditLogOptions(); + + // Ensure that we create a new audit log directory to separate outputs + Path tmpDir = Files.createTempDirectory("GetAuditLogTest"); + options.audit_logs_dir = tmpDir.toString(); + + return options; + } } From a540eea80731752c3c3b2f295f73085be99fb5e0 Mon Sep 17 00:00:00 2001 From: maoling <maoling@apache.org> Date: Wed, 5 Mar 2025 23:48:00 +0800 Subject: [PATCH 175/225] Add more resources to .gitignore after CASSANDRA-19915 patch by Ling Mao; reviewed by Stefan Miklosovic, Michael Semb Wever for CASSANDRA-20391 --- .gitignore | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.gitignore b/.gitignore index 16cc10a5ae47..2508b527a67b 100644 --- a/.gitignore +++ b/.gitignore @@ -77,6 +77,8 @@ Thumbs.db # Generated files from the documentation doc/modules/cassandra/pages/managing/configuration/cass_yaml_file.adoc +doc/modules/cassandra/pages/reference/native-protocol.adoc +doc/modules/cassandra/attachments/native_protocol_v*.html doc/modules/cassandra/pages/managing/tools/nodetool/ doc/modules/cassandra/examples/TEXT/NODETOOL/ From 393901a85842b952a25e2c2d6dfc4de6d0d59c2c Mon Sep 17 00:00:00 2001 From: Brandon Williams <brandonwilliams@apache.org> Date: Thu, 6 Mar 2025 12:36:32 -0600 Subject: [PATCH 176/225] Suppress CVE-2024-12798 Patch by brandonwilliams; reviewed by smiklosovic for CASSANDRA-20408 --- .build/dependency-check-suppressions.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.build/dependency-check-suppressions.xml b/.build/dependency-check-suppressions.xml index f8bf01b41fdd..a25230a60b2b 100644 --- a/.build/dependency-check-suppressions.xml +++ b/.build/dependency-check-suppressions.xml @@ -63,9 +63,11 @@ <suppress> <packageUrl regex="true">^pkg:maven/ch\.qos\.logback/logback\-core@.*$</packageUrl> <cve>CVE-2023-6378</cve> + <cve>CVE-2024-12798</cve> </suppress> <suppress> <packageUrl regex="true">^pkg:maven/ch\.qos\.logback/logback\-classic@.*$</packageUrl> <cve>CVE-2023-6378</cve> + <cve>CVE-2024-12798</cve> </suppress> </suppressions> From 0e0ff79dd6b1b10bafcef93ab2437d707716716f Mon Sep 17 00:00:00 2001 From: Brandon Williams <brandonwilliams@apache.org> Date: Thu, 6 Mar 2025 12:38:23 -0600 Subject: [PATCH 177/225] Suppress CVE-2024-12798 Patch by brandonwilliams; reviewed by smiklosovic for CASSANDRA-20408 --- .build/owasp/dependency-check-suppressions.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.build/owasp/dependency-check-suppressions.xml b/.build/owasp/dependency-check-suppressions.xml index 994f3cb39562..a6a2816d956e 100644 --- a/.build/owasp/dependency-check-suppressions.xml +++ b/.build/owasp/dependency-check-suppressions.xml @@ -51,11 +51,13 @@ <packageUrl regex="true">^pkg:maven/ch\.qos\.logback/logback\-core@.*$</packageUrl> <cve>CVE-2023-6378</cve> <cve>CVE-2023-6481</cve> + <cve>CVE-2024-12798</cve> </suppress> <suppress> <packageUrl regex="true">^pkg:maven/ch\.qos\.logback/logback\-classic@.*$</packageUrl> <cve>CVE-2023-6378</cve> <cve>CVE-2023-6481</cve> + <cve>CVE-2024-12798</cve> </suppress> <!-- https://issues.apache.org/jira/browse/CASSANDRA-20024 --> From 4a66f68cba8b8b3c27c387eac88463579115770f Mon Sep 17 00:00:00 2001 From: Brandon Williams <brandonwilliams@apache.org> Date: Thu, 6 Mar 2025 12:21:20 -0600 Subject: [PATCH 178/225] Suppress CVE-2024-12801 Patch by brandonwilliams; reviewed by smiklosovic for CASSANDRA-20412 --- .build/dependency-check-suppressions.xml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.build/dependency-check-suppressions.xml b/.build/dependency-check-suppressions.xml index a25230a60b2b..4cdcc9149c7c 100644 --- a/.build/dependency-check-suppressions.xml +++ b/.build/dependency-check-suppressions.xml @@ -60,14 +60,17 @@ </suppress> <!-- https://issues.apache.org/jira/browse/CASSANDRA-19142 --> + <!-- https://issues.apache.org/jira/browse/CASSANDRA-20412 --> <suppress> <packageUrl regex="true">^pkg:maven/ch\.qos\.logback/logback\-core@.*$</packageUrl> <cve>CVE-2023-6378</cve> <cve>CVE-2024-12798</cve> + <cve>CVE-2024-12801</cve> </suppress> <suppress> <packageUrl regex="true">^pkg:maven/ch\.qos\.logback/logback\-classic@.*$</packageUrl> <cve>CVE-2023-6378</cve> <cve>CVE-2024-12798</cve> + <cve>CVE-2024-12801</cve> </suppress> </suppressions> From d509d2fcbe70d50013a9982f97832b3049d681f2 Mon Sep 17 00:00:00 2001 From: Brandon Williams <brandonwilliams@apache.org> Date: Thu, 6 Mar 2025 12:25:00 -0600 Subject: [PATCH 179/225] Suppress CVE-2024-12801 Patch by brandonwilliams; reviewed by smiklosovic for CASSANDRA-20412 --- .build/owasp/dependency-check-suppressions.xml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.build/owasp/dependency-check-suppressions.xml b/.build/owasp/dependency-check-suppressions.xml index a6a2816d956e..987b29d185bc 100644 --- a/.build/owasp/dependency-check-suppressions.xml +++ b/.build/owasp/dependency-check-suppressions.xml @@ -47,17 +47,20 @@ </suppress> <!-- https://issues.apache.org/jira/browse/CASSANDRA-19142 --> + <!-- https://issues.apache.org/jira/browse/CASSANDRA-20412 --> <suppress> <packageUrl regex="true">^pkg:maven/ch\.qos\.logback/logback\-core@.*$</packageUrl> <cve>CVE-2023-6378</cve> <cve>CVE-2023-6481</cve> <cve>CVE-2024-12798</cve> + <cve>CVE-2024-12801</cve> </suppress> <suppress> <packageUrl regex="true">^pkg:maven/ch\.qos\.logback/logback\-classic@.*$</packageUrl> <cve>CVE-2023-6378</cve> <cve>CVE-2023-6481</cve> <cve>CVE-2024-12798</cve> + <cve>CVE-2024-12801</cve> </suppress> <!-- https://issues.apache.org/jira/browse/CASSANDRA-20024 --> From 763994ba43a2d3cdf168fea2c0cbb8fa1f688159 Mon Sep 17 00:00:00 2001 From: Brandon Williams <brandonwilliams@apache.org> Date: Tue, 11 Mar 2025 15:14:15 -0500 Subject: [PATCH 180/225] Update changes for 20408/20412 --- CHANGES.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index 30d253d29b56..0ad954d81eed 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,6 @@ 4.0.18 + * Suppress CVE-2024-12801 (CASSANDRA-20412) + * Suppress CVE-2024-12798 (CASSANDRA-20408) * Support null column value tombstones in FQL batch statements (CASSANDRA-20397) * Update Zstd library to 1.5.7-1 (CASSANDRA-20367) * Fix premature auto-failing of long-running repairs (CASSANDRA-20312) From 5c845f3b090d3a53fa8837a1d9e2c874abad1b4a Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Fri, 28 Feb 2025 11:10:14 +0100 Subject: [PATCH 181/225] Move all disk error logic to DiskErrorsHandler to enable pluggability patch by Stefan Miklosovic; reviewed by Tommy Stendahl, Brandon Williams for CASSANDRA-20363 --- CHANGES.txt | 1 + .../config/CassandraRelevantProperties.java | 1 + .../cassandra/db/commitlog/CommitLog.java | 21 +- .../db/commitlog/CommitLogSegment.java | 2 +- .../apache/cassandra/io/FSErrorHandler.java | 31 --- .../apache/cassandra/io/util/FileUtils.java | 25 -- .../cassandra/service/CassandraDaemon.java | 2 +- ...ler.java => DefaultDiskErrorsHandler.java} | 122 ++++++++-- .../cassandra/service/DiskErrorsHandler.java | 77 ++++++ .../service/DiskErrorsHandlerService.java | 83 +++++++ .../cassandra/service/StorageService.java | 3 + .../utils/JVMStabilityInspector.java | 31 +-- .../cassandra/distributed/impl/Instance.java | 8 +- .../distributed/impl/InstanceKiller.java | 2 +- .../org/apache/cassandra/ServerTestUtils.java | 3 + .../apache/cassandra/cql3/OutOfSpaceTest.java | 4 + .../apache/cassandra/db/DirectoriesTest.java | 4 +- .../CommitLogInitWithExceptionTest.java | 3 + .../service/DefaultFSErrorHandlerTest.java | 3 +- .../service/DiskErrorsHandlerTest.java | 221 ++++++++++++++++++ .../service/DiskFailurePolicyTest.java | 3 +- .../snapshot/MetadataSnapshotsTest.java | 5 +- .../utils/JVMStabilityInspectorTest.java | 8 +- .../cassandra/utils/KillerForTests.java | 2 +- 24 files changed, 525 insertions(+), 140 deletions(-) delete mode 100644 src/java/org/apache/cassandra/io/FSErrorHandler.java rename src/java/org/apache/cassandra/service/{DefaultFSErrorHandler.java => DefaultDiskErrorsHandler.java} (60%) create mode 100644 src/java/org/apache/cassandra/service/DiskErrorsHandler.java create mode 100644 src/java/org/apache/cassandra/service/DiskErrorsHandlerService.java create mode 100644 test/unit/org/apache/cassandra/service/DiskErrorsHandlerTest.java diff --git a/CHANGES.txt b/CHANGES.txt index c70b5e8c8b19..a44294d6064a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Move all disk error logic to DiskErrorsHandler to enable pluggability (CASSANDRA-20363) * Fix marking an SSTable as suspected and BufferPool leakage in case of a corrupted SSTable read during a compaction (CASSANDRA-20396) * Add missed documentation for CREATE TABLE LIKE (CASSANDRA-20401) * Add OCTET_LENGTH constraint (CASSANDRA-20340) diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java index e212a9f72cdc..515bee33bb9a 100644 --- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java +++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java @@ -179,6 +179,7 @@ public enum CassandraRelevantProperties CONSISTENT_RANGE_MOVEMENT("cassandra.consistent.rangemovement", "true"), CONSISTENT_SIMULTANEOUS_MOVES_ALLOW("cassandra.consistent.simultaneousmoves.allow"), CRYPTO_PROVIDER_CLASS_NAME("cassandra.crypto_provider_class_name"), + CUSTOM_DISK_ERROR_HANDLER("cassandra.custom_disk_error_handler"), CUSTOM_GUARDRAILS_CONFIG_PROVIDER_CLASS("cassandra.custom_guardrails_config_provider_class"), CUSTOM_QUERY_HANDLER_CLASS("cassandra.custom_query_handler_class"), CUSTOM_TRACING_CLASS("cassandra.custom_tracing_class"), diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java index 30c8e08b8745..76cf38a804d6 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java @@ -57,8 +57,8 @@ import org.apache.cassandra.schema.CompressionParams; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.security.EncryptionContext; +import org.apache.cassandra.service.DiskErrorsHandlerService; import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.MBeanWrapper; import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException; @@ -576,24 +576,7 @@ public static long freeDiskSpace() @VisibleForTesting public static boolean handleCommitError(String message, Throwable t) { - JVMStabilityInspector.inspectCommitLogThrowable(t); - switch (DatabaseDescriptor.getCommitFailurePolicy()) - { - // Needed here for unit tests to not fail on default assertion - case die: - case stop: - StorageService.instance.stopTransports(); - //$FALL-THROUGH$ - case stop_commit: - String errorMsg = String.format("%s. Commit disk failure policy is %s; terminating thread.", message, DatabaseDescriptor.getCommitFailurePolicy()); - logger.error(addAdditionalInformationIfPossible(errorMsg), t); - return false; - case ignore: - logger.error(addAdditionalInformationIfPossible(message), t); - return true; - default: - throw new AssertionError(DatabaseDescriptor.getCommitFailurePolicy()); - } + return DiskErrorsHandlerService.get().handleCommitError(message, t); } /** diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java index e2aeef1dd247..d3c8e40de392 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java @@ -387,7 +387,7 @@ public static void writeCDCIndexFile(CommitLogDescriptor desc, int offset, boole } catch (IOException e) { - if (!CommitLog.instance.handleCommitError("Failed to sync CDC Index: " + desc.cdcIndexFileName(), e)) + if (!CommitLog.handleCommitError("Failed to sync CDC Index: " + desc.cdcIndexFileName(), e)) throw new RuntimeException(e); } } diff --git a/src/java/org/apache/cassandra/io/FSErrorHandler.java b/src/java/org/apache/cassandra/io/FSErrorHandler.java deleted file mode 100644 index b7d283640bd5..000000000000 --- a/src/java/org/apache/cassandra/io/FSErrorHandler.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.io; - -import org.apache.cassandra.io.sstable.CorruptSSTableException; - -/** - * Interface for handling file system errors. - */ -public interface FSErrorHandler -{ - void handleCorruptSSTable(CorruptSSTableException e); - void handleFSError(FSError e); - default void handleStartupFSError(Throwable t) {} -} diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java b/src/java/org/apache/cassandra/io/util/FileUtils.java index cf6ea52be16e..d5ea8dcde1be 100644 --- a/src/java/org/apache/cassandra/io/util/FileUtils.java +++ b/src/java/org/apache/cassandra/io/util/FileUtils.java @@ -44,10 +44,8 @@ import java.util.Collections; import java.util.EnumSet; import java.util.List; -import java.util.Optional; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -57,9 +55,7 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.io.FSError; -import org.apache.cassandra.io.FSErrorHandler; import org.apache.cassandra.io.FSWriteError; -import org.apache.cassandra.io.sstable.CorruptSSTableException; import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.SyncUtil; @@ -78,7 +74,6 @@ public final class FileUtils public static final long ONE_TIB = 1024 * ONE_GIB; private static final DecimalFormat df = new DecimalFormat("#.##"); - private static final AtomicReference<Optional<FSErrorHandler>> fsErrorHandler = new AtomicReference<>(Optional.empty()); private static final Class clsDirectBuffer; private static final MethodHandle mhDirectBufferCleaner; @@ -468,21 +463,6 @@ else if (value >= ONE_KIB) } } - public static void handleCorruptSSTable(CorruptSSTableException e) - { - fsErrorHandler.get().ifPresent(handler -> handler.handleCorruptSSTable(e)); - } - - public static void handleFSError(FSError e) - { - fsErrorHandler.get().ifPresent(handler -> handler.handleFSError(e)); - } - - public static void handleStartupFSError(Throwable t) - { - fsErrorHandler.get().ifPresent(handler -> handler.handleStartupFSError(t)); - } - /** * handleFSErrorAndPropagate will invoke the disk failure policy error handler, * which may or may not stop the daemon or transports. However, if we don't exit, @@ -626,11 +606,6 @@ public static List<String> readLines(File file) } } - public static void setFSErrorHandler(FSErrorHandler handler) - { - fsErrorHandler.getAndSet(Optional.ofNullable(handler)); - } - /** @deprecated See CASSANDRA-16926 */ @Deprecated(since = "4.1") public static void createDirectory(String directory) diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java index 8c44c4286d36..57f38e93e9a0 100644 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@ -224,7 +224,7 @@ public CassandraDaemon(boolean runManaged) */ protected void setup() { - FileUtils.setFSErrorHandler(new DefaultFSErrorHandler()); + DiskErrorsHandlerService.configure(); // Since CASSANDRA-14793 the local system keyspaces data are not dispatched across the data directories // anymore to reduce the risks in case of disk failures. By consequence, the system need to ensure in case of diff --git a/src/java/org/apache/cassandra/service/DefaultFSErrorHandler.java b/src/java/org/apache/cassandra/service/DefaultDiskErrorsHandler.java similarity index 60% rename from src/java/org/apache/cassandra/service/DefaultFSErrorHandler.java rename to src/java/org/apache/cassandra/service/DefaultDiskErrorsHandler.java index 8b182942b23e..cc350861f493 100644 --- a/src/java/org/apache/cassandra/service/DefaultFSErrorHandler.java +++ b/src/java/org/apache/cassandra/service/DefaultDiskErrorsHandler.java @@ -18,28 +18,44 @@ package org.apache.cassandra.service; - +import java.nio.file.FileStore; import java.util.Set; import com.google.common.collect.ImmutableSet; - -import org.apache.cassandra.io.util.File; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.config.Config; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.DisallowedDirectories; import org.apache.cassandra.db.Keyspace; -import org.apache.cassandra.io.*; +import org.apache.cassandra.io.FSDiskFullWriteError; +import org.apache.cassandra.io.FSError; +import org.apache.cassandra.io.FSNoDiskAvailableForWriteError; +import org.apache.cassandra.io.FSReadError; import org.apache.cassandra.io.sstable.CorruptSSTableException; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.io.util.PathUtils; import org.apache.cassandra.utils.JVMStabilityInspector; -public class DefaultFSErrorHandler implements FSErrorHandler +public class DefaultDiskErrorsHandler implements DiskErrorsHandler { - private static final Logger logger = LoggerFactory.getLogger(DefaultFSErrorHandler.class); + private static final Logger logger = LoggerFactory.getLogger(DefaultDiskErrorsHandler.class); private static final Set<Class<?>> exceptionsSkippingDataRemoval = ImmutableSet.of(OutOfMemoryError.class); + @Override + public void init() + { + // intentionally empty + } + + @Override + public void close() throws Exception + { + // intentionally empty + } + @Override public void handleCorruptSSTable(CorruptSSTableException e) { @@ -102,6 +118,68 @@ public void handleFSError(FSError e) } } + @Override + public void handleStartupFSError(Throwable t) + { + switch (DatabaseDescriptor.getDiskFailurePolicy()) + { + case stop_paranoid: + case stop: + case die: + logger.error("Exiting forcefully due to file system exception on startup, disk failure policy \"{}\"", + DatabaseDescriptor.getDiskFailurePolicy(), + t); + JVMStabilityInspector.killCurrentJVM(t, true); + break; + default: + break; + } + } + + @Override + public boolean handleCommitError(String message, Throwable t) + { + JVMStabilityInspector.inspectCommitLogThrowable(t); + switch (DatabaseDescriptor.getCommitFailurePolicy()) + { + // Needed here for unit tests to not fail on default assertion + case die: + case stop: + StorageService.instance.stopTransports(); + //$FALL-THROUGH$ + case stop_commit: + String errorMsg = String.format("%s. Commit disk failure policy is %s; terminating thread.", message, DatabaseDescriptor.getCommitFailurePolicy()); + logger.error(addAdditionalInformationIfPossible(errorMsg), t); + return false; + case ignore: + logger.error(addAdditionalInformationIfPossible(message), t); + return true; + default: + throw new AssertionError(DatabaseDescriptor.getCommitFailurePolicy()); + } + } + + @Override + public void inspectDiskError(Throwable t) + { + if (t instanceof CorruptSSTableException) + handleCorruptSSTable((CorruptSSTableException) t); + else if (t instanceof FSError) + handleFSError((FSError) t); + } + + @Override + public void inspectCommitLogError(Throwable t) + { + if (!StorageService.instance.isDaemonSetupCompleted()) + { + logger.error("Exiting due to error while processing commit log during initialization.", t); + JVMStabilityInspector.killCurrentJVM(t, true); + } + else if (DatabaseDescriptor.getCommitFailurePolicy() == Config.CommitFailurePolicy.die) + JVMStabilityInspector.killCurrentJVM(t, false); + } + private boolean shouldMaybeRemoveData(Throwable error) { for (Throwable t = error; t != null; t = t.getCause()) @@ -118,21 +196,27 @@ private boolean shouldMaybeRemoveData(Throwable error) return true; } - @Override - public void handleStartupFSError(Throwable t) + /** + * Add additional information to the error message if the commit directory does not have enough free space. + * + * @param msg the original error message + * @return the message with additional information if possible + */ + private static String addAdditionalInformationIfPossible(String msg) { - switch (DatabaseDescriptor.getDiskFailurePolicy()) + long unallocatedSpace = freeDiskSpace(); + int segmentSize = DatabaseDescriptor.getCommitLogSegmentSize(); + + if (unallocatedSpace < segmentSize) { - case stop_paranoid: - case stop: - case die: - logger.error("Exiting forcefully due to file system exception on startup, disk failure policy \"{}\"", - DatabaseDescriptor.getDiskFailurePolicy(), - t); - JVMStabilityInspector.killCurrentJVM(t, true); - break; - default: - break; + return String.format("%s. %d bytes required for next commitlog segment but only %d bytes available. Check %s to see if not enough free space is the reason for this error.", + msg, segmentSize, unallocatedSpace, DatabaseDescriptor.getCommitLogLocation()); } + return msg; + } + + private static long freeDiskSpace() + { + return PathUtils.tryGetSpace(new File(DatabaseDescriptor.getCommitLogLocation()).toPath(), FileStore::getTotalSpace); } } diff --git a/src/java/org/apache/cassandra/service/DiskErrorsHandler.java b/src/java/org/apache/cassandra/service/DiskErrorsHandler.java new file mode 100644 index 000000000000..b4fe9d67db67 --- /dev/null +++ b/src/java/org/apache/cassandra/service/DiskErrorsHandler.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.io.FSError; +import org.apache.cassandra.io.sstable.CorruptSSTableException; + +public interface DiskErrorsHandler extends AutoCloseable +{ + void init(); + + void handleCorruptSSTable(CorruptSSTableException e); + + void handleFSError(FSError e); + + boolean handleCommitError(String message, Throwable t); + + void handleStartupFSError(Throwable t); + + void inspectDiskError(Throwable t); + + void inspectCommitLogError(Throwable t); + + class NoOpDiskErrorHandler implements DiskErrorsHandler + { + public static final DiskErrorsHandler NO_OP = new NoOpDiskErrorHandler(); + + @VisibleForTesting + NoOpDiskErrorHandler() {} + + @Override + public void inspectCommitLogError(Throwable t) {} + + @Override + public boolean handleCommitError(String message, Throwable t) + { + // tracks what DefaultDiskErrorsHandler does when commit_failure_policy = ignore + return true; + } + + @Override + public void handleCorruptSSTable(CorruptSSTableException e) {} + + @Override + public void handleFSError(FSError e) {} + + @Override + public void handleStartupFSError(Throwable t) {} + + @Override + public void inspectDiskError(Throwable t) {} + + @Override + public void init() {} + + @Override + public void close() throws Exception {} + } +} diff --git a/src/java/org/apache/cassandra/service/DiskErrorsHandlerService.java b/src/java/org/apache/cassandra/service/DiskErrorsHandlerService.java new file mode 100644 index 000000000000..97e7ecde5fba --- /dev/null +++ b/src/java/org/apache/cassandra/service/DiskErrorsHandlerService.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.utils.FBUtilities; + +import static org.apache.cassandra.service.DiskErrorsHandler.NoOpDiskErrorHandler.NO_OP; + +public class DiskErrorsHandlerService +{ + private static final Logger logger = LoggerFactory.getLogger(DiskErrorsHandlerService.class); + + private static volatile DiskErrorsHandler instance = NO_OP; + + @VisibleForTesting + public static synchronized void set(DiskErrorsHandler newInstance) + { + if (newInstance == null) + return; + + DiskErrorsHandler oldInstance = DiskErrorsHandlerService.instance; + + try + { + newInstance.init(); + instance = newInstance; + + try + { + oldInstance.close(); + } + catch (Throwable t) + { + logger.warn("Exception occured while closing disk error handler of class " + oldInstance.getClass().getName(), t); + } + } + catch (Throwable t) + { + logger.warn("Exception occured while initializing disk error handler of class " + newInstance.getClass().getName(), t); + } + } + + public static DiskErrorsHandler get() + { + return instance; + } + + public static void close() throws Throwable + { + get().close(); + } + + public static void configure() throws ConfigurationException + { + String fsErrorHandlerClass = CassandraRelevantProperties.CUSTOM_DISK_ERROR_HANDLER.getString(); + DiskErrorsHandler fsErrorHandler = fsErrorHandlerClass == null + ? new DefaultDiskErrorsHandler() + : FBUtilities.construct(fsErrorHandlerClass, "disk error handler"); + DiskErrorsHandlerService.set(fsErrorHandler); + } +} diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index ab0cb41ee3f4..1aae64970e87 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -3838,6 +3838,9 @@ protected synchronized void drain(boolean isFinalShutdown) throws IOException, I // wait for miscellaneous tasks like sstable and commitlog segment deletion ColumnFamilyStore.shutdownPostFlushExecutor(); + if (isFinalShutdown) + DiskErrorsHandlerService.get().close(); + try { // we are not shutting down ScheduledExecutors#scheduledFastTasks to be still able to progress time diff --git a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java index 983e75f252fb..fdd678efd04b 100644 --- a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java +++ b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java @@ -32,6 +32,7 @@ import org.apache.cassandra.exceptions.UnrecoverableIllegalStateException; import org.apache.cassandra.metrics.StorageMetrics; +import org.apache.cassandra.service.DiskErrorsHandlerService; import org.apache.cassandra.tracing.Tracing; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,7 +43,6 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.io.FSError; import org.apache.cassandra.io.sstable.CorruptSSTableException; -import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException; @@ -86,20 +86,12 @@ public static void uncaughtException(Thread thread, Throwable t) */ public static void inspectThrowable(Throwable t) throws OutOfMemoryError { - inspectThrowable(t, JVMStabilityInspector::inspectDiskError); + inspectThrowable(t, DiskErrorsHandlerService.get()::inspectDiskError); } public static void inspectCommitLogThrowable(Throwable t) { - inspectThrowable(t, JVMStabilityInspector::inspectCommitLogError); - } - - private static void inspectDiskError(Throwable t) - { - if (t instanceof CorruptSSTableException) - FileUtils.handleCorruptSSTable((CorruptSSTableException) t); - else if (t instanceof FSError) - FileUtils.handleFSError((FSError) t); + inspectThrowable(t, ex -> DiskErrorsHandlerService.get().inspectCommitLogError(ex)); } public static void inspectThrowable(Throwable t, Consumer<Throwable> fn) throws OutOfMemoryError @@ -156,7 +148,7 @@ else if (t instanceof UnrecoverableIllegalStateException) if (isUnstable) { if (!StorageService.instance.isDaemonSetupCompleted()) - FileUtils.handleStartupFSError(t); + DiskErrorsHandlerService.get().handleStartupFSError(t); killer.killCurrentJVM(t); } @@ -197,17 +189,6 @@ private static void forceHeapSpaceOomMaybe(OutOfMemoryError oom) } } - private static void inspectCommitLogError(Throwable t) - { - if (!StorageService.instance.isDaemonSetupCompleted()) - { - logger.error("Exiting due to error while processing commit log during initialization.", t); - killer.killCurrentJVM(t, true); - } - else if (DatabaseDescriptor.getCommitFailurePolicy() == Config.CommitFailurePolicy.die) - killer.killCurrentJVM(t); - } - public static void killCurrentJVM(Throwable t, boolean quiet) { killer.killCurrentJVM(t, quiet); @@ -249,12 +230,12 @@ public static class Killer * @param t * The Throwable to log before killing the current JVM */ - protected void killCurrentJVM(Throwable t) + public void killCurrentJVM(Throwable t) { killCurrentJVM(t, false); } - protected void killCurrentJVM(Throwable t, boolean quiet) + public void killCurrentJVM(Throwable t, boolean quiet) { if (!quiet) { diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java index b98b36c47c93..a7cb238119db 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java @@ -109,7 +109,6 @@ import org.apache.cassandra.io.util.DataOutputBuffer; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.io.util.File; -import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.io.util.PathUtils; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.metrics.CassandraMetricsRegistry; @@ -124,7 +123,7 @@ import org.apache.cassandra.service.CassandraDaemon; import org.apache.cassandra.service.ClientState; import org.apache.cassandra.service.ClientWarn; -import org.apache.cassandra.service.DefaultFSErrorHandler; +import org.apache.cassandra.service.DiskErrorsHandlerService; import org.apache.cassandra.service.QueryState; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.service.StorageServiceMBean; @@ -715,7 +714,7 @@ protected void partialStartup(ICluster<?> cluster) throws IOException, NoSuchFie logSystemInfo(inInstancelogger); Config.log(DatabaseDescriptor.getRawConfig()); - FileUtils.setFSErrorHandler(new DefaultFSErrorHandler()); + DiskErrorsHandlerService.configure(); DatabaseDescriptor.createAllDirectories(); CassandraDaemon.getInstanceForTesting().migrateSystemDataIfNeeded(); @@ -956,7 +955,8 @@ public Future<Void> shutdown(boolean runOnExitThreads, boolean shutdownMessaging () -> shutdownAndWait(Collections.singletonList(ActiveRepairService.repairCommandExecutor())), () -> ActiveRepairService.instance().shutdownNowAndWait(1L, MINUTES), () -> EpochAwareDebounce.instance.close(), - SnapshotManager.instance::close + SnapshotManager.instance::close, + DiskErrorsHandlerService::close ); internodeMessagingStarted = false; diff --git a/test/distributed/org/apache/cassandra/distributed/impl/InstanceKiller.java b/test/distributed/org/apache/cassandra/distributed/impl/InstanceKiller.java index 38b045b381dc..6f2576cebc47 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/InstanceKiller.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/InstanceKiller.java @@ -45,7 +45,7 @@ public static void clear() } @Override - protected void killCurrentJVM(Throwable t, boolean quiet) + public void killCurrentJVM(Throwable t, boolean quiet) { KILL_ATTEMPTS.incrementAndGet(); onKill.accept(quiet); diff --git a/test/unit/org/apache/cassandra/ServerTestUtils.java b/test/unit/org/apache/cassandra/ServerTestUtils.java index 596312ff0b11..ead4a1a558cb 100644 --- a/test/unit/org/apache/cassandra/ServerTestUtils.java +++ b/test/unit/org/apache/cassandra/ServerTestUtils.java @@ -48,6 +48,7 @@ import org.apache.cassandra.locator.Replica; import org.apache.cassandra.locator.BaseProximity; import org.apache.cassandra.security.ThreadAwareSecurityManager; +import org.apache.cassandra.service.DiskErrorsHandlerService; import org.apache.cassandra.service.EmbeddedCassandraService; import org.apache.cassandra.tcm.AtomicLongBackedProcessor; import org.apache.cassandra.tcm.ClusterMetadata; @@ -178,6 +179,8 @@ public void uncaughtException(Thread t, Throwable e) SystemKeyspace.persistLocalMetadata(); AuditLogManager.instance.initialize(); + DiskErrorsHandlerService.configure(); + isServerPrepared = true; } diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java index 30e12ea173c7..072ee0872006 100644 --- a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java +++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java @@ -36,6 +36,8 @@ import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.io.FSWriteError; import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.service.CassandraDaemon; +import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.KillerForTests; @@ -59,6 +61,8 @@ public static void setUpClass() DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); ServerTestUtils.prepareServerNoRegister(); ServerTestUtils.markCMS(); + StorageService.instance.registerDaemon(CassandraDaemon.getInstanceForTesting()); + CassandraDaemon.getInstanceForTesting().completeSetup(); } @Test diff --git a/test/unit/org/apache/cassandra/db/DirectoriesTest.java b/test/unit/org/apache/cassandra/db/DirectoriesTest.java index 06db7e4a02a8..496312503470 100644 --- a/test/unit/org/apache/cassandra/db/DirectoriesTest.java +++ b/test/unit/org/apache/cassandra/db/DirectoriesTest.java @@ -95,7 +95,7 @@ import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.SchemaKeyspaceTables; import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.service.DefaultFSErrorHandler; +import org.apache.cassandra.service.DiskErrorsHandlerService; import org.apache.cassandra.service.snapshot.SnapshotLoader; import org.apache.cassandra.service.snapshot.SnapshotManifest; import org.apache.cassandra.service.snapshot.TableSnapshot; @@ -151,7 +151,7 @@ public static Collection<Object[]> idBuilders() public static void beforeClass() { DatabaseDescriptor.daemonInitialization(); - FileUtils.setFSErrorHandler(new DefaultFSErrorHandler()); + DiskErrorsHandlerService.configure(); } @Before diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogInitWithExceptionTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogInitWithExceptionTest.java index b3cff94c66af..05acf17f8d9d 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogInitWithExceptionTest.java +++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogInitWithExceptionTest.java @@ -20,6 +20,7 @@ import org.apache.cassandra.Util; +import org.apache.cassandra.service.DiskErrorsHandlerService; import org.apache.cassandra.utils.concurrent.Condition; import org.junit.Assert; import org.junit.BeforeClass; @@ -62,6 +63,8 @@ public static void setUp() killed.signal(); } }; + + DiskErrorsHandlerService.configure(); } @Test diff --git a/test/unit/org/apache/cassandra/service/DefaultFSErrorHandlerTest.java b/test/unit/org/apache/cassandra/service/DefaultFSErrorHandlerTest.java index b1da62fcdcf7..652af819169f 100644 --- a/test/unit/org/apache/cassandra/service/DefaultFSErrorHandlerTest.java +++ b/test/unit/org/apache/cassandra/service/DefaultFSErrorHandlerTest.java @@ -35,7 +35,6 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.gms.Gossiper; -import org.apache.cassandra.io.FSErrorHandler; import org.apache.cassandra.io.FSReadError; import org.apache.cassandra.io.sstable.CorruptSSTableException; @@ -46,7 +45,7 @@ @RunWith(Parameterized.class) public class DefaultFSErrorHandlerTest { - private FSErrorHandler handler = new DefaultFSErrorHandler(); + private DiskErrorsHandler handler = new DefaultDiskErrorsHandler(); Config.DiskFailurePolicy oldDiskPolicy; Config.DiskFailurePolicy testDiskPolicy; private boolean gossipRunningFSError; diff --git a/test/unit/org/apache/cassandra/service/DiskErrorsHandlerTest.java b/test/unit/org/apache/cassandra/service/DiskErrorsHandlerTest.java new file mode 100644 index 000000000000..6465164fe0ee --- /dev/null +++ b/test/unit/org/apache/cassandra/service/DiskErrorsHandlerTest.java @@ -0,0 +1,221 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service; + +import org.junit.Test; + +import org.apache.cassandra.distributed.shared.WithProperties; +import org.apache.cassandra.io.FSError; +import org.apache.cassandra.io.sstable.CorruptSSTableException; + +import static org.apache.cassandra.config.CassandraRelevantProperties.CUSTOM_DISK_ERROR_HANDLER; +import static org.apache.cassandra.service.DiskErrorsHandlerService.get; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; + +public class DiskErrorsHandlerTest +{ + @Test + public void testSetting() throws Throwable + { + try (WithProperties ignore = new WithProperties().set(CUSTOM_DISK_ERROR_HANDLER, + HandlerA.class.getName())) + { + DiskErrorsHandlerService.configure(); + + assertSame(HandlerA.class, get().getClass()); + + assertTrue(HandlerA.initialized); + assertFalse(HandlerA.closed); + assertFalse(HandlerB.initialized); + assertFalse(HandlerB.closed); + } + + try (WithProperties ignore = new WithProperties().set(CUSTOM_DISK_ERROR_HANDLER, + HandlerB.class.getName())) + { + DiskErrorsHandlerService.configure(); + + assertTrue(HandlerA.initialized); + assertTrue(HandlerA.closed); + + assertTrue(HandlerB.initialized); + assertFalse(HandlerB.closed); + + assertSame(HandlerB.class, get().getClass()); + + get().close(); + + assertTrue(HandlerB.closed); + } + } + + @Test + public void testFailures() + { + // failed closing + try (WithProperties ignore = new WithProperties().set(CUSTOM_DISK_ERROR_HANDLER, + HandlerC.class.getName())) + { + DiskErrorsHandlerService.configure(); + assertTrue(HandlerC.initialized); + assertSame(HandlerC.class, get().getClass()); + } + + // this will call close() on C handler + try (WithProperties ignore = new WithProperties().set(CUSTOM_DISK_ERROR_HANDLER, + HandlerE.class.getName())) + { + DiskErrorsHandlerService.configure(); + assertTrue(HandlerE.initialized); + assertSame(HandlerE.class, get().getClass()); + } + + try (WithProperties ignore = new WithProperties().set(CUSTOM_DISK_ERROR_HANDLER, + HandlerD.class.getName())) + { + DiskErrorsHandlerService.configure(); + // still handler E as handler D failed to init + assertSame(HandlerE.class, get().getClass()); + } + } + + public static class HandlerA extends DummyErrorHandler + { + public static boolean initialized = false; + public static boolean closed = false; + + @Override + public void init() + { + initialized = true; + } + + @Override + public void close() throws Exception + { + closed = true; + } + } + + public static class HandlerB extends DummyErrorHandler + { + public static boolean initialized = false; + public static boolean closed = false; + + @Override + public void init() + { + initialized = true; + } + + @Override + public void close() throws Exception + { + closed = true; + } + } + + public static class HandlerC extends DummyErrorHandler + { + public static boolean initialized = false; + + @Override + public void init() + { + initialized = true; + } + + @Override + public void close() throws Exception + { + throw new RuntimeException("failed to close"); + } + } + + public static class HandlerD extends DummyErrorHandler + { + public static boolean closed = false; + + @Override + public void init() + { + throw new RuntimeException("failed to init"); + } + + @Override + public void close() throws Exception + { + closed = true; + } + } + + public static class HandlerE extends DummyErrorHandler + { + public static boolean initialized = false; + public static boolean closed = false; + + @Override + public void init() + { + initialized = true; + } + + @Override + public void close() throws Exception + { + closed = true; + } + } + + private static abstract class DummyErrorHandler implements DiskErrorsHandler + { + @Override + public void handleCorruptSSTable(CorruptSSTableException e) + { + } + + @Override + public void handleFSError(FSError e) + { + } + + @Override + public void handleStartupFSError(Throwable t) + { + } + + @Override + public void inspectDiskError(Throwable t) + { + } + + @Override + public void inspectCommitLogError(Throwable t) + { + } + + @Override + public boolean handleCommitError(String message, Throwable t) + { + return true; + } + } +} diff --git a/test/unit/org/apache/cassandra/service/DiskFailurePolicyTest.java b/test/unit/org/apache/cassandra/service/DiskFailurePolicyTest.java index 832c631eeddf..f1485d0ac694 100644 --- a/test/unit/org/apache/cassandra/service/DiskFailurePolicyTest.java +++ b/test/unit/org/apache/cassandra/service/DiskFailurePolicyTest.java @@ -38,7 +38,6 @@ import org.apache.cassandra.io.FSReadError; import org.apache.cassandra.io.sstable.CorruptSSTableException; import org.apache.cassandra.io.util.File; -import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.KillerForTests; @@ -72,7 +71,7 @@ public static void defineSchema() throws ConfigurationException JOIN_RING.setBoolean(false); // required to start gossiper without setting tokens SchemaLoader.prepareServer(); StorageService.instance.initServer(); - FileUtils.setFSErrorHandler(new DefaultFSErrorHandler()); + DiskErrorsHandlerService.configure(); } public DiskFailurePolicyTest(DiskFailurePolicy testPolicy, boolean isStartUpInProgress, Throwable t, diff --git a/test/unit/org/apache/cassandra/service/snapshot/MetadataSnapshotsTest.java b/test/unit/org/apache/cassandra/service/snapshot/MetadataSnapshotsTest.java index ee996e3a333e..ab2ae50bf565 100644 --- a/test/unit/org/apache/cassandra/service/snapshot/MetadataSnapshotsTest.java +++ b/test/unit/org/apache/cassandra/service/snapshot/MetadataSnapshotsTest.java @@ -32,8 +32,7 @@ import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.io.util.FileUtils; -import org.apache.cassandra.service.DefaultFSErrorHandler; +import org.apache.cassandra.service.DiskErrorsHandlerService; import static org.apache.cassandra.service.snapshot.TableSnapshotTest.createFolders; import static org.apache.cassandra.utils.FBUtilities.now; @@ -50,7 +49,7 @@ public static void beforeClass() CassandraRelevantProperties.SNAPSHOT_CLEANUP_PERIOD_SECONDS.setInt(3); DatabaseDescriptor.daemonInitialization(); - FileUtils.setFSErrorHandler(new DefaultFSErrorHandler()); + DiskErrorsHandlerService.configure(); } @ClassRule diff --git a/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java b/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java index 3a3415e4d4ff..172e5eb584f8 100644 --- a/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java +++ b/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java @@ -30,10 +30,10 @@ import org.apache.cassandra.io.FSReadError; import org.apache.cassandra.io.FSWriteError; import org.apache.cassandra.io.sstable.CorruptSSTableException; +import org.apache.cassandra.service.DiskErrorsHandler; +import org.apache.cassandra.service.DiskErrorsHandlerService; import org.assertj.core.api.Assertions; -import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.service.CassandraDaemon; -import org.apache.cassandra.service.DefaultFSErrorHandler; import org.apache.cassandra.service.StorageService; import static java.util.Arrays.asList; @@ -59,7 +59,7 @@ public void testKill() throws Exception Config.DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy(); Config.CommitFailurePolicy oldCommitPolicy = DatabaseDescriptor.getCommitFailurePolicy(); - FileUtils.setFSErrorHandler(new DefaultFSErrorHandler()); + DiskErrorsHandlerService.configure(); try { CassandraDaemon daemon = new CassandraDaemon(); @@ -114,7 +114,7 @@ public void testKill() throws Exception DatabaseDescriptor.setDiskFailurePolicy(oldPolicy); DatabaseDescriptor.setCommitFailurePolicy(oldCommitPolicy); StorageService.instance.registerDaemon(null); - FileUtils.setFSErrorHandler(null); + DiskErrorsHandlerService.set(DiskErrorsHandler.NoOpDiskErrorHandler.NO_OP); } } diff --git a/test/unit/org/apache/cassandra/utils/KillerForTests.java b/test/unit/org/apache/cassandra/utils/KillerForTests.java index b6c48d52e819..6f7f3c7b1020 100644 --- a/test/unit/org/apache/cassandra/utils/KillerForTests.java +++ b/test/unit/org/apache/cassandra/utils/KillerForTests.java @@ -40,7 +40,7 @@ public KillerForTests(boolean expectFailure) } @Override - protected void killCurrentJVM(Throwable t, boolean quiet) + public void killCurrentJVM(Throwable t, boolean quiet) { if (!expected) Assert.fail("Saw JVM Kill but did not expect it."); From 7c0a86323e1486a557e9d86a3bb8b5b799fb22a6 Mon Sep 17 00:00:00 2001 From: maoling <maoling@apache.org> Date: Thu, 20 Jun 2024 23:44:00 +0800 Subject: [PATCH 182/225] Add JVM version and Cassandra build date to nodetool version -v patch by Ling Mao; reviewed by Maxwell Guo, Stefan Miklosovic for CASSANDRA-19721 --- CHANGES.txt | 1 + build.xml | 15 ++- .../config/CassandraRelevantProperties.java | 1 + .../cassandra/service/StorageService.java | 7 ++ .../service/StorageServiceMBean.java | 7 ++ .../org/apache/cassandra/tools/NodeProbe.java | 5 + .../cassandra/tools/nodetool/Version.java | 7 ++ .../apache/cassandra/utils/FBUtilities.java | 20 ++-- .../cassandra/tools/nodetool/VersionTest.java | 99 +++++++++++++++++++ 9 files changed, 154 insertions(+), 8 deletions(-) create mode 100644 test/unit/org/apache/cassandra/tools/nodetool/VersionTest.java diff --git a/CHANGES.txt b/CHANGES.txt index a44294d6064a..1b38765b6296 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Add JVM version and Cassandra build date to nodetool version -v (CASSANDRA-19721) * Move all disk error logic to DiskErrorsHandler to enable pluggability (CASSANDRA-20363) * Fix marking an SSTable as suspected and BufferPool leakage in case of a corrupted SSTable read during a compaction (CASSANDRA-20396) * Add missed documentation for CREATE TABLE LIKE (CASSANDRA-20401) diff --git a/build.xml b/build.xml index ccc38717d28c..7b5351e203e5 100644 --- a/build.xml +++ b/build.xml @@ -490,15 +490,25 @@ </target> <!-- create properties file with C version --> - <target name="_createVersionPropFile" depends="_get-git-sha,set-cqlsh-version"> + <target name="_createVersionPropFile" depends="_get-git-sha,set-cqlsh-version,_set-build-date"> <taskdef name="propertyfile" classname="org.apache.tools.ant.taskdefs.optional.PropertyFile"/> <mkdir dir="${version.properties.dir}"/> <propertyfile file="${version.properties.dir}/version.properties"> <entry key="CassandraVersion" value="${version}"/> <entry key="GitSHA" value="${git.sha}"/> + <entry key="BuildDate" value="${build.date}"/> </propertyfile> </target> + <!-- set ant build date --> + <target name="_set-build-date"> + <tstamp> + <format property="build.date" pattern="yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"/> + </tstamp> + <!-- Include timezone information --> + <property name="build.date" value="${build.date}"/> + </target> + <target name="test-run" depends="jar" description="Run in test mode. Not for production use!"> <java classname="org.apache.cassandra.service.CassandraDaemon" fork="true"> @@ -735,7 +745,7 @@ The jar target makes cassandra.jar output. --> <target name="_main-jar" - depends="build" + depends="build,_get-git-sha,_set-build-date" description="Assemble Cassandra JAR files"> <mkdir dir="${build.classes.main}/META-INF" /> <copy file="LICENSE.txt" @@ -754,6 +764,7 @@ <attribute name="Implementation-Version" value="${version}"/> <attribute name="Implementation-Vendor" value="Apache"/> <attribute name="Implementation-Git-SHA" value="${git.sha}"/> + <attribute name="Implementation-Build-Date" value="${build.date}"/> <!-- </section> --> </manifest> </jar> diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java index 515bee33bb9a..e83dfaf53aba 100644 --- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java +++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java @@ -74,6 +74,7 @@ public enum CassandraRelevantProperties BROADCAST_INTERVAL_MS("cassandra.broadcast_interval_ms", "60000"), BTREE_BRANCH_SHIFT("cassandra.btree.branchshift", "5"), BTREE_FAN_FACTOR("cassandra.btree.fanfactor"), + BUILD_DATE("cassandra.buildDate"), /** Represents the maximum size (in bytes) of a serialized mutation that can be cached **/ CACHEABLE_MUTATION_SIZE_LIMIT("cassandra.cacheable_mutation_size_limit_bytes", convertToString(1_000_000)), CASSANDRA_ALLOW_SIMPLE_STRATEGY("cassandra.allow_simplestrategy"), diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 1aae64970e87..d0f8711ea51b 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -695,6 +695,7 @@ public void unsafeInitialize() throws ConfigurationException public synchronized void initServer() throws ConfigurationException { logger.info("Cassandra version: {}", FBUtilities.getReleaseVersionString()); + logger.info("Build Date: {}", FBUtilities.getBuildDate()); logger.info("Git SHA: {}", FBUtilities.getGitSHA()); logger.info("CQL version: {}", QueryProcessor.CQL_VERSION); logger.info("Native protocol supported versions: {} (default: {})", @@ -2385,6 +2386,12 @@ public String getGitSHA() return FBUtilities.getGitSHA(); } + @Override + public String getBuildDate() + { + return FBUtilities.getBuildDate(); + } + public String getSchemaVersion() { return Schema.instance.getVersion().toString(); diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java index 57dfcea67338..dfaa436cdcc8 100644 --- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java +++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java @@ -113,6 +113,13 @@ public interface StorageServiceMBean extends NotificationEmitter */ public String getGitSHA(); + /** + * Fetch a string representation of the Cassandra's build date. + * The format: {@code yyyy-MM-dd'T'HH:mm:ss.SSS'Z'} + * @return A string representation of the Cassandra's build date. + */ + String getBuildDate(); + /** * Fetch a string representation of the current Schema version. * @return A string representation of the Schema version. diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index 5a6e8b359821..747da8348564 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -871,6 +871,11 @@ public String getGitSHA() return ssProxy.getGitSHA(); } + public String getBuildDate() + { + return ssProxy.getBuildDate(); + } + public int getCurrentGenerationNumber() { return ssProxy.getCurrentGenerationNumber(); diff --git a/src/java/org/apache/cassandra/tools/nodetool/Version.java b/src/java/org/apache/cassandra/tools/nodetool/Version.java index 6556a046272f..9b92249b3c08 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/Version.java +++ b/src/java/org/apache/cassandra/tools/nodetool/Version.java @@ -22,6 +22,9 @@ import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.tools.NodeTool.NodeToolCmd; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_VERSION; +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_VM_NAME; + @Command(name = "version", description = "Print cassandra version") public class Version extends NodeToolCmd { @@ -35,6 +38,10 @@ public void execute(NodeProbe probe) { probe.output().out.println("ReleaseVersion: " + probe.getReleaseVersion()); if (verbose) + { + probe.output().out.println("BuildDate: " + probe.getBuildDate()); probe.output().out.println("GitSHA: " + probe.getGitSHA()); + probe.output().out.printf("JVM vendor/version: %s/%s%n", JAVA_VM_NAME.getString(), JAVA_VERSION.getString()); + } } } diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java index d1eb0798d896..91b608d18553 100644 --- a/src/java/org/apache/cassandra/utils/FBUtilities.java +++ b/src/java/org/apache/cassandra/utils/FBUtilities.java @@ -95,6 +95,7 @@ import org.objectweb.asm.Opcodes; import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_AVAILABLE_PROCESSORS; +import static org.apache.cassandra.config.CassandraRelevantProperties.BUILD_DATE; import static org.apache.cassandra.config.CassandraRelevantProperties.GIT_SHA; import static org.apache.cassandra.config.CassandraRelevantProperties.LINE_SEPARATOR; import static org.apache.cassandra.config.CassandraRelevantProperties.OS_NAME; @@ -113,8 +114,7 @@ public class FBUtilities } private static final Logger logger = LoggerFactory.getLogger(FBUtilities.class); - public static final String UNKNOWN_RELEASE_VERSION = "Unknown"; - public static final String UNKNOWN_GIT_SHA = "Unknown"; + private static final String UNKNOWN = "Unknown"; public static final BigInteger TWO = new BigInteger("2"); private static final String DEFAULT_TRIGGER_DIR = "triggers"; @@ -467,7 +467,7 @@ public static String getReleaseVersionString() { Properties props = loadedProperties.get(); if (props == null) - return RELEASE_VERSION.getString(UNKNOWN_RELEASE_VERSION); + return RELEASE_VERSION.getString(UNKNOWN); return props.getProperty("CassandraVersion"); } @@ -475,14 +475,22 @@ public static String getGitSHA() { Properties props = loadedProperties.get(); if (props == null) - return GIT_SHA.getString(UNKNOWN_GIT_SHA); - return props.getProperty("GitSHA", UNKNOWN_GIT_SHA); + return GIT_SHA.getString(UNKNOWN); + return props.getProperty("GitSHA", UNKNOWN); + } + + public static String getBuildDate() + { + Properties props = loadedProperties.get(); + if (props == null) + return BUILD_DATE.getString(UNKNOWN); + return props.getProperty("BuildDate", UNKNOWN); } public static String getReleaseVersionMajor() { String releaseVersion = FBUtilities.getReleaseVersionString(); - if (FBUtilities.UNKNOWN_RELEASE_VERSION.equals(releaseVersion)) + if (FBUtilities.UNKNOWN.equals(releaseVersion)) { throw new AssertionError("Release version is unknown"); } diff --git a/test/unit/org/apache/cassandra/tools/nodetool/VersionTest.java b/test/unit/org/apache/cassandra/tools/nodetool/VersionTest.java new file mode 100644 index 000000000000..1b17fffa2abd --- /dev/null +++ b/test/unit/org/apache/cassandra/tools/nodetool/VersionTest.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.tools.nodetool; + +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.tools.ToolRunner; + +import static org.assertj.core.api.Assertions.assertThat; + +public class VersionTest extends CQLTester +{ + + @BeforeClass + public static void setup() throws Exception + { + requireNetwork(); + startJMXServer(); + } + + @Test + public void testHelp() + { + ToolRunner.ToolResult tool = ToolRunner.invokeNodetool("help", "version"); + tool.assertOnExitCode(); + + String help = "NAME\n" + + " nodetool version - Print cassandra version\n" + + "\n" + + "SYNOPSIS\n" + + " nodetool [(-h <host> | --host <host>)] [(-p <port> | --port <port>)]\n" + + " [(-pp | --print-port)] [(-pw <password> | --password <password>)]\n" + + " [(-pwf <passwordFilePath> | --password-file <passwordFilePath>)]\n" + + " [(-u <username> | --username <username>)] version [(-v | --verbose)]\n" + + "\n" + + "OPTIONS\n" + + " -h <host>, --host <host>\n" + + " Node hostname or ip address\n" + + "\n" + + " -p <port>, --port <port>\n" + + " Remote jmx agent port number\n" + + "\n" + + " -pp, --print-port\n" + + " Operate in 4.0 mode with hosts disambiguated by port number\n" + + "\n" + + " -pw <password>, --password <password>\n" + + " Remote jmx agent password\n" + + "\n" + + " -pwf <passwordFilePath>, --password-file <passwordFilePath>\n" + + " Path to the JMX password file\n" + + "\n" + + " -u <username>, --username <username>\n" + + " Remote jmx agent username\n" + + "\n" + + " -v, --verbose\n" + + " Include additional information\n" + + "\n" + + "\n"; + assertThat(tool.getStdout()).isEqualTo(help); + } + + @Test + public void testBasic() + { + ToolRunner.ToolResult tool = ToolRunner.invokeNodetool("version"); + tool.assertOnExitCode(); + String stdout = tool.getStdout(); + assertThat(stdout).containsPattern("ReleaseVersion:\\s+\\S+"); + } + + @Test + public void testVOption() + { + ToolRunner.ToolResult tool = ToolRunner.invokeNodetool("version", "-v"); + tool.assertOnExitCode(); + String stdout = tool.getStdout(); + assertThat(stdout).containsPattern("ReleaseVersion:\\s+\\S+"); + assertThat(stdout).containsPattern("BuildDate:\\s+\\S+"); + assertThat(stdout).containsPattern("GitSHA:\\s+\\S+"); + assertThat(stdout).containsPattern("JVM vendor/version:\\s+\\S+"); + } +} From 4dd70cec7685d6ee5074752269f3c81f17560b18 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Thu, 13 Mar 2025 09:47:58 +0100 Subject: [PATCH 183/225] Un-ignore HintsUpgradeTest patch by Stefan Miklosovic; reviewed by Alex Petrov for CASSANDRA-20435 --- test/unit/org/apache/cassandra/hints/HintsUpgradeTest.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/test/unit/org/apache/cassandra/hints/HintsUpgradeTest.java b/test/unit/org/apache/cassandra/hints/HintsUpgradeTest.java index fd44a963174b..78b8f56b5034 100644 --- a/test/unit/org/apache/cassandra/hints/HintsUpgradeTest.java +++ b/test/unit/org/apache/cassandra/hints/HintsUpgradeTest.java @@ -26,7 +26,6 @@ import com.google.common.collect.ImmutableMap; import org.junit.After; import org.junit.BeforeClass; -import org.junit.Ignore; import org.junit.Test; import org.apache.cassandra.SchemaLoader; @@ -50,7 +49,6 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -@Ignore("TODO: TCM") public class HintsUpgradeTest { static From 8cb58bf97a0f8cf81e6600d595e6b50c7b271b01 Mon Sep 17 00:00:00 2001 From: Dmitry Konstantinov <konstantinov@netcracker.com> Date: Tue, 11 Mar 2025 23:29:17 +0000 Subject: [PATCH 184/225] Disable role and network permission caches to avoid interference between test methods Patch by Dmitry Konstantinov; reviewed by TBD for CASSANDRA-20423 --- test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java b/test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java index 4ce56721d74d..3572fafb118a 100644 --- a/test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java +++ b/test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java @@ -60,6 +60,7 @@ public static void setUpClass() DatabaseDescriptor.setAuthorizer(new CassandraAuthorizer()); DatabaseDescriptor.setRoleManager(new CassandraRoleManager()); DatabaseDescriptor.setPermissionsValidity(0); + DatabaseDescriptor.setRolesValidity(0); CQLTester.setUpClass(); requireNetworkWithoutDriver(); DatabaseDescriptor.getRoleManager().setup(); From 740d6d080504da1a072ab98d3b0793031b87ca98 Mon Sep 17 00:00:00 2001 From: Caleb Rackliffe <calebrackliffe@gmail.com> Date: Tue, 11 Mar 2025 17:07:28 -0500 Subject: [PATCH 185/225] Prioritize legacy 2i over SAI for columns with multiple indexes patch by Caleb Rackliffe; reviewed by David Capwell for CASSANDRA-20334 --- CHANGES.txt | 1 + NEWS.txt | 10 +++ conf/cassandra.yaml | 5 ++ .../cassandra/config/DatabaseDescriptor.java | 10 +++ .../config/StorageAttachedIndexOptions.java | 2 + .../plan/StorageAttachedIndexQueryPlan.java | 5 +- .../cassandra/service/StorageService.java | 11 +++ .../service/StorageServiceMBean.java | 3 + .../sai/metrics/AbstractMetricsTest.java | 5 ++ .../index/sai/metrics/IndexSelectionTest.java | 72 +++++++++++++++++++ .../index/sai/metrics/QueryMetricsTest.java | 5 -- 11 files changed, 120 insertions(+), 9 deletions(-) create mode 100644 test/unit/org/apache/cassandra/index/sai/metrics/IndexSelectionTest.java diff --git a/CHANGES.txt b/CHANGES.txt index fa2ab2e885aa..8b02c3ffae42 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.4 + * Prioritize legacy 2i over SAI for columns with multiple indexes (CASSANDRA-20334) * Ensure only offline tools can build IntervalTrees without first/last key fields (CASSANDRA-20407) * Improve IntervalTree build throughput (CASSANDRA-19596) * Avoid limit on RFP fetch in the case of an unresolved static row (CASSANDRA-20323) diff --git a/NEWS.txt b/NEWS.txt index 717b5cf37de6..5b5ec9b92a1a 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -65,6 +65,16 @@ restore snapshots created with the previous major version using the 'sstableloader' tool. You can upgrade the file format of your snapshots using the provided 'sstableupgrade' tool. +5.0.4 +===== + +Upgrading +--------- + - When more than one index exists on a single column, legacy table-backed indexes will now be prioritized over + storage-attached indexes (SAI) to make migration between the two safer. This behavior can be switched off via + the flag `sai_options.prioritize_over_legacy_index` (which defaults to `false`) in `cassandra.yaml` or via + `setPrioritizeSAIOverLegacyIndex(boolean)` in the JMX MBean `org.apache.cassandra.db:type=StorageService`. + 5.0.1 ===== diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index 6f70cd669c7e..e09518188a60 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -1763,6 +1763,11 @@ transparent_data_encryption_options: ## is split between all SAI indexes being built so more indexes will mean smaller ## segment sizes. # segment_write_buffer_size: 1024MiB + # + ## When more than one index exists on a single column, this flag determines whether or not SAI + ## should take precedence over legacy table-backed indexes during reads. By default, legacy + ## indexes take precedence to maintain continuity during migration. + # prioritize_over_legacy_index: false ##################### # SAFETY THRESHOLDS # diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index bc58fe151ba3..9797bf82e6bc 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -5247,6 +5247,16 @@ public static DataStorageSpec.IntMebibytesBound getSAISegmentWriteBufferSpace() return conf.sai_options.segment_write_buffer_size; } + public static boolean getPrioritizeSAIOverLegacyIndex() + { + return conf.sai_options.prioritize_over_legacy_index; + } + + public static void setPrioritizeSAIOverLegacyIndex(boolean value) + { + conf.sai_options.prioritize_over_legacy_index = value; + } + public static RepairRetrySpec getRepairRetrySpec() { return conf == null ? new RepairRetrySpec() : conf.repair.retries; diff --git a/src/java/org/apache/cassandra/config/StorageAttachedIndexOptions.java b/src/java/org/apache/cassandra/config/StorageAttachedIndexOptions.java index 18bf3506fed6..48aea80f3700 100644 --- a/src/java/org/apache/cassandra/config/StorageAttachedIndexOptions.java +++ b/src/java/org/apache/cassandra/config/StorageAttachedIndexOptions.java @@ -33,6 +33,8 @@ public class StorageAttachedIndexOptions "Value must be a positive integer less than " + MAXIMUM_SEGMENT_BUFFER_MB + "MiB"; public DataStorageSpec.IntMebibytesBound segment_write_buffer_size = new DataStorageSpec.IntMebibytesBound(DEFAULT_SEGMENT_BUFFER_MB); + + public volatile boolean prioritize_over_legacy_index = false; public void validate() { diff --git a/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexQueryPlan.java b/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexQueryPlan.java index e352f7fcdd55..b53632407f9d 100644 --- a/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexQueryPlan.java +++ b/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexQueryPlan.java @@ -121,10 +121,7 @@ public Set<Index> getIndexes() @Override public long getEstimatedResultRows() { - // this is temporary (until proper QueryPlan is integrated into Cassandra) - // and allows us to priority storage-attached indexes if any in the query since they - // are going to be more efficient, to query and intersect, than built-in indexes. - return Long.MIN_VALUE; + return DatabaseDescriptor.getPrioritizeSAIOverLegacyIndex() ? Long.MIN_VALUE : Long.MAX_VALUE; } @Override diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 533ca08e6d5e..9f2c81861992 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -7657,4 +7657,15 @@ public void setEnforceNativeDeadlineForHints(boolean value) DatabaseDescriptor.setEnforceNativeDeadlineForHints(value); } + @Override + public boolean getPrioritizeSAIOverLegacyIndex() + { + return DatabaseDescriptor.getPrioritizeSAIOverLegacyIndex(); + } + + @Override + public void setPrioritizeSAIOverLegacyIndex(boolean value) + { + DatabaseDescriptor.setPrioritizeSAIOverLegacyIndex(value); + } } diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java index 0bfcfd500b30..66396e03f68e 100644 --- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java +++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java @@ -1319,4 +1319,7 @@ public void enableAuditLog(String loggerName, String includedKeyspaces, String e * e.g. keyspace_name -> [reads, writes, paxos]. */ Map<String, long[]> getOutOfRangeOperationCounts(); + + boolean getPrioritizeSAIOverLegacyIndex(); + void setPrioritizeSAIOverLegacyIndex(boolean value); } diff --git a/test/unit/org/apache/cassandra/index/sai/metrics/AbstractMetricsTest.java b/test/unit/org/apache/cassandra/index/sai/metrics/AbstractMetricsTest.java index 24134bccdef7..e09e44f32648 100644 --- a/test/unit/org/apache/cassandra/index/sai/metrics/AbstractMetricsTest.java +++ b/test/unit/org/apache/cassandra/index/sai/metrics/AbstractMetricsTest.java @@ -34,4 +34,9 @@ public void initializeTest() throws Throwable createMBeanServerConnection(); } + + protected long getTableQueryMetrics(String keyspace, String table, String metricsName) + { + return (long) getMetricValue(objectNameNoIndex(metricsName, keyspace, table, TableQueryMetrics.TABLE_QUERY_METRIC_TYPE)); + } } diff --git a/test/unit/org/apache/cassandra/index/sai/metrics/IndexSelectionTest.java b/test/unit/org/apache/cassandra/index/sai/metrics/IndexSelectionTest.java new file mode 100644 index 000000000000..db44c21d52b7 --- /dev/null +++ b/test/unit/org/apache/cassandra/index/sai/metrics/IndexSelectionTest.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.index.sai.metrics; + +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; + +import static org.junit.Assert.assertEquals; + +public class IndexSelectionTest extends AbstractMetricsTest +{ + @Test + public void shouldSelectLegacyIndexByDefault() + { + createTable("CREATE TABLE %s (pk int, ck int, val1 int, val2 int, PRIMARY KEY(pk, ck))"); + createIndex("CREATE INDEX ON %s(val1) USING 'legacy_local_table'"); + createIndex("CREATE INDEX ON %s(val1) USING 'sai'"); + + execute("INSERT INTO %s(pk, ck, val1, val2) VALUES(?, ?, ?, ?)", 1, 1, 2, 1); + + assertRows(execute("SELECT pk, ck, val1, val2 FROM %s WHERE val1 = 2"), row(1, 1, 2, 1)); + assertEquals(0L, getTableQueryMetrics(KEYSPACE, currentTable(), "TotalQueriesCompleted")); + + DatabaseDescriptor.setPrioritizeSAIOverLegacyIndex(true); + assertRows(execute("SELECT pk, ck, val1, val2 FROM %s WHERE val1 = 2"), row(1, 1, 2, 1)); + assertEquals(1L, getTableQueryMetrics(KEYSPACE, currentTable(), "TotalQueriesCompleted")); + + DatabaseDescriptor.setPrioritizeSAIOverLegacyIndex(false); + assertRows(execute("SELECT pk, ck, val1, val2 FROM %s WHERE val1 = 2"), row(1, 1, 2, 1)); + assertEquals(1L, getTableQueryMetrics(KEYSPACE, currentTable(), "TotalQueriesCompleted")); + } + + @Test + public void shouldSelectLegacyIndexByDefaultForAndQueries() + { + createTable("CREATE TABLE %s (pk int, ck int, val1 int, val2 int, PRIMARY KEY(pk, ck))"); + createIndex("CREATE INDEX ON %s(val1) USING 'legacy_local_table'"); + createIndex("CREATE INDEX ON %s(val1) USING 'sai'"); + createIndex("CREATE INDEX ON %s(val2) USING 'legacy_local_table'"); + createIndex("CREATE INDEX ON %s(val2) USING 'sai'"); + + execute("INSERT INTO %s(pk, ck, val1, val2) VALUES(?, ?, ?, ?)", 1, 1, 2, 1); + + assertRows(execute("SELECT pk, ck, val1, val2 FROM %s WHERE val1 = 2 AND val2 = 1"), row(1, 1, 2, 1)); + assertEquals(0L, getTableQueryMetrics(KEYSPACE, currentTable(), "TotalQueriesCompleted")); + + DatabaseDescriptor.setPrioritizeSAIOverLegacyIndex(true); + assertRows(execute("SELECT pk, ck, val1, val2 FROM %s WHERE val1 = 2 AND val2 = 1"), row(1, 1, 2, 1)); + assertEquals(1L, getTableQueryMetrics(KEYSPACE, currentTable(), "TotalQueriesCompleted")); + + DatabaseDescriptor.setPrioritizeSAIOverLegacyIndex(false); + assertRows(execute("SELECT pk, ck, val1, val2 FROM %s WHERE val1 = 2 AND val2 = 1"), row(1, 1, 2, 1)); + assertEquals(1L, getTableQueryMetrics(KEYSPACE, currentTable(), "TotalQueriesCompleted")); + } +} diff --git a/test/unit/org/apache/cassandra/index/sai/metrics/QueryMetricsTest.java b/test/unit/org/apache/cassandra/index/sai/metrics/QueryMetricsTest.java index 846024ebe097..f6b808e1e0d9 100644 --- a/test/unit/org/apache/cassandra/index/sai/metrics/QueryMetricsTest.java +++ b/test/unit/org/apache/cassandra/index/sai/metrics/QueryMetricsTest.java @@ -99,9 +99,4 @@ public void testMetricRelease() throws Throwable dropIndex(String.format("DROP INDEX %s." + index, keyspace)); assertThatThrownBy(() -> getTableQueryMetrics(keyspace, table, "TotalQueriesCompleted")).hasCauseInstanceOf(InstanceNotFoundException.class); } - - private long getTableQueryMetrics(String keyspace, String table, String metricsName) - { - return (long) getMetricValue(objectNameNoIndex(metricsName, keyspace, table, TableQueryMetrics.TABLE_QUERY_METRIC_TYPE)); - } } From 5b0a9c3aa4b9b15d493b7403e5985799b728e607 Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi <contacto@bernardobotella.com> Date: Tue, 11 Mar 2025 06:53:20 -0700 Subject: [PATCH 186/225] Improve constraints autocompletion Improve constraints autocompletion patch by Bernardo Botella; reviewed by Maxwell Guo for CASSANDRA-20341 --- CHANGES.txt | 1 + pylib/cqlshlib/cql3handling.py | 11 ++++++++++- pylib/cqlshlib/test/test_cqlsh_completion.py | 4 ++++ 3 files changed, 15 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 6281a6076b1e..7df26992c301 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Improve constraints autocompletion (CASSANDRA-20341) * Add JVM version and Cassandra build date to nodetool version -v (CASSANDRA-19721) * Move all disk error logic to DiskErrorsHandler to enable pluggability (CASSANDRA-20363) * Fix marking an SSTable as suspected and BufferPool leakage in case of a corrupted SSTable read during a compaction (CASSANDRA-20396) diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py index e231d5c8a8ee..cddcdb34d68a 100644 --- a/pylib/cqlshlib/cql3handling.py +++ b/pylib/cqlshlib/cql3handling.py @@ -328,9 +328,18 @@ def dequote_value(cqlword): ; <constraint> ::= <cident> <cmp> <term> - | <functionArguments> <cmp> <term> + | <constraintComparableFunction> <functionArguments> <cmp> <term> + | <constraintStandaloneFunction> <functionArguments> ; +<constraintComparableFunction> ::= "LENGTH" + | "OCTET_LENGTH" + ; + +<constraintStandaloneFunction> ::= "JSON" + | "NOT_NULL" + ; + <column_mask> ::= "MASKED" "WITH" ( "DEFAULT" | <functionName> <selectionFunctionArguments> ); # Note: autocomplete for frozen collection types does not handle nesting past depth 1 properly, diff --git a/pylib/cqlshlib/test/test_cqlsh_completion.py b/pylib/cqlshlib/test/test_cqlsh_completion.py index 55f5511ef1a2..d3f95ec89e1d 100644 --- a/pylib/cqlshlib/test/test_cqlsh_completion.py +++ b/pylib/cqlshlib/test/test_cqlsh_completion.py @@ -1107,6 +1107,10 @@ def test_complete_in_alter_table(self): self.trycompletions('ALTER TABLE new_table ADD col int MASKED WITH ', choices=['DEFAULT', self.cqlsh.keyspace + '.', 'system.'], other_choices_ok=True) + self.trycompletions('ALTER TABLE new_table ADD col int C', immediate='HECK ') + self.trycompletions('ALTER TABLE new_table ADD col int CHECK ', + choices=['<identifier', '<quotedName>', 'JSON', 'LENGTH', 'NOT_NULL', 'OCTET_LENGTH'], + other_choices_ok=True) self.trycompletions('ALTER TABLE IF EXISTS new_table RENAME ', choices=['IF', '<quotedName>', '<identifier>']) self.trycompletions('ALTER TABLE new_table RENAME ', choices=['IF', '<quotedName>', '<identifier>']) self.trycompletions('ALTER TABLE IF EXISTS new_table DROP ', choices=['IF', '<quotedName>', '<identifier>']) From acaff7bf449035047b60967cd89f9fabe7a0f543 Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi <contacto@bernardobotella.com> Date: Thu, 13 Mar 2025 13:49:17 -0700 Subject: [PATCH 187/225] Fix the constraints mutability on ColumnMetadata patch by Bernardo Botella; reviewed by Sam Tunnicliffe for CASSANDRA-20357 --- src/antlr/Parser.g | 4 +- .../schema/AlterTableStatement.java | 83 +++++++------------ .../schema/CreateTableStatement.java | 12 +-- .../cassandra/schema/ColumnMetadata.java | 9 +- .../cassandra/schema/TableMetadata.java | 14 ++++ .../test/ColumnConstraintsTest.java | 10 ++- ...ableWithTableConstraintValidationTest.java | 17 ++++ 7 files changed, 79 insertions(+), 70 deletions(-) diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g index 30e07fddf94d..dd9eb181dfe7 100644 --- a/src/antlr/Parser.g +++ b/src/antlr/Parser.g @@ -989,8 +989,8 @@ alterTableStatement returns [AlterTableStatement.Raw stmt] | K_ALTER ( K_IF K_EXISTS { $stmt.ifColumnExists(true); } )? id=cident ( mask=columnMask { $stmt.mask(id, mask); } | K_DROP K_MASKED { $stmt.mask(id, null); } - | K_DROP K_CHECK { $stmt.dropConstraints(id); } - | (constraints=columnConstraints) { $stmt.alterConstraints(id, constraints); }) + | K_DROP K_CHECK { $stmt.constraint(id, null); } + | (constraints=columnConstraints) { $stmt.constraint(id, constraints); }) | K_ADD ( K_IF K_NOT K_EXISTS { $stmt.ifColumnNotExists(true); } )? ( id=ident v=comparatorType b=isStaticColumn (m=columnMask)? { $stmt.add(id, v, b, m); } diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java index 08dc3ea918d6..fc2ab582f754 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java @@ -710,65 +710,46 @@ private void validateCanDropCompactStorage() } } - public static class DropConstraints extends AlterTableStatement - { - final ColumnIdentifier columnName; - - DropConstraints(String keyspaceName, String tableName, boolean ifTableExists, ColumnIdentifier columnName) - { - super(keyspaceName, tableName, ifTableExists); - this.columnName = columnName; - } - - @Override - public KeyspaceMetadata apply(Epoch epoch, KeyspaceMetadata keyspace, TableMetadata table, ClusterMetadata metadata) - { - ColumnMetadata columnMetadata = table.getColumn(columnName); - columnMetadata.removeColumnConstraints(); - - TableMetadata.Builder tableBuilder = table.unbuild().epoch(epoch); - Views.Builder viewsBuilder = keyspace.views.unbuild(); - TableMetadata tableMetadata = tableBuilder.build(); - tableMetadata.validate(); - - return keyspace.withSwapped(keyspace.tables.withSwapped(tableMetadata)) - .withSwapped(viewsBuilder.build()); - } - } - public static class AlterConstraints extends AlterTableStatement { final ColumnIdentifier columnName; - final ColumnConstraints constraints; + final ColumnConstraints.Raw constraints; + final boolean ifColumnExists; - AlterConstraints(String keyspaceName, String tableName, boolean ifTableExists, ColumnIdentifier columnName, ColumnConstraints constraints) + AlterConstraints(String keyspaceName, String tableName, boolean ifTableExists, boolean ifColumnExists, ColumnIdentifier columnName, ColumnConstraints.Raw constraints) { super(keyspaceName, tableName, ifTableExists); this.columnName = columnName; this.constraints = constraints; + this.ifColumnExists = ifColumnExists; } @Override public KeyspaceMetadata apply(Epoch epoch, KeyspaceMetadata keyspace, TableMetadata table, ClusterMetadata metadata) { - TableMetadata.Builder tableBuilder = table.unbuild().epoch(epoch); - for (ColumnMetadata column : tableBuilder.columns()) + ColumnMetadata column = table.getColumn(columnName); + if (column != null) { - if (column.name == columnName) - { - constraints.validate(column); - column.setColumnConstraints(constraints); - break; - } + ColumnConstraints oldConstraints = column.getColumnConstraints(); + ColumnConstraints newConstraints = constraints == null ? ColumnConstraints.NO_OP : constraints.prepare(); + if (Objects.equals(oldConstraints, newConstraints)) + return keyspace; + newConstraints.validate(column); + TableMetadata.Builder tableBuilder = table.unbuild().epoch(epoch); + tableBuilder.alterColumnConstraints(columnName, newConstraints); + + TableMetadata newTable = tableBuilder.build(); + newTable.validate(); + + return keyspace.withSwapped(keyspace.tables.withSwapped(newTable)); } - - Views.Builder viewsBuilder = keyspace.views.unbuild(); - TableMetadata tableMetadata = tableBuilder.build(); - tableMetadata.validate(); - - return keyspace.withSwapped(keyspace.tables.withSwapped(tableMetadata)) - .withSwapped(viewsBuilder.build()); + else + { + if (!ifColumnExists) + throw ire("Column '%s' doesn't exist", columnName); + } + return keyspace; } } @@ -783,7 +764,6 @@ private enum Kind RENAME_COLUMNS, ALTER_OPTIONS, DROP_COMPACT_STORAGE, - DROP_CONSTRAINTS, ALTER_CONSTRAINTS } @@ -792,7 +772,7 @@ private enum Kind private boolean ifColumnExists; private boolean ifColumnNotExists; private ColumnIdentifier constraintName; - private ColumnConstraints constraints; + private ColumnConstraints.Raw constraints; private Kind kind; @@ -839,8 +819,7 @@ public AlterTableStatement prepare(ClientState state) case RENAME_COLUMNS: return new RenameColumns(keyspaceName, tableName, renamedColumns, ifTableExists, ifColumnExists); case ALTER_OPTIONS: return new AlterOptions(keyspaceName, tableName, attrs, ifTableExists); case DROP_COMPACT_STORAGE: return new DropCompactStorage(keyspaceName, tableName, ifTableExists); - case DROP_CONSTRAINTS: return new DropConstraints(keyspaceName, tableName, ifTableExists, constraintName); - case ALTER_CONSTRAINTS: return new AlterConstraints(keyspaceName, tableName, ifTableExists, constraintName, constraints); + case ALTER_CONSTRAINTS: return new AlterConstraints(keyspaceName, tableName, ifTableExists, ifColumnExists, constraintName, constraints); } throw new AssertionError(); @@ -885,17 +864,11 @@ public void dropCompactStorage() kind = Kind.DROP_COMPACT_STORAGE; } - public void dropConstraints(ColumnIdentifier name) - { - kind = Kind.DROP_CONSTRAINTS; - this.constraintName = name; - } - - public void alterConstraints(ColumnIdentifier name, ColumnConstraints.Raw rawConstraints) + public void constraint(ColumnIdentifier name, ColumnConstraints.Raw rawConstraints) { kind = Kind.ALTER_CONSTRAINTS; this.constraintName = name; - this.constraints = rawConstraints.prepare(); + this.constraints = rawConstraints; } public void timestamp(long timestamp) diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java index a527a1c2c5bd..997a40200077 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java @@ -348,16 +348,14 @@ public TableMetadata.Builder builder(Types types, UserFunctions functions) { ColumnProperties properties = partitionKeyColumnProperties.get(i); ColumnIdentifier columnIdentifier = partitionKeyColumns.get(i); - builder.addPartitionKeyColumn(columnIdentifier, properties.type, properties.mask); - builder.getColumn(columnIdentifier).setColumnConstraints(columnConstraints.get(columnIdentifier)); + builder.addPartitionKeyColumn(columnIdentifier, properties.type, properties.mask, columnConstraints.get(columnIdentifier)); } for (int i = 0; i < clusteringColumns.size(); i++) { ColumnProperties properties = clusteringColumnProperties.get(i); ColumnIdentifier columnIdentifier = clusteringColumns.get(i); - builder.addClusteringColumn(columnIdentifier, properties.type, properties.mask); - builder.getColumn(columnIdentifier).setColumnConstraints(columnConstraints.get(columnIdentifier)); + builder.addClusteringColumn(columnIdentifier, properties.type, properties.mask, columnConstraints.get(columnIdentifier)); } if (useCompactStorage) @@ -368,14 +366,12 @@ public TableMetadata.Builder builder(Types types, UserFunctions functions) { columns.forEach((column, properties) -> { if (staticColumns.contains(column)) - builder.addStaticColumn(column, properties.type, properties.mask); + builder.addStaticColumn(column, properties.type, properties.mask, columnConstraints.get(column)); else - builder.addRegularColumn(column, properties.type, properties.mask); + builder.addRegularColumn(column, properties.type, properties.mask, columnConstraints.get(column)); }); } - columns.keySet().forEach(id -> builder.getColumn(id).setColumnConstraints(columnConstraints.get(id))); - return builder; } diff --git a/src/java/org/apache/cassandra/schema/ColumnMetadata.java b/src/java/org/apache/cassandra/schema/ColumnMetadata.java index cdaec3f80b78..e28312532be0 100644 --- a/src/java/org/apache/cassandra/schema/ColumnMetadata.java +++ b/src/java/org/apache/cassandra/schema/ColumnMetadata.java @@ -363,10 +363,15 @@ public ColumnConstraints getColumnConstraints() return columnConstraints; } - public void setColumnConstraints(ColumnConstraints constraints) + public ColumnConstraints setColumnConstraints() + { + return columnConstraints; + } + + public ColumnMetadata withNewColumnConstraints(ColumnConstraints constraints) { constraints.validate(this); - this.columnConstraints = constraints; + return new ColumnMetadata(ksName, cfName, name, type, position, kind, mask, constraints); } public void removeColumnConstraints() diff --git a/src/java/org/apache/cassandra/schema/TableMetadata.java b/src/java/org/apache/cassandra/schema/TableMetadata.java index 4dcec20f92a1..9fdf5e821542 100644 --- a/src/java/org/apache/cassandra/schema/TableMetadata.java +++ b/src/java/org/apache/cassandra/schema/TableMetadata.java @@ -455,6 +455,7 @@ public ColumnMetadata getColumn(ColumnIdentifier name) { return columns.get(name.bytes); } + /** * Returns the column of the provided name if it exists, but throws a user-visible exception if that column doesn't * exist. @@ -1294,6 +1295,19 @@ public Builder alterColumnMask(ColumnIdentifier name, @Nullable ColumnMask mask) return this; } + public Builder alterColumnConstraints(ColumnIdentifier name, ColumnConstraints constraints) + { + ColumnMetadata column = columns.get(name.bytes); + if (column == null) + throw new IllegalArgumentException(); + + ColumnMetadata newColumn = column.withNewColumnConstraints(constraints); + + updateColumn(column, newColumn); + + return this; + } + Builder alterColumnType(ColumnIdentifier name, AbstractType<?> type) { ColumnMetadata column = columns.get(name.bytes); diff --git a/test/distributed/org/apache/cassandra/distributed/test/ColumnConstraintsTest.java b/test/distributed/org/apache/cassandra/distributed/test/ColumnConstraintsTest.java index 16fc38c55c6c..77b72ccca1d2 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/ColumnConstraintsTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/ColumnConstraintsTest.java @@ -23,7 +23,6 @@ import java.util.Set; import org.apache.cassandra.cql3.constraints.ConstraintViolationException; -import org.apache.cassandra.cql3.constraints.InvalidConstraintDefinitionException; import org.apache.cassandra.exceptions.InvalidRequestException; import org.junit.Test; @@ -52,7 +51,12 @@ public void testInvalidConstraintsExceptions() throws IOException { assertThrowsInvalidConstraintException(cluster, String.format("CREATE TABLE %s (pk int, ck1 text CHECK ck1 < 100, ck2 int, v int, " + "PRIMARY KEY ((pk), ck1, ck2));", tableName), - "Column 'ck1' is not a number type."); + "Constraint 'ck1 <' can be used only for columns of type " + + "[org.apache.cassandra.db.marshal.ByteType, org.apache.cassandra.db.marshal.CounterColumnType, " + + "org.apache.cassandra.db.marshal.DecimalType, org.apache.cassandra.db.marshal.DoubleType, " + + "org.apache.cassandra.db.marshal.FloatType, org.apache.cassandra.db.marshal.Int32Type, " + + "org.apache.cassandra.db.marshal.IntegerType, org.apache.cassandra.db.marshal.LongType, " + + "org.apache.cassandra.db.marshal.ShortType] but it was class org.apache.cassandra.db.marshal.UTF8Type"); assertThrowsInvalidConstraintException(cluster, String.format("CREATE TABLE %s (pk int, ck1 int CHECK LENGTH(ck1) < 100, ck2 int, v int, " + "PRIMARY KEY ((pk), ck1, ck2));", tableName), @@ -320,6 +324,6 @@ private void assertThrowsInvalidConstraintException(Cluster cluster, String stat assertThatThrownBy(() -> cluster.schemaChange(statement)) .describedAs(description) .has(new Condition<Throwable>(t -> t.getClass().getCanonicalName() - .equals(InvalidConstraintDefinitionException.class.getCanonicalName()), description)); + .equals(InvalidRequestException.class.getCanonicalName()), description)); } } diff --git a/test/unit/org/apache/cassandra/contraints/AlterTableWithTableConstraintValidationTest.java b/test/unit/org/apache/cassandra/contraints/AlterTableWithTableConstraintValidationTest.java index 584cd955c7dd..4db12f5a6979 100644 --- a/test/unit/org/apache/cassandra/contraints/AlterTableWithTableConstraintValidationTest.java +++ b/test/unit/org/apache/cassandra/contraints/AlterTableWithTableConstraintValidationTest.java @@ -20,6 +20,8 @@ import org.junit.Test; +import org.apache.cassandra.exceptions.InvalidRequestException; + public class AlterTableWithTableConstraintValidationTest extends CqlConstraintValidationTester { @@ -222,4 +224,19 @@ public void testAlterWithCdcAndClusteringConstraintsEnabled() throws Throwable // It works execute("ALTER TABLE %s WITH cdc = true"); } + + @Test + public void testCreateTableAddConstraintWithIfExists() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + execute("ALTER TABLE %s ALTER IF EXISTS foo CHECK foo < 100"); + } + + @Test + public void testCreateTableAddConstraintWithNonExistingColumn() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v int, PRIMARY KEY ((pk),ck1, ck2)) WITH CLUSTERING ORDER BY (ck1 ASC);"); + String expectedErrorMessage = "Column 'foo' doesn't exist"; + assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "ALTER TABLE %s ALTER foo CHECK foo < 100"); + } } From 68b88134db419ba4d1713b9b076b29fe0215840d Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Fri, 14 Mar 2025 13:49:46 +0100 Subject: [PATCH 188/225] ninja: fix typo in test_cqlsh_completion.py after CASSANDRA-20341 --- pylib/cqlshlib/test/test_cqlsh_completion.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pylib/cqlshlib/test/test_cqlsh_completion.py b/pylib/cqlshlib/test/test_cqlsh_completion.py index d3f95ec89e1d..aa38ec45db66 100644 --- a/pylib/cqlshlib/test/test_cqlsh_completion.py +++ b/pylib/cqlshlib/test/test_cqlsh_completion.py @@ -1109,7 +1109,7 @@ def test_complete_in_alter_table(self): other_choices_ok=True) self.trycompletions('ALTER TABLE new_table ADD col int C', immediate='HECK ') self.trycompletions('ALTER TABLE new_table ADD col int CHECK ', - choices=['<identifier', '<quotedName>', 'JSON', 'LENGTH', 'NOT_NULL', 'OCTET_LENGTH'], + choices=['<identifier>', '<quotedName>', 'JSON', 'LENGTH', 'NOT_NULL', 'OCTET_LENGTH'], other_choices_ok=True) self.trycompletions('ALTER TABLE IF EXISTS new_table RENAME ', choices=['IF', '<quotedName>', '<identifier>']) self.trycompletions('ALTER TABLE new_table RENAME ', choices=['IF', '<quotedName>', '<identifier>']) From ef2b7e82a0e01420239a263b49d251e29245579c Mon Sep 17 00:00:00 2001 From: maoling <maoling@apache.org> Date: Wed, 5 Mar 2025 23:48:00 +0800 Subject: [PATCH 189/225] Add more resources to .gitignore after CASSANDRA-19915 patch by Ling Mao; reviewed by Stefan Miklosovic, Michael Semb Wever for CASSANDRA-20391 --- .gitignore | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.gitignore b/.gitignore index aa9e76c9323d..db7562cd3aea 100644 --- a/.gitignore +++ b/.gitignore @@ -76,6 +76,8 @@ Thumbs.db # Generated files from the documentation doc/modules/cassandra/pages/managing/configuration/cass_yaml_file.adoc +doc/modules/cassandra/pages/reference/native-protocol.adoc +doc/modules/cassandra/attachments/native_protocol_v*.html doc/modules/cassandra/pages/managing/tools/nodetool/ doc/modules/cassandra/examples/TEXT/NODETOOL/ From 85b25f2256e34da4ae3756621704242895f29f51 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Thu, 13 Mar 2025 16:26:33 +0100 Subject: [PATCH 190/225] Add regular expression constraint patch by Stefan Miklosovic; reviewed by Bernardo Botella for CASSANDRA-20275 --- CHANGES.txt | 1 + .../pages/developing/cql/constraints.adoc | 31 +++++ pylib/cqlshlib/cql3handling.py | 1 + pylib/cqlshlib/test/test_cqlsh_completion.py | 2 +- .../cql3/constraints/ConstraintFunction.java | 2 +- .../constraints/FunctionColumnConstraint.java | 5 +- .../cql3/constraints/NotNullConstraint.java | 2 +- .../cql3/constraints/RegexpConstraint.java | 112 ++++++++++++++++++ .../UnaryFunctionColumnConstraint.java | 2 +- .../cql3/functions/types/ParseUtils.java | 2 +- .../contraints/RegexpConstraintTest.java | 103 ++++++++++++++++ 11 files changed, 256 insertions(+), 7 deletions(-) create mode 100644 src/java/org/apache/cassandra/cql3/constraints/RegexpConstraint.java create mode 100644 test/unit/org/apache/cassandra/contraints/RegexpConstraintTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 7df26992c301..c02e830092e9 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Add regular expression constraint (CASSANDRA-20275) * Improve constraints autocompletion (CASSANDRA-20341) * Add JVM version and Cassandra build date to nodetool version -v (CASSANDRA-19721) * Move all disk error logic to DiskErrorsHandler to enable pluggability (CASSANDRA-20363) diff --git a/doc/modules/cassandra/pages/developing/cql/constraints.adoc b/doc/modules/cassandra/pages/developing/cql/constraints.adoc index f768d4fd8ff7..7e1893872929 100644 --- a/doc/modules/cassandra/pages/developing/cql/constraints.adoc +++ b/doc/modules/cassandra/pages/developing/cql/constraints.adoc @@ -191,4 +191,35 @@ INSERT INTO ks.tb (id, val) VALUES (1, 'abc'); ... [Invalid query] message="Value for column 'val' violated JSON constraint as it is not a valid JSON." +---- + +=== REGEXP constraint + +Defines a constraint which checks text-like values againt a regular expression. + +---- +CREATE TABLE ks.tb ( + id int primary key, + value CHECK REGEXP(value) = 'a.*b' +) +---- + +---- +cassandra@cqlsh> INSERT INTO ks.tb (id , value ) VALUES ( 1, 'asdadasdabb'); +cassandra@cqlsh> INSERT INTO ks.tb (id , value ) VALUES ( 1, 'aaaaa'); +... [Invalid query] message="Value does not match regular expression 'a.*b'" +---- + +Negation can be also used: + +---- +ALTER TABLE ks.tb ALTER value CHECK REGEXP(value) != 'a.*b'; +---- + +which would logically invert the condition: + +---- +cassandra@cqlsh> INSERT INTO ks.tb (id , value ) VALUES ( 1, 'asdadasdabb'); +... [Invalid query] message="Value does match regular expression 'a.*b'" +cassandra@cqlsh> INSERT INTO ks.tb (id , value ) VALUES ( 1, 'aaaaa'); ---- \ No newline at end of file diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py index cddcdb34d68a..f1baf2ad2952 100644 --- a/pylib/cqlshlib/cql3handling.py +++ b/pylib/cqlshlib/cql3handling.py @@ -334,6 +334,7 @@ def dequote_value(cqlword): <constraintComparableFunction> ::= "LENGTH" | "OCTET_LENGTH" + | "REGEXP" ; <constraintStandaloneFunction> ::= "JSON" diff --git a/pylib/cqlshlib/test/test_cqlsh_completion.py b/pylib/cqlshlib/test/test_cqlsh_completion.py index aa38ec45db66..bf2385a46fef 100644 --- a/pylib/cqlshlib/test/test_cqlsh_completion.py +++ b/pylib/cqlshlib/test/test_cqlsh_completion.py @@ -1109,7 +1109,7 @@ def test_complete_in_alter_table(self): other_choices_ok=True) self.trycompletions('ALTER TABLE new_table ADD col int C', immediate='HECK ') self.trycompletions('ALTER TABLE new_table ADD col int CHECK ', - choices=['<identifier>', '<quotedName>', 'JSON', 'LENGTH', 'NOT_NULL', 'OCTET_LENGTH'], + choices=['<identifier>', '<quotedName>', 'JSON', 'LENGTH', 'NOT_NULL', 'OCTET_LENGTH', 'REGEXP'], other_choices_ok=True) self.trycompletions('ALTER TABLE IF EXISTS new_table RENAME ', choices=['IF', '<quotedName>', '<identifier>']) self.trycompletions('ALTER TABLE new_table RENAME ', choices=['IF', '<quotedName>', '<identifier>']) diff --git a/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java b/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java index 1dda89093f4c..9952ab32d94f 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java +++ b/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java @@ -79,7 +79,7 @@ public void evaluate(AbstractType<?> valueType, ByteBuffer columnValue) throws C * Method that validates that a condition is valid. This method is called when the CQL constraint is created to determine * if the CQL statement is valid or needs to be rejected as invalid throwing a {@link InvalidConstraintDefinitionException} */ - public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException + public void validate(ColumnMetadata columnMetadata, String term) throws InvalidConstraintDefinitionException { } diff --git a/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java index dac62ddfc69a..a94b4bd0bdcb 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/FunctionColumnConstraint.java @@ -78,7 +78,8 @@ public static SatisfiabilityChecker[] getSatisfiabilityCheckers() public enum Functions { LENGTH(LengthConstraint::new), - OCTET_LENGTH(OctetLengthConstraint::new); + OCTET_LENGTH(OctetLengthConstraint::new), + REGEXP(RegexpConstraint::new); private final Function<ColumnIdentifier, ConstraintFunction> functionCreator; @@ -157,7 +158,7 @@ public void validate(ColumnMetadata columnMetadata) { validateArgs(columnMetadata); validateTypes(columnMetadata); - function.validate(columnMetadata); + function.validate(columnMetadata, term); } @Override diff --git a/src/java/org/apache/cassandra/cql3/constraints/NotNullConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/NotNullConstraint.java index 465db80933bd..fb9f7de95b2d 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/NotNullConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/NotNullConstraint.java @@ -49,7 +49,7 @@ public void internalEvaluate(AbstractType<?> valueType, Operator relationType, S } @Override - public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException + public void validate(ColumnMetadata columnMetadata, String term) throws InvalidConstraintDefinitionException { if (columnMetadata.isPrimaryKeyColumn()) throw new InvalidConstraintDefinitionException(format("%s constraint can not be specified on a %s key column '%s'", diff --git a/src/java/org/apache/cassandra/cql3/constraints/RegexpConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/RegexpConstraint.java new file mode 100644 index 000000000000..a2e439585f05 --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/constraints/RegexpConstraint.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.constraints; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.cql3.functions.types.ParseUtils; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.AsciiType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.schema.ColumnMetadata; + +import static java.lang.String.format; +import static org.apache.cassandra.cql3.Operator.EQ; +import static org.apache.cassandra.cql3.Operator.NEQ; + +public class RegexpConstraint extends ConstraintFunction +{ + public static final String FUNCTION_NAME = "REGEXP"; + private static final List<AbstractType<?>> SUPPORTED_TYPES = List.of(UTF8Type.instance, AsciiType.instance); + private static final List<Operator> ALLOWED_FUNCTION_OPERATORS = List.of(EQ, NEQ); + + private Pattern pattern; + + public RegexpConstraint(ColumnIdentifier columnName) + { + super(columnName, FUNCTION_NAME); + } + + @Override + protected void internalEvaluate(AbstractType<?> valueType, Operator relationType, String regexp, ByteBuffer columnValue) + { + assert pattern != null; + Matcher matcher = pattern.matcher(valueType.getString(columnValue)); + + switch (relationType) + { + case EQ: + if (!matcher.matches()) + throw new ConstraintViolationException(format("Value does not match regular expression %s", regexp)); + break; + case NEQ: + if (matcher.matches()) + throw new ConstraintViolationException(format("Value does match regular expression %s", regexp)); + break; + default: + throw new IllegalStateException("Unsupported operator: " + relationType); + } + } + + @Override + public List<AbstractType<?>> getSupportedTypes() + { + return SUPPORTED_TYPES; + } + + @Override + public List<Operator> getSupportedOperators() + { + return ALLOWED_FUNCTION_OPERATORS; + } + + @Override + public void validate(ColumnMetadata columnMetadata, String regexp) throws InvalidConstraintDefinitionException + { + try + { + // compilation of a regexp every single time upon evaluation is not performance friendly + // so we "cache" the compiled regexp for further reuse upon actual validation + pattern = Pattern.compile(ParseUtils.unquote(regexp)); + } + catch (Exception e) + { + throw new InvalidConstraintDefinitionException(format("String '%s' is not a valid regular expression", ParseUtils.unquote(regexp))); + } + } + + @Override + public boolean equals(Object o) + { + if (this == o) + return true; + + if (!(o instanceof RegexpConstraint)) + return false; + + RegexpConstraint other = (RegexpConstraint) o; + + return columnName.equals(other.columnName); + } +} diff --git a/src/java/org/apache/cassandra/cql3/constraints/UnaryFunctionColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/UnaryFunctionColumnConstraint.java index d39f7f865d97..80fd443e0e4d 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/UnaryFunctionColumnConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/UnaryFunctionColumnConstraint.java @@ -124,7 +124,7 @@ public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefi { validateArgs(columnMetadata); validateTypes(columnMetadata); - function.validate(columnMetadata); + function.validate(columnMetadata, term); } @Override diff --git a/src/java/org/apache/cassandra/cql3/functions/types/ParseUtils.java b/src/java/org/apache/cassandra/cql3/functions/types/ParseUtils.java index 8d0f29b45b09..45bf720c0c1d 100644 --- a/src/java/org/apache/cassandra/cql3/functions/types/ParseUtils.java +++ b/src/java/org/apache/cassandra/cql3/functions/types/ParseUtils.java @@ -268,7 +268,7 @@ public static String quote(String value) * @param value The string to unquote. * @return The unquoted string. */ - static String unquote(String value) + public static String unquote(String value) { return unquote(value, '\''); } diff --git a/test/unit/org/apache/cassandra/contraints/RegexpConstraintTest.java b/test/unit/org/apache/cassandra/contraints/RegexpConstraintTest.java new file mode 100644 index 000000000000..fd4800e24787 --- /dev/null +++ b/test/unit/org/apache/cassandra/contraints/RegexpConstraintTest.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.contraints; + +import org.junit.Test; + +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.cql3.constraints.ColumnConstraints; +import org.apache.cassandra.cql3.constraints.FunctionColumnConstraint.Raw; +import org.apache.cassandra.cql3.constraints.InvalidConstraintDefinitionException; +import org.apache.cassandra.cql3.constraints.RegexpConstraint; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.AsciiType; +import org.apache.cassandra.db.marshal.IntegerType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.schema.ColumnMetadata; +import org.assertj.core.api.ThrowableAssert; + +import static java.util.List.of; +import static org.apache.cassandra.schema.ColumnMetadata.Kind.REGULAR; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public class RegexpConstraintTest +{ + private static final ColumnIdentifier columnIdentifier = new ColumnIdentifier("a_column", false); + private static final ColumnIdentifier regexpFunctionIdentifier = new ColumnIdentifier(RegexpConstraint.FUNCTION_NAME, false); + private static final ColumnMetadata regularStringColumn = getColumnOfType(UTF8Type.instance); + private static final ColumnMetadata regularAsciiColumn = getColumnOfType(AsciiType.instance); + + private static final ColumnConstraints regexp = new ColumnConstraints(of(new Raw(regexpFunctionIdentifier, columnIdentifier, Operator.EQ, "'a..b'").prepare())); + private static final ColumnConstraints negatedRegexp = new ColumnConstraints(of(new Raw(regexpFunctionIdentifier, columnIdentifier, Operator.NEQ, "'a..b'").prepare())); + + @Test + public void testRegexpConstraint() throws Throwable + { + run(regexp, "acdb"); + run(regexp, "aaaaaaa", "Value does not match regular expression 'a..b'"); + run(negatedRegexp, "acdb", "Value does match regular expression 'a..b'"); + run(negatedRegexp, "aaaaa"); + } + + @Test + public void testInvalidPattern() + { + ColumnConstraints invalid = new ColumnConstraints(of(new Raw(regexpFunctionIdentifier, columnIdentifier, Operator.EQ, "'*abc'").prepare())); + assertThatThrownBy(() -> invalid.validate(regularStringColumn)) + .hasMessage("String '*abc' is not a valid regular expression") + .isInstanceOf(InvalidConstraintDefinitionException.class); + } + + @Test + public void testInvalidTypes() + { + assertThatThrownBy(() -> regexp.validate(getColumnOfType(IntegerType.instance))) + .hasMessage("Constraint 'REGEXP' can be used only for columns of type " + + "[org.apache.cassandra.db.marshal.UTF8Type, org.apache.cassandra.db.marshal.AsciiType] " + + "but it was class org.apache.cassandra.db.marshal.IntegerType"); + } + + private void run(ColumnConstraints regexp, String input) throws Throwable + { + run(regexp, input, null); + } + + private void run(ColumnConstraints regexp, String input, String exceptionMessage) throws Throwable + { + ThrowableAssert.ThrowingCallable callable = () -> + { + regexp.validate(regularStringColumn); + regexp.evaluate(regularStringColumn.type, regularStringColumn.type.fromString(input)); + + regexp.validate(regularAsciiColumn); + regexp.evaluate(regularAsciiColumn.type, regularAsciiColumn.type.fromString(input)); + }; + + if (exceptionMessage == null) + callable.call(); + else + assertThatThrownBy(callable).hasMessageContaining(exceptionMessage); + } + + private static ColumnMetadata getColumnOfType(AbstractType<?> type) + { + return new ColumnMetadata("a", "b", columnIdentifier, type, -1, REGULAR, null); + } +} From b17e4ee88aca91e6446598b23cbeb3d6a58528fd Mon Sep 17 00:00:00 2001 From: Ariel Weisberg <aweisberg@apple.com> Date: Fri, 14 Mar 2025 14:01:38 -0400 Subject: [PATCH 191/225] Fix SSTableReader interval mock in UnifiedCompactionStrategyTest patch by Ariel Weisberg; reviewed by Caleb Rackliffe for CASSANDRA-20437 --- .../cassandra/io/sstable/format/SSTableReader.java | 11 +++++++++-- .../db/compaction/UnifiedCompactionStrategyTest.java | 4 ++++ 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java index 75272a0b5f04..8a554ae2c5e4 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java @@ -179,7 +179,7 @@ private static ScheduledExecutorPlus initSyncExecutor() public static final Comparator<SSTableReader> maxTimestampAscending = Comparator.comparingLong(SSTableReader::getMaxTimestamp); public static final Comparator<SSTableReader> maxTimestampDescending = maxTimestampAscending.reversed(); - private static final TimeUUID.Generator.Factory<UniqueIdentifier> UNIQUE_IDENTIFIER_FACTORY = new TimeUUID.Generator.Factory<UniqueIdentifier>() + public static final TimeUUID.Generator.Factory<UniqueIdentifier> UNIQUE_IDENTIFIER_FACTORY = new TimeUUID.Generator.Factory<UniqueIdentifier>() { @Override public UniqueIdentifier atUnixMicrosWithLsb(long unixMicros, long clockSeqAndNode) @@ -197,6 +197,7 @@ private UniqueIdentifier(long unixMicros, long clockSeqAndNode) super(unixMicros, clockSeqAndNode); } } + public final UniqueIdentifier instanceId = TimeUUID.Generator.nextTimeUUID(UNIQUE_IDENTIFIER_FACTORY); public static final Comparator<SSTableReader> firstKeyComparator = (o1, o2) -> o1.getFirst().compareTo(o2.getFirst()); @@ -1769,11 +1770,17 @@ public abstract IVerifier getVerifier(ColumnFamilyStore cfs, boolean isOffline, IVerifier.Options options); + public UniqueIdentifier instanceId() + { + return instanceId; + } + @Override public int compareTo(SSTableReader other) { // Used in IntervalTree with the expecation that compareTo uniquely identifies an SSTableReader - return instanceId.compareTo(other.instanceId); + // Use accessor for instanceId for mocks + return instanceId().compareTo(other.instanceId()); } /** diff --git a/test/unit/org/apache/cassandra/db/compaction/UnifiedCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/UnifiedCompactionStrategyTest.java index 82fee2144c91..9ca29cb19063 100644 --- a/test/unit/org/apache/cassandra/db/compaction/UnifiedCompactionStrategyTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/UnifiedCompactionStrategyTest.java @@ -58,11 +58,13 @@ import org.apache.cassandra.utils.Interval; import org.apache.cassandra.utils.Overlaps; import org.apache.cassandra.utils.Pair; +import org.apache.cassandra.utils.TimeUUID; import org.mockito.Answers; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.MockitoAnnotations; +import static org.apache.cassandra.io.sstable.format.SSTableReader.UNIQUE_IDENTIFIER_FACTORY; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; @@ -851,6 +853,8 @@ SSTableReader mockSSTable(int level, when(ret.getFirst()).thenReturn(first); when(ret.getLast()).thenReturn(last); when(ret.getInterval()).thenReturn(new Interval<>(first, last, ret)); + when(ret.instanceId()).thenReturn(TimeUUID.Generator.nextTimeUUID(UNIQUE_IDENTIFIER_FACTORY)); + when(ret.compareTo(any())).thenCallRealMethod(); when(ret.isMarkedSuspect()).thenReturn(false); when(ret.isRepaired()).thenReturn(false); when(ret.getRepairedAt()).thenReturn(repairedAt); From ca04d6772026e93834f361dbea02a6e2612c52b2 Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi <contacto@bernardobotella.com> Date: Fri, 14 Mar 2025 11:30:52 -0700 Subject: [PATCH 192/225] Fix constraints package name typo patch by Bernardo Botella; reviewed by Sam Tunnicliffe for CASSANDRA-20438 --- .../AlterTableWithTableConstraintValidationTest.java | 2 +- .../{contraints => constraints}/ColumnConstraintsTest.java | 2 +- .../ConstraintsSatisfiabilityTest.java | 2 +- .../CqlConstraintValidationTester.java | 2 +- .../CreateTableWithColumnCqlConstraintValidationTest.java | 2 +- .../CreateTableWithColumnNotNullConstraintInvalidTest.java | 2 +- .../CreateTableWithColumnNotNullConstraintValidTest.java | 2 +- ...reateTableWithColumnOctetLengthConstraintValidationTest.java | 2 +- .../{contraints => constraints}/JsonConstraintTest.java | 2 +- .../{contraints => constraints}/RegexpConstraintTest.java | 2 +- 10 files changed, 10 insertions(+), 10 deletions(-) rename test/unit/org/apache/cassandra/{contraints => constraints}/AlterTableWithTableConstraintValidationTest.java (99%) rename test/unit/org/apache/cassandra/{contraints => constraints}/ColumnConstraintsTest.java (98%) rename test/unit/org/apache/cassandra/{contraints => constraints}/ConstraintsSatisfiabilityTest.java (99%) rename test/unit/org/apache/cassandra/{contraints => constraints}/CqlConstraintValidationTester.java (98%) rename test/unit/org/apache/cassandra/{contraints => constraints}/CreateTableWithColumnCqlConstraintValidationTest.java (99%) rename test/unit/org/apache/cassandra/{contraints => constraints}/CreateTableWithColumnNotNullConstraintInvalidTest.java (98%) rename test/unit/org/apache/cassandra/{contraints => constraints}/CreateTableWithColumnNotNullConstraintValidTest.java (98%) rename test/unit/org/apache/cassandra/{contraints => constraints}/CreateTableWithColumnOctetLengthConstraintValidationTest.java (99%) rename test/unit/org/apache/cassandra/{contraints => constraints}/JsonConstraintTest.java (98%) rename test/unit/org/apache/cassandra/{contraints => constraints}/RegexpConstraintTest.java (99%) diff --git a/test/unit/org/apache/cassandra/contraints/AlterTableWithTableConstraintValidationTest.java b/test/unit/org/apache/cassandra/constraints/AlterTableWithTableConstraintValidationTest.java similarity index 99% rename from test/unit/org/apache/cassandra/contraints/AlterTableWithTableConstraintValidationTest.java rename to test/unit/org/apache/cassandra/constraints/AlterTableWithTableConstraintValidationTest.java index 4db12f5a6979..d5f3f4a32c50 100644 --- a/test/unit/org/apache/cassandra/contraints/AlterTableWithTableConstraintValidationTest.java +++ b/test/unit/org/apache/cassandra/constraints/AlterTableWithTableConstraintValidationTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.contraints; +package org.apache.cassandra.constraints; import org.junit.Test; diff --git a/test/unit/org/apache/cassandra/contraints/ColumnConstraintsTest.java b/test/unit/org/apache/cassandra/constraints/ColumnConstraintsTest.java similarity index 98% rename from test/unit/org/apache/cassandra/contraints/ColumnConstraintsTest.java rename to test/unit/org/apache/cassandra/constraints/ColumnConstraintsTest.java index f43974e09eec..8d3bbe50f47e 100644 --- a/test/unit/org/apache/cassandra/contraints/ColumnConstraintsTest.java +++ b/test/unit/org/apache/cassandra/constraints/ColumnConstraintsTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.contraints; +package org.apache.cassandra.constraints; import org.junit.Test; diff --git a/test/unit/org/apache/cassandra/contraints/ConstraintsSatisfiabilityTest.java b/test/unit/org/apache/cassandra/constraints/ConstraintsSatisfiabilityTest.java similarity index 99% rename from test/unit/org/apache/cassandra/contraints/ConstraintsSatisfiabilityTest.java rename to test/unit/org/apache/cassandra/constraints/ConstraintsSatisfiabilityTest.java index a0a87b7d7b0d..6f087c5850b5 100644 --- a/test/unit/org/apache/cassandra/contraints/ConstraintsSatisfiabilityTest.java +++ b/test/unit/org/apache/cassandra/constraints/ConstraintsSatisfiabilityTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.contraints; +package org.apache.cassandra.constraints; import org.junit.Test; diff --git a/test/unit/org/apache/cassandra/contraints/CqlConstraintValidationTester.java b/test/unit/org/apache/cassandra/constraints/CqlConstraintValidationTester.java similarity index 98% rename from test/unit/org/apache/cassandra/contraints/CqlConstraintValidationTester.java rename to test/unit/org/apache/cassandra/constraints/CqlConstraintValidationTester.java index f2bc3dd1b464..8d2e52ff0e4f 100644 --- a/test/unit/org/apache/cassandra/contraints/CqlConstraintValidationTester.java +++ b/test/unit/org/apache/cassandra/constraints/CqlConstraintValidationTester.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.contraints; +package org.apache.cassandra.constraints; import java.util.Map; diff --git a/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnCqlConstraintValidationTest.java b/test/unit/org/apache/cassandra/constraints/CreateTableWithColumnCqlConstraintValidationTest.java similarity index 99% rename from test/unit/org/apache/cassandra/contraints/CreateTableWithColumnCqlConstraintValidationTest.java rename to test/unit/org/apache/cassandra/constraints/CreateTableWithColumnCqlConstraintValidationTest.java index 17c093c9f480..6dc160468642 100644 --- a/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnCqlConstraintValidationTest.java +++ b/test/unit/org/apache/cassandra/constraints/CreateTableWithColumnCqlConstraintValidationTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.contraints; +package org.apache.cassandra.constraints; import java.util.Arrays; diff --git a/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnNotNullConstraintInvalidTest.java b/test/unit/org/apache/cassandra/constraints/CreateTableWithColumnNotNullConstraintInvalidTest.java similarity index 98% rename from test/unit/org/apache/cassandra/contraints/CreateTableWithColumnNotNullConstraintInvalidTest.java rename to test/unit/org/apache/cassandra/constraints/CreateTableWithColumnNotNullConstraintInvalidTest.java index 73664f848b4a..dcb172163807 100644 --- a/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnNotNullConstraintInvalidTest.java +++ b/test/unit/org/apache/cassandra/constraints/CreateTableWithColumnNotNullConstraintInvalidTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.contraints; +package org.apache.cassandra.constraints; import java.util.Arrays; import java.util.Collection; diff --git a/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnNotNullConstraintValidTest.java b/test/unit/org/apache/cassandra/constraints/CreateTableWithColumnNotNullConstraintValidTest.java similarity index 98% rename from test/unit/org/apache/cassandra/contraints/CreateTableWithColumnNotNullConstraintValidTest.java rename to test/unit/org/apache/cassandra/constraints/CreateTableWithColumnNotNullConstraintValidTest.java index 5513a00251c4..b1ad79a43f1a 100644 --- a/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnNotNullConstraintValidTest.java +++ b/test/unit/org/apache/cassandra/constraints/CreateTableWithColumnNotNullConstraintValidTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.contraints; +package org.apache.cassandra.constraints; import java.util.Arrays; import java.util.Collection; diff --git a/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnOctetLengthConstraintValidationTest.java b/test/unit/org/apache/cassandra/constraints/CreateTableWithColumnOctetLengthConstraintValidationTest.java similarity index 99% rename from test/unit/org/apache/cassandra/contraints/CreateTableWithColumnOctetLengthConstraintValidationTest.java rename to test/unit/org/apache/cassandra/constraints/CreateTableWithColumnOctetLengthConstraintValidationTest.java index 5c79c957a513..6f9260f022ec 100644 --- a/test/unit/org/apache/cassandra/contraints/CreateTableWithColumnOctetLengthConstraintValidationTest.java +++ b/test/unit/org/apache/cassandra/constraints/CreateTableWithColumnOctetLengthConstraintValidationTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.contraints; +package org.apache.cassandra.constraints; import java.util.Arrays; diff --git a/test/unit/org/apache/cassandra/contraints/JsonConstraintTest.java b/test/unit/org/apache/cassandra/constraints/JsonConstraintTest.java similarity index 98% rename from test/unit/org/apache/cassandra/contraints/JsonConstraintTest.java rename to test/unit/org/apache/cassandra/constraints/JsonConstraintTest.java index 5a436892af18..95db5b7604de 100644 --- a/test/unit/org/apache/cassandra/contraints/JsonConstraintTest.java +++ b/test/unit/org/apache/cassandra/constraints/JsonConstraintTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.contraints; +package org.apache.cassandra.constraints; import org.junit.Test; diff --git a/test/unit/org/apache/cassandra/contraints/RegexpConstraintTest.java b/test/unit/org/apache/cassandra/constraints/RegexpConstraintTest.java similarity index 99% rename from test/unit/org/apache/cassandra/contraints/RegexpConstraintTest.java rename to test/unit/org/apache/cassandra/constraints/RegexpConstraintTest.java index fd4800e24787..3c865509416c 100644 --- a/test/unit/org/apache/cassandra/contraints/RegexpConstraintTest.java +++ b/test/unit/org/apache/cassandra/constraints/RegexpConstraintTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.contraints; +package org.apache.cassandra.constraints; import org.junit.Test; From dff453350aaaca3f85555d13bd7b2e1b999204db Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Thu, 13 Mar 2025 14:22:36 +0100 Subject: [PATCH 193/225] Optionally skip exception logging on invalid legacy protocol magic exception patch by Stefan Miklosovic; reviewed by David Capwell, Brad Schoening for CASSANDRA-19483 --- CHANGES.txt | 1 + .../org/apache/cassandra/config/Config.java | 1 + .../cassandra/config/DatabaseDescriptor.java | 5 +++ .../net/InboundConnectionInitiator.java | 7 ++++ .../test/InternodeErrorExclusionTest.java | 40 ++++++++++++++----- 5 files changed, 45 insertions(+), 9 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index ff0be82b8d5f..3f972dd9e3b0 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.1.9 + * Optionally skip exception logging on invalid legacy protocol magic exception (CASSANDRA-19483) * Fix SimpleClient ability to release acquired capacity (CASSANDRA-20202) * Fix WaitQueue.Signal.awaitUninterruptibly may block forever if invoking thread is interrupted (CASSANDRA-20084) Merged from 4.0: diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index a2324f76ddbf..b841d9b7c0fc 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -820,6 +820,7 @@ public static void setClientMode(boolean clientMode) public volatile SubnetGroups client_error_reporting_exclusions = new SubnetGroups(); public volatile SubnetGroups internode_error_reporting_exclusions = new SubnetGroups(); + public volatile boolean invalid_legacy_protocol_magic_no_spam_enabled = false; public volatile int keyspaces_warn_threshold = -1; public volatile int keyspaces_fail_threshold = -1; diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index cfec73758521..853b29f6df2e 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -4339,6 +4339,11 @@ public static SubnetGroups getInternodeErrorReportingExclusions() return conf.internode_error_reporting_exclusions; } + public static boolean getInvalidLegacyProtocolMagicNoSpamEnabled() + { + return conf.invalid_legacy_protocol_magic_no_spam_enabled; + } + public static boolean getReadThresholdsEnabled() { return conf.read_thresholds_enabled; diff --git a/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java b/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java index 3067b587c9c4..2257d0005045 100644 --- a/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java +++ b/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java @@ -23,9 +23,11 @@ import java.util.List; import java.util.NoSuchElementException; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import java.util.function.Consumer; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Throwables; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,6 +58,7 @@ import org.apache.cassandra.streaming.StreamDeserializingTask; import org.apache.cassandra.streaming.StreamingChannel; import org.apache.cassandra.streaming.async.NettyStreamingChannel; +import org.apache.cassandra.utils.NoSpamLogger; import org.apache.cassandra.utils.memory.BufferPools; import static java.lang.Math.*; @@ -69,6 +72,8 @@ public class InboundConnectionInitiator { private static final Logger logger = LoggerFactory.getLogger(InboundConnectionInitiator.class); + private static final NoSpamLogger noSpam5m = NoSpamLogger.getLogger(logger, 5, TimeUnit.MINUTES); + private static class Initializer extends ChannelInitializer<SocketChannel> { private static final String PIPELINE_INTERNODE_ERROR_EXCLUSIONS = "Internode Error Exclusions"; @@ -396,6 +401,8 @@ private void exceptionCaught(Channel channel, Throwable cause) if (reportingExclusion) logger.debug("Excluding internode exception for {}; address contained in internode_error_reporting_exclusions", remoteAddress, cause); + else if (cause != null && Throwables.getRootCause(cause) instanceof Message.InvalidLegacyProtocolMagic && DatabaseDescriptor.getInvalidLegacyProtocolMagicNoSpamEnabled()) + noSpam5m.warn("Failed to properly handshake with peer {}. Closing the channel. Invalid legacy protocol magic.", ((InetSocketAddress) channel.remoteAddress()).getHostName()); else logger.error("Failed to properly handshake with peer {}. Closing the channel.", remoteAddress, cause); diff --git a/test/distributed/org/apache/cassandra/distributed/test/InternodeErrorExclusionTest.java b/test/distributed/org/apache/cassandra/distributed/test/InternodeErrorExclusionTest.java index 08fd1229a5b0..81d0e80eaf8b 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/InternodeErrorExclusionTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/InternodeErrorExclusionTest.java @@ -53,16 +53,38 @@ public void ignoreExcludedInternodeErrors() throws IOException, TimeoutException .set("internode_error_reporting_exclusions", ImmutableMap.of("subnets", Arrays.asList("127.0.0.1")))) .start()) { - try (SimpleClient client = SimpleClient.builder("127.0.0.1", 7012).build()) - { - client.connect(true); - Assert.fail("Connection should fail"); - } - catch (Exception e) - { - // expected - } + + causeException(); assertThat(cluster.get(1).logs().watchFor("address contained in internode_error_reporting_exclusions").getResult()).hasSize(1); } } + + @Test + public void testNoSpammingInvalidLegacyProtocolMagicException() throws Throwable + { + try (Cluster cluster = Cluster.build(1) + .withConfig(c -> c + .with(Feature.NETWORK) + .set("invalid_legacy_protocol_magic_no_spam_enabled", true)) + .start()) + { + causeException(); + causeException(); + // we used no spam logger so the second message will not be emitted (the size is still 1). + assertThat(cluster.get(1).logs().watchFor("Failed to properly handshake with peer localhost. Closing the channel. Invalid legacy protocol magic.").getResult()).hasSize(1); + } + } + + private void causeException() + { + try (SimpleClient client = SimpleClient.builder("127.0.0.1", 7012).build()) + { + client.connect(true); + Assert.fail("Connection should fail"); + } + catch (Exception e) + { + // expected + } + } } From c925228a4df4cacd508be678cf3cca493544a691 Mon Sep 17 00:00:00 2001 From: Tiago Alves <tiago.alves@datastax.com> Date: Tue, 25 Feb 2025 15:17:58 +0000 Subject: [PATCH 194/225] Improve error messages when initializing auth classes patch by Tiago Alves; reviewed by Stefan Miklosovic, Maxwell Guo for CASSANDRA-20368 --- CHANGES.txt | 1 + .../org/apache/cassandra/auth/AuthConfig.java | 16 ++- .../cassandra/config/ParameterizedClass.java | 78 +++++++++----- .../cassandra/audit/AuditLoggerAuthTest.java | 12 ++- .../config/ParameterizedClassExample.java | 43 ++++++++ .../config/ParameterizedClassTest.java | 101 ++++++++++++++++++ .../cassandra/transport/CQLUserAuditTest.java | 10 +- 7 files changed, 218 insertions(+), 43 deletions(-) create mode 100644 test/unit/org/apache/cassandra/config/ParameterizedClassExample.java create mode 100644 test/unit/org/apache/cassandra/config/ParameterizedClassTest.java diff --git a/CHANGES.txt b/CHANGES.txt index d778d0c9595f..eea30f01a477 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.4 + * Improve error messages when initializing auth classes (CASSANDRA-20368) * Prioritize legacy 2i over SAI for columns with multiple indexes (CASSANDRA-20334) * Ensure only offline tools can build IntervalTrees without first/last key fields (CASSANDRA-20407) * Improve IntervalTree build throughput (CASSANDRA-19596) diff --git a/src/java/org/apache/cassandra/auth/AuthConfig.java b/src/java/org/apache/cassandra/auth/AuthConfig.java index f357825b10b2..1363b24a6fe2 100644 --- a/src/java/org/apache/cassandra/auth/AuthConfig.java +++ b/src/java/org/apache/cassandra/auth/AuthConfig.java @@ -73,7 +73,10 @@ public static void applyAuth() IAuthorizer authorizer = authInstantiate(conf.authorizer, AllowAllAuthorizer.class); if (!authenticator.requireAuthentication() && authorizer.requireAuthorization()) - throw new ConfigurationException(conf.authenticator.class_name + " can't be used with " + conf.authorizer, false); + { + throw new ConfigurationException(authorizer.getClass().getName() + " has authorization enabled which requires " + + authenticator.getClass().getName() + " to enable authentication", false); + } DatabaseDescriptor.setAuthorizer(authorizer); @@ -82,7 +85,7 @@ public static void applyAuth() IRoleManager roleManager = authInstantiate(conf.role_manager, CassandraRoleManager.class); if (authenticator instanceof PasswordAuthenticator && !(roleManager instanceof CassandraRoleManager)) - throw new ConfigurationException("CassandraRoleManager must be used with PasswordAuthenticator", false); + throw new ConfigurationException(authenticator.getClass().getName() + " requires " + CassandraRoleManager.class.getName(), false); DatabaseDescriptor.setRoleManager(roleManager); @@ -132,13 +135,6 @@ private static <T> T authInstantiate(ParameterizedClass authCls, Class<T> defaul return ParameterizedClass.newInstance(authCls, List.of("", authPackage)); } - try - { - return defaultCls.newInstance(); - } - catch (InstantiationException | IllegalAccessException e) - { - throw new ConfigurationException("Failed to instantiate " + defaultCls.getName(), e); - } + return ParameterizedClass.newInstance(new ParameterizedClass(defaultCls.getName()), List.of()); } } diff --git a/src/java/org/apache/cassandra/config/ParameterizedClass.java b/src/java/org/apache/cassandra/config/ParameterizedClass.java index 2d0390ef4486..d772629f194c 100644 --- a/src/java/org/apache/cassandra/config/ParameterizedClass.java +++ b/src/java/org/apache/cassandra/config/ParameterizedClass.java @@ -18,9 +18,12 @@ package org.apache.cassandra.config; import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.function.Predicate; +import java.util.stream.Collectors; import com.google.common.base.Objects; @@ -45,8 +48,7 @@ public ParameterizedClass() public ParameterizedClass(String class_name) { - this.class_name = class_name; - this.parameters = Collections.emptyMap(); + this(class_name, Collections.emptyMap()); } public ParameterizedClass(String class_name, Map<String, String> parameters) @@ -64,7 +66,7 @@ public ParameterizedClass(Map<String, ?> p) static public <K> K newInstance(ParameterizedClass parameterizedClass, List<String> searchPackages) { - Exception last = null; + Class<?> providerClass = null; if (searchPackages == null || searchPackages.isEmpty()) searchPackages = Collections.singletonList(""); for (String searchPackage : searchPackages) @@ -74,32 +76,56 @@ static public <K> K newInstance(ParameterizedClass parameterizedClass, List<Stri if (!searchPackage.isEmpty() && !searchPackage.endsWith(".")) searchPackage = searchPackage + '.'; String name = searchPackage + parameterizedClass.class_name; - Class<?> providerClass = Class.forName(name); - try - { - Constructor<?> constructor = providerClass.getConstructor(Map.class); - K instance = (K) constructor.newInstance(parameterizedClass.parameters); - return instance; - } - catch (Exception constructorEx) - { - //no-op - } - // fallback to no arg constructor if no params present - if (parameterizedClass.parameters == null || parameterizedClass.parameters.isEmpty()) - { - Constructor<?> constructor = providerClass.getConstructor(); - K instance = (K) constructor.newInstance(); - return instance; - } + providerClass = Class.forName(name); } - // there are about 5 checked exceptions that could be thrown here. - catch (Exception e) + catch (ClassNotFoundException e) { - last = e; + //no-op } } - throw new ConfigurationException("Unable to create parameterized class " + parameterizedClass.class_name, last); + + if (providerClass == null) + { + String pkgList = '[' + searchPackages.stream().map(p -> '"' + p + '"').collect(Collectors.joining(",")) + ']'; + String error = "Unable to find class " + parameterizedClass.class_name + " in packages " + pkgList; + throw new ConfigurationException(error); + } + + try + { + Constructor<?> mapConstructor = filterConstructor(providerClass, c -> c.getParameterTypes().length == 1 && c.getParameterTypes()[0].equals(Map.class)); + if (mapConstructor != null) + return (K) mapConstructor.newInstance(parameterizedClass.parameters == null ? Collections.emptyMap() : parameterizedClass.parameters); + + // Falls-back to no-arg constructor + Constructor<?> noArgsConstructor = filterConstructor(providerClass, c -> c.getParameterTypes().length == 0); + if (noArgsConstructor != null) + return (K) noArgsConstructor.newInstance(); + + throw new ConfigurationException("No valid constructor found for class " + parameterizedClass.class_name); + } + catch (IllegalAccessException | InstantiationException | ExceptionInInitializerError e) + { + throw new ConfigurationException("Unable to instantiate parameterized class " + parameterizedClass.class_name, e); + } + catch (InvocationTargetException e) + { + Throwable cause = e.getCause(); + String error = "Failed to instantiate class " + parameterizedClass.class_name + + (cause.getMessage() != null ? ": " + cause.getMessage() : ""); + throw new ConfigurationException(error, cause); + } + } + + private static Constructor<?> filterConstructor(Class<?> providerClass, Predicate<Constructor<?>> filter) + { + for (Constructor<?> constructor : providerClass.getDeclaredConstructors()) + { + if (filter.test(constructor)) + return constructor; + } + + return null; } @Override @@ -122,6 +148,6 @@ public int hashCode() @Override public String toString() { - return class_name + (parameters == null ? "" : parameters.toString()); + return class_name + parameters; } } diff --git a/test/unit/org/apache/cassandra/audit/AuditLoggerAuthTest.java b/test/unit/org/apache/cassandra/audit/AuditLoggerAuthTest.java index 33a57eb29d91..0e81afa1e273 100644 --- a/test/unit/org/apache/cassandra/audit/AuditLoggerAuthTest.java +++ b/test/unit/org/apache/cassandra/audit/AuditLoggerAuthTest.java @@ -23,6 +23,10 @@ import java.util.List; import java.util.Queue; +import org.apache.cassandra.auth.CassandraAuthorizer; +import org.apache.cassandra.auth.CassandraRoleManager; +import org.apache.cassandra.auth.PasswordAuthenticator; + import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; @@ -69,11 +73,11 @@ public class AuditLoggerAuthTest public static void setup() throws Exception { OverrideConfigurationLoader.override((config) -> { - config.authenticator = new ParameterizedClass("PasswordAuthenticator"); - config.role_manager = new ParameterizedClass("CassandraRoleManager"); - config.authorizer = new ParameterizedClass("CassandraAuthorizer"); + config.authenticator = new ParameterizedClass(PasswordAuthenticator.class.getName()); + config.role_manager = new ParameterizedClass(CassandraRoleManager.class.getName()); + config.authorizer = new ParameterizedClass(CassandraAuthorizer.class.getName()); config.audit_logging_options.enabled = true; - config.audit_logging_options.logger = new ParameterizedClass("InMemoryAuditLogger", null); + config.audit_logging_options.logger = new ParameterizedClass(InMemoryAuditLogger.class.getName()); }); SUPERUSER_SETUP_DELAY_MS.setLong(0); diff --git a/test/unit/org/apache/cassandra/config/ParameterizedClassExample.java b/test/unit/org/apache/cassandra/config/ParameterizedClassExample.java new file mode 100644 index 000000000000..0bd3c7a6d4be --- /dev/null +++ b/test/unit/org/apache/cassandra/config/ParameterizedClassExample.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.config; + +import java.util.Map; + +import org.junit.Assert; + +public class ParameterizedClassExample +{ + public ParameterizedClassExample() + { + Assert.fail("This constructor should not be called"); + } + + public ParameterizedClassExample(Map<String, String> parameters) + { + if (parameters == null) + throw new IllegalArgumentException("Parameters must not be null"); + + boolean simulateFailure = Boolean.parseBoolean(parameters.getOrDefault("fail", "false")); + if (simulateFailure) + { + throw new IllegalArgumentException("Simulated failure"); + } + } +} diff --git a/test/unit/org/apache/cassandra/config/ParameterizedClassTest.java b/test/unit/org/apache/cassandra/config/ParameterizedClassTest.java new file mode 100644 index 000000000000..732812aefd15 --- /dev/null +++ b/test/unit/org/apache/cassandra/config/ParameterizedClassTest.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.config; + +import java.util.List; +import java.util.Map; + +import org.junit.Test; + +import org.apache.cassandra.auth.AllowAllAuthorizer; +import org.apache.cassandra.auth.IAuthorizer; +import org.apache.cassandra.exceptions.ConfigurationException; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +public class ParameterizedClassTest +{ + @Test + public void testParameterizedClassEmptyConstructorHasNullParameters() + { + ParameterizedClass parameterizedClass = new ParameterizedClass(); + assertNull(parameterizedClass.parameters); + } + + @Test + public void testParameterizedClassConstructorWithClassNameHasNonNullParameters() + { + ParameterizedClass parameterizedClass = new ParameterizedClass("TestClass"); + assertNotNull(parameterizedClass.parameters); + } + + @Test + public void testParameterizedClassConstructorWithClassNameAndParametersHasNullParamters() + { + ParameterizedClass parameterizedClass = new ParameterizedClass("TestClass", null); + assertNull(parameterizedClass.parameters); + } + + @Test + public void testNewInstanceWithNonExistentClassFailsWithConfigurationException() + { + assertThatThrownBy(() -> ParameterizedClass.newInstance(new ParameterizedClass("NonExistentClass"), + List.of("org.apache.cassandra.config"))) + .hasMessage("Unable to find class NonExistentClass in packages [\"org.apache.cassandra.config\"]") + .isInstanceOf(ConfigurationException.class); + } + + @Test + public void testNewInstanceWithSingleEmptyConstructorUsesEmptyConstructor() + { + ParameterizedClass parameterizedClass = new ParameterizedClass(AllowAllAuthorizer.class.getName()); + IAuthorizer instance = ParameterizedClass.newInstance(parameterizedClass, null); + assertNotNull(instance); + } + + @Test + public void testNewInstanceWithValidConstructorsFavorsMapConstructor() + { + ParameterizedClass parameterizedClass = new ParameterizedClass(ParameterizedClassExample.class.getName()); + ParameterizedClassExample instance = ParameterizedClass.newInstance(parameterizedClass, null); + assertNotNull(instance); + } + + @Test + public void testNewInstanceWithValidConstructorsUsingNullParamtersFavorsMapConstructor() + { + ParameterizedClass parameterizedClass = new ParameterizedClass(ParameterizedClassExample.class.getName()); + parameterizedClass.parameters = null; + + ParameterizedClassExample instance = ParameterizedClass.newInstance(parameterizedClass, null); + assertNotNull(instance); + } + + @Test + public void testNewInstanceWithConstructorExceptionPreservesOriginalFailure() + { + assertThatThrownBy(() -> ParameterizedClass.newInstance(new ParameterizedClass(ParameterizedClassExample.class.getName(), + Map.of("fail", "true")), null)) + .hasMessageStartingWith("Failed to instantiate class") + .hasMessageContaining("Simulated failure") + .isInstanceOf(ConfigurationException.class); + } +} diff --git a/test/unit/org/apache/cassandra/transport/CQLUserAuditTest.java b/test/unit/org/apache/cassandra/transport/CQLUserAuditTest.java index b0d1debfda74..e31aa73c5920 100644 --- a/test/unit/org/apache/cassandra/transport/CQLUserAuditTest.java +++ b/test/unit/org/apache/cassandra/transport/CQLUserAuditTest.java @@ -29,6 +29,10 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import org.apache.cassandra.audit.DiagnosticEventAuditLogger; +import org.apache.cassandra.auth.CassandraRoleManager; +import org.apache.cassandra.auth.PasswordAuthenticator; + import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -64,11 +68,11 @@ public class CQLUserAuditTest public static void setup() throws Exception { OverrideConfigurationLoader.override((config) -> { - config.authenticator = new ParameterizedClass("PasswordAuthenticator"); - config.role_manager = new ParameterizedClass("CassandraRoleManager"); + config.authenticator = new ParameterizedClass(PasswordAuthenticator.class.getName()); + config.role_manager = new ParameterizedClass(CassandraRoleManager.class.getName()); config.diagnostic_events_enabled = true; config.audit_logging_options.enabled = true; - config.audit_logging_options.logger = new ParameterizedClass("DiagnosticEventAuditLogger", null); + config.audit_logging_options.logger = new ParameterizedClass(DiagnosticEventAuditLogger.class.getName()); }); SUPERUSER_SETUP_DELAY_MS.setLong(0); From e16f30c58711185d333af2e8ef2ed1e91af9e32c Mon Sep 17 00:00:00 2001 From: maoling <maoling@apache.org> Date: Sun, 9 Mar 2025 23:56:00 +0800 Subject: [PATCH 195/225] Improve Go download when building source code patch by Ling Mao; reviewed by Stefan Miklosovic, Michael Semb Wever for CASSANDRA-20422 Co-authored-by: Stefan Miklosovic <smiklosovic@apache.org> --- .jenkins/Jenkinsfile | 4 +- build.xml | 8 +- ...process-native-protocol-specs-in-docker.sh | 87 ++++++++++++++++--- 3 files changed, 80 insertions(+), 19 deletions(-) diff --git a/.jenkins/Jenkinsfile b/.jenkins/Jenkinsfile index 808d3cb44952..4d5cb6dfa189 100644 --- a/.jenkins/Jenkinsfile +++ b/.jenkins/Jenkinsfile @@ -221,7 +221,7 @@ def tasks() { // Disable splits for all but proper stages !(axis['split'] > 1 && !stepsMap.findAll { entry -> entry.value.splits >= axis['split'] }.keySet().contains(axis['step'])) && // run only the build types on non-amd64 - !(axis['arch'] != 'amd64' && stepsMap.findAll { entry -> 'build' == entry.value.type }.keySet().contains(axis['step'])) + !(axis['arch'] != 'amd64' && !stepsMap.findAll { entry -> 'build' == entry.value.type }.keySet().contains(axis['step'])) } def Map tasks = [ @@ -337,7 +337,7 @@ def build(command, cell) { test -f .jenkins/Jenkinsfile || { echo "Invalid git fork/branch"; exit 1; } grep -q "Jenkins CI declaration" .jenkins/Jenkinsfile || { echo "Only Cassandra 5.0+ supported"; exit 1; } """ - fetchDockerImages(['almalinux-build', 'bullseye-build', 'centos7-build']) + fetchDockerImages(['almalinux-build', 'bullseye-build']) def cell_suffix = "_jdk${cell.jdk}_${cell.arch}" def logfile = "stage-logs/${JOB_NAME}_${BUILD_NUMBER}_${cell.step}${cell_suffix}_attempt${attempt}.log.xz" def script_vars = "#!/bin/bash \n set -o pipefail ; " // pipe to tee needs pipefail diff --git a/build.xml b/build.xml index 326178df174a..f1d6e925158b 100644 --- a/build.xml +++ b/build.xml @@ -468,13 +468,13 @@ </target> <target name="gen-asciidoc" description="Generate dynamic asciidoc pages" depends="jar" unless="ant.gen-doc.skip"> - <exec executable="make" osfamily="unix" dir="${doc.dir}"> + <exec executable="make" osfamily="unix" dir="${doc.dir}" failonerror="true"> <arg value="gen-asciidoc"/> </exec> </target> <target name="gen-doc" description="Generate documentation" depends="gen-asciidoc,generate-cql-html" unless="ant.gen-doc.skip"> - <exec executable="make" osfamily="unix" dir="${doc.dir}"> + <exec executable="make" osfamily="unix" dir="${doc.dir}" failonerror="true"> <arg value="html"/> </exec> </target> @@ -548,7 +548,7 @@ <copy todir="${basedir}/conf" file="${build.classes.main}/META-INF/hotspot_compiler"/> </target> - <target name="check" depends="_main-jar,build-test" description="Verifies the source code and dependencies. This task is intended to run on pre-commit and locally. It should verify mostly modified files compared to the upstream base branch." unless="check.skip"> + <target name="check" depends="_main-jar,build-test,gen-doc" description="Verifies the source code and dependencies. This task is intended to run on pre-commit and locally. It should verify mostly modified files compared to the upstream base branch." unless="check.skip"> <antcall target="rat-check" inheritrefs="true"/> <antcall target="checkstyle" inheritrefs="true"/> <antcall target="checkstyle-test" inheritrefs="true"/> @@ -909,7 +909,7 @@ </target> <!-- creates release tarballs --> - <target name="artifacts" depends="_artifacts-init,check,gen-doc,sources-jar" + <target name="artifacts" depends="_artifacts-init,check,sources-jar" description="Create Cassandra tarball and maven artifacts"> <tar compression="gzip" longfile="gnu" destfile="${build.dir}/${final.name}-bin.tar.gz"> diff --git a/doc/scripts/process-native-protocol-specs-in-docker.sh b/doc/scripts/process-native-protocol-specs-in-docker.sh index 3af83f4a310f..05565c02b93d 100755 --- a/doc/scripts/process-native-protocol-specs-in-docker.sh +++ b/doc/scripts/process-native-protocol-specs-in-docker.sh @@ -20,33 +20,94 @@ # Variables GO_VERSION="1.23.1" -GO_TAR="go${GO_VERSION}.linux-amd64.tar.gz" -TMPDIR="${TMPDIR:-/tmp}" -# Step 0: Download and install Go -echo "Downloading Go $GO_VERSION..." -wget -q "https://golang.org/dl/$GO_TAR" -O "$TMPDIR/$GO_TAR" +GO_OS=linux + +if [ $(uname) = "Darwin" ]; then + GO_OS=darwin +fi + +GO_PLATFORM=amd64 -echo "Installing Go..." -tar -C "$TMPDIR" -xzf "$TMPDIR/$GO_TAR" -rm "$TMPDIR/$GO_TAR" +if [ $(uname -m) = "aarch64" ]; then + GO_PLATFORM=arm64 +fi -# Set Go environment variables -export PATH="$PATH:$TMPDIR/go/bin" -export GOPATH="$TMPDIR/go" +GO_TAR="go${GO_VERSION}.${GO_OS}-${GO_PLATFORM}.tar.gz" +TMPDIR="${TMPDIR:-/tmp}" + +check_go_version() { + if command -v go &>/dev/null; then + local installed_version=$(go version | awk '{print $3}' | sed 's/go//') + + if [ "$(printf '%s\n' "$GO_VERSION" "$installed_version" | sort -V | head -n1)" = "$GO_VERSION" ]; then + echo "Detected Go $installed_version (>= $GO_VERSION), skipping installation." + return 0 + else + if [ -z $installed_version ]; then + echo "No Go installation detected, proceeding with installation." + else + echo "Detected Go $installed_version (< $GO_VERSION), proceeding with installation." + fi + return 1 + fi + else + echo "Go env not found in your system, proceeding with installation." + return 1 + fi +} + +if ! check_go_version; then + + if ls $TMPDIR/go$GO_VERSION > /dev/null 2>&1; then + echo "Reusing cached installation in $TMPDIR/go$GO_VERSION" + export PATH="$PATH:$TMPDIR/go$GO_VERSION/go/bin" + export GOPATH="$TMPDIR/go$GO_VERSION/go/bin" + export GOROOT="$TMPDIR/go$GO_VERSION/go" + else + if ! ls $TMPDIR/$GO_TAR > /dev/null 2>&1; then + echo "Downloading Go $GO_VERSION..." + + curl -L --fail --silent --retry 2 --retry-delay 5 --max-time 30 https://golang.org/dl/$GO_TAR -o $TMPDIR/$GO_TAR + + if [ $? != "0" ]; then + echo "Network error. Specify '-Dant.gen-doc.skip=true' to skip if offline." + exit 1 + fi + fi + + echo "Installing Go $GO_VERSION..." + mkdir -p $TMPDIR/go$GO_VERSION + tar -C "$TMPDIR/go$GO_VERSION" -xzf "$TMPDIR/$GO_TAR" + + # Set Go environment variables + export PATH="$PATH:$TMPDIR/go$GO_VERSION/go/bin" + export GOPATH="$TMPDIR/go$GO_VERSION/go/bin" + export GOROOT="$TMPDIR/go$GO_VERSION/go" + fi +else + echo "Using system-installed Go." +fi # Step 1: Building the parser echo "Building the cqlprotodoc..." DIR="$(pwd)" cd "${TMPDIR}" +rm -rf "${TMPDIR}/cassandra-website" git clone -n --depth=1 --filter=tree:0 https://github.com/apache/cassandra-website +if [ $? != "0" ]; then + echo "Error occured while cloning https://github.com/apache/cassandra-website" + exit 1 +fi + cd "${TMPDIR}/cassandra-website" git sparse-checkout set --no-cone /cqlprotodoc git checkout cd "${TMPDIR}/cassandra-website/cqlprotodoc" -go build -o "$TMPDIR"/cqlprotodoc +rm -rf "${TMPDIR}/cqlprotodoc" +$TMPDIR/go$GO_VERSION/go/bin/go build -o "$TMPDIR"/cqlprotodoc # Step 2: Process the spec files using the parser echo "Processing the .spec files..." @@ -121,6 +182,6 @@ echo " </script>" >> "$summary_file" # Step 3: Cleanup - Remove the Cassandra and parser directories echo "Cleaning up..." cd "${DIR}" -rm -rf "${TMPDIR}/go" "${TMPDIR}/cassandra-website" "${TMPDIR}/cqlprotodoc" 2>/dev/null +rm -rf "${TMPDIR}/cassandra-website" "${TMPDIR}/cqlprotodoc" 2>/dev/null echo "Script completed successfully." From 35d51725ef73cd6749393f3c08ecc0fab3f33513 Mon Sep 17 00:00:00 2001 From: Jordan West <jordan@moonwest.dev> Date: Wed, 17 Apr 2024 10:27:02 -0700 Subject: [PATCH 196/225] Add read ahead buffer for scans of compressed data files Patch by Jordan West, Jon Haddad; Reviewed by David Capwell, Caleb Rackliffe, Dmitry Konstantinov for CASSANDRA-15452 NOTE: This was originally merged up via merge commit but something went wrong, this is the fix commit to bring this back to trunk --- .../apache/cassandra/cache/ChunkCache.java | 2 +- .../org/apache/cassandra/config/Config.java | 2 + .../cassandra/config/DatabaseDescriptor.java | 21 ++ .../io/sstable/format/SSTableReader.java | 5 + .../io/sstable/format/SSTableScanner.java | 2 +- .../io/util/BufferManagingRebufferer.java | 1 + .../apache/cassandra/io/util/ChunkReader.java | 2 + .../io/util/CompressedChunkReader.java | 202 ++++++++++++++--- .../cassandra/io/util/EmptyRebufferer.java | 2 +- .../apache/cassandra/io/util/FileHandle.java | 19 +- .../cassandra/io/util/MmapRebufferer.java | 2 +- .../cassandra/io/util/RandomAccessReader.java | 2 +- .../cassandra/io/util/RebuffererFactory.java | 2 +- .../cassandra/io/util/SimpleChunkReader.java | 2 +- .../io/util/ThreadLocalReadAheadBuffer.java | 154 +++++++++++++ .../cassandra/service/StorageService.java | 13 ++ .../service/StorageServiceMBean.java | 3 + .../io/util/CompressedChunkReaderTest.java | 141 ++++++++++++ .../util/ThreadLocalReadAheadBufferTest.java | 204 ++++++++++++++++++ 19 files changed, 743 insertions(+), 38 deletions(-) create mode 100644 src/java/org/apache/cassandra/io/util/ThreadLocalReadAheadBuffer.java create mode 100644 test/unit/org/apache/cassandra/io/util/CompressedChunkReaderTest.java create mode 100644 test/unit/org/apache/cassandra/io/util/ThreadLocalReadAheadBufferTest.java diff --git a/src/java/org/apache/cassandra/cache/ChunkCache.java b/src/java/org/apache/cassandra/cache/ChunkCache.java index afd5804089bc..958007c50460 100644 --- a/src/java/org/apache/cassandra/cache/ChunkCache.java +++ b/src/java/org/apache/cassandra/cache/ChunkCache.java @@ -258,7 +258,7 @@ public void invalidate(long position) } @Override - public Rebufferer instantiateRebufferer() + public Rebufferer instantiateRebufferer(boolean isScan) { return this; } diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 518469a14987..2ec1d78e301f 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -345,6 +345,8 @@ public MemtableOptions() @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"); + public DataStorageSpec.IntKibibytesBound compressed_read_ahead_buffer_size = new DataStorageSpec.IntKibibytesBound("256KiB"); + // fraction of free disk space available for compaction after min free space is subtracted public volatile Double max_space_usable_for_compactions_in_percentage = .95; diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 4033aa7918b0..f6fd1b52ff44 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -1025,6 +1025,9 @@ else if (conf.max_value_size.toMebibytes() >= 2048) break; } + if (conf.compressed_read_ahead_buffer_size.toKibibytes() > 0 && conf.compressed_read_ahead_buffer_size.toKibibytes() < 256) + throw new ConfigurationException("compressed_read_ahead_buffer_size must be at least 256KiB (set to 0 to disable), but was " + conf.compressed_read_ahead_buffer_size, false); + if (conf.server_encryption_options != null) { conf.server_encryption_options.applyConfig(); @@ -2730,6 +2733,24 @@ public static void setConcurrentViewBuilders(int value) conf.concurrent_materialized_view_builders = value; } + public static int getCompressedReadAheadBufferSize() + { + return conf.compressed_read_ahead_buffer_size.toBytes(); + } + + public static int getCompressedReadAheadBufferSizeInKB() + { + return conf.compressed_read_ahead_buffer_size.toKibibytes(); + } + + public static void setCompressedReadAheadBufferSizeInKb(int sizeInKb) + { + if (sizeInKb < 256) + throw new IllegalArgumentException("compressed_read_ahead_buffer_size_in_kb must be at least 256KiB"); + + conf.compressed_read_ahead_buffer_size = createIntKibibyteBoundAndEnsureItIsValidForByteConversion(sizeInKb, "compressed_read_ahead_buffer_size"); + } + public static long getMinFreeSpacePerDriveInMebibytes() { return conf.min_free_space_per_drive.toMebibytes(); diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java index 92359e4d0c6d..ff488694dc82 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java @@ -1375,6 +1375,11 @@ public RandomAccessReader openDataReader() return dfile.createReader(); } + public RandomAccessReader openDataReaderForScan() + { + return dfile.createReaderForScan(); + } + public void trySkipFileCacheBefore(DecoratedKey key) { long position = getPosition(key, SSTableReader.Operator.GE); diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableScanner.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableScanner.java index 86c0f7ec2dc0..28035a85da0b 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableScanner.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableScanner.java @@ -75,7 +75,7 @@ protected SSTableScanner(S sstable, { assert sstable != null; - this.dfile = sstable.openDataReader(); + this.dfile = sstable.openDataReaderForScan(); this.sstable = sstable; this.columns = columns; this.dataRange = dataRange; diff --git a/src/java/org/apache/cassandra/io/util/BufferManagingRebufferer.java b/src/java/org/apache/cassandra/io/util/BufferManagingRebufferer.java index 3a297ee0e260..13b7c9d44141 100644 --- a/src/java/org/apache/cassandra/io/util/BufferManagingRebufferer.java +++ b/src/java/org/apache/cassandra/io/util/BufferManagingRebufferer.java @@ -50,6 +50,7 @@ protected BufferManagingRebufferer(ChunkReader wrapped) public void closeReader() { BufferPools.forChunkCache().put(buffer); + source.releaseUnderlyingResources(); offset = -1; } diff --git a/src/java/org/apache/cassandra/io/util/ChunkReader.java b/src/java/org/apache/cassandra/io/util/ChunkReader.java index 33bf7921edd6..779e7c35f94a 100644 --- a/src/java/org/apache/cassandra/io/util/ChunkReader.java +++ b/src/java/org/apache/cassandra/io/util/ChunkReader.java @@ -48,4 +48,6 @@ public interface ChunkReader extends RebuffererFactory * This is not guaranteed to be fulfilled. */ BufferType preferredBufferType(); + + default void releaseUnderlyingResources() {} } diff --git a/src/java/org/apache/cassandra/io/util/CompressedChunkReader.java b/src/java/org/apache/cassandra/io/util/CompressedChunkReader.java index b0aa24bd8f6f..b6b3c9a6a267 100644 --- a/src/java/org/apache/cassandra/io/util/CompressedChunkReader.java +++ b/src/java/org/apache/cassandra/io/util/CompressedChunkReader.java @@ -26,11 +26,13 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.primitives.Ints; +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.io.compress.BufferType; import org.apache.cassandra.io.compress.CompressionMetadata; import org.apache.cassandra.io.compress.CorruptBlockException; import org.apache.cassandra.io.sstable.CorruptSSTableException; import org.apache.cassandra.utils.ChecksumType; +import org.apache.cassandra.utils.Closeable; public abstract class CompressedChunkReader extends AbstractReaderFileProxy implements ChunkReader { @@ -47,6 +49,11 @@ protected CompressedChunkReader(ChannelProxy channel, CompressionMetadata metada assert Integer.bitCount(metadata.chunkLength()) == 1; //must be a power of two } + protected CompressedChunkReader forScan() + { + return this; + } + @VisibleForTesting public double getCrcCheckChance() { @@ -83,20 +90,167 @@ public BufferType preferredBufferType() } @Override - public Rebufferer instantiateRebufferer() + public Rebufferer instantiateRebufferer(boolean isScan) { - return new BufferManagingRebufferer.Aligned(this); + return new BufferManagingRebufferer.Aligned(isScan ? forScan() : this); } - public static class Standard extends CompressedChunkReader + protected interface CompressedReader extends Closeable { - // we read the raw compressed bytes into this buffer, then uncompressed them into the provided one. + default void allocateResources() + { + } + + default void deallocateResources() + { + } + + default boolean allocated() + { + return false; + } + + default void close() + { + + } + + + ByteBuffer read(CompressionMetadata.Chunk chunk, boolean shouldCheckCrc) throws CorruptBlockException; + } + + private static class RandomAccessCompressedReader implements CompressedReader + { + private final ChannelProxy channel; private final ThreadLocalByteBufferHolder bufferHolder; + private RandomAccessCompressedReader(ChannelProxy channel, CompressionMetadata metadata) + { + this.channel = channel; + this.bufferHolder = new ThreadLocalByteBufferHolder(metadata.compressor().preferredBufferType()); + } + + @Override + public ByteBuffer read(CompressionMetadata.Chunk chunk, boolean shouldCheckCrc) throws CorruptBlockException + { + int length = shouldCheckCrc ? chunk.length + Integer.BYTES // compressed length + checksum length + : chunk.length; + ByteBuffer compressed = bufferHolder.getBuffer(length); + if (channel.read(compressed, chunk.offset) != length) + throw new CorruptBlockException(channel.filePath(), chunk); + compressed.flip(); + compressed.limit(chunk.length); + + if (shouldCheckCrc) + { + int checksum = (int) ChecksumType.CRC32.of(compressed); + compressed.limit(length); + if (compressed.getInt() != checksum) + throw new CorruptBlockException(channel.filePath(), chunk); + compressed.position(0).limit(chunk.length); + } + return compressed; + } + } + + private static class ScanCompressedReader implements CompressedReader + { + private final ChannelProxy channel; + private final ThreadLocalByteBufferHolder bufferHolder; + private final ThreadLocalReadAheadBuffer readAheadBuffer; + + private ScanCompressedReader(ChannelProxy channel, CompressionMetadata metadata, int readAheadBufferSize) + { + this.channel = channel; + this.bufferHolder = new ThreadLocalByteBufferHolder(metadata.compressor().preferredBufferType()); + this.readAheadBuffer = new ThreadLocalReadAheadBuffer(channel, readAheadBufferSize, metadata.compressor().preferredBufferType()); + } + + @Override + public ByteBuffer read(CompressionMetadata.Chunk chunk, boolean shouldCheckCrc) throws CorruptBlockException + { + int length = shouldCheckCrc ? chunk.length + Integer.BYTES // compressed length + checksum length + : chunk.length; + ByteBuffer compressed = bufferHolder.getBuffer(length); + + int copied = 0; + while (copied < length) + { + readAheadBuffer.fill(chunk.offset + copied); + int leftToRead = length - copied; + if (readAheadBuffer.remaining() >= leftToRead) + copied += readAheadBuffer.read(compressed, leftToRead); + else + copied += readAheadBuffer.read(compressed, readAheadBuffer.remaining()); + } + + compressed.flip(); + compressed.limit(chunk.length); + + if (shouldCheckCrc) + { + int checksum = (int) ChecksumType.CRC32.of(compressed); + compressed.limit(length); + if (compressed.getInt() != checksum) + throw new CorruptBlockException(channel.filePath(), chunk); + compressed.position(0).limit(chunk.length); + } + return compressed; + } + + @Override + public void allocateResources() + { + readAheadBuffer.allocateBuffer(); + } + + @Override + public void deallocateResources() + { + readAheadBuffer.clear(true); + } + + @Override + public boolean allocated() + { + return readAheadBuffer.hasBuffer(); + } + + public void close() + { + readAheadBuffer.close(); + } + } + + public static class Standard extends CompressedChunkReader + { + + private final CompressedReader reader; + private final CompressedReader scanReader; + public Standard(ChannelProxy channel, CompressionMetadata metadata, Supplier<Double> crcCheckChanceSupplier) { super(channel, metadata, crcCheckChanceSupplier); - bufferHolder = new ThreadLocalByteBufferHolder(metadata.compressor().preferredBufferType()); + reader = new RandomAccessCompressedReader(channel, metadata); + + int readAheadBufferSize = DatabaseDescriptor.getCompressedReadAheadBufferSize(); + scanReader = (readAheadBufferSize > 0 && readAheadBufferSize > metadata.chunkLength()) + ? new ScanCompressedReader(channel, metadata, readAheadBufferSize) : null; + } + + protected CompressedChunkReader forScan() + { + if (scanReader != null) + scanReader.allocateResources(); + + return this; + } + + @Override + public void releaseUnderlyingResources() + { + if (scanReader != null) + scanReader.deallocateResources(); } @Override @@ -110,31 +264,13 @@ public void readChunk(long position, ByteBuffer uncompressed) CompressionMetadata.Chunk chunk = metadata.chunkFor(position); boolean shouldCheckCrc = shouldCheckCrc(); - int length = shouldCheckCrc ? chunk.length + Integer.BYTES // compressed length + checksum length - : chunk.length; + CompressedReader readFrom = (scanReader != null && scanReader.allocated()) ? scanReader : reader; if (chunk.length < maxCompressedLength) { - ByteBuffer compressed = bufferHolder.getBuffer(length); - - if (channel.read(compressed, chunk.offset) != length) - throw new CorruptBlockException(channel.filePath(), chunk); - - compressed.flip(); - compressed.limit(chunk.length); + ByteBuffer compressed = readFrom.read(chunk, shouldCheckCrc); uncompressed.clear(); - if (shouldCheckCrc) - { - int checksum = (int) ChecksumType.CRC32.of(compressed); - - compressed.limit(length); - if (compressed.getInt() != checksum) - throw new CorruptBlockException(channel.filePath(), chunk); - - compressed.position(0).limit(chunk.length); - } - try { metadata.compressor().uncompress(compressed, uncompressed); @@ -155,10 +291,9 @@ public void readChunk(long position, ByteBuffer uncompressed) uncompressed.flip(); int checksum = (int) ChecksumType.CRC32.of(uncompressed); - ByteBuffer scratch = bufferHolder.getBuffer(Integer.BYTES); - + ByteBuffer scratch = ByteBuffer.allocate(Integer.BYTES); if (channel.read(scratch, chunk.offset + chunk.length) != Integer.BYTES - || scratch.getInt(0) != checksum) + || scratch.getInt(0) != checksum) throw new CorruptBlockException(channel.filePath(), chunk); } } @@ -171,6 +306,16 @@ public void readChunk(long position, ByteBuffer uncompressed) throw new CorruptSSTableException(e, channel.filePath()); } } + + @Override + public void close() + { + reader.close(); + if (scanReader != null) + scanReader.close(); + + super.close(); + } } public static class Mmap extends CompressedChunkReader @@ -233,7 +378,6 @@ public void readChunk(long position, ByteBuffer uncompressed) uncompressed.position(0).limit(0); throw new CorruptSSTableException(e, channel.filePath()); } - } public void close() diff --git a/src/java/org/apache/cassandra/io/util/EmptyRebufferer.java b/src/java/org/apache/cassandra/io/util/EmptyRebufferer.java index aa8e7e046f39..7f54a6b180f2 100644 --- a/src/java/org/apache/cassandra/io/util/EmptyRebufferer.java +++ b/src/java/org/apache/cassandra/io/util/EmptyRebufferer.java @@ -64,7 +64,7 @@ public void closeReader() } @Override - public Rebufferer instantiateRebufferer() + public Rebufferer instantiateRebufferer(boolean isScan) { return this; } diff --git a/src/java/org/apache/cassandra/io/util/FileHandle.java b/src/java/org/apache/cassandra/io/util/FileHandle.java index 943355d01dd7..67bfd239d61e 100644 --- a/src/java/org/apache/cassandra/io/util/FileHandle.java +++ b/src/java/org/apache/cassandra/io/util/FileHandle.java @@ -134,6 +134,11 @@ public RandomAccessReader createReader() return createReader(null); } + public RandomAccessReader createReaderForScan() + { + return createReader(null, true); + } + /** * Create {@link RandomAccessReader} with configured method of reading content of the file. * Reading from file will be rate limited by given {@link RateLimiter}. @@ -143,7 +148,12 @@ public RandomAccessReader createReader() */ public RandomAccessReader createReader(RateLimiter limiter) { - return new RandomAccessReader(instantiateRebufferer(limiter)); + return createReader(limiter, false); + } + + public RandomAccessReader createReader(RateLimiter limiter, boolean forScan) + { + return new RandomAccessReader(instantiateRebufferer(limiter, forScan)); } public FileDataInput createReader(long position) @@ -186,7 +196,12 @@ public void dropPageCache(long before) public Rebufferer instantiateRebufferer(RateLimiter limiter) { - Rebufferer rebufferer = rebuffererFactory.instantiateRebufferer(); + return instantiateRebufferer(limiter, false); + } + + public Rebufferer instantiateRebufferer(RateLimiter limiter, boolean forScan) + { + Rebufferer rebufferer = rebuffererFactory.instantiateRebufferer(forScan); if (limiter != null) rebufferer = new LimitingRebufferer(rebufferer, limiter, DiskOptimizationStrategy.MAX_BUFFER_SIZE); diff --git a/src/java/org/apache/cassandra/io/util/MmapRebufferer.java b/src/java/org/apache/cassandra/io/util/MmapRebufferer.java index 8df6370c5e69..884bc9718642 100644 --- a/src/java/org/apache/cassandra/io/util/MmapRebufferer.java +++ b/src/java/org/apache/cassandra/io/util/MmapRebufferer.java @@ -41,7 +41,7 @@ public BufferHolder rebuffer(long position) } @Override - public Rebufferer instantiateRebufferer() + public Rebufferer instantiateRebufferer(boolean isScan) { return this; } diff --git a/src/java/org/apache/cassandra/io/util/RandomAccessReader.java b/src/java/org/apache/cassandra/io/util/RandomAccessReader.java index 3ce1a2eb0862..b89e59eb5291 100644 --- a/src/java/org/apache/cassandra/io/util/RandomAccessReader.java +++ b/src/java/org/apache/cassandra/io/util/RandomAccessReader.java @@ -332,7 +332,7 @@ public static RandomAccessReader open(File file) try { ChunkReader reader = new SimpleChunkReader(channel, -1, BufferType.OFF_HEAP, DEFAULT_BUFFER_SIZE); - Rebufferer rebufferer = reader.instantiateRebufferer(); + Rebufferer rebufferer = reader.instantiateRebufferer(false); return new RandomAccessReaderWithOwnChannel(rebufferer); } catch (Throwable t) diff --git a/src/java/org/apache/cassandra/io/util/RebuffererFactory.java b/src/java/org/apache/cassandra/io/util/RebuffererFactory.java index ec35f0ba530b..192fb8ea0cd8 100644 --- a/src/java/org/apache/cassandra/io/util/RebuffererFactory.java +++ b/src/java/org/apache/cassandra/io/util/RebuffererFactory.java @@ -28,5 +28,5 @@ */ public interface RebuffererFactory extends ReaderFileProxy { - Rebufferer instantiateRebufferer(); + Rebufferer instantiateRebufferer(boolean isScan); } diff --git a/src/java/org/apache/cassandra/io/util/SimpleChunkReader.java b/src/java/org/apache/cassandra/io/util/SimpleChunkReader.java index 8d00ce5d4000..fec1216bf4e4 100644 --- a/src/java/org/apache/cassandra/io/util/SimpleChunkReader.java +++ b/src/java/org/apache/cassandra/io/util/SimpleChunkReader.java @@ -55,7 +55,7 @@ public BufferType preferredBufferType() } @Override - public Rebufferer instantiateRebufferer() + public Rebufferer instantiateRebufferer(boolean forScan) { if (Integer.bitCount(bufferSize) == 1) return new BufferManagingRebufferer.Aligned(this); diff --git a/src/java/org/apache/cassandra/io/util/ThreadLocalReadAheadBuffer.java b/src/java/org/apache/cassandra/io/util/ThreadLocalReadAheadBuffer.java new file mode 100644 index 000000000000..824acaa8d88f --- /dev/null +++ b/src/java/org/apache/cassandra/io/util/ThreadLocalReadAheadBuffer.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.io.util; + +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; + +import io.netty.util.concurrent.FastThreadLocal; +import org.apache.cassandra.io.compress.BufferType; +import org.apache.cassandra.io.sstable.CorruptSSTableException; + +public final class ThreadLocalReadAheadBuffer +{ + private static class Block + { + ByteBuffer buffer = null; + int index = -1; + } + + private final ChannelProxy channel; + + private final BufferType bufferType; + + private static final FastThreadLocal<Map<String, Block>> blockMap = new FastThreadLocal<>() + { + @Override + protected Map<String, Block> initialValue() + { + return new HashMap<>(); + } + }; + + private final int bufferSize; + private final long channelSize; + + public ThreadLocalReadAheadBuffer(ChannelProxy channel, int bufferSize, BufferType bufferType) + { + this.channel = channel; + this.channelSize = channel.size(); + this.bufferSize = bufferSize; + this.bufferType = bufferType; + } + + public boolean hasBuffer() + { + return block().buffer != null; + } + + /** + * Safe to call only if {@link #hasBuffer()} is true + */ + public int remaining() + { + return getBlock().buffer.remaining(); + } + + public void allocateBuffer() + { + getBlock(); + } + + private Block getBlock() + { + Block block = block(); + if (block.buffer == null) + { + block.buffer = bufferType.allocate(bufferSize); + block.buffer.clear(); + } + return block; + } + + private Block block() + { + return blockMap.get().computeIfAbsent(channel.filePath(), k -> new Block()); + } + + public void fill(long position) + { + Block block = getBlock(); + ByteBuffer blockBuffer = block.buffer; + long realPosition = Math.min(channelSize, position); + int blockNo = (int) (realPosition / bufferSize); + long blockPosition = blockNo * (long) bufferSize; + + long remaining = channelSize - blockPosition; + int sizeToRead = (int) Math.min(remaining, bufferSize); + if (block.index != blockNo) + { + blockBuffer.flip(); + blockBuffer.limit(sizeToRead); + if (channel.read(blockBuffer, blockPosition) != sizeToRead) + throw new CorruptSSTableException(null, channel.filePath()); + + block.index = blockNo; + } + + blockBuffer.flip(); + blockBuffer.limit(sizeToRead); + blockBuffer.position((int) (realPosition - blockPosition)); + } + + public int read(ByteBuffer dest, int length) + { + Block block = getBlock(); + ByteBuffer blockBuffer = block.buffer; + ByteBuffer tmp = blockBuffer.duplicate(); + tmp.limit(tmp.position() + length); + dest.put(tmp); + blockBuffer.position(blockBuffer.position() + length); + + return length; + } + + public void clear(boolean deallocate) + { + Block block = getBlock(); + block.index = -1; + + ByteBuffer blockBuffer = block.buffer; + if (blockBuffer != null) + { + blockBuffer.clear(); + if (deallocate) + { + FileUtils.clean(blockBuffer); + block.buffer = null; + } + } + } + + public void close() + { + clear(true); + blockMap.get().remove(channel.filePath()); + } +} diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 5d90bcb57203..53f815a586db 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -1430,6 +1430,19 @@ public Map<String, String> getCurrentCompactionThroughputMebibytesPerSec() return result; } + @Override + public int getCompressedReadAheadBufferInKB() + { + return DatabaseDescriptor.getCompressedReadAheadBufferSizeInKB(); + } + + @Override + public void setCompressedReadAheadBufferInKB(int sizeInKb) + { + DatabaseDescriptor.setCompressedReadAheadBufferSizeInKb(sizeInKb); + logger.info("set compressed read ahead buffer size to {}KiB", sizeInKb); + } + public int getBatchlogReplayThrottleInKB() { return DatabaseDescriptor.getBatchlogReplayThrottleInKiB(); diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java index bfb9eb97b728..7760ea01883e 100644 --- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java +++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java @@ -862,6 +862,9 @@ default int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, public void setCompactionThroughputMbPerSec(int value); Map<String, String> getCurrentCompactionThroughputMebibytesPerSec(); + public int getCompressedReadAheadBufferInKB(); + public void setCompressedReadAheadBufferInKB(int sizeInKb); + public int getBatchlogReplayThrottleInKB(); public void setBatchlogReplayThrottleInKB(int value); diff --git a/test/unit/org/apache/cassandra/io/util/CompressedChunkReaderTest.java b/test/unit/org/apache/cassandra/io/util/CompressedChunkReaderTest.java new file mode 100644 index 000000000000..af4b458fecc4 --- /dev/null +++ b/test/unit/org/apache/cassandra/io/util/CompressedChunkReaderTest.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.io.util; + +import accord.utils.Gen; +import accord.utils.Gens; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ClusteringComparator; +import org.apache.cassandra.io.compress.CompressedSequentialWriter; +import org.apache.cassandra.io.compress.CompressionMetadata; +import org.apache.cassandra.io.filesystem.ListenableFileSystem; +import org.apache.cassandra.io.sstable.metadata.MetadataCollector; +import org.apache.cassandra.schema.CompressionParams; +import org.assertj.core.api.Assertions; + +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.nio.file.Files; +import java.util.concurrent.atomic.AtomicInteger; + +import static accord.utils.Property.qt; + +public class CompressedChunkReaderTest +{ + static + { + DatabaseDescriptor.clientInitialization(); + } + + @Test + public void scanReaderReadsLessThanRAReader() + { + var optionGen = options(); + var paramsGen = params(); + var lengthGen = Gens.longs().between(1, 1 << 16); + qt().withSeed(-1871070464864118891L).forAll(Gens.random(), optionGen, paramsGen).check((rs, option, params) -> { + ListenableFileSystem fs = FileSystems.newGlobalInMemoryFileSystem(); + + File f = new File("/file.db"); + AtomicInteger reads = new AtomicInteger(); + fs.onPostRead(f.path::equals, (p, c, pos, dst, r) -> { + reads.incrementAndGet(); + }); + long length = lengthGen.nextLong(rs); + CompressionMetadata metadata1, metadata2; + try (CompressedSequentialWriter writer = new CompressedSequentialWriter(f, new File("/file.offset"), new File("/file.digest"), option, params, new MetadataCollector(new ClusteringComparator()))) + { + for (long i = 0; i < length; i++) + writer.writeLong(i); + + writer.sync(); + metadata1 = writer.open(0); + metadata2 = writer.open(0); + } + + doReads(f, metadata1, length, true); + int scanReads = reads.getAndSet(0); + + doReads(f, metadata2, length, false); + int raReads = reads.getAndSet(0); + + if (Files.size(f.toPath()) > DatabaseDescriptor.getCompressedReadAheadBufferSize()) + Assert.assertTrue(scanReads < raReads); + }); + } + + private void doReads(File f, CompressionMetadata metadata, long length, boolean useReadAhead) + { + ByteBuffer buffer = ByteBuffer.allocateDirect(metadata.chunkLength()); + + try (ChannelProxy channel = new ChannelProxy(f); + CompressedChunkReader reader = new CompressedChunkReader.Standard(channel, metadata, () -> 1.1); + metadata) + { + if (useReadAhead) + reader.forScan(); + + long offset = 0; + long maxOffset = length * Long.BYTES; + do + { + reader.readChunk(offset, buffer); + for (long expected = offset / Long.BYTES; buffer.hasRemaining(); expected++) + Assertions.assertThat(buffer.getLong()).isEqualTo(expected); + + offset += metadata.chunkLength(); + } + while (offset < maxOffset); + } + finally + { + FileUtils.clean(buffer); + }} + + private static Gen<SequentialWriterOption> options() + { + Gen<Integer> bufferSizes = Gens.constant(1 << 10); //.pickInt(1 << 4, 1 << 10, 1 << 15); + return rs -> SequentialWriterOption.newBuilder() + .finishOnClose(false) + .bufferSize(bufferSizes.next(rs)) + .build(); + } + + private enum CompressionKind { Noop, Snappy, Deflate, Lz4, Zstd } + + private static Gen<CompressionParams> params() + { + Gen<Integer> chunkLengths = Gens.constant(CompressionParams.DEFAULT_CHUNK_LENGTH); + Gen<Double> compressionRatio = Gens.pick(1.1D); + return rs -> { + CompressionKind kind = rs.pick(CompressionKind.values()); + switch (kind) + { + case Noop: return CompressionParams.noop(); + case Snappy: return CompressionParams.snappy(chunkLengths.next(rs), compressionRatio.next(rs)); + case Deflate: return CompressionParams.deflate(chunkLengths.next(rs)); + case Lz4: return CompressionParams.lz4(chunkLengths.next(rs)); + case Zstd: return CompressionParams.zstd(chunkLengths.next(rs)); + default: throw new UnsupportedOperationException(kind.name()); + } + }; + } +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/io/util/ThreadLocalReadAheadBufferTest.java b/test/unit/org/apache/cassandra/io/util/ThreadLocalReadAheadBufferTest.java new file mode 100644 index 000000000000..4d43017b2a5a --- /dev/null +++ b/test/unit/org/apache/cassandra/io/util/ThreadLocalReadAheadBufferTest.java @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.io.util; + + +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.file.Files; +import java.util.List; +import java.util.Random; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.DataStorageSpec; +import org.apache.cassandra.io.compress.BufferType; +import org.apache.cassandra.io.sstable.CorruptSSTableException; +import org.apache.cassandra.utils.Pair; +import org.quicktheories.WithQuickTheories; +import org.quicktheories.core.Gen; + +import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_IO_TMPDIR; + +public class ThreadLocalReadAheadBufferTest implements WithQuickTheories +{ + private static final int numFiles = 5; + private static final File[] files = new File[numFiles]; + private static final Logger logger = LoggerFactory.getLogger(ThreadLocalReadAheadBufferTest.class); + + @BeforeClass + public static void setup() + { + int seed = new Random().nextInt(); + logger.info("Seed: {}", seed); + + for (int i = 0; i < numFiles; i++) + { + int size = new Random().nextInt((Integer.MAX_VALUE - 1) / 8); + files[i] = writeFile(seed, size); + } + } + + @AfterClass + public static void cleanup() + { + for (File f : files) + { + try + { + f.delete(); + } + catch (Exception e) + { + // ignore + } + } + } + + @Test + public void testLastBlockReads() + { + qt().forAll(lastBlockReads()) + .checkAssert(this::testReads); + } + + @Test + public void testReadsLikeChannelProxy() + { + + qt().forAll(randomReads()) + .checkAssert(this::testReads); + } + + private void testReads(InputData propertyInputs) + { + try (ChannelProxy channel = new ChannelProxy(propertyInputs.file)) + { + ThreadLocalReadAheadBuffer trlab = new ThreadLocalReadAheadBuffer(channel, new DataStorageSpec.IntKibibytesBound("256KiB").toBytes(), BufferType.OFF_HEAP); + for (Pair<Long, Integer> read : propertyInputs.positionsAndLengths) + { + int readSize = Math.min(read.right,(int) (channel.size() - read.left)); + ByteBuffer buf1 = ByteBuffer.allocate(readSize); + channel.read(buf1, read.left); + + ByteBuffer buf2 = ByteBuffer.allocate(readSize); + try + { + int copied = 0; + while (copied < readSize) { + trlab.fill(read.left + copied); + int leftToRead = readSize - copied; + if (trlab.remaining() >= leftToRead) + copied += trlab.read(buf2, leftToRead); + else + copied += trlab.read(buf2, trlab.remaining()); + } + } + catch (CorruptSSTableException e) + { + throw new RuntimeException(e); + } + + Assert.assertEquals(buf1, buf2); + } + } + } + + private Gen<InputData> lastBlockReads() + { + return arbitrary().pick(List.of(files)) + .flatMap((file) -> + lists().of(longs().between(0, fileSize(file)).zip(integers().between(1, 100), Pair::create)) + .ofSizeBetween(5, 10) + .map(positionsAndLengths -> new InputData(file, positionsAndLengths))); + + } + + private Gen<InputData> randomReads() + { + int blockSize = new DataStorageSpec.IntKibibytesBound("256KiB").toBytes(); + return arbitrary().pick(List.of(files)) + .flatMap((file) -> + lists().of(longs().between(fileSize(file) - blockSize, fileSize(file)).zip(integers().between(1, 100), Pair::create)) + .ofSizeBetween(5, 10) + .map(positionsAndLengths -> new InputData(file, positionsAndLengths))); + + } + + // need this becasue generators don't handle the IOException + private long fileSize(File file) + { + try + { + return Files.size(file.toPath()); + } catch (IOException e) + { + throw new RuntimeException(e); + } + } + + private static class InputData + { + + private final File file; + private final List<Pair<Long, Integer>> positionsAndLengths; + + public InputData(File file, List<Pair<Long, Integer>> positionsAndLengths) + { + this.file = file; + this.positionsAndLengths = positionsAndLengths; + } + } + + private static File writeFile(int seed, int length) + { + String fileName = JAVA_IO_TMPDIR.getString() + "data+" + length + ".bin"; + + byte[] dataChunk = new byte[4096 * 8]; + java.util.Random random = new Random(seed); + int writtenData = 0; + + File file = new File(fileName); + try (FileOutputStream fos = new FileOutputStream(file.toJavaIOFile())) + { + while (writtenData < length) + { + random.nextBytes(dataChunk); + int toWrite = Math.min((length - writtenData), dataChunk.length); + fos.write(dataChunk, 0, toWrite); + writtenData += toWrite; + } + fos.flush(); + } + catch (IOException e) + { + throw new RuntimeException(e); + } + + return file; + } + +} From 19007f374cf516557ed9a63e9d0a0db69b559369 Mon Sep 17 00:00:00 2001 From: Andy Tolbert <6889771+tolbertam@users.noreply.github.com> Date: Thu, 13 Feb 2025 14:48:47 -0600 Subject: [PATCH 197/225] Remove remaining driver dependencies from server code patch by Andy Tolbert; reviewed by Stefan Miklosovic for CASSANDRA-20327 --- .../org/apache/cassandra/cql3/UntypedResultSet.java | 9 +++++++-- .../apache/cassandra/cql3/functions/UDFunction.java | 6 ------ .../cassandra/security/ThreadAwareSecurityManager.java | 3 +-- .../datastax/driver/core/PreparedStatementHelper.java | 10 ---------- 4 files changed, 8 insertions(+), 20 deletions(-) rename {src/java => test/unit}/com/datastax/driver/core/PreparedStatementHelper.java (91%) diff --git a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java index a0201c500a39..f82ff3eb835e 100644 --- a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java +++ b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java @@ -33,7 +33,6 @@ import com.google.common.annotations.VisibleForTesting; -import com.datastax.driver.core.CodecUtils; import org.apache.cassandra.cql3.functions.types.LocalDate; import org.apache.cassandra.cql3.statements.SelectStatement; import org.apache.cassandra.db.Clustering; @@ -442,7 +441,13 @@ public Date getTimestamp(String column) return TimestampType.instance.compose(data.get(column)); } - public LocalDate getDate(String column) { return LocalDate.fromDaysSinceEpoch(CodecUtils.fromUnsignedToSignedInt(data.get(column).getInt()));} + public LocalDate getDate(String column) + { + // date type is stored as an unsigned byte; convert it back by adding MIN_VALUE. + int unsigned = data.get(column).getInt(); + int signed = unsigned + Integer.MIN_VALUE; + return LocalDate.fromDaysSinceEpoch(signed); + } public long getLong(String column) { diff --git a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java index 39f8f9b7fa9e..ded2a1589e53 100644 --- a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java +++ b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java @@ -134,12 +134,6 @@ public abstract class UDFunction extends UserFunction implements ScalarFunction // Only need to disallow a pattern, if it would otherwise be allowed via allowedPatterns private static final String[] disallowedPatterns = { - "com/datastax/driver/core/Cluster.class", - "com/datastax/driver/core/Metrics.class", - "com/datastax/driver/core/NettyOptions.class", - "com/datastax/driver/core/Session.class", - "com/datastax/driver/core/Statement.class", - "com/datastax/driver/core/TimestampGenerator.class", // indirectly covers ServerSideTimestampGenerator + ThreadLocalMonotonicTimestampGenerator "java/lang/Compiler.class", "java/lang/InheritableThreadLocal.class", "java/lang/Package.class", diff --git a/src/java/org/apache/cassandra/security/ThreadAwareSecurityManager.java b/src/java/org/apache/cassandra/security/ThreadAwareSecurityManager.java index 8f86831cc337..65793a63a128 100644 --- a/src/java/org/apache/cassandra/security/ThreadAwareSecurityManager.java +++ b/src/java/org/apache/cassandra/security/ThreadAwareSecurityManager.java @@ -41,8 +41,6 @@ * Custom {@link SecurityManager} and {@link Policy} implementation that only performs access checks * if explicitly enabled. * <p> - * This implementation gives no measurable performance penalty - * (see <a href="http://cstar.datastax.com/tests/id/1d461628-12ba-11e5-918f-42010af0688f">see cstar test</a>). * This is better than the penalty of 1 to 3 percent using a standard {@code SecurityManager} with an <i>allow all</i> policy. * </p> */ @@ -216,6 +214,7 @@ public void checkPermission(Permission perm) // required by JavaDriver 2.2.0-rc3 and 3.0.0-a2 or newer // code in com.datastax.driver.core.CodecUtils uses Guava stuff, which in turns requires this permission + // TODO: Evaluate removing this once the driver is removed as a dependency (see CASSANDRA-20326). if (CHECK_MEMBER_ACCESS_PERMISSION.equals(perm)) return; diff --git a/src/java/com/datastax/driver/core/PreparedStatementHelper.java b/test/unit/com/datastax/driver/core/PreparedStatementHelper.java similarity index 91% rename from src/java/com/datastax/driver/core/PreparedStatementHelper.java rename to test/unit/com/datastax/driver/core/PreparedStatementHelper.java index 3df33fb0ec72..26e0408e9f27 100644 --- a/src/java/com/datastax/driver/core/PreparedStatementHelper.java +++ b/test/unit/com/datastax/driver/core/PreparedStatementHelper.java @@ -41,16 +41,6 @@ private static MD5Digest id(PreparedStatement statement) return statement.getPreparedId().boundValuesMetadata.id; } - public static void assertStable(PreparedStatement first, PreparedStatement subsequent) - { - if (!id(first).equals(id(subsequent))) - { - throw new AssertionError(String.format("Subsequent id (%s) is different from the first one (%s)", - id(first), - id(subsequent))); - } - } - public static void assertHashWithoutKeyspace(PreparedStatement statement, String queryString, String ks) { MD5Digest returned = id(statement); From 779bae02c8c9902c36ac0fca47610c5704abb357 Mon Sep 17 00:00:00 2001 From: David Capwell <dcapwell@apache.org> Date: Mon, 17 Mar 2025 14:07:00 -0700 Subject: [PATCH 198/225] Add LIMIT and PER PARTITION LIMIT to the AST SingleNodeTableWalkTest patch by David Capwell; reviewed by Abe Ratnofsky for CASSANDRA-20399 --- .../AbstractCompactionStrategy.java | 4 +- .../compaction/LeveledCompactionStrategy.java | 6 +- .../SizeTieredCompactionStrategyOptions.java | 6 +- .../db/compaction/unified/Controller.java | 18 +- .../test/cql3/SingleNodeTableWalkTest.java | 7 +- .../test/cql3/StatefulASTBase.java | 53 ++- .../harry/model/ASTSingleTableModel.java | 39 +- .../harry/model/ASTSingleTableModelTest.java | 102 ++++- .../org/apache/cassandra/cql3/ast/Select.java | 58 ++- .../cassandra/utils/CassandraGenerators.java | 361 +++++++++++++++++- 10 files changed, 615 insertions(+), 39 deletions(-) diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java index 9d189b9975ce..99a509c5f2a8 100644 --- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java +++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java @@ -72,10 +72,10 @@ public abstract class AbstractCompactionStrategy protected static final boolean DEFAULT_UNCHECKED_TOMBSTONE_COMPACTION_OPTION = false; protected static final boolean DEFAULT_LOG_ALL_OPTION = false; - protected static final String TOMBSTONE_THRESHOLD_OPTION = "tombstone_threshold"; + public static final String TOMBSTONE_THRESHOLD_OPTION = "tombstone_threshold"; protected static final String TOMBSTONE_COMPACTION_INTERVAL_OPTION = "tombstone_compaction_interval"; // disable range overlap check when deciding if an SSTable is candidate for tombstone compaction (CASSANDRA-6563) - protected static final String UNCHECKED_TOMBSTONE_COMPACTION_OPTION = "unchecked_tombstone_compaction"; + public static final String UNCHECKED_TOMBSTONE_COMPACTION_OPTION = "unchecked_tombstone_compaction"; protected static final String LOG_ALL_OPTION = "log_all"; protected static final String COMPACTION_ENABLED = "enabled"; public static final String ONLY_PURGE_REPAIRED_TOMBSTONES = "only_purge_repaired_tombstones"; diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java index ff90219224c5..a68efa120c18 100644 --- a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java +++ b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java @@ -47,10 +47,10 @@ public class LeveledCompactionStrategy extends AbstractCompactionStrategy { private static final Logger logger = LoggerFactory.getLogger(LeveledCompactionStrategy.class); - private static final String SSTABLE_SIZE_OPTION = "sstable_size_in_mb"; + public static final String SSTABLE_SIZE_OPTION = "sstable_size_in_mb"; private static final boolean tolerateSstableSize = TOLERATE_SSTABLE_SIZE.getBoolean(); - private static final String LEVEL_FANOUT_SIZE_OPTION = "fanout_size"; - private static final String SINGLE_SSTABLE_UPLEVEL_OPTION = "single_sstable_uplevel"; + public static final String LEVEL_FANOUT_SIZE_OPTION = "fanout_size"; + public static final String SINGLE_SSTABLE_UPLEVEL_OPTION = "single_sstable_uplevel"; public static final int DEFAULT_LEVEL_FANOUT_SIZE = 10; @VisibleForTesting diff --git a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyOptions.java b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyOptions.java index eb1d8f97afe2..f95a19bc021c 100644 --- a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyOptions.java +++ b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyOptions.java @@ -26,9 +26,9 @@ public final class SizeTieredCompactionStrategyOptions protected static final long DEFAULT_MIN_SSTABLE_SIZE = 50L * 1024L * 1024L; protected static final double DEFAULT_BUCKET_LOW = 0.5; protected static final double DEFAULT_BUCKET_HIGH = 1.5; - protected static final String MIN_SSTABLE_SIZE_KEY = "min_sstable_size"; - protected static final String BUCKET_LOW_KEY = "bucket_low"; - protected static final String BUCKET_HIGH_KEY = "bucket_high"; + public static final String MIN_SSTABLE_SIZE_KEY = "min_sstable_size"; + public static final String BUCKET_LOW_KEY = "bucket_low"; + public static final String BUCKET_HIGH_KEY = "bucket_high"; protected long minSSTableSize; protected double bucketLow; diff --git a/src/java/org/apache/cassandra/db/compaction/unified/Controller.java b/src/java/org/apache/cassandra/db/compaction/unified/Controller.java index df2a66590c30..28684aa59645 100644 --- a/src/java/org/apache/cassandra/db/compaction/unified/Controller.java +++ b/src/java/org/apache/cassandra/db/compaction/unified/Controller.java @@ -50,7 +50,7 @@ public class Controller * The scaling parameters W, one per bucket index and separated by a comma. * Higher indexes will use the value of the last index with a W specified. */ - final static String SCALING_PARAMETERS_OPTION = "scaling_parameters"; + public final static String SCALING_PARAMETERS_OPTION = "scaling_parameters"; private final static String DEFAULT_SCALING_PARAMETERS = CassandraRelevantProperties.UCS_SCALING_PARAMETER.getString(); @@ -58,7 +58,7 @@ public class Controller * The minimum sstable size. Sharded writers split sstables over shard only if they are at least as large as the * minimum size. */ - static final String MIN_SSTABLE_SIZE_OPTION = "min_sstable_size"; + public static final String MIN_SSTABLE_SIZE_OPTION = "min_sstable_size"; private static final String DEFAULT_MIN_SSTABLE_SIZE = CassandraRelevantProperties.UCS_MIN_SSTABLE_SIZE.getString(); @@ -66,9 +66,9 @@ public class Controller * Override for the flush size in MB. The database should be able to calculate this from executing flushes, this * should only be necessary in rare cases. */ - static final String FLUSH_SIZE_OVERRIDE_OPTION = "flush_size_override"; + public static final String FLUSH_SIZE_OVERRIDE_OPTION = "flush_size_override"; - static final String BASE_SHARD_COUNT_OPTION = "base_shard_count"; + public static final String BASE_SHARD_COUNT_OPTION = "base_shard_count"; /** * Default base shard count, used when a base count is not explicitly supplied. This value applies as long as the * table is not a system one, and directories are not defined. @@ -79,10 +79,10 @@ public class Controller public static final int DEFAULT_BASE_SHARD_COUNT = CassandraRelevantProperties.UCS_BASE_SHARD_COUNT.getInt(); - static final String TARGET_SSTABLE_SIZE_OPTION = "target_sstable_size"; + public static final String TARGET_SSTABLE_SIZE_OPTION = "target_sstable_size"; public static final long DEFAULT_TARGET_SSTABLE_SIZE = CassandraRelevantProperties.UCS_TARGET_SSTABLE_SIZE.getSizeInBytes(); - static final long MIN_TARGET_SSTABLE_SIZE = 1L << 20; + public static final long MIN_TARGET_SSTABLE_SIZE = 1L << 20; /** * Provision for growth of the constructed SSTables as the size of the data grows. By default, the target SSTable @@ -109,7 +109,7 @@ public class Controller * base count of 4, the number of SSTables will be 4 (~256GiB each) for a growth value of 1, 128 (~8GiB each) for * a growth value of 0.333, and 64 (~16GiB each) for a growth value of 0.5. */ - static final String SSTABLE_GROWTH_OPTION = "sstable_growth"; + public static final String SSTABLE_GROWTH_OPTION = "sstable_growth"; private static final double DEFAULT_SSTABLE_GROWTH = CassandraRelevantProperties.UCS_SSTABLE_GROWTH.getDouble(); /** @@ -127,7 +127,7 @@ public class Controller * * If the fanout factor is larger than the maximum number of sstables, the strategy will ignore the latter. */ - static final String MAX_SSTABLES_TO_COMPACT_OPTION = "max_sstables_to_compact"; + public static final String MAX_SSTABLES_TO_COMPACT_OPTION = "max_sstables_to_compact"; static final String ALLOW_UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION_OPTION = "unsafe_aggressive_sstable_expiration"; static final boolean ALLOW_UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION = @@ -147,7 +147,7 @@ public class Controller * that overlap with participating (LCS-like, higher concurrency during upgrades but some double compaction), * TRANSITIVE to include overlaps of overlaps (likely to trigger whole level compactions, safest). */ - static final String OVERLAP_INCLUSION_METHOD_OPTION = "overlap_inclusion_method"; + public static final String OVERLAP_INCLUSION_METHOD_OPTION = "overlap_inclusion_method"; static final Overlaps.InclusionMethod DEFAULT_OVERLAP_INCLUSION_METHOD = CassandraRelevantProperties.UCS_OVERLAP_INCLUSION_METHOD.getEnum(Overlaps.InclusionMethod.TRANSITIVE); diff --git a/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTableWalkTest.java b/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTableWalkTest.java index c18d6cfecfbf..2ba02ae76940 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTableWalkTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTableWalkTest.java @@ -130,7 +130,7 @@ protected List<CreateIndexDDL.Indexer> supportedIndexers() } } Select select = builder.build(); - return state.command(rs, select, (wholePartition ? "Whole Partition" : "Single Row")); + return state.command(rs, select, (wholePartition ? "By Partition Key" : "By Primary Key")); } public Property.Command<State, Void, ?> selectToken(RandomSource rs, State state) @@ -357,7 +357,10 @@ protected TableMetadata defineTable(RandomSource rs, String ks) //TODO (coverage): partition is defined at the cluster level, so have to hard code in this model as the table is changed rather than cluster being recreated... this limits coverage return toGen(new TableMetadataBuilder() .withTableKinds(TableMetadata.Kind.REGULAR) - .withKnownMemtables() + .withParams(b -> b.withKnownMemtables() + .withCaching() + .withCompaction() + .withCompression()) .withKeyspaceName(ks).withTableName("tbl") .withSimpleColumnNames() .withDefaultTypeGen(supportedTypes()) diff --git a/test/distributed/org/apache/cassandra/distributed/test/cql3/StatefulASTBase.java b/test/distributed/org/apache/cassandra/distributed/test/cql3/StatefulASTBase.java index ac5df71ddf97..ec530b97fdc5 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/cql3/StatefulASTBase.java +++ b/test/distributed/org/apache/cassandra/distributed/test/cql3/StatefulASTBase.java @@ -99,8 +99,7 @@ public class StatefulASTBase extends TestBaseImpl */ protected static boolean CQL_DEBUG_APPLY_OPERATOR = false; - protected static final Gen<Gen<Boolean>> BIND_OR_LITERAL_DISTRO = Gens.bools().mixedDistribution(); - protected static final Gen<Gen<Boolean>> BETWEEN_EQ_DISTRO = Gens.bools().mixedDistribution(); + protected static final Gen<Gen<Boolean>> BOOL_DISTRIBUTION = Gens.bools().mixedDistribution(); protected static final Gen<Gen<Conditional.Where.Inequality>> LESS_THAN_DISTRO = Gens.mixedDistribution(Stream.of(Conditional.Where.Inequality.values()) .filter(i -> i == Conditional.Where.Inequality.LESS_THAN || i == Conditional.Where.Inequality.LESS_THAN_EQ) .collect(Collectors.toList())); @@ -111,6 +110,7 @@ public class StatefulASTBase extends TestBaseImpl .filter(i -> i != Conditional.Where.Inequality.EQUAL && i != Conditional.Where.Inequality.NOT_EQUAL) .collect(Collectors.toList())); protected static final Gen<Gen.IntGen> FETCH_SIZE_DISTRO = Gens.mixedDistribution(new int[] {1, 10, 100, 1000, 5000}); + protected static final Gen<Gen.IntGen> LIMIT_DISTRO = Gens.mixedDistribution(1, 1001); static { @@ -196,6 +196,8 @@ protected static abstract class BaseState implements AutoCloseable protected final Session session; protected final Gen<Boolean> bindOrLiteralGen; protected final Gen<Boolean> betweenEqGen; + protected final Gen<Boolean> useFetchSizeGen, usePerPartitionLimitGen, useLimitGen; + protected final Gen.IntGen perPartitionLimitGen, limitGen; protected final Gen<Conditional.Where.Inequality> lessThanGen; protected final Gen<Conditional.Where.Inequality> greaterThanGen; protected final Gen<Conditional.Where.Inequality> rangeInequalityGen; @@ -221,12 +223,17 @@ protected BaseState(RandomSource rs, Cluster cluster, TableMetadata metadata) this.debug = CQL_DEBUG_APPLY_OPERATOR ? CompositeVisitor.of(StandardVisitors.APPLY_OPERATOR, StandardVisitors.DEBUG) : StandardVisitors.DEBUG; - this.bindOrLiteralGen = BIND_OR_LITERAL_DISTRO.next(rs); - this.betweenEqGen = BETWEEN_EQ_DISTRO.next(rs); + this.bindOrLiteralGen = BOOL_DISTRIBUTION.next(rs); + this.betweenEqGen = BOOL_DISTRIBUTION.next(rs); this.lessThanGen = LESS_THAN_DISTRO.next(rs); this.greaterThanGen = GREATER_THAN_DISTRO.next(rs); this.rangeInequalityGen = RANGE_INEQUALITY_DISTRO.next(rs); this.fetchSizeGen = FETCH_SIZE_DISTRO.next(rs); + this.useFetchSizeGen = BOOL_DISTRIBUTION.next(rs); + this.usePerPartitionLimitGen = BOOL_DISTRIBUTION.next(rs); + this.useLimitGen = BOOL_DISTRIBUTION.next(rs); + this.perPartitionLimitGen = LIMIT_DISTRO.next(rs); + this.limitGen = LIMIT_DISTRO.next(rs); this.enoughMemtables = rs.pickInt(3, 10, 50); this.enoughSSTables = rs.pickInt(3, 10, 50); @@ -260,18 +267,50 @@ private String createKeyspaceCQL(String ks) return command(rs, select, null); } + protected boolean allowLimit(Select select) + { + //TODO (coverage): allow this in the model! + // LIMIT with IN clause on partition columns is non-deterministic which is not currently supported by the model + if (select.where.isEmpty()) return true; + return !select.where.get() + .streamRecursive(true) + .filter(e -> e instanceof Conditional.In) + .anyMatch(e -> { + var in = (Conditional.In) e; + // when expression is size 1, then this is deterministic + if (in.expressions.size() == 1) return false; + return model.factory.partitionColumns.contains(in.ref); + }); + } + + protected boolean allowPerPartitionLimit(Select select) + { + return true; + } + + protected boolean allowPaging(Select select) + { + return true; + } + protected <S extends BaseState> Property.Command<S, Void, ?> command(RandomSource rs, Select select, @Nullable String annotate) { var inst = selectInstance(rs); - //TODO (coverage): don't limit this to all selects, only those doing range queries! - int fetchSize = fetchSizeGen.nextInt(rs); + if (allowPerPartitionLimit(select) && usePerPartitionLimitGen.next(rs)) + select = select.withPerPartitionLimit(perPartitionLimitGen.nextInt(rs)); + if (allowLimit(select) && useLimitGen.next(rs)) + select = select.withLimit(limitGen.nextInt(rs)); + int fetchSize = allowPaging(select) && useFetchSizeGen.next(rs) + ? fetchSizeGen.nextInt(rs) + : Integer.MAX_VALUE; String postfix = "on " + inst; if (fetchSize != Integer.MAX_VALUE) postfix += ", fetch size " + fetchSize; if (annotate == null) annotate = postfix; else annotate += ", " + postfix; + Select finalSelect = select; return new Property.SimpleCommand<>(humanReadable(select, annotate), s -> { - s.model.validate(s.executeQuery(inst, fetchSize, s.selectCl(), select), select); + s.model.validate(s.executeQuery(inst, fetchSize, s.selectCl(), finalSelect), finalSelect); }); } diff --git a/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModel.java b/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModel.java index 13180c061bf7..4c0a5f5cf51a 100644 --- a/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModel.java +++ b/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModel.java @@ -29,6 +29,7 @@ import java.util.Map; import java.util.NavigableSet; import java.util.Objects; +import java.util.Optional; import java.util.Set; import java.util.TreeMap; import java.util.function.IntFunction; @@ -52,9 +53,11 @@ import org.apache.cassandra.cql3.ast.Select; import org.apache.cassandra.cql3.ast.StandardVisitors; import org.apache.cassandra.cql3.ast.Symbol; +import org.apache.cassandra.cql3.ast.Value; import org.apache.cassandra.db.BufferClustering; import org.apache.cassandra.db.Clustering; import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.Int32Type; import org.apache.cassandra.dht.Token; import org.apache.cassandra.harry.model.BytesPartitionState.PrimaryKey; import org.apache.cassandra.harry.util.StringUtils; @@ -572,7 +575,7 @@ private SelectResult(ByteBuffer[][] rows, boolean unordered) private SelectResult getRowsAsByteBuffer(Select select) { if (select.where.isEmpty()) - return all(); + return new SelectResult(getRowsAsByteBuffer(applyLimits(all(), select.perPartitionLimit, select.limit)), false); LookupContext ctx = context(select); List<PrimaryKey> primaryKeys; if (ctx.unmatchable) @@ -598,11 +601,41 @@ else if (ctx.tokenLowerBound != null || ctx.tokenUpperBound != null) // partial tested (handles many columns, tests are single column) primaryKeys = search(ctx); } + primaryKeys = applyLimits(primaryKeys, select.perPartitionLimit, select.limit); //TODO (correctness): now that we have the rows we need to handle the selections/aggregation/limit/group-by/etc. return new SelectResult(getRowsAsByteBuffer(primaryKeys), ctx.unordered); } - private SelectResult all() + private List<PrimaryKey> applyLimits(List<PrimaryKey> primaryKeys, Optional<Value> perPartitionLimitOpt, Optional<Value> limitOpt) + { + if (perPartitionLimitOpt.isPresent()) + { + int limit = Int32Type.instance.compose(eval(perPartitionLimitOpt.get())); + var it = primaryKeys.iterator(); + BytesPartitionState.Ref current = null; + int count = 0; + while (it.hasNext()) + { + PrimaryKey next = it.next(); + if (current == null || !current.equals(next.partition)) + { + current = next.partition; + count = 0; + } + if (++count > limit) + it.remove(); + } + } + if (limitOpt.isPresent()) + { + int limit = Int32Type.instance.compose(eval(limitOpt.get())); + if (primaryKeys.size() > limit) + primaryKeys = primaryKeys.subList(0, limit); + } + return primaryKeys; + } + + private List<PrimaryKey> all() { List<PrimaryKey> primaryKeys = new ArrayList<>(); for (var partition : partitions.values()) @@ -610,7 +643,7 @@ private SelectResult all() if (partition.staticOnly()) primaryKeys.add(partition.partitionRowRef()); else partition.rows().stream().map(BytesPartitionState.Row::ref).forEach(primaryKeys::add); } - return new SelectResult(getRowsAsByteBuffer(primaryKeys), false); + return primaryKeys; } public ByteBuffer[][] getRowsAsByteBuffer(List<PrimaryKey> primaryKeys) diff --git a/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModelTest.java b/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModelTest.java index af0aa22de904..16f4d6681934 100644 --- a/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModelTest.java +++ b/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModelTest.java @@ -26,6 +26,7 @@ import java.util.TreeMap; import java.util.function.BooleanSupplier; import java.util.function.Consumer; +import java.util.function.Function; import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -83,8 +84,36 @@ public void singlePartition() Select.Builder builder = Select.builder().table(metadata); for (var pk : metadata.partitionKeyColumns()) builder.value(new Symbol(pk), ZERO); - Select select = builder.build(); - model.validate(expected, select); + + model.validate(expected, builder.build()); + } + } + + @Test + public void singlePartitionLimit() + { + for (TableMetadata metadata : defaultTables()) + { + if (metadata.clusteringColumns().isEmpty()) continue; + ASTSingleTableModel model = new ASTSingleTableModel(metadata); + Function<ByteBuffer, ColumnValue> update = bb -> partitionLevelUpdate(ZERO, bb); + ByteBuffer[][] expected = new ByteBuffer[][]{ + insert(model, ZERO), + insert(model, update.apply(ONE)), + insert(model, update.apply(TWO)), + insert(model, update.apply(THREE)), + }; + + Select.Builder builder = Select.builder().table(metadata); + for (var pk : metadata.partitionKeyColumns()) + builder.value(new Symbol(pk), ZERO); + // without limit + model.validate(expected, builder.build()); + for (int limit = 1; limit <= expected.length; limit++) + { + builder.limit(limit); + model.validate(Arrays.copyOf(expected, limit), builder.build()); + } } } @@ -328,7 +357,38 @@ public void selectStar() modelModel.add(insert(model, value)); } - model.validate(modelModel.all(), Select.builder(metadata).build()); + var builder = Select.builder(metadata); + ByteBuffer[][] all = modelModel.all(); + model.validate(all, builder.build()); + for (int i = 1; i < all.length; i++) + model.validate(Arrays.copyOf(all, i), builder.limit(i).build()); + model.validate(all, builder.limit(all.length).build()); + model.validate(all, builder.limit(all.length + 1).build()); + } + } + + @Test + public void selectStarPerPartitionLimit() + { + List<ByteBuffer> values = Arrays.asList(ZERO, ONE, TWO, THREE); + for (TableMetadata metadata : defaultTables()) + { + if (metadata.clusteringColumns().isEmpty()) continue; + + ASTSingleTableModel model = new ASTSingleTableModel(metadata); + ModelModel modelModel = new ModelModel(model); + for (ByteBuffer pk : values) + { + for (ByteBuffer row : values) + modelModel.add(insert(model, partitionLevelUpdate(pk, row))); + } + + var builder = Select.builder(metadata); + model.validate(modelModel.all(), builder.build()); + for (int i = 1; i < values.size(); i++) + model.validate(modelModel.allPerPartitionLimit(i), builder.perPartitionLimit(i).build()); + + model.validate(modelModel.all(), builder.perPartitionLimit(values.size()).build()); } } @@ -522,6 +582,20 @@ private static ByteBuffer[] insert(ASTSingleTableModel model, ByteBuffer value) return insert(model, (i1, i2) -> value); } + private static ColumnValue partitionLevelUpdate(ByteBuffer partitionLevel, ByteBuffer rowLevel) + { + return (kind, offset) -> { + switch (kind) + { + case PARTITION_KEY: + case STATIC: + return partitionLevel; + default: + return rowLevel; + } + }; + } + private static ByteBuffer[] insert(ASTSingleTableModel model, ColumnValue fn) { TableMetadata metadata = model.factory.metadata; @@ -617,6 +691,28 @@ public ByteBuffer[][] all() return allWhere(i -> true); } + public ByteBuffer[][] allPerPartitionLimit(int limit) + { + class State + { + BytesPartitionState.Ref current = null; + int count = 0; + boolean nextPartition(BytesPartitionState.Ref ref) + { + current = ref; + count = 0; + return true; + } + + boolean nextRow(ByteBuffer[] row) + { + return ++count <= limit; + } + } + State state = new State(); + return allWhere(state::nextPartition, state::nextRow); + } + public ByteBuffer[][] allEq(Symbol column, ByteBuffer value) { return allWhere(column, Inequality.EQUAL, value); diff --git a/test/unit/org/apache/cassandra/cql3/ast/Select.java b/test/unit/org/apache/cassandra/cql3/ast/Select.java index 55f984df9182..7fa497e16c89 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/Select.java +++ b/test/unit/org/apache/cassandra/cql3/ast/Select.java @@ -39,6 +39,7 @@ public class Select implements Statement [AND clustering_filters [AND static_filters]]] [ORDER BY PK_column_name ASC|DESC] +[PER PARTITION LIMIT N] [LIMIT N] [ALLOW FILTERING] */ @@ -49,7 +50,7 @@ public class Select implements Statement // where public final Optional<Conditional> where; public final Optional<OrderBy> orderBy; - public final Optional<Value> limit; + public final Optional<Value> perPartitionLimit, limit; public final boolean allowFiltering; public Select(List<Expression> selections) @@ -59,15 +60,16 @@ public Select(List<Expression> selections) public Select(List<Expression> selections, Optional<TableReference> source, Optional<Conditional> where, Optional<OrderBy> orderBy, Optional<Value> limit) { - this(selections, source, where, orderBy, limit, false); + this(selections, source, where, orderBy, Optional.empty(), limit, false); } - public Select(List<Expression> selections, Optional<TableReference> source, Optional<Conditional> where, Optional<OrderBy> orderBy, Optional<Value> limit, boolean allowFiltering) + public Select(List<Expression> selections, Optional<TableReference> source, Optional<Conditional> where, Optional<OrderBy> orderBy, Optional<Value> perPartitionLimit, Optional<Value> limit, boolean allowFiltering) { this.selections = selections; this.source = source; this.where = where; this.orderBy = orderBy; + this.perPartitionLimit = perPartitionLimit; this.limit = limit; this.allowFiltering = allowFiltering; @@ -96,7 +98,17 @@ public static TableBasedBuilder builder(TableMetadata metadata) public Select withAllowFiltering() { - return new Select(selections, source, where, orderBy, limit, true); + return new Select(selections, source, where, orderBy, perPartitionLimit, limit, true); + } + + public Select withLimit(int limit) + { + return new Select(selections, source, where, orderBy, perPartitionLimit, Optional.of(Literal.of(limit)), allowFiltering); + } + + public Select withPerPartitionLimit(int perPartitionLimit) + { + return new Select(selections, source, where, orderBy, Optional.of(Literal.of(perPartitionLimit)), limit, allowFiltering); } @Override @@ -132,6 +144,12 @@ public void toCQL(StringBuilder sb, CQLFormatter formatter) sb.append("ORDER BY "); orderBy.get().toCQL(sb, formatter); } + if (perPartitionLimit.isPresent()) + { + formatter.section(sb); + sb.append("PER PARTITION LIMIT "); + perPartitionLimit.get().toCQL(sb, formatter); + } if (limit.isPresent()) { formatter.section(sb); @@ -154,6 +172,7 @@ public Stream<? extends Element> stream() + (source.isPresent() ? 1 : 0) + (where.isPresent() ? 1 : 0) + (orderBy.isPresent() ? 1 : 0) + + (perPartitionLimit.isPresent() ? 1 : 0) + (limit.isPresent() ? 1 : 0)); es.addAll(selections); if (source.isPresent()) @@ -162,6 +181,8 @@ public Stream<? extends Element> stream() es.add(where.get()); if (orderBy.isPresent()) es.add(orderBy.get()); + if (perPartitionLimit.isPresent()) + es.add(perPartitionLimit.get()); if (limit.isPresent()) es.add(limit.get()); return es.stream(); @@ -204,6 +225,18 @@ public Statement visit(Visitor v) { where = this.where; } + Optional<Value> perPartitionLimit; + if (this.perPartitionLimit.isPresent()) + { + var l = this.perPartitionLimit.get(); + var update = l.visit(v); + updated |= l != update; + perPartitionLimit = Optional.ofNullable(update); + } + else + { + perPartitionLimit = this.perPartitionLimit; + } Optional<Value> limit; if (this.limit.isPresent()) { @@ -217,9 +250,10 @@ public Statement visit(Visitor v) limit = this.limit; } if (!updated) return this; - return new Select(selections, source, where, orderBy, limit, allowFiltering); + return new Select(selections, source, where, orderBy, perPartitionLimit, limit, allowFiltering); } + public static class OrderBy implements Element { public enum Ordering @@ -313,6 +347,7 @@ public static class BaseBuilder<T extends BaseBuilder<T>> implements Conditional protected Optional<TableReference> source = Optional.empty(); private Conditional.Builder where = new Conditional.Builder(); private OrderBy.Builder orderBy = new OrderBy.Builder(); + private Optional<Value> perPartitionLimit = Optional.empty(); private Optional<Value> limit = Optional.empty(); private boolean allowFiltering = false; @@ -377,6 +412,17 @@ public T orderByColumn(String name, AbstractType<?> type, OrderBy.Ordering order return (T) this; } + public T perPartitionLimit(Value limit) + { + this.perPartitionLimit = Optional.of(limit); + return (T) this; + } + + public T perPartitionLimit(int limit) + { + return perPartitionLimit(Bind.of(limit)); + } + public T limit(Value limit) { this.limit = Optional.of(limit); @@ -394,7 +440,7 @@ public Select build() source, where.isEmpty() ? Optional.empty() : Optional.of(where.build()), orderBy.isEmpty() ? Optional.empty() : Optional.of(orderBy.build()), - limit, + perPartitionLimit, limit, allowFiltering); } } diff --git a/test/unit/org/apache/cassandra/utils/CassandraGenerators.java b/test/unit/org/apache/cassandra/utils/CassandraGenerators.java index ca2576cdcc17..8867d6141e1e 100644 --- a/test/unit/org/apache/cassandra/utils/CassandraGenerators.java +++ b/test/unit/org/apache/cassandra/utils/CassandraGenerators.java @@ -28,6 +28,7 @@ import java.util.Collections; import java.util.Comparator; import java.util.Deque; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -51,6 +52,7 @@ import org.apache.commons.lang3.builder.MultilineRecursiveToStringStyle; import org.apache.commons.lang3.builder.ReflectionToStringBuilder; +import org.apache.cassandra.config.DataStorageSpec; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.Duration; @@ -60,6 +62,12 @@ import org.apache.cassandra.db.SchemaCQLHelper; import org.apache.cassandra.db.SinglePartitionReadCommand; import org.apache.cassandra.db.Slices; +import org.apache.cassandra.db.compaction.AbstractCompactionStrategy; +import org.apache.cassandra.db.compaction.LeveledCompactionStrategy; +import org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy; +import org.apache.cassandra.db.compaction.SizeTieredCompactionStrategyOptions; +import org.apache.cassandra.db.compaction.UnifiedCompactionStrategy; +import org.apache.cassandra.db.compaction.unified.Controller; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.ByteBufferAccessor; import org.apache.cassandra.db.marshal.CompositeType; @@ -82,6 +90,8 @@ import org.apache.cassandra.gms.EndpointState; import org.apache.cassandra.gms.HeartBeatState; import org.apache.cassandra.gms.VersionedValue; +import org.apache.cassandra.io.compress.LZ4Compressor; +import org.apache.cassandra.io.compress.ZstdCompressor; import org.apache.cassandra.io.sstable.format.SSTableFormat; import org.apache.cassandra.locator.AbstractReplicationStrategy; import org.apache.cassandra.locator.InetAddressAndPort; @@ -94,7 +104,10 @@ import org.apache.cassandra.net.NoPayload; import org.apache.cassandra.net.PingRequest; import org.apache.cassandra.net.Verb; +import org.apache.cassandra.schema.CachingParams; import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.CompactionParams; +import org.apache.cassandra.schema.CompressionParams; import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.schema.MemtableParams; @@ -196,6 +209,71 @@ private CassandraGenerators() } + private static String humanReadableSignPrefix(RandomnessSource rnd) + { + switch (SourceDSL.integers().between(0, 2).generate(rnd)) + { + case 0: return ""; + case 1: return "-"; + case 2: return "+"; + default: + throw new AssertionError(); + } + } + + public static Gen<String> humanReadableStorageValue() + { + Gen<Long> valueGen = SourceDSL.longs().between(0, 1000); + return rnd -> { + // [+-]?\d+(\.\d+)?([eE]([+-]?)\d+)? + StringBuilder sb = new StringBuilder(); + sb.append(humanReadableSignPrefix(rnd)); + sb.append(valueGen.generate(rnd)); + if (nextBoolean(rnd)) + { + sb.append('.'); + sb.append(valueGen.generate(rnd)); + } + if (nextBoolean(rnd)) + { + sb.append('E'); + sb.append(humanReadableSignPrefix(rnd)); + sb.append(valueGen.generate(rnd)); + } + return sb.toString(); + }; + } + + public static Gen<String> humanReadableStorage() + { + Gen<DataStorageSpec.DataStorageUnit> unitGen = SourceDSL.arbitrary().enumValues(DataStorageSpec.DataStorageUnit.class); + return rnd -> { + DataStorageSpec.DataStorageUnit unit = unitGen.generate(rnd); + String value; + switch (SourceDSL.integers().between(0, 2).generate(rnd)) + { + case 0: + value = "NaN"; + break; + case 1: + value = humanReadableSignPrefix(rnd) + "Infinity"; + break; + case 2: + value = humanReadableStorageValue().generate(rnd); + break; + default: + throw new AssertionError(); + } + return value + ' ' + unit.getSymbol(); + }; + } + + public static Gen<String> humanReadableStorageSimple() + { + Gen<DataStorageSpec.DataStorageUnit> unitGen = SourceDSL.arbitrary().enumValues(DataStorageSpec.DataStorageUnit.class); + return rnd -> humanReadableStorageValue().generate(rnd) + ' ' + unitGen.generate(rnd).getSymbol(); + } + public static Set<UserType> extractUDTs(TableMetadata metadata) { Set<UserType> matches = new HashSet<>(); @@ -413,10 +491,261 @@ public Gen<KeyspaceMetadata> build() } } + public static Gen<CachingParams> cachingParamsGen() + { + return rnd -> { + boolean cacheKeys = nextBoolean(rnd); + int rowsPerPartitionToCache; + switch (SourceDSL.integers().between(1, 3).generate(rnd)) + { + case 1: // ALL + rowsPerPartitionToCache = Integer.MAX_VALUE; + break; + case 2: // NONE + rowsPerPartitionToCache = 0; + break; + case 3: // num values + rowsPerPartitionToCache = Math.toIntExact(rnd.next(Constraint.between(1, Integer.MAX_VALUE - 1))); + break; + default: + throw new AssertionError(); + } + return new CachingParams(cacheKeys, rowsPerPartitionToCache); + }; + } + + public enum KnownCompactionAlgo + { + SizeTiered(SizeTieredCompactionStrategy.class), + Leveled(LeveledCompactionStrategy.class), + Unified(UnifiedCompactionStrategy.class); + private final Class<? extends AbstractCompactionStrategy> klass; + + KnownCompactionAlgo(Class<? extends AbstractCompactionStrategy> klass) + { + this.klass = klass; + } + } + + public static class CompactionParamsBuilder + { + private Gen<KnownCompactionAlgo> algoGen = SourceDSL.arbitrary().enumValues(KnownCompactionAlgo.class); + private Gen<CompactionParams.TombstoneOption> tombstoneOptionGen = SourceDSL.arbitrary().enumValues(CompactionParams.TombstoneOption.class); + private Gen<Map<String, String>> sizeTieredOptions = rnd -> { + if (nextBoolean(rnd)) return Map.of(); + Map<String, String> options = new HashMap<>(); + if (nextBoolean(rnd)) + // computes mb then converts to bytes + options.put(SizeTieredCompactionStrategyOptions.MIN_SSTABLE_SIZE_KEY, Long.toString(SourceDSL.longs().between(1, 100).generate(rnd) * 1024L * 1024L)); + if (nextBoolean(rnd)) + options.put(SizeTieredCompactionStrategyOptions.BUCKET_LOW_KEY, Double.toString(SourceDSL.doubles().between(0.1, 0.9).generate(rnd))); + if (nextBoolean(rnd)) + options.put(SizeTieredCompactionStrategyOptions.BUCKET_HIGH_KEY, Double.toString(SourceDSL.doubles().between(1.1, 1.9).generate(rnd))); + return options; + }; + private Gen<Map<String, String>> leveledOptions = rnd -> { + if (nextBoolean(rnd)) return Map.of(); + Map<String, String> options = new HashMap<>(); + if (nextBoolean(rnd)) + options.putAll(sizeTieredOptions.generate(rnd)); + if (nextBoolean(rnd)) + // size in mb + options.put(LeveledCompactionStrategy.SSTABLE_SIZE_OPTION, SourceDSL.integers().between(1, 2_000).generate(rnd).toString()); + if (nextBoolean(rnd)) + options.put(LeveledCompactionStrategy.LEVEL_FANOUT_SIZE_OPTION, SourceDSL.integers().between(1, 100).generate(rnd).toString()); + if (nextBoolean(rnd)) + options.put(LeveledCompactionStrategy.SINGLE_SSTABLE_UPLEVEL_OPTION, nextBoolean(rnd).toString()); + return options; + }; + private Gen<Map<String, String>> unifiedOptions = rnd -> { + if (nextBoolean(rnd)) return Map.of(); + Gen<String> storageSizeGen = Generators.filter(humanReadableStorageSimple(), s -> Controller.MIN_TARGET_SSTABLE_SIZE <= FBUtilities.parseHumanReadableBytes(s)); + Map<String, String> options = new HashMap<>(); + if (nextBoolean(rnd)) + options.put(Controller.BASE_SHARD_COUNT_OPTION, SourceDSL.integers().between(1, 10).generate(rnd).toString()); + if (nextBoolean(rnd)) + options.put(Controller.FLUSH_SIZE_OVERRIDE_OPTION, storageSizeGen.generate(rnd)); + if (nextBoolean(rnd)) + options.put(Controller.MAX_SSTABLES_TO_COMPACT_OPTION, SourceDSL.integers().between(0, 32).generate(rnd).toString()); + if (nextBoolean(rnd)) + options.put(Controller.SSTABLE_GROWTH_OPTION, SourceDSL.integers().between(0, 100).generate(rnd) + "%"); + if (nextBoolean(rnd)) + options.put(Controller.OVERLAP_INCLUSION_METHOD_OPTION, SourceDSL.arbitrary().enumValues(Overlaps.InclusionMethod.class).generate(rnd).name()); + if (nextBoolean(rnd)) + { + int numLevels = SourceDSL.integers().between(1, 10).generate(rnd); + String[] scalingParams = new String[numLevels]; + Gen<Integer> levelSize = SourceDSL.integers().between(2, 10); + for (int i = 0; i < numLevels; i++) + { + String value; + switch (SourceDSL.integers().between(0, 3).generate(rnd)) + { + case 0: + value = "N"; + break; + case 1: + value = "L" + levelSize.generate(rnd); + break; + case 2: + value = "T" + levelSize.generate(rnd); + break; + case 3: + value = SourceDSL.integers().all().generate(rnd).toString(); + break; + default: + throw new AssertionError(); + } + scalingParams[i] = value; + } + options.put(Controller.SCALING_PARAMETERS_OPTION, String.join(",", scalingParams)); + } + if (nextBoolean(rnd)) + { + // Calculate TARGET then compute the MIN from that. The issue is that there is a hidden relationship + // between these 2 fields more complex than simple comparability, MIN must be < 70% * TARGET! + // See CASSANDRA-20398 + // 1MiB to 128MiB target + long targetBytes = SourceDSL.longs().between(1L << 20, 1L << 27).generate(rnd); + long limit = (long) Math.ceil(targetBytes * Math.sqrt(0.5)); + long minBytes = SourceDSL.longs().between(1, limit - 1).generate(rnd); + options.put(Controller.MIN_SSTABLE_SIZE_OPTION, minBytes + "B"); + options.put(Controller.TARGET_SSTABLE_SIZE_OPTION, targetBytes + "B"); + } + return options; + }; + //TODO (coverage): doesn't look to validate > 1, what does that even mean? + private Gen<Float> tombstoneThreshold = SourceDSL.floats().between(0, 1); + private Gen<Boolean> uncheckedTombstoneCompaction = SourceDSL.booleans().all(); + private Gen<Boolean> onlyPurgeRepairedTombstones = SourceDSL.booleans().all(); + + public Gen<CompactionParams> build() + { + return rnd -> { + KnownCompactionAlgo algo = algoGen.generate(rnd); + Map<String, String> options = new HashMap<>(); + if (nextBoolean(rnd)) + options.put(CompactionParams.Option.PROVIDE_OVERLAPPING_TOMBSTONES.toString(), tombstoneOptionGen.generate(rnd).name()); + if (CompactionParams.supportsThresholdParams(algo.klass) && nextBoolean(rnd)) + { + options.put(CompactionParams.Option.MIN_THRESHOLD.toString(), Long.toString(rnd.next(Constraint.between(2, 4)))); + options.put(CompactionParams.Option.MAX_THRESHOLD.toString(), Long.toString(rnd.next(Constraint.between(5, 32)))); + } + if (nextBoolean(rnd)) + options.put(AbstractCompactionStrategy.TOMBSTONE_THRESHOLD_OPTION, tombstoneThreshold.generate(rnd).toString()); + if (nextBoolean(rnd)) + options.put(AbstractCompactionStrategy.UNCHECKED_TOMBSTONE_COMPACTION_OPTION, uncheckedTombstoneCompaction.generate(rnd).toString()); + if (nextBoolean(rnd)) + options.put(AbstractCompactionStrategy.ONLY_PURGE_REPAIRED_TOMBSTONES, onlyPurgeRepairedTombstones.generate(rnd).toString()); + switch (algo) + { + case SizeTiered: + options.putAll(sizeTieredOptions.generate(rnd)); + break; + case Leveled: + options.putAll(leveledOptions.generate(rnd)); + break; + case Unified: + options.putAll(unifiedOptions.generate(rnd)); + break; + default: + throw new UnsupportedOperationException(algo.name()); + } + return CompactionParams.create(algo.klass, options); + }; + } + } + + private static Boolean nextBoolean(RandomnessSource rnd) + { + return SourceDSL.booleans().all().generate(rnd); + } + + public static Gen<CompactionParams> compactionParamsGen() + { + return new CompactionParamsBuilder().build(); + } + + public enum KnownCompressionAlgo + { + snappy("SnappyCompressor"), + deflate("DeflateCompressor"), + lz4("LZ4Compressor"), + zstd("ZstdCompressor"), + noop("NoopCompressor"); + + private final String compressor; + + KnownCompressionAlgo(String compressor) + { + this.compressor = compressor; + } + } + + public static class CompressionParamsBuilder + { + private Gen<Boolean> enabledGen = SourceDSL.booleans().all(); + private Gen<KnownCompressionAlgo> algoGen = SourceDSL.arbitrary().enumValues(KnownCompressionAlgo.class); + private Gen<Map<String, String>> lz4OptionsGen = rnd -> { + if (nextBoolean(rnd)) + return Map.of(); + Map<String, String> options = new HashMap<>(); + if (nextBoolean(rnd)) + options.put(LZ4Compressor.LZ4_COMPRESSOR_TYPE, nextBoolean(rnd) ? LZ4Compressor.LZ4_FAST_COMPRESSOR : LZ4Compressor.LZ4_HIGH_COMPRESSOR); + if (nextBoolean(rnd)) + options.put(LZ4Compressor.LZ4_HIGH_COMPRESSION_LEVEL, Integer.toString(Math.toIntExact(rnd.next(Constraint.between(1, 17))))); + return options; + }; + private Gen<Map<String, String>> zstdOptionsGen = rnd -> { + if (nextBoolean(rnd)) + return Map.of(); + int level = Math.toIntExact(rnd.next(Constraint.between(ZstdCompressor.FAST_COMPRESSION_LEVEL, ZstdCompressor.BEST_COMPRESSION_LEVEL))); + return Map.of(ZstdCompressor.COMPRESSION_LEVEL_OPTION_NAME, Integer.toString(level)); + }; + + public Gen<CompressionParams> build() + { + return rnd -> { + if (!enabledGen.generate(rnd)) + return CompressionParams.noCompression(); + KnownCompressionAlgo algo = algoGen.generate(rnd); + if (algo == KnownCompressionAlgo.noop) + return CompressionParams.noop(); + // when null disabled + int chunkLength = CompressionParams.DEFAULT_CHUNK_LENGTH; + double minCompressRatio = CompressionParams.DEFAULT_MIN_COMPRESS_RATIO; + Map<String, String> options; + switch (algo) + { + case lz4: + options = lz4OptionsGen.generate(rnd); + break; + case zstd: + options = zstdOptionsGen.generate(rnd); + break; + default: + options = Map.of(); + } + return new CompressionParams(algo.compressor, options, chunkLength, minCompressRatio); + }; + } + } + + public static Gen<CompressionParams> compressionParamsGen() + { + return new CompressionParamsBuilder().build(); + } + public static class TableParamsBuilder { @Nullable private Gen<String> memtableKeyGen = null; + @Nullable + private Gen<CachingParams> cachingParamsGen = null; + @Nullable + private Gen<CompactionParams> compactionParamsGen = null; + @Nullable + private Gen<CompressionParams> compressionParamsGen = null; public TableParamsBuilder withKnownMemtables() { @@ -427,12 +756,36 @@ public TableParamsBuilder withKnownMemtables() return this; } + public TableParamsBuilder withCaching() + { + cachingParamsGen = cachingParamsGen(); + return this; + } + + public TableParamsBuilder withCompaction() + { + compactionParamsGen = compactionParamsGen(); + return this; + } + + public TableParamsBuilder withCompression() + { + compressionParamsGen = compressionParamsGen(); + return this; + } + public Gen<TableParams> build() { return rnd -> { TableParams.Builder params = TableParams.builder(); if (memtableKeyGen != null) params.memtable(MemtableParams.get(memtableKeyGen.generate(rnd))); + if (cachingParamsGen != null) + params.caching(cachingParamsGen.generate(rnd)); + if (compactionParamsGen != null) + params.compaction(compactionParamsGen.generate(rnd)); + if (compressionParamsGen != null) + params.compression(compressionParamsGen.generate(rnd)); return params.build(); }; } @@ -515,6 +868,12 @@ public TableMetadataBuilder withKnownMemtables() return this; } + public TableMetadataBuilder withParams(Consumer<TableParamsBuilder> fn) + { + fn.accept(paramsBuilder); + return this; + } + public TableMetadataBuilder withKeyspaceName(Gen<String> ksNameGen) { this.ksNameGen = ksNameGen; @@ -1358,7 +1717,7 @@ private enum EpochConstants { FIRST, EMPTY, UPGRADE_STARTUP, UPGRADE_GOSSIP} public static Gen<Epoch> epochs() { return rnd -> { - if (SourceDSL.booleans().all().generate(rnd)) + if (nextBoolean(rnd)) { switch (SourceDSL.arbitrary().enumValues(EpochConstants.class).generate(rnd)) { From 2d3d7788f45a1867f5d47d60fb5a69d042b05f2d Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Tue, 18 Mar 2025 11:58:16 +0100 Subject: [PATCH 199/225] Fix failing simulator dtests after CASSANDRA-20368 patch by Stefan Miklosovic; reviewed by David Capwell for CASSANDRA-20450 --- src/java/org/apache/cassandra/auth/AuthConfig.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/auth/AuthConfig.java b/src/java/org/apache/cassandra/auth/AuthConfig.java index 1363b24a6fe2..67d4490790ee 100644 --- a/src/java/org/apache/cassandra/auth/AuthConfig.java +++ b/src/java/org/apache/cassandra/auth/AuthConfig.java @@ -135,6 +135,15 @@ private static <T> T authInstantiate(ParameterizedClass authCls, Class<T> defaul return ParameterizedClass.newInstance(authCls, List.of("", authPackage)); } - return ParameterizedClass.newInstance(new ParameterizedClass(defaultCls.getName()), List.of()); + // for now, this has to stay and can not be replaced by ParameterizedClass.newInstance as above + // due to that failing for simulator dtests. See CASSANDRA-20450 for more information. + try + { + return defaultCls.newInstance(); + } + catch (InstantiationException | IllegalAccessException e) + { + throw new ConfigurationException("Failed to instantiate " + defaultCls.getName(), e); + } } } From a16333387cf75bf8900ed116e5941e951480181c Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Tue, 18 Mar 2025 14:29:35 +0100 Subject: [PATCH 200/225] Update zstd-jni to 1.5.7-2 patch by Stefan Miklosovic; reviewed by Dmitry Konstantinov for CASSANDRA-20453 --- CHANGES.txt | 1 + build.xml | 2 +- ide/nbproject/project.xml | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 0ad954d81eed..a8a6062366fc 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0.18 + * Update zstd-jni to 1.5.7-2 (CASSANDRA-20453) * Suppress CVE-2024-12801 (CASSANDRA-20412) * Suppress CVE-2024-12798 (CASSANDRA-20408) * Support null column value tombstones in FQL batch statements (CASSANDRA-20397) diff --git a/build.xml b/build.xml index 41e59c9f1b99..59749daa57f5 100644 --- a/build.xml +++ b/build.xml @@ -547,7 +547,7 @@ <dependencyManagement> <dependency groupId="org.xerial.snappy" artifactId="snappy-java" version="1.1.10.4"/> <dependency groupId="org.lz4" artifactId="lz4-java" version="1.8.0"/> - <dependency groupId="com.github.luben" artifactId="zstd-jni" version="1.5.7-1"/> + <dependency groupId="com.github.luben" artifactId="zstd-jni" version="1.5.7-2"/> <dependency groupId="com.google.guava" artifactId="guava" version="27.0-jre"> <exclusion groupId="com.google.code.findbugs" artifactId="jsr305" /> <exclusion groupId="org.codehaus.mojo" artifactId="animal-sniffer-annotations" /> diff --git a/ide/nbproject/project.xml b/ide/nbproject/project.xml index 49efd63261ca..552b9d46c3bd 100644 --- a/ide/nbproject/project.xml +++ b/ide/nbproject/project.xml @@ -7,7 +7,7 @@ <properties> <property name="project.dir">..</property> <!-- the compile classpaths should be distinct per compilation unit… but it is kept simple and the build will catch errors --> - <property name="cassandra.classpath.jars">${project.dir}/build/lib/jars/HdrHistogram-2.1.12.jar:${project.dir}/build/lib/jars/ST4-4.0.8.jar:${project.dir}/build/lib/jars/airline-0.8.jar:${project.dir}/build/lib/jars/antlr-3.5.2.jar:${project.dir}/build/lib/jars/antlr-runtime-3.5.2.jar:${project.dir}/build/lib/jars/asm-7.1.jar:${project.dir}/build/lib/jars/byteman-4.0.6.jar:${project.dir}/build/lib/jars/byteman-bmunit-4.0.6.jar:${project.dir}/build/lib/jars/byteman-install-4.0.6.jar:${project.dir}/build/lib/jars/byteman-submit-4.0.6.jar:${project.dir}/build/lib/jars/caffeine-2.5.6.jar:${project.dir}/build/lib/jars/cassandra-driver-core-3.11.0-shaded.jar:${project.dir}/build/lib/jars/chronicle-bytes-2.20.111.jar:${project.dir}/build/lib/jars/chronicle-core-2.20.126.jar:${project.dir}/build/lib/jars/chronicle-queue-5.20.123.jar:${project.dir}/build/lib/jars/chronicle-threads-2.20.111.jar:${project.dir}/build/lib/jars/chronicle-wire-2.20.117.jar:${project.dir}/build/lib/jars/commons-beanutils-1.7.0.jar:${project.dir}/build/lib/jars/commons-beanutils-core-1.8.0.jar:${project.dir}/build/lib/jars/commons-cli-1.1.jar:${project.dir}/build/lib/jars/commons-codec-1.9.jar:${project.dir}/build/lib/jars/commons-collections-3.2.1.jar:${project.dir}/build/lib/jars/commons-configuration-1.6.jar:${project.dir}/build/lib/jars/commons-digester-1.8.jar:${project.dir}/build/lib/jars/commons-el-1.0.jar:${project.dir}/build/lib/jars/commons-httpclient-3.0.1.jar:${project.dir}/build/lib/jars/commons-lang-2.4.jar:${project.dir}/build/lib/jars/commons-lang3-3.11.jar:${project.dir}/build/lib/jars/commons-math-2.1.jar:${project.dir}/build/lib/jars/commons-math3-3.2.jar:${project.dir}/build/lib/jars/commons-net-1.4.1.jar:${project.dir}/build/lib/jars/compile-command-annotations-1.2.0.jar:${project.dir}/build/lib/jars/concurrent-trees-2.4.0.jar:${project.dir}/build/lib/jars/ecj-4.6.1.jar:${project.dir}/build/lib/jars/ftplet-api-1.0.0.jar:${project.dir}/build/lib/jars/ftpserver-core-1.0.0.jar:${project.dir}/build/lib/jars/ftpserver-deprecated-1.0.0-M2.jar:${project.dir}/build/lib/jars/guava-27.0-jre.jar:${project.dir}/build/lib/jars/hadoop-core-1.0.3.jar:${project.dir}/build/lib/jars/hadoop-minicluster-1.0.3.jar:${project.dir}/build/lib/jars/hadoop-test-1.0.3.jar:${project.dir}/build/lib/jars/high-scale-lib-1.0.6.jar:${project.dir}/build/lib/jars/hppc-0.8.1.jar:${project.dir}/build/lib/jars/hsqldb-1.8.0.10.jar:${project.dir}/build/lib/jars/j2objc-annotations-1.3.jar:${project.dir}/build/lib/jars/jackson-annotations-2.13.2.jar:${project.dir}/build/lib/jars/jackson-core-2.13.2.jar:${project.dir}/build/lib/jars/jackson-databind-2.13.2.2.jar:${project.dir}/build/lib/jars/jacocoagent.jar:${project.dir}/build/lib/jars/jamm-0.3.2.jar:${project.dir}/build/lib/jars/jasper-compiler-5.5.12.jar:${project.dir}/build/lib/jars/jasper-runtime-5.5.12.jar:${project.dir}/build/lib/jars/javax.inject-1.jar:${project.dir}/build/lib/jars/jbcrypt-0.4.jar:${project.dir}/build/lib/jars/jcl-over-slf4j-1.7.25.jar:${project.dir}/build/lib/jars/jcommander-1.30.jar:${project.dir}/build/lib/jars/jctools-core-3.1.0.jar:${project.dir}/build/lib/jars/jersey-core-1.0.jar:${project.dir}/build/lib/jars/jersey-server-1.0.jar:${project.dir}/build/lib/jars/jets3t-0.7.1.jar:${project.dir}/build/lib/jars/jetty-6.1.26.jar:${project.dir}/build/lib/jars/jetty-util-6.1.26.jar:${project.dir}/build/lib/jars/jna-5.6.0.jar:${project.dir}/build/lib/jars/json-simple-1.1.jar:${project.dir}/build/lib/jars/jsp-2.1-6.1.14.jar:${project.dir}/build/lib/jars/jsp-api-2.1-6.1.14.jar:${project.dir}/build/lib/jars/jsr305-2.0.2.jar:${project.dir}/build/lib/jars/jsr311-api-1.0.jar:${project.dir}/build/lib/jars/jvm-attach-api-1.5.jar:${project.dir}/build/lib/jars/kfs-0.3.jar:${project.dir}/build/lib/jars/log4j-over-slf4j-1.7.25.jar:${project.dir}/build/lib/jars/logback-classic-1.2.9.jar:${project.dir}/build/lib/jars/logback-core-1.2.9.jar:${project.dir}/build/lib/jars/lz4-java-1.8.0.jar:${project.dir}/build/lib/jars/metrics-core-3.1.5.jar:${project.dir}/build/lib/jars/metrics-jvm-3.1.5.jar:${project.dir}/build/lib/jars/metrics-logback-3.1.5.jar:${project.dir}/build/lib/jars/mina-core-2.0.0-M5.jar:${project.dir}/build/lib/jars/mxdump-0.14.jar:${project.dir}/build/lib/jars/netty-all-4.1.58.Final.jar:${project.dir}/build/lib/jars/netty-tcnative-boringssl-static-2.0.36.Final.jar:${project.dir}/build/lib/jars/ohc-core-0.5.1.jar:${project.dir}/build/lib/jars/ohc-core-j8-0.5.1.jar:${project.dir}/build/lib/jars/oro-2.0.8.jar:${project.dir}/build/lib/jars/psjava-0.1.19.jar:${project.dir}/build/lib/jars/reporter-config-base-3.0.3.jar:${project.dir}/build/lib/jars/reporter-config3-3.0.3.jar:${project.dir}/build/lib/jars/servlet-api-2.5-6.1.14.jar:${project.dir}/build/lib/jars/sigar-1.6.4.jar:${project.dir}/build/lib/jars/sjk-cli-0.14.jar:${project.dir}/build/lib/jars/sjk-core-0.14.jar:${project.dir}/build/lib/jars/sjk-json-0.14.jar:${project.dir}/build/lib/jars/sjk-stacktrace-0.14.jar:${project.dir}/build/lib/jars/slf4j-api-1.7.25.jar:${project.dir}/build/lib/jars/snakeyaml-1.26.jar:${project.dir}/build/lib/jars/snappy-java-1.1.10.4.jar:${project.dir}/build/lib/jars/snowball-stemmer-1.3.0.581.1.jar:${project.dir}/build/lib/jars/stream-2.5.2.jar:${project.dir}/build/lib/jars/xmlenc-0.52.jar:${project.dir}/build/lib/jars/zstd-jni-1.5.7-1.jar:${project.dir}/build/test/lib/jars/ant-1.10.12.jar:${project.dir}/build/test/lib/jars/ant-junit-1.10.12.jar:${project.dir}/build/test/lib/jars/ant-launcher-1.10.12.jar:${project.dir}/build/test/lib/jars/asm-6.0.jar:${project.dir}/build/test/lib/jars/asm-analysis-6.0.jar:${project.dir}/build/test/lib/jars/asm-commons-6.0.jar:${project.dir}/build/test/lib/jars/asm-tree-6.0.jar:${project.dir}/build/test/lib/jars/asm-util-6.0.jar:${project.dir}/build/test/lib/jars/asm-xml-6.0.jar:${project.dir}/build/test/lib/jars/assertj-core-3.25.3.jar:${project.dir}/build/test/lib/jars/awaitility-4.0.3.jar:${project.dir}/build/test/lib/jars/byte-buddy-1.14.11.jar:${project.dir}/build/test/lib/jars/byte-buddy-agent-1.10.5.jar:${project.dir}/build/test/lib/jars/commons-io-2.6.jar:${project.dir}/build/test/lib/jars/commons-math3-3.6.1.jar:${project.dir}/build/test/lib/jars/dtest-api-0.0.16.jar:${project.dir}/build/test/lib/jars/hamcrest-2.2.jar:${project.dir}/build/test/lib/jars/java-allocation-instrumenter-3.1.0.jar:${project.dir}/build/test/lib/jars/javassist-3.28.0-GA.jar:${project.dir}/build/test/lib/jars/jmh-core-1.37.jar:${project.dir}/build/test/lib/jars/jmh-generator-annprocess-1.37.jar:${project.dir}/build/test/lib/jars/jopt-simple-5.0.4.jar:${project.dir}/build/test/lib/jars/jsr305-3.0.2.jar:${project.dir}/build/test/lib/jars/junit-4.12.jar:${project.dir}/build/test/lib/jars/mockito-core-3.2.4.jar:${project.dir}/build/test/lib/jars/objenesis-2.6.jar:${project.dir}/build/test/lib/jars/org.jacoco.agent-0.8.6.jar:${project.dir}/build/test/lib/jars/org.jacoco.ant-0.8.6.jar:${project.dir}/build/test/lib/jars/org.jacoco.core-0.8.6.jar:${project.dir}/build/test/lib/jars/org.jacoco.report-0.8.6.jar:${project.dir}/build/test/lib/jars/quicktheories-0.26.jar:${project.dir}/build/test/lib/jars/reflections-0.10.2.jar:${project.dir}/build/test/lib/jars/semver4j-3.1.0.jar:${project.dir}/build/test/lib/jars/slf4j-api-1.7.32.jar:</property> + <property name="cassandra.classpath.jars">${project.dir}/build/lib/jars/HdrHistogram-2.1.12.jar:${project.dir}/build/lib/jars/ST4-4.0.8.jar:${project.dir}/build/lib/jars/airline-0.8.jar:${project.dir}/build/lib/jars/antlr-3.5.2.jar:${project.dir}/build/lib/jars/antlr-runtime-3.5.2.jar:${project.dir}/build/lib/jars/asm-7.1.jar:${project.dir}/build/lib/jars/byteman-4.0.6.jar:${project.dir}/build/lib/jars/byteman-bmunit-4.0.6.jar:${project.dir}/build/lib/jars/byteman-install-4.0.6.jar:${project.dir}/build/lib/jars/byteman-submit-4.0.6.jar:${project.dir}/build/lib/jars/caffeine-2.5.6.jar:${project.dir}/build/lib/jars/cassandra-driver-core-3.11.0-shaded.jar:${project.dir}/build/lib/jars/chronicle-bytes-2.20.111.jar:${project.dir}/build/lib/jars/chronicle-core-2.20.126.jar:${project.dir}/build/lib/jars/chronicle-queue-5.20.123.jar:${project.dir}/build/lib/jars/chronicle-threads-2.20.111.jar:${project.dir}/build/lib/jars/chronicle-wire-2.20.117.jar:${project.dir}/build/lib/jars/commons-beanutils-1.7.0.jar:${project.dir}/build/lib/jars/commons-beanutils-core-1.8.0.jar:${project.dir}/build/lib/jars/commons-cli-1.1.jar:${project.dir}/build/lib/jars/commons-codec-1.9.jar:${project.dir}/build/lib/jars/commons-collections-3.2.1.jar:${project.dir}/build/lib/jars/commons-configuration-1.6.jar:${project.dir}/build/lib/jars/commons-digester-1.8.jar:${project.dir}/build/lib/jars/commons-el-1.0.jar:${project.dir}/build/lib/jars/commons-httpclient-3.0.1.jar:${project.dir}/build/lib/jars/commons-lang-2.4.jar:${project.dir}/build/lib/jars/commons-lang3-3.11.jar:${project.dir}/build/lib/jars/commons-math-2.1.jar:${project.dir}/build/lib/jars/commons-math3-3.2.jar:${project.dir}/build/lib/jars/commons-net-1.4.1.jar:${project.dir}/build/lib/jars/compile-command-annotations-1.2.0.jar:${project.dir}/build/lib/jars/concurrent-trees-2.4.0.jar:${project.dir}/build/lib/jars/ecj-4.6.1.jar:${project.dir}/build/lib/jars/ftplet-api-1.0.0.jar:${project.dir}/build/lib/jars/ftpserver-core-1.0.0.jar:${project.dir}/build/lib/jars/ftpserver-deprecated-1.0.0-M2.jar:${project.dir}/build/lib/jars/guava-27.0-jre.jar:${project.dir}/build/lib/jars/hadoop-core-1.0.3.jar:${project.dir}/build/lib/jars/hadoop-minicluster-1.0.3.jar:${project.dir}/build/lib/jars/hadoop-test-1.0.3.jar:${project.dir}/build/lib/jars/high-scale-lib-1.0.6.jar:${project.dir}/build/lib/jars/hppc-0.8.1.jar:${project.dir}/build/lib/jars/hsqldb-1.8.0.10.jar:${project.dir}/build/lib/jars/j2objc-annotations-1.3.jar:${project.dir}/build/lib/jars/jackson-annotations-2.13.2.jar:${project.dir}/build/lib/jars/jackson-core-2.13.2.jar:${project.dir}/build/lib/jars/jackson-databind-2.13.2.2.jar:${project.dir}/build/lib/jars/jacocoagent.jar:${project.dir}/build/lib/jars/jamm-0.3.2.jar:${project.dir}/build/lib/jars/jasper-compiler-5.5.12.jar:${project.dir}/build/lib/jars/jasper-runtime-5.5.12.jar:${project.dir}/build/lib/jars/javax.inject-1.jar:${project.dir}/build/lib/jars/jbcrypt-0.4.jar:${project.dir}/build/lib/jars/jcl-over-slf4j-1.7.25.jar:${project.dir}/build/lib/jars/jcommander-1.30.jar:${project.dir}/build/lib/jars/jctools-core-3.1.0.jar:${project.dir}/build/lib/jars/jersey-core-1.0.jar:${project.dir}/build/lib/jars/jersey-server-1.0.jar:${project.dir}/build/lib/jars/jets3t-0.7.1.jar:${project.dir}/build/lib/jars/jetty-6.1.26.jar:${project.dir}/build/lib/jars/jetty-util-6.1.26.jar:${project.dir}/build/lib/jars/jna-5.6.0.jar:${project.dir}/build/lib/jars/json-simple-1.1.jar:${project.dir}/build/lib/jars/jsp-2.1-6.1.14.jar:${project.dir}/build/lib/jars/jsp-api-2.1-6.1.14.jar:${project.dir}/build/lib/jars/jsr305-2.0.2.jar:${project.dir}/build/lib/jars/jsr311-api-1.0.jar:${project.dir}/build/lib/jars/jvm-attach-api-1.5.jar:${project.dir}/build/lib/jars/kfs-0.3.jar:${project.dir}/build/lib/jars/log4j-over-slf4j-1.7.25.jar:${project.dir}/build/lib/jars/logback-classic-1.2.9.jar:${project.dir}/build/lib/jars/logback-core-1.2.9.jar:${project.dir}/build/lib/jars/lz4-java-1.8.0.jar:${project.dir}/build/lib/jars/metrics-core-3.1.5.jar:${project.dir}/build/lib/jars/metrics-jvm-3.1.5.jar:${project.dir}/build/lib/jars/metrics-logback-3.1.5.jar:${project.dir}/build/lib/jars/mina-core-2.0.0-M5.jar:${project.dir}/build/lib/jars/mxdump-0.14.jar:${project.dir}/build/lib/jars/netty-all-4.1.58.Final.jar:${project.dir}/build/lib/jars/netty-tcnative-boringssl-static-2.0.36.Final.jar:${project.dir}/build/lib/jars/ohc-core-0.5.1.jar:${project.dir}/build/lib/jars/ohc-core-j8-0.5.1.jar:${project.dir}/build/lib/jars/oro-2.0.8.jar:${project.dir}/build/lib/jars/psjava-0.1.19.jar:${project.dir}/build/lib/jars/reporter-config-base-3.0.3.jar:${project.dir}/build/lib/jars/reporter-config3-3.0.3.jar:${project.dir}/build/lib/jars/servlet-api-2.5-6.1.14.jar:${project.dir}/build/lib/jars/sigar-1.6.4.jar:${project.dir}/build/lib/jars/sjk-cli-0.14.jar:${project.dir}/build/lib/jars/sjk-core-0.14.jar:${project.dir}/build/lib/jars/sjk-json-0.14.jar:${project.dir}/build/lib/jars/sjk-stacktrace-0.14.jar:${project.dir}/build/lib/jars/slf4j-api-1.7.25.jar:${project.dir}/build/lib/jars/snakeyaml-1.26.jar:${project.dir}/build/lib/jars/snappy-java-1.1.10.4.jar:${project.dir}/build/lib/jars/snowball-stemmer-1.3.0.581.1.jar:${project.dir}/build/lib/jars/stream-2.5.2.jar:${project.dir}/build/lib/jars/xmlenc-0.52.jar:${project.dir}/build/lib/jars/zstd-jni-1.5.7-2.jar:${project.dir}/build/test/lib/jars/ant-1.10.12.jar:${project.dir}/build/test/lib/jars/ant-junit-1.10.12.jar:${project.dir}/build/test/lib/jars/ant-launcher-1.10.12.jar:${project.dir}/build/test/lib/jars/asm-6.0.jar:${project.dir}/build/test/lib/jars/asm-analysis-6.0.jar:${project.dir}/build/test/lib/jars/asm-commons-6.0.jar:${project.dir}/build/test/lib/jars/asm-tree-6.0.jar:${project.dir}/build/test/lib/jars/asm-util-6.0.jar:${project.dir}/build/test/lib/jars/asm-xml-6.0.jar:${project.dir}/build/test/lib/jars/assertj-core-3.25.3.jar:${project.dir}/build/test/lib/jars/awaitility-4.0.3.jar:${project.dir}/build/test/lib/jars/byte-buddy-1.14.11.jar:${project.dir}/build/test/lib/jars/byte-buddy-agent-1.10.5.jar:${project.dir}/build/test/lib/jars/commons-io-2.6.jar:${project.dir}/build/test/lib/jars/commons-math3-3.6.1.jar:${project.dir}/build/test/lib/jars/dtest-api-0.0.16.jar:${project.dir}/build/test/lib/jars/hamcrest-2.2.jar:${project.dir}/build/test/lib/jars/java-allocation-instrumenter-3.1.0.jar:${project.dir}/build/test/lib/jars/javassist-3.28.0-GA.jar:${project.dir}/build/test/lib/jars/jmh-core-1.37.jar:${project.dir}/build/test/lib/jars/jmh-generator-annprocess-1.37.jar:${project.dir}/build/test/lib/jars/jopt-simple-5.0.4.jar:${project.dir}/build/test/lib/jars/jsr305-3.0.2.jar:${project.dir}/build/test/lib/jars/junit-4.12.jar:${project.dir}/build/test/lib/jars/mockito-core-3.2.4.jar:${project.dir}/build/test/lib/jars/objenesis-2.6.jar:${project.dir}/build/test/lib/jars/org.jacoco.agent-0.8.6.jar:${project.dir}/build/test/lib/jars/org.jacoco.ant-0.8.6.jar:${project.dir}/build/test/lib/jars/org.jacoco.core-0.8.6.jar:${project.dir}/build/test/lib/jars/org.jacoco.report-0.8.6.jar:${project.dir}/build/test/lib/jars/quicktheories-0.26.jar:${project.dir}/build/test/lib/jars/reflections-0.10.2.jar:${project.dir}/build/test/lib/jars/semver4j-3.1.0.jar:${project.dir}/build/test/lib/jars/slf4j-api-1.7.32.jar:</property> </properties> <folders> <source-folder> From 1d47fab638e16e103cbeb19fe979806c16b26b45 Mon Sep 17 00:00:00 2001 From: Caleb Rackliffe <calebrackliffe@gmail.com> Date: Mon, 17 Mar 2025 22:49:23 -0500 Subject: [PATCH 201/225] Serialization can lose complex deletions in a mutation with multiple collections in a row patch by Caleb Rackliffe; reviewed by Berenguer Blasi and Abe Ratnofsky for CASSANDRA-20449 --- CHANGES.txt | 1 + .../apache/cassandra/db/rows/BTreeRow.java | 6 +- .../cassandra/utils/ByteBufferUtil.java | 15 +++++ .../apache/cassandra/utils/btree/BTree.java | 5 +- .../distributed/test/CollectionsTest.java | 55 +++++++++++++++++++ 5 files changed, 78 insertions(+), 4 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/test/CollectionsTest.java diff --git a/CHANGES.txt b/CHANGES.txt index a7b52483c85a..bb53fe01076b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0.4 + * Serialization can lose complex deletions in a mutation with multiple collections in a row (CASSANDRA-20449) * Improve error messages when initializing auth classes (CASSANDRA-20368) * Prioritize legacy 2i over SAI for columns with multiple indexes (CASSANDRA-20334) * Ensure only offline tools can build IntervalTrees without first/last key fields (CASSANDRA-20407) diff --git a/src/java/org/apache/cassandra/db/rows/BTreeRow.java b/src/java/org/apache/cassandra/db/rows/BTreeRow.java index 075a4f67fe6d..52f0639e8e8a 100644 --- a/src/java/org/apache/cassandra/db/rows/BTreeRow.java +++ b/src/java/org/apache/cassandra/db/rows/BTreeRow.java @@ -60,6 +60,8 @@ import org.apache.cassandra.utils.btree.UpdateFunction; import org.apache.cassandra.utils.memory.Cloner; +import static org.apache.cassandra.utils.btree.BTree.STOP_SENTINEL_VALUE; + /** * Immutable implementation of a Row object. */ @@ -399,9 +401,9 @@ public boolean hasComplex() public boolean hasComplexDeletion() { - long result = accumulate((cd, v) -> ((ComplexColumnData) cd).complexDeletion().isLive() ? 0 : Cell.MAX_DELETION_TIME, + long result = accumulate((cd, v) -> ((ComplexColumnData) cd).complexDeletion().isLive() ? 0 : STOP_SENTINEL_VALUE, COLUMN_COMPARATOR, isStatic() ? FIRST_COMPLEX_STATIC : FIRST_COMPLEX_REGULAR, 0L); - return result == Cell.MAX_DELETION_TIME; + return result == STOP_SENTINEL_VALUE; } public Row markCounterLocalToBeCleared() diff --git a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java index 156c1c4bc840..4d3d0ca0f32c 100644 --- a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java +++ b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java @@ -34,10 +34,14 @@ import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; import java.util.Arrays; +import java.util.LinkedHashSet; +import java.util.Set; import java.util.UUID; import net.nicoulaj.compilecommand.annotations.Inline; import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.marshal.SetType; import org.apache.cassandra.io.compress.BufferType; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; @@ -544,6 +548,17 @@ else if (obj instanceof byte[]) return ByteBuffer.wrap((byte[]) obj); else if (obj instanceof ByteBuffer) return (ByteBuffer) obj; + else if (obj instanceof Set) + { + Set<?> set = (Set<?>) obj; + // convert subtypes to BB + Set<ByteBuffer> bbs = new LinkedHashSet<>(); + for (Object o : set) + if (!bbs.add(objectToBytes(o))) + throw new IllegalStateException("Object " + o + " maps to a buffer that already exists in the set"); + // decompose/serializer doesn't use the isMultiCell, so safe to do this + return SetType.getInstance(BytesType.instance, false).decompose(bbs); + } else throw new IllegalArgumentException(String.format("Cannot convert value %s of type %s", obj, diff --git a/src/java/org/apache/cassandra/utils/btree/BTree.java b/src/java/org/apache/cassandra/utils/btree/BTree.java index 2ac80df48ea3..8674d714daf8 100644 --- a/src/java/org/apache/cassandra/utils/btree/BTree.java +++ b/src/java/org/apache/cassandra/utils/btree/BTree.java @@ -68,6 +68,7 @@ public class BTree private static final int BRANCH_FACTOR = 1 << BRANCH_SHIFT; public static final int MIN_KEYS = BRANCH_FACTOR / 2 - 1; public static final int MAX_KEYS = BRANCH_FACTOR - 1; + public static final long STOP_SENTINEL_VALUE = Long.MAX_VALUE; // An empty BTree Leaf - which is the same as an empty BTree private static final Object[] EMPTY_LEAF = new Object[1]; @@ -1823,7 +1824,7 @@ private static <V> int find(Object[] btree, V from, Comparator<V> comparator) private static boolean isStopSentinel(long v) { - return v == Long.MAX_VALUE; + return v == STOP_SENTINEL_VALUE; } private static <V, A> long accumulateLeaf(Object[] btree, BiLongAccumulator<A, V> accumulator, A arg, Comparator<V> comparator, V from, long initialValue) @@ -1852,7 +1853,7 @@ private static <V, A> long accumulateLeaf(Object[] btree, BiLongAccumulator<A, V /** * Walk the btree and accumulate a long value using the supplied accumulator function. Iteration will stop if the - * accumulator function returns the sentinel values Long.MIN_VALUE or Long.MAX_VALUE + * accumulator function returns the sentinel value {@link #STOP_SENTINEL_VALUE} * <p> * If the optional from argument is not null, iteration will start from that value (or the one after it's insertion * point if an exact match isn't found) diff --git a/test/distributed/org/apache/cassandra/distributed/test/CollectionsTest.java b/test/distributed/org/apache/cassandra/distributed/test/CollectionsTest.java new file mode 100644 index 000000000000..8e8ab242408c --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/CollectionsTest.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import java.io.IOException; + +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; + +import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows; +import static org.apache.cassandra.distributed.shared.AssertUtils.row; + +public class CollectionsTest extends TestBaseImpl +{ + private static Cluster CLUSTER; + + @BeforeClass + public static void setUpCluster() throws IOException + { + CLUSTER = init(Cluster.build(2).start()); + } + + @Test + public void testMultipleSetsComplexDeletion() + { + CLUSTER.schemaChange(withKeyspace("CREATE TABLE %s.multi_collection (k int, c int, s1 set<int>, s2 set<int>, s3 set<int>, PRIMARY KEY (k, c)) WITH read_repair = 'NONE'")); + CLUSTER.coordinator(1).execute(withKeyspace("INSERT INTO %s.multi_collection (k, c, s1, s2, s3) VALUES (?, ?, ?, ?, ?)"), ConsistencyLevel.ALL, 0, 0, set(1), set(1), set(1)); + CLUSTER.coordinator(1).execute(withKeyspace("UPDATE %s.multi_collection SET s2 = ?, s1 = s1 + ?, s3 = s3 + ? WHERE k = ? AND c = ?"), ConsistencyLevel.ALL, set(2), set(2), set(2), 0, 0); + + String select = withKeyspace("SELECT k, c, s1, s2, s3 FROM %s.multi_collection"); + assertRows(CLUSTER.get(1).executeInternal(select), row(0, 0, set(1, 2), set(2), set(1, 2))); + + // If the complex deletion is not properly serialized, node 2 will think the update on s2 was an append... + assertRows(CLUSTER.get(2).executeInternal(select), row(0, 0, set(1, 2), set(2), set(1, 2))); + } +} From 12c061b3bb61d9c7c3699ca9a160e08760dd60d4 Mon Sep 17 00:00:00 2001 From: "Arra, Praveen R" <praveen.r.arra@jpmchase.com> Date: Mon, 17 Mar 2025 11:32:25 -0500 Subject: [PATCH 202/225] Updated documentation with additional function examples patch by Arra Praveen; reviewed by Stefan Miklosovic for CASSANDRA-20254 --- .../examples/BNF/insert_statement.bnf | 3 +- .../examples/CQL/update_statement.cql | 6 +- .../pages/developing/cql/definitions.adoc | 1 + .../cassandra/pages/developing/cql/dml.adoc | 61 +++++----- .../pages/developing/cql/functions.adoc | 106 ++++++++++-------- .../cassandra/pages/developing/cql/types.adoc | 2 + .../cassandra/partials/masking_functions.adoc | 28 +++-- 7 files changed, 110 insertions(+), 97 deletions(-) diff --git a/doc/modules/cassandra/examples/BNF/insert_statement.bnf b/doc/modules/cassandra/examples/BNF/insert_statement.bnf index ed80c3ed05b7..514af382ef6b 100644 --- a/doc/modules/cassandra/examples/BNF/insert_statement.bnf +++ b/doc/modules/cassandra/examples/BNF/insert_statement.bnf @@ -1,6 +1,7 @@ insert_statement::= INSERT INTO table_name ( names_values | json_clause ) [ IF NOT EXISTS ] - [ USING update_parameter ( AND update_parameter )* ] + [ USING insert_parameter ( AND insert_parameter )* ] names_values::= names VALUES tuple_literal json_clause::= JSON string [ DEFAULT ( NULL | UNSET ) ] names::= '(' column_name ( ',' column_name )* ')' +insert_parameter ::= ( TIMESTAMP | TTL ) ( integer | bind_marker ) diff --git a/doc/modules/cassandra/examples/CQL/update_statement.cql b/doc/modules/cassandra/examples/CQL/update_statement.cql index 7e1cfa76fecf..c3b4edca0e1e 100644 --- a/doc/modules/cassandra/examples/CQL/update_statement.cql +++ b/doc/modules/cassandra/examples/CQL/update_statement.cql @@ -1,10 +1,14 @@ +UPDATE NerdMovies +SET director = 'Joss Whedon', +WHERE movie = 'Serenity'; + UPDATE NerdMovies USING TTL 400 SET director = 'Joss Whedon', main_actor = 'Nathan Fillion', year = 2005 WHERE movie = 'Serenity'; -UPDATE UserActions +UPDATE UserActions USING TIMESTAMP 1735689600 SET total = total + 2 WHERE user = B70DE1D0-9908-4AE3-BE34-5573E5B09F14 AND action = 'click'; diff --git a/doc/modules/cassandra/pages/developing/cql/definitions.adoc b/doc/modules/cassandra/pages/developing/cql/definitions.adoc index 9d494c85b490..e139c28ef9f8 100644 --- a/doc/modules/cassandra/pages/developing/cql/definitions.adoc +++ b/doc/modules/cassandra/pages/developing/cql/definitions.adoc @@ -172,6 +172,7 @@ this documentation (see links above): include::cassandra:example$BNF/cql_statement.bnf[] ---- +[[prepared-statements]] == Prepared Statements CQL supports _prepared statements_. Prepared statements are an diff --git a/doc/modules/cassandra/pages/developing/cql/dml.adoc b/doc/modules/cassandra/pages/developing/cql/dml.adoc index 674ede814518..66981129cd54 100644 --- a/doc/modules/cassandra/pages/developing/cql/dml.adoc +++ b/doc/modules/cassandra/pages/developing/cql/dml.adoc @@ -72,25 +72,11 @@ include::cassandra:example$CQL/as.cql[] [NOTE] ==== Currently, aliases aren't recognized in the `WHERE` or `ORDER BY` clauses in the statement. -You must use the orignal column name instead. +You must use the original column name instead. ==== -[[writetime-and-ttl-function]] -==== `WRITETIME`, `MAXWRITETIME` and `TTL` function - -Selection supports three special functions that aren't allowed anywhere -else: `WRITETIME`, `MAXWRITETIME` and `TTL`. -All functions take only one argument, a column name. If the column is a collection or UDT, it's possible to add element -selectors, such as `WRITETTIME(phones[2..4])` or `WRITETTIME(user.name)`. -These functions retrieve meta-information that is stored internally for each column: - -* `WRITETIME` stores the timestamp of the value of the column. -* `MAXWRITETIME` stores the largest timestamp of the value of the column. For non-collection and non-UDT columns, `MAXWRITETIME` -is equivalent to `WRITETIME`. In the other cases, it returns the largest timestamp of the values in the column. -* `TTL` stores the remaining time to live (in seconds) for the value of the column if it is set to expire; otherwise the value is `null`. - -The `WRITETIME` and `TTL` functions can be used on multi-cell columns such as non-frozen collections or non-frozen -user-defined types. In that case, the functions will return the list of timestamps or TTLs for each selected cell. +Selection supports four special functions: `WRITETIME`, `MINWRITETIME`, `MAXWRITETIME` and `TTL`. +See the xref:cassandra:developing/cql/functions.adoc#writetime-and-ttl-functions[functions] section for details. [[where-clause]] === The `WHERE` clause @@ -120,7 +106,7 @@ include::cassandra:example$CQL/where.cql[] ---- But the following one is not, as it does not select a contiguous set of -rows (and we suppose no secondary indexes are set): +rows (assuming no secondary indexes): [source,cql] ---- @@ -133,7 +119,7 @@ Rows will be selected based on the token of the `PARTITION_KEY` rather than on t ==== The token of a key depends on the partitioner in use, and that in particular the `RandomPartitioner` won't yield a meaningful order. -Also note that ordering partitioners always order token values by bytes (so +Also note that the `ByteOrderedPartitioner` always orders token values by bytes (so even if the partition key is of type int, `token(-1) > token(0)` in particular). ==== @@ -216,7 +202,7 @@ or the reverse The `LIMIT` option to a `SELECT` statement limits the number of rows returned by a query. The `PER PARTITION LIMIT` option limits the -number of rows returned for a given partition by the query. Both types of limits can used in the same statement. +number of rows returned for a given partition by the query. Both types of limits can be used in the same statement. [[allow-filtering]] === Allowing filtering @@ -246,7 +232,7 @@ The first query returns all rows, because all users are selected. The second query returns only the rows defined by the secondary index, a per-node implementation; the results will depend on the number of nodes in the cluster, and is indirectly proportional to the amount of data stored. The number of nodes will always be multiple number of magnitude lower than the number of user profiles stored. -Both queries may return very large result sets, but the addition of a `LIMIT` clause can reduced the latency. +Both queries may return very large result sets, but the addition of a `LIMIT` clause would reduce the latency. The following query will be rejected: @@ -283,10 +269,10 @@ include::cassandra:example$CQL/insert_statement.cql[] The `INSERT` statement writes one or more columns for a given row in a table. -Since a row is identified by its `PRIMARY KEY`, at least one columns must be specified. +Since a row is identified by its `PRIMARY KEY`, at least one column must be specified. The list of columns to insert must be supplied with the `VALUES` syntax. When using the `JSON` syntax, `VALUES` are optional. -See the section on xref:cassandra:developing/cql/dml.adoc#cql-json[JSON support] for more detail. +See the section on xref:cassandra:developing/cql/json.adoc[JSON support] for more detail. All updates for an `INSERT` are applied atomically and in isolation. Unlike in SQL, `INSERT` does not check the prior existence of the row by default. @@ -297,7 +283,9 @@ The `IF NOT EXISTS` condition can restrict the insertion if the row does not exi However, note that using `IF NOT EXISTS` will incur a non-negligible performance cost, because Paxos is used, so this should be used sparingly. -Please refer to the xref:cassandra:developing/cql/dml.adoc#update-parameters[UPDATE] section for informations on the `update_parameter`. +When using xref:cassandra:developing/cql/definitions.adoc#prepared-statements[Prepared Statements] bind_markers can be used instead of actual values. + +Please refer to the xref:cassandra:developing/cql/dml.adoc#upsert-parameters[INSERT PARAMETERS] section for information on the `insert_parameter`. Also note that `INSERT` does not support counters, while `UPDATE` does. [[update-statement]] @@ -310,13 +298,6 @@ Updating a row is done using an `UPDATE` statement: include::cassandra:example$BNF/update_statement.bnf[] ---- -For instance: - -[source,cql] ----- -include::cassandra:example$CQL/update_statement.cql[] ----- - The `UPDATE` statement writes one or more columns for a given row in a table. The `WHERE` clause is used to select the row to update and must include all columns of the `PRIMARY KEY`. @@ -333,15 +314,15 @@ However, like the `IF NOT EXISTS` condition, a non-negligible performance cost c Regarding the `SET` assignment: * `c = c + 3` will increment/decrement counters, the only operation allowed. -The column name after the '=' sign *must* be the same than the one before the '=' sign. +The column name after the '=' sign *must* be the same as the one before the '=' sign. Increment/decrement is only allowed on counters. -See the section on xref:cassandra:developing/cql/dml.adoc#counters[counters] for details. +See the section on xref:cassandra:developing/cql/counter-column.adoc[counters] for details. * `id = id + <some-collection>` and `id[value1] = value2` are for collections. See the xref:cassandra:developing/cql/types.adoc#collections[collections] for details. * `id.field = 3` is for setting the value of a field on a non-frozen user-defined types. See the xref:cassandra:developing/cql/types.adoc#udts[UDTs] for details. -=== Update parameters +=== [[upsert-parameters]]Insert and Update parameters `UPDATE` and `INSERT` statements support the following parameters: @@ -362,6 +343,14 @@ the coordinator will use the current time (in microseconds) at the start of statement execution as the timestamp. This is usually a suitable default. +For instance: + +[source,cql] +---- +include::cassandra:example$CQL/update_statement.cql[] +---- + + [[delete_statement]] == DELETE @@ -389,7 +378,7 @@ may be deleted with one statement by using an `IN` operator. A range of rows may be deleted using an inequality operator (such as `>=`). `DELETE` supports the `TIMESTAMP` option with the same semantics as in -xref:cassandra:developing/cql/dml.adoc#update-parameters[updates]. +xref:cassandra:developing/cql/dml.adoc#upsert-parameters[updates]. In a `DELETE` statement, all deletions within the same partition key are applied atomically and in isolation. @@ -455,7 +444,7 @@ only isolated within a single partition). There is a performance penalty for batch atomicity when a batch spans multiple partitions. If you do not want to incur this penalty, you can -tell Cassandra to skip the batchlog with the `UNLOGGED` option. If the +tell Cassandra to skip the batch log with the `UNLOGGED` option. If the `UNLOGGED` option is used, a failed batch might leave the batch only partly applied. diff --git a/doc/modules/cassandra/pages/developing/cql/functions.adoc b/doc/modules/cassandra/pages/developing/cql/functions.adoc index ec8cd085276b..43b95257eede 100644 --- a/doc/modules/cassandra/pages/developing/cql/functions.adoc +++ b/doc/modules/cassandra/pages/developing/cql/functions.adoc @@ -1,14 +1,12 @@ -// Need some intro for UDF and native functions in general and point those to it. -// [[cql-functions]][[native-functions]] +[[cql-functions]] = Functions CQL supports 2 main categories of functions: -* xref:cassandra:developing/cql/functions.adoc#scalar-functions[scalar functions] that take a number of values and produce an output +* xref:cassandra:developing/cql/functions.adoc#scalar-native-functions[scalar functions] that take a number of values and produce an output * xref:cassandra:developing/cql/functions.adoc#aggregate-functions[aggregate functions] that aggregate multiple rows resulting from a `SELECT` statement -In both cases, CQL provides a number of native "hard-coded" functions as -well as the ability to create new user-defined functions. +In both cases, CQL provides a number of native "hard-coded" functions and also allows for the creation of custom user-defined functions. [NOTE] ==== @@ -92,9 +90,9 @@ into its own datatype. The conversions rely strictly on Java's semantics. For example, the double value 1 will be converted to the text value '1.0'. For instance: -[source,cql] +[source,sql] ---- -SELECT avg(cast(count as double)) FROM myTable +SELECT avg(cast(count as double)) FROM myTable; ---- ==== Token @@ -114,7 +112,7 @@ The type of the arguments of the `token` depend on the partition key column type For example, consider the following table: -[source,cql] +[source,sql] ---- include::cassandra:example$CQL/create_table_simple.cql[] ---- @@ -134,12 +132,12 @@ uuid suitable for use in `INSERT` or `UPDATE` statements. The `now` function takes no arguments and generates, on the coordinator node, a new unique timeuuid at the time the function is invoked. Note -that this method is useful for insertion but is largely non-sensical in +that this method is useful for insertion but is largely nonsensical in `WHERE` clauses. For example, a query of the form: -[source,cql] +[source,sql] ---- include::cassandra:example$CQL/timeuuid_now.cql[] ---- @@ -157,7 +155,7 @@ The `max_timeuuid` works similarly, but returns the _largest_ possible `timeuuid For example: -[source,cql] +[source,sql] ---- include::cassandra:example$CQL/timeuuid_min_max.cql[] ---- @@ -167,7 +165,7 @@ The clause `t >= maxTimeuuid('2013-01-01 00:05+0000')` would still _not_ select [NOTE] ==== -The values generated by `min_timeuuid` and `max_timeuuid` are called _fake_ UUID because they do no respect the time-based UUID generation process +The values generated by `min_timeuuid` and `max_timeuuid` are called _fake_ UUIDs because they do not respect the time-based UUID generation process specified by the http://www.ietf.org/rfc/rfc4122.txt[IETF RFC 4122]. In particular, the value returned by these two methods will not be unique. Thus, only use these methods for *querying*, not for *insertion*, to prevent possible data overwriting. @@ -175,10 +173,7 @@ Thus, only use these methods for *querying*, not for *insertion*, to prevent pos ==== Datetime functions -===== Retrieving the current date/time - -The following functions can be used to retrieve the date/time at the -time where the function is invoked: +Retrieving the current date and time: [cols=",",options="header",] |=== @@ -195,7 +190,7 @@ time where the function is invoked: For example the last two days of data can be retrieved using: -[source,cql] +[source,sql] ---- include::cassandra:example$CQL/current_date.cql[] ---- @@ -227,7 +222,7 @@ A number of functions are provided to convert a `timeuuid`, a `timestamp` or a ` A number of functions are provided to convert the native types into binary data, or a `blob`. -For every xref:cassandra:developing/cql/types.adoc#native-types[type] supported by CQL, the function `type_as_blob` takes a argument of type `type` and returns it as a `blob`. +For every xref:cassandra:developing/cql/types.adoc#native-types[type] supported by CQL, the function `type_as_blob` takes an argument of type `type` and returns it as a `blob`. Conversely, the function `blob_as_type` takes a 64-bit `blob` argument and converts it to a `bigint` value. For example, `bigint_as_blob(3)` returns `0x0000000000000003` and `blob_as_bigint(0x0000000000000003)` returns `3`. @@ -339,7 +334,7 @@ include::cassandra:partial$vector-search/vector_functions.adoc[] [[human-helper-functions]] ==== Human helper functions -For user's convenience, there are currently two functions which are converting values to more human-friendly represetations. +For user's convenience, there are currently two functions which are converting values to more human-friendly representations. [cols=",,",options="header",] |=== @@ -369,7 +364,7 @@ The actual return value of the `Long.MAX_VALUE` will be 9223372036854776000 due There are three ways how to call this function. Let's have this table: -[source,cql] +[source,sql] ---- cqlsh> select * from ks.tb; @@ -387,12 +382,12 @@ cqlsh> select * from ks.tb; with schema -[source,cql] +[source,sql] ---- CREATE TABLE ks.tb ( id int PRIMARY KEY, val bigint -) +); ---- Imagine that we wanted to look at `val` values as if they were in mebibytes. We would like to have more human-friendly output in order to not visually divide the values by 1024 in order to get them in respective bigger units. The following function call may take just a column itself as an argument, and it will @@ -403,7 +398,7 @@ automatically convert it. The default source unit for `format_bytes` function is _bytes_, (`B`). ==== -[source,cql] +[source,sql] ---- cqlsh> select format_bytes(val) from ks.tb; @@ -421,7 +416,7 @@ cqlsh> select format_bytes(val) from ks.tb; The second way to call `format_bytes` functions is to specify into what size unit we would like to see all values to be converted to. For example, we want all size to be represented in mebibytes, hence we do: -[source,cql] +[source,sql] ---- cqlsh> select format_bytes(val, 'MiB') from ks.tb; @@ -437,9 +432,9 @@ cqlsh> select format_bytes(val, 'MiB') from ks.tb; ---- Lastly, we can specify a source unit and a target unit. A source unit tells what unit that column is logically of, the target unit tells what unit we want these values to be converted to. For example, -if we know that our column is logically in kibibytes and we want them to be converted into mebibytes, we would do: +if our column values are in kibibytes and we want to convert them to mebibytes, we would perform the following conversion: -[source,cql] +[source,sql] ---- cqlsh> select format_bytes(val, 'Kib', 'MiB') from ks.tb; @@ -470,7 +465,7 @@ Return values can be max of `Double.MAX_VALUE`, If the conversion produces overf The default source unit for `format_time` function is _milliseconds_, (`ms`). ==== -[source,cql] +[source,sql] ---- cqlsh> select format_time(val) from ks.tb; @@ -485,9 +480,9 @@ cqlsh> select format_time(val) from ks.tb; 2.06 m ---- -We may specify what unit we want that value to be converted to, give the column's values are in millseconds: +We may specify what unit we want that value to be converted to, give the column's values are in milliseconds: -[source,cql] +[source,sql] ---- cqlsh> select format_time(val, 'm') from ks.tb; @@ -504,7 +499,7 @@ cqlsh> select format_time(val, 'm') from ks.tb; Lastly, we can specify both source and target values: -[source,cql] +[source,sql] ---- cqlsh> select format_time(val, 's', 'h') from ks.tb; @@ -537,13 +532,13 @@ already in progress. For more information - CASSANDRA-17281, CASSANDRA-18252. For example: -[source,cql] +[source,sql] ---- include::cassandra:example$CQL/function_overload.cql[] ---- -UDFs are susceptible to all of the normal problems with the chosen programming language. -Accordingly, implementations should be safe against null pointer exceptions, illegal arguments, or any other potential source of exceptions. +User-defined functions (UDFs) are prone to the typical issues associated with the programming language they are written in. +Accordingly, implementations should be protected against null pointer exceptions, illegal arguments, or any other potential source of exceptions. An exception during function execution will result in the entire statement failing. Valid queries for UDF use are `SELECT`, `INSERT` and `UPDATE` statements. @@ -558,14 +553,14 @@ Note the use the double dollar-sign syntax to enclose the UDF source code. For example: -[source,cql] +[source,sql] ---- include::cassandra:example$CQL/function_dollarsign.cql[] ---- The implicitly available `udfContext` field (or binding for script UDFs) provides the necessary functionality to create new UDT and tuple values: -[source,cql] +[source,sql] ---- include::cassandra:example$CQL/function_udfcontext.cql[] ---- @@ -598,7 +593,7 @@ include::cassandra:example$BNF/create_function_statement.bnf[] For example: -[source,cql] +[source,sql] ---- include::cassandra:example$CQL/create_function.cql[] ---- @@ -635,7 +630,7 @@ include::cassandra:example$BNF/drop_function_statement.bnf[] For example: -[source,cql] +[source,sql] ---- include::cassandra:example$CQL/drop_function.cql[] ---- @@ -661,14 +656,14 @@ The `count` function can be used to count the rows returned by a query. For example: -[source,cql] +[source,sql] ---- include::cassandra:example$CQL/count.cql[] ---- It also can count the non-null values of a given column: -[source,cql] +[source,sql] ---- include::cassandra:example$CQL/count_nonnull.cql[] ---- @@ -679,7 +674,7 @@ The `max` and `min` functions compute the maximum and the minimum value returned For example: -[source,cql] +[source,sql] ---- include::cassandra:example$CQL/min_max.cql[] ---- @@ -692,7 +687,7 @@ The returned value is of the same type as the input collection elements, so ther For example: -[source,cql] +[source,sql] ---- include::cassandra:example$CQL/sum.cql[] ---- @@ -701,7 +696,7 @@ The returned value is of the same type as the input values, so there is a risk o values exceeds the maximum value that the type can represent. You can use type casting to cast the input values as a type large enough to contain the type. For example: -[source,cql] +[source,sql] ---- include::cassandra:example$CQL/sum_with_cast.cql[] ---- @@ -712,7 +707,7 @@ The `avg` function computes the average of all the values returned by a query fo For example: -[source,cql] +[source,sql] ---- include::cassandra:example$CQL/avg.cql[] ---- @@ -723,7 +718,7 @@ The returned value is of the same type as the input values, which might include For example `collection_avg([1, 2])` returns `1` instead of `1.5`. You can use type casting to cast to a type with the desired decimal precision. For example: -[source,cql] +[source,sql] ---- include::cassandra:example$CQL/avg_with_cast.cql[] ---- @@ -747,7 +742,7 @@ overload can appear after creation of the aggregate. A complete working example for user-defined aggregates (assuming that a keyspace has been selected using the `USE` statement): -[source,cql] +[source,sql] ---- include::cassandra:example$CQL/uda.cql[] ---- @@ -790,7 +785,7 @@ If a `FINALFUNC` is defined, it is the return type of that function. [[drop-aggregate-statement]] === DROP AGGREGATE statement -Dropping an user-defined aggregate function uses the `DROP AGGREGATE` +Dropping a user-defined aggregate function uses the `DROP AGGREGATE` statement: [source, bnf] @@ -800,7 +795,7 @@ include::cassandra:example$BNF/drop_aggregate_statement.bnf[] For instance: -[source,cql] +[source,sql] ---- include::cassandra:example$CQL/drop_aggregate.cql[] ---- @@ -811,3 +806,20 @@ different signature. The `DROP AGGREGATE` command with the optional `IF EXISTS` keywords drops an aggregate if it exists, and does nothing if a function with the signature does not exist. + +[[writetime-and-ttl-functions]] +==== `WRITETIME`, `MINWRITETIME`, `MAXWRITETIME` and `TTL` functions + +These metadata functions are only allowed in `SELECT` statements: `WRITETIME`, `MINWRITETIME`, `MAXWRITETIME` and `TTL`. +The functions take only one argument, a column name, and retrieve metadata stored internally for the column. If the column is a collection or UDT, it's possible to add element +selectors, such as `WRITETTIME(phones[2..4])` or `WRITETTIME(user.name)`. + +* `WRITETIME` stores the timestamp of the value of the column. +* `MINWRITETIME` stores the smallest timestamp of the value of the column. For non-collection and non-UDT columns, `MINWRITETIME` +is equivalent to `WRITETIME`. In the other cases, it returns the smallest timestamp of the values in the column. +* `MAXWRITETIME` stores the largest timestamp of the value of the column. For non-collection and non-UDT columns, `MAXWRITETIME` +is equivalent to `WRITETIME`. In the other cases, it returns the largest timestamp of the values in the column. +* `TTL` stores the remaining time to live (in seconds) for the value of the column if it is set to expire; otherwise the value is `null`. + +The `WRITETIME` and `TTL` functions can be used on multi-cell columns such as non-frozen collections or non-frozen +user-defined types. In that case, the functions will return the list of timestamps or TTLs for each selected cell. diff --git a/doc/modules/cassandra/pages/developing/cql/types.adoc b/doc/modules/cassandra/pages/developing/cql/types.adoc index db46d4c6ac3f..d792e707af98 100644 --- a/doc/modules/cassandra/pages/developing/cql/types.adoc +++ b/doc/modules/cassandra/pages/developing/cql/types.adoc @@ -10,6 +10,7 @@ types]: include::cassandra:example$BNF/cql_type.bnf[] ---- +[[native-types]] == Native types The native types supported by CQL are: @@ -191,6 +192,7 @@ a date context. A `1d` duration is not equal to a `24h` one as the duration type has been created to be able to support daylight saving. +[[collections]] == Collections CQL supports three kinds of collections: `maps`, `sets` and `lists`. The diff --git a/doc/modules/cassandra/partials/masking_functions.adoc b/doc/modules/cassandra/partials/masking_functions.adoc index 528da7d825a6..c7a6f289bf21 100644 --- a/doc/modules/cassandra/partials/masking_functions.adoc +++ b/doc/modules/cassandra/partials/masking_functions.adoc @@ -14,7 +14,7 @@ Examples: | `mask_default(value)` | Replaces its argument by an arbitrary, fixed default value of the same type. -This will be `\***\***` for text values, zero for numeric values, `false` for booleans, etc. +This will be `$$****$$` for text values, zero for numeric values, `false` for booleans, etc. Variable-length multi-valued types such as lists, sets and maps are masked as empty collections. @@ -22,7 +22,7 @@ Fixed-length multi-valued types such as tuples, user-defined types (UDTs) and ve Examples: -`mask_default('Alice')` -> `'\****'` +`mask_default('Alice')` -> `'$$****$$'` `mask_default(123)` -> `0` @@ -43,32 +43,36 @@ Examples: | `mask_inner(value, begin, end, [padding])` | Returns a copy of the first `text`, `varchar` or `ascii` argument, replacing each character except the first and last ones by a padding character. The second and third arguments are the size of the exposed prefix and suffix. -The optional fourth argument is the padding character, `\*` by default. +The optional fourth argument is the padding character, `*` by default. Examples: -`mask_inner('Alice', 1, 2)` -> `'A**ce'` +`mask_inner('Alice', 1, 2)` -> `'A$$**$$ce'` -`mask_inner('Alice', 1, null)` -> `'A****'` +`mask_inner('Alice', 1, null)` -> `'A$$****$$'` -`mask_inner('Alice', null, 2)` -> `'***ce'` +`mask_inner('Alice', null, 2)` -> `'$$***$$ce'` -`mask_inner('Alice', 2, 1, '\#')` -> `'Al##e'` +`mask_inner('Alice', 2, 1, '$$#$$')` -> `'Al##e'` + +`mask_inner('078-05-1120', 0, 4)` -> `'$$*******$$1120'` | `mask_outer(value, begin, end, [padding])` | Returns a copy of the first `text`, `varchar` or `ascii` argument, replacing the first and last character by a padding character. The second and third arguments are the size of the exposed prefix and suffix. -The optional fourth argument is the padding character, `\*` by default. +The optional fourth argument is the padding character, `*` by default. Examples: -`mask_outer('Alice', 1, 2)` -> `'*li**'` +`mask_outer('Alice', 1, 2)` -> `'$$*$$li$$**$$'` + +`mask_outer('Alice', 1, null)` -> `'$$*$$lice'` -`mask_outer('Alice', 1, null)` -> `'*lice'` +`mask_outer('Alice', null, 2)` -> `'Ali$$**$$'` -`mask_outer('Alice', null, 2)` -> `'Ali**'` +`mask_outer('Alice', 2, 1, '$$#$$')` -> `'$$##$$ic$$#$$'` -`mask_outer('Alice', 2, 1, '\#')` -> `'##ic#'` +`mask_outer('11:39:33', 2, 2, '0')` -> `'00:39:00'` | `mask_hash(value, [algorithm])` | Returns a `blob` containing the hash of the first argument. From 02d71cee45aa3b62a49a339b615124b1dabcf53a Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Wed, 19 Mar 2025 15:57:46 +0100 Subject: [PATCH 203/225] Add support for time, date, timestamp types in scalar constraint patch by Stefan Miklosovic; reviewed by Bernardo Botella for CASSANDRA-20274 --- CHANGES.txt | 1 + .../pages/developing/cql/constraints.adoc | 54 +++-- ...AbstractFunctionSatisfiabilityChecker.java | 5 +- .../constraints/ScalarColumnConstraint.java | 27 +-- .../constraints/TimeConstraintsTest.java | 187 ++++++++++++++++++ 5 files changed, 249 insertions(+), 25 deletions(-) create mode 100644 test/unit/org/apache/cassandra/constraints/TimeConstraintsTest.java diff --git a/CHANGES.txt b/CHANGES.txt index b2a3004838cb..ba9c7aae3afd 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Add support for time, date, timestamp types in scalar constraint (CASSANDRA-20274) * Add regular expression constraint (CASSANDRA-20275) * Improve constraints autocompletion (CASSANDRA-20341) * Add JVM version and Cassandra build date to nodetool version -v (CASSANDRA-19721) diff --git a/doc/modules/cassandra/pages/developing/cql/constraints.adoc b/doc/modules/cassandra/pages/developing/cql/constraints.adoc index 7e1893872929..390d6c27a979 100644 --- a/doc/modules/cassandra/pages/developing/cql/constraints.adoc +++ b/doc/modules/cassandra/pages/developing/cql/constraints.adoc @@ -12,7 +12,7 @@ The main syntax to define a constraint is as follows: [source,bnf] ---- -CREATE TABLE keyspace.table ( +CREATE TABLE ks.tb ( name text, i int CHECK (condition) (AND (condition))* ..., @@ -48,7 +48,7 @@ For example, we can define constraints that ensure that i is bigger or equal tha [source,bnf] ---- -CREATE TABLE keyspace.table ( +CREATE TABLE ks.tb ( name text, i int CHECK i < 1000 AND i > 100 ..., @@ -58,13 +58,43 @@ CREATE TABLE keyspace.table ( Altering that constraint can be done with: ---- -ALTER TABLE keyspace.table ALTER i CHECK i >= 500; +ALTER TABLE ks.tb ALTER i CHECK i >= 500; ---- Finally, the constraint can be removed: ---- -ALTER TABLE keyspace.table ALTER i DROP CHECK; +ALTER TABLE ks.tb ALTER i DROP CHECK; +---- + +This constraint also works for `time`, `date` and `timestamp` types. For example, a user +can express constraints like this: + +---- +CREATE TABLE ks.tb + name text, + dob date CHECK dob > '1900-01-01' + ... +) +---- + +Hence, we can enforce that date of birth is later than January 1st, 1900. + +---- +INSERT INTO ks.tb (name, dob) VALUES ( 'Joe Doe', '1899-08-06'); +... [Invalid query] message="Column value does not satisfy value constraint for column 'dob'. It should be dob > '1900-01-01'" +-- this passes as it is > January 1st, 1900 +INSERT INTO ks.tb (name, dob) VALUES ( 'Joe Doe', '1976-12-06'); +---- + +A user can also use ranges on time-related columns with `>` and `<` operators, e.g. + +---- +CREATE TABLE ks.tb + name text, + afternoon time CHECK afternoon >= '12:00:00' AND afternoon <= '23:59:59'; + ... +) ---- === LENGTH CONSTRAINT @@ -74,7 +104,7 @@ Defines a condition that checks the length of text or binary type. For example, we can create a constraint that checks that name can't be longer than 256 characters: ---- -CREATE TABLE keyspace.table ( +CREATE TABLE ks.tb ( name text CHECK LENGTH(name) < 256 ..., ); @@ -83,13 +113,13 @@ CREATE TABLE keyspace.table ( Altering that constraint can be done with: ---- -ALTER TABLE keyspace.table ALTER name LENGTH(name) < 512; +ALTER TABLE ks.tb ALTER name LENGTH(name) < 512; ---- Finally, the constraint can be removed: ---- -ALTER TABLE keyspace.table ALTER name DROP CHECK; +ALTER TABLE ks.tb ALTER name DROP CHECK; ---- === OCTET_LENGTH CONSTRAINT @@ -99,7 +129,7 @@ Defines a condition that checks the size in bytes of text or binary type. For example, we can create a constraint that checks that name can't be bigger than 256 characters: ---- -CREATE TABLE keyspace.table ( +CREATE TABLE ks.tb ( name text CHECK OCTET_LENGTH(name) < 2 ..., ); @@ -107,12 +137,12 @@ CREATE TABLE keyspace.table ( Inserting a valid row: ---- -INSERT INTO keyspace.table (name) VALUES ("f") +INSERT INTO ks.tb (name) VALUES ("f") ---- Inserting an invalid row: ---- -INSERT INTO keyspace.table (name) VALUES ("fooooooo") +INSERT INTO ks.tb (name) VALUES ("fooooooo") ERROR: Column value does not satisfy value constraint for column 'name'. It has a length of 8 and and it should be should be < 2 @@ -152,8 +182,8 @@ A column which has `NOT_NULL` constraint has to be specified in every modificati The constraint can be removed: ---- -ALTER TABLE keyspace.table ALTER col1 DROP CHECK; -ALTER TABLE keyspace.table ALTER col2 DROP CHECK; +ALTER TABLE ks.tb ALTER col1 DROP CHECK; +ALTER TABLE ks.tb ALTER col2 DROP CHECK; ---- We can not remove the value of a column where `NOT_NULL` constraint is present: diff --git a/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionSatisfiabilityChecker.java b/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionSatisfiabilityChecker.java index 33f261535c91..91d448da012a 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionSatisfiabilityChecker.java +++ b/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionSatisfiabilityChecker.java @@ -25,6 +25,7 @@ import java.util.TreeSet; import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.cql3.functions.types.ParseUtils; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.utils.Pair; @@ -146,8 +147,8 @@ else if (firstRelation == NEQ && secondRelation == NEQ) } else { - ByteBuffer firstTermBuffer = columnMetadata.type.fromString(firstTerm); - ByteBuffer secondTermBuffer = columnMetadata.type.fromString(secondTerm); + ByteBuffer firstTermBuffer = columnMetadata.type.fromString(ParseUtils.unquote(firstTerm)); + ByteBuffer secondTermBuffer = columnMetadata.type.fromString(ParseUtils.unquote(secondTerm)); boolean firstSatisfaction = firstRelation.isSatisfiedBy(columnMetadata.type, secondTermBuffer, firstTermBuffer); boolean secondSatisfaction = secondRelation.isSatisfiedBy(columnMetadata.type, firstTermBuffer, secondTermBuffer); diff --git a/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java index 70b33a7569a7..80671a6bf39f 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/ScalarColumnConstraint.java @@ -26,6 +26,7 @@ import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.cql3.functions.types.ParseUtils; import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.ByteType; @@ -37,6 +38,9 @@ import org.apache.cassandra.db.marshal.IntegerType; import org.apache.cassandra.db.marshal.LongType; import org.apache.cassandra.db.marshal.ShortType; +import org.apache.cassandra.db.marshal.SimpleDateType; +import org.apache.cassandra.db.marshal.TimeType; +import org.apache.cassandra.db.marshal.TimestampType; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.schema.ColumnMetadata; @@ -56,7 +60,7 @@ public class ScalarColumnConstraint extends AbstractFunctionConstraint<ScalarCol private static final List<AbstractType<?>> SUPPORTED_TYPES = List.of(ByteType.instance, CounterColumnType.instance, DecimalType.instance, DoubleType.instance, FloatType.instance, Int32Type.instance, IntegerType.instance, LongType.instance, - ShortType.instance); + ShortType.instance, TimeType.instance, SimpleDateType.instance, TimestampType.instance); @VisibleForTesting public static final List<Operator> SUPPORTED_OPERATORS = List.of(EQ, NEQ, GTE, GT, LTE, LT); @@ -91,6 +95,8 @@ public void checkSatisfiability(List<ColumnConstraint<?>> constraints, ColumnMet } } + private ByteBuffer value; + private ScalarColumnConstraint(ColumnIdentifier param, Operator relationType, String term) { super(param, relationType, term); @@ -111,16 +117,6 @@ public List<AbstractType<?>> getSupportedTypes() @Override protected void internalEvaluate(AbstractType<?> valueType, ByteBuffer columnValue) { - ByteBuffer value; - try - { - value = valueType.fromString(term); - } - catch (NumberFormatException exception) - { - throw new ConstraintViolationException(columnName + " and " + term + " need to be numbers."); - } - if (!relationType.isSatisfiedBy(valueType, columnValue, value)) throw new ConstraintViolationException("Column value does not satisfy value constraint for column '" + columnName + "'. " + "It should be " + columnName + " " + relationType + " " + term); @@ -130,6 +126,15 @@ protected void internalEvaluate(AbstractType<?> valueType, ByteBuffer columnValu public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException { validateTypes(columnMetadata); + + try + { + value = columnMetadata.type.fromString(ParseUtils.unquote(term)); + } + catch (Throwable t) + { + throw new ConstraintViolationException("Cannot parse constraint value from " + term + " for column '" + columnName + '\''); + } } @Override diff --git a/test/unit/org/apache/cassandra/constraints/TimeConstraintsTest.java b/test/unit/org/apache/cassandra/constraints/TimeConstraintsTest.java new file mode 100644 index 000000000000..830abb444a8c --- /dev/null +++ b/test/unit/org/apache/cassandra/constraints/TimeConstraintsTest.java @@ -0,0 +1,187 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.constraints; + +import org.junit.Test; + +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.cql3.constraints.ColumnConstraints; +import org.apache.cassandra.cql3.constraints.ConstraintViolationException; +import org.apache.cassandra.cql3.constraints.InvalidConstraintDefinitionException; +import org.apache.cassandra.cql3.constraints.ScalarColumnConstraint.Raw; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.SimpleDateType; +import org.apache.cassandra.db.marshal.TimeType; +import org.apache.cassandra.db.marshal.TimestampType; +import org.apache.cassandra.schema.ColumnMetadata; + +import static java.util.List.of; +import static org.apache.cassandra.cql3.Operator.EQ; +import static org.apache.cassandra.cql3.Operator.GT; +import static org.apache.cassandra.cql3.Operator.GTE; +import static org.apache.cassandra.cql3.Operator.LT; +import static org.apache.cassandra.cql3.Operator.LTE; +import static org.apache.cassandra.cql3.Operator.NEQ; +import static org.apache.cassandra.cql3.functions.types.ParseUtils.quote; +import static org.apache.cassandra.schema.ColumnMetadata.Kind.REGULAR; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public class TimeConstraintsTest +{ + private static final ColumnIdentifier columnIdentifier = new ColumnIdentifier("a_column", false); + private static final ColumnMetadata timeColumn = getColumnOfType(TimeType.instance); + private static final ColumnMetadata dateColumn = getColumnOfType(SimpleDateType.instance); + private static final ColumnMetadata timestampColumn = getColumnOfType(TimestampType.instance); + + @Test + public void testTimeConstraint() + { + evaluateTime(EQ, "12:00:00", "12:00:00"); + evaluateTime(NEQ, "12:00:00", "11:00:00"); + evaluateTime(LT, "12:00:00", "11:00:00"); + evaluateTime(GT, "12:00:00", "13:00:00"); + evaluateTime(GT, "12:00:00", "12:00:00.1234"); + evaluateTime(GTE, "12:00:00", "12:00:00"); + evaluateTime(LTE, "12:00:00", "12:00:00"); + + assertThatThrownBy(() -> evaluateTime(GT, "12:00:00", "01:00:00")) + .isInstanceOf(ConstraintViolationException.class); + + assertThatThrownBy(() -> evaluateTime(LT, "12:00:00", "13:00:00")) + .isInstanceOf(ConstraintViolationException.class); + + assertThatThrownBy(() -> evaluateTime(LT, "-3:00:00", "13:00:00")) + .isInstanceOf(ConstraintViolationException.class) + .hasMessage(cantParse("-3:00:00")); + + evaluate(timeColumn, TimeType.instance, GT, "06:00:00", LT, "15:00:00", "12:00:00"); + + assertThatThrownBy(() -> evaluate(timeColumn, TimeType.instance, GT, "06:00:00", LT, "15:00:00", "18:00:00")) + .isInstanceOf(ConstraintViolationException.class) + .hasMessage("Column value does not satisfy value constraint for column 'a_column'. It should be a_column < '15:00:00'"); + + assertThatThrownBy(() -> evaluate(timeColumn, TimeType.instance, GT, "06:00:00", LT, "03:00:00", "18:00:00")) + .isInstanceOf(InvalidConstraintDefinitionException.class) + .hasMessage("Constraints of scalar are not satisfiable: a_column > '06:00:00', a_column < '03:00:00'"); + } + + @Test + public void testDateConstraint() + { + evaluateDate(EQ, "2000-01-01", "2000-01-01"); + evaluateDate(NEQ, "2000-01-01", "1999-12-31"); + evaluateDate(LT, "2000-01-01", "1999-12-31"); + evaluateDate(GT, "2000-01-01", "2000-01-02"); + evaluateDate(GTE, "2000-01-01", "2000-01-01"); + evaluateDate(LTE, "2000-01-01", "2000-01-01"); + + assertThatThrownBy(() -> evaluateDate(GT, "2000-01-01", "1999-12-31")) + .isInstanceOf(ConstraintViolationException.class); + + assertThatThrownBy(() -> evaluateDate(LT, "2000-01-01", "2000-01-02")) + .isInstanceOf(ConstraintViolationException.class); + + assertThatThrownBy(() -> evaluateDate(LT, "2000-54-01", "13:00:00")) + .isInstanceOf(ConstraintViolationException.class) + .hasMessage(cantParse("2000-54-01")); + + evaluate(dateColumn, SimpleDateType.instance, GT, "2000-01-01", LT, "2000-01-31", "2000-01-10"); + + assertThatThrownBy(() -> evaluate(dateColumn, SimpleDateType.instance, GT, "2000-01-01", LT, "2000-01-31", "2000-02-10")) + .isInstanceOf(ConstraintViolationException.class) + .hasMessage("Column value does not satisfy value constraint for column 'a_column'. It should be a_column < '2000-01-31'"); + + assertThatThrownBy(() -> evaluate(dateColumn, SimpleDateType.instance, GT, "2000-01-31", LT, "2000-01-01", "2000-01-10")) + .isInstanceOf(InvalidConstraintDefinitionException.class) + .hasMessage("Constraints of scalar are not satisfiable: a_column > '2000-01-31', a_column < '2000-01-01'"); + } + + @Test + public void testTimestampConstraint() + { + evaluateTimestamp(EQ, "2025-03-18 12:34:56", "2025-03-18 12:34:56"); + evaluateTimestamp(NEQ, "2025-03-18 12:34:56", "2025-03-18 12:34:55"); + evaluateTimestamp(LT, "2025-03-18 12:34:56", "2025-03-18 12:34:55"); + evaluateTimestamp(GT, "2025-03-18 12:34:56", "2025-03-18 12:34:57"); + evaluateTimestamp(GTE, "2025-03-18 12:34:56", "2025-03-18 12:34:56"); + evaluateTimestamp(LTE, "2025-03-18 12:34:56", "2025-03-18 12:34:56"); + + assertThatThrownBy(() -> evaluateTimestamp(GT, "2025-03-18 12:34:56", "2025-03-18 12:34:55")) + .isInstanceOf(ConstraintViolationException.class); + + assertThatThrownBy(() -> evaluateTimestamp(LT, "2025-03-18 12:34:56", "2025-03-18 12:34:57")) + .isInstanceOf(ConstraintViolationException.class); + + assertThatThrownBy(() -> evaluateTimestamp(LT, "2025-55-18 12:34:56", "13:00:00")) + .isInstanceOf(ConstraintViolationException.class) + .hasMessage(cantParse("2025-55-18 12:34:56")); + + evaluate(timestampColumn, TimestampType.instance, GT, "2025-03-18 12:34:56", LT, "2025-03-18 12:35:56", "2025-03-18 12:35:12"); + + assertThatThrownBy(() -> evaluate(timestampColumn, TimestampType.instance, GT, "2025-03-18 12:34:56", LT, "2025-03-18 12:35:56", "2025-03-18 13:35:12")) + .isInstanceOf(ConstraintViolationException.class) + .hasMessage("Column value does not satisfy value constraint for column 'a_column'. It should be a_column < '2025-03-18 12:35:56'"); + + assertThatThrownBy(() -> evaluate(timestampColumn, TimestampType.instance, GT, "2025-03-18 12:34:56", LT, "2025-03-18 12:33:56", "2025-03-18 12:35:12")) + .isInstanceOf(InvalidConstraintDefinitionException.class) + .hasMessage("Constraints of scalar are not satisfiable: a_column > '2025-03-18 12:34:56', a_column < '2025-03-18 12:33:56'"); + } + + private void evaluate(ColumnMetadata columnMetadata, AbstractType<?> type, Operator operator1, String term1, Operator operator2, String term2, String value) + { + ColumnConstraints constraint = new ColumnConstraints(of(new Raw(columnIdentifier, operator1, quote(term1)).prepare(), + new Raw(columnIdentifier, operator2, quote(term2)).prepare())); + constraint.validate(columnMetadata); + constraint.evaluate(type, type.fromString(value)); + } + + private void evaluateTime(Operator operator, String term, String value) + { + evaluate(TimeType.instance, timeColumn, operator, quote(term), value); + } + + private void evaluateDate(Operator operator, String term, String value) + { + evaluate(SimpleDateType.instance, dateColumn, operator, quote(term), value); + } + + private void evaluateTimestamp(Operator operator, String term, String value) + { + evaluate(TimestampType.instance, timestampColumn, operator, quote(term), value); + } + + private void evaluate(AbstractType<?> type, ColumnMetadata columnMetadata, Operator operator, String term, String value) + { + ColumnConstraints constraint = new ColumnConstraints(of(new Raw(columnIdentifier, operator, term).prepare())); + constraint.validate(columnMetadata); + + constraint.evaluate(type, type.fromString(value)); + } + + private String cantParse(String value) + { + return "Cannot parse constraint value from '" + value + "' for column '" + columnIdentifier + '\''; + } + + private static ColumnMetadata getColumnOfType(AbstractType<?> type) + { + return new ColumnMetadata("a", "b", columnIdentifier, type, -1, REGULAR, null); + } +} From b22c0b1317c010227859cc1cd371e3cd40167ff6 Mon Sep 17 00:00:00 2001 From: mck <mck@apache.org> Date: Tue, 18 Mar 2025 11:34:14 +0100 Subject: [PATCH 204/225] Fix reference nav section in in-tree docs MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit patch by Mick Semb Wever; reviewed by Štefan Miklošovič for CASSANDRA-20451 --- doc/modules/cassandra/nav.adoc | 5 +- .../reference/cql-commands/commands-toc.adoc | 82 +++++++++---------- .../cassandra/pages/reference/index.adoc | 5 +- 3 files changed, 49 insertions(+), 43 deletions(-) diff --git a/doc/modules/cassandra/nav.adoc b/doc/modules/cassandra/nav.adoc index 6a4c7a7a2fe5..7c1a02cfaa94 100644 --- a/doc/modules/cassandra/nav.adoc +++ b/doc/modules/cassandra/nav.adoc @@ -118,8 +118,11 @@ ** xref:reference/index.adoc[] *** xref:reference/cql-commands/commands-toc.adoc[CQL commands] +*** xref:developing/cql/cql_singlefile.html[CQL specification] *** xref:reference/java17.adoc[Java 17] -*** xref:reference/static.adoc[Static columns] +*** xref:reference/native-protocol.adoc[Native Protocol specification] *** xref:reference/sai-virtual-table-indexes.adoc[SAI virtual table] +*** xref:reference/static.adoc[Static columns] +*** xref:reference/vector-data-type.adoc[Vector data type] ** xref:integrating/plugins/index.adoc[] \ No newline at end of file diff --git a/doc/modules/cassandra/pages/reference/cql-commands/commands-toc.adoc b/doc/modules/cassandra/pages/reference/cql-commands/commands-toc.adoc index e1fe0fe0fd35..89ec71787679 100644 --- a/doc/modules/cassandra/pages/reference/cql-commands/commands-toc.adoc +++ b/doc/modules/cassandra/pages/reference/cql-commands/commands-toc.adoc @@ -4,123 +4,123 @@ This section describes the Cassandra Query Language (CQL) commands supported by the {product} database. ''' -xref:cql-commands/alter-keyspace.adoc[ALTER KEYSPACE] :: +xref:reference/cql-commands/alter-keyspace.adoc[ALTER KEYSPACE] :: Changes keyspace replication strategy and enables or disables commit log. -xref:cql-commands/alter-materialized-view.adoc[ALTER MATERIALIZED VIEW] :: +xref:reference/cql-commands/alter-materialized-view.adoc[ALTER MATERIALIZED VIEW] :: Changes the table properties of a materialized view. -xref:cql-commands/alter-role.adoc[ALTER ROLE] :: +xref:reference/cql-commands/alter-role.adoc[ALTER ROLE] :: Changes password and sets superuser or login options. -xref:cql-commands/alter-table.adoc[ALTER TABLE] :: +xref:reference/cql-commands/alter-table.adoc[ALTER TABLE] :: Modifies the columns and properties of a table, or modify -xref:cql-commands/alter-type.adoc[ALTER TYPE] :: +xref:reference/cql-commands/alter-type.adoc[ALTER TYPE] :: Modifies an existing user-defined type (UDT). -xref:reference:cql-commands/alter-user.adoc[ALTER USER (Deprecated)] :: +xref:reference/cql-commands/alter-user.adoc[ALTER USER (Deprecated)] :: Deprecated. Alter existing user options. -xref:reference:cql-commands/batch.adoc[BATCH] :: +xref:reference/cql-commands/batch.adoc[BATCH] :: Applies multiple data modification language (DML) statements with atomicity and/or in isolation. -xref:reference:cql-commands/create-aggregate.adoc[CREATE AGGREGATE] :: +xref:reference/cql-commands/create-aggregate.adoc[CREATE AGGREGATE] :: Defines a user-defined aggregate. -xref:reference:cql-commands/create-custom-index.adoc[CREATE CUSTOM INDEX] :: +xref:reference/cql-commands/create-custom-index.adoc[CREATE CUSTOM INDEX] :: Creates a storage-attached index. -xref:reference:cql-commands/create-function.adoc[CREATE FUNCTION] :: +xref:reference/cql-commands/create-function.adoc[CREATE FUNCTION] :: Creates custom function to execute user provided code. -xref:reference:cql-commands/create-index.adoc[CREATE INDEX] :: +xref:reference/cql-commands/create-index.adoc[CREATE INDEX] :: Defines a new index for a single column of a table. -xref:reference:cql-commands/create-keyspace.adoc[CREATE KEYSPACE] :: +xref:reference/cql-commands/create-keyspace.adoc[CREATE KEYSPACE] :: -xref:reference:cql-commands/create-materialized-view.adoc[CREATE MATERIALIZED VIEW] :: +xref:reference/cql-commands/create-materialized-view.adoc[CREATE MATERIALIZED VIEW] :: Optimizes read requests and eliminates the need for multiple write requests by duplicating data from a base table. -xref:reference:cql-commands/create-role.adoc[CREATE ROLE] :: +xref:reference/cql-commands/create-role.adoc[CREATE ROLE] :: Creates a cluster wide database object used for access control. -xref:reference:cql-commands/create-table.adoc[CREATE TABLE] :: +xref:reference/cql-commands/create-table.adoc[CREATE TABLE] :: Creates a new table. -xref:reference:cql-commands/create-type.adoc[CREATE TYPE] :: +xref:reference/cql-commands/create-type.adoc[CREATE TYPE] :: Creates a custom data type in the keyspace that contains one or more fields of related information. -xref:reference:cql-commands/create-user.adoc[CREATE USER (Deprecated)] :: +xref:reference/cql-commands/create-user.adoc[CREATE USER (Deprecated)] :: Deprecated. Creates a new user. -xref:reference:cql-commands/delete.adoc[DELETE] :: +xref:reference/cql-commands/delete.adoc[DELETE] :: Removes data from one or more columns or removes the entire row -xref:reference:cql-commands/drop-aggregate.adoc[DROP AGGREGATE] :: +xref:reference/cql-commands/drop-aggregate.adoc[DROP AGGREGATE] :: Deletes a user-defined aggregate from a keyspace. -xref:reference:cql-commands/drop-function.adoc[DROP FUNCTION] :: +xref:reference/cql-commands/drop-function.adoc[DROP FUNCTION] :: Deletes a user-defined function (UDF) from a keyspace. -xref:reference:cql-commands/drop-index.adoc[DROP INDEX] :: +xref:reference/cql-commands/drop-index.adoc[DROP INDEX] :: Removes an index from a table. -xref:reference:cql-commands/drop-keyspace.adoc[DROP KEYSPACE] :: +xref:reference/cql-commands/drop-keyspace.adoc[DROP KEYSPACE] :: Removes the keyspace. -xref:reference:cql-commands/drop-materialized-view.adoc[DROP MATERIALIZED VIEW] :: +xref:reference/cql-commands/drop-materialized-view.adoc[DROP MATERIALIZED VIEW] :: Removes the named materialized view. -xref:reference:cql-commands/drop-role.adoc[DROP ROLE] :: +xref:reference/cql-commands/drop-role.adoc[DROP ROLE] :: Removes a role. -xref:reference:cql-commands/drop-table.adoc[DROP TABLE] :: +xref:reference/cql-commands/drop-table.adoc[DROP TABLE] :: Removes the table. -xref:reference:cql-commands/drop-type.adoc[DROP TYPE] :: +xref:reference/cql-commands/drop-type.adoc[DROP TYPE] :: Drop a user-defined type. -xref:reference:cql-commands/drop-user.adoc[DROP USER (Deprecated)] :: +xref:reference/cql-commands/drop-user.adoc[DROP USER (Deprecated)] :: Removes a user. -xref:reference:cql-commands/grant.adoc[GRANT] :: +xref:reference/cql-commands/grant.adoc[GRANT] :: Allow access to database resources. -xref:reference:cql-commands/insert.adoc[INSERT] :: +xref:reference/cql-commands/insert.adoc[INSERT] :: Inserts an entire row or upserts data into existing rows. -xref:reference:cql-commands/list-permissions.adoc[LIST PERMISSIONS] :: +xref:reference/cql-commands/list-permissions.adoc[LIST PERMISSIONS] :: Lists permissions on resources. -xref:reference:cql-commands/list-roles.adoc[LIST ROLES] :: +xref:reference/cql-commands/list-roles.adoc[LIST ROLES] :: Lists roles and shows superuser and login status. -xref:reference:cql-commands/list-users.adoc[LIST USERS (Deprecated)] :: +xref:reference/cql-commands/list-users.adoc[LIST USERS (Deprecated)] :: Lists existing internal authentication users and their superuser status. -xref:reference:cql-commands/restrict.adoc[RESTRICT] :: +xref:reference/cql-commands/restrict.adoc[RESTRICT] :: Denies the permission on a resource, even if the role is directly granted or inherits permissions. -xref:reference:cql-commands/restrict-rows.adoc[RESTRICT ROWS] :: +xref:reference/cql-commands/restrict-rows.adoc[RESTRICT ROWS] :: Configures the column used for row-level access control. -xref:reference:cql-commands/revoke.adoc[REVOKE] :: +xref:reference/cql-commands/revoke.adoc[REVOKE] :: Removes privileges on database objects from roles. -xref:reference:cql-commands/select.adoc[SELECT] :: +xref:reference/cql-commands/select.adoc[SELECT] :: Returns data from a table. -xref:reference:cql-commands/truncate.adoc[TRUNCATE] :: +xref:reference/cql-commands/truncate.adoc[TRUNCATE] :: Removes all data from a table. -xref:reference:cql-commands/unrestrict.adoc[UNRESTRICT] :: +xref:reference/cql-commands/unrestrict.adoc[UNRESTRICT] :: Removes a restriction from a role. -xref:reference:cql-commands/unrestrict-rows.adoc[UNRESTRICT ROWS] :: +xref:reference/cql-commands/unrestrict-rows.adoc[UNRESTRICT ROWS] :: Removes the column definition for row-level access control. -xref:reference:cql-commands/update.adoc[UPDATE] :: +xref:reference/cql-commands/update.adoc[UPDATE] :: Modifies one or more column values to a row in a table. -xref:reference:cql-commands/use.adoc[USE] :: +xref:reference/cql-commands/use.adoc[USE] :: Selects the keyspace for the current client session. \ No newline at end of file diff --git a/doc/modules/cassandra/pages/reference/index.adoc b/doc/modules/cassandra/pages/reference/index.adoc index 6c61a37925f5..8e09e8f8d804 100644 --- a/doc/modules/cassandra/pages/reference/index.adoc +++ b/doc/modules/cassandra/pages/reference/index.adoc @@ -1,6 +1,9 @@ = Reference * xref:reference/cql-commands/commands-toc.adoc[CQL commands] +* xref:developing/cql/cql_singlefile.html[CQL specification] * xref:reference/java17.adoc[Java 17] +* xref:reference/native-protocol.adoc[Native Protocol specification] +* xref:reference/sai-virtual-table-indexes.adoc[SAI virtual table] * xref:reference/static.adoc[Static columns] -* xref:reference/sai-virtual-table-indexes.adoc[SAI virtual table] \ No newline at end of file +* xref:reference/vector-data-type.adoc[Vector data type] \ No newline at end of file From 91d78afc2b595b2b8c17c6843aea836f33956c48 Mon Sep 17 00:00:00 2001 From: Brandon Williams <brandonwilliams@apache.org> Date: Fri, 21 Mar 2025 09:53:13 -0500 Subject: [PATCH 205/225] Add CASSANDRA-15452 to CHANGES --- CHANGES.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGES.txt b/CHANGES.txt index bb53fe01076b..14174d858ba7 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -3,6 +3,7 @@ * Improve error messages when initializing auth classes (CASSANDRA-20368) * Prioritize legacy 2i over SAI for columns with multiple indexes (CASSANDRA-20334) * Ensure only offline tools can build IntervalTrees without first/last key fields (CASSANDRA-20407) + * Improve disk access patterns during compaction and range reads (CASSANDRA-15452) * Improve IntervalTree build throughput (CASSANDRA-19596) * Avoid limit on RFP fetch in the case of an unresolved static row (CASSANDRA-20323) * Include materialized views to the output of DESCRIBE TABLE statements (CASSANDRA-20365) From ea4ff0e9663f8de01a56ac6b9e9ffc2bc363dc5e Mon Sep 17 00:00:00 2001 From: David Capwell <dcapwell@apache.org> Date: Fri, 21 Mar 2025 11:45:33 -0700 Subject: [PATCH 206/225] Expand TableWalk tests to include collections and add support for += and -= for these types patch by David Capwell; reviewed by Abe Ratnofsky for CASSANDRA-20460 --- .../test/cql3/SingleNodeTableWalkTest.java | 70 ++++++-- .../harry/model/ASTSingleTableModel.java | 125 +++++++++++++- .../harry/model/ASTSingleTableModelTest.java | 128 ++++++++++++++- .../harry/model/BytesPartitionState.java | 27 ++- .../org/apache/cassandra/cql3/KnownIssue.java | 2 + .../cql3/ast/AssignmentOperator.java | 31 ++-- .../cassandra/cql3/ast/Conditional.java | 2 +- .../cassandra/cql3/ast/CreateIndexDDL.java | 2 +- .../cql3/ast/ExpressionEvaluator.java | 154 ++++++++++-------- .../apache/cassandra/cql3/ast/Mutation.java | 6 + .../org/apache/cassandra/cql3/ast/Select.java | 7 + .../cassandra/cql3/ast/StandardVisitors.java | 2 +- .../apache/cassandra/utils/ASTGenerators.java | 20 ++- .../utils/AbstractTypeGenerators.java | 78 +++++++-- .../apache/cassandra/utils/Generators.java | 31 +++- .../cassandra/utils/ImmutableUniqueList.java | 24 ++- 16 files changed, 570 insertions(+), 139 deletions(-) diff --git a/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTableWalkTest.java b/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTableWalkTest.java index 2ba02ae76940..755d479e92e7 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTableWalkTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTableWalkTest.java @@ -66,10 +66,11 @@ import org.apache.cassandra.utils.ASTGenerators; import org.apache.cassandra.utils.AbstractTypeGenerators; import org.apache.cassandra.utils.AbstractTypeGenerators.TypeGenBuilder; +import org.apache.cassandra.utils.AbstractTypeGenerators.TypeKind; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.CassandraGenerators.TableMetadataBuilder; +import org.apache.cassandra.utils.Generators; import org.apache.cassandra.utils.ImmutableUniqueList; -import org.quicktheories.generators.SourceDSL; import static accord.utils.Property.commands; import static accord.utils.Property.stateful; @@ -78,6 +79,17 @@ public class SingleNodeTableWalkTest extends StatefulASTBase { + private static final Gen<Gen<Boolean>> BOOLEAN_DISTRIBUTION = Gens.bools().mixedDistribution(); + //TODO (coverage): COMPOSITE, DYNAMIC_COMPOSITE + private static final Gen<Gen<TypeKind>> TYPE_KIND_DISTRIBUTION = Gens.mixedDistribution(TypeKind.PRIMITIVE, + TypeKind.SET, TypeKind.LIST, TypeKind.MAP, + TypeKind.TUPLE, TypeKind.UDT, + TypeKind.VECTOR + ); + private static final Gen<Gen<AbstractType<?>>> PRIMITIVE_DISTRIBUTION = Gens.mixedDistribution(AbstractTypeGenerators.knownPrimitiveTypes() + .stream() + .filter(t -> !AbstractTypeGenerators.isUnsafeEquality(t)) + .collect(Collectors.toList())); private static final Logger logger = LoggerFactory.getLogger(SingleNodeTableWalkTest.class); protected void preCheck(Cluster cluster, Property.StatefulBuilder builder) @@ -88,10 +100,20 @@ protected void preCheck(Cluster cluster, Property.StatefulBuilder builder) // CQL_DEBUG_APPLY_OPERATOR = true; } - protected TypeGenBuilder supportedTypes() + protected TypeGenBuilder supportedTypes(RandomSource rs) { - return AbstractTypeGenerators.withoutUnsafeEquality(AbstractTypeGenerators.builder() - .withTypeKinds(AbstractTypeGenerators.TypeKind.PRIMITIVE)); + return AbstractTypeGenerators.builder() + .withTypeKinds(Generators.fromGen(TYPE_KIND_DISTRIBUTION.next(rs))) + .withPrimitives(Generators.fromGen(PRIMITIVE_DISTRIBUTION.next(rs))) + .withUserTypeFields(AbstractTypeGenerators.UserTypeFieldsGen.simpleNames()) + .withMaxDepth(1); + } + + protected TypeGenBuilder supportedPrimaryColumnTypes(RandomSource rs) + { + return AbstractTypeGenerators.builder() + .withTypeKinds(TypeKind.PRIMITIVE) + .withPrimitives(Generators.fromGen(PRIMITIVE_DISTRIBUTION.next(rs))); } protected List<CreateIndexDDL.Indexer> supportedIndexers() @@ -206,7 +228,7 @@ protected List<CreateIndexDDL.Indexer> supportedIndexers() builder.value(pk, key.bufferAt(pks.indexOf(pk))); - List<Symbol> searchableColumns = state.nonPartitionColumns; + List<Symbol> searchableColumns = state.searchableNonPartitionColumns; Symbol symbol = rs.pick(searchableColumns); TreeMap<ByteBuffer, List<BytesPartitionState.PrimaryKey>> universe = state.model.index(ref, symbol); @@ -363,7 +385,8 @@ protected TableMetadata defineTable(RandomSource rs, String ks) .withCompression()) .withKeyspaceName(ks).withTableName("tbl") .withSimpleColumnNames() - .withDefaultTypeGen(supportedTypes()) + .withDefaultTypeGen(supportedTypes(rs)) + .withPrimaryColumnTypeGen(supportedPrimaryColumnTypes(rs)) .withPartitioner(Murmur3Partitioner.instance) .build()) .next(rs); @@ -393,7 +416,7 @@ public class State extends CommonState { protected final LinkedHashMap<Symbol, IndexedColumn> indexes; private final Gen<Mutation> mutationGen; - private final List<Symbol> nonPartitionColumns; + private final List<Symbol> searchableNonPartitionColumns; private final List<Symbol> searchableColumns; private final List<Symbol> nonPkIndexedColumns; @@ -424,7 +447,8 @@ public State(RandomSource rs, Cluster cluster) .withoutTransaction() .withoutTtl() .withoutTimestamp() - .withPartitions(SourceDSL.arbitrary().pick(uniquePartitions)); + .withPartitions(Generators.fromGen(Gens.mixedDistribution(uniquePartitions).next(rs))) + .withColumnExpressions(e -> e.withOperators(Generators.fromGen(BOOLEAN_DISTRIBUTION.next(rs)))); if (IGNORED_ISSUES.contains(KnownIssue.SAI_EMPTY_TYPE)) { model.factory.regularAndStaticColumns.stream() @@ -438,16 +462,30 @@ public State(RandomSource rs, Cluster cluster) } this.mutationGen = toGen(mutationGenBuilder.build()); - nonPartitionColumns = ImmutableList.<Symbol>builder() - .addAll(model.factory.clusteringColumns) - .addAll(model.factory.staticColumns) - .addAll(model.factory.regularColumns) - .build(); + var nonPartitionColumns = ImmutableList.<Symbol>builder() + .addAll(model.factory.clusteringColumns) + .addAll(model.factory.staticColumns) + .addAll(model.factory.regularColumns) + .build(); + searchableNonPartitionColumns = nonPartitionColumns.stream() + .filter(this::isSearchable) + .collect(Collectors.toList()); nonPkIndexedColumns = nonPartitionColumns.stream() .filter(indexes::containsKey) .collect(Collectors.toList()); - searchableColumns = metadata.partitionKeyColumns().size() > 1 ? model.factory.selectionOrder : nonPartitionColumns; + searchableColumns = (metadata.partitionKeyColumns().size() > 1 ? model.factory.selectionOrder : nonPartitionColumns) + .stream() + .filter(this::isSearchable) + .collect(Collectors.toList()); + } + + private boolean isSearchable(Symbol symbol) + { + // See org.apache.cassandra.cql3.Operator.validateFor + // multi cell collections can only be searched if you search their elements, not the collection as a whole + //TODO (coverage): can you query for UDT fields? its a single cell so you "should"? + return !(symbol.type().isMultiCell() && (symbol.type().isCollection() || symbol.type().isUDT())); } @Override @@ -523,6 +561,8 @@ private List<Symbol> multiColumnQueryColumns() List<Symbol> allowedColumns = searchableColumns; if (hasMultiNodeMultiColumnAllowFilteringWithLocalWritesIssue()) allowedColumns = nonPkIndexedColumns; + if (IGNORED_ISSUES.contains(KnownIssue.SAI_AND_VECTOR_COLUMNS) && !indexes.isEmpty()) + allowedColumns = allowedColumns.stream().filter(s -> !s.type().isVector()).collect(Collectors.toList()); return allowedColumns; } @@ -533,7 +573,7 @@ private boolean hasMultiNodeMultiColumnAllowFilteringWithLocalWritesIssue() public boolean allowPartitionQuery() { - return !(model.isEmpty() || nonPartitionColumns.isEmpty()); + return !(model.isEmpty() || searchableNonPartitionColumns.isEmpty()); } @Override diff --git a/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModel.java b/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModel.java index 4c0a5f5cf51a..d2fbb6edcca0 100644 --- a/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModel.java +++ b/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModel.java @@ -21,6 +21,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; +import java.util.BitSet; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -43,13 +44,16 @@ import com.google.common.collect.Sets; import accord.utils.Invariants; +import org.apache.cassandra.cql3.ast.AssignmentOperator; import org.apache.cassandra.cql3.ast.Conditional; import org.apache.cassandra.cql3.ast.Conditional.Where.Inequality; import org.apache.cassandra.cql3.ast.Element; import org.apache.cassandra.cql3.ast.Expression; import org.apache.cassandra.cql3.ast.ExpressionEvaluator; import org.apache.cassandra.cql3.ast.FunctionCall; +import org.apache.cassandra.cql3.ast.Literal; import org.apache.cassandra.cql3.ast.Mutation; +import org.apache.cassandra.cql3.ast.Operator; import org.apache.cassandra.cql3.ast.Select; import org.apache.cassandra.cql3.ast.StandardVisitors; import org.apache.cassandra.cql3.ast.Symbol; @@ -244,7 +248,10 @@ public void update(Mutation.Update update) // static columns to add in. If we are doing something like += to a row that doesn't exist, we still update statics... Map<Symbol, ByteBuffer> write = new HashMap<>(); for (Symbol col : Sets.intersection(factory.staticColumns.asSet(), set.keySet())) - write.put(col, eval(set.get(col))); + { + ByteBuffer current = partition.staticRow().get(col); + write.put(col, eval(col, current, set.get(col))); + } partition.setStaticColumns(write); } // table has clustering but non are in the write, so only pk/static can be updated @@ -254,7 +261,10 @@ public void update(Mutation.Update update) { Map<Symbol, ByteBuffer> write = new HashMap<>(); for (Symbol col : Sets.intersection(factory.regularColumns.asSet(), set.keySet())) - write.put(col, eval(set.get(col))); + { + ByteBuffer current = partition.get(cd, col); + write.put(col, eval(col, current, set.get(col))); + } partition.setColumns(cd, write, false); } @@ -493,6 +503,45 @@ private static void validateAnyOrder(ImmutableUniqueList<Symbol> columns, Set<Ro if (actual.isEmpty()) sb.append("No rows returned"); else sb.append("Missing rows:\n").append(table(columns, missing)); } + if (!unexpected.isEmpty() && unexpected.size() == missing.size()) + { + // good chance a column differs + StringBuilder finalSb = sb; + Runnable runOnce = new Runnable() + { + boolean ran = false; + @Override + public void run() + { + if (ran) return; + finalSb.append("\nPossible column conflicts:"); + ran = true; + } + }; + for (var e : missing) + { + Row smallest = null; + BitSet smallestDiff = null; + for (var a : unexpected) + { + BitSet diff = e.diff(a); + if (smallestDiff == null || diff.cardinality() < smallestDiff.cardinality()) + { + smallest = a; + smallestDiff = diff; + } + } + // if every column differs then ignore + if (smallestDiff.cardinality() == e.values.length) + continue; + runOnce.run(); + sb.append("\n\tExpected: ").append(e); + sb.append("\n\tDiff (expected over actual):\n"); + Row eSmall = e.select(smallestDiff); + Row aSmall = smallest.select(smallestDiff); + sb.append(table(eSmall.columns, Arrays.asList(eSmall, aSmall))); + } + } if (sb != null) { sb.append("\nExpected:\n").append(table(columns, expected)); @@ -731,7 +780,7 @@ private static boolean matches(ByteBuffer value, List<? extends Expression> cond for (Expression e : conditions) { ByteBuffer expected = eval(e); - if (expected.equals(value)) + if (expected != null && expected.equals(value)) return true; } return false; @@ -893,13 +942,41 @@ private List<Clustering<ByteBuffer>> keys(Map<Symbol, List<? extends Expression> return current.stream().map(BufferClustering::new).collect(Collectors.toList()); } + private static ByteBuffer eval(Symbol col, @Nullable ByteBuffer current, Expression e) + { + if (!(e instanceof AssignmentOperator)) return eval(e); + // multi cell collections have the property that they do update even if the current value is null + boolean isFancy = col.type().isCollection() && col.type().isMultiCell(); + if (current == null && !isFancy) return null; // null + ? == null + var assignment = (AssignmentOperator) e; + if (isFancy && current == null) + { + return assignment.kind == AssignmentOperator.Kind.SUBTRACT + // if it doesn't exist, then there is nothing to subtract + ? null + : eval(assignment.right); + } + switch (assignment.kind) + { + case ADD: + return eval(new Operator(Operator.Kind.ADD, new Literal(current, e.type()), assignment.right)); + case SUBTRACT: + return eval(new Operator(Operator.Kind.SUBTRACT, new Literal(current, e.type()), assignment.right)); + default: + throw new UnsupportedOperationException(assignment.kind + ": " + assignment.toCQL()); + } + } + + @Nullable private static ByteBuffer eval(Expression e) { - return ExpressionEvaluator.tryEvalEncoded(e).get(); + return ExpressionEvaluator.evalEncoded(e); } private static class Row { + private static final Row EMPTY = new Row(ImmutableUniqueList.empty(), ByteBufferUtil.EMPTY_ARRAY); + private final ImmutableUniqueList<Symbol> columns; private final ByteBuffer[] values; @@ -907,6 +984,8 @@ private Row(ImmutableUniqueList<Symbol> columns, ByteBuffer[] values) { this.columns = columns; this.values = values; + if (columns.size() != values.length) + throw new IllegalArgumentException("Columns " + columns + " should have the same size as values, but had " + values.length); } public String asCQL(Symbol symbol) @@ -914,7 +993,9 @@ public String asCQL(Symbol symbol) int offset = columns.indexOf(symbol); assert offset >= 0; ByteBuffer b = values[offset]; - return (b == null || ByteBufferUtil.EMPTY_BYTE_BUFFER.equals(b)) ? "null" : symbol.type().asCQL3Type().toCQLLiteral(b); + if (b == null) return "null"; + if (ByteBufferUtil.EMPTY_BYTE_BUFFER.equals(b)) return "<empty>"; + return symbol.type().asCQL3Type().toCQLLiteral(b); } public List<String> asCQL() @@ -925,6 +1006,40 @@ public List<String> asCQL() return human; } + public BitSet diff(Row other) + { + if (!columns.equals(other.columns)) + throw new UnsupportedOperationException("Columns do not match: expected " + columns + " but given " + other.columns); + int maxLength = Math.max(values.length, other.values.length); + int minLength = Math.min(values.length, other.values.length); + BitSet set = new BitSet(maxLength); + for (int i = 0; i < minLength; i++) + { + ByteBuffer a = values[i]; + ByteBuffer b = other.values[i]; + if (!Objects.equals(a, b)) + set.set(i); + } + for (int i = minLength; i < maxLength; i++) + set.set(i); + return set; + } + + public Row select(BitSet selection) + { + if (selection.isEmpty()) return EMPTY; + var names = ImmutableUniqueList.<Symbol>builder(selection.cardinality()); + ByteBuffer[] copy = new ByteBuffer[selection.cardinality()]; + int offset = 0; + for (int i = 0; i < this.values.length; i++) + { + if (!selection.get(i)) continue; + names.add(this.columns.get(i)); + copy[offset++] = this.values[i]; + } + return new Row(names.build(), copy); + } + @Override public boolean equals(Object o) { diff --git a/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModelTest.java b/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModelTest.java index 16f4d6681934..a04425f82722 100644 --- a/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModelTest.java +++ b/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModelTest.java @@ -23,6 +23,8 @@ import java.util.Arrays; import java.util.EnumSet; import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.TreeMap; import java.util.function.BooleanSupplier; import java.util.function.Consumer; @@ -34,9 +36,11 @@ import com.google.common.collect.ImmutableList; import org.junit.Test; +import org.apache.cassandra.cql3.ast.AssignmentOperator; import org.apache.cassandra.cql3.ast.Bind; import org.apache.cassandra.cql3.ast.Conditional.Where.Inequality; import org.apache.cassandra.cql3.ast.FunctionCall; +import org.apache.cassandra.cql3.ast.Literal; import org.apache.cassandra.cql3.ast.Mutation; import org.apache.cassandra.cql3.ast.Select; import org.apache.cassandra.cql3.ast.Symbol; @@ -48,7 +52,10 @@ import org.apache.cassandra.db.marshal.InetAddressType; import org.apache.cassandra.db.marshal.Int32Type; import org.apache.cassandra.db.marshal.LexicalUUIDType; +import org.apache.cassandra.db.marshal.ListType; +import org.apache.cassandra.db.marshal.MapType; import org.apache.cassandra.db.marshal.ReversedType; +import org.apache.cassandra.db.marshal.SetType; import org.apache.cassandra.db.marshal.ShortType; import org.apache.cassandra.db.marshal.TimestampType; import org.apache.cassandra.dht.Murmur3Partitioner; @@ -67,6 +74,9 @@ public class ASTSingleTableModelTest private static final EnumSet<Inequality> RANGE_INEQUALITY = EnumSet.of(Inequality.LESS_THAN, Inequality.LESS_THAN_EQ, Inequality.GREATER_THAN, Inequality.GREATER_THAN_EQ); + public static final ListType<Integer> LIST_INT = ListType.getInstance(Int32Type.instance, true); + public static final SetType<Integer> SET_INT = SetType.getInstance(Int32Type.instance, true); + public static final MapType<Integer, Integer> MAP_INT = MapType.getInstance(Int32Type.instance, Int32Type.instance, true); @Test public void singlePartition() @@ -296,12 +306,12 @@ public void nullColumnSelect() String pk0 = "'e44b:bdaf:aeb:f68b:1cff:ecbd:8b54:2295'"; ByteBuffer pk0BB = InetAddressType.instance.asCQL3Type().fromCQLLiteral(pk0); - Short row1 = Short.valueOf((short) -14407); + Short row1 = (short) -14407; ByteBuffer row1BB = ShortType.instance.decompose(row1); String row1V1 = "0x00000000000049008a00000000000000"; ByteBuffer row1V1BB = LexicalUUIDType.instance.asCQL3Type().fromCQLLiteral(row1V1); - Short row2 = Short.valueOf((short) ((short) 18175 - (short) 23847)); + Short row2 = (short) ((short) 18175 - (short) 23847); ByteBuffer row2BB = ShortType.instance.decompose(row2); String row2V0 = "'1989-01-11T15:00:30.950Z'"; ByteBuffer row2V0BB = TimestampType.instance.asCQL3Type().fromCQLLiteral(row2V0); @@ -324,7 +334,7 @@ public void nullColumnSelect() .build()); model.validate(new ByteBuffer[][]{ new ByteBuffer[]{ pk0BB, row1BB, null, row1V1BB } }, selectPk); - model.validate(new ByteBuffer[0][], selectColumn); + model.validate(EMPTY, selectColumn); model.update(Mutation.insert(metadata) @@ -339,7 +349,7 @@ public void nullColumnSelect() new ByteBuffer[]{ pk0BB, row1BB, null, row1V1BB }, }, selectPk); - model.validate(new ByteBuffer[0][], selectColumn); + model.validate(EMPTY, selectColumn); } @Test @@ -565,6 +575,116 @@ public void tokenEqIncludesEmptyPartition() .build()); } + @Test + public void assignmentOperator() + { + // not testing if assignment / operators are "corrrect", other tests can cover that + // the goal of this test is to test the plumbing and null handling within the model + TableMetadata metadata = defaultTable() + .addPartitionKeyColumn("pk", Int32Type.instance) + .addStaticColumn("s", Int32Type.instance) + .addRegularColumn("r", Int32Type.instance) + .build(); + ASTSingleTableModel model = new ASTSingleTableModel(metadata); + + // pk=0 doesn't exist, so s/r are null; so the operation should end with a null... this shouldn't create the partition + model.update(Mutation.update(metadata) + .value("pk", 0) + .set("s", subtract(42)) + .set("r", subtract(42)) + .build()); + + model.validate(EMPTY, Select.builder(metadata).build()); + + model.update(Mutation.insert(metadata).value("pk", 0).value("s", 40).value("r", 40).build()); + model.update(Mutation.update(metadata) + .value("pk", 0) + .set("s", subtract(42)) + .set("r", subtract(42)) + .build()); + + model.validate(rows(row(metadata, 0, -2, -2)), Select.builder(metadata).build()); + } + + @Test + public void assignmentOperatorMultiCellCollections() + { + // not testing if assignment / operators are "corrrect", other tests can cover that + // the goal of this test is to test the plumbing and null handling within the model + TableMetadata metadata = defaultTable() + .addPartitionKeyColumn("pk", Int32Type.instance) + .addStaticColumn("s0", LIST_INT) + .addStaticColumn("r0", LIST_INT) + .addStaticColumn("s1", SET_INT) + .addStaticColumn("r1", SET_INT) + .addStaticColumn("s2", MAP_INT) + .addStaticColumn("r2", MAP_INT) + .build(); + ASTSingleTableModel model = new ASTSingleTableModel(metadata); + + // pk=0 doesn't exist, so s/r are null; but these are multi cell collections, so the update happens! + model.update(Mutation.update(metadata) + .value("pk", 0) + .set("s0", add(List.of(42))) + .set("r0", add(List.of(42))) + .set("s1", add(Set.of(42))) + .set("r1", add(Set.of(42))) + .set("s2", add(Map.of(42, 42))) + .set("r2", add(Map.of(42, 42))) + .build()); + + // Expected: + //pk | r0 | r1 | r2 | s0 | s1 | s2 + //0 | [42] | {42} | {42: 42} | [42] | {42} | {42: 42} + model.validate(rows(row(metadata, 0, List.of(42), Set.of(42), Map.of(42, 42), List.of(42), Set.of(42), Map.of(42, 42))), Select.builder(metadata).build()); + + // add to existing + model.update(Mutation.update(metadata) + .value("pk", 0) + .set("s0", add(List.of(42))) + .set("r0", add(List.of(42))) + .set("s1", add(Set.of(0))) + .set("r1", add(Set.of(0))) + .set("s2", add(Map.of(42, 0))) + .set("r2", add(Map.of(42, 0))) + .build()); + model.validate(rows(row(metadata, 0, List.of(42, 42), Set.of(0, 42), Map.of(42, 0), List.of(42, 42), Set.of(0, 42), Map.of(42, 0))), Select.builder(metadata).build()); + } + + private static ByteBuffer[][] rows(ByteBuffer[]... rows) + { + return rows; + } + + private static ByteBuffer[] row(TableMetadata metadata, Object... values) + { + ByteBuffer[] row = new ByteBuffer[values.length]; + var it = metadata.allColumnsInSelectOrder(); + for (int i = 0; i < values.length && it.hasNext(); i++) + row[i] = it.next().type.decomposeUntyped(values[i]); + return row; + } + + private static AssignmentOperator subtract(int value) + { + return new AssignmentOperator(AssignmentOperator.Kind.SUBTRACT, Literal.of(value)); + } + + private static AssignmentOperator add(List<Integer> value) + { + return new AssignmentOperator(AssignmentOperator.Kind.ADD, new Literal(value, LIST_INT)); + } + + private static AssignmentOperator add(Set<Integer> value) + { + return new AssignmentOperator(AssignmentOperator.Kind.ADD, new Literal(value, SET_INT)); + } + + private static AssignmentOperator add(Map<Integer, Integer> value) + { + return new AssignmentOperator(AssignmentOperator.Kind.ADD, new Literal(value, MAP_INT)); + } + private static TableMetadata.Builder defaultTable() { return TableMetadata.builder("ks", "tbl") diff --git a/test/harry/main/org/apache/cassandra/harry/model/BytesPartitionState.java b/test/harry/main/org/apache/cassandra/harry/model/BytesPartitionState.java index 6b8f61259dcb..a10524968ae7 100644 --- a/test/harry/main/org/apache/cassandra/harry/model/BytesPartitionState.java +++ b/test/harry/main/org/apache/cassandra/harry/model/BytesPartitionState.java @@ -125,6 +125,12 @@ public void setColumns(Clustering<ByteBuffer> clustering, Map<Symbol, ByteBuffer long cd = factory.clusteringCache.deflate(clustering); long[] vds = toDescriptor(factory.regularColumns, values); state.writeRegular(cd, vds, MagicConstants.NO_TIMESTAMP, writePrimaryKeyLiveness); + + // UDT's have the ability to "update" that triggers a delete; this allows creating an "empty" row. + // When an empty row exists without liveness info, then purge the row + var row = state.rows.get(cd); + if (row.isEmpty() && !row.hasPrimaryKeyLivenessInfo) + state.delete(cd, MagicConstants.NO_TIMESTAMP); } private long[] toDescriptor(ImmutableUniqueList<Symbol> positions, Map<Symbol, ByteBuffer> values) @@ -135,8 +141,14 @@ private long[] toDescriptor(ImmutableUniqueList<Symbol> positions, Map<Symbol, B Symbol column = positions.get(i); if (values.containsKey(column)) { - long vd = factory.valueCache.deflate(new Value(column.type(), values.get(column))); - vds[i] = vd; + ByteBuffer value = values.get(column); + // user type is the only multi cell type that allows <empty> so this check should be fine; can expand if we find more cases + if (value == null || !value.hasRemaining() && (column.type().isUDT() && column.type().isMultiCell())) + { + vds[i] = MagicConstants.NIL_DESCR; + continue; + } + vds[i] = factory.valueCache.deflate(new Value(column.type(), value)); } else { @@ -197,6 +209,13 @@ public Row get(Clustering<ByteBuffer> clustering) return toRow(rowState); } + @Nullable + public ByteBuffer get(Clustering<ByteBuffer> clustering, Symbol column) + { + Row row = get(clustering); + return row == null ? null : row.get(column); + } + private Row toRow(PartitionState.RowState rowState) { Clustering<ByteBuffer> clustering; @@ -548,8 +567,8 @@ private static class Value private Value(AbstractType<?> type, ByteBuffer value) { - this.type = type; - this.value = value; + this.type = Objects.requireNonNull(type); + this.value = Objects.requireNonNull(value); } @Override diff --git a/test/unit/org/apache/cassandra/cql3/KnownIssue.java b/test/unit/org/apache/cassandra/cql3/KnownIssue.java index a1924a91f975..b1c2c09d6624 100644 --- a/test/unit/org/apache/cassandra/cql3/KnownIssue.java +++ b/test/unit/org/apache/cassandra/cql3/KnownIssue.java @@ -39,6 +39,8 @@ public enum KnownIssue "Some types allow empty bytes, but define them as meaningless. AF can be used to query them using <, <=, and =; but SAI can not"), AF_MULTI_NODE_MULTI_COLUMN_AND_NODE_LOCAL_WRITES("https://issues.apache.org/jira/browse/CASSANDRA-19007", "When doing multi node/multi column queries, AF can miss data when the nodes are not in-sync"), + SAI_AND_VECTOR_COLUMNS("https://issues.apache.org/jira/browse/CASSANDRA-20464", + "When doing an SAI query, if the where clause also contains a vector column bad results can be produced") ; KnownIssue(String url, String description) diff --git a/test/unit/org/apache/cassandra/cql3/ast/AssignmentOperator.java b/test/unit/org/apache/cassandra/cql3/ast/AssignmentOperator.java index f72fceb817eb..0ffb65411b46 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/AssignmentOperator.java +++ b/test/unit/org/apache/cassandra/cql3/ast/AssignmentOperator.java @@ -53,31 +53,30 @@ public AssignmentOperator(Kind kind, Expression right) this.right = right; } - public static EnumSet<Kind> supportsOperators(AbstractType<?> type) + public static EnumSet<Kind> supportsOperators(AbstractType<?> type, boolean isTransaction) { type = type.unwrap(); EnumSet<Kind> result = EnumSet.noneOf(Kind.class); + if (type instanceof CollectionType && type.isMultiCell()) + { + if (type instanceof SetType || type instanceof ListType) + return EnumSet.of(Kind.ADD, Kind.SUBTRACT); + if (type instanceof MapType) + { + // map supports subtract, but not map - map; only map - set! + // since this is annoying to support, for now dropping - + return EnumSet.of(Kind.ADD); + } + throw new AssertionError("Unexpected collection type: " + type); + } + if (!isTransaction) + return result; // only multi-cell collections can be updated outside of transactions for (Operator.Kind supported : Operator.supportsOperators(type)) { Kind kind = toKind(supported); if (kind != null) result.add(kind); } - if (result.isEmpty()) - { - if (type instanceof CollectionType && type.isMultiCell()) - { - if (type instanceof SetType || type instanceof ListType) - return EnumSet.of(Kind.ADD, Kind.SUBTRACT); - if (type instanceof MapType) - { - // map supports subtract, but not map - map; only map - set! - // since this is annoying to support, for now dropping - - return EnumSet.of(Kind.ADD); - } - throw new AssertionError("Unexpected collection type: " + type); - } - } return result; } diff --git a/test/unit/org/apache/cassandra/cql3/ast/Conditional.java b/test/unit/org/apache/cassandra/cql3/ast/Conditional.java index 7fdcd17bea4b..52f79bb1dc95 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/Conditional.java +++ b/test/unit/org/apache/cassandra/cql3/ast/Conditional.java @@ -433,7 +433,7 @@ public boolean isEmpty() return sub.isEmpty(); } - private Builder add(Conditional conditional) + public Builder add(Conditional conditional) { sub.add(conditional); return this; diff --git a/test/unit/org/apache/cassandra/cql3/ast/CreateIndexDDL.java b/test/unit/org/apache/cassandra/cql3/ast/CreateIndexDDL.java index 0984ee4620d9..ce86d6bbb500 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/CreateIndexDDL.java +++ b/test/unit/org/apache/cassandra/cql3/ast/CreateIndexDDL.java @@ -141,7 +141,7 @@ else if (((isFrozen(type) && !type.isVector()) || StorageAttachedIndex.SUPPORTED public EnumSet<QueryType> supportedQueries(AbstractType<?> type) { type = type.unwrap(); - if (IndexTermType.isEqOnlyType(type)) + if (IndexTermType.isEqOnlyType(type) || type.isCollection() || type.isUDT() || type.isTuple()) return EnumSet.of(QueryType.Eq); return EnumSet.allOf(QueryType.class); } diff --git a/test/unit/org/apache/cassandra/cql3/ast/ExpressionEvaluator.java b/test/unit/org/apache/cassandra/cql3/ast/ExpressionEvaluator.java index 8270b438f46c..34acb843c7c6 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/ExpressionEvaluator.java +++ b/test/unit/org/apache/cassandra/cql3/ast/ExpressionEvaluator.java @@ -21,107 +21,139 @@ import java.math.BigDecimal; import java.math.BigInteger; import java.nio.ByteBuffer; -import java.util.Optional; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; -import org.apache.cassandra.db.marshal.AbstractType; +import javax.annotation.Nullable; -import static java.util.Optional.of; +import org.apache.cassandra.db.marshal.AbstractType; public class ExpressionEvaluator { - public static Optional<Object> tryEval(Expression e) + @Nullable + public static Object eval(Expression e) { if (e instanceof Value) - return of(((Value) e).value()); + return ((Value) e).value(); if (e instanceof TypeHint) - return tryEval(((TypeHint) e).e); + return eval(((TypeHint) e).e); if (e instanceof Operator) - return tryEval((Operator) e); - return Optional.empty(); + return eval((Operator) e); + throw new UnsupportedOperationException("Unexpected expression type " + e.getClass() + ": " + e.toCQL()); } - public static Optional<Object> tryEval(Operator e) + @Nullable + public static Object eval(Operator e) { + Object lhs = eval(e.left); + if (lhs instanceof ByteBuffer) + lhs = e.left.type().compose((ByteBuffer) lhs); + Object rhs = eval(e.right); + if (rhs instanceof ByteBuffer) + rhs = e.right.type().compose((ByteBuffer) rhs); switch (e.kind) { case ADD: { - var lhsOpt = tryEval(e.left); - var rhsOpt = tryEval(e.right); - if (lhsOpt.isEmpty() || rhsOpt.isEmpty()) - return Optional.empty(); - Object lhs = lhsOpt.get(); - Object rhs = rhsOpt.get(); if (lhs instanceof Byte) - return of((byte) (((Byte) lhs) + ((Byte) rhs))); + return (byte) (((Byte) lhs) + ((Byte) rhs)); if (lhs instanceof Short) - return of((short) (((Short) lhs) + ((Short) rhs))); + return (short) (((Short) lhs) + ((Short) rhs)); if (lhs instanceof Integer) - return of((int) (((Integer) lhs) + ((Integer) rhs))); + return (int) (((Integer) lhs) + ((Integer) rhs)); if (lhs instanceof Long) - return of((long) (((Long) lhs) + ((Long) rhs))); + return (long) (((Long) lhs) + ((Long) rhs)); if (lhs instanceof Float) - return of((float) (((Float) lhs) + ((Float) rhs))); + return (float) (((Float) lhs) + ((Float) rhs)); if (lhs instanceof Double) - return of((double) (((Double) lhs) + ((Double) rhs))); + return (double) (((Double) lhs) + ((Double) rhs)); if (lhs instanceof BigInteger) - return of(((BigInteger) lhs).add((BigInteger) rhs)); + return ((BigInteger) lhs).add((BigInteger) rhs); if (lhs instanceof BigDecimal) - return of(((BigDecimal) lhs).add((BigDecimal) rhs)); + return ((BigDecimal) lhs).add((BigDecimal) rhs); if (lhs instanceof String) - return of(lhs.toString() + rhs.toString()); + return lhs.toString() + rhs.toString(); + if (lhs instanceof Set) + { + Set<Object> accum = new HashSet<>((Set<Object>) lhs); + accum.addAll((Set<Object>) rhs); + return accum; + } + if (lhs instanceof List) + { + List<Object> accum = new ArrayList<>((List<Object>) lhs); + accum.addAll((List<Object>) rhs); + return accum; + } + if (lhs instanceof Map) + { + Map<Object, Object> accum = new HashMap<>((Map<Object, Object>) lhs); + accum.putAll((Map<Object, Object>) rhs); + return accum; + } throw new UnsupportedOperationException("Unexpected type: " + lhs.getClass()); } case SUBTRACT: { - var lhsOpt = tryEval(e.left); - var rhsOpt = tryEval(e.right); - if (lhsOpt.isEmpty() || rhsOpt.isEmpty()) - return Optional.empty(); - Object lhs = lhsOpt.get(); - Object rhs = rhsOpt.get(); if (lhs instanceof Byte) - return of((byte) (((Byte) lhs) - ((Byte) rhs))); + return (byte) (((Byte) lhs) - ((Byte) rhs)); if (lhs instanceof Short) - return of((short) (((Short) lhs) - ((Short) rhs))); + return (short) (((Short) lhs) - ((Short) rhs)); if (lhs instanceof Integer) - return of((int) (((Integer) lhs) - ((Integer) rhs))); + return (int) (((Integer) lhs) - ((Integer) rhs)); if (lhs instanceof Long) - return of((long) (((Long) lhs) - ((Long) rhs))); + return (long) (((Long) lhs) - ((Long) rhs)); if (lhs instanceof Float) - return of((float) (((Float) lhs) - ((Float) rhs))); + return (float) (((Float) lhs) - ((Float) rhs)); if (lhs instanceof Double) - return of((double) (((Double) lhs) - ((Double) rhs))); + return (double) (((Double) lhs) - ((Double) rhs)); if (lhs instanceof BigInteger) - return of(((BigInteger) lhs).subtract((BigInteger) rhs)); + return ((BigInteger) lhs).subtract((BigInteger) rhs); if (lhs instanceof BigDecimal) - return of(((BigDecimal) lhs).subtract((BigDecimal) rhs)); + return ((BigDecimal) lhs).subtract((BigDecimal) rhs); + if (lhs instanceof Set) + { + Set<Object> accum = new HashSet<>((Set<Object>) lhs); + accum.removeAll((Set<Object>) rhs); + return accum.isEmpty() ? null : accum; + } + if (lhs instanceof List) + { + List<Object> accum = new ArrayList<>((List<Object>) lhs); + accum.removeAll((List<Object>) rhs); + return accum.isEmpty() ? null : accum; + } + if (lhs instanceof Map) + { + // rhs is a Set<Object> as CQL doesn't allow removing if the key and value both match + Map<Object, Object> accum = new HashMap<>((Map<Object, Object>) lhs); + ((Set<Object>) rhs).forEach(accum::remove); + return accum.isEmpty() ? null : accum; + } throw new UnsupportedOperationException("Unexpected type: " + lhs.getClass()); } case MULTIPLY: { - var lhsOpt = tryEval(e.left); - var rhsOpt = tryEval(e.right); - if (lhsOpt.isEmpty() || rhsOpt.isEmpty()) - return Optional.empty(); - Object lhs = lhsOpt.get(); - Object rhs = rhsOpt.get(); if (lhs instanceof Byte) - return of((byte) (((Byte) lhs) * ((Byte) rhs))); + return (byte) (((Byte) lhs) * ((Byte) rhs)); if (lhs instanceof Short) - return of((short) (((Short) lhs) * ((Short) rhs))); + return (short) (((Short) lhs) * ((Short) rhs)); if (lhs instanceof Integer) - return of((int) (((Integer) lhs) * ((Integer) rhs))); + return (int) (((Integer) lhs) * ((Integer) rhs)); if (lhs instanceof Long) - return of((long) (((Long) lhs) * ((Long) rhs))); + return (long) (((Long) lhs) * ((Long) rhs)); if (lhs instanceof Float) - return of((float) (((Float) lhs) * ((Float) rhs))); + return (float) (((Float) lhs) * ((Float) rhs)); if (lhs instanceof Double) - return of((double) ((Double) lhs) * ((Double) rhs)); + return (double) ((Double) lhs) * ((Double) rhs); if (lhs instanceof BigInteger) - return of(((BigInteger) lhs).multiply((BigInteger) rhs)); + return ((BigInteger) lhs).multiply((BigInteger) rhs); if (lhs instanceof BigDecimal) - return of(((BigDecimal) lhs).multiply((BigDecimal) rhs)); + return ((BigDecimal) lhs).multiply((BigDecimal) rhs); throw new UnsupportedOperationException("Unexpected type: " + lhs.getClass()); } default: @@ -129,18 +161,12 @@ public static Optional<Object> tryEval(Operator e) } } - public static Optional<ByteBuffer> tryEvalEncoded(Expression e) + @Nullable + public static ByteBuffer evalEncoded(Expression e) { - return tryEval(e).map(v -> { - if (v instanceof ByteBuffer) return (ByteBuffer) v; - try - { - return ((AbstractType) e.type()).decompose(v); - } - catch (Throwable t) - { - throw t; - } - }); + Object v = eval(e); + if (v == null) return null; + if (v instanceof ByteBuffer) return (ByteBuffer) v; + return ((AbstractType) e.type()).decompose(v); } } diff --git a/test/unit/org/apache/cassandra/cql3/ast/Mutation.java b/test/unit/org/apache/cassandra/cql3/ast/Mutation.java index c21508f4d21c..9126d8cbda76 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/Mutation.java +++ b/test/unit/org/apache/cassandra/cql3/ast/Mutation.java @@ -720,6 +720,12 @@ public UpdateBuilder set(String column, int value) return set(new Symbol(column, Int32Type.instance), Bind.of(value)); } + public UpdateBuilder set(String column, Expression expression) + { + Symbol symbol = new Symbol(metadata.getColumn(new ColumnIdentifier(column, true))); + return set(symbol, expression); + } + public UpdateBuilder set(String column, String value) { Symbol symbol = new Symbol(metadata.getColumn(new ColumnIdentifier(column, true))); diff --git a/test/unit/org/apache/cassandra/cql3/ast/Select.java b/test/unit/org/apache/cassandra/cql3/ast/Select.java index 7fa497e16c89..28134dde8e95 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/Select.java +++ b/test/unit/org/apache/cassandra/cql3/ast/Select.java @@ -378,6 +378,13 @@ public T selection(Expression e) return (T) this; } + public T where(Conditional conditional) + { + where = new Conditional.Builder(); + where.add(conditional); + return (T) this; + } + @Override public T where(Expression ref, Conditional.Where.Inequality kind, Expression expression) { diff --git a/test/unit/org/apache/cassandra/cql3/ast/StandardVisitors.java b/test/unit/org/apache/cassandra/cql3/ast/StandardVisitors.java index 854c096c0723..4cbf3d989f03 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/StandardVisitors.java +++ b/test/unit/org/apache/cassandra/cql3/ast/StandardVisitors.java @@ -50,7 +50,7 @@ public Expression visit(Expression e) public Expression visit(Expression e) { if (!(e instanceof Operator)) return e; - return new Bind(ExpressionEvaluator.tryEval((Operator) e).get(), e.type()); + return new Bind(ExpressionEvaluator.eval((Operator) e), e.type()); } }; diff --git a/test/unit/org/apache/cassandra/utils/ASTGenerators.java b/test/unit/org/apache/cassandra/utils/ASTGenerators.java index a02bf65610c0..47b2267de3f9 100644 --- a/test/unit/org/apache/cassandra/utils/ASTGenerators.java +++ b/test/unit/org/apache/cassandra/utils/ASTGenerators.java @@ -35,6 +35,7 @@ import java.util.TreeMap; import java.util.concurrent.TimeUnit; import java.util.function.BiFunction; +import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.IntStream; import javax.annotation.Nullable; @@ -98,7 +99,7 @@ private static <K, V> Map<K, V> assertDeterministic(Map<K, V> map) throw new AssertionError("Unsupported map type: " + map.getClass()); } - public static Gen<AssignmentOperator> assignmentOperatorGen(EnumSet<AssignmentOperator.Kind> allowed, Expression right) + private static Gen<AssignmentOperator> assignmentOperatorGen(EnumSet<AssignmentOperator.Kind> allowed, Expression right) { if (allowed.isEmpty()) throw new IllegalArgumentException("Unable to create a operator gen for empty set of allowed operators"); @@ -187,6 +188,12 @@ public ExpressionBuilder withOperators() return this; } + public ExpressionBuilder withOperators(Gen<Boolean> useOperator) + { + this.useOperator = Objects.requireNonNull(useOperator); + return this; + } + public ExpressionBuilder withoutOperators() { useOperator = i -> false; @@ -375,6 +382,13 @@ public MutationGenBuilder(TableMetadata metadata) columnExpressions.put(symbol, new ExpressionBuilder(symbol.type())); } + public MutationGenBuilder withColumnExpressions(Consumer<ExpressionBuilder> fn) + { + for (Symbol symbol : allColumns) + fn.accept(columnExpressions.get(symbol)); + return this; + } + public MutationGenBuilder allowEmpty(Symbol symbol) { columnExpressions.get(symbol).allowEmpty(); @@ -770,12 +784,12 @@ private void generateRemaining(RandomnessSource rnd, } } } - if (kind == Mutation.Kind.UPDATE && isTransaction) + if (kind == Mutation.Kind.UPDATE) { for (Symbol c : new ArrayList<>(columnsToGenerate)) { var useOperator = columnExpressions.get(c).useOperator; - EnumSet<AssignmentOperator.Kind> additionOperatorAllowed = AssignmentOperator.supportsOperators(c.type()); + EnumSet<AssignmentOperator.Kind> additionOperatorAllowed = AssignmentOperator.supportsOperators(c.type(), isTransaction); if (!additionOperatorAllowed.isEmpty() && useOperator.generate(rnd)) { Expression expression = columnExpressions.get(c).build().generate(rnd); diff --git a/test/unit/org/apache/cassandra/utils/AbstractTypeGenerators.java b/test/unit/org/apache/cassandra/utils/AbstractTypeGenerators.java index 902c37e92d60..ea9a128233dd 100644 --- a/test/unit/org/apache/cassandra/utils/AbstractTypeGenerators.java +++ b/test/unit/org/apache/cassandra/utils/AbstractTypeGenerators.java @@ -26,7 +26,6 @@ import java.util.EnumSet; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Objects; @@ -165,12 +164,13 @@ public static Comparator<String> stringComparator(StringType st) ).collect(Collectors.toMap(t -> t.type, t -> t)); // NOTE not supporting reversed as CQL doesn't allow nested reversed types // when generating part of the clustering key, it would be good to allow reversed types as the top level - private static final Gen<AbstractType<?>> PRIMITIVE_TYPE_GEN; - static + private static final Gen<AbstractType<?>> PRIMITIVE_TYPE_GEN = SourceDSL.arbitrary().pick(knownPrimitiveTypes()); + + public static List<AbstractType<?>> knownPrimitiveTypes() { ArrayList<AbstractType<?>> types = new ArrayList<>(PRIMITIVE_TYPE_DATA_GENS.keySet()); types.sort(Comparator.comparing(a -> a.getClass().getName())); - PRIMITIVE_TYPE_GEN = SourceDSL.arbitrary().pick(types); + return types; } private static final Set<Class<? extends AbstractType>> NON_PRIMITIVE_TYPES = ImmutableSet.<Class<? extends AbstractType>>builder() @@ -244,6 +244,14 @@ public static <T> Releaser overridePrimitiveTypeSupport(AbstractType<T> type, Ty return () -> PRIMITIVE_TYPE_DATA_GENS.put(type, original); } + public static boolean isUnsafeEquality(AbstractType<?> type) + { + return type == EmptyType.instance + || type == DurationType.instance + || type == DecimalType.instance + || type == CounterColumnType.instance; + } + public static TypeGenBuilder withoutUnsafeEquality(TypeGenBuilder builder) { // make sure to keep UNSAFE_EQUALITY in-sync @@ -281,6 +289,7 @@ public static class TypeGenBuilder private Predicate<AbstractType<?>> typeFilter = null; private Gen<String> udtName = null; private Gen<Boolean> multiCellGen = BOOLEAN_GEN; + private UserTypeFieldsGen fieldNamesGen = UserTypeFieldsGen.random(); public TypeGenBuilder() { @@ -289,19 +298,27 @@ public TypeGenBuilder() public TypeGenBuilder(TypeGenBuilder other) { maxDepth = other.maxDepth; - kinds = other.kinds == null ? null : EnumSet.copyOf(other.kinds); + kinds = other.kinds; typeKindGen = other.typeKindGen; defaultSizeGen = other.defaultSizeGen; vectorSizeGen = other.vectorSizeGen; tupleSizeGen = other.tupleSizeGen; - udtName = other.udtName; udtSizeGen = other.udtSizeGen; + compositeSizeGen = other.compositeSizeGen; primitiveGen = other.primitiveGen; + compositeElementGen = other.compositeElementGen; userTypeKeyspaceGen = other.userTypeKeyspaceGen; defaultSetKeyFunc = other.defaultSetKeyFunc; - compositeElementGen = other.compositeElementGen; - compositeSizeGen = other.compositeSizeGen; typeFilter = other.typeFilter; + udtName = other.udtName; + multiCellGen = other.multiCellGen; + fieldNamesGen = other.fieldNamesGen; + } + + public TypeGenBuilder withUserTypeFields(UserTypeFieldsGen fieldNamesGen) + { + this.fieldNamesGen = fieldNamesGen; + return this; } public TypeGenBuilder withMultiCell(Gen<Boolean> multiCellGen) @@ -406,6 +423,13 @@ public TypeGenBuilder withPrimitives(AbstractType<?> first, AbstractType<?>... r return this; } + public TypeGenBuilder withPrimitives(Gen<AbstractType<?>> gen) + { + // any previous filters will be ignored... + primitiveGen = Objects.requireNonNull(gen); + return this; + } + public TypeGenBuilder withMaxDepth(int value) { this.maxDepth = value; @@ -509,7 +533,7 @@ private Gen<AbstractType<?>> buildRecursive(int maxDepth, int level, Gen<TypeKin case TUPLE: return tupleTypeGen(atBottom ? primitiveGen : buildRecursive(maxDepth, level - 1, typeKindGen, SourceDSL.arbitrary().constant(false)), tupleSizeGen != null ? tupleSizeGen : defaultSizeGen).generate(rnd); case UDT: - return userTypeGen(next.get(), udtSizeGen != null ? udtSizeGen : defaultSizeGen, userTypeKeyspaceGen, udtName, multiCellGen).generate(rnd); + return userTypeGen(fieldNamesGen, next.get(), udtSizeGen != null ? udtSizeGen : defaultSizeGen, userTypeKeyspaceGen, udtName, multiCellGen).generate(rnd); case VECTOR: { Gen<Integer> sizeGen = vectorSizeGen != null ? vectorSizeGen : defaultSizeGen; @@ -762,27 +786,47 @@ public static void clearUDTKeyspace() OVERRIDE_KEYSPACE.remove(); } + public interface UserTypeFieldsGen + { + List<FieldIdentifier> generate(RandomnessSource rnd, int size); + + static UserTypeFieldsGen random() + { + Gen<FieldIdentifier> fieldNameGen = IDENTIFIER_GEN.map(FieldIdentifier::forQuoted); + return (rnd, size) -> Generators.uniqueList(fieldNameGen, i -> size).generate(rnd); + } + + static UserTypeFieldsGen simpleNames() + { + return (rnd, size) -> { + List<FieldIdentifier> output = new ArrayList<>(size); + for (int i = 0; i < size; i++) + output.add(FieldIdentifier.forUnquoted("f" + i)); + return output; + }; + } + } + public static Gen<UserType> userTypeGen(Gen<AbstractType<?>> elementGen, Gen<Integer> sizeGen, Gen<String> ksGen, Gen<String> nameGen, Gen<Boolean> multiCellGen) { - Gen<FieldIdentifier> fieldNameGen = IDENTIFIER_GEN.map(FieldIdentifier::forQuoted); + return userTypeGen(UserTypeFieldsGen.random(), elementGen, sizeGen, ksGen, nameGen, multiCellGen); + } + + public static Gen<UserType> userTypeGen(UserTypeFieldsGen fieldNamesGen, Gen<AbstractType<?>> elementGen, Gen<Integer> sizeGen, Gen<String> ksGen, Gen<String> nameGen, Gen<Boolean> multiCellGen) + { return rnd -> { boolean multiCell = multiCellGen.generate(rnd); int numElements = sizeGen.generate(rnd); List<AbstractType<?>> fieldTypes = new ArrayList<>(numElements); - LinkedHashSet<FieldIdentifier> fieldNames = new LinkedHashSet<>(numElements); + List<FieldIdentifier> fieldNames = fieldNamesGen.generate(rnd, numElements); String ks = OVERRIDE_KEYSPACE.get(); if (ks == null) ks = ksGen.generate(rnd); String name = nameGen.generate(rnd); ByteBuffer nameBB = AsciiType.instance.decompose(name); - Gen<FieldIdentifier> distinctNameGen = filter(fieldNameGen, 30, e -> !fieldNames.contains(e)); - // UDTs don't allow duplicate names, so make sure all names are unique for (int i = 0; i < numElements; i++) { - FieldIdentifier fieldName = distinctNameGen.generate(rnd); - fieldNames.add(fieldName); - AbstractType<?> element = elementGen.generate(rnd); element = multiCell ? element.freeze() : element.unfreeze(); // a UDT cannot contain a non-frozen UDT; as defined by CreateType @@ -790,7 +834,7 @@ public static Gen<UserType> userTypeGen(Gen<AbstractType<?>> elementGen, Gen<Int element = element.freeze(); fieldTypes.add(element); } - return new UserType(ks, nameBB, new ArrayList<>(fieldNames), fieldTypes, multiCell); + return new UserType(ks, nameBB, fieldNames, fieldTypes, multiCell); }; } diff --git a/test/unit/org/apache/cassandra/utils/Generators.java b/test/unit/org/apache/cassandra/utils/Generators.java index 5f99421240a3..6bb7f56a8d5c 100644 --- a/test/unit/org/apache/cassandra/utils/Generators.java +++ b/test/unit/org/apache/cassandra/utils/Generators.java @@ -27,7 +27,6 @@ import java.time.ZoneOffset; import java.time.ZonedDateTime; import java.util.ArrayList; -import java.util.Comparator; import java.util.Date; import java.util.HashSet; import java.util.List; @@ -38,10 +37,13 @@ import java.util.function.Predicate; import com.google.common.collect.Range; +import com.google.common.collect.Sets; import org.apache.commons.lang3.ArrayUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import accord.utils.DefaultRandom; +import accord.utils.RandomSource; import org.apache.cassandra.cql3.ReservedKeywords; import org.quicktheories.core.Gen; import org.quicktheories.core.RandomnessSource; @@ -492,13 +494,20 @@ public static <T> Gen<Set<T>> set(Gen<T> gen, Gen<Integer> sizeGen) }; } - public static <T extends Comparable<? super T>> Gen<List<T>> uniqueList(Gen<T> gen, Gen<Integer> sizeGen) + public static <T> Gen<List<T>> uniqueList(Gen<T> gen, Gen<Integer> sizeGen) { - return set(gen, sizeGen).map(t -> { - List<T> list = new ArrayList<>(t); - list.sort(Comparator.naturalOrder()); - return list; - }); + return rnd -> { + int size = sizeGen.generate(rnd); + Set<T> set = Sets.newHashSetWithExpectedSize(size); + List<T> output = new ArrayList<>(size); + for (int i = 0; i < size; i++) + { + T value; + while (!set.add(value = gen.generate(rnd))) {} + output.add(value); + } + return output; + }; } public static <T> Gen<T> cached(Gen<T> gen) @@ -612,6 +621,14 @@ public static <T> accord.utils.Gen<T> toGen(org.quicktheories.core.Gen<T> qt) }; } + public static <T> org.quicktheories.core.Gen<T> fromGen(accord.utils.Gen<T> accord) + { + return rnd -> { + RandomSource rs = new DefaultRandom(rnd.next(Constraint.none())); + return accord.next(rs); + }; + } + public static Gen<TimeUUID> timeUUID() { ZonedDateTime now = ZonedDateTime.of(2020, 8, 20, diff --git a/test/unit/org/apache/cassandra/utils/ImmutableUniqueList.java b/test/unit/org/apache/cassandra/utils/ImmutableUniqueList.java index 73e7da9a9121..00fabea136b5 100644 --- a/test/unit/org/apache/cassandra/utils/ImmutableUniqueList.java +++ b/test/unit/org/apache/cassandra/utils/ImmutableUniqueList.java @@ -32,6 +32,8 @@ public class ImmutableUniqueList<T> extends AbstractList<T> implements RandomAccess { + private static final ImmutableUniqueList<Object> EMPTY = ImmutableUniqueList.builder().build(); + private final T[] values; private final Object2IntHashMap<T> indexLookup; private transient AsSet asSet = null; @@ -46,6 +48,16 @@ public static <T> Builder<T> builder() return new Builder<>(); } + public static <T> Builder<T> builder(int expectedSize) + { + return new Builder<>(expectedSize); + } + + public static <T> ImmutableUniqueList<T> empty() + { + return (ImmutableUniqueList<T>) EMPTY; + } + public AsSet asSet() { if (asSet != null) return asSet; @@ -85,10 +97,20 @@ public int size() public static final class Builder<T> extends AbstractSet<T> { - private final List<T> values = new ArrayList<>(); + private final List<T> values; private final Object2IntHashMap<T> indexLookup = new Object2IntHashMap<>(-1); private int idx; + public Builder() + { + this.values = new ArrayList<>(); + } + + public Builder(int expectedSize) + { + this.values = new ArrayList<>(expectedSize); + } + public Builder<T> mayAddAll(Collection<? extends T> values) { addAll(values); From 4226a7770ea17dd7b545b0997ace704a7a8efec5 Mon Sep 17 00:00:00 2001 From: Edgar Modesto <edgarmodesto23@gmail.com> Date: Sun, 23 Mar 2025 20:22:57 -0700 Subject: [PATCH 207/225] Fix typo on architecture/overview.adoc patch by Edgar Modesto; reviewed by Mick Semb Wever for CASSANDRA-20473 --- doc/modules/cassandra/pages/architecture/overview.adoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/doc/modules/cassandra/pages/architecture/overview.adoc b/doc/modules/cassandra/pages/architecture/overview.adoc index dd0e28739649..2b9949951a97 100644 --- a/doc/modules/cassandra/pages/architecture/overview.adoc +++ b/doc/modules/cassandra/pages/architecture/overview.adoc @@ -10,7 +10,7 @@ Dynamo and Bigtable were both developed to meet emerging requirements for scalab {product} was designed as a best-in-class combination of both systems to meet emerging largescale, both in data footprint and query volume, storage requirements. As applications began to require full global replication and always available low-latency reads and writes, a new kind of database model was required to meet these new requirements. -Relational database systems at that time struggled to meet the the requirements. +Relational database systems at that time struggled to meet the requirements. {product} was designed to meet these challenges with the following design objectives in mind: From 73cd2c56ca07fc4bd1c77a7a45e1d6e27941fefa Mon Sep 17 00:00:00 2001 From: Sam Tunnicliffe <samt@apache.org> Date: Thu, 20 Mar 2025 16:35:47 +0000 Subject: [PATCH 208/225] Add histogram columns to timer metrics vtable Patch by Sam Tunnicliffe; reviewed by Maxim Muzafarov for CASSANDRA-20466 --- CHANGES.txt | 1 + .../db/virtual/model/TimerMetricRow.java | 51 +++++++++++++++++++ .../virtual/walker/TimerMetricRowWalker.java | 21 +++++++- .../metrics/JmxVirtualTableMetricsTest.java | 8 +++ 4 files changed, 80 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 3f622f89765a..324af6e28193 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Add min/max/mean/percentiles to timer metrics vtable (CASSANDRA-20466) * Add support for time, date, timestamp types in scalar constraint (CASSANDRA-20274) * Add regular expression constraint (CASSANDRA-20275) * Improve constraints autocompletion (CASSANDRA-20341) diff --git a/src/java/org/apache/cassandra/db/virtual/model/TimerMetricRow.java b/src/java/org/apache/cassandra/db/virtual/model/TimerMetricRow.java index 4fd4b8175835..ce38bd4e42bc 100644 --- a/src/java/org/apache/cassandra/db/virtual/model/TimerMetricRow.java +++ b/src/java/org/apache/cassandra/db/virtual/model/TimerMetricRow.java @@ -19,6 +19,7 @@ package org.apache.cassandra.db.virtual.model; import com.codahale.metrics.Metric; +import com.codahale.metrics.Snapshot; import com.codahale.metrics.Timer; import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; @@ -30,11 +31,13 @@ public class TimerMetricRow { private final String key; private final Timer value; + private final Snapshot snapshot; public TimerMetricRow(String key, Metric value) { this.key = key; this.value = (Timer) value; + this.snapshot = ((Timer) value).getSnapshot(); } @Column @@ -78,4 +81,52 @@ public double oneMinuteRate() { return value.getOneMinuteRate(); } + + @Column + public double p75th() + { + return snapshot.get75thPercentile(); + } + + @Column + public double p95th() + { + return snapshot.get95thPercentile(); + } + + @Column + public double p98th() + { + return snapshot.get98thPercentile(); + } + + @Column + public double p99th() + { + return snapshot.get99thPercentile(); + } + + @Column + public double p999th() + { + return snapshot.get999thPercentile(); + } + + @Column + public double max() + { + return snapshot.getMax(); + } + + @Column + public double mean() + { + return snapshot.getMean(); + } + + @Column + public double min() + { + return snapshot.getMin(); + } } diff --git a/src/java/org/apache/cassandra/db/virtual/walker/TimerMetricRowWalker.java b/src/java/org/apache/cassandra/db/virtual/walker/TimerMetricRowWalker.java index e4e15ffe6737..c1e6a5005f1b 100644 --- a/src/java/org/apache/cassandra/db/virtual/walker/TimerMetricRowWalker.java +++ b/src/java/org/apache/cassandra/db/virtual/walker/TimerMetricRowWalker.java @@ -28,10 +28,21 @@ */ public class TimerMetricRowWalker implements RowWalker<TimerMetricRow> { + // Note: max & min are defined as doubles here despite Timer itself exposing them as longs, + // via its histogram's snapshot. This is because historically, JMXTimerMBean defined these + // fields as doubles and the vtable representation should be consistent with that. @Override public void visitMeta(MetadataVisitor visitor) { visitor.accept(Column.Type.PARTITION_KEY, "name", String.class); + visitor.accept(Column.Type.REGULAR, "max", Double.TYPE); + visitor.accept(Column.Type.REGULAR, "mean", Double.TYPE); + visitor.accept(Column.Type.REGULAR, "min", Double.TYPE); + visitor.accept(Column.Type.REGULAR, "p75th", Double.TYPE); + visitor.accept(Column.Type.REGULAR, "p95th", Double.TYPE); + visitor.accept(Column.Type.REGULAR, "p98th", Double.TYPE); + visitor.accept(Column.Type.REGULAR, "p999th", Double.TYPE); + visitor.accept(Column.Type.REGULAR, "p99th", Double.TYPE); visitor.accept(Column.Type.REGULAR, "count", Long.TYPE); visitor.accept(Column.Type.REGULAR, "fifteen_minute_rate", Double.TYPE); visitor.accept(Column.Type.REGULAR, "five_minute_rate", Double.TYPE); @@ -44,6 +55,14 @@ public void visitMeta(MetadataVisitor visitor) public void visitRow(TimerMetricRow row, RowMetadataVisitor visitor) { visitor.accept(Column.Type.PARTITION_KEY, "name", String.class, row::name); + visitor.accept(Column.Type.REGULAR, "max", Double.TYPE, row::max); + visitor.accept(Column.Type.REGULAR, "mean", Double.TYPE, row::mean); + visitor.accept(Column.Type.REGULAR, "min", Double.TYPE, row::min); + visitor.accept(Column.Type.REGULAR, "p75th", Double.TYPE, row::p75th); + visitor.accept(Column.Type.REGULAR, "p95th", Double.TYPE, row::p95th); + visitor.accept(Column.Type.REGULAR, "p98th", Double.TYPE, row::p98th); + visitor.accept(Column.Type.REGULAR, "p999th", Double.TYPE, row::p999th); + visitor.accept(Column.Type.REGULAR, "p99th", Double.TYPE, row::p99th); visitor.accept(Column.Type.REGULAR, "count", Long.TYPE, row::count); visitor.accept(Column.Type.REGULAR, "fifteen_minute_rate", Double.TYPE, row::fifteenMinuteRate); visitor.accept(Column.Type.REGULAR, "five_minute_rate", Double.TYPE, row::fiveMinuteRate); @@ -62,7 +81,7 @@ public int count(Column.Type type) case CLUSTERING: return 0; case REGULAR: - return 6; + return 14; default: throw new IllegalStateException("Unknown column type: " + type); } diff --git a/test/unit/org/apache/cassandra/metrics/JmxVirtualTableMetricsTest.java b/test/unit/org/apache/cassandra/metrics/JmxVirtualTableMetricsTest.java index 6f8c8b982261..6cf433e43394 100644 --- a/test/unit/org/apache/cassandra/metrics/JmxVirtualTableMetricsTest.java +++ b/test/unit/org/apache/cassandra/metrics/JmxVirtualTableMetricsTest.java @@ -276,6 +276,14 @@ private Object[] makeTimerRow(ObjectName objectName) return CQLTester.row(getFullMetricName(objectName), bean.getCount(), + bean.getMax(), + bean.getMean(), + bean.getMin(), + bean.get75thPercentile(), + bean.get95thPercentile(), + bean.get98thPercentile(), + bean.get999thPercentile(), + bean.get99thPercentile(), bean.getFifteenMinuteRate(), bean.getFiveMinuteRate(), bean.getMeanRate(), From f7d39dfff4fe4c5e0793297bffb783b97c61eca8 Mon Sep 17 00:00:00 2001 From: "Arra, Praveen R" <praveen.r.arra@jpmchase.com> Date: Fri, 21 Mar 2025 13:05:43 -0500 Subject: [PATCH 209/225] Updated documentation related to commitlog, storage engine and other improvements patch by Arra Praveen; reviewed by Stefan Miklosovic for CASSANDRA-20454 --- .../pages/architecture/messaging.adoc | 14 ++-- .../pages/architecture/overview.adoc | 8 +- .../pages/architecture/storage-engine.adoc | 76 ++++++++++--------- .../pages/architecture/streaming.adoc | 7 +- 4 files changed, 53 insertions(+), 52 deletions(-) diff --git a/doc/modules/cassandra/pages/architecture/messaging.adoc b/doc/modules/cassandra/pages/architecture/messaging.adoc index 2e01fd4c545c..b3d1da95bd47 100644 --- a/doc/modules/cassandra/pages/architecture/messaging.adoc +++ b/doc/modules/cassandra/pages/architecture/messaging.adoc @@ -110,7 +110,7 @@ internode_application_receive_queue_reserve_global_capacity: 512MiB == Virtual Tables for Messaging Metrics Metrics is improved by keeping metrics using virtual tables for -inter-node inbound and outbound messaging +internode inbound and outbound messaging (https://issues.apache.org/jira/browse/CASSANDRA-15066[CASSANDRA-15066]). For inbound messaging a virtual table (`internode_inbound`) has been added to keep metrics for: @@ -122,10 +122,10 @@ due to an error * Bytes and count of messages successfully received * Nanos and count of messages throttled * Bytes and count of messages expired -* Corrupt frames recovered and unrecovered +* Corrupt frames recovered and un-recovered A separate virtual table (`internode_outbound`) has been added for -outbound inter-node messaging. The outbound virtual table keeps metrics +outbound internode messaging. The outbound virtual table keeps metrics for: * Bytes and count of messages pending @@ -216,7 +216,7 @@ The Unprotected option is still available. For resilience, all frames are written with a separate CRC protected header, of 8 and 6 bytes respectively. If corruption occurs in this header, the connection must be reset, as before. If corruption occurs -anywhere outside of the header, the corrupt frame will be skipped, +outside the header, the corrupt frame will be skipped, leaving the connection intact and avoiding the loss of any messages unnecessarily. @@ -255,7 +255,7 @@ expired at the time when a message is encountered the message is just skipped in the byte stream altogether. And if a message fails to be deserialized while still on the receiving side - say, because of table id or column being unknown - bytes are skipped, without dropping the -entire connection and losing all the buffered messages. An immediately +entire connection and losing all the buffered messages. An immediate reply back is sent to the coordinator node with the failure reason, rather than waiting for the coordinator callback to expire. This logic is extended to a corrupted frame; a corrupted frame is safely skipped @@ -342,7 +342,7 @@ sent. == Added a Message size limit Cassandra pre-4.0 doesn't protect the server from allocating huge -buffers for the inter-node Message objects. Adding a message size limit +buffers for the internode Message objects. Adding a message size limit would be good to deal with issues such as a malfunctioning cluster participant. Version 4.0 introduced max message size config param, akin to max mutation size - set to endpoint reserve capacity by default. @@ -356,5 +356,5 @@ message from another node. Pre-4.0, we close the connection and reconnect, which can cause other concurrent queries to fail. Version 4.0 fixes the issue by wrapping message in-stream with `TrackedDataInputPlus`, catching `UnknownCFException`, and skipping the -remaining bytes in this message. TCP won't be closed and it will remain +remaining bytes in this message. TCP won't be closed, and it will remain connected for other messages. diff --git a/doc/modules/cassandra/pages/architecture/overview.adoc b/doc/modules/cassandra/pages/architecture/overview.adoc index 2b9949951a97..003f2198bc34 100644 --- a/doc/modules/cassandra/pages/architecture/overview.adoc +++ b/doc/modules/cassandra/pages/architecture/overview.adoc @@ -8,7 +8,7 @@ It implements a partitioned wide-column storage model with eventually consistent This initial design implemented a combination of Amazon's https://www.cs.cornell.edu/courses/cs5414/2017fa/papers/dynamo.pdf[Dynamo] distributed storage and replication techniques and Google's https://static.googleusercontent.com/media/research.google.com/en//archive/bigtable-osdi06.pdf[Bigtable] data and storage engine model. Dynamo and Bigtable were both developed to meet emerging requirements for scalable, reliable and highly available storage systems, but each had areas that could be improved. -{product} was designed as a best-in-class combination of both systems to meet emerging largescale, both in data footprint and query volume, storage requirements. +{product} was designed as a best-in-class combination of both systems to meet emerging large scale, both in data footprint and query volume, storage requirements. As applications began to require full global replication and always available low-latency reads and writes, a new kind of database model was required to meet these new requirements. Relational database systems at that time struggled to meet the requirements. @@ -28,9 +28,9 @@ Relational database systems at that time struggled to meet the requirements. language, to create, modify, and delete database schema, as well as access data. CQL allows users to organize data within a cluster of {cassandra} nodes using: -* Keyspace: Defines how a dataset is replicated, per datacenter. -Replication is the number of copies saved per cluster. -Keyspaces contain tables. +* Keyspace: Specifies the replication strategy for a dataset across different datacenters. +Replication refers to the number of copies stored within a cluster. +Keyspaces serve as containers for tables. * Table: Tables are composed of rows and columns. Columns define the typed schema for a single datum in a table. Tables are partitioned based on the columns provided in the partition key. diff --git a/doc/modules/cassandra/pages/architecture/storage-engine.adoc b/doc/modules/cassandra/pages/architecture/storage-engine.adoc index 61bd549af191..5580120a83ff 100644 --- a/doc/modules/cassandra/pages/architecture/storage-engine.adoc +++ b/doc/modules/cassandra/pages/architecture/storage-engine.adoc @@ -1,12 +1,12 @@ = Storage Engine -The {Cassandra} storage engine is optimized for high performance, write-oriented workloads. The architecture is based on Log Structured Merge (LSM) trees, which utilize an append-only approach instead of the traditional relational database design with B-trees. This creates a write path free of read lookups and bottlenecks. +The Cassandra storage engine is optimized for high performance, write-oriented workloads. The architecture is based on Log Structured Merge (LSM) trees, which utilize an append-only approach instead of the traditional relational database design with B-trees. This creates a write path free of read lookups and bottlenecks. While the write path is highly optimized, it comes with tradeoffs in terms of read performance and write amplification. To enhance read operations, Cassandra uses Bloom filters when accessing data from stables. Bloom filters are remarkably efficient, leading to generally well-balanced performance for both reads and writes. -Compaction is a necessary background activity required by the ‘merge’ phase of Log Structured Merge trees. Compaction creates write amplification when several small SSTables on disk are read, merged, updates and deletes processed, and a new ssstable is re-written. Every write of data in Cassandra is re-written multiple times, known as write amplification, and this adds background I/O to the database workload. +Compaction is a necessary background activity required by the ‘merge’ phase of Log Structured Merge trees. Compaction creates write amplification when several small SSTables on disk are read, merged, updates and deletes processed, and a new SSTable is re-written. Every write of data in Cassandra is re-written multiple times, known as write amplification, and this adds background I/O to the database workload. -The core storage engine consists of memtables for in-memory data and immutable SSTables (Sorted String Tables) on disk. Data in SSTables is stored sorted to enable efficent merge sort during compaction. Additionally, a write-ahead log (WAL), referred to as the commit log, ensures resiliency for crash and transaction recovery. +The core storage engine consists of memtables for in-memory data and immutable SSTables (Sorted String Tables) on disk. Data in SSTables is stored sorted to enable efficient merge sort during compaction. Additionally, a write-ahead log (WAL), referred to as the commit log, ensures resiliency for crash and transaction recovery. The sequence of the steps in the write path: @@ -18,19 +18,19 @@ The sequence of the steps in the write path: [[commit-log]] == Logging writes to commit logs -When a write occurs, {cassandra} writes the data to a local append-only (https://cassandra.apache.org/_/glossary.html#commit-log)[commit log] on disk. -This action provides xref:cassandra:managing/configuration/cass_yaml_file.adoc[configurable durability] by logging every write made to a {cassandra} node. +When a write operation takes place, Cassandra records the data in a local append-only https://cassandra.apache.org/_/glossary.html#commit-log[commit log] on disk. +This action provides xref:cassandra:managing/configuration/cass_yaml_file.adoc[configurable durability] by logging every write made to a Cassandra node. If an unexpected shutdown occurs, the commit log provides permanent durable writes of the data. -On startup, any mutations in the commit log will be applied to (https://cassandra.apache.org/_/glossary.html#memtable)[memtables]. +On startup, any mutations in the commit log will be applied to https://cassandra.apache.org/_/glossary.html#memtable[memtables]. The commit log is shared among tables. All mutations are write-optimized on storage in commit log segments, reducing the number of seeks needed to write to disk. Commit log segments are limited by the xref:cassandra:managing/configuration/cass_yaml_file.adoc#commitlog_segment_size[`commitlog_segment_size`] option. Once the defined size is reached, a new commit log segment is created. Commit log segments can be archived, deleted, or recycled once all the data is flushed to -(https://cassandra.apache.org/_/glossary.html#sstable)[SSTables]. -Commit log segments are truncated when {cassandra} has written data older than a certain point to the SSTables. -Running xref:managing:tools/nodetool/drain.adoc[`nodetool drain`] before stopping {cassandra} will write everything in the memtables +https://cassandra.apache.org/_/glossary.html#sstable[SSTables]. +Commit log segments are truncated when Cassandra has written data older than a certain point to the SSTables. +Running xref:managing:tools/nodetool/drain.adoc[`nodetool drain`] before stopping Cassandra will write everything in the memtables to SSTables and remove the need to sync with the commit logs on startup. * xref:cassandra:managing/configuration/cass_yaml_file.adoc#commitlog_segment_size [`commitlog_segment_size`]: The default size is 32MiB, which is almost always fine, but if you are archiving commitlog segments (see commitlog_archiving.properties), then you probably want a finer granularity of archiving; 8 or 16 MiB is reasonable. @@ -42,18 +42,18 @@ If `max_mutation_size` is set explicitly then `commitlog_segment_size` must be s ==== * xref:cassandra:managing/configuration/cass_yaml_file.adoc#commitlog_sync[`commitlog_sync`]: may be either _periodic_ or _batch_. -** `batch`: In batch mode, {cassandra} won't acknowledge writes until the commit log has been fsynced to disk. +** `batch`: In batch mode, Cassandra won't acknowledge writes until the commit log has been fsynced to disk. + ** `periodic`: In periodic mode, writes are immediately acknowledged, and the commit log is simply synced every "commitlog_sync_period" milliseconds. + - `commitlog_sync_period`: Time to wait between "periodic" fsyncs _Default Value:_ 10000ms -_Default Value:_ batch +_Default Value:_ periodic [NOTE] ==== -In the event of an unexpected shutdown, {cassandra} can lose up to the sync period or more if the sync is delayed. +In the event of an unexpected shutdown, Cassandra can lose up to the sync period or more if the sync is delayed. If using `batch` mode, it is recommended to store commit logs in a separate, dedicated device. ==== @@ -67,7 +67,7 @@ _Default Value:_ `/var/lib/cassandra/commitlog` If omitted, the commit log will be written uncompressed. LZ4, Snappy,Deflate and Zstd compressors are supported. -_Default Value:_ (complex option): +_Default Value:_ [source, yaml] ---- @@ -77,7 +77,7 @@ _Default Value:_ (complex option): * xref:cassandra:managing/configuration/cass_yaml_file.adoc#commitlog_total_space[`commitlog_total_space`]: Total space to use for commit logs on disk. This option is commented out by default. -If space gets above this value, {cassandra} will flush every dirty table in the oldest segment and remove it. +If space gets above this value, Cassandra will flush every dirty table in the oldest segment and remove it. So a small total commit log space will tend to cause more flush activity on less-active tables. The default value is the smallest between 8192 and 1/4 of the total space of the commitlog volume. @@ -85,11 +85,12 @@ _Default Value:_ 8192MiB == Memtables -When a write occurs, {cassandra} also writes the data to a memtable. -Memtables are in-memory structures where {cassandra} buffers writes. -In general, there is one active memtable per table. -The memtable is a write-back cache of data partitions that {cassandra} looks up by key. -Memtables may be stored entirely on-heap or partially off-heap, depending on xref:cassandra:managing/configuration/cass_yaml_file.adoc#memtable_allocation_type[`memtable_allocation_type`]. +When Cassandra accepts new write requests, it saves the data in two places: an in-memory write-back cache called a memtable and the CommitLog. +The memtable buffers writes and allows serving reads without accessing the disk, while the CommitLog ensures durability by appending new mutations. +Typically, there is one active memtable per table, acting as a cache for data partitions that Cassandra accesses by key. + +Depending on the xref:cassandra:managing/configuration/cass_yaml_file.adoc#memtable_allocation_type[`memtable_allocation_type`], memtables can be stored entirely on-heap or partially off-heap. +If Cassandra crashes before flushing the memtable, it can restore acknowledged writes by replaying the commitLog. The memtable stores writes in sorted order until reaching a configurable limit. When the limit is reached, memtables are flushed onto disk and become immutable xref:#sstables[SSTables]. @@ -102,8 +103,8 @@ When a triggering event occurs, the memtable is put in a queue that is flushed t Flushing writes the data to disk, in the memtable-sorted order. A partition index is also created on the disk that maps the tokens to a location on disk. -The queue can be configured with either the xref:cassandra:managing/configuration/cass_yaml_file.adoc#memtable_heap_space[`memtable_heap_space`] or xref:cassandra:managing/configuration/cass_yaml_file.adoc#memtable_offheap_space[`memtable_offheap_space`] setting in the `cassandra.yaml` file. -If the data to be flushed exceeds the `memtable_cleanup_threshold`, {cassandra} blocks writes until the next flush succeeds. +The queue can be configured with either the xref:cassandra:managing/configuration/cass_yaml_file.adoc#memtable_heap_space[`memtable_heap_space`] or xref:cassandra:managing/configuration/cass_yaml_file.adoc#memtable_offheap_space[`memtable_offheap_space`] setting in the `cassandra.yaml` file. +If the data to be flushed exceeds the `memtable_cleanup_threshold`, Cassandra blocks writes until the next flush succeeds. You can manually flush a table using xref:managing:tools/nodetool/flush.adoc[`nodetool flush`] or `nodetool drain` (flushes memtables without listening for connections to other nodes). To reduce the commit log replay time, the recommended best practice is to flush the memtable before you restart the nodes. If a node stops working, replaying the commit log restores writes to the memtable that were there before it stopped. @@ -112,12 +113,12 @@ Data in the commit log is purged after its corresponding data in the memtable is == SSTables -https://cassandra.apache.org/_/glossary.html#sstable[SSTables] are the immutable data files that {cassandra} uses for persisting data on disk. +https://cassandra.apache.org/_/glossary.html#sstable[SSTables] are the immutable data files that Cassandra uses for persisting data on disk. SSTables are maintained per table. SSTables are immutable, and never written to again after the memtable is flushed. Thus, a partition is typically stored across multiple SSTable files, as data is added or modified. -Each SSTable is comprised of multiple components stored in separate files: +Each SSTable consists of multiple components stored in separate files: `Data.db`:: The actual data, i.e. the contents of rows. @@ -148,7 +149,7 @@ Only present if SAI is enabled for the table. [NOTE] ==== Note that the `Index.db` file type is replaced by `Partitions.db` and `Rows.db`. -This change is a consequence of the inclusion of Big Trie indexes in Cassandra (https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-25%3A+Trie-indexed+SSTable+format[CEP-25]). +This change is a consequence of the inclusion of Big Trie indexes in Cassandra https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-25%3A+Trie-indexed+SSTable+format[CEP-25]. ==== Within the `Data.db` file, rows are organized by partition. @@ -157,18 +158,18 @@ Within a partition, rows are stored in the order of their clustering keys. SSTables can be optionally compressed using block-based compression. -As SSTables are flushed to disk from `memtables` or are streamed from other nodes, {cassandra} triggers compactions which combine multiple SSTables into one. +As SSTables are flushed to disk from `memtables` or are streamed from other nodes, Cassandra triggers compactions which combine multiple SSTables into one. Once the new SSTable has been written, the old SSTables can be removed. == SSTable Versions -From (https://github.com/apache/cassandra/blob/cassandra-5.0/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java#L338)[BigFormat#BigVersion]. +From https://github.com/apache/cassandra/blob/cassandra-5.0/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java#L338[BigFormat#BigVersion]. The version numbers, to date are: === Version 0 -* b (0.7.0): added version to sstable filenames +* b (0.7.0): added version to SSTable filenames * c (0.7.0): bloom filter component computes hashes over raw key bytes instead of strings * d (0.7.0): row size in data component becomes a long instead of int @@ -199,7 +200,7 @@ millis-based id (see CASSANDRA-4782) real format change, this is mostly a marker to know if we should expect super columns or not. We do need a major version bump however, because we should not allow streaming of super columns into this new format) -** tracks max local deletiontime in sstable metadata +** tracks max local deletiontime in SSTable metadata ** records bloom_filter_fp_chance in metadata component ** remove data size and column count from data file (CASSANDRA-4180) ** tracks max/min column values (according to comparator) @@ -208,7 +209,7 @@ we should not allow streaming of super columns into this new format) ** checksum the compressed data * ka (2.1.0): ** new Statistics.db file format -** index summaries can be downsampled and the sampling level is +** index summaries can be down sampled and the sampling level is persisted ** switch uncompressed checksums to adler32 ** tracks presence of legacy (local and remote) counter shards @@ -222,12 +223,12 @@ persisted ** store rows natively * mb (3.0.7, 3.7): commit log lower bound included * mc (3.0.8, 3.9): commit log intervals included -* md (3.0.18, 3.11.4): corrected sstable min/max clustering -* me (3.0.25, 3.11.11): added hostId of the node from which the sstable originated +* md (3.0.18, 3.11.4): corrected SSTable min/max clustering +* me (3.0.25, 3.11.11): added hostId of the node from which the SSTable originated === Version 4 -* na (4.0-rc1): uncompressed chunks, pending repair session, isTransient, checksummed sstable metadata file, new Bloomfilter format +* na (4.0-rc1): uncompressed chunks, pending repair session, isTransient, checksummed SSTable metadata file, new Bloom filter format * nb (4.0.0): originating host id === Version 5 @@ -241,14 +242,15 @@ persisted Cassandra 5.0 introduced new SSTable formats BTI for Trie-indexed SSTables. To use the BTI formats configure it `cassandra.yaml` like -``` +[source] +---- sstable: selected_format: bti -``` +---- -Versions come from (https://github.com/apache/cassandra/blob/cassandra-5.0/src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java#L287)[BtiFormat#BtiVersion]. +Versions come from https://github.com/apache/cassandra/blob/f16fb6765b8a3ff8f49accf61c908791520c0d6e/src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java#L302[BtiFormat#BtiVersion]. -For implementation docs see (https://github.com/apache/cassandra/blob/cassandra-5.0/src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.md)[BtiFormat.md]. +For implementation docs see https://github.com/apache/cassandra/blob/cassandra-5.0/src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.md[BtiFormat.md]. === Version 5 @@ -256,7 +258,7 @@ For implementation docs see (https://github.com/apache/cassandra/blob/cassandra- === Example Code -The following example is useful for finding all sstables that do not +The following example is useful for finding all SSTables that do not match the "ib" SSTable version [source,bash] diff --git a/doc/modules/cassandra/pages/architecture/streaming.adoc b/doc/modules/cassandra/pages/architecture/streaming.adoc index 1758f66abb65..dd3859b5c671 100644 --- a/doc/modules/cassandra/pages/architecture/streaming.adoc +++ b/doc/modules/cassandra/pages/architecture/streaming.adoc @@ -68,8 +68,7 @@ Zero copy streaming is enabled by setting the following setting in stream_entire_sstables: true .... -By default zero copy streaming is enabled. - +Zero copy streaming is enabled by default. === SSTables Eligible for Zero Copy Streaming Zero copy streaming is used if all partitions within the SSTable need to @@ -88,7 +87,7 @@ network transfer significantly subject to throttling specified by Enabling this will reduce the GC pressure on sending and receiving node. While this feature tries to keep the disks balanced, it cannot guarantee it. This feature will be automatically disabled if internode encryption -is enabled. Currently this can be used with Leveled Compaction. +is enabled. Currently, this can be used with Leveled Compaction. === Configuring for Zero Copy Streaming @@ -161,7 +160,7 @@ Repair with `nodetool repair` involves streaming of repaired SSTables and a repair preview has been added to provide an estimate of the amount of repair streaming that would need to be performed. Repair preview (https://issues.apache.org/jira/browse/CASSANDRA-13257[CASSANDRA-13257]) -is invoke with `nodetool repair --preview` using option: +is invoked with `nodetool repair --preview` using option: .... -prv, --preview From 615aa04edfdccf25f37a8fc30dd94029e7ab9d35 Mon Sep 17 00:00:00 2001 From: Marcus Eriksson <marcuse@apache.org> Date: Wed, 5 Mar 2025 11:59:37 +0100 Subject: [PATCH 210/225] Reintroduce CASSANDRA-17411 in trunk Patch by marcuse; reviewed by Caleb Rackliffe and Sam Tunnicliffe for CASSANDRA-19346 --- CHANGES.txt | 1 + src/java/org/apache/cassandra/locator/ReplicaPlans.java | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 324af6e28193..98389b77cc40 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Reintroduce CASSANDRA-17411 in trunk (CASSANDRA-19346) * Add min/max/mean/percentiles to timer metrics vtable (CASSANDRA-20466) * Add support for time, date, timestamp types in scalar constraint (CASSANDRA-20274) * Add regular expression constraint (CASSANDRA-20275) diff --git a/src/java/org/apache/cassandra/locator/ReplicaPlans.java b/src/java/org/apache/cassandra/locator/ReplicaPlans.java index 62b7f0ca2d44..b6a03b683bd1 100644 --- a/src/java/org/apache/cassandra/locator/ReplicaPlans.java +++ b/src/java/org/apache/cassandra/locator/ReplicaPlans.java @@ -234,10 +234,11 @@ public static Replica findCounterLeaderReplica(ClusterMetadata metadata, String EndpointsForToken replicas = metadata.placements.get(keyspace.getMetadata().params.replication).reads.forToken(key.getToken()).get(); // CASSANDRA-13043: filter out those endpoints not accepting clients yet, maybe because still bootstrapping - // TODO: replace this with JOINED state. - // TODO don't forget adding replicas = replicas.filter(replica -> FailureDetector.instance.isAlive(replica.endpoint())); after rebase (from CASSANDRA-17411) replicas = replicas.filter(replica -> StorageService.instance.isRpcReady(replica.endpoint())); + // CASSANDRA-17411: filter out endpoints that are not alive + replicas = replicas.filter(replica -> FailureDetector.instance.isAlive(replica.endpoint())); + // TODO have a way to compute the consistency level if (replicas.isEmpty()) throw UnavailableException.create(cl, cl.blockFor(replicationStrategy), 0); From 51e01a3862afc1ebaffd765b2490a581461c3f14 Mon Sep 17 00:00:00 2001 From: Sam Tunnicliffe <samt@apache.org> Date: Thu, 20 Mar 2025 17:55:44 +0000 Subject: [PATCH 211/225] Repair Paxos for the distributed metadata log when CMS membership changes Patch by Josh McKenzie and Sam Tunnicliffe; reviewed by Marcus Ericksson for CASSANDRA-20467 Co-authored-by: Josh McKenzie <jmckenzie@apache.org> Co-authored-by: Sam Tunnicliffe <samt@apache.org> --- CHANGES.txt | 1 + .../cassandra/db/DiskBoundaryManager.java | 2 +- .../cassandra/tcm/MultiStepOperation.java | 4 +- .../apache/cassandra/tcm/Transformation.java | 6 ++ .../cassandra/tcm/sequences/AddToCMS.java | 2 +- .../tcm/sequences/ReconfigureCMS.java | 67 +++++++++++++++---- .../distributed/test/PaxosRepairTest.java | 21 ++++-- .../test/log/ReconfigureCMSTest.java | 64 ++++++++++++++++++ 8 files changed, 142 insertions(+), 25 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 98389b77cc40..1360ab01bca3 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Repair Paxos for the distributed metadata log when CMS membership changes (CASSANDRA-20467) * Reintroduce CASSANDRA-17411 in trunk (CASSANDRA-19346) * Add min/max/mean/percentiles to timer metrics vtable (CASSANDRA-20466) * Add support for time, date, timestamp types in scalar constraint (CASSANDRA-20274) diff --git a/src/java/org/apache/cassandra/db/DiskBoundaryManager.java b/src/java/org/apache/cassandra/db/DiskBoundaryManager.java index 2a25dc4efc5c..5c6b59a0b165 100644 --- a/src/java/org/apache/cassandra/db/DiskBoundaryManager.java +++ b/src/java/org/apache/cassandra/db/DiskBoundaryManager.java @@ -152,7 +152,7 @@ private static RangesAtEndpoint getLocalRanges(ColumnFamilyStore cfs, ClusterMet } else { - // Reason we use use the future settled metadata is that if we decommission a node, we want to stream + // Reason we use the future settled metadata is that if we decommission a node, we want to stream // from that node to the correct location on disk, if we didn't, we would put new files in the wrong places. // We do this to minimize the amount of data we need to move in rebalancedisks once everything settled placement = metadata.writePlacementAllSettled(cfs.keyspace.getMetadata()); diff --git a/src/java/org/apache/cassandra/tcm/MultiStepOperation.java b/src/java/org/apache/cassandra/tcm/MultiStepOperation.java index 019086dccd69..d447974f85d5 100644 --- a/src/java/org/apache/cassandra/tcm/MultiStepOperation.java +++ b/src/java/org/apache/cassandra/tcm/MultiStepOperation.java @@ -40,7 +40,7 @@ * For example, in order to join, the joining node has to execute the following steps: * * PrepareJoin, which introduces node's tokens, but makes no changes to range ownership, and creates BootstrapAndJoin * in-progress sequence - * * StartJoin, which adds the bootstrapping node to the write placements for the ranges it gains + * * StartJoin, which adds the bootstrapping node to the write placements for the ranges it gains * * MidJoin, which adds the bootstrapping node to the read placements for the ranges it has gained, and removes * owners of these ranges from the read placements * * FinishJoin, which removes owners of the gained ranges from the write placements. @@ -126,7 +126,7 @@ public boolean finishDuringStartup() /** * Returns the {@link Transformation.Kind} of the next step due to be executed in the sequence. Used when executing - * a {@link Transformation} which is part of a sequence (specifically, subclasses of + * a {@link Transformation} which is part of a sequence (often, this is an implementation of * {@link org.apache.cassandra.tcm.transformations.ApplyPlacementDeltas}) to validate that it is being applied at * the correct point (i.e. that the type of the transform matches the expected next) * matches the If all steps diff --git a/src/java/org/apache/cassandra/tcm/Transformation.java b/src/java/org/apache/cassandra/tcm/Transformation.java index e4b81bc7653a..8cfda01e26c0 100644 --- a/src/java/org/apache/cassandra/tcm/Transformation.java +++ b/src/java/org/apache/cassandra/tcm/Transformation.java @@ -48,6 +48,12 @@ import org.apache.cassandra.tcm.transformations.cms.StartAddToCMS; import org.apache.cassandra.tcm.transformations.cms.PrepareCMSReconfiguration; +/** + * Implementations should be pure transformations from one ClusterMetadata state to another. They are likely to be + * replayed during startup to rebuild the node's current state and so should be free of side effects and should not + * depend on external state, configuration or resources. They must produce consistent outputs when run on every instance + * in a cluster, regardless of any specific characteristics of the instance. + */ public interface Transformation { Serializer transformationSerializer = new Serializer(); diff --git a/src/java/org/apache/cassandra/tcm/sequences/AddToCMS.java b/src/java/org/apache/cassandra/tcm/sequences/AddToCMS.java index 149c99b8b2bd..0d5ee2f06710 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/AddToCMS.java +++ b/src/java/org/apache/cassandra/tcm/sequences/AddToCMS.java @@ -82,7 +82,7 @@ public static void initiate(NodeId nodeId, InetAddressAndPort addr) .commit(new StartAddToCMS(addr)) .inProgressSequences.get(nodeId); InProgressSequences.resume(sequence); - ReconfigureCMS.repairPaxosTopology(); + ReconfigureCMS.repairPaxosForCMSTopologyChange(); } public AddToCMS(Epoch latestModification, diff --git a/src/java/org/apache/cassandra/tcm/sequences/ReconfigureCMS.java b/src/java/org/apache/cassandra/tcm/sequences/ReconfigureCMS.java index 57b5e68e807e..38566812bf54 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/ReconfigureCMS.java +++ b/src/java/org/apache/cassandra/tcm/sequences/ReconfigureCMS.java @@ -33,9 +33,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.dht.Range; -import org.apache.cassandra.dht.Token; import org.apache.cassandra.gms.FailureDetector; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; @@ -74,9 +71,18 @@ import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.concurrent.Future; -import static org.apache.cassandra.streaming.StreamOperation.RESTORE_REPLICA_COUNT; import static org.apache.cassandra.locator.MetaStrategy.entireRange; +import static org.apache.cassandra.streaming.StreamOperation.RESTORE_REPLICA_COUNT; +/** + * This class is slightly different from most other MultiStepOperations in that it doesn't reify every component + * transformation when it is constructed (see how {@link BootstrapAndJoin} encloses its StartJoin/MidJoin/FinishJoin + * transforms for a counter example). Instead, each instance includes a single transformation with kind {@link + * Transformation.Kind#ADVANCE_CMS_RECONFIGURATION}, representing the _next_ step to be executed. That transformation + * instance holds all the state necessary to generate the subsequent ADVANCE_CMS_RECONFIGURATION step. As each of these + * transformations is applied, they logically progress the multi-step operation by installing a new ReconfigureCMS + * instance with the idx pointer bumped and the next step encoded. + */ public class ReconfigureCMS extends MultiStepOperation<AdvanceCMSReconfiguration> { public static final Serializer serializer = new Serializer(); @@ -119,6 +125,7 @@ public Kind kind() { return MultiStepOperation.Kind.RECONFIGURE_CMS; } + @Override protected SequenceKey sequenceKey() { @@ -176,6 +183,13 @@ public SequenceState executeNext() Replica replica = new Replica(endpoint, entireRange, true); streamRanges(replica, activeTransition.streamCandidates); } + else + { + // Run a paxos repair before starting either the addition or removal of a CMS member, where in both + // cases there is no active transition. + repairPaxosForCMSTopologyChange(); + } + // Commit the next step in the sequence ClusterMetadataService.instance().commit(transitionCMS.next); return SequenceState.continuable(); @@ -298,17 +312,42 @@ public String toString() '}'; } - static void repairPaxosTopology() + static void repairPaxosForCMSTopologyChange() { - Retry.Backoff retry = new Retry.Backoff(TCMMetrics.instance.repairPaxosTopologyRetries); + // This *should* be redundant, primarily because the state machine which manages a node's cluster metadata + // doesn't rely on the distributed metadata log table directly but is driven by metadata replication messages + // which distribute log entries around the cluster. + // + // However, it is still worthwhile to guard against a failure to sync paxos accept/commit operations by a subset + // of replicas, in this case the CMS members. Paxos repair is designed to ensure that any operation witnessed by + // at least one replica prior to a topology change is witnessed by a majority of the replica set after the + // topology change. Essentially ensuring that the pre- and post- change quorums overlap. + // + // The way that CMS reconfiguration proceeds is to first expand the membership to the maximal state by adding all + // new members and then to shrink back down to the desired size by pruning out the leaving members. Each step + // only modifies the membership group by adding or removing a single member and unlike operations on other + // keyspaces, there are no changes to the ranges involved. As all CMS members replicate the entire token + // range for that keyspace, these ownership changes are relatively simple. + // + // For example, if the CMS membership is currently {1, 2, 3} and we want to transition it to {4, 5, 6} the + // reconfiguration goes through these steps: + // * {1, 2, 3} + // * {1, 2, 3, 4} + // * {1, 2, 3, 4, 5} + // * {1, 2, 3, 4, 5, 6} + // * {2, 3, 4, 5, 6} + // * {3, 4, 5, 6} + // * {4, 5, 6} + // When adding a member, the new member streams data in from an existing member, analogous to bootstrapping. + // When removing, there is no need for streaming as the existing members' ownership is not changing. Running a + // paxos repair at the beginning of each step, before streaming where applicable, will ensure that the + // overlapping quorums invariant holds. - // The system.paxos table is what we're actually repairing and that uses the system configured partitioner - // so although we use MetaStrategy.entireRange for streaming between CMS members, we don't use it here - Range<Token> entirePaxosRange = new Range<>(DatabaseDescriptor.getPartitioner().getMinimumToken(), - DatabaseDescriptor.getPartitioner().getMinimumToken()); - List<Supplier<Future<?>>> remaining = ActiveRepairService.instance().repairPaxosForTopologyChangeAsync(SchemaConstants.METADATA_KEYSPACE_NAME, - Collections.singletonList(entirePaxosRange), - "bootstrap"); + Retry.Backoff retry = new Retry.Backoff(TCMMetrics.instance.repairPaxosTopologyRetries); + List<Supplier<Future<?>>> remaining = ActiveRepairService.instance() + .repairPaxosForTopologyChangeAsync(SchemaConstants.METADATA_KEYSPACE_NAME, + Collections.singletonList(entireRange), + "CMS reconfiguration"); while (!retry.reachedMax()) { @@ -316,7 +355,6 @@ static void repairPaxosTopology() for (Supplier<Future<?>> supplier : remaining) tasks.put(supplier, supplier.get()); remaining.clear(); - logger.info("Performing paxos topology repair on: {}", remaining); for (Map.Entry<Supplier<Future<?>>, Future<?>> e : tasks.entrySet()) { @@ -331,6 +369,7 @@ static void repairPaxosTopology() } catch (InterruptedException t) { + logger.info("Interrupted while repairing paxos topology, aborting.", t); return; } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/PaxosRepairTest.java b/test/distributed/org/apache/cassandra/distributed/test/PaxosRepairTest.java index 0197106bc631..a2af4a213731 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/PaxosRepairTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/PaxosRepairTest.java @@ -34,7 +34,6 @@ import com.google.common.collect.Sets; import com.google.common.util.concurrent.Uninterruptibles; import org.junit.Assert; -import org.junit.Ignore; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,6 +48,7 @@ import org.apache.cassandra.db.partitions.PartitionIterator; import org.apache.cassandra.db.rows.*; import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.Constants; import org.apache.cassandra.distributed.api.ConsistencyLevel; import org.apache.cassandra.distributed.api.Feature; import org.apache.cassandra.distributed.api.ICluster; @@ -57,6 +57,8 @@ import org.apache.cassandra.distributed.api.IInvokableInstance; import org.apache.cassandra.distributed.api.IMessageFilters; import org.apache.cassandra.gms.FailureDetector; +import org.apache.cassandra.distributed.api.TokenSupplier; +import org.apache.cassandra.distributed.shared.NetworkTopology; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.repair.RepairParallelism; import org.apache.cassandra.repair.SharedContext; @@ -304,16 +306,17 @@ public void epochBadDeserializationTest() throws Throwable } - @Ignore @Test public void topologyChangePaxosTest() throws Throwable { // TODO: fails with vnode enabled - try (Cluster cluster = Cluster.build(4).withConfig(WITH_NETWORK).withoutVNodes().createWithoutStarting()) + try (Cluster cluster = builder().withNodes(3) + .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(4)) + .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(4, "dc0", "rack0")) + .withConfig(WITH_NETWORK) + .withoutVNodes() + .start()) { - for (int i=1; i<=3; i++) - cluster.get(i).startup(); - init(cluster); cluster.schemaChange("CREATE TABLE " + KEYSPACE + '.' + TABLE + " (pk int, ck int, v int, PRIMARY KEY (pk, ck))"); cluster.coordinator(1).execute("INSERT INTO " + KEYSPACE + '.' + TABLE + " (pk, ck, v) VALUES (1, 1, 1) IF NOT EXISTS", ConsistencyLevel.QUORUM); @@ -333,7 +336,11 @@ public void topologyChangePaxosTest() throws Throwable cluster.filters().reset(); // node 4 starting should repair paxos and inform the other nodes of its gossip state - cluster.get(4).startup(); + IInstanceConfig config = cluster.newInstanceConfig() + .set("auto_bootstrap", true) + .set(Constants.KEY_DTEST_FULL_STARTUP, true); + IInvokableInstance node4 = cluster.bootstrap(config); + node4.startup(); Assert.assertFalse(hasUncommittedQuorum(cluster, KEYSPACE, TABLE)); } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/log/ReconfigureCMSTest.java b/test/distributed/org/apache/cassandra/distributed/test/log/ReconfigureCMSTest.java index d1f983f19af2..2869fe913ac8 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/log/ReconfigureCMSTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/log/ReconfigureCMSTest.java @@ -19,8 +19,10 @@ package org.apache.cassandra.distributed.test.log; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Collections; import java.util.HashSet; +import java.util.List; import java.util.Random; import java.util.Set; import java.util.concurrent.ExecutionException; @@ -40,6 +42,8 @@ import org.apache.cassandra.schema.DistributedMetadataLogKeyspace; import org.apache.cassandra.schema.ReplicationParams; import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.service.paxos.Ballot; +import org.apache.cassandra.service.paxos.PaxosRepairHistory; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.ownership.DataPlacement; @@ -257,6 +261,66 @@ public void testReconfigurationViolatesRackDiversityIfNecessary() throws Excepti } } + @Test + public void cmsTopologyChangePaxosTest() throws Throwable + { + // Use a 4 node cluster so we have room to decommission one node while still maintaining RF + try (Cluster cluster = builder().withNodes(4) + .withConfig(c -> c.with(Feature.NETWORK)) + .withoutVNodes() + .start()) + { + IInvokableInstance node1 = cluster.get(1); + IInvokableInstance node2 = cluster.get(2); + IInvokableInstance node3 = cluster.get(3); + IInvokableInstance node4 = cluster.get(4); + + // no paxos repair history initially + PaxosRepairHistory empty = PaxosRepairHistory.empty(MetaStrategy.partitioner); + cluster.forEach(i -> assertEquals(empty, paxosRepairHistory(i))); + + node1.nodetoolResult("cms", "reconfigure", "2").asserts().success(); + // Nodes 3 & 4 are not involved in the first cms reconfiguration, so should still have no paxos repair + // history for the metadata log table + assertEquals(empty, paxosRepairHistory(node3)); + assertEquals(empty, paxosRepairHistory(node4)); + // Node 1 & 2 should have completed a paxos repair. For this keyspace, that is always over the entire + // range, so there is only ever a single entry in the repair history which equates to prh.size() == 0 + PaxosRepairHistory node1History = paxosRepairHistory(node1); + assertEquals(0, node1History.size()); + assertEquals(node1History, paxosRepairHistory(node2)); + + // node 1 leaving should cause a cms reconfiguration which runs a paxos repair which involves nodes 2 & 3 + // does participate in while node 4 remains uninvolved. + node1.nodetoolResult("decommission").asserts().success(); + assertEquals(empty, paxosRepairHistory(node4)); + + PaxosRepairHistory node3History = paxosRepairHistory(node3); + assertEquals(0, node3History.size()); + assertEquals(node3History, paxosRepairHistory(node2)); + // verify that the ballot for this second repair is > the one for the first + Ballot node3Ballot = node3History.ballotForToken(MetaStrategy.partitioner.getMinimumToken()); + Ballot node1Ballot = node1History.ballotForToken(MetaStrategy.partitioner.getMinimumToken()); + assertTrue(node3Ballot.unixMicros() > node1Ballot.unixMicros()); + } + } + + private PaxosRepairHistory paxosRepairHistory(IInvokableInstance instance) + { + Object[][] rows = instance.executeInternal("select points from system.paxos_repair_history " + + "where keyspace_name = ? " + + "and table_name = ?", + SchemaConstants.METADATA_KEYSPACE_NAME, + DistributedMetadataLogKeyspace.TABLE_NAME); + + if (rows.length == 0) + return PaxosRepairHistory.empty(SchemaConstants.METADATA_KEYSPACE_NAME, DistributedMetadataLogKeyspace.TABLE_NAME); + assertEquals(1, rows.length); + //noinspection unchecked + List<ByteBuffer> points = (List<ByteBuffer>)rows[0][0]; + return PaxosRepairHistory.fromTupleBufferList(MetaStrategy.partitioner, points); + } + // We can't assume that nodeId matches endpoint (ie node3 = 127.0.0.3 etc) private Set<String> expectedCMS(Cluster cluster, int... instanceIds) { From 4fb81ea483dc8d95dac11c234ca6cb3aaeac44a8 Mon Sep 17 00:00:00 2001 From: Marcus Eriksson <marcuse@apache.org> Date: Mon, 24 Mar 2025 13:27:38 +0100 Subject: [PATCH 212/225] Add nodetool command to abort failed nodetool cms initialize Patch by marcuse; reviewed by Sam Tunnicliffe for CASSANDRA-20482 --- CHANGES.txt | 1 + .../apache/cassandra/tcm/CMSOperations.java | 6 ++ .../cassandra/tcm/CMSOperationsMBean.java | 1 + .../cassandra/tcm/migration/Election.java | 44 ++++++++++++--- .../org/apache/cassandra/tools/NodeTool.java | 3 +- .../cassandra/tools/nodetool/CMSAdmin.java | 13 +++++ ...sterMetadataUpgradeAbortMigrationTest.java | 55 +++++++++++++++++++ .../upgrade/ClusterMetadataUpgradeTest.java | 3 +- 8 files changed, 115 insertions(+), 11 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeAbortMigrationTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 1360ab01bca3..864e66462043 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Add nodetool command to abort failed nodetool cms initialize (CASSANDRA-20482) * Repair Paxos for the distributed metadata log when CMS membership changes (CASSANDRA-20467) * Reintroduce CASSANDRA-17411 in trunk (CASSANDRA-19346) * Add min/max/mean/percentiles to timer metrics vtable (CASSANDRA-20466) diff --git a/src/java/org/apache/cassandra/tcm/CMSOperations.java b/src/java/org/apache/cassandra/tcm/CMSOperations.java index f4a608b31e25..b37da9dd94b9 100644 --- a/src/java/org/apache/cassandra/tcm/CMSOperations.java +++ b/src/java/org/apache/cassandra/tcm/CMSOperations.java @@ -35,6 +35,7 @@ import org.apache.cassandra.tcm.membership.NodeId; import org.apache.cassandra.tcm.membership.NodeState; import org.apache.cassandra.tcm.membership.NodeVersion; +import org.apache.cassandra.tcm.migration.Election; import org.apache.cassandra.tcm.sequences.CancelCMSReconfiguration; import org.apache.cassandra.tcm.sequences.InProgressSequences; import org.apache.cassandra.tcm.sequences.ReconfigureCMS; @@ -80,6 +81,11 @@ public void initializeCMS(List<String> ignoredEndpoints) cms.upgradeFromGossip(ignoredEndpoints); } + public void abortInitialization(String initiator) + { + Election.instance.abortInitialization(initiator); + } + @Override public void resumeReconfigureCms() { diff --git a/src/java/org/apache/cassandra/tcm/CMSOperationsMBean.java b/src/java/org/apache/cassandra/tcm/CMSOperationsMBean.java index 3b6e4b526bf9..1e2d9e147313 100644 --- a/src/java/org/apache/cassandra/tcm/CMSOperationsMBean.java +++ b/src/java/org/apache/cassandra/tcm/CMSOperationsMBean.java @@ -25,6 +25,7 @@ public interface CMSOperationsMBean { public void initializeCMS(List<String> ignore); + public void abortInitialization(String initiator); public void resumeReconfigureCms(); public void reconfigureCMS(int rf); public void reconfigureCMS(Map<String, Integer> rf); diff --git a/src/java/org/apache/cassandra/tcm/migration/Election.java b/src/java/org/apache/cassandra/tcm/migration/Election.java index 04e1a8fa4c8f..69fac6f4ba7b 100644 --- a/src/java/org/apache/cassandra/tcm/migration/Election.java +++ b/src/java/org/apache/cassandra/tcm/migration/Election.java @@ -56,6 +56,7 @@ public class Election { private static final Logger logger = LoggerFactory.getLogger(Election.class); + private static final CMSInitializationRequest.Initiator MIGRATING = new CMSInitializationRequest.Initiator(null, null); private static final CMSInitializationRequest.Initiator MIGRATED = new CMSInitializationRequest.Initiator(null, null); private final AtomicReference<CMSInitializationRequest.Initiator> initiator = new AtomicReference<>(); @@ -127,14 +128,21 @@ private void initiate(Set<InetAddressAndPort> sendTo, ClusterMetadata metadata, private void finish(Set<InetAddressAndPort> sendTo) { CMSInitializationRequest.Initiator currentInitiator = initiator.get(); - assert currentInitiator.initiator.equals(FBUtilities.getBroadcastAddressAndPort()); - - Startup.initializeAsFirstCMSNode(); - Register.maybeRegister(); - SystemKeyspace.setLocalHostId(ClusterMetadata.current().myNodeId().toUUID()); + if (currentInitiator != null && + Objects.equals(currentInitiator.initiator, FBUtilities.getBroadcastAddressAndPort()) && + initiator.compareAndSet(currentInitiator, MIGRATING)) + { + Startup.initializeAsFirstCMSNode(); + Register.maybeRegister(); + SystemKeyspace.setLocalHostId(ClusterMetadata.current().myNodeId().toUUID()); - updateInitiator(currentInitiator, MIGRATED); - MessageDelivery.fanoutAndWait(messaging, sendTo, Verb.TCM_NOTIFY_REQ, DistributedMetadataLogKeyspace.getLogState(Epoch.EMPTY, false)); + updateInitiator(MIGRATING, MIGRATED); + MessageDelivery.fanoutAndWait(messaging, sendTo, Verb.TCM_NOTIFY_REQ, DistributedMetadataLogKeyspace.getLogState(Epoch.EMPTY, false)); + } + else + { + throw new IllegalStateException("Can't finish migration, initiator="+currentInitiator); + } } private void abort(Set<InetAddressAndPort> sendTo) @@ -166,6 +174,25 @@ public boolean isMigrating() return initiator != null && initiator != MIGRATED; } + public void abortInitialization(String initiatorEp) + { + InetAddressAndPort expectedInitiator = InetAddressAndPort.getByNameUnchecked(initiatorEp); + CMSInitializationRequest.Initiator currentInitiator = initiator.get(); + if (currentInitiator != null && Objects.equals(currentInitiator.initiator, expectedInitiator) && initiator.compareAndSet(currentInitiator, null)) + { + for (InetAddressAndPort ep : ClusterMetadata.current().directory.allJoinedEndpoints()) + { + if (!ep.equals(FBUtilities.getBroadcastAddressAndPort())) + messaging.send(Message.out(Verb.TCM_ABORT_MIG, currentInitiator), ep); + } + } + else + { + throw new IllegalStateException("Current initiator [" + currentInitiator +"] does not match provided " + expectedInitiator + + " - run this command on a node where initialization has not yet been cleared, with the correct expected initiator"); + } + } + public class PrepareHandler implements IVerbHandler<CMSInitializationRequest> { @Override @@ -209,7 +236,8 @@ public class AbortHandler implements IVerbHandler<CMSInitializationRequest.Initi public void doVerb(Message<CMSInitializationRequest.Initiator> message) throws IOException { logger.info("Received election abort message {} from {}", message.payload, message.from()); - if (!message.from().equals(initiator().initiator) || !updateInitiator(message.payload, null)) + CMSInitializationRequest.Initiator initiator = message.payload; + if (!initiator.initiator.equals(initiator().initiator) || !updateInitiator(message.payload, null)) logger.error("Could not clear initiator - initiator is set to {}, abort message received from {}", initiator(), message.payload); } } diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java index 49be441b89a6..d7bcc25b7110 100644 --- a/src/java/org/apache/cassandra/tools/NodeTool.java +++ b/src/java/org/apache/cassandra/tools/NodeTool.java @@ -268,7 +268,8 @@ public int execute(String... args) .withCommand(CMSAdmin.InitializeCMS.class) .withCommand(CMSAdmin.ReconfigureCMS.class) .withCommand(CMSAdmin.Snapshot.class) - .withCommand(CMSAdmin.Unregister.class); + .withCommand(CMSAdmin.Unregister.class) + .withCommand(CMSAdmin.AbortInitialization.class); Cli<NodeToolCmdRunnable> parser = builder.build(); diff --git a/src/java/org/apache/cassandra/tools/nodetool/CMSAdmin.java b/src/java/org/apache/cassandra/tools/nodetool/CMSAdmin.java index 0ed853ba68d2..02cc045545b7 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/CMSAdmin.java +++ b/src/java/org/apache/cassandra/tools/nodetool/CMSAdmin.java @@ -194,4 +194,17 @@ protected void execute(NodeProbe probe) probe.getCMSOperationsProxy().unregisterLeftNodes(nodeIds); } } + + @Command(name = "abortinitialization", description = "Abort an incomplete initialization") + public static class AbortInitialization extends NodeTool.NodeToolCmd + { + @Option(required = true, name = "--initiator", title = "Initiator", description = "The address of the node where `cms initialize` was run.") + public String initiator; + + @Override + protected void execute(NodeProbe probe) + { + probe.getCMSOperationsProxy().abortInitialization(initiator); + } + } } diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeAbortMigrationTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeAbortMigrationTest.java new file mode 100644 index 000000000000..bab722346d42 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeAbortMigrationTest.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.upgrade; + +import org.junit.Test; + +import org.apache.cassandra.distributed.Constants; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.net.Verb; + +public class ClusterMetadataUpgradeAbortMigrationTest extends UpgradeTestBase +{ + @Test + public void upgradeLostNotifyTest() throws Throwable + { + new UpgradeTestBase.TestCase() + .nodes(3) + .nodesToUpgrade(1, 2, 3) + .withConfig((cfg) -> cfg.with(Feature.NETWORK, Feature.GOSSIP) + .set(Constants.KEY_DTEST_FULL_STARTUP, true)) + .upgradesToCurrentFrom(v50) + .setup((cluster) -> { + cluster.schemaChange(withKeyspace("ALTER KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor':2}")); + cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))"); + }) + .runAfterClusterUpgrade((cluster) -> { + cluster.filters().verbs(Verb.TCM_INIT_MIG_RSP.id, + Verb.TCM_ABORT_MIG.id).drop(); + cluster.get(1).nodetoolResult("cms", "initialize").asserts().failure(); + cluster.filters().reset(); + // it was already cleared on node1: + cluster.get(1).nodetoolResult("cms", "abortinitialization", "--initiator", "127.0.0.1").asserts().failure(); + // but not on node2 + cluster.get(2).nodetoolResult("cms", "abortinitialization", "--initiator", "127.0.0.1").asserts().success(); + + cluster.get(1).nodetoolResult("cms", "initialize").asserts().success(); + }).run(); + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeTest.java index 930933425c25..eb717602f36b 100644 --- a/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeTest.java +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeTest.java @@ -113,6 +113,5 @@ public void upgradeMismatchTest() throws Throwable cluster.get(1).nodetoolResult("cms", "initialize", "--ignore", "127.0.0.3").asserts().success(); cluster.schemaChange(withKeyspace("create table %s.tbl2 (id int primary key)")); }).run(); - } - + } } From 4318e74180d710844b075fedc33e8ca008f87f63 Mon Sep 17 00:00:00 2001 From: Marcus Eriksson <marcuse@apache.org> Date: Mon, 24 Mar 2025 15:15:53 +0100 Subject: [PATCH 213/225] Various gossip to TCM upgrade fixes patch by marcuse; reviewed by Sam Tunnicliffe for CASSANDRA-20483 --- CHANGES.txt | 1 + .../apache/cassandra/gms/FailureDetector.java | 29 +++-- .../org/apache/cassandra/gms/Gossiper.java | 42 ++++++- .../cassandra/service/StorageService.java | 3 +- .../cassandra/tcm/ClusterMetadataService.java | 7 ++ .../org/apache/cassandra/tcm/Startup.java | 25 ++-- .../tcm/compatibility/GossipHelper.java | 72 ++++++++++- .../cassandra/tcm/membership/Directory.java | 2 +- .../cassandra/tcm/membership/NodeVersion.java | 2 +- .../cassandra/tcm/migration/Election.java | 13 +- .../tcm/migration/GossipCMSListener.java | 3 + .../tcm/sequences/BootstrapAndReplace.java | 12 ++ .../tcm/sequences/ReplaceSameAddress.java | 1 + .../cassandra/utils/CassandraVersion.java | 1 + .../distributed/UpgradeableCluster.java | 15 +++ ...ClusterMetadataUpgradeAssassinateTest.java | 29 +++-- .../ClusterMetadataUpgradeHibernateTest.java | 119 ++++++++++++++++++ .../ClusterMetadataUpgradeJoinRingTest.java | 102 +++++++++++++++ .../distributed/upgrade/UpgradeTestBase.java | 19 ++- .../tcm/compatibility/GossipHelperTest.java | 35 +++++- 20 files changed, 488 insertions(+), 44 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeHibernateTest.java create mode 100644 test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeJoinRingTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 864e66462043..0449ba45d5e0 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Various gossip to TCM upgrade fixes (CASSANDRA-20483) * Add nodetool command to abort failed nodetool cms initialize (CASSANDRA-20482) * Repair Paxos for the distributed metadata log when CMS membership changes (CASSANDRA-20467) * Reintroduce CASSANDRA-17411 in trunk (CASSANDRA-19346) diff --git a/src/java/org/apache/cassandra/gms/FailureDetector.java b/src/java/org/apache/cassandra/gms/FailureDetector.java index 34c98da4b7e4..49b208929748 100644 --- a/src/java/org/apache/cassandra/gms/FailureDetector.java +++ b/src/java/org/apache/cassandra/gms/FailureDetector.java @@ -47,12 +47,13 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.FSWriteError; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.MultiStepOperation; import org.apache.cassandra.tcm.membership.NodeId; +import org.apache.cassandra.tcm.sequences.BootstrapAndReplace; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.MBeanWrapper; @@ -150,7 +151,7 @@ public String getAllEndpointStates(boolean withPort, boolean resolveIp) for (Map.Entry<InetAddressAndPort, EndpointState> entry : Gossiper.instance.endpointStateMap.entrySet()) { sb.append(resolveIp ? entry.getKey().getHostName(withPort) : entry.getKey().toString(withPort)).append("\n"); - appendEndpointState(sb, entry.getValue()); + appendEndpointState(sb, entry.getKey(), entry.getValue()); } return sb.toString(); } @@ -242,12 +243,13 @@ private TabularData getPhiValues(boolean withPort) throws OpenDataException public String getEndpointState(String address) throws UnknownHostException { StringBuilder sb = new StringBuilder(); - EndpointState endpointState = Gossiper.instance.getEndpointStateForEndpoint(InetAddressAndPort.getByName(address)); - appendEndpointState(sb, endpointState); + InetAddressAndPort endpoint = InetAddressAndPort.getByName(address); + EndpointState endpointState = Gossiper.instance.getEndpointStateForEndpoint(endpoint); + appendEndpointState(sb, endpoint, endpointState); return sb.toString(); } - private void appendEndpointState(StringBuilder sb, EndpointState endpointState) + private void appendEndpointState(StringBuilder sb, InetAddressAndPort endpoint, EndpointState endpointState) { sb.append(" generation:").append(endpointState.getHeartBeatState().getGeneration()).append("\n"); sb.append(" heartbeat:").append(endpointState.getHeartBeatState().getHeartBeatVersion()).append("\n"); @@ -258,9 +260,20 @@ private void appendEndpointState(StringBuilder sb, EndpointState endpointState) sb.append(" ").append(state.getKey()).append(":").append(state.getValue().version).append(":").append(state.getValue().value).append("\n"); } ClusterMetadata metadata = ClusterMetadata.current(); - NodeId nodeId = metadata.directory.peerId(FBUtilities.getBroadcastAddressAndPort()); - List<Token> tokens = metadata.tokenMap.tokens(nodeId); - if (tokens != null && !tokens.isEmpty()) + NodeId nodeId = metadata.directory.peerId(endpoint); + boolean foundTokens = false; + if (nodeId != null) + { + foundTokens = !metadata.tokenMap.tokens(nodeId).isEmpty(); + if (!foundTokens) + { + MultiStepOperation<?> mso = metadata.inProgressSequences.get(nodeId); + if (mso instanceof BootstrapAndReplace) + foundTokens = true; + } + } + + if (foundTokens) sb.append(" TOKENS:").append(metadata.epoch.getEpoch()).append(":<hidden>\n"); else sb.append(" TOKENS: not present\n"); diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java index 5274a57ffb53..14cc5f5adaaf 100644 --- a/src/java/org/apache/cassandra/gms/Gossiper.java +++ b/src/java/org/apache/cassandra/gms/Gossiper.java @@ -454,6 +454,26 @@ public static boolean isShutdown(VersionedValue vv) return state.equals(VersionedValue.SHUTDOWN); } + public static boolean isHibernate(EndpointState epState) + { + VersionedValue versionedValue = epState.getApplicationState(ApplicationState.STATUS_WITH_PORT); + if (versionedValue == null) + versionedValue = epState.getApplicationState(ApplicationState.STATUS); + return isHibernate(versionedValue); + } + + public static boolean isHibernate(VersionedValue vv) + { + if (vv == null) + return false; + + String value = vv.value; + String[] pieces = value.split(VersionedValue.DELIMITER_STR, -1); + assert (pieces.length > 0); + String state = pieces[0]; + return state.equals(VersionedValue.HIBERNATE); + } + public static void runInGossipStageBlocking(Runnable runnable) { // run immediately if we're already in the gossip stage @@ -2106,10 +2126,18 @@ private void unsafeUpdateEpStates(InetAddressAndPort endpoint, EndpointState eps */ public void mergeNodeToGossip(NodeId nodeId, ClusterMetadata metadata) { - mergeNodeToGossip(nodeId, metadata, metadata.tokenMap.tokens(nodeId)); + mergeNodeToGossip(nodeId, metadata, metadata.tokenMap.tokens(nodeId), false); + } + public void mergeNodeToGossip(NodeId nodeId, ClusterMetadata metadata, boolean forceHibernate) + { + mergeNodeToGossip(nodeId, metadata, metadata.tokenMap.tokens(nodeId), forceHibernate); } - public void mergeNodeToGossip(NodeId nodeId, ClusterMetadata metadata, Collection<Token> tokens) + { + mergeNodeToGossip(nodeId, metadata, tokens, false); + } + + private void mergeNodeToGossip(NodeId nodeId, ClusterMetadata metadata, Collection<Token> tokens, boolean forceHibernate) { taskLock.lock(); try @@ -2156,7 +2184,7 @@ public void mergeNodeToGossip(NodeId nodeId, ClusterMetadata metadata, Collectio newValue = valueFactory.hostId(uuid); break; case TOKENS: - if (tokens != null) + if (tokens != null && !tokens.isEmpty()) newValue = valueFactory.tokens(tokens); break; case INTERNAL_ADDRESS_AND_PORT: @@ -2175,6 +2203,14 @@ public void mergeNodeToGossip(NodeId nodeId, ClusterMetadata metadata, Collectio // In this case, the app state will be set to `hibernate` by StorageService, so // don't set it here as nodeStateToStatus only considers persistent states (e.g. // ones stored in ClusterMetadata), it isn't aware of transient states like hibernate. + // forceHibernate can be true when upgrading from pre-tcm versions - if a node is hibernating + // we have no state for this in cluster metadata, so we need to explicitly keep that from + // the pre-upgrade gossip states + if (forceHibernate) + { + newValue = valueFactory.hibernate(true); + break; + } if (isLocal && !StorageService.instance.shouldJoinRing()) break; newValue = GossipHelper.nodeStateToStatus(nodeId, metadata, tokens, valueFactory, oldValue); diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 53f815a586db..812dc31cd492 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -2084,8 +2084,7 @@ public void onChange(InetAddressAndPort endpoint, ApplicationState state, Versio // normal STATUS. if (state == ApplicationState.STATUS_WITH_PORT) { - String[] pieces = splitValue(value); - if (pieces[0].equals(VersionedValue.HIBERNATE)) + if (Gossiper.isHibernate(value)) { logger.info("Node {} state jump to hibernate", endpoint); Gossiper.runInGossipStageBlocking(() -> { diff --git a/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java b/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java index 4c6eed11b4a1..3e3d8389ae2f 100644 --- a/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java +++ b/src/java/org/apache/cassandra/tcm/ClusterMetadataService.java @@ -318,6 +318,13 @@ public void upgradeFromGossip(List<String> ignoredEndpoints) } ClusterMetadata metadata = metadata(); + if (metadata.myNodeState() != NodeState.JOINED) + { + String msg = String.format("Initial CMS node needs to be fully joined, not: %s", metadata.myNodeState()); + logger.error(msg); + throw new IllegalStateException(msg); + } + Set<InetAddressAndPort> existingMembers = metadata.fullCMSMembers(); if (!metadata.directory.allAddresses().containsAll(ignored)) diff --git a/src/java/org/apache/cassandra/tcm/Startup.java b/src/java/org/apache/cassandra/tcm/Startup.java index edd8734fca27..d17b68769886 100644 --- a/src/java/org/apache/cassandra/tcm/Startup.java +++ b/src/java/org/apache/cassandra/tcm/Startup.java @@ -74,6 +74,7 @@ import static org.apache.cassandra.tcm.compatibility.GossipHelper.emptyWithSchemaFromSystemTables; import static org.apache.cassandra.tcm.compatibility.GossipHelper.fromEndpointStates; import static org.apache.cassandra.tcm.membership.NodeState.JOINED; +import static org.apache.cassandra.tcm.membership.NodeState.LEFT; import static org.apache.cassandra.utils.FBUtilities.getBroadcastAddressAndPort; /** @@ -313,18 +314,26 @@ public static void initializeFromGossip(Function<Processor, Processor> wrapProce logger.debug("Created initial ClusterMetadata {}", initial); ClusterMetadataService.instance().setFromGossip(initial); Gossiper.instance.clearUnsafe(); - if (switchIp != null) - { - // quarantine the old ip to make sure it doesn't get re-added via gossip - InetAddressAndPort removeEp = switchIp; - Gossiper.runInGossipStageBlocking(() -> Gossiper.instance.removeEndpoint(removeEp)); - } + + // find any endpoints that were ignored on upgrade and make sure they don't get re-added in gossip + InetAddressAndPort removeEp = switchIp; + Gossiper.runInGossipStageBlocking(() -> { + if (removeEp != null) + Gossiper.instance.removeEndpoint(removeEp); + for (InetAddressAndPort ep : epStates.keySet()) + { + if (initial.directory.peerId(ep) == null) + Gossiper.instance.removeEndpoint(ep); // just quarantines the ep - endpoint states should be empty + // here (this is run before Gossiper is started) + } + }); + Gossiper.instance.maybeInitializeLocalState(SystemKeyspace.incrementAndGetGeneration()); for (Map.Entry<NodeId, NodeState> entry : initial.directory.states.entrySet()) { InetAddressAndPort ep = initial.directory.addresses.get(entry.getKey()).broadcastAddress; - if (entry.getValue() != NodeState.LEFT) - Gossiper.instance.mergeNodeToGossip(entry.getKey(), initial); + if (entry.getValue() != LEFT) + Gossiper.instance.mergeNodeToGossip(entry.getKey(), initial, Gossiper.isHibernate(epStates.get(ep))); else Gossiper.runInGossipStageBlocking(() -> Gossiper.instance.endpointStateMap.put(ep, epStates.get(ep))); } diff --git a/src/java/org/apache/cassandra/tcm/compatibility/GossipHelper.java b/src/java/org/apache/cassandra/tcm/compatibility/GossipHelper.java index 0e1acbc4eac3..1a555fce4d36 100644 --- a/src/java/org/apache/cassandra/tcm/compatibility/GossipHelper.java +++ b/src/java/org/apache/cassandra/tcm/compatibility/GossipHelper.java @@ -87,6 +87,7 @@ import static org.apache.cassandra.gms.ApplicationState.RPC_ADDRESS; import static org.apache.cassandra.gms.ApplicationState.STATUS_WITH_PORT; import static org.apache.cassandra.gms.ApplicationState.TOKENS; +import static org.apache.cassandra.gms.Gossiper.isHibernate; import static org.apache.cassandra.gms.Gossiper.isShutdown; import static org.apache.cassandra.locator.InetAddressAndPort.getByName; import static org.apache.cassandra.locator.InetAddressAndPort.getByNameOverrideDefaults; @@ -123,6 +124,8 @@ public static VersionedValue nodeStateToStatus(NodeId nodeId, case JOINED: if (isShutdown(oldValue)) status = valueFactory.shutdown(true); + else if (isHibernate(oldValue)) + status = valueFactory.hibernate(true); else status = valueFactory.normal(tokens); break; @@ -224,10 +227,13 @@ private static NodeState toNodeState(InetAddressAndPort endpoint, EndpointState String status = epState.getStatus(); if (status.equals(VersionedValue.STATUS_NORMAL) || - status.equals(VersionedValue.SHUTDOWN)) + status.equals(VersionedValue.SHUTDOWN) || + status.equals(VersionedValue.HIBERNATE)) return NodeState.JOINED; if (status.equals(VersionedValue.STATUS_LEFT)) return NodeState.LEFT; + if (status.isEmpty()) + return NodeState.REGISTERED; throw new IllegalStateException("Can't upgrade the first node when STATUS = " + status + " for node " + endpoint); } @@ -332,6 +338,12 @@ public static ClusterMetadata fromEndpointStates(Map<InetAddressAndPort, Endpoin { Directory directory = new Directory().withLastModified(Epoch.UPGRADE_GOSSIP); TokenMap tokenMap = new TokenMap(partitioner).withLastModified(Epoch.UPGRADE_GOSSIP); + + // gossip can contain old hosts with duplicate host ids during upgrades from pre-TCM versions. We need to clean + // those up during upgrades since TCM is more strict. Here we simply keep the host with the newest gossip + // generation if there is a duplicate hostid. + if (containsDuplicateHostIds(epStates)) + epStates = cleanupDuplicateHostIds(epStates); List<InetAddressAndPort> sortedEps = Lists.newArrayList(epStates.keySet()); Collections.sort(sortedEps); Map<ExtensionKey<?, ?>, ExtensionValue<?>> extensions = new HashMap<>(); @@ -344,14 +356,20 @@ public static ClusterMetadata fromEndpointStates(Map<InetAddressAndPort, Endpoin NodeAddresses nodeAddresses = getAddressesFromEndpointState(endpoint, epState); NodeVersion nodeVersion = getVersionFromEndpointState(endpoint, epState); assert hostIdString != null; - NodeState nodeState = toNodeState(endpoint, epState); + // some clusters have old, removed hibernating endpoints in gossip, ignore these if they don't exist in our system.peers_v2 table + if (!endpoint.equals(FBUtilities.getBroadcastAddressAndPort()) && Gossiper.isHibernate(epState) && !SystemKeyspace.loadTokens().containsKey(endpoint)) + { + logger.info("Ignoring endpoint {} with endpoint states {} since it is missing from system.peers_v2", endpoint, epState); + continue; + } + NodeState nodeState = toNodeState(endpoint, epState); directory = directory.withNonUpgradedNode(nodeAddresses, new Location(dc, rack), nodeVersion, nodeState, UUID.fromString(hostIdString)); - if (nodeState != NodeState.LEFT) + if (nodeState == NodeState.JOINED) { NodeId nodeId = directory.peerId(endpoint); tokenMap = tokenMap.assignTokens(nodeId, getTokensIn(partitioner, epState)); @@ -385,7 +403,7 @@ public static boolean isValidForClusterMetadata(Map<InetAddressAndPort, Endpoint { if (epstates.isEmpty()) return false; - EnumSet<ApplicationState> requiredStates = EnumSet.of(DC, RACK, HOST_ID, TOKENS, RELEASE_VERSION); + EnumSet<ApplicationState> requiredStates = EnumSet.of(DC, RACK, HOST_ID, RELEASE_VERSION); for (Map.Entry<InetAddressAndPort, EndpointState> entry : epstates.entrySet()) { EndpointState epstate = entry.getValue(); @@ -398,4 +416,50 @@ public static boolean isValidForClusterMetadata(Map<InetAddressAndPort, Endpoint } return true; } + + private static boolean containsDuplicateHostIds(Map<InetAddressAndPort, EndpointState> epstates) + { + Set<String> hostIds = new HashSet<>(); + for (EndpointState epstate : epstates.values()) + { + String hostIdString = epstate.getApplicationState(HOST_ID).value; + if (hostIds.contains(hostIdString)) + return true; + hostIds.add(hostIdString); + } + return false; + } + + private static Map<InetAddressAndPort, EndpointState> cleanupDuplicateHostIds(Map<InetAddressAndPort, EndpointState> epstates) + { + Map<InetAddressAndPort, EndpointState> cleanEpstates = new HashMap<>(); + Map<String, InetAddressAndPort> seenHostIds = new HashMap<>(); + for (Map.Entry<InetAddressAndPort, EndpointState> entry : epstates.entrySet()) + { + InetAddressAndPort endpoint = entry.getKey(); + EndpointState epstate = entry.getValue(); + String hostIdString = epstate.getApplicationState(HOST_ID).value; + if (seenHostIds.containsKey(hostIdString)) + { + int thisGeneration = epstate.getHeartBeatState().getGeneration(); + + InetAddressAndPort seenHost = seenHostIds.get(hostIdString); + int seenGeneration = epstates.get(seenHost).getHeartBeatState().getGeneration(); + logger.warn("Duplicate host id {} found: {} with generation {} and {} with generation {}, keeping the one with the newest generation", + hostIdString, seenHost, seenGeneration, endpoint, thisGeneration); + if (thisGeneration > seenGeneration) + { + cleanEpstates.remove(seenHost); + cleanEpstates.put(endpoint, epstate); + seenHostIds.put(hostIdString, endpoint); + } + } + else + { + seenHostIds.put(hostIdString, endpoint); + cleanEpstates.put(endpoint, epstate); + } + } + return cleanEpstates; + } } diff --git a/src/java/org/apache/cassandra/tcm/membership/Directory.java b/src/java/org/apache/cassandra/tcm/membership/Directory.java index 90af4ff79f51..51ab84c52051 100644 --- a/src/java/org/apache/cassandra/tcm/membership/Directory.java +++ b/src/java/org/apache/cassandra/tcm/membership/Directory.java @@ -157,7 +157,7 @@ public Directory withNonUpgradedNode(NodeAddresses addresses, { NodeId id = new NodeId(nextId); Directory updated = with(addresses, id, hostId, location, version).withNodeState(id, state); - if (state != NodeState.LEFT) + if (state == NodeState.JOINED) updated = updated.withRackAndDC(id); return updated; } diff --git a/src/java/org/apache/cassandra/tcm/membership/NodeVersion.java b/src/java/org/apache/cassandra/tcm/membership/NodeVersion.java index e5b608994aa0..bc1bcc707e3d 100644 --- a/src/java/org/apache/cassandra/tcm/membership/NodeVersion.java +++ b/src/java/org/apache/cassandra/tcm/membership/NodeVersion.java @@ -36,7 +36,7 @@ public class NodeVersion implements Comparable<NodeVersion> public static final Serializer serializer = new Serializer(); public static final Version CURRENT_METADATA_VERSION = Version.V6; public static final NodeVersion CURRENT = new NodeVersion(new CassandraVersion(FBUtilities.getReleaseVersionString()), CURRENT_METADATA_VERSION); - private static final CassandraVersion SINCE_VERSION = CassandraVersion.CASSANDRA_5_0; + private static final CassandraVersion SINCE_VERSION = CassandraVersion.CASSANDRA_5_1; public final CassandraVersion cassandraVersion; public final int serializationVersion; diff --git a/src/java/org/apache/cassandra/tcm/migration/Election.java b/src/java/org/apache/cassandra/tcm/migration/Election.java index 69fac6f4ba7b..94f5dc4a06d3 100644 --- a/src/java/org/apache/cassandra/tcm/migration/Election.java +++ b/src/java/org/apache/cassandra/tcm/migration/Election.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Collection; import java.util.HashSet; +import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.UUID; @@ -38,6 +39,8 @@ import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.Startup; import org.apache.cassandra.tcm.membership.Directory; +import org.apache.cassandra.tcm.membership.NodeId; +import org.apache.cassandra.tcm.membership.NodeState; import org.apache.cassandra.tcm.ownership.TokenMap; import org.apache.cassandra.tcm.transformations.Register; import org.apache.cassandra.net.MessageDelivery; @@ -50,6 +53,8 @@ import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.Pair; +import static org.apache.cassandra.tcm.membership.NodeState.LEFT; + /** * Election process establishes initial CMS leader, from which you can further evolve cluster metadata. */ @@ -180,10 +185,12 @@ public void abortInitialization(String initiatorEp) CMSInitializationRequest.Initiator currentInitiator = initiator.get(); if (currentInitiator != null && Objects.equals(currentInitiator.initiator, expectedInitiator) && initiator.compareAndSet(currentInitiator, null)) { - for (InetAddressAndPort ep : ClusterMetadata.current().directory.allJoinedEndpoints()) + ClusterMetadata metadata = ClusterMetadata.current(); + for (Map.Entry<NodeId, NodeState> entry : metadata.directory.states.entrySet()) { - if (!ep.equals(FBUtilities.getBroadcastAddressAndPort())) - messaging.send(Message.out(Verb.TCM_ABORT_MIG, currentInitiator), ep); + NodeId nodeId = entry.getKey(); + if (!Objects.equals(metadata.myNodeId(), nodeId) && entry.getValue() != LEFT) + messaging.send(Message.out(Verb.TCM_ABORT_MIG, currentInitiator), metadata.directory.endpoint(nodeId)); } } else diff --git a/src/java/org/apache/cassandra/tcm/migration/GossipCMSListener.java b/src/java/org/apache/cassandra/tcm/migration/GossipCMSListener.java index b712a695a2d9..97ff11b0e7e3 100644 --- a/src/java/org/apache/cassandra/tcm/migration/GossipCMSListener.java +++ b/src/java/org/apache/cassandra/tcm/migration/GossipCMSListener.java @@ -54,6 +54,9 @@ public void onJoin(InetAddressAndPort endpoint, EndpointState epState) if (nodeId == null) { VersionedValue hostIdValue = epState.getApplicationState(ApplicationState.HOST_ID); + if (Gossiper.isHibernate(epState)) + return; + if (hostIdValue != null) { UUID hostId = UUID.fromString(hostIdValue.value); diff --git a/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndReplace.java b/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndReplace.java index e774b77ebe9c..2b283d6905af 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndReplace.java +++ b/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndReplace.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.HashSet; import java.util.List; import java.util.Objects; @@ -443,6 +444,17 @@ public static void gossipStateToHibernate(ClusterMetadata metadata, NodeId nodeI Gossiper.instance.addLocalApplicationStates(states); } + public static void gossipStateToNormal(ClusterMetadata metadata, NodeId nodeId) + { + List<Pair<ApplicationState, VersionedValue>> states = new ArrayList<>(); + VersionedValue.VersionedValueFactory valueFactory = StorageService.instance.valueFactory; + Collection<Token> tokens = metadata.tokenMap.tokens(nodeId); + states.add(Pair.create(ApplicationState.TOKENS, valueFactory.tokens(tokens))); + states.add(Pair.create(ApplicationState.STATUS_WITH_PORT, valueFactory.normal(tokens))); + states.add(Pair.create(ApplicationState.STATUS, valueFactory.normal(tokens))); + Gossiper.instance.addLocalApplicationStates(states); + } + public static class Serializer implements AsymmetricMetadataSerializer<MultiStepOperation<?>, BootstrapAndReplace> { public void serialize(MultiStepOperation<?> t, DataOutputPlus out, Version version) throws IOException diff --git a/src/java/org/apache/cassandra/tcm/sequences/ReplaceSameAddress.java b/src/java/org/apache/cassandra/tcm/sequences/ReplaceSameAddress.java index 2c4553a14f70..be75538d6888 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/ReplaceSameAddress.java +++ b/src/java/org/apache/cassandra/tcm/sequences/ReplaceSameAddress.java @@ -93,6 +93,7 @@ public static void streamData(NodeId nodeId, ClusterMetadata metadata, boolean s StreamSupport.stream(ColumnFamilyStore.all().spliterator(), false) .filter(cfs -> Schema.instance.getUserKeyspaces().names().contains(cfs.keyspace.getName())) .forEach(cfs -> cfs.indexManager.executePreJoinTasksBlocking(true)); + BootstrapAndReplace.gossipStateToNormal(metadata, metadata.myNodeId()); Gossiper.instance.mergeNodeToGossip(metadata.myNodeId(), metadata); } } diff --git a/src/java/org/apache/cassandra/utils/CassandraVersion.java b/src/java/org/apache/cassandra/utils/CassandraVersion.java index 587e06d0618d..27891f560a8c 100644 --- a/src/java/org/apache/cassandra/utils/CassandraVersion.java +++ b/src/java/org/apache/cassandra/utils/CassandraVersion.java @@ -50,6 +50,7 @@ public class CassandraVersion implements Comparable<CassandraVersion> private static final Pattern PATTERN = Pattern.compile(VERSION_REGEXP); + public static final CassandraVersion CASSANDRA_5_1 = new CassandraVersion("5.1").familyLowerBound.get(); public static final CassandraVersion CASSANDRA_5_0 = new CassandraVersion("5.0").familyLowerBound.get(); public static final CassandraVersion CASSANDRA_4_1 = new CassandraVersion("4.1").familyLowerBound.get(); public static final CassandraVersion CASSANDRA_4_0 = new CassandraVersion("4.0").familyLowerBound.get(); diff --git a/test/distributed/org/apache/cassandra/distributed/UpgradeableCluster.java b/test/distributed/org/apache/cassandra/distributed/UpgradeableCluster.java index b7bd1d6d662d..68b542c84868 100644 --- a/test/distributed/org/apache/cassandra/distributed/UpgradeableCluster.java +++ b/test/distributed/org/apache/cassandra/distributed/UpgradeableCluster.java @@ -19,11 +19,14 @@ package org.apache.cassandra.distributed; import java.io.IOException; +import java.util.Map; import java.util.function.Consumer; import org.apache.cassandra.distributed.api.IInstanceConfig; import org.apache.cassandra.distributed.api.IUpgradeableInstance; +import org.apache.cassandra.distributed.api.TokenSupplier; import org.apache.cassandra.distributed.impl.AbstractCluster; +import org.apache.cassandra.distributed.shared.NetworkTopology; import org.apache.cassandra.distributed.shared.Versions; /** @@ -73,6 +76,18 @@ public static UpgradeableCluster create(int nodeCount, Versions.Version version, return builder.start(); } + public static UpgradeableCluster create(int nodeCount, Versions.Version version, Consumer<IInstanceConfig> configUpdater, Consumer<Builder> builderUpdater, TokenSupplier tokenSupplier, Map<Integer, NetworkTopology.DcAndRack> nodeIdTopology) throws IOException + { + Builder builder = build(nodeCount).withConfig(configUpdater).withVersion(version); + if (tokenSupplier != null) + builder = builder.withTokenSupplier(tokenSupplier); + if (nodeIdTopology != null) + builder = builder.withNodeIdTopology(nodeIdTopology); + if (builderUpdater != null) + builderUpdater.accept(builder); + return builder.start(); + } + public static UpgradeableCluster create(int nodeCount, Versions.Version version) throws Throwable { return build(nodeCount).withVersion(version).start(); diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeAssassinateTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeAssassinateTest.java index 9e5be9639a14..7712a040d932 100644 --- a/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeAssassinateTest.java +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeAssassinateTest.java @@ -43,24 +43,31 @@ public void simpleUpgradeTest() throws Throwable cluster.get(1).nodetoolResult("assassinate", "127.0.0.3").asserts().success(); }) .runAfterClusterUpgrade((cluster) -> { - checkPlacements(cluster.get(1)); - checkPlacements(cluster.get(2)); + checkPlacements(cluster.get(1), false); + checkPlacements(cluster.get(2), false); cluster.get(1).nodetoolResult("cms", "initialize").asserts().success(); - checkPlacements(cluster.get(1)); - checkPlacements(cluster.get(2)); + checkPlacements(cluster.get(1), false); + checkPlacements(cluster.get(2), false); }).run(); } - - private void checkPlacements(IUpgradeableInstance i) + static void checkPlacements(IUpgradeableInstance i, boolean shouldExist) + { + checkPlacements(i, "127.0.0.3", shouldExist); + } + static void checkPlacements(IUpgradeableInstance i, String host, boolean shouldExist) { ((IInvokableInstance) i).runOnInstance(() -> { ClusterMetadata metadata = ClusterMetadata.current(); - InetAddressAndPort ep = InetAddressAndPort.getByNameUnchecked("127.0.0.3"); + InetAddressAndPort ep = InetAddressAndPort.getByNameUnchecked(host); metadata.placements.asMap().forEach((key, value) -> { - if (Streams.concat(value.reads.endpoints.stream(), - value.writes.endpoints.stream()) - .anyMatch(fr -> fr.endpoints().contains(ep))) - throw new IllegalStateException(ep + " should not be in placements " + metadata.placements); + if (key.isMeta()) + return; + boolean existsInPlacements = Streams.concat(value.reads.endpoints.stream(), + value.writes.endpoints.stream()) + .anyMatch(fr -> fr.endpoints().contains(ep)); + if (shouldExist != existsInPlacements) + throw new IllegalStateException(ep + " should" + (shouldExist ? "" : " not")+ " be in placements " + key + " : " + value); + }); }); } diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeHibernateTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeHibernateTest.java new file mode 100644 index 000000000000..35c108193abc --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeHibernateTest.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.upgrade; + +import java.util.Map; + +import org.junit.Test; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.api.IInstance; +import org.apache.cassandra.distributed.shared.ClusterUtils; +import org.apache.cassandra.distributed.shared.Versions; +import org.apache.cassandra.gms.ApplicationState; + +import static org.apache.cassandra.distributed.action.GossipHelper.withProperty; +import static org.apache.cassandra.distributed.upgrade.ClusterMetadataUpgradeAssassinateTest.checkPlacements; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class ClusterMetadataUpgradeHibernateTest extends UpgradeTestBase +{ + @Test + public void hibernateUpgradeTest() throws Throwable + { + new TestCase() + .nodes(3) + .nodesToUpgrade(1, 2) // not node3 - we manually upgrade that below + .withConfig((cfg) -> cfg.with(Feature.NETWORK, Feature.GOSSIP)) + .upgradesToCurrentFrom(v50) + .setup((cluster) -> { + cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))"); + cluster.get(3).shutdown().get(); + // stopping a fully joined node and then starting it with join_ring=false puts it in hibernate status - it still owns tokens: + withProperty(CassandraRelevantProperties.JOIN_RING, false, () -> cluster.get(3).startup()); + assertTrue(hibernating(cluster.get(1), "127.0.0.3")); + }) + .runAfterClusterUpgrade((cluster) -> { + // manually upgrade node3 to be able to keep join_ring=false + cluster.get(3).shutdown().get(); + cluster.get(3).setVersion(Versions.find().getLatest(v51)); + assertTrue(hibernating(cluster.get(1), "127.0.0.3")); + withProperty(CassandraRelevantProperties.JOIN_RING, false, () -> cluster.get(3).startup()); + cluster.forEach(i -> checkPlacements(i, true)); + assertTrue(hibernating(cluster.get(1), "127.0.0.3")); + cluster.forEach(i -> checkPlacements(i, true)); + cluster.get(1).nodetoolResult("cms", "initialize").asserts().success(); + assertTrue(hibernating(cluster.get(1), "127.0.0.3")); + cluster.forEach(i -> checkPlacements(i, true)); + + // and remove join_ring=false and make sure it is no longer hibernating + cluster.get(3).shutdown().get(); + cluster.get(3).startup(); + assertFalse(hibernating(cluster.get(1), "127.0.0.3")); + cluster.forEach(i -> checkPlacements(i, true)); + }).run(); + } + + @Test + public void hibernateBadGossipUpgradeTest() throws Throwable + { + new TestCase() + .nodes(3) + .nodesToUpgrade(1, 2) + .withConfig((cfg) -> cfg.with(Feature.NETWORK, Feature.GOSSIP)) + .upgradesToCurrentFrom(v50) + .setup((cluster) -> { + cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))"); + cluster.get(3).shutdown().get(); + withProperty(CassandraRelevantProperties.JOIN_RING, false, () -> cluster.get(3).startup()); + cluster.get(3).shutdown(); + assertTrue(hibernating(cluster.get(1), "127.0.0.3")); + assertTrue(hibernating(cluster.get(2), "127.0.0.3")); + // terrible - we might have old hibernating nodes in gossip which don't exist in peers_v2 - this + // is an approximation of that state to be able to upgrade and ignore these nodes + for (int i = 1; i <= 2; i++) + cluster.get(i).executeInternal("delete from system.peers_v2 where peer = '127.0.0.3'"); + }) + .runAfterClusterUpgrade((cluster) -> { + checkPlacements(cluster.get(1), false); + checkPlacements(cluster.get(2), false); + // 127.0.0.3 should have been ignored on upgrade: + assertFalse(hibernating(cluster.get(1), "127.0.0.3")); + cluster.get(1).nodetoolResult("cms", "initialize").asserts().success(); + assertFalse(hibernating(cluster.get(1), "127.0.0.3")); + cluster.get(2).shutdown().get(); + cluster.get(2).startup(); + assertFalse(hibernating(cluster.get(2), "127.0.0.3")); + checkPlacements(cluster.get(1), false); + checkPlacements(cluster.get(2), false); + }).run(); + } + + private static boolean hibernating(IInstance instance, String host) + { + Map<String, Map<String, String>> states = ClusterUtils.gossipInfo(instance); + Map<String, String> state = states.get('/'+host); + if (state == null) + return false; + String status = state.get(ApplicationState.STATUS_WITH_PORT.name()); + return status != null && status.contains("hibernate"); + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeJoinRingTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeJoinRingTest.java new file mode 100644 index 000000000000..d9c2ef8b4cff --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/ClusterMetadataUpgradeJoinRingTest.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.upgrade; + +import org.junit.Test; + +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.distributed.UpgradeableCluster; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.api.IInstanceConfig; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.api.IUpgradeableInstance; +import org.apache.cassandra.distributed.api.TokenSupplier; +import org.apache.cassandra.distributed.shared.ClusterUtils; +import org.apache.cassandra.distributed.shared.NetworkTopology; +import org.apache.cassandra.distributed.shared.Uninterruptibles; +import org.apache.cassandra.distributed.shared.Versions; +import org.apache.cassandra.tcm.ClusterMetadataService; + +import static org.apache.cassandra.distributed.action.GossipHelper.withProperty; +import static org.apache.cassandra.distributed.upgrade.ClusterMetadataUpgradeAssassinateTest.checkPlacements; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class ClusterMetadataUpgradeJoinRingTest extends UpgradeTestBase +{ + @Test + public void joinRingUpgradeTest() throws Throwable + { + TokenSupplier ts = TokenSupplier.evenlyDistributedTokens(4); + + new TestCase() + .nodes(3) + .nodesToUpgrade(1, 2, 3) + .withTokenSupplier(ts::tokens) + .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(4, "dc0", "rack0")) + .withConfig((cfg) -> cfg.with(Feature.NETWORK, Feature.GOSSIP)) + .upgradesToCurrentFrom(v50) + .setup((cluster) -> { + cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))"); + IInstanceConfig nodeConfig = cluster.newInstanceConfig(); + IUpgradeableInstance newInstance = cluster.bootstrap(nodeConfig); + withProperty(CassandraRelevantProperties.JOIN_RING, false, newInstance::startup); + checkGossipinfo(cluster, false); + }) + .runAfterClusterUpgrade((cluster) -> { + checkGossipinfo(cluster, false); + // node4 not upgraded yet - should be allowed to vote despite being join_ring=false: + cluster.get(1).nodetoolResult("cms", "initialize").asserts().failure(); + cluster.get(4).shutdown().get(); + cluster.get(4).setVersion(Versions.find().getLatest(v51)); + withProperty(CassandraRelevantProperties.JOIN_RING, false, () -> cluster.get(4).startup()); + checkGossipinfo(cluster, false); + checkPlacements(cluster.get(1), "127.0.0.4", false); + + // before "cms initialize" - shouldn't be allowed to join + cluster.get(4).nodetoolResult("join").asserts().failure(); + checkGossipinfo(cluster, false); + // don't allow non-joined nodes to become initial cms: + cluster.get(4).nodetoolResult("cms", "initialize").asserts().failure(); + + cluster.get(1).nodetoolResult("cms", "initialize").asserts().success(); + checkGossipinfo(cluster, false); + ((IInvokableInstance)cluster.get(4)).runOnInstance(() -> { + while (ClusterMetadataService.state() == ClusterMetadataService.State.GOSSIP) + Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS); + }); + cluster.get(4).nodetoolResult("join").asserts().success(); + checkGossipinfo(cluster, true); + checkPlacements(cluster.get(1), "127.0.0.4", true); + + }).run(); + } + + private void checkGossipinfo(UpgradeableCluster cluster, boolean shouldBeJoined) + { + Map<String, Map<String, String>> states = ClusterUtils.gossipInfo(cluster.get(1)); + Map<String, String> node4State = states.get("/127.0.0.4"); + assertTrue(node4State != null && !node4State.isEmpty()); + assertEquals(!shouldBeJoined, node4State.get("TOKENS").contains("not present")); + assertEquals(shouldBeJoined, node4State.containsKey("STATUS_WITH_PORT")); + } + +} diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTestBase.java b/test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTestBase.java index 44d8d35b40bf..6e6aaaef7669 100644 --- a/test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTestBase.java +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/UpgradeTestBase.java @@ -24,6 +24,7 @@ import java.util.HashSet; import java.util.LinkedHashSet; import java.util.List; +import java.util.Map; import java.util.NavigableSet; import java.util.Objects; import java.util.Set; @@ -44,7 +45,9 @@ import org.apache.cassandra.distributed.api.Feature; import org.apache.cassandra.distributed.api.ICluster; import org.apache.cassandra.distributed.api.IInstanceConfig; +import org.apache.cassandra.distributed.api.TokenSupplier; import org.apache.cassandra.distributed.shared.DistributedTestBase; +import org.apache.cassandra.distributed.shared.NetworkTopology; import org.apache.cassandra.distributed.shared.ThrowingRunnable; import org.apache.cassandra.distributed.shared.Versions; import org.apache.cassandra.utils.ByteBufferUtil; @@ -159,6 +162,8 @@ public static class TestCase implements ThrowingRunnable private final Set<Integer> nodesToUpgrade = new LinkedHashSet<>(); private Consumer<IInstanceConfig> configConsumer; private Consumer<UpgradeableCluster.Builder> builderConsumer; + private TokenSupplier tokenSupplier; + private Map<Integer, NetworkTopology.DcAndRack> nodeIdTopology; private UpgradeListener upgradeListener = new UpgradeListener() { @Override @@ -334,6 +339,18 @@ public TestCase withUpgradeListener(UpgradeListener listener) return this; } + public TestCase withTokenSupplier(TokenSupplier tokenSupplier) + { + this.tokenSupplier = tokenSupplier; + return this; + } + + public TestCase withNodeIdTopology(Map<Integer, NetworkTopology.DcAndRack> nodeIdTopology) + { + this.nodeIdTopology = nodeIdTopology; + return this; + } + public void run() throws Throwable { if (setup == null) @@ -358,7 +375,7 @@ public void run() throws Throwable for (TestVersions upgrade : this.upgrade) { logger.info("testing upgrade from {} to {}", upgrade.initial.version, upgrade.upgradeVersions); - try (UpgradeableCluster cluster = init(UpgradeableCluster.create(nodeCount, upgrade.initial, configConsumer, builderConsumer))) + try (UpgradeableCluster cluster = init(UpgradeableCluster.create(nodeCount, upgrade.initial, configConsumer, builderConsumer, tokenSupplier, nodeIdTopology))) { setup.run(cluster); diff --git a/test/unit/org/apache/cassandra/tcm/compatibility/GossipHelperTest.java b/test/unit/org/apache/cassandra/tcm/compatibility/GossipHelperTest.java index 3ff7c03e8323..a7ec4ad78003 100644 --- a/test/unit/org/apache/cassandra/tcm/compatibility/GossipHelperTest.java +++ b/test/unit/org/apache/cassandra/tcm/compatibility/GossipHelperTest.java @@ -54,6 +54,7 @@ import static org.apache.cassandra.locator.InetAddressAndPort.getByName; import static org.apache.cassandra.tcm.compatibility.GossipHelper.fromEndpointStates; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -146,6 +147,33 @@ public void testPlacements() throws UnknownHostException verifyPlacements(endpoints, metadata); } + @Test + public void duplicateHostIdTest() throws UnknownHostException + { + int nodes = 10; + Keyspaces kss = Keyspaces.NONE.with(KSM_NTS); + DistributedSchema schema = new DistributedSchema(kss); + + Map<InetAddressAndPort, EndpointState> epstates = new HashMap<>(); + UUID toDupe = UUID.randomUUID(); + for (int i = 1; i < nodes; i++) + { + long t = i * 1000L; + InetAddressAndPort endpoint = getByName("127.0.0."+i); + Token token = t(t); + UUID hostId = i == 1 ? toDupe : UUID.randomUUID(); + EndpointState endpointState = epstate(100, endpoint, endpoint, token, hostId, i % 2 == 1 ? "dc1" : "dc2"); + epstates.put(endpoint, endpointState); + } + + InetAddressAndPort oldDuplicate = getByName("127.0.0." + nodes); + epstates.put(oldDuplicate, epstate(50, oldDuplicate, oldDuplicate, t(nodes), toDupe, "dc2")); + + ClusterMetadata metadata = fromEndpointStates(epstates, Murmur3Partitioner.instance, schema); + assertEquals(epstates.size() - 1, metadata.directory.addresses.size()); + assertNull(metadata.directory.peerId(oldDuplicate)); + } + private static void verifyPlacements(Map<Integer, Token> endpoints, ClusterMetadata metadata) throws UnknownHostException { // quick check to make sure cm.placements is populated @@ -185,8 +213,11 @@ private static void verify(EndpointsForToken eps, int ... endpoints) throws Unkn assertTrue("endpoint "+ep+" should be in " + eps, eps.contains(ep)); } } - private static EndpointState epstate(InetAddressAndPort internalAddress, InetAddressAndPort nativeAddress, Token token, UUID hostId, String dc) + { + return epstate(1, internalAddress, nativeAddress, token, hostId, dc); + } + private static EndpointState epstate(int generation, InetAddressAndPort internalAddress, InetAddressAndPort nativeAddress, Token token, UUID hostId, String dc) { Map<ApplicationState, VersionedValue> versionedValues = new EnumMap<>(ApplicationState.class); versionedValues.put(STATUS_WITH_PORT, vvf.normal(Collections.singleton(token))); @@ -197,7 +228,7 @@ private static EndpointState epstate(InetAddressAndPort internalAddress, InetAdd versionedValues.put(RELEASE_VERSION, vvf.releaseVersion("3.0.24")); versionedValues.put(NATIVE_ADDRESS_AND_PORT, vvf.nativeaddressAndPort(nativeAddress)); versionedValues.put(INTERNAL_ADDRESS_AND_PORT, vvf.internalAddressAndPort(internalAddress)); - return new EndpointState(new HeartBeatState(1, 1), versionedValues); + return new EndpointState(new HeartBeatState(generation, 1), versionedValues); } private EndpointState withState(String status) throws UnknownHostException From bc8941e377db8d48ad70142ad94524743c3ef897 Mon Sep 17 00:00:00 2001 From: Dmitry Konstantinov <konstantinov@netcracker.com> Date: Wed, 19 Mar 2025 22:05:01 +0000 Subject: [PATCH 214/225] Fix CommitlogShutdownTest flakiness by testing a real drain logic instead an emulation which is affected by background activities for system tables Patch by Dmitry Konstantinov; reviewed by Josh McKenzie for CASSANDRA-19101 --- .../db/commitlog/CommitlogShutdownTest.java | 28 +++++++++++++------ 1 file changed, 19 insertions(+), 9 deletions(-) diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitlogShutdownTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitlogShutdownTest.java index 711cf65c257e..a6b12bc00c85 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/CommitlogShutdownTest.java +++ b/test/unit/org/apache/cassandra/db/commitlog/CommitlogShutdownTest.java @@ -20,8 +20,10 @@ import java.io.File; import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; import java.util.Random; -import java.util.UUID; import com.google.common.collect.ImmutableMap; import org.junit.Assert; @@ -40,7 +42,7 @@ import org.apache.cassandra.db.marshal.AsciiType; import org.apache.cassandra.db.marshal.BytesType; import org.apache.cassandra.schema.KeyspaceParams; -import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.service.StorageService; import org.jboss.byteman.contrib.bmunit.BMRule; import org.jboss.byteman.contrib.bmunit.BMUnitRunner; @@ -85,15 +87,23 @@ public void testShutdownWithPendingTasks() throws Exception // force creating several commitlog files for (int i = 0; i < 10; i++) { - CommitLog.instance.add(m); + m.apply(); } - // schedule discarding completed segments and immediately issue a shutdown - TableId tableId = m.getTableIds().iterator().next(); - CommitLog.instance.discardCompletedSegments(tableId, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition()); - CommitLog.instance.shutdownBlocking(); + StorageService.instance.drain(); - // the shutdown should block until all logs except the currently active one and perhaps a new, empty one are gone - Assert.assertTrue(new File(DatabaseDescriptor.getCommitLogLocation()).listFiles().length <= 2); + List<CommitLogSegment> segmentsToCheck = new ArrayList<>(CommitLog.instance.segmentManager.getActiveSegments()); + // remove the last, potentially active segment from the check + if (!segmentsToCheck.isEmpty()) + segmentsToCheck.remove(segmentsToCheck.size() - 1); + + for (CommitLogSegment segment : segmentsToCheck) + { + Assert.assertFalse("An unused segment is left after drain: " + segment.getName() + + ", dirty tables: " + segment.dirtyString() + + ", total segments: " + CommitLog.instance.segmentManager.getActiveSegments().size() + + ", commit log files: " + Arrays.toString(new File(DatabaseDescriptor.getCommitLogLocation()).listFiles()), + segment.isUnused()); + } } } From 009146959afccc67adbae24e59a7ba38930fea87 Mon Sep 17 00:00:00 2001 From: maoling <maoling@apache.org> Date: Wed, 5 Jun 2024 23:50:00 +0800 Subject: [PATCH 215/225] Fix nodetool gcstats output, support human-readable units and more output formats nodetool gcstats output was broken as such / not aligned. This command was using a completely custom way of displaying the statistics which was fixed. This patch also supports three output modes: table, yaml, json under -F flag. There is also a possibility to output the figures in human-friendly format via -H flag. The patch also adds more JVM statistics related to direct memory besides already existing "allocated direct memory" being: - max direct memory - reserved direct memory patch by Ling Mao; reviewed by Brad Schoening, Stefan Miklosovic for CASSANDRA-19022 Co-authored-by: Stefan Miklosovic <smiklosovic@apache.org> --- CHANGES.txt | 1 + .../apache/cassandra/service/GCInspector.java | 70 ++++++-- .../cassandra/tools/nodetool/GcStats.java | 23 +-- .../tools/nodetool/stats/GcStatsHolder.java | 55 +++++-- .../tools/nodetool/stats/GcStatsPrinter.java | 48 ++++-- .../cassandra/tools/nodetool/GcStatsTest.java | 154 ++++++++++-------- 6 files changed, 243 insertions(+), 108 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 0449ba45d5e0..94399222f388 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Fix nodetool gcstats output, support human-readable units and more output formats (CASSANDRA-19022) * Various gossip to TCM upgrade fixes (CASSANDRA-20483) * Add nodetool command to abort failed nodetool cms initialize (CASSANDRA-20482) * Repair Paxos for the distributed metadata log when CMS membership changes (CASSANDRA-20467) diff --git a/src/java/org/apache/cassandra/service/GCInspector.java b/src/java/org/apache/cassandra/service/GCInspector.java index 16b6665c6917..4bdc3a05bade 100644 --- a/src/java/org/apache/cassandra/service/GCInspector.java +++ b/src/java/org/apache/cassandra/service/GCInspector.java @@ -26,6 +26,7 @@ import java.util.HashMap; import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import javax.management.MBeanServer; @@ -59,24 +60,32 @@ public class GCInspector implements NotificationListener, GCInspectorMXBean * bytes of direct memory requires via ByteBuffer.allocateDirect that have not been GCed. */ final static Field BITS_TOTAL_CAPACITY; + // The hard limit for direct memory allocation, typically controlled by the -XX:MaxDirectMemorySize JVM option. + final static Field BITS_MAX; + // This represents the amount of direct memory that has been reserved for future use but not necessarily allocated yet. + final static Field BITS_RESERVED; - static { - Field temp = null; + Field totalTempField = null; + Field maxTempField = null; + Field reservedTempField = null; try { Class<?> bitsClass = Class.forName("java.nio.Bits"); - Field f = bitsClass.getDeclaredField("TOTAL_CAPACITY"); - f.setAccessible(true); - temp = f; + totalTempField = getField(bitsClass, "TOTAL_CAPACITY"); + // Returns the maximum amount of allocatable direct buffer memory. + maxTempField = getField(bitsClass, "MAX_MEMORY"); + reservedTempField = getField(bitsClass, "RESERVED_MEMORY"); } catch (Throwable t) { logger.debug("Error accessing field of java.nio.Bits", t); //Don't care, will just return the dummy value -1 if we can't get at the field in this JVM } - BITS_TOTAL_CAPACITY = temp; + BITS_TOTAL_CAPACITY = totalTempField; + BITS_MAX = maxTempField; + BITS_RESERVED = reservedTempField; } static final class State @@ -303,28 +312,65 @@ public State getTotalSinceLastCheck() public double[] getAndResetStats() { State state = getTotalSinceLastCheck(); - double[] r = new double[7]; + double[] r = new double[9]; r[0] = TimeUnit.NANOSECONDS.toMillis(nanoTime() - state.startNanos); r[1] = state.maxRealTimeElapsed; r[2] = state.totalRealTimeElapsed; r[3] = state.sumSquaresRealTimeElapsed; r[4] = state.totalBytesReclaimed; r[5] = state.count; - r[6] = getAllocatedDirectMemory(); + r[6] = getTotalDirectMemory(); + r[7] = getMaxDirectMemory(); + r[8] = getReservedDirectMemory(); return r; } - private static long getAllocatedDirectMemory() + private static long getTotalDirectMemory() { - if (BITS_TOTAL_CAPACITY == null) return -1; + return getFieldValue(BITS_TOTAL_CAPACITY, true); + } + + private static long getMaxDirectMemory() + { + return getFieldValue(BITS_MAX, false); + } + + private static long getReservedDirectMemory() + { + return getFieldValue(BITS_RESERVED, true); + } + + private static Field getField(Class<?> clazz, String fieldName) + { + try + { + Field field = clazz.getDeclaredField(fieldName); + field.setAccessible(true); + return field; + } + catch (Throwable t) + { + logger.trace("Error accessing field {} of {}", fieldName, clazz.getName(), t); + // Return null to indicate failure + return null; + } + } + + /** + * From the implementation of java.nio.Bits, we can infer that TOTAL_CAPACITY/RESERVED_MEMORY is AtomicLong + * and MAX_MEMORY is long. This method works well with JDK 11/17 + * */ + private static long getFieldValue(Field field, boolean isAtomicLong) + { + if (field == null) return -1; try { - return BITS_TOTAL_CAPACITY.getLong(null); + return isAtomicLong ? ((AtomicLong) field.get(null)).get() : field.getLong(null); } catch (Throwable t) { - logger.trace("Error accessing field of java.nio.Bits", t); + logger.trace("Error accessing field value of {}", field.getName(), t); //Don't care how or why we failed to get the value in this JVM. Return -1 to indicate failure return -1; } diff --git a/src/java/org/apache/cassandra/tools/nodetool/GcStats.java b/src/java/org/apache/cassandra/tools/nodetool/GcStats.java index f65095bcaf68..2378226f759b 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/GcStats.java +++ b/src/java/org/apache/cassandra/tools/nodetool/GcStats.java @@ -17,31 +17,32 @@ */ package org.apache.cassandra.tools.nodetool; +import io.airlift.airline.Command; +import io.airlift.airline.Option; import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.tools.NodeTool.NodeToolCmd; import org.apache.cassandra.tools.nodetool.stats.GcStatsHolder; import org.apache.cassandra.tools.nodetool.stats.GcStatsPrinter; -import org.apache.cassandra.tools.nodetool.stats.StatsPrinter; - -import io.airlift.airline.Command; -import io.airlift.airline.Option; @Command(name = "gcstats", description = "Print GC Statistics") public class GcStats extends NodeToolCmd { @Option(title = "format", - name = {"-F", "--format"}, - description = "Output format (json, yaml)") + name = { "-F", "--format" }, + description = "Output format (json, yaml, table)") private String outputFormat = ""; + @Option(title = "human_readable", + name = { "-H", "--human-readable" }, + description = "Display gcstats with human-readable units") + private boolean humanReadable = false; + @Override public void execute(NodeProbe probe) { - if (!outputFormat.isEmpty() && !"json".equals(outputFormat) && !"yaml".equals(outputFormat)) - throw new IllegalArgumentException("arguments for -F are json, yaml only."); + if (!outputFormat.isEmpty() && !"json".equals(outputFormat) && !"yaml".equals(outputFormat) && !"table".equals(outputFormat)) + throw new IllegalArgumentException("arguments for -F are json, yaml, table only."); - GcStatsHolder data = new GcStatsHolder(probe); - StatsPrinter<GcStatsHolder> printer = GcStatsPrinter.from(outputFormat); - printer.print(data, probe.output().out); + GcStatsPrinter.from(outputFormat).print(new GcStatsHolder(probe, humanReadable), probe.output().out); } } \ No newline at end of file diff --git a/src/java/org/apache/cassandra/tools/nodetool/stats/GcStatsHolder.java b/src/java/org/apache/cassandra/tools/nodetool/stats/GcStatsHolder.java index 7533de0e33a6..e6ecb1c917c4 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/stats/GcStatsHolder.java +++ b/src/java/org/apache/cassandra/tools/nodetool/stats/GcStatsHolder.java @@ -18,21 +18,50 @@ package org.apache.cassandra.tools.nodetool.stats; +import java.util.Collections; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.Map; import org.apache.cassandra.tools.NodeProbe; +import static org.apache.cassandra.io.util.FileUtils.stringifyFileSize; + /** * Holds and converts GC statistics to a map structure. */ public class GcStatsHolder implements StatsHolder { - public final NodeProbe probe; + public static final String INTERVAL = "interval_ms"; + public static final String MAX_GC = "max_gc_elapsed_ms"; + public static final String TOTAL_GC = "total_gc_elapsed_ms"; + public static final String STDEV_GC = "stdev_gc_elapsed_ms"; + public static final String RECLAIMED_GC = "gc_reclaimed_mb"; + public static final String GC_COUNT = "gc_count"; + public static final String ALLOCATED_DIRECT_MEMORY = "allocated_direct_memory_bytes"; + public static final String MAX_DIRECT_MEMORY = "max_direct_memory_bytes"; + public static final String RESERVED_DIRECT_MEMORY = "reserved_direct_memory_bytes"; + + public static final Map<String, String> columnDescriptionMap = Collections.unmodifiableMap(new LinkedHashMap<>() + {{ + put(INTERVAL, "Interval (ms)"); + put(MAX_GC, "Max GC Elapsed (ms)"); + put(TOTAL_GC, "Total GC Elapsed (ms)"); + put(STDEV_GC, "Stdev GC Elapsed (ms)"); + put(RECLAIMED_GC, "GC Reclaimed Bytes"); + put(GC_COUNT, "GC Count"); + put(ALLOCATED_DIRECT_MEMORY, "Allocated Direct Memory Bytes"); + put(MAX_DIRECT_MEMORY, "Max Direct Memory Bytes"); + put(RESERVED_DIRECT_MEMORY, "Reserved Direct Memory Bytes"); + }}); - public GcStatsHolder(NodeProbe probe) + private final NodeProbe probe; + private final boolean humanReadable; + + public GcStatsHolder(NodeProbe probe, boolean humanReadable) { this.probe = probe; + this.humanReadable = humanReadable; } /** @@ -43,19 +72,25 @@ public GcStatsHolder(NodeProbe probe) @Override public Map<String, Object> convert2Map() { - HashMap<String, Object> result = new HashMap<>(); + HashMap<String, Object> result = new LinkedHashMap<>(); double[] stats = probe.getAndResetGCStats(); double mean = stats[2] / stats[5]; double stdev = Math.sqrt((stats[3] / stats[5]) - (mean * mean)); - result.put("interval_ms", stats[0]); - result.put("max_gc_elapsed_ms", stats[1]); - result.put("total_gc_elapsed_ms", stats[2]); - result.put("stdev_gc_elapsed_ms", stdev); - result.put("gc_reclaimed_mb", stats[4]); - result.put("collections", stats[5]); - result.put("direct_memory_bytes", (long) stats[6]); + long totalDirect = (long) stats[6]; + long maxDirect = (long) stats[7]; + long reservedDirect = (long) stats[8]; + + result.put(INTERVAL, String.format("%.0f", stats[0])); + result.put(MAX_GC, String.format("%.0f", stats[1])); + result.put(TOTAL_GC, String.format("%.0f", stats[2])); + result.put(STDEV_GC, String.format("%.0f", stdev)); + result.put(RECLAIMED_GC, stringifyFileSize((long) stats[4], humanReadable)); + result.put(GC_COUNT, String.valueOf((long) stats[5])); + result.put(ALLOCATED_DIRECT_MEMORY, totalDirect == -1 ? Double.NaN : stringifyFileSize((long) stats[6], humanReadable)); + result.put(MAX_DIRECT_MEMORY, maxDirect == -1 ? Double.NaN : stringifyFileSize((long) stats[7], humanReadable)); + result.put(RESERVED_DIRECT_MEMORY, reservedDirect == -1 ? Double.NaN : stringifyFileSize((long) stats[8], humanReadable)); return result; } diff --git a/src/java/org/apache/cassandra/tools/nodetool/stats/GcStatsPrinter.java b/src/java/org/apache/cassandra/tools/nodetool/stats/GcStatsPrinter.java index 87b639184dba..d84b5ffff689 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/stats/GcStatsPrinter.java +++ b/src/java/org/apache/cassandra/tools/nodetool/stats/GcStatsPrinter.java @@ -19,8 +19,12 @@ package org.apache.cassandra.tools.nodetool.stats; import java.io.PrintStream; +import java.util.ArrayList; +import java.util.List; import java.util.Map; +import org.apache.cassandra.tools.nodetool.formatter.TableBuilder; + /** * Printer for GC statistics. */ @@ -40,32 +44,54 @@ public static StatsPrinter<GcStatsHolder> from(String format) return new StatsPrinter.JsonPrinter<>(); case "yaml": return new StatsPrinter.YamlPrinter<>(); + case "table": + return new TablePrinter(); default: - return new DefaultPrinter(); + return new LegacyPrinter(); } } - /** - * Default printer for GC statistics. - */ - public static class DefaultPrinter implements StatsPrinter<GcStatsHolder> + public static class TablePrinter implements StatsPrinter<GcStatsHolder> { /** * Prints GC statistics in a human-readable table format. * * @param data The GC statistics data holder. - * @param out The output stream to print to. + * @param out The output stream to print to. */ @Override public void print(GcStatsHolder data, PrintStream out) { Map<String, Object> stats = data.convert2Map(); + TableBuilder tableBuilder = new TableBuilder(); + + for (Map.Entry<String, Object> entry : stats.entrySet()) + tableBuilder.add(GcStatsHolder.columnDescriptionMap.get(entry.getKey()), entry.getValue().toString()); + + tableBuilder.printTo(out); + } + } + + /** + * Default printer for GC statistics. + */ + public static class LegacyPrinter implements StatsPrinter<GcStatsHolder> + { + @Override + public void print(GcStatsHolder data, PrintStream out) + { + Map<String, Object> stats = data.convert2Map(); + TableBuilder tableBuilder = new TableBuilder(); + + tableBuilder.add(new ArrayList<>(GcStatsHolder.columnDescriptionMap.values())); + + List<String> values = new ArrayList<>(); + for (String key : GcStatsHolder.columnDescriptionMap.keySet()) + values.add(stats.get(key).toString()); + + tableBuilder.add(values); - out.printf("%20s%20s%20s%20s%20s%20s%25s%n", "Interval (ms)", "Max GC Elapsed (ms)", "Total GC Elapsed (ms)", - "Stdev GC Elapsed (ms)", "GC Reclaimed (MB)", "Collections", "Direct Memory Bytes"); - out.printf("%20.0f%20.0f%20.0f%20.0f%20.0f%20.0f%25d%n", stats.get("interval_ms"), stats.get("max_gc_elapsed_ms"), - stats.get("total_gc_elapsed_ms"), stats.get("stdev_gc_elapsed_ms"), stats.get("gc_reclaimed_mb"), - stats.get("collections"), (long) stats.get("direct_memory_bytes")); + tableBuilder.printTo(out); } } } \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/tools/nodetool/GcStatsTest.java b/test/unit/org/apache/cassandra/tools/nodetool/GcStatsTest.java index 107b37a87342..56ad5d26118b 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/GcStatsTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/GcStatsTest.java @@ -18,16 +18,23 @@ package org.apache.cassandra.tools.nodetool; -import java.util.Arrays; +import org.apache.commons.lang3.StringUtils; +import org.junit.BeforeClass; +import org.junit.Test; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.service.GCInspector; import org.apache.cassandra.tools.ToolRunner; +import org.apache.cassandra.tools.ToolRunner.ToolResult; +import org.apache.cassandra.tools.nodetool.stats.GcStatsHolder; import org.apache.cassandra.utils.JsonUtils; -import org.junit.BeforeClass; -import org.junit.Test; import org.yaml.snakeyaml.Yaml; +import static java.lang.Double.parseDouble; +import static java.util.Arrays.asList; +import static org.apache.cassandra.tools.nodetool.stats.GcStatsHolder.MAX_DIRECT_MEMORY; +import static org.apache.cassandra.tools.nodetool.stats.GcStatsHolder.RESERVED_DIRECT_MEMORY; +import static org.apache.cassandra.tools.nodetool.stats.GcStatsHolder.ALLOCATED_DIRECT_MEMORY; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatCode; @@ -46,41 +53,44 @@ public static void setUp() throws Exception public void testMaybeChangeDocs() { // If you added, modified options or help, please update docs if necessary - ToolRunner.ToolResult tool = ToolRunner.invokeNodetool("help", "gcstats"); + ToolResult tool = ToolRunner.invokeNodetool("help", "gcstats"); tool.assertOnCleanExit(); - String help = "NAME\n" + - " nodetool gcstats - Print GC Statistics\n" + - "\n" + - "SYNOPSIS\n" + - " nodetool [(-h <host> | --host <host>)] [(-p <port> | --port <port>)]\n" + - " [(-pp | --print-port)] [(-pw <password> | --password <password>)]\n" + - " [(-pwf <passwordFilePath> | --password-file <passwordFilePath>)]\n" + - " [(-u <username> | --username <username>)] gcstats\n" + - " [(-F <format> | --format <format>)]\n" + - "\n" + - "OPTIONS\n" + - " -F <format>, --format <format>\n" + - " Output format (json, yaml)\n" + - "\n" + - " -h <host>, --host <host>\n" + - " Node hostname or ip address\n" + - "\n" + - " -p <port>, --port <port>\n" + - " Remote jmx agent port number\n" + - "\n" + - " -pp, --print-port\n" + - " Operate in 4.0 mode with hosts disambiguated by port number\n" + - "\n" + - " -pw <password>, --password <password>\n" + - " Remote jmx agent password\n" + - "\n" + - " -pwf <passwordFilePath>, --password-file <passwordFilePath>\n" + - " Path to the JMX password file\n" + - "\n" + - " -u <username>, --username <username>\n" + - " Remote jmx agent username\n" + - "\n"; + String help = "NAME\n" + + " nodetool gcstats - Print GC Statistics\n" + + "\n" + + "SYNOPSIS\n" + + " nodetool [(-h <host> | --host <host>)] [(-p <port> | --port <port>)]\n" + + " [(-pp | --print-port)] [(-pw <password> | --password <password>)]\n" + + " [(-pwf <passwordFilePath> | --password-file <passwordFilePath>)]\n" + + " [(-u <username> | --username <username>)] gcstats\n" + + " [(-F <format> | --format <format>)] [(-H | --human-readable)]\n" + + "\n" + + "OPTIONS\n" + + " -F <format>, --format <format>\n" + + " Output format (json, yaml, table)\n" + + "\n" + + " -h <host>, --host <host>\n" + + " Node hostname or ip address\n" + + "\n" + + " -H, --human-readable\n" + + " Display gcstats with human-readable units\n" + + "\n" + + " -p <port>, --port <port>\n" + + " Remote jmx agent port number\n" + + "\n" + + " -pp, --print-port\n" + + " Operate in 4.0 mode with hosts disambiguated by port number\n" + + "\n" + + " -pw <password>, --password <password>\n" + + " Remote jmx agent password\n" + + "\n" + + " -pwf <passwordFilePath>, --password-file <passwordFilePath>\n" + + " Path to the JMX password file\n" + + "\n" + + " -u <username>, --username <username>\n" + + " Remote jmx agent username\n" + + "\n"; assertThat(tool.getStdout().trim()).isEqualTo(help.trim()); } @@ -88,59 +98,75 @@ public void testMaybeChangeDocs() @Test public void testDefaultGcStatsOutput() { - ToolRunner.ToolResult tool = ToolRunner.invokeNodetool("gcstats"); + ToolResult tool = ToolRunner.invokeNodetool("gcstats"); tool.assertOnCleanExit(); String output = tool.getStdout(); - assertThat(output).contains("Interval (ms)"); - assertThat(output).contains("Max GC Elapsed (ms)"); - assertThat(output).contains("Total GC Elapsed (ms)"); - assertThat(output).contains("GC Reclaimed (MB)"); - assertThat(output).contains("Collections"); - assertThat(output).contains("Direct Memory Bytes"); + for (String value : GcStatsHolder.columnDescriptionMap.values()) + assertThat(output).contains(value); } @Test public void testJsonGcStatsOutput() { - Arrays.asList("-F", "--format").forEach(arg -> { - ToolRunner.ToolResult tool = ToolRunner.invokeNodetool("gcstats", arg, "json"); + asList("-F", "--format").forEach(arg -> { + ToolResult tool = ToolRunner.invokeNodetool("gcstats", arg, "json"); tool.assertOnCleanExit(); String json = tool.getStdout(); assertThatCode(() -> JsonUtils.JSON_OBJECT_MAPPER.readTree(json)).doesNotThrowAnyException(); - assertThat(json).containsPattern("\"interval_ms\""); - assertThat(json).containsPattern("\"stdev_gc_elapsed_ms\""); - assertThat(json).containsPattern("\"collections\""); - assertThat(json).containsPattern("\"max_gc_elapsed_ms\""); - assertThat(json).containsPattern("\"gc_reclaimed_mb\""); - assertThat(json).containsPattern("\"total_gc_elapsed_ms\""); - assertThat(json).containsPattern("\"direct_memory_bytes\""); + + for (String key : GcStatsHolder.columnDescriptionMap.keySet()) + assertThat(json).contains(key); }); } @Test public void testYamlGcStatsOutput() { - Arrays.asList("-F", "--format").forEach(arg -> { - ToolRunner.ToolResult tool = ToolRunner.invokeNodetool("gcstats", arg, "yaml"); + asList("-F", "--format").forEach(arg -> { + ToolResult tool = ToolRunner.invokeNodetool("gcstats", arg, "yaml"); tool.assertOnCleanExit(); String yamlOutput = tool.getStdout(); Yaml yaml = new Yaml(); assertThatCode(() -> yaml.load(yamlOutput)).doesNotThrowAnyException(); - assertThat(yamlOutput).containsPattern("interval_ms:"); - assertThat(yamlOutput).containsPattern("stdev_gc_elapsed_ms:"); - assertThat(yamlOutput).containsPattern("collections:"); - assertThat(yamlOutput).containsPattern("max_gc_elapsed_ms:"); - assertThat(yamlOutput).containsPattern("gc_reclaimed_mb:"); - assertThat(yamlOutput).containsPattern("total_gc_elapsed_ms:"); - assertThat(yamlOutput).containsPattern("direct_memory_bytes:"); + + for (String key : GcStatsHolder.columnDescriptionMap.keySet()) + assertThat(yamlOutput).containsPattern(key); }); } @Test - public void testInvalidFormatOption() throws Exception + public void testInvalidFormatOption() { - ToolRunner.ToolResult tool = ToolRunner.invokeNodetool("gcstats", "-F", "invalid_format"); + ToolResult tool = ToolRunner.invokeNodetool("gcstats", "-F", "invalid_format"); assertThat(tool.getExitCode()).isEqualTo(1); - assertThat(tool.getStdout()).contains("arguments for -F are json, yaml only."); + assertThat(tool.getStdout()).contains("arguments for -F are json, yaml, table only."); + } + + @Test + public void testWithoutNoOption() + { + ToolResult tool = ToolRunner.invokeNodetool("gcstats"); + tool.assertOnCleanExit(); + + for (String value : GcStatsHolder.columnDescriptionMap.values()) + assertThat(tool.getStdout()).contains(value); + } + + @Test + public void testWithHumanReadableOption() + { + ToolResult tool = ToolRunner.invokeNodetool("gcstats", "--human-readable", "-F", "table"); + tool.assertOnCleanExit(); + String gcStatsOutput = tool.getStdout(); + + for (String value : GcStatsHolder.columnDescriptionMap.values()) + assertThat(tool.getStdout()).contains(value); + + String total = StringUtils.substringBetween(gcStatsOutput, GcStatsHolder.columnDescriptionMap.get(ALLOCATED_DIRECT_MEMORY), "\n").trim(); + assertThat(parseDouble(total.split(" ")[0])).isGreaterThan(0); + String max = StringUtils.substringBetween(gcStatsOutput, GcStatsHolder.columnDescriptionMap.get(MAX_DIRECT_MEMORY), "\n").trim(); + assertThat(parseDouble(max.split(" ")[0])).isGreaterThan(0); + String reserved = StringUtils.substringBetween(gcStatsOutput, GcStatsHolder.columnDescriptionMap.get(RESERVED_DIRECT_MEMORY), "\n").trim(); + assertThat(parseDouble(reserved.split(" ")[0])).isGreaterThan(0); } } \ No newline at end of file From 9843b3d40e7aa7a305a078ae3bb21988d87b7a5e Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Tue, 18 Mar 2025 13:23:29 +0100 Subject: [PATCH 216/225] Do not fail to start a node with materialized views after they are turned off in config patch by Stefan Miklosovic; reviewed by Maxwell Guo for CASSANDRA-20452 --- CHANGES.txt | 1 + .../schema/CreateViewStatement.java | 7 ++- .../distributed/MaterializedViewTest.java | 49 +++++++++++++++++++ 3 files changed, 53 insertions(+), 4 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/MaterializedViewTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 94399222f388..a7dcac723c12 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Do not fail to start a node with materialized views after they are turned off in config (CASSANDRA-20452) * Fix nodetool gcstats output, support human-readable units and more output formats (CASSANDRA-19022) * Various gossip to TCM upgrade fixes (CASSANDRA-20483) * Add nodetool command to abort failed nodetool cms initialize (CASSANDRA-20482) diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateViewStatement.java index f6c8135bb160..40cbee967b60 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateViewStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateViewStatement.java @@ -105,6 +105,9 @@ public CreateViewStatement(String keyspaceName, @Override public void validate(ClientState state) { + if (!DatabaseDescriptor.getMaterializedViewsEnabled()) + throw ire("Materialized views are disabled. Enable in cassandra.yaml to use."); + super.validate(state); // save the query state to use it for guardrails validation in #apply @@ -114,13 +117,9 @@ public void validate(ClientState state) @Override public Keyspaces apply(ClusterMetadata metadata) { - if (!DatabaseDescriptor.getMaterializedViewsEnabled()) - throw ire("Materialized views are disabled. Enable in cassandra.yaml to use."); - /* * Basic dependency validations */ - Keyspaces schema = metadata.schema.getKeyspaces(); KeyspaceMetadata keyspace = schema.getNullable(keyspaceName); if (null == keyspace) diff --git a/test/distributed/org/apache/cassandra/distributed/MaterializedViewTest.java b/test/distributed/org/apache/cassandra/distributed/MaterializedViewTest.java new file mode 100644 index 000000000000..4f42977339fe --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/MaterializedViewTest.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed; + +import org.junit.Test; + +import org.apache.cassandra.distributed.test.TestBaseImpl; + +import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL; + +public class MaterializedViewTest extends TestBaseImpl +{ + @Test + public void testDisablingMaterializedViewsDontFailNodeToStart() throws Throwable + { + try (Cluster cluster = init(Cluster.build(1) + .withConfig(c -> c.set("materialized_views_enabled", true)) + .start())) + { + cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (id uuid, col1 text, col2 text, primary key (id));"); + cluster.schemaChange("CREATE MATERIALIZED VIEW " + KEYSPACE + ".a_view AS SELECT id, col1, col2 " + + "FROM tbl WHERE col2 IS NOT NULL AND id IS NOT NULL PRIMARY KEY (col2, id) " + + "WITH CLUSTERING ORDER BY (id ASC);"); + cluster.coordinator(1).execute(withKeyspace("select * from %s.a_view"), ALL); + + cluster.get(1).shutdown().get(); + cluster.get(1).config().set("materialized_views_enabled", false); + cluster.get(1).startup(); + + cluster.coordinator(1).execute(withKeyspace("select * from %s.a_view"), ALL); + } + } +} From e41a71b34b45bcc40573e0d0782336aac5ea55a5 Mon Sep 17 00:00:00 2001 From: Mick Semb Wever <mck@apache.org> Date: Fri, 28 Mar 2025 10:41:37 +0100 Subject: [PATCH 217/225] Fix comment in BigFormat wrt when nb was introduced patch by Mick Semb Wever; reviewed by Dmitry Konstantinov for CASSANDRA-20485 --- .../org/apache/cassandra/io/sstable/format/big/BigFormat.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java index ff0d7916672c..2c0d02f65997 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java +++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java @@ -123,7 +123,7 @@ static class BigVersion extends Version // me (3.0.25, 3.11.11): added hostId of the node from which the sstable originated // na (4.0-rc1): uncompressed chunks, pending repair session, isTransient, checksummed sstable metadata file, new Bloomfilter format - // nb (4.0.0): originating host id + // nb (4.0-rc2): originating host id // // NOTE: when adding a new version, please add that to LegacySSTableTest, too. From ab51b794735bd5731357d6fd4cb92cf0059a7ad1 Mon Sep 17 00:00:00 2001 From: Roxana Neophytou <roxana.neophytou@datastax.com> Date: Wed, 27 Nov 2024 12:47:55 -0500 Subject: [PATCH 218/225] Add LegacySSTableTest data for past sstable formats Also add test method testVerifyOldTupleSSTables for frozen tuples and dropping of them. The test data also adds different versions of me format sstables from C* version 3.0.25 and 3.11.11 as they do (unfortunately) differ. Test sstable files have had their sequence generation ids changed to represent the server version used to create them, while this shouldn't be necessary it is for debug purposes when sstable formats have changed between versions. patch by Roxana Neophytou, Mick Semb Wever; reviewed by Dmitry Konstantinov for CASSANDRA-20485 Co-authored-by: mck <mck@apache.org> --- ...nInfo.db => da-500-bti-CompressionInfo.db} | Bin .../{da-1-bti-Data.db => da-500-bti-Data.db} | Bin ...i-Digest.crc32 => da-500-bti-Digest.crc32} | 0 ...a-1-bti-Filter.db => da-500-bti-Filter.db} | Bin ...Partitions.db => da-500-bti-Partitions.db} | Bin .../{da-1-bti-Rows.db => da-500-bti-Rows.db} | Bin ...Statistics.db => da-500-bti-Statistics.db} | Bin .../{da-1-bti-TOC.txt => da-500-bti-TOC.txt} | 0 ...nInfo.db => da-500-bti-CompressionInfo.db} | Bin .../{da-1-bti-Data.db => da-500-bti-Data.db} | Bin ...i-Digest.crc32 => da-500-bti-Digest.crc32} | 0 ...a-1-bti-Filter.db => da-500-bti-Filter.db} | Bin ...Partitions.db => da-500-bti-Partitions.db} | Bin .../{da-1-bti-Rows.db => da-500-bti-Rows.db} | Bin ...Statistics.db => da-500-bti-Statistics.db} | Bin .../{da-1-bti-TOC.txt => da-500-bti-TOC.txt} | 0 ...nInfo.db => da-500-bti-CompressionInfo.db} | Bin .../{da-1-bti-Data.db => da-500-bti-Data.db} | Bin ...i-Digest.crc32 => da-500-bti-Digest.crc32} | 0 ...a-1-bti-Filter.db => da-500-bti-Filter.db} | Bin ...Partitions.db => da-500-bti-Partitions.db} | Bin .../{da-1-bti-Rows.db => da-500-bti-Rows.db} | 0 ...Statistics.db => da-500-bti-Statistics.db} | Bin .../{da-1-bti-TOC.txt => da-500-bti-TOC.txt} | 0 ...nInfo.db => da-500-bti-CompressionInfo.db} | Bin .../{da-1-bti-Data.db => da-500-bti-Data.db} | Bin ...i-Digest.crc32 => da-500-bti-Digest.crc32} | 0 ...a-1-bti-Filter.db => da-500-bti-Filter.db} | Bin ...Partitions.db => da-500-bti-Partitions.db} | Bin .../{da-1-bti-Rows.db => da-500-bti-Rows.db} | 0 ...Statistics.db => da-500-bti-Statistics.db} | Bin .../{da-1-bti-TOC.txt => da-500-bti-TOC.txt} | 0 .../da-500-bti-CompressionInfo.db | Bin 0 -> 47 bytes .../legacy_da_tuple/da-500-bti-Data.db | Bin 0 -> 300 bytes .../legacy_da_tuple/da-500-bti-Digest.crc32 | 1 + .../legacy_da_tuple/da-500-bti-Filter.db} | Bin .../legacy_da_tuple/da-500-bti-Partitions.db | Bin 0 -> 62 bytes .../legacy_da_tuple/da-500-bti-Rows.db | 0 .../legacy_da_tuple/da-500-bti-Statistics.db | Bin 0 -> 5785 bytes .../legacy_da_tuple/da-500-bti-TOC.txt | 8 + ...nInfo.db => ma-306-big-CompressionInfo.db} | Bin .../{ma-1-big-Data.db => ma-306-big-Data.db} | Bin ...g-Digest.crc32 => ma-306-big-Digest.crc32} | 0 ...a-1-big-Filter.db => ma-306-big-Filter.db} | Bin ...{ma-1-big-Index.db => ma-306-big-Index.db} | Bin ...Statistics.db => ma-306-big-Statistics.db} | Bin ...1-big-Summary.db => ma-306-big-Summary.db} | Bin .../{ma-1-big-TOC.txt => ma-306-big-TOC.txt} | 0 ...nInfo.db => ma-306-big-CompressionInfo.db} | Bin .../{ma-1-big-Data.db => ma-306-big-Data.db} | Bin ...g-Digest.crc32 => ma-306-big-Digest.crc32} | 0 ...a-1-big-Filter.db => ma-306-big-Filter.db} | Bin ...{ma-1-big-Index.db => ma-306-big-Index.db} | Bin ...Statistics.db => ma-306-big-Statistics.db} | Bin ...1-big-Summary.db => ma-306-big-Summary.db} | Bin .../{ma-1-big-TOC.txt => ma-306-big-TOC.txt} | 0 ...nInfo.db => ma-306-big-CompressionInfo.db} | Bin .../{ma-1-big-Data.db => ma-306-big-Data.db} | Bin ...g-Digest.crc32 => ma-306-big-Digest.crc32} | 0 ...a-1-big-Filter.db => ma-306-big-Filter.db} | Bin ...{ma-1-big-Index.db => ma-306-big-Index.db} | Bin ...Statistics.db => ma-306-big-Statistics.db} | Bin ...1-big-Summary.db => ma-306-big-Summary.db} | Bin .../{ma-1-big-TOC.txt => ma-306-big-TOC.txt} | 0 ...nInfo.db => ma-306-big-CompressionInfo.db} | Bin .../{ma-1-big-Data.db => ma-306-big-Data.db} | Bin ...g-Digest.crc32 => ma-306-big-Digest.crc32} | 0 ...a-1-big-Filter.db => ma-306-big-Filter.db} | Bin ...{ma-1-big-Index.db => ma-306-big-Index.db} | Bin ...Statistics.db => ma-306-big-Statistics.db} | Bin ...1-big-Summary.db => ma-306-big-Summary.db} | Bin .../{ma-1-big-TOC.txt => ma-306-big-TOC.txt} | 0 ...nInfo.db => ma-306-big-CompressionInfo.db} | Bin .../{ma-1-big-Data.db => ma-306-big-Data.db} | Bin ...g-Digest.crc32 => ma-306-big-Digest.crc32} | 0 ...a-1-big-Filter.db => ma-306-big-Filter.db} | Bin ...{ma-1-big-Index.db => ma-306-big-Index.db} | Bin ...Statistics.db => ma-306-big-Statistics.db} | Bin ...1-big-Summary.db => ma-306-big-Summary.db} | Bin .../{ma-1-big-TOC.txt => ma-306-big-TOC.txt} | 0 ...nInfo.db => ma-306-big-CompressionInfo.db} | Bin .../{ma-1-big-Data.db => ma-306-big-Data.db} | Bin ...g-Digest.crc32 => ma-306-big-Digest.crc32} | 0 ...a-1-big-Filter.db => ma-306-big-Filter.db} | Bin ...{ma-1-big-Index.db => ma-306-big-Index.db} | Bin ...Statistics.db => ma-306-big-Statistics.db} | Bin ...1-big-Summary.db => ma-306-big-Summary.db} | Bin .../{ma-1-big-TOC.txt => ma-306-big-TOC.txt} | 0 ...nInfo.db => ma-306-big-CompressionInfo.db} | Bin .../{ma-1-big-Data.db => ma-306-big-Data.db} | Bin ...g-Digest.crc32 => ma-306-big-Digest.crc32} | 0 ...a-1-big-Filter.db => ma-306-big-Filter.db} | Bin ...{ma-1-big-Index.db => ma-306-big-Index.db} | Bin ...Statistics.db => ma-306-big-Statistics.db} | Bin ...1-big-Summary.db => ma-306-big-Summary.db} | Bin .../{ma-1-big-TOC.txt => ma-306-big-TOC.txt} | 0 ...nInfo.db => ma-306-big-CompressionInfo.db} | Bin .../{ma-1-big-Data.db => ma-306-big-Data.db} | Bin ...g-Digest.crc32 => ma-306-big-Digest.crc32} | 0 ...a-1-big-Filter.db => ma-306-big-Filter.db} | Bin ...{ma-1-big-Index.db => ma-306-big-Index.db} | Bin ...Statistics.db => ma-306-big-Statistics.db} | Bin ...1-big-Summary.db => ma-306-big-Summary.db} | Bin .../{ma-1-big-TOC.txt => ma-306-big-TOC.txt} | 0 .../ma-306-big-CompressionInfo.db | Bin 0 -> 43 bytes .../legacy_ma_tuple/ma-306-big-Data.db | Bin 0 -> 121 bytes .../legacy_ma_tuple/ma-306-big-Digest.crc32 | 1 + .../legacy_ma_tuple/ma-306-big-Filter.db} | Bin .../legacy_ma_tuple/ma-306-big-Index.db | Bin 0 -> 28 bytes .../legacy_ma_tuple/ma-306-big-Statistics.db | Bin 0 -> 4767 bytes .../legacy_ma_tuple/ma-306-big-Summary.db} | Bin .../legacy_ma_tuple/ma-306-big-TOC.txt} | 8 +- ...nInfo.db => mb-307-big-CompressionInfo.db} | Bin .../{mb-1-big-Data.db => mb-307-big-Data.db} | Bin ...g-Digest.crc32 => mb-307-big-Digest.crc32} | 0 .../mb-307-big-Filter.db} | Bin ...{mb-1-big-Index.db => mb-307-big-Index.db} | Bin ...Statistics.db => mb-307-big-Statistics.db} | Bin .../mb-307-big-Summary.db} | Bin .../{mb-1-big-TOC.txt => mb-307-big-TOC.txt} | 0 ...nInfo.db => mb-307-big-CompressionInfo.db} | Bin .../{mb-1-big-Data.db => mb-307-big-Data.db} | Bin ...g-Digest.crc32 => mb-307-big-Digest.crc32} | 0 .../mb-307-big-Filter.db} | Bin ...{mb-1-big-Index.db => mb-307-big-Index.db} | Bin ...Statistics.db => mb-307-big-Statistics.db} | Bin .../mb-307-big-Summary.db} | Bin .../{mb-1-big-TOC.txt => mb-307-big-TOC.txt} | 0 ...nInfo.db => mb-307-big-CompressionInfo.db} | Bin .../{mb-1-big-Data.db => mb-307-big-Data.db} | Bin ...g-Digest.crc32 => mb-307-big-Digest.crc32} | 0 .../mb-307-big-Filter.db} | Bin ...{mb-1-big-Index.db => mb-307-big-Index.db} | Bin ...Statistics.db => mb-307-big-Statistics.db} | Bin .../mb-307-big-Summary.db} | Bin .../{mb-1-big-TOC.txt => mb-307-big-TOC.txt} | 0 ...nInfo.db => mb-307-big-CompressionInfo.db} | Bin .../{mb-1-big-Data.db => mb-307-big-Data.db} | Bin ...g-Digest.crc32 => mb-307-big-Digest.crc32} | 0 .../mb-307-big-Filter.db} | Bin ...{mb-1-big-Index.db => mb-307-big-Index.db} | Bin ...Statistics.db => mb-307-big-Statistics.db} | Bin .../mb-307-big-Summary.db} | Bin .../{mb-1-big-TOC.txt => mb-307-big-TOC.txt} | 0 ...nInfo.db => mb-307-big-CompressionInfo.db} | Bin .../{mb-1-big-Data.db => mb-307-big-Data.db} | Bin ...g-Digest.crc32 => mb-307-big-Digest.crc32} | 0 .../mb-307-big-Filter.db} | Bin ...{mb-1-big-Index.db => mb-307-big-Index.db} | Bin ...Statistics.db => mb-307-big-Statistics.db} | Bin .../mb-307-big-Summary.db} | Bin .../{mb-1-big-TOC.txt => mb-307-big-TOC.txt} | 0 ...nInfo.db => mb-307-big-CompressionInfo.db} | Bin .../{mb-1-big-Data.db => mb-307-big-Data.db} | Bin ...g-Digest.crc32 => mb-307-big-Digest.crc32} | 0 .../mb-307-big-Filter.db} | Bin ...{mb-1-big-Index.db => mb-307-big-Index.db} | Bin ...Statistics.db => mb-307-big-Statistics.db} | Bin .../mb-307-big-Summary.db} | Bin .../{mb-1-big-TOC.txt => mb-307-big-TOC.txt} | 0 ...nInfo.db => mb-307-big-CompressionInfo.db} | Bin .../{mb-1-big-Data.db => mb-307-big-Data.db} | Bin ...g-Digest.crc32 => mb-307-big-Digest.crc32} | 0 .../mb-307-big-Filter.db} | Bin ...{mb-1-big-Index.db => mb-307-big-Index.db} | Bin ...Statistics.db => mb-307-big-Statistics.db} | Bin .../mb-307-big-Summary.db} | Bin .../{mb-1-big-TOC.txt => mb-307-big-TOC.txt} | 0 ...nInfo.db => mb-307-big-CompressionInfo.db} | Bin .../{mb-1-big-Data.db => mb-307-big-Data.db} | Bin ...g-Digest.crc32 => mb-307-big-Digest.crc32} | 0 .../mb-307-big-Filter.db} | Bin ...{mb-1-big-Index.db => mb-307-big-Index.db} | Bin ...Statistics.db => mb-307-big-Statistics.db} | Bin .../mb-307-big-Summary.db} | Bin .../{mb-1-big-TOC.txt => mb-307-big-TOC.txt} | 0 .../mb-307-big-CompressionInfo.db | Bin 0 -> 43 bytes .../legacy_mb_tuple/mb-307-big-Data.db | Bin 0 -> 276 bytes .../legacy_mb_tuple/mb-307-big-Digest.crc32 | 1 + .../legacy_mb_tuple/mb-307-big-Filter.db} | Bin .../legacy_mb_tuple/mb-307-big-Index.db | Bin 0 -> 29 bytes .../legacy_mb_tuple/mb-307-big-Statistics.db | Bin 0 -> 5275 bytes .../legacy_mb_tuple/mb-307-big-Summary.db} | Bin .../legacy_mb_tuple/mb-307-big-TOC.txt} | 10 +- ...Info.db => mc-3113-big-CompressionInfo.db} | Bin .../{mc-1-big-Data.db => mc-3113-big-Data.db} | Bin ...-Digest.crc32 => mc-3113-big-Digest.crc32} | 0 .../mc-3113-big-Filter.db} | Bin ...mc-1-big-Index.db => mc-3113-big-Index.db} | Bin ...tatistics.db => mc-3113-big-Statistics.db} | Bin .../mc-3113-big-Summary.db} | Bin .../{mc-1-big-TOC.txt => mc-3113-big-TOC.txt} | 0 ...Info.db => mc-3113-big-CompressionInfo.db} | Bin .../{mc-1-big-Data.db => mc-3113-big-Data.db} | Bin ...-Digest.crc32 => mc-3113-big-Digest.crc32} | 0 .../mc-3113-big-Filter.db} | Bin ...mc-1-big-Index.db => mc-3113-big-Index.db} | Bin ...tatistics.db => mc-3113-big-Statistics.db} | Bin .../mc-3113-big-Summary.db} | Bin .../{mc-1-big-TOC.txt => mc-3113-big-TOC.txt} | 0 ...Info.db => mc-3113-big-CompressionInfo.db} | Bin .../{mc-1-big-Data.db => mc-3113-big-Data.db} | Bin ...-Digest.crc32 => mc-3113-big-Digest.crc32} | 0 .../mc-3113-big-Filter.db} | Bin ...mc-1-big-Index.db => mc-3113-big-Index.db} | Bin ...tatistics.db => mc-3113-big-Statistics.db} | Bin .../mc-3113-big-Summary.db} | Bin .../{mc-1-big-TOC.txt => mc-3113-big-TOC.txt} | 0 ...Info.db => mc-3113-big-CompressionInfo.db} | Bin .../{mc-1-big-Data.db => mc-3113-big-Data.db} | Bin ...-Digest.crc32 => mc-3113-big-Digest.crc32} | 0 .../mc-3113-big-Filter.db} | Bin ...mc-1-big-Index.db => mc-3113-big-Index.db} | Bin ...tatistics.db => mc-3113-big-Statistics.db} | Bin .../mc-3113-big-Summary.db} | Bin .../{mc-1-big-TOC.txt => mc-3113-big-TOC.txt} | 0 ...Info.db => mc-3113-big-CompressionInfo.db} | Bin .../{mc-1-big-Data.db => mc-3113-big-Data.db} | Bin ...-Digest.crc32 => mc-3113-big-Digest.crc32} | 0 ...-1-big-Filter.db => mc-3113-big-Filter.db} | Bin ...mc-1-big-Index.db => mc-3113-big-Index.db} | Bin ...tatistics.db => mc-3113-big-Statistics.db} | Bin ...-big-Summary.db => mc-3113-big-Summary.db} | Bin .../{mc-1-big-TOC.txt => mc-3113-big-TOC.txt} | 0 ...Info.db => mc-3113-big-CompressionInfo.db} | Bin .../{mc-1-big-Data.db => mc-3113-big-Data.db} | Bin ...-Digest.crc32 => mc-3113-big-Digest.crc32} | 0 .../mc-3113-big-Filter.db} | Bin ...mc-1-big-Index.db => mc-3113-big-Index.db} | Bin ...tatistics.db => mc-3113-big-Statistics.db} | Bin .../mc-3113-big-Summary.db} | Bin .../{mc-1-big-TOC.txt => mc-3113-big-TOC.txt} | 0 ...Info.db => mc-3113-big-CompressionInfo.db} | Bin .../{mc-1-big-Data.db => mc-3113-big-Data.db} | Bin ...-Digest.crc32 => mc-3113-big-Digest.crc32} | 0 .../mc-3113-big-Filter.db} | Bin ...mc-1-big-Index.db => mc-3113-big-Index.db} | Bin ...tatistics.db => mc-3113-big-Statistics.db} | Bin .../mc-3113-big-Summary.db} | Bin .../{mc-1-big-TOC.txt => mc-3113-big-TOC.txt} | 0 ...Info.db => mc-3113-big-CompressionInfo.db} | Bin .../{mc-1-big-Data.db => mc-3113-big-Data.db} | Bin ...-Digest.crc32 => mc-3113-big-Digest.crc32} | 0 .../mc-3113-big-Filter.db} | Bin ...mc-1-big-Index.db => mc-3113-big-Index.db} | Bin ...tatistics.db => mc-3113-big-Statistics.db} | Bin .../mc-3113-big-Summary.db} | Bin .../{mc-1-big-TOC.txt => mc-3113-big-TOC.txt} | 0 ...Info.db => mc-3113-big-CompressionInfo.db} | Bin .../{mc-1-big-Data.db => mc-3113-big-Data.db} | Bin ...-Digest.crc32 => mc-3113-big-Digest.crc32} | 0 .../mc-3113-big-Filter.db} | Bin ...mc-1-big-Index.db => mc-3113-big-Index.db} | Bin ...tatistics.db => mc-3113-big-Statistics.db} | Bin .../mc-3113-big-Summary.db} | Bin .../{mc-1-big-TOC.txt => mc-3113-big-TOC.txt} | 0 .../mc-3113-big-CompressionInfo.db | Bin 0 -> 43 bytes .../legacy_mc_tuple/mc-3113-big-Data.db | Bin 0 -> 121 bytes .../legacy_mc_tuple/mc-3113-big-Digest.crc32 | 1 + .../legacy_mc_tuple/mc-3113-big-Filter.db} | Bin .../legacy_mc_tuple/mc-3113-big-Index.db | Bin 0 -> 28 bytes .../legacy_mc_tuple/mc-3113-big-Statistics.db | Bin 0 -> 4791 bytes .../legacy_mc_tuple/mc-3113-big-Summary.db} | Bin .../legacy_mc_tuple/mc-3113-big-TOC.txt} | 8 +- ...nfo.db => md-31110-big-CompressionInfo.db} | Bin ...{md-1-big-Data.db => md-31110-big-Data.db} | Bin ...Digest.crc32 => md-31110-big-Digest.crc32} | 0 .../md-31110-big-Filter.db} | Bin ...d-1-big-Index.db => md-31110-big-Index.db} | Bin ...atistics.db => md-31110-big-Statistics.db} | Bin .../md-31110-big-Summary.db} | Bin ...{md-1-big-TOC.txt => md-31110-big-TOC.txt} | 0 ...nfo.db => md-31110-big-CompressionInfo.db} | Bin ...{md-1-big-Data.db => md-31110-big-Data.db} | Bin ...Digest.crc32 => md-31110-big-Digest.crc32} | 0 .../md-31110-big-Filter.db} | Bin ...d-1-big-Index.db => md-31110-big-Index.db} | Bin ...atistics.db => md-31110-big-Statistics.db} | Bin .../md-31110-big-Summary.db} | Bin ...{md-1-big-TOC.txt => md-31110-big-TOC.txt} | 0 ...nfo.db => md-31110-big-CompressionInfo.db} | Bin ...{md-1-big-Data.db => md-31110-big-Data.db} | Bin ...Digest.crc32 => md-31110-big-Digest.crc32} | 0 .../md-31110-big-Filter.db} | Bin ...d-1-big-Index.db => md-31110-big-Index.db} | Bin ...atistics.db => md-31110-big-Statistics.db} | Bin .../md-31110-big-Summary.db} | Bin ...{md-1-big-TOC.txt => md-31110-big-TOC.txt} | 0 ...nfo.db => md-31110-big-CompressionInfo.db} | Bin ...{md-1-big-Data.db => md-31110-big-Data.db} | Bin ...Digest.crc32 => md-31110-big-Digest.crc32} | 0 .../md-31110-big-Filter.db} | Bin ...d-1-big-Index.db => md-31110-big-Index.db} | Bin ...atistics.db => md-31110-big-Statistics.db} | Bin .../md-31110-big-Summary.db} | Bin ...{md-1-big-TOC.txt => md-31110-big-TOC.txt} | 0 ...nfo.db => md-31110-big-CompressionInfo.db} | Bin ...{md-1-big-Data.db => md-31110-big-Data.db} | Bin ...Digest.crc32 => md-31110-big-Digest.crc32} | 0 .../md-31110-big-Filter.db} | Bin ...d-1-big-Index.db => md-31110-big-Index.db} | Bin ...atistics.db => md-31110-big-Statistics.db} | Bin .../md-31110-big-Summary.db} | Bin ...{md-1-big-TOC.txt => md-31110-big-TOC.txt} | 0 ...nfo.db => md-31110-big-CompressionInfo.db} | Bin ...{md-1-big-Data.db => md-31110-big-Data.db} | Bin ...Digest.crc32 => md-31110-big-Digest.crc32} | 0 .../md-31110-big-Filter.db} | Bin ...d-1-big-Index.db => md-31110-big-Index.db} | Bin ...atistics.db => md-31110-big-Statistics.db} | Bin .../md-31110-big-Summary.db} | Bin ...{md-1-big-TOC.txt => md-31110-big-TOC.txt} | 0 ...nfo.db => md-31110-big-CompressionInfo.db} | Bin ...{md-1-big-Data.db => md-31110-big-Data.db} | Bin ...Digest.crc32 => md-31110-big-Digest.crc32} | 0 .../md-31110-big-Filter.db} | Bin ...d-1-big-Index.db => md-31110-big-Index.db} | Bin ...atistics.db => md-31110-big-Statistics.db} | Bin .../md-31110-big-Summary.db} | Bin ...{md-1-big-TOC.txt => md-31110-big-TOC.txt} | 0 ...nfo.db => md-31110-big-CompressionInfo.db} | Bin ...{md-1-big-Data.db => md-31110-big-Data.db} | Bin ...Digest.crc32 => md-31110-big-Digest.crc32} | 0 .../md-31110-big-Filter.db} | Bin ...d-1-big-Index.db => md-31110-big-Index.db} | Bin ...atistics.db => md-31110-big-Statistics.db} | Bin .../md-31110-big-Summary.db} | Bin ...{md-1-big-TOC.txt => md-31110-big-TOC.txt} | 0 .../md-31110-big-CompressionInfo.db | Bin 0 -> 43 bytes .../legacy_md_tuple/md-31110-big-Data.db | Bin 0 -> 121 bytes .../legacy_md_tuple/md-31110-big-Digest.crc32 | 1 + .../legacy_md_tuple/md-31110-big-Filter.db} | Bin .../legacy_md_tuple/md-31110-big-Index.db | Bin 0 -> 28 bytes .../md-31110-big-Statistics.db | Bin 0 -> 4855 bytes .../legacy_md_tuple/md-31110-big-Summary.db} | Bin .../legacy_md_tuple/md-31110-big-TOC.txt} | 4 +- .../me-3025-big-CompressionInfo.db | Bin 0 -> 83 bytes .../legacy_me_clust/me-3025-big-Data.db | Bin 0 -> 5236 bytes .../legacy_me_clust/me-3025-big-Digest.crc32 | 1 + .../me-3025-big-Filter.db} | Bin .../legacy_me_clust/me-3025-big-Index.db | Bin 0 -> 157553 bytes .../me-3025-big-Statistics.db} | Bin 9718 -> 7151 bytes .../me-3025-big-Summary.db} | Bin .../legacy_me_clust/me-3025-big-TOC.txt | 8 + ...nfo.db => me-31111-big-CompressionInfo.db} | Bin ...{me-1-big-Data.db => me-31111-big-Data.db} | Bin ...Digest.crc32 => me-31111-big-Digest.crc32} | 0 .../me-31111-big-Filter.db} | Bin ...e-1-big-Index.db => me-31111-big-Index.db} | Bin ...atistics.db => me-31111-big-Statistics.db} | Bin .../me-31111-big-Summary.db} | Bin ...{me-1-big-TOC.txt => me-31111-big-TOC.txt} | 0 .../me-3025-big-CompressionInfo.db | Bin 0 -> 83 bytes .../me-3025-big-Data.db | Bin 0 -> 5213 bytes .../me-3025-big-Digest.crc32 | 1 + .../me-3025-big-Filter.db} | Bin .../me-3025-big-Index.db} | Bin 157553 -> 157553 bytes .../me-3025-big-Statistics.db} | Bin 9727 -> 7151 bytes .../me-3025-big-Summary.db} | Bin .../me-3025-big-TOC.txt | 8 + ...nfo.db => me-31111-big-CompressionInfo.db} | Bin ...{me-1-big-Data.db => me-31111-big-Data.db} | Bin ...Digest.crc32 => me-31111-big-Digest.crc32} | 0 .../me-31111-big-Filter.db} | Bin ...e-1-big-Index.db => me-31111-big-Index.db} | Bin ...atistics.db => me-31111-big-Statistics.db} | Bin .../me-31111-big-Summary.db} | Bin ...{me-1-big-TOC.txt => me-31111-big-TOC.txt} | 0 ...nfo.db => me-31111-big-CompressionInfo.db} | Bin ...{me-1-big-Data.db => me-31111-big-Data.db} | Bin ...Digest.crc32 => me-31111-big-Digest.crc32} | 0 .../me-31111-big-Filter.db} | Bin ...e-1-big-Index.db => me-31111-big-Index.db} | Bin ...atistics.db => me-31111-big-Statistics.db} | Bin .../me-31111-big-Summary.db} | Bin ...{me-1-big-TOC.txt => me-31111-big-TOC.txt} | 0 .../me-3025-big-CompressionInfo.db | Bin 0 -> 75 bytes .../me-3025-big-Data.db | Bin 0 -> 4482 bytes .../me-3025-big-Digest.crc32 | 1 + .../me-3025-big-Filter.db} | Bin .../me-3025-big-Index.db} | Bin 157553 -> 157553 bytes .../me-3025-big-Statistics.db | Bin 0 -> 7160 bytes .../me-3025-big-Summary.db} | Bin .../me-3025-big-TOC.txt | 8 + ...nfo.db => me-31111-big-CompressionInfo.db} | Bin ...{me-1-big-Data.db => me-31111-big-Data.db} | Bin ...Digest.crc32 => me-31111-big-Digest.crc32} | 0 .../me-31111-big-Filter.db} | Bin ...e-1-big-Index.db => me-31111-big-Index.db} | Bin ...atistics.db => me-31111-big-Statistics.db} | Bin .../me-31111-big-Summary.db} | Bin ...{me-1-big-TOC.txt => me-31111-big-TOC.txt} | 0 ...Info.db => me-3025-big-CompressionInfo.db} | Bin .../legacy_me_simple/me-3025-big-Data.db | Bin 0 -> 89 bytes .../legacy_me_simple/me-3025-big-Digest.crc32 | 1 + .../legacy_me_simple/me-3025-big-Filter.db} | Bin ...me-1-big-Index.db => me-3025-big-Index.db} | Bin .../me-3025-big-Statistics.db | Bin 0 -> 4704 bytes .../legacy_me_simple/me-3025-big-Summary.db} | Bin .../legacy_me_simple/me-3025-big-TOC.txt | 8 + .../me-31111-big-CompressionInfo.db | Bin 0 -> 43 bytes ...{me-1-big-Data.db => me-31111-big-Data.db} | Bin ...Digest.crc32 => me-31111-big-Digest.crc32} | 0 .../legacy_me_simple/me-31111-big-Filter.db | Bin 0 -> 24 bytes .../legacy_me_simple/me-31111-big-Index.db} | Bin ...atistics.db => me-31111-big-Statistics.db} | Bin .../legacy_me_simple/me-31111-big-Summary.db} | Bin ...{me-1-big-TOC.txt => me-31111-big-TOC.txt} | 0 ...Info.db => me-3025-big-CompressionInfo.db} | Bin .../me-3025-big-Data.db | Bin 0 -> 89 bytes .../me-3025-big-Digest.crc32 | 1 + .../me-3025-big-Filter.db | Bin 0 -> 24 bytes ...me-1-big-Index.db => me-3025-big-Index.db} | Bin .../me-3025-big-Statistics.db | Bin 0 -> 4745 bytes .../me-3025-big-Summary.db} | Bin .../me-3025-big-TOC.txt | 8 + .../me-31111-big-CompressionInfo.db | Bin 0 -> 43 bytes ...{me-1-big-Data.db => me-31111-big-Data.db} | Bin ...Digest.crc32 => me-31111-big-Digest.crc32} | 0 .../me-31111-big-Filter.db | Bin 0 -> 24 bytes .../me-31111-big-Index.db} | Bin ...atistics.db => me-31111-big-Statistics.db} | Bin .../me-31111-big-Summary.db} | Bin ...{me-1-big-TOC.txt => me-31111-big-TOC.txt} | 0 ...nfo.db => me-31111-big-CompressionInfo.db} | Bin ...{me-1-big-Data.db => me-31111-big-Data.db} | Bin ...Digest.crc32 => me-31111-big-Digest.crc32} | 0 .../me-31111-big-Filter.db | Bin 0 -> 24 bytes ...e-1-big-Index.db => me-31111-big-Index.db} | Bin ...atistics.db => me-31111-big-Statistics.db} | Bin .../me-31111-big-Summary.db} | Bin ...{me-1-big-TOC.txt => me-31111-big-TOC.txt} | 0 ...Info.db => me-3025-big-CompressionInfo.db} | Bin .../me-3025-big-Data.db | Bin 0 -> 110 bytes .../me-3025-big-Digest.crc32 | 1 + .../me-3025-big-Filter.db | Bin 0 -> 24 bytes ...me-1-big-Index.db => me-3025-big-Index.db} | Bin .../me-3025-big-Statistics.db | Bin 0 -> 4754 bytes .../me-3025-big-Summary.db} | Bin .../me-3025-big-TOC.txt | 8 + .../me-31111-big-CompressionInfo.db | Bin 0 -> 43 bytes ...{me-1-big-Data.db => me-31111-big-Data.db} | Bin ...Digest.crc32 => me-31111-big-Digest.crc32} | 0 .../me-31111-big-Filter.db | Bin 0 -> 24 bytes .../me-31111-big-Index.db} | Bin ...atistics.db => me-31111-big-Statistics.db} | Bin .../me-31111-big-Summary.db} | Bin ...{me-1-big-TOC.txt => me-31111-big-TOC.txt} | 0 .../me-31111-big-CompressionInfo.db | Bin 0 -> 43 bytes .../legacy_me_tuple/me-31111-big-Data.db | Bin 0 -> 284 bytes .../legacy_me_tuple/me-31111-big-Digest.crc32 | 1 + .../legacy_me_tuple/me-31111-big-Filter.db | Bin 0 -> 24 bytes .../legacy_me_tuple/me-31111-big-Index.db | Bin 0 -> 29 bytes .../me-31111-big-Statistics.db | Bin 0 -> 5603 bytes .../legacy_me_tuple/me-31111-big-Summary.db} | Bin .../legacy_me_tuple/me-31111-big-TOC.txt | 8 + .../me-31111-big-CompressionInfo.db | Bin 0 -> 43 bytes .../me-31111-big-Data.db | Bin 0 -> 282 bytes .../me-31111-big-Digest.crc32 | 1 + .../me-31111-big-Filter.db | Bin 0 -> 24 bytes .../me-31111-big-Index.db | Bin 0 -> 29 bytes .../me-31111-big-Statistics.db | Bin 0 -> 5603 bytes .../me-31111-big-Summary.db} | Bin .../me-31111-big-TOC.txt | 8 + ...nInfo.db => na-400-big-CompressionInfo.db} | Bin .../{na-1-big-Data.db => na-400-big-Data.db} | Bin ...g-Digest.crc32 => na-400-big-Digest.crc32} | 0 .../na-400-big-Filter.db} | Bin ...{na-1-big-Index.db => na-400-big-Index.db} | Bin ...Statistics.db => na-400-big-Statistics.db} | Bin .../legacy_na_clust/na-400-big-Summary.db} | Bin .../{na-1-big-TOC.txt => na-400-big-TOC.txt} | 0 ...nInfo.db => na-400-big-CompressionInfo.db} | Bin .../{na-1-big-Data.db => na-400-big-Data.db} | Bin ...g-Digest.crc32 => na-400-big-Digest.crc32} | 0 .../na-400-big-Filter.db | Bin 0 -> 24 bytes ...{na-1-big-Index.db => na-400-big-Index.db} | Bin ...Statistics.db => na-400-big-Statistics.db} | Bin .../na-400-big-Summary.db} | Bin .../{na-1-big-TOC.txt => na-400-big-TOC.txt} | 0 ...nInfo.db => na-400-big-CompressionInfo.db} | Bin .../{na-1-big-Data.db => na-400-big-Data.db} | Bin ...g-Digest.crc32 => na-400-big-Digest.crc32} | 0 .../na-400-big-Filter.db} | Bin ...{na-1-big-Index.db => na-400-big-Index.db} | Bin ...Statistics.db => na-400-big-Statistics.db} | Bin .../na-400-big-Summary.db} | Bin .../{na-1-big-TOC.txt => na-400-big-TOC.txt} | 0 ...nInfo.db => na-400-big-CompressionInfo.db} | Bin .../{na-1-big-Data.db => na-400-big-Data.db} | Bin ...g-Digest.crc32 => na-400-big-Digest.crc32} | 0 .../na-400-big-Filter.db | Bin 0 -> 24 bytes ...{na-1-big-Index.db => na-400-big-Index.db} | Bin ...Statistics.db => na-400-big-Statistics.db} | Bin .../na-400-big-Summary.db} | Bin .../{na-1-big-TOC.txt => na-400-big-TOC.txt} | 0 ...nInfo.db => na-400-big-CompressionInfo.db} | Bin .../{na-1-big-Data.db => na-400-big-Data.db} | Bin ...g-Digest.crc32 => na-400-big-Digest.crc32} | 0 .../na-400-big-Filter.db} | Bin .../legacy_na_simple/na-400-big-Index.db} | Bin ...Statistics.db => na-400-big-Statistics.db} | Bin .../legacy_na_simple/na-400-big-Summary.db} | Bin .../{na-1-big-TOC.txt => na-400-big-TOC.txt} | 0 ...nInfo.db => na-400-big-CompressionInfo.db} | Bin .../{na-1-big-Data.db => na-400-big-Data.db} | Bin ...g-Digest.crc32 => na-400-big-Digest.crc32} | 0 .../na-400-big-Filter.db | Bin 0 -> 24 bytes .../na-400-big-Index.db | Bin 0 -> 26 bytes ...Statistics.db => na-400-big-Statistics.db} | Bin .../na-400-big-Summary.db} | Bin .../{na-1-big-TOC.txt => na-400-big-TOC.txt} | 0 ...nInfo.db => na-400-big-CompressionInfo.db} | Bin .../{na-1-big-Data.db => na-400-big-Data.db} | Bin ...g-Digest.crc32 => na-400-big-Digest.crc32} | 0 .../na-400-big-Filter.db} | Bin ...{na-1-big-Index.db => na-400-big-Index.db} | Bin ...Statistics.db => na-400-big-Statistics.db} | Bin .../na-400-big-Summary.db} | Bin .../{na-1-big-TOC.txt => na-400-big-TOC.txt} | 0 ...nInfo.db => na-400-big-CompressionInfo.db} | Bin .../{na-1-big-Data.db => na-400-big-Data.db} | Bin ...g-Digest.crc32 => na-400-big-Digest.crc32} | 0 .../na-400-big-Filter.db | Bin 0 -> 24 bytes .../na-400-big-Index.db | Bin 0 -> 27 bytes ...Statistics.db => na-400-big-Statistics.db} | Bin .../na-400-big-Summary.db} | Bin .../{na-1-big-TOC.txt => na-400-big-TOC.txt} | 0 .../na-400-big-CompressionInfo.db | Bin 0 -> 47 bytes .../legacy_na_tuple/na-400-big-Data.db | Bin 0 -> 307 bytes .../legacy_na_tuple/na-400-big-Digest.crc32 | 1 + .../legacy_na_tuple/na-400-big-Filter.db} | Bin .../legacy_na_tuple/na-400-big-Index.db | Bin 0 -> 29 bytes .../legacy_na_tuple/na-400-big-Statistics.db | Bin 0 -> 5676 bytes .../legacy_na_tuple/na-400-big-Summary.db | Bin 0 -> 47 bytes .../legacy_na_tuple/na-400-big-TOC.txt | 8 + ...nInfo.db => nb-400-big-CompressionInfo.db} | Bin .../{nb-1-big-Data.db => nb-400-big-Data.db} | Bin ...g-Digest.crc32 => nb-400-big-Digest.crc32} | 0 .../nb-400-big-Filter.db} | Bin ...{nb-1-big-Index.db => nb-400-big-Index.db} | Bin ...Statistics.db => nb-400-big-Statistics.db} | Bin .../legacy_nb_clust/nb-400-big-Summary.db | Bin 0 -> 47 bytes .../{nb-1-big-TOC.txt => nb-400-big-TOC.txt} | 0 ...nInfo.db => nb-400-big-CompressionInfo.db} | Bin .../{nb-1-big-Data.db => nb-400-big-Data.db} | Bin ...g-Digest.crc32 => nb-400-big-Digest.crc32} | 0 .../nb-400-big-Filter.db} | Bin ...{nb-1-big-Index.db => nb-400-big-Index.db} | Bin ...Statistics.db => nb-400-big-Statistics.db} | Bin .../nb-400-big-Summary.db | Bin 0 -> 47 bytes .../{nb-1-big-TOC.txt => nb-400-big-TOC.txt} | 0 ...3qdyo2i4kzfpt3cqcy-big-CompressionInfo.db} | Bin ...b-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-Data.db | Bin 0 -> 89 bytes ...l_1kke_3qdyo2i4kzfpt3cqcy-big-Digest.crc32 | 1 + ...gll_1kke_3qdyo2i4kzfpt3cqcy-big-Filter.db} | Bin ...3gll_1kke_3qdyo2i4kzfpt3cqcy-big-Index.db} | Bin ...1kke_3qdyo2i4kzfpt3cqcy-big-Statistics.db} | Bin 4822 -> 4730 bytes ...gll_1kke_3qdyo2i4kzfpt3cqcy-big-Summary.db | Bin 0 -> 47 bytes ...b-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-TOC.txt | 8 + .../nb-400-big-CompressionInfo.db} | Bin .../{nb-1-big-Data.db => nb-400-big-Data.db} | Bin ...g-Digest.crc32 => nb-400-big-Digest.crc32} | 0 .../legacy_nb_simple/nb-400-big-Filter.db} | Bin .../legacy_nb_simple/nb-400-big-Index.db | Bin 0 -> 26 bytes ...Statistics.db => nb-400-big-Statistics.db} | Bin .../legacy_nb_simple/nb-400-big-Summary.db | Bin 0 -> 47 bytes .../{nb-1-big-TOC.txt => nb-400-big-TOC.txt} | 0 ...nInfo.db => nb-400-big-CompressionInfo.db} | Bin .../{nb-1-big-Data.db => nb-400-big-Data.db} | Bin ...g-Digest.crc32 => nb-400-big-Digest.crc32} | 0 .../nb-400-big-Filter.db} | Bin ...{nb-1-big-Index.db => nb-400-big-Index.db} | Bin ...Statistics.db => nb-400-big-Statistics.db} | Bin .../nb-400-big-Summary.db | Bin 0 -> 47 bytes .../{nb-1-big-TOC.txt => nb-400-big-TOC.txt} | 0 ..._4abk02i4kzfpt3cqcy-big-CompressionInfo.db | Bin 0 -> 47 bytes ...b-3gll_1kke_4abk02i4kzfpt3cqcy-big-Data.db | Bin 0 -> 122 bytes ...l_1kke_4abk02i4kzfpt3cqcy-big-Digest.crc32 | 1 + ...gll_1kke_4abk02i4kzfpt3cqcy-big-Filter.db} | Bin ...-3gll_1kke_4abk02i4kzfpt3cqcy-big-Index.db | Bin 0 -> 28 bytes ...1kke_4abk02i4kzfpt3cqcy-big-Statistics.db} | Bin 4831 -> 4942 bytes ...gll_1kke_4abk02i4kzfpt3cqcy-big-Summary.db | Bin 0 -> 47 bytes ...b-3gll_1kke_4abk02i4kzfpt3cqcy-big-TOC.txt | 8 + .../nb-400-big-CompressionInfo.db | Bin 0 -> 47 bytes .../legacy_nb_tuple/nb-400-big-Data.db | Bin 0 -> 309 bytes .../legacy_nb_tuple/nb-400-big-Digest.crc32 | 1 + .../legacy_nb_tuple/nb-400-big-Filter.db} | Bin .../legacy_nb_tuple/nb-400-big-Index.db | Bin 0 -> 29 bytes .../legacy_nb_tuple/nb-400-big-Statistics.db | Bin 0 -> 5773 bytes .../legacy_nb_tuple/nb-400-big-Summary.db | Bin 0 -> 47 bytes .../legacy_nb_tuple/nb-400-big-TOC.txt} | 0 .../nc-1-big-CompressionInfo.db | Bin 207 -> 0 bytes .../legacy_nc_clust/nc-1-big-Data.db | Bin 8713 -> 0 bytes .../legacy_nc_clust/nc-1-big-Digest.crc32 | 1 - .../nc-1-big-CompressionInfo.db | Bin 199 -> 0 bytes .../legacy_nc_clust_counter/nc-1-big-Data.db | Bin 7529 -> 0 bytes .../nc-1-big-Digest.crc32 | 1 - .../legacy_nc_simple/nc-1-big-Data.db | Bin 87 -> 0 bytes .../legacy_nc_simple/nc-1-big-Digest.crc32 | 1 - .../nc-1-big-CompressionInfo.db | Bin 47 -> 0 bytes .../legacy_nc_simple_counter/nc-1-big-Data.db | Bin 138 -> 0 bytes .../nc-1-big-Digest.crc32 | 1 - .../nc-1-big-Index.db | Bin 27 -> 0 bytes ...nInfo.db => oa-500-big-CompressionInfo.db} | Bin .../{oa-1-big-Data.db => oa-500-big-Data.db} | Bin ...g-Digest.crc32 => oa-500-big-Digest.crc32} | 0 .../oa-500-big-Filter.db} | Bin ...{oa-1-big-Index.db => oa-500-big-Index.db} | Bin ...Statistics.db => oa-500-big-Statistics.db} | Bin .../legacy_oa_clust/oa-500-big-Summary.db | Bin 0 -> 47 bytes .../oa-500-big-TOC.txt} | 0 ...nInfo.db => oa-500-big-CompressionInfo.db} | Bin .../{oa-1-big-Data.db => oa-500-big-Data.db} | Bin ...g-Digest.crc32 => oa-500-big-Digest.crc32} | 0 .../oa-500-big-Filter.db} | Bin ...{oa-1-big-Index.db => oa-500-big-Index.db} | Bin ...Statistics.db => oa-500-big-Statistics.db} | Bin .../oa-500-big-Summary.db | Bin 0 -> 47 bytes .../oa-500-big-TOC.txt} | 0 ...nInfo.db => oa-500-big-CompressionInfo.db} | Bin .../{oa-1-big-Data.db => oa-500-big-Data.db} | Bin ...g-Digest.crc32 => oa-500-big-Digest.crc32} | 0 .../oa-500-big-Filter.db} | Bin ...{oa-1-big-Index.db => oa-500-big-Index.db} | Bin ...Statistics.db => oa-500-big-Statistics.db} | Bin .../legacy_oa_simple/oa-500-big-Summary.db | Bin 0 -> 47 bytes .../oa-500-big-TOC.txt} | 0 ...nInfo.db => oa-500-big-CompressionInfo.db} | Bin .../{oa-1-big-Data.db => oa-500-big-Data.db} | Bin ...g-Digest.crc32 => oa-500-big-Digest.crc32} | 0 .../oa-500-big-Filter.db | Bin 0 -> 24 bytes ...{oa-1-big-Index.db => oa-500-big-Index.db} | Bin ...Statistics.db => oa-500-big-Statistics.db} | Bin .../oa-500-big-Summary.db | Bin 0 -> 47 bytes .../oa-500-big-TOC.txt | 8 + .../oa-500-big-CompressionInfo.db | Bin 0 -> 47 bytes .../legacy_oa_tuple/oa-500-big-Data.db | Bin 0 -> 301 bytes .../legacy_oa_tuple/oa-500-big-Digest.crc32 | 1 + .../legacy_oa_tuple/oa-500-big-Filter.db | Bin 0 -> 24 bytes .../legacy_oa_tuple/oa-500-big-Index.db | Bin 0 -> 29 bytes .../legacy_oa_tuple/oa-500-big-Statistics.db | Bin 0 -> 5785 bytes .../legacy_oa_tuple/oa-500-big-Summary.db | Bin 0 -> 47 bytes .../legacy_oa_tuple/oa-500-big-TOC.txt | 8 + .../io/sstable/LegacySSTableTest.java | 205 ++++++++++++++---- .../tools/SSTablePartitionsTest.java | 68 +++--- .../utils/concurrent/RefCountedTest.java | 2 +- 647 files changed, 338 insertions(+), 101 deletions(-) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/{da-1-bti-CompressionInfo.db => da-500-bti-CompressionInfo.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/{da-1-bti-Data.db => da-500-bti-Data.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/{da-1-bti-Digest.crc32 => da-500-bti-Digest.crc32} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/{da-1-bti-Filter.db => da-500-bti-Filter.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/{da-1-bti-Partitions.db => da-500-bti-Partitions.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/{da-1-bti-Rows.db => da-500-bti-Rows.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/{da-1-bti-Statistics.db => da-500-bti-Statistics.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/{da-1-bti-TOC.txt => da-500-bti-TOC.txt} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/{da-1-bti-CompressionInfo.db => da-500-bti-CompressionInfo.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/{da-1-bti-Data.db => da-500-bti-Data.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/{da-1-bti-Digest.crc32 => da-500-bti-Digest.crc32} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/{da-1-bti-Filter.db => da-500-bti-Filter.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/{da-1-bti-Partitions.db => da-500-bti-Partitions.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/{da-1-bti-Rows.db => da-500-bti-Rows.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/{da-1-bti-Statistics.db => da-500-bti-Statistics.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/{da-1-bti-TOC.txt => da-500-bti-TOC.txt} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/{da-1-bti-CompressionInfo.db => da-500-bti-CompressionInfo.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/{da-1-bti-Data.db => da-500-bti-Data.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/{da-1-bti-Digest.crc32 => da-500-bti-Digest.crc32} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/{da-1-bti-Filter.db => da-500-bti-Filter.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/{da-1-bti-Partitions.db => da-500-bti-Partitions.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/{da-1-bti-Rows.db => da-500-bti-Rows.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/{da-1-bti-Statistics.db => da-500-bti-Statistics.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/{da-1-bti-TOC.txt => da-500-bti-TOC.txt} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/{da-1-bti-CompressionInfo.db => da-500-bti-CompressionInfo.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/{da-1-bti-Data.db => da-500-bti-Data.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/{da-1-bti-Digest.crc32 => da-500-bti-Digest.crc32} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/{da-1-bti-Filter.db => da-500-bti-Filter.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/{da-1-bti-Partitions.db => da-500-bti-Partitions.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/{da-1-bti-Rows.db => da-500-bti-Rows.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/{da-1-bti-Statistics.db => da-500-bti-Statistics.db} (100%) rename test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/{da-1-bti-TOC.txt => da-500-bti-TOC.txt} (100%) create mode 100644 test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-CompressionInfo.db create mode 100644 test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-Data.db create mode 100644 test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-Digest.crc32 rename test/data/legacy-sstables/{na/legacy_tables/legacy_na_clust/na-1-big-Filter.db => da/legacy_tables/legacy_da_tuple/da-500-bti-Filter.db} (100%) create mode 100644 test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-Partitions.db create mode 100644 test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-Rows.db create mode 100644 test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-Statistics.db create mode 100644 test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-TOC.txt rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/{ma-1-big-CompressionInfo.db => ma-306-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/{ma-1-big-Data.db => ma-306-big-Data.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/{ma-1-big-Digest.crc32 => ma-306-big-Digest.crc32} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/{ma-1-big-Filter.db => ma-306-big-Filter.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/{ma-1-big-Index.db => ma-306-big-Index.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/{ma-1-big-Statistics.db => ma-306-big-Statistics.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/{ma-1-big-Summary.db => ma-306-big-Summary.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/{ma-1-big-TOC.txt => ma-306-big-TOC.txt} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/{ma-1-big-CompressionInfo.db => ma-306-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/{ma-1-big-Data.db => ma-306-big-Data.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/{ma-1-big-Digest.crc32 => ma-306-big-Digest.crc32} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/{ma-1-big-Filter.db => ma-306-big-Filter.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/{ma-1-big-Index.db => ma-306-big-Index.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/{ma-1-big-Statistics.db => ma-306-big-Statistics.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/{ma-1-big-Summary.db => ma-306-big-Summary.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/{ma-1-big-TOC.txt => ma-306-big-TOC.txt} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/{ma-1-big-CompressionInfo.db => ma-306-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/{ma-1-big-Data.db => ma-306-big-Data.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/{ma-1-big-Digest.crc32 => ma-306-big-Digest.crc32} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/{ma-1-big-Filter.db => ma-306-big-Filter.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/{ma-1-big-Index.db => ma-306-big-Index.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/{ma-1-big-Statistics.db => ma-306-big-Statistics.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/{ma-1-big-Summary.db => ma-306-big-Summary.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/{ma-1-big-TOC.txt => ma-306-big-TOC.txt} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/{ma-1-big-CompressionInfo.db => ma-306-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/{ma-1-big-Data.db => ma-306-big-Data.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/{ma-1-big-Digest.crc32 => ma-306-big-Digest.crc32} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/{ma-1-big-Filter.db => ma-306-big-Filter.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/{ma-1-big-Index.db => ma-306-big-Index.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/{ma-1-big-Statistics.db => ma-306-big-Statistics.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/{ma-1-big-Summary.db => ma-306-big-Summary.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/{ma-1-big-TOC.txt => ma-306-big-TOC.txt} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/{ma-1-big-CompressionInfo.db => ma-306-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/{ma-1-big-Data.db => ma-306-big-Data.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/{ma-1-big-Digest.crc32 => ma-306-big-Digest.crc32} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/{ma-1-big-Filter.db => ma-306-big-Filter.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/{ma-1-big-Index.db => ma-306-big-Index.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/{ma-1-big-Statistics.db => ma-306-big-Statistics.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/{ma-1-big-Summary.db => ma-306-big-Summary.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/{ma-1-big-TOC.txt => ma-306-big-TOC.txt} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/{ma-1-big-CompressionInfo.db => ma-306-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/{ma-1-big-Data.db => ma-306-big-Data.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/{ma-1-big-Digest.crc32 => ma-306-big-Digest.crc32} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/{ma-1-big-Filter.db => ma-306-big-Filter.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/{ma-1-big-Index.db => ma-306-big-Index.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/{ma-1-big-Statistics.db => ma-306-big-Statistics.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/{ma-1-big-Summary.db => ma-306-big-Summary.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/{ma-1-big-TOC.txt => ma-306-big-TOC.txt} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/{ma-1-big-CompressionInfo.db => ma-306-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/{ma-1-big-Data.db => ma-306-big-Data.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/{ma-1-big-Digest.crc32 => ma-306-big-Digest.crc32} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/{ma-1-big-Filter.db => ma-306-big-Filter.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/{ma-1-big-Index.db => ma-306-big-Index.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/{ma-1-big-Statistics.db => ma-306-big-Statistics.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/{ma-1-big-Summary.db => ma-306-big-Summary.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/{ma-1-big-TOC.txt => ma-306-big-TOC.txt} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/{ma-1-big-CompressionInfo.db => ma-306-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/{ma-1-big-Data.db => ma-306-big-Data.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/{ma-1-big-Digest.crc32 => ma-306-big-Digest.crc32} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/{ma-1-big-Filter.db => ma-306-big-Filter.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/{ma-1-big-Index.db => ma-306-big-Index.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/{ma-1-big-Statistics.db => ma-306-big-Statistics.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/{ma-1-big-Summary.db => ma-306-big-Summary.db} (100%) rename test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/{ma-1-big-TOC.txt => ma-306-big-TOC.txt} (100%) create mode 100644 test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-CompressionInfo.db create mode 100644 test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-Data.db create mode 100644 test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-Digest.crc32 rename test/data/legacy-sstables/{mb/legacy_tables/legacy_mb_clust/mb-1-big-Filter.db => ma/legacy_tables/legacy_ma_tuple/ma-306-big-Filter.db} (100%) create mode 100644 test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-Index.db create mode 100644 test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-Statistics.db rename test/data/legacy-sstables/{mb/legacy_tables/legacy_mb_clust/mb-1-big-Summary.db => ma/legacy_tables/legacy_ma_tuple/ma-306-big-Summary.db} (100%) rename test/data/legacy-sstables/{nc/legacy_tables/legacy_nc_clust/nc-1-big-TOC.txt => ma/legacy_tables/legacy_ma_tuple/ma-306-big-TOC.txt} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/{mb-1-big-CompressionInfo.db => mb-307-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/{mb-1-big-Data.db => mb-307-big-Data.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/{mb-1-big-Digest.crc32 => mb-307-big-Digest.crc32} (100%) rename test/data/legacy-sstables/mb/legacy_tables/{legacy_mb_clust_compact/mb-1-big-Filter.db => legacy_mb_clust/mb-307-big-Filter.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/{mb-1-big-Index.db => mb-307-big-Index.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/{mb-1-big-Statistics.db => mb-307-big-Statistics.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/{legacy_mb_clust_compact/mb-1-big-Summary.db => legacy_mb_clust/mb-307-big-Summary.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/{mb-1-big-TOC.txt => mb-307-big-TOC.txt} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/{mb-1-big-CompressionInfo.db => mb-307-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/{mb-1-big-Data.db => mb-307-big-Data.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/{mb-1-big-Digest.crc32 => mb-307-big-Digest.crc32} (100%) rename test/data/legacy-sstables/mb/legacy_tables/{legacy_mb_clust_counter/mb-1-big-Filter.db => legacy_mb_clust_compact/mb-307-big-Filter.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/{mb-1-big-Index.db => mb-307-big-Index.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/{mb-1-big-Statistics.db => mb-307-big-Statistics.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/{legacy_mb_clust_counter/mb-1-big-Summary.db => legacy_mb_clust_compact/mb-307-big-Summary.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/{mb-1-big-TOC.txt => mb-307-big-TOC.txt} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/{mb-1-big-CompressionInfo.db => mb-307-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/{mb-1-big-Data.db => mb-307-big-Data.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/{mb-1-big-Digest.crc32 => mb-307-big-Digest.crc32} (100%) rename test/data/legacy-sstables/mb/legacy_tables/{legacy_mb_clust_counter_compact/mb-1-big-Filter.db => legacy_mb_clust_counter/mb-307-big-Filter.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/{mb-1-big-Index.db => mb-307-big-Index.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/{mb-1-big-Statistics.db => mb-307-big-Statistics.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/{legacy_mb_clust_counter_compact/mb-1-big-Summary.db => legacy_mb_clust_counter/mb-307-big-Summary.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/{mb-1-big-TOC.txt => mb-307-big-TOC.txt} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/{mb-1-big-CompressionInfo.db => mb-307-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/{mb-1-big-Data.db => mb-307-big-Data.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/{mb-1-big-Digest.crc32 => mb-307-big-Digest.crc32} (100%) rename test/data/legacy-sstables/mb/legacy_tables/{legacy_mb_simple/mb-1-big-Filter.db => legacy_mb_clust_counter_compact/mb-307-big-Filter.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/{mb-1-big-Index.db => mb-307-big-Index.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/{mb-1-big-Statistics.db => mb-307-big-Statistics.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/{legacy_mb_simple/mb-1-big-Summary.db => legacy_mb_clust_counter_compact/mb-307-big-Summary.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/{mb-1-big-TOC.txt => mb-307-big-TOC.txt} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/{mb-1-big-CompressionInfo.db => mb-307-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/{mb-1-big-Data.db => mb-307-big-Data.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/{mb-1-big-Digest.crc32 => mb-307-big-Digest.crc32} (100%) rename test/data/legacy-sstables/mb/legacy_tables/{legacy_mb_simple_compact/mb-1-big-Filter.db => legacy_mb_simple/mb-307-big-Filter.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/{mb-1-big-Index.db => mb-307-big-Index.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/{mb-1-big-Statistics.db => mb-307-big-Statistics.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/{legacy_mb_simple_compact/mb-1-big-Summary.db => legacy_mb_simple/mb-307-big-Summary.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/{mb-1-big-TOC.txt => mb-307-big-TOC.txt} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/{mb-1-big-CompressionInfo.db => mb-307-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/{mb-1-big-Data.db => mb-307-big-Data.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/{mb-1-big-Digest.crc32 => mb-307-big-Digest.crc32} (100%) rename test/data/legacy-sstables/mb/legacy_tables/{legacy_mb_simple_counter/mb-1-big-Filter.db => legacy_mb_simple_compact/mb-307-big-Filter.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/{mb-1-big-Index.db => mb-307-big-Index.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/{mb-1-big-Statistics.db => mb-307-big-Statistics.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/{legacy_mb_simple_counter/mb-1-big-Summary.db => legacy_mb_simple_compact/mb-307-big-Summary.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/{mb-1-big-TOC.txt => mb-307-big-TOC.txt} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/{mb-1-big-CompressionInfo.db => mb-307-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/{mb-1-big-Data.db => mb-307-big-Data.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/{mb-1-big-Digest.crc32 => mb-307-big-Digest.crc32} (100%) rename test/data/legacy-sstables/mb/legacy_tables/{legacy_mb_simple_counter_compact/mb-1-big-Filter.db => legacy_mb_simple_counter/mb-307-big-Filter.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/{mb-1-big-Index.db => mb-307-big-Index.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/{mb-1-big-Statistics.db => mb-307-big-Statistics.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/{legacy_mb_simple_counter_compact/mb-1-big-Summary.db => legacy_mb_simple_counter/mb-307-big-Summary.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/{mb-1-big-TOC.txt => mb-307-big-TOC.txt} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/{mb-1-big-CompressionInfo.db => mb-307-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/{mb-1-big-Data.db => mb-307-big-Data.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/{mb-1-big-Digest.crc32 => mb-307-big-Digest.crc32} (100%) rename test/data/legacy-sstables/{mc/legacy_tables/legacy_mc_clust/mc-1-big-Filter.db => mb/legacy_tables/legacy_mb_simple_counter_compact/mb-307-big-Filter.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/{mb-1-big-Index.db => mb-307-big-Index.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/{mb-1-big-Statistics.db => mb-307-big-Statistics.db} (100%) rename test/data/legacy-sstables/{mc/legacy_tables/legacy_mc_clust/mc-1-big-Summary.db => mb/legacy_tables/legacy_mb_simple_counter_compact/mb-307-big-Summary.db} (100%) rename test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/{mb-1-big-TOC.txt => mb-307-big-TOC.txt} (100%) create mode 100644 test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-CompressionInfo.db create mode 100644 test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-Data.db create mode 100644 test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-Digest.crc32 rename test/data/legacy-sstables/{mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Filter.db => mb/legacy_tables/legacy_mb_tuple/mb-307-big-Filter.db} (100%) create mode 100644 test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-Index.db create mode 100644 test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-Statistics.db rename test/data/legacy-sstables/{mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Summary.db => mb/legacy_tables/legacy_mb_tuple/mb-307-big-Summary.db} (100%) rename test/data/legacy-sstables/{nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-TOC.txt => mb/legacy_tables/legacy_mb_tuple/mb-307-big-TOC.txt} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/{mc-1-big-CompressionInfo.db => mc-3113-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/{mc-1-big-Data.db => mc-3113-big-Data.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/{mc-1-big-Digest.crc32 => mc-3113-big-Digest.crc32} (100%) rename test/data/legacy-sstables/mc/legacy_tables/{legacy_mc_clust_counter/mc-1-big-Filter.db => legacy_mc_clust/mc-3113-big-Filter.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/{mc-1-big-Index.db => mc-3113-big-Index.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/{mc-1-big-Statistics.db => mc-3113-big-Statistics.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/{legacy_mc_clust_counter/mc-1-big-Summary.db => legacy_mc_clust/mc-3113-big-Summary.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/{mc-1-big-TOC.txt => mc-3113-big-TOC.txt} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/{mc-1-big-CompressionInfo.db => mc-3113-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/{mc-1-big-Data.db => mc-3113-big-Data.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/{mc-1-big-Digest.crc32 => mc-3113-big-Digest.crc32} (100%) rename test/data/legacy-sstables/mc/legacy_tables/{legacy_mc_clust_counter_compact/mc-1-big-Filter.db => legacy_mc_clust_compact/mc-3113-big-Filter.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/{mc-1-big-Index.db => mc-3113-big-Index.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/{mc-1-big-Statistics.db => mc-3113-big-Statistics.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/{legacy_mc_clust_counter_compact/mc-1-big-Summary.db => legacy_mc_clust_compact/mc-3113-big-Summary.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/{mc-1-big-TOC.txt => mc-3113-big-TOC.txt} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/{mc-1-big-CompressionInfo.db => mc-3113-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/{mc-1-big-Data.db => mc-3113-big-Data.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/{mc-1-big-Digest.crc32 => mc-3113-big-Digest.crc32} (100%) rename test/data/legacy-sstables/mc/legacy_tables/{legacy_mc_simple/mc-1-big-Filter.db => legacy_mc_clust_counter/mc-3113-big-Filter.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/{mc-1-big-Index.db => mc-3113-big-Index.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/{mc-1-big-Statistics.db => mc-3113-big-Statistics.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/{legacy_mc_simple/mc-1-big-Summary.db => legacy_mc_clust_counter/mc-3113-big-Summary.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/{mc-1-big-TOC.txt => mc-3113-big-TOC.txt} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/{mc-1-big-CompressionInfo.db => mc-3113-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/{mc-1-big-Data.db => mc-3113-big-Data.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/{mc-1-big-Digest.crc32 => mc-3113-big-Digest.crc32} (100%) rename test/data/legacy-sstables/mc/legacy_tables/{legacy_mc_simple_compact/mc-1-big-Filter.db => legacy_mc_clust_counter_compact/mc-3113-big-Filter.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/{mc-1-big-Index.db => mc-3113-big-Index.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/{mc-1-big-Statistics.db => mc-3113-big-Statistics.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/{legacy_mc_simple_compact/mc-1-big-Summary.db => legacy_mc_clust_counter_compact/mc-3113-big-Summary.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/{mc-1-big-TOC.txt => mc-3113-big-TOC.txt} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/{mc-1-big-CompressionInfo.db => mc-3113-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/{mc-1-big-Data.db => mc-3113-big-Data.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/{mc-1-big-Digest.crc32 => mc-3113-big-Digest.crc32} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/{mc-1-big-Filter.db => mc-3113-big-Filter.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/{mc-1-big-Index.db => mc-3113-big-Index.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/{mc-1-big-Statistics.db => mc-3113-big-Statistics.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/{mc-1-big-Summary.db => mc-3113-big-Summary.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/{mc-1-big-TOC.txt => mc-3113-big-TOC.txt} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/{mc-1-big-CompressionInfo.db => mc-3113-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/{mc-1-big-Data.db => mc-3113-big-Data.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/{mc-1-big-Digest.crc32 => mc-3113-big-Digest.crc32} (100%) rename test/data/legacy-sstables/mc/legacy_tables/{legacy_mc_simple_counter/mc-1-big-Filter.db => legacy_mc_simple/mc-3113-big-Filter.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/{mc-1-big-Index.db => mc-3113-big-Index.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/{mc-1-big-Statistics.db => mc-3113-big-Statistics.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/{legacy_mc_simple_counter/mc-1-big-Summary.db => legacy_mc_simple/mc-3113-big-Summary.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/{mc-1-big-TOC.txt => mc-3113-big-TOC.txt} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/{mc-1-big-CompressionInfo.db => mc-3113-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/{mc-1-big-Data.db => mc-3113-big-Data.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/{mc-1-big-Digest.crc32 => mc-3113-big-Digest.crc32} (100%) rename test/data/legacy-sstables/mc/legacy_tables/{legacy_mc_simple_counter_compact/mc-1-big-Filter.db => legacy_mc_simple_compact/mc-3113-big-Filter.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/{mc-1-big-Index.db => mc-3113-big-Index.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/{mc-1-big-Statistics.db => mc-3113-big-Statistics.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/{legacy_mc_simple_counter_compact/mc-1-big-Summary.db => legacy_mc_simple_compact/mc-3113-big-Summary.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/{mc-1-big-TOC.txt => mc-3113-big-TOC.txt} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/{mc-1-big-CompressionInfo.db => mc-3113-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/{mc-1-big-Data.db => mc-3113-big-Data.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/{mc-1-big-Digest.crc32 => mc-3113-big-Digest.crc32} (100%) rename test/data/legacy-sstables/{md/legacy_tables/legacy_md_clust/md-1-big-Filter.db => mc/legacy_tables/legacy_mc_simple_counter/mc-3113-big-Filter.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/{mc-1-big-Index.db => mc-3113-big-Index.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/{mc-1-big-Statistics.db => mc-3113-big-Statistics.db} (100%) rename test/data/legacy-sstables/{md/legacy_tables/legacy_md_clust/md-1-big-Summary.db => mc/legacy_tables/legacy_mc_simple_counter/mc-3113-big-Summary.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/{mc-1-big-TOC.txt => mc-3113-big-TOC.txt} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/{mc-1-big-CompressionInfo.db => mc-3113-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/{mc-1-big-Data.db => mc-3113-big-Data.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/{mc-1-big-Digest.crc32 => mc-3113-big-Digest.crc32} (100%) rename test/data/legacy-sstables/{md/legacy_tables/legacy_md_clust_compact/md-1-big-Filter.db => mc/legacy_tables/legacy_mc_simple_counter_compact/mc-3113-big-Filter.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/{mc-1-big-Index.db => mc-3113-big-Index.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/{mc-1-big-Statistics.db => mc-3113-big-Statistics.db} (100%) rename test/data/legacy-sstables/{md/legacy_tables/legacy_md_clust_compact/md-1-big-Summary.db => mc/legacy_tables/legacy_mc_simple_counter_compact/mc-3113-big-Summary.db} (100%) rename test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/{mc-1-big-TOC.txt => mc-3113-big-TOC.txt} (100%) create mode 100644 test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-CompressionInfo.db create mode 100644 test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-Data.db create mode 100644 test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-Digest.crc32 rename test/data/legacy-sstables/{md/legacy_tables/legacy_md_clust_counter/md-1-big-Filter.db => mc/legacy_tables/legacy_mc_tuple/mc-3113-big-Filter.db} (100%) create mode 100644 test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-Index.db create mode 100644 test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-Statistics.db rename test/data/legacy-sstables/{md/legacy_tables/legacy_md_clust_counter/md-1-big-Summary.db => mc/legacy_tables/legacy_mc_tuple/mc-3113-big-Summary.db} (100%) rename test/data/legacy-sstables/{nc/legacy_tables/legacy_nc_simple/nc-1-big-TOC.txt => mc/legacy_tables/legacy_mc_tuple/mc-3113-big-TOC.txt} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/{md-1-big-CompressionInfo.db => md-31110-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/{md-1-big-Data.db => md-31110-big-Data.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/{md-1-big-Digest.crc32 => md-31110-big-Digest.crc32} (100%) rename test/data/legacy-sstables/md/legacy_tables/{legacy_md_clust_counter_compact/md-1-big-Filter.db => legacy_md_clust/md-31110-big-Filter.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/{md-1-big-Index.db => md-31110-big-Index.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/{md-1-big-Statistics.db => md-31110-big-Statistics.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/{legacy_md_clust_counter_compact/md-1-big-Summary.db => legacy_md_clust/md-31110-big-Summary.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/{md-1-big-TOC.txt => md-31110-big-TOC.txt} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/{md-1-big-CompressionInfo.db => md-31110-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/{md-1-big-Data.db => md-31110-big-Data.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/{md-1-big-Digest.crc32 => md-31110-big-Digest.crc32} (100%) rename test/data/legacy-sstables/md/legacy_tables/{legacy_md_simple/md-1-big-Filter.db => legacy_md_clust_compact/md-31110-big-Filter.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/{md-1-big-Index.db => md-31110-big-Index.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/{md-1-big-Statistics.db => md-31110-big-Statistics.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/{legacy_md_simple/md-1-big-Summary.db => legacy_md_clust_compact/md-31110-big-Summary.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/{md-1-big-TOC.txt => md-31110-big-TOC.txt} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/{md-1-big-CompressionInfo.db => md-31110-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/{md-1-big-Data.db => md-31110-big-Data.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/{md-1-big-Digest.crc32 => md-31110-big-Digest.crc32} (100%) rename test/data/legacy-sstables/md/legacy_tables/{legacy_md_simple_compact/md-1-big-Filter.db => legacy_md_clust_counter/md-31110-big-Filter.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/{md-1-big-Index.db => md-31110-big-Index.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/{md-1-big-Statistics.db => md-31110-big-Statistics.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/{legacy_md_simple_compact/md-1-big-Summary.db => legacy_md_clust_counter/md-31110-big-Summary.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/{md-1-big-TOC.txt => md-31110-big-TOC.txt} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/{md-1-big-CompressionInfo.db => md-31110-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/{md-1-big-Data.db => md-31110-big-Data.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/{md-1-big-Digest.crc32 => md-31110-big-Digest.crc32} (100%) rename test/data/legacy-sstables/md/legacy_tables/{legacy_md_simple_counter/md-1-big-Filter.db => legacy_md_clust_counter_compact/md-31110-big-Filter.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/{md-1-big-Index.db => md-31110-big-Index.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/{md-1-big-Statistics.db => md-31110-big-Statistics.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/{legacy_md_simple_counter/md-1-big-Summary.db => legacy_md_clust_counter_compact/md-31110-big-Summary.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/{md-1-big-TOC.txt => md-31110-big-TOC.txt} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/{md-1-big-CompressionInfo.db => md-31110-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/{md-1-big-Data.db => md-31110-big-Data.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/{md-1-big-Digest.crc32 => md-31110-big-Digest.crc32} (100%) rename test/data/legacy-sstables/md/legacy_tables/{legacy_md_simple_counter_compact/md-1-big-Filter.db => legacy_md_simple/md-31110-big-Filter.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/{md-1-big-Index.db => md-31110-big-Index.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/{md-1-big-Statistics.db => md-31110-big-Statistics.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/{legacy_md_simple_counter_compact/md-1-big-Summary.db => legacy_md_simple/md-31110-big-Summary.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/{md-1-big-TOC.txt => md-31110-big-TOC.txt} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/{md-1-big-CompressionInfo.db => md-31110-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/{md-1-big-Data.db => md-31110-big-Data.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/{md-1-big-Digest.crc32 => md-31110-big-Digest.crc32} (100%) rename test/data/legacy-sstables/{me/legacy_tables/legacy_me_clust/me-1-big-Filter.db => md/legacy_tables/legacy_md_simple_compact/md-31110-big-Filter.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/{md-1-big-Index.db => md-31110-big-Index.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/{md-1-big-Statistics.db => md-31110-big-Statistics.db} (100%) rename test/data/legacy-sstables/{me/legacy_tables/legacy_me_clust/me-1-big-Summary.db => md/legacy_tables/legacy_md_simple_compact/md-31110-big-Summary.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/{md-1-big-TOC.txt => md-31110-big-TOC.txt} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/{md-1-big-CompressionInfo.db => md-31110-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/{md-1-big-Data.db => md-31110-big-Data.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/{md-1-big-Digest.crc32 => md-31110-big-Digest.crc32} (100%) rename test/data/legacy-sstables/{me/legacy_tables/legacy_me_clust_compact/me-1-big-Filter.db => md/legacy_tables/legacy_md_simple_counter/md-31110-big-Filter.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/{md-1-big-Index.db => md-31110-big-Index.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/{md-1-big-Statistics.db => md-31110-big-Statistics.db} (100%) rename test/data/legacy-sstables/{me/legacy_tables/legacy_me_clust_compact/me-1-big-Summary.db => md/legacy_tables/legacy_md_simple_counter/md-31110-big-Summary.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/{md-1-big-TOC.txt => md-31110-big-TOC.txt} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/{md-1-big-CompressionInfo.db => md-31110-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/{md-1-big-Data.db => md-31110-big-Data.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/{md-1-big-Digest.crc32 => md-31110-big-Digest.crc32} (100%) rename test/data/legacy-sstables/{me/legacy_tables/legacy_me_clust_counter/me-1-big-Filter.db => md/legacy_tables/legacy_md_simple_counter_compact/md-31110-big-Filter.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/{md-1-big-Index.db => md-31110-big-Index.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/{md-1-big-Statistics.db => md-31110-big-Statistics.db} (100%) rename test/data/legacy-sstables/{me/legacy_tables/legacy_me_clust_counter/me-1-big-Summary.db => md/legacy_tables/legacy_md_simple_counter_compact/md-31110-big-Summary.db} (100%) rename test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/{md-1-big-TOC.txt => md-31110-big-TOC.txt} (100%) create mode 100644 test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-CompressionInfo.db create mode 100644 test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-Data.db create mode 100644 test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-Digest.crc32 rename test/data/legacy-sstables/{me/legacy_tables/legacy_me_clust_counter_compact/me-1-big-Filter.db => md/legacy_tables/legacy_md_tuple/md-31110-big-Filter.db} (100%) create mode 100644 test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-Index.db create mode 100644 test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-Statistics.db rename test/data/legacy-sstables/{me/legacy_tables/legacy_me_clust_counter_compact/me-1-big-Summary.db => md/legacy_tables/legacy_md_tuple/md-31110-big-Summary.db} (100%) rename test/data/legacy-sstables/{nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-TOC.txt => md/legacy_tables/legacy_md_tuple/md-31110-big-TOC.txt} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-CompressionInfo.db create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-Data.db create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-Digest.crc32 rename test/data/legacy-sstables/me/legacy_tables/{legacy_me_simple/me-1-big-Filter.db => legacy_me_clust/me-3025-big-Filter.db} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-Index.db rename test/data/legacy-sstables/{nc/legacy_tables/legacy_nc_clust/nc-1-big-Statistics.db => me/legacy_tables/legacy_me_clust/me-3025-big-Statistics.db} (64%) rename test/data/legacy-sstables/me/legacy_tables/{legacy_me_simple/me-1-big-Summary.db => legacy_me_clust/me-3025-big-Summary.db} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-TOC.txt rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/{me-1-big-CompressionInfo.db => me-31111-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/{me-1-big-Data.db => me-31111-big-Data.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/{me-1-big-Digest.crc32 => me-31111-big-Digest.crc32} (100%) rename test/data/legacy-sstables/me/legacy_tables/{legacy_me_simple_compact/me-1-big-Filter.db => legacy_me_clust/me-31111-big-Filter.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/{me-1-big-Index.db => me-31111-big-Index.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/{me-1-big-Statistics.db => me-31111-big-Statistics.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/{legacy_me_simple_compact/me-1-big-Summary.db => legacy_me_clust/me-31111-big-Summary.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/{me-1-big-TOC.txt => me-31111-big-TOC.txt} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-CompressionInfo.db create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-Data.db create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-Digest.crc32 rename test/data/legacy-sstables/me/legacy_tables/{legacy_me_simple_counter/me-1-big-Filter.db => legacy_me_clust_compact/me-3025-big-Filter.db} (100%) rename test/data/legacy-sstables/{nc/legacy_tables/legacy_nc_clust/nc-1-big-Index.db => me/legacy_tables/legacy_me_clust_compact/me-3025-big-Index.db} (99%) rename test/data/legacy-sstables/{nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-Statistics.db => me/legacy_tables/legacy_me_clust_compact/me-3025-big-Statistics.db} (64%) rename test/data/legacy-sstables/me/legacy_tables/{legacy_me_simple_counter/me-1-big-Summary.db => legacy_me_clust_compact/me-3025-big-Summary.db} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-TOC.txt rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/{me-1-big-CompressionInfo.db => me-31111-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/{me-1-big-Data.db => me-31111-big-Data.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/{me-1-big-Digest.crc32 => me-31111-big-Digest.crc32} (100%) rename test/data/legacy-sstables/me/legacy_tables/{legacy_me_simple_counter_compact/me-1-big-Filter.db => legacy_me_clust_compact/me-31111-big-Filter.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/{me-1-big-Index.db => me-31111-big-Index.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/{me-1-big-Statistics.db => me-31111-big-Statistics.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/{legacy_me_simple_counter_compact/me-1-big-Summary.db => legacy_me_clust_compact/me-31111-big-Summary.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/{me-1-big-TOC.txt => me-31111-big-TOC.txt} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/{me-1-big-CompressionInfo.db => me-31111-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/{me-1-big-Data.db => me-31111-big-Data.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/{me-1-big-Digest.crc32 => me-31111-big-Digest.crc32} (100%) rename test/data/legacy-sstables/{na/legacy_tables/legacy_na_clust_compact/na-1-big-Filter.db => me/legacy_tables/legacy_me_clust_counter/me-31111-big-Filter.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/{me-1-big-Index.db => me-31111-big-Index.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/{me-1-big-Statistics.db => me-31111-big-Statistics.db} (100%) rename test/data/legacy-sstables/{na/legacy_tables/legacy_na_clust/na-1-big-Summary.db => me/legacy_tables/legacy_me_clust_counter/me-31111-big-Summary.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/{me-1-big-TOC.txt => me-31111-big-TOC.txt} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-CompressionInfo.db create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-Data.db create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-Digest.crc32 rename test/data/legacy-sstables/{na/legacy_tables/legacy_na_clust_counter_compact/na-1-big-Filter.db => me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-Filter.db} (100%) rename test/data/legacy-sstables/{nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-Index.db => me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-Index.db} (99%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-Statistics.db rename test/data/legacy-sstables/{na/legacy_tables/legacy_na_clust_compact/na-1-big-Summary.db => me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-Summary.db} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-TOC.txt rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/{me-1-big-CompressionInfo.db => me-31111-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/{me-1-big-Data.db => me-31111-big-Data.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/{me-1-big-Digest.crc32 => me-31111-big-Digest.crc32} (100%) rename test/data/legacy-sstables/{na/legacy_tables/legacy_na_simple_compact/na-1-big-Filter.db => me/legacy_tables/legacy_me_clust_counter_compact/me-31111-big-Filter.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/{me-1-big-Index.db => me-31111-big-Index.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/{me-1-big-Statistics.db => me-31111-big-Statistics.db} (100%) rename test/data/legacy-sstables/{na/legacy_tables/legacy_na_clust_counter/na-1-big-Summary.db => me/legacy_tables/legacy_me_clust_counter_compact/me-31111-big-Summary.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/{me-1-big-TOC.txt => me-31111-big-TOC.txt} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/{me-1-big-CompressionInfo.db => me-3025-big-CompressionInfo.db} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-Data.db create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-Digest.crc32 rename test/data/legacy-sstables/{na/legacy_tables/legacy_na_simple_counter_compact/na-1-big-Filter.db => me/legacy_tables/legacy_me_simple/me-3025-big-Filter.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/{me-1-big-Index.db => me-3025-big-Index.db} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-Statistics.db rename test/data/legacy-sstables/{na/legacy_tables/legacy_na_clust_counter_compact/na-1-big-Summary.db => me/legacy_tables/legacy_me_simple/me-3025-big-Summary.db} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-TOC.txt create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-31111-big-CompressionInfo.db rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/{me-1-big-Data.db => me-31111-big-Data.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/{me-1-big-Digest.crc32 => me-31111-big-Digest.crc32} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-31111-big-Filter.db rename test/data/legacy-sstables/{na/legacy_tables/legacy_na_simple/na-1-big-Index.db => me/legacy_tables/legacy_me_simple/me-31111-big-Index.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/{me-1-big-Statistics.db => me-31111-big-Statistics.db} (100%) rename test/data/legacy-sstables/{na/legacy_tables/legacy_na_simple/na-1-big-Summary.db => me/legacy_tables/legacy_me_simple/me-31111-big-Summary.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/{me-1-big-TOC.txt => me-31111-big-TOC.txt} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/{me-1-big-CompressionInfo.db => me-3025-big-CompressionInfo.db} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-Data.db create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-Digest.crc32 create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-Filter.db rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/{me-1-big-Index.db => me-3025-big-Index.db} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-Statistics.db rename test/data/legacy-sstables/{na/legacy_tables/legacy_na_simple_compact/na-1-big-Summary.db => me/legacy_tables/legacy_me_simple_compact/me-3025-big-Summary.db} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-TOC.txt create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-31111-big-CompressionInfo.db rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/{me-1-big-Data.db => me-31111-big-Data.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/{me-1-big-Digest.crc32 => me-31111-big-Digest.crc32} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-31111-big-Filter.db rename test/data/legacy-sstables/{na/legacy_tables/legacy_na_simple_compact/na-1-big-Index.db => me/legacy_tables/legacy_me_simple_compact/me-31111-big-Index.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/{me-1-big-Statistics.db => me-31111-big-Statistics.db} (100%) rename test/data/legacy-sstables/{na/legacy_tables/legacy_na_simple_counter/na-1-big-Summary.db => me/legacy_tables/legacy_me_simple_compact/me-31111-big-Summary.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/{me-1-big-TOC.txt => me-31111-big-TOC.txt} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/{me-1-big-CompressionInfo.db => me-31111-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/{me-1-big-Data.db => me-31111-big-Data.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/{me-1-big-Digest.crc32 => me-31111-big-Digest.crc32} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-31111-big-Filter.db rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/{me-1-big-Index.db => me-31111-big-Index.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/{me-1-big-Statistics.db => me-31111-big-Statistics.db} (100%) rename test/data/legacy-sstables/{na/legacy_tables/legacy_na_simple_counter_compact/na-1-big-Summary.db => me/legacy_tables/legacy_me_simple_counter/me-31111-big-Summary.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/{me-1-big-TOC.txt => me-31111-big-TOC.txt} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/{me-1-big-CompressionInfo.db => me-3025-big-CompressionInfo.db} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-Data.db create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-Digest.crc32 create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-Filter.db rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/{me-1-big-Index.db => me-3025-big-Index.db} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-Statistics.db rename test/data/legacy-sstables/{nb/legacy_tables/legacy_nb_clust/nb-1-big-Summary.db => me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-Summary.db} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-TOC.txt create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-31111-big-CompressionInfo.db rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/{me-1-big-Data.db => me-31111-big-Data.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/{me-1-big-Digest.crc32 => me-31111-big-Digest.crc32} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-31111-big-Filter.db rename test/data/legacy-sstables/{na/legacy_tables/legacy_na_simple_counter_compact/na-1-big-Index.db => me/legacy_tables/legacy_me_simple_counter_compact/me-31111-big-Index.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/{me-1-big-Statistics.db => me-31111-big-Statistics.db} (100%) rename test/data/legacy-sstables/{nb/legacy_tables/legacy_nb_clust_counter/nb-1-big-Summary.db => me/legacy_tables/legacy_me_simple_counter_compact/me-31111-big-Summary.db} (100%) rename test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/{me-1-big-TOC.txt => me-31111-big-TOC.txt} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-CompressionInfo.db create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-Data.db create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-Digest.crc32 create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-Filter.db create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-Index.db create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-Statistics.db rename test/data/legacy-sstables/{nb/legacy_tables/legacy_nb_simple/nb-1-big-Summary.db => me/legacy_tables/legacy_me_tuple/me-31111-big-Summary.db} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-TOC.txt create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-CompressionInfo.db create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-Data.db create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-Digest.crc32 create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-Filter.db create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-Index.db create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-Statistics.db rename test/data/legacy-sstables/{nb/legacy_tables/legacy_nb_simple_counter/nb-1-big-Summary.db => me/legacy_tables/legacy_me_tuple_compact/me-31111-big-Summary.db} (100%) create mode 100644 test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-TOC.txt rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/{na-1-big-CompressionInfo.db => na-400-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/{na-1-big-Data.db => na-400-big-Data.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/{na-1-big-Digest.crc32 => na-400-big-Digest.crc32} (100%) rename test/data/legacy-sstables/na/legacy_tables/{legacy_na_clust_counter/na-1-big-Filter.db => legacy_na_clust/na-400-big-Filter.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/{na-1-big-Index.db => na-400-big-Index.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/{na-1-big-Statistics.db => na-400-big-Statistics.db} (100%) rename test/data/legacy-sstables/{nc/legacy_tables/legacy_nc_clust/nc-1-big-Summary.db => na/legacy_tables/legacy_na_clust/na-400-big-Summary.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/{na-1-big-TOC.txt => na-400-big-TOC.txt} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/{na-1-big-CompressionInfo.db => na-400-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/{na-1-big-Data.db => na-400-big-Data.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/{na-1-big-Digest.crc32 => na-400-big-Digest.crc32} (100%) create mode 100644 test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-400-big-Filter.db rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/{na-1-big-Index.db => na-400-big-Index.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/{na-1-big-Statistics.db => na-400-big-Statistics.db} (100%) rename test/data/legacy-sstables/{nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-Summary.db => na/legacy_tables/legacy_na_clust_compact/na-400-big-Summary.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/{na-1-big-TOC.txt => na-400-big-TOC.txt} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/{na-1-big-CompressionInfo.db => na-400-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/{na-1-big-Data.db => na-400-big-Data.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/{na-1-big-Digest.crc32 => na-400-big-Digest.crc32} (100%) rename test/data/legacy-sstables/na/legacy_tables/{legacy_na_simple/na-1-big-Filter.db => legacy_na_clust_counter/na-400-big-Filter.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/{na-1-big-Index.db => na-400-big-Index.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/{na-1-big-Statistics.db => na-400-big-Statistics.db} (100%) rename test/data/legacy-sstables/{nc/legacy_tables/legacy_nc_simple/nc-1-big-Summary.db => na/legacy_tables/legacy_na_clust_counter/na-400-big-Summary.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/{na-1-big-TOC.txt => na-400-big-TOC.txt} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/{na-1-big-CompressionInfo.db => na-400-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/{na-1-big-Data.db => na-400-big-Data.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/{na-1-big-Digest.crc32 => na-400-big-Digest.crc32} (100%) create mode 100644 test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-400-big-Filter.db rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/{na-1-big-Index.db => na-400-big-Index.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/{na-1-big-Statistics.db => na-400-big-Statistics.db} (100%) rename test/data/legacy-sstables/{nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-Summary.db => na/legacy_tables/legacy_na_clust_counter_compact/na-400-big-Summary.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/{na-1-big-TOC.txt => na-400-big-TOC.txt} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/{na-1-big-CompressionInfo.db => na-400-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/{na-1-big-Data.db => na-400-big-Data.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/{na-1-big-Digest.crc32 => na-400-big-Digest.crc32} (100%) rename test/data/legacy-sstables/na/legacy_tables/{legacy_na_simple_counter/na-1-big-Filter.db => legacy_na_simple/na-400-big-Filter.db} (100%) rename test/data/legacy-sstables/{nb/legacy_tables/legacy_nb_simple/nb-1-big-Index.db => na/legacy_tables/legacy_na_simple/na-400-big-Index.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/{na-1-big-Statistics.db => na-400-big-Statistics.db} (100%) rename test/data/legacy-sstables/{oa/legacy_tables/legacy_oa_clust/oa-1-big-Summary.db => na/legacy_tables/legacy_na_simple/na-400-big-Summary.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/{na-1-big-TOC.txt => na-400-big-TOC.txt} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/{na-1-big-CompressionInfo.db => na-400-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/{na-1-big-Data.db => na-400-big-Data.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/{na-1-big-Digest.crc32 => na-400-big-Digest.crc32} (100%) create mode 100644 test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-400-big-Filter.db create mode 100644 test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-400-big-Index.db rename test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/{na-1-big-Statistics.db => na-400-big-Statistics.db} (100%) rename test/data/legacy-sstables/{oa/legacy_tables/legacy_oa_clust_counter/oa-1-big-Summary.db => na/legacy_tables/legacy_na_simple_compact/na-400-big-Summary.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/{na-1-big-TOC.txt => na-400-big-TOC.txt} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/{na-1-big-CompressionInfo.db => na-400-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/{na-1-big-Data.db => na-400-big-Data.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/{na-1-big-Digest.crc32 => na-400-big-Digest.crc32} (100%) rename test/data/legacy-sstables/{nb/legacy_tables/legacy_nb_clust/nb-1-big-Filter.db => na/legacy_tables/legacy_na_simple_counter/na-400-big-Filter.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/{na-1-big-Index.db => na-400-big-Index.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/{na-1-big-Statistics.db => na-400-big-Statistics.db} (100%) rename test/data/legacy-sstables/{oa/legacy_tables/legacy_oa_simple/oa-1-big-Summary.db => na/legacy_tables/legacy_na_simple_counter/na-400-big-Summary.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/{na-1-big-TOC.txt => na-400-big-TOC.txt} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/{na-1-big-CompressionInfo.db => na-400-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/{na-1-big-Data.db => na-400-big-Data.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/{na-1-big-Digest.crc32 => na-400-big-Digest.crc32} (100%) create mode 100644 test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-400-big-Filter.db create mode 100644 test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-400-big-Index.db rename test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/{na-1-big-Statistics.db => na-400-big-Statistics.db} (100%) rename test/data/legacy-sstables/{oa/legacy_tables/legacy_oa_simple_counter/oa-1-big-Summary.db => na/legacy_tables/legacy_na_simple_counter_compact/na-400-big-Summary.db} (100%) rename test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/{na-1-big-TOC.txt => na-400-big-TOC.txt} (100%) create mode 100644 test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-CompressionInfo.db create mode 100644 test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-Data.db create mode 100644 test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-Digest.crc32 rename test/data/legacy-sstables/{nb/legacy_tables/legacy_nb_clust_counter/nb-1-big-Filter.db => na/legacy_tables/legacy_na_tuple/na-400-big-Filter.db} (100%) create mode 100644 test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-Index.db create mode 100644 test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-Statistics.db create mode 100644 test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-Summary.db create mode 100644 test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-TOC.txt rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/{nb-1-big-CompressionInfo.db => nb-400-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/{nb-1-big-Data.db => nb-400-big-Data.db} (100%) rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/{nb-1-big-Digest.crc32 => nb-400-big-Digest.crc32} (100%) rename test/data/legacy-sstables/nb/legacy_tables/{legacy_nb_simple/nb-1-big-Filter.db => legacy_nb_clust/nb-400-big-Filter.db} (100%) rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/{nb-1-big-Index.db => nb-400-big-Index.db} (100%) rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/{nb-1-big-Statistics.db => nb-400-big-Statistics.db} (100%) create mode 100644 test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-400-big-Summary.db rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/{nb-1-big-TOC.txt => nb-400-big-TOC.txt} (100%) rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/{nb-1-big-CompressionInfo.db => nb-400-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/{nb-1-big-Data.db => nb-400-big-Data.db} (100%) rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/{nb-1-big-Digest.crc32 => nb-400-big-Digest.crc32} (100%) rename test/data/legacy-sstables/nb/legacy_tables/{legacy_nb_simple_counter/nb-1-big-Filter.db => legacy_nb_clust_counter/nb-400-big-Filter.db} (100%) rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/{nb-1-big-Index.db => nb-400-big-Index.db} (100%) rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/{nb-1-big-Statistics.db => nb-400-big-Statistics.db} (100%) create mode 100644 test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-400-big-Summary.db rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/{nb-1-big-TOC.txt => nb-400-big-TOC.txt} (100%) rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/{nb-1-big-CompressionInfo.db => nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-CompressionInfo.db} (100%) create mode 100644 test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-Data.db create mode 100644 test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-Digest.crc32 rename test/data/legacy-sstables/{nc/legacy_tables/legacy_nc_clust/nc-1-big-Filter.db => nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-Filter.db} (100%) rename test/data/legacy-sstables/{nc/legacy_tables/legacy_nc_simple/nc-1-big-Index.db => nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-Index.db} (100%) rename test/data/legacy-sstables/{nc/legacy_tables/legacy_nc_simple/nc-1-big-Statistics.db => nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-Statistics.db} (84%) create mode 100644 test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-Summary.db create mode 100644 test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-TOC.txt rename test/data/legacy-sstables/{nc/legacy_tables/legacy_nc_simple/nc-1-big-CompressionInfo.db => nb/legacy_tables/legacy_nb_simple/nb-400-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/{nb-1-big-Data.db => nb-400-big-Data.db} (100%) rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/{nb-1-big-Digest.crc32 => nb-400-big-Digest.crc32} (100%) rename test/data/legacy-sstables/{nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-Filter.db => nb/legacy_tables/legacy_nb_simple/nb-400-big-Filter.db} (100%) create mode 100644 test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-400-big-Index.db rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/{nb-1-big-Statistics.db => nb-400-big-Statistics.db} (100%) create mode 100644 test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-400-big-Summary.db rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/{nb-1-big-TOC.txt => nb-400-big-TOC.txt} (100%) rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/{nb-1-big-CompressionInfo.db => nb-400-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/{nb-1-big-Data.db => nb-400-big-Data.db} (100%) rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/{nb-1-big-Digest.crc32 => nb-400-big-Digest.crc32} (100%) rename test/data/legacy-sstables/{nc/legacy_tables/legacy_nc_simple/nc-1-big-Filter.db => nb/legacy_tables/legacy_nb_simple_counter/nb-400-big-Filter.db} (100%) rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/{nb-1-big-Index.db => nb-400-big-Index.db} (100%) rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/{nb-1-big-Statistics.db => nb-400-big-Statistics.db} (100%) create mode 100644 test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-400-big-Summary.db rename test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/{nb-1-big-TOC.txt => nb-400-big-TOC.txt} (100%) create mode 100644 test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-CompressionInfo.db create mode 100644 test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-Data.db create mode 100644 test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-Digest.crc32 rename test/data/legacy-sstables/{nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-Filter.db => nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-Filter.db} (100%) create mode 100644 test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-Index.db rename test/data/legacy-sstables/{nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-Statistics.db => nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-Statistics.db} (81%) create mode 100644 test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-Summary.db create mode 100644 test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-TOC.txt create mode 100644 test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-CompressionInfo.db create mode 100644 test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-Data.db create mode 100644 test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-Digest.crc32 rename test/data/legacy-sstables/{oa/legacy_tables/legacy_oa_clust/oa-1-big-Filter.db => nb/legacy_tables/legacy_nb_tuple/nb-400-big-Filter.db} (100%) create mode 100644 test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-Index.db create mode 100644 test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-Statistics.db create mode 100644 test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-Summary.db rename test/data/legacy-sstables/{oa/legacy_tables/legacy_oa_clust/oa-1-big-TOC.txt => nb/legacy_tables/legacy_nb_tuple/nb-400-big-TOC.txt} (100%) delete mode 100644 test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust/nc-1-big-CompressionInfo.db delete mode 100644 test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust/nc-1-big-Data.db delete mode 100644 test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust/nc-1-big-Digest.crc32 delete mode 100644 test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-CompressionInfo.db delete mode 100644 test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-Data.db delete mode 100644 test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-Digest.crc32 delete mode 100644 test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple/nc-1-big-Data.db delete mode 100644 test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple/nc-1-big-Digest.crc32 delete mode 100644 test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-CompressionInfo.db delete mode 100644 test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-Data.db delete mode 100644 test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-Digest.crc32 delete mode 100644 test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-Index.db rename test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/{oa-1-big-CompressionInfo.db => oa-500-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/{oa-1-big-Data.db => oa-500-big-Data.db} (100%) rename test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/{oa-1-big-Digest.crc32 => oa-500-big-Digest.crc32} (100%) rename test/data/legacy-sstables/oa/legacy_tables/{legacy_oa_clust_counter/oa-1-big-Filter.db => legacy_oa_clust/oa-500-big-Filter.db} (100%) rename test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/{oa-1-big-Index.db => oa-500-big-Index.db} (100%) rename test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/{oa-1-big-Statistics.db => oa-500-big-Statistics.db} (100%) create mode 100644 test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-500-big-Summary.db rename test/data/legacy-sstables/oa/legacy_tables/{legacy_oa_clust_counter/oa-1-big-TOC.txt => legacy_oa_clust/oa-500-big-TOC.txt} (100%) rename test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/{oa-1-big-CompressionInfo.db => oa-500-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/{oa-1-big-Data.db => oa-500-big-Data.db} (100%) rename test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/{oa-1-big-Digest.crc32 => oa-500-big-Digest.crc32} (100%) rename test/data/legacy-sstables/oa/legacy_tables/{legacy_oa_simple/oa-1-big-Filter.db => legacy_oa_clust_counter/oa-500-big-Filter.db} (100%) rename test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/{oa-1-big-Index.db => oa-500-big-Index.db} (100%) rename test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/{oa-1-big-Statistics.db => oa-500-big-Statistics.db} (100%) create mode 100644 test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-500-big-Summary.db rename test/data/legacy-sstables/oa/legacy_tables/{legacy_oa_simple/oa-1-big-TOC.txt => legacy_oa_clust_counter/oa-500-big-TOC.txt} (100%) rename test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/{oa-1-big-CompressionInfo.db => oa-500-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/{oa-1-big-Data.db => oa-500-big-Data.db} (100%) rename test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/{oa-1-big-Digest.crc32 => oa-500-big-Digest.crc32} (100%) rename test/data/legacy-sstables/oa/legacy_tables/{legacy_oa_simple_counter/oa-1-big-Filter.db => legacy_oa_simple/oa-500-big-Filter.db} (100%) rename test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/{oa-1-big-Index.db => oa-500-big-Index.db} (100%) rename test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/{oa-1-big-Statistics.db => oa-500-big-Statistics.db} (100%) create mode 100644 test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-500-big-Summary.db rename test/data/legacy-sstables/oa/legacy_tables/{legacy_oa_simple_counter/oa-1-big-TOC.txt => legacy_oa_simple/oa-500-big-TOC.txt} (100%) rename test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/{oa-1-big-CompressionInfo.db => oa-500-big-CompressionInfo.db} (100%) rename test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/{oa-1-big-Data.db => oa-500-big-Data.db} (100%) rename test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/{oa-1-big-Digest.crc32 => oa-500-big-Digest.crc32} (100%) create mode 100644 test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-500-big-Filter.db rename test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/{oa-1-big-Index.db => oa-500-big-Index.db} (100%) rename test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/{oa-1-big-Statistics.db => oa-500-big-Statistics.db} (100%) create mode 100644 test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-500-big-Summary.db create mode 100644 test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-500-big-TOC.txt create mode 100644 test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-CompressionInfo.db create mode 100644 test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-Data.db create mode 100644 test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-Digest.crc32 create mode 100644 test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-Filter.db create mode 100644 test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-Index.db create mode 100644 test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-Statistics.db create mode 100644 test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-Summary.db create mode 100644 test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-TOC.txt diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-1-bti-CompressionInfo.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-500-bti-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-1-bti-CompressionInfo.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-500-bti-CompressionInfo.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-1-bti-Data.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-500-bti-Data.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-1-bti-Data.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-500-bti-Data.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-1-bti-Digest.crc32 b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-500-bti-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-1-bti-Digest.crc32 rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-500-bti-Digest.crc32 diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-1-bti-Filter.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-500-bti-Filter.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-1-bti-Filter.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-500-bti-Filter.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-1-bti-Partitions.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-500-bti-Partitions.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-1-bti-Partitions.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-500-bti-Partitions.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-1-bti-Rows.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-500-bti-Rows.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-1-bti-Rows.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-500-bti-Rows.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-1-bti-Statistics.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-500-bti-Statistics.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-1-bti-Statistics.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-500-bti-Statistics.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-1-bti-TOC.txt b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-500-bti-TOC.txt similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-1-bti-TOC.txt rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_clust/da-500-bti-TOC.txt diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-1-bti-CompressionInfo.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-500-bti-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-1-bti-CompressionInfo.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-500-bti-CompressionInfo.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-1-bti-Data.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-500-bti-Data.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-1-bti-Data.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-500-bti-Data.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-1-bti-Digest.crc32 b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-500-bti-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-1-bti-Digest.crc32 rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-500-bti-Digest.crc32 diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-1-bti-Filter.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-500-bti-Filter.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-1-bti-Filter.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-500-bti-Filter.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-1-bti-Partitions.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-500-bti-Partitions.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-1-bti-Partitions.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-500-bti-Partitions.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-1-bti-Rows.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-500-bti-Rows.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-1-bti-Rows.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-500-bti-Rows.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-1-bti-Statistics.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-500-bti-Statistics.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-1-bti-Statistics.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-500-bti-Statistics.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-1-bti-TOC.txt b/test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-500-bti-TOC.txt similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-1-bti-TOC.txt rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_clust_counter/da-500-bti-TOC.txt diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-1-bti-CompressionInfo.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-500-bti-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-1-bti-CompressionInfo.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-500-bti-CompressionInfo.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-1-bti-Data.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-500-bti-Data.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-1-bti-Data.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-500-bti-Data.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-1-bti-Digest.crc32 b/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-500-bti-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-1-bti-Digest.crc32 rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-500-bti-Digest.crc32 diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-1-bti-Filter.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-500-bti-Filter.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-1-bti-Filter.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-500-bti-Filter.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-1-bti-Partitions.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-500-bti-Partitions.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-1-bti-Partitions.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-500-bti-Partitions.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-1-bti-Rows.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-500-bti-Rows.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-1-bti-Rows.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-500-bti-Rows.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-1-bti-Statistics.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-500-bti-Statistics.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-1-bti-Statistics.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-500-bti-Statistics.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-1-bti-TOC.txt b/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-500-bti-TOC.txt similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-1-bti-TOC.txt rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_simple/da-500-bti-TOC.txt diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-1-bti-CompressionInfo.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-500-bti-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-1-bti-CompressionInfo.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-500-bti-CompressionInfo.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-1-bti-Data.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-500-bti-Data.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-1-bti-Data.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-500-bti-Data.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-1-bti-Digest.crc32 b/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-500-bti-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-1-bti-Digest.crc32 rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-500-bti-Digest.crc32 diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-1-bti-Filter.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-500-bti-Filter.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-1-bti-Filter.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-500-bti-Filter.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-1-bti-Partitions.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-500-bti-Partitions.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-1-bti-Partitions.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-500-bti-Partitions.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-1-bti-Rows.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-500-bti-Rows.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-1-bti-Rows.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-500-bti-Rows.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-1-bti-Statistics.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-500-bti-Statistics.db similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-1-bti-Statistics.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-500-bti-Statistics.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-1-bti-TOC.txt b/test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-500-bti-TOC.txt similarity index 100% rename from test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-1-bti-TOC.txt rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_simple_counter/da-500-bti-TOC.txt diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-CompressionInfo.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-CompressionInfo.db new file mode 100644 index 0000000000000000000000000000000000000000..5a82349dd2e22fcbf900172acbc7f84c8e531651 GIT binary patch literal 47 pcmZSJ^@%cZ&d)6<N-ZwVFJb@#2Zs9p|NleStapJlBZSSs001M23A6wJ literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-Data.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-Data.db new file mode 100644 index 0000000000000000000000000000000000000000..d281b869c65400a90257a4898011af14db4ca994 GIT binary patch literal 300 zcmWlUJx;?w5QS$FN4P@yk*Gi<1)@ME=0`$Qv``U2M}xff#%m{8uYa;$Zxs%JXgLNP zA#earfJB)dl(ejiZ=T-M%t-IM1@QX<=q`e%fra*B5s9eCvNB+C8Ouq)Sm8}eqN$F< zh{U`~sFw@LML3)Dgyu3T6Hf~*MUmxF>WR-u#hF%YRurmA6_0YJSej;OAry(jB-JG^ zs_C3%Bn;#vq{>rcKTb5w#!*h~3jo+PG!2#kZRj;@xUYS$4}J6o&?YU|_b=ga)HUh$ z9k?Fxx8OOT<NBbn178SuVoDuYOZWW<*SAae4IQ+)rw$HRHx6!wt7`{aoihioK3+Qx au8p+o<lJn0K0Y)y$MqaF%~Sjle|!HJ1YI)# literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-Digest.crc32 b/test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-Digest.crc32 new file mode 100644 index 000000000000..0778d5baa4e5 --- /dev/null +++ b/test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-Digest.crc32 @@ -0,0 +1 @@ +798454228 \ No newline at end of file diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Filter.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-Filter.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Filter.db rename to test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-Filter.db diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-Partitions.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-Partitions.db new file mode 100644 index 0000000000000000000000000000000000000000..b096fdac14902ec67edde3aa414f23396dac4511 GIT binary patch literal 62 zcmd<!{Lg9r?;7XbzbiSJ{?u^l|Mp*Cz`|%Cz;J+r!9m)Afzg10(S!j46reOKl$HYk DG=B@% literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-Rows.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-Rows.db new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-Statistics.db b/test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-Statistics.db new file mode 100644 index 0000000000000000000000000000000000000000..fdc9bb3bd64ad355df7a87ba5e156bf88706eb35 GIT binary patch literal 5785 zcmeI$dr%a0902g$d+-uYIPd|CWGas%I*@xwfe4~NkAZk37LF`;3moCW9H#-{fQJ82 z5it<i_$YH21qUXm1S0!G1x*WOn64ZloQTS%Fh>v)p*^^T@Ap^D@n16zJF~O<x$k%P zey`o%?~bA<k)O<4OwDaU+=1qHJ+}+E{Y`EcQk1k@-!*xk3eGoX%b5a3o5RYrOkp8o z&>0!IF2^K~E;g|VMjdNpb%~78q&MmF4XiP;qU2U=v&U;0{a5O_8+p&n%uKs@(&C!C zhv@3@$L_T?3R=II5)VxcxV~~>dXP&w!tIvLQU<t@wv-KSTp7_OLZT?Xe*DLF{E78w zPuPL56Jck<62fkTJqWKRyoT_4!hwWC35OGoBm5%aY{I34>j<|IHe=I#e^}d@Rugd( z;d9u6RfG>>3;Fw7KNj-i#A;D0&KE1N?IN)4iTXM`z;UM@Z0Awz<%h7PA7HP13EQg= zdu=eb-*Iet8n)&rc61kZ;tK2xPwc`Muqzqt24ej7#Nha-4>IlZCw5v2_E9`O>Ctad zT#$^PXTew4!nAKtzDR_}zv%i^6c=BrLbkipi0qI&gX}oXA}^bEN0#XDcyx>T1;sr& zRwAz+^g>=^PC{Pai~Bop0{6F~`b(4_G4vDiGf}vFJH3WcJok4ga(O}v^5Mx%$n6(! z`wX<B_N1L$$6!7k8Ecsb<~O=*-KlwSC-jx`A<zQ<v*`UQcvsR7<3g_@G(LnKyBuL$ zROGlBTAXf1^|i~_7s0rF`|volW6ZXD(93)}(evbzCcOsZuEO|kXm^b-8qcdFlkdT} zchft!p&wVqwL<%Uw<8=n_*zagbXdmzW6&`-LUuqW9p9sZ*7|<71$y6KE+f!Y7aL=t zo01Q&gZ{d{;%(^hPy4n3(*Z7-9njjXTjBFhHzuBi&p$m=D}m3yfE^zE99%CLI<vV1 zSZJb$Yk)-?dZqEu6BDJdf5i8z>)`xhFLNJ>0{IU0AD4Ur?3D0fw-MNRchw)T{*v$A zZo~S!U5{&%g8UUsWdWQIJWoY-%0b-c@`x1n*M_^X@s%JRlsf7Q*8?iWZ`~3Qk2L?> z`6}>pM_SIn^1M76rh?<ykQCEU4e~28+k1w9kKG*TY5+cW;-eN=-r@6seVT<hEq(p! zR5WmW?%0Eaz;8V7|70)ljr6+Qpal!Uo*pU70dDeW4&A?Cp`tbS#MFW*zOMM4{?9d? z09<1^{jW8g*qX;5`ODBdhWl`74ac|4+_4UmTI?FmdfCz%4!7UZ8V<F`(i(1Q4F?9q z(i-mnT*J|IyXra^$~m-_vJ2i@J$BPPdz=hko89ern)};j{`XJh<%nN>ciuY5b)98? zo}Ht)oJ-s}o?p2Lm|*$elU$DVyQqgx%v}!+%0bZv!GY2r0-p7GE4L;}Z;-!F(IK?5 zY=XPWy4Cvi7d>1a_@j5a`HtJg{L@sxzyH6KDc{2w3v-w}d1}h`O)13%ELT`DTVyga zi@rj*kI5_VwRQiLy#;x!<qK@yH<>jpDlgt(QiNJEWkI&Se6hFW8$l)$ag_>vBx|KO z;A*S6wj1+HSi_<^A+?Y-&L-xu*^IV$w~5KjV+(`STD3~8QEyZ$G-0YNwNkxNm8DT? zRGJV=ggaNMl^SJ`n&nb-YSmM=O0@XV!*`+0F~KLw<~mzhK5VNHk3wD9lUeqE10ga6 AA^-pY literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-TOC.txt b/test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-TOC.txt new file mode 100644 index 000000000000..298910cfdc58 --- /dev/null +++ b/test/data/legacy-sstables/da/legacy_tables/legacy_da_tuple/da-500-bti-TOC.txt @@ -0,0 +1,8 @@ +Data.db +Statistics.db +Digest.crc32 +TOC.txt +CompressionInfo.db +Filter.db +Partitions.db +Rows.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-1-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-306-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-1-big-CompressionInfo.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-306-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-1-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-306-big-Data.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-1-big-Data.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-306-big-Data.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-1-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-306-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-1-big-Digest.crc32 rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-306-big-Digest.crc32 diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-1-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-306-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-1-big-Filter.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-306-big-Filter.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-1-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-306-big-Index.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-1-big-Index.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-306-big-Index.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-1-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-306-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-1-big-Statistics.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-306-big-Statistics.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-1-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-306-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-1-big-Summary.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-306-big-Summary.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-1-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-306-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-1-big-TOC.txt rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust/ma-306-big-TOC.txt diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-306-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-CompressionInfo.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-306-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-306-big-Data.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Data.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-306-big-Data.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-306-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Digest.crc32 rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-306-big-Digest.crc32 diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-306-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Filter.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-306-big-Filter.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-306-big-Index.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Index.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-306-big-Index.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-306-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Statistics.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-306-big-Statistics.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-306-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Summary.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-306-big-Summary.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-306-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-TOC.txt rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-306-big-TOC.txt diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-1-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-306-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-1-big-CompressionInfo.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-306-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-1-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-306-big-Data.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-1-big-Data.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-306-big-Data.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-1-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-306-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-1-big-Digest.crc32 rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-306-big-Digest.crc32 diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-1-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-306-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-1-big-Filter.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-306-big-Filter.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-1-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-306-big-Index.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-1-big-Index.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-306-big-Index.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-1-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-306-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-1-big-Statistics.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-306-big-Statistics.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-1-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-306-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-1-big-Summary.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-306-big-Summary.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-1-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-306-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-1-big-TOC.txt rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter/ma-306-big-TOC.txt diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-306-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-CompressionInfo.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-306-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-306-big-Data.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Data.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-306-big-Data.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-306-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Digest.crc32 rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-306-big-Digest.crc32 diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-306-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Filter.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-306-big-Filter.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-306-big-Index.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Index.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-306-big-Index.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-306-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Statistics.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-306-big-Statistics.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-306-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Summary.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-306-big-Summary.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-306-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-TOC.txt rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-306-big-TOC.txt diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-1-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-306-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-1-big-CompressionInfo.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-306-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-1-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-306-big-Data.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-1-big-Data.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-306-big-Data.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-1-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-306-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-1-big-Digest.crc32 rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-306-big-Digest.crc32 diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-1-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-306-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-1-big-Filter.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-306-big-Filter.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-1-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-306-big-Index.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-1-big-Index.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-306-big-Index.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-1-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-306-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-1-big-Statistics.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-306-big-Statistics.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-1-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-306-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-1-big-Summary.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-306-big-Summary.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-1-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-306-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-1-big-TOC.txt rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple/ma-306-big-TOC.txt diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-306-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-CompressionInfo.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-306-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-306-big-Data.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Data.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-306-big-Data.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-306-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Digest.crc32 rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-306-big-Digest.crc32 diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-306-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Filter.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-306-big-Filter.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-306-big-Index.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Index.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-306-big-Index.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-306-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Statistics.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-306-big-Statistics.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-306-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Summary.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-306-big-Summary.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-306-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-TOC.txt rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-306-big-TOC.txt diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-1-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-306-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-1-big-CompressionInfo.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-306-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-1-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-306-big-Data.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-1-big-Data.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-306-big-Data.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-1-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-306-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-1-big-Digest.crc32 rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-306-big-Digest.crc32 diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-1-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-306-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-1-big-Filter.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-306-big-Filter.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-1-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-306-big-Index.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-1-big-Index.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-306-big-Index.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-1-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-306-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-1-big-Statistics.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-306-big-Statistics.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-1-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-306-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-1-big-Summary.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-306-big-Summary.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-1-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-306-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-1-big-TOC.txt rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter/ma-306-big-TOC.txt diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-306-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-CompressionInfo.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-306-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-306-big-Data.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Data.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-306-big-Data.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-306-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Digest.crc32 rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-306-big-Digest.crc32 diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-306-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Filter.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-306-big-Filter.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-306-big-Index.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Index.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-306-big-Index.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-306-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Statistics.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-306-big-Statistics.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-306-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Summary.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-306-big-Summary.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-306-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-TOC.txt rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-306-big-TOC.txt diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-CompressionInfo.db new file mode 100644 index 0000000000000000000000000000000000000000..8e08a5dca913775242791d79b15823b407030b47 GIT binary patch literal 43 fcmZSJ^@%cZ&d)6<N-ZwVFJb@zMlcCtbRzQrwwnc^ literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-Data.db new file mode 100644 index 0000000000000000000000000000000000000000..38aa3536628b60017637e6dee8e3f98f181d934e GIT binary patch literal 121 zcmeBVWMG)Yz-Una|Ns9621bU@>?+p$3>@YR3=A?r%nZaVK+MR&Ak4(Uz#zdOEUdtw zz?i7WAdtk$5Sh%#z-Z{p5O2$WIPaxDgQm2RKZE^|j29rLF_3voYzB}iW8%-Czi+ZX QgHiy{sAR_Zes}f&08a=RB>(^b literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-Digest.crc32 new file mode 100644 index 000000000000..90049599e034 --- /dev/null +++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-Digest.crc32 @@ -0,0 +1 @@ +2353277624 \ No newline at end of file diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-1-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-1-big-Filter.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-Filter.db diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-Index.db new file mode 100644 index 0000000000000000000000000000000000000000..0398b1f8a0daf8ad065034f4b71658c0cb693a9e GIT binary patch literal 28 fcmZQzG+<z0U^Mgv5=ISkfTVH5YanUTXvF{kGaLku literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-Statistics.db new file mode 100644 index 0000000000000000000000000000000000000000..99471825afc6a6ea228826445d6461f1f8af0658 GIT binary patch literal 4767 zcmeI$e@v8h90%~{$31s&au;`?2u@0c5atdh5xS`-cm`K(5I>ZDxP0KA2@ZkXod%@S z8UDi>kwxU@kFt$9G+9y!WY$JOKP-e=R12f+B4^f>Q-UO}=jxvPyuWk9Kl|gp+jr0R zb)WC^Jl|i>^L%$4$LY9#Q<8e%)x1T`f|_@$S>!lN8<$?|-e!~Q<cdngRv~*la*fL^ z+gz1iTW*6_DR8?Kx8f?4-QFs1Rc(#pwl_8Ykas%vY4rk&0)fD+DV~=OUOT|IPTqaJ zyHhyx?u;}(GZw9e+tnOQ!TZw0I@lhX)OJDA`k?(NX@$^H&)C2?g0Yct6k{`E3*%VE z3mGqFyo_-g<CTopFwPHI>-Q!mzKwBkZfR<xP|MwnN6|bpAN>7H+{^epT8Lx3A1yN1 z7Y}1xSBjQ0(fYM$1JmD#n;16^qNDuiyAGhOub>w^hE5nk-;<8M_ZZq%hR!~M&OL`N zoR8ibkM=x*ZkEv<%=#U4VBDVw<`aKMmzAK8;QHkKUqM_b#^+i11TB_*4)t|9T>rY^ z3lNt+Y60ti>jXy>2f&fD3V7aZ4A|_#^=NTihInl60`S613E;(}Mc`#aINxbgINzDA z??L^w<3EBQ%)$0On&5-@_Ft^veFa_ML({9kJs;rsjP<~H^2X>Jv_5aodltNR=?U<s z{Tb9k%2{~73NM+zq;WB!9@dB0yCsswb@h?!sHG=Hp}+dts(KnX^!O&JBOM#BQ_oB6 zgXhUqX1z$`(V}yJIwm_A)^nVB`ehnlbn^Kt)JyKmKSQ1J_2xCy=@%<cQ?K5-=P0$~ z+l<ZBMaOnzQCB3ty`Fm4pQZ`wmJd7gs81FjT1x#{d((5&lW&b|B<7c!%6q9RHf*5J zKi^q+mOlS{;6*ci{sqN%=^b*uFn(%XBeCe^eFuqkNkdjA_0-gEIzQ4Mt!?!CVMvXc z$RYJ3+TUn=lh|1B>VTU#>dBU0Y5&dNSgz3iTZZ$yt)%{Zxw(#h58_YQ`)nkh`0<35 z&R5d4JZCeBuPF5=)BQkJ=FbCW61R{3)b|we!-u<0(e^y<U!6tQb4`(>qm|TeD(@K_ zCqDY!*trhk^T%K7qV4tdr;TLaiu2ZIF3jW-JGbAsx&M|qPpGWDgC)jVRY5C+j^RBQ z4v4nr(th2f6+*}Go(l#fyyt@9;KF+@7?1FtE4=3-3*!H^=i*<e9ZO%r#aV|reah9T z%MBU-9R1n-d)5Cpf>~?2{R`9US9h-8quQlrKIeNi_`Vk9jQ^ExQ_w!d(qD$-51FRb zJ@)e3|M608+abF>m2$PMwB&(RB@K0odM9z0T)l5-t`kaj)>SLPGv>T>vEsd>dd?be U=E`8r)D?5x{7x^a^(r;>Z)1VMTmS$7 literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-1-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-1-big-Summary.db rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-Summary.db diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust/nc-1-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust/nc-1-big-TOC.txt rename to test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-TOC.txt index 6ea912e92c15..4ab645afba51 100644 --- a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust/nc-1-big-TOC.txt +++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_tuple/ma-306-big-TOC.txt @@ -1,8 +1,8 @@ -TOC.txt +Summary.db Data.db +Index.db Statistics.db -Summary.db Filter.db -Digest.crc32 -Index.db +TOC.txt CompressionInfo.db +Digest.crc32 diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-1-big-CompressionInfo.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-307-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-1-big-CompressionInfo.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-307-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-1-big-Data.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-307-big-Data.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-1-big-Data.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-307-big-Data.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-1-big-Digest.crc32 b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-307-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-1-big-Digest.crc32 rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-307-big-Digest.crc32 diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-1-big-Filter.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-307-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-1-big-Filter.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-307-big-Filter.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-1-big-Index.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-307-big-Index.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-1-big-Index.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-307-big-Index.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-1-big-Statistics.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-307-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-1-big-Statistics.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-307-big-Statistics.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-1-big-Summary.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-307-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-1-big-Summary.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-307-big-Summary.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-1-big-TOC.txt b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-307-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-1-big-TOC.txt rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust/mb-307-big-TOC.txt diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-1-big-CompressionInfo.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-307-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-1-big-CompressionInfo.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-307-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-1-big-Data.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-307-big-Data.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-1-big-Data.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-307-big-Data.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-1-big-Digest.crc32 b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-307-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-1-big-Digest.crc32 rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-307-big-Digest.crc32 diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-1-big-Filter.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-307-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-1-big-Filter.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-307-big-Filter.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-1-big-Index.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-307-big-Index.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-1-big-Index.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-307-big-Index.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-1-big-Statistics.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-307-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-1-big-Statistics.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-307-big-Statistics.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-1-big-Summary.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-307-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-1-big-Summary.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-307-big-Summary.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-1-big-TOC.txt b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-307-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-1-big-TOC.txt rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_compact/mb-307-big-TOC.txt diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-1-big-CompressionInfo.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-307-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-1-big-CompressionInfo.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-307-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-1-big-Data.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-307-big-Data.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-1-big-Data.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-307-big-Data.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-1-big-Digest.crc32 b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-307-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-1-big-Digest.crc32 rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-307-big-Digest.crc32 diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-1-big-Filter.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-307-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-1-big-Filter.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-307-big-Filter.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-1-big-Index.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-307-big-Index.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-1-big-Index.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-307-big-Index.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-1-big-Statistics.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-307-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-1-big-Statistics.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-307-big-Statistics.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-1-big-Summary.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-307-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-1-big-Summary.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-307-big-Summary.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-1-big-TOC.txt b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-307-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-1-big-TOC.txt rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter/mb-307-big-TOC.txt diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-1-big-CompressionInfo.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-307-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-1-big-CompressionInfo.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-307-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-1-big-Data.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-307-big-Data.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-1-big-Data.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-307-big-Data.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-1-big-Digest.crc32 b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-307-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-1-big-Digest.crc32 rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-307-big-Digest.crc32 diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-1-big-Filter.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-307-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-1-big-Filter.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-307-big-Filter.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-1-big-Index.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-307-big-Index.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-1-big-Index.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-307-big-Index.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-1-big-Statistics.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-307-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-1-big-Statistics.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-307-big-Statistics.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-1-big-Summary.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-307-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-1-big-Summary.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-307-big-Summary.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-1-big-TOC.txt b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-307-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-1-big-TOC.txt rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_clust_counter_compact/mb-307-big-TOC.txt diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-1-big-CompressionInfo.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-307-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-1-big-CompressionInfo.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-307-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-1-big-Data.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-307-big-Data.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-1-big-Data.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-307-big-Data.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-1-big-Digest.crc32 b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-307-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-1-big-Digest.crc32 rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-307-big-Digest.crc32 diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-1-big-Filter.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-307-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-1-big-Filter.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-307-big-Filter.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-1-big-Index.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-307-big-Index.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-1-big-Index.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-307-big-Index.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-1-big-Statistics.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-307-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-1-big-Statistics.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-307-big-Statistics.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-1-big-Summary.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-307-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-1-big-Summary.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-307-big-Summary.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-1-big-TOC.txt b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-307-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-1-big-TOC.txt rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple/mb-307-big-TOC.txt diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-1-big-CompressionInfo.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-307-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-1-big-CompressionInfo.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-307-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-1-big-Data.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-307-big-Data.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-1-big-Data.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-307-big-Data.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-1-big-Digest.crc32 b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-307-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-1-big-Digest.crc32 rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-307-big-Digest.crc32 diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-1-big-Filter.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-307-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-1-big-Filter.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-307-big-Filter.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-1-big-Index.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-307-big-Index.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-1-big-Index.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-307-big-Index.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-1-big-Statistics.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-307-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-1-big-Statistics.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-307-big-Statistics.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-1-big-Summary.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-307-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-1-big-Summary.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-307-big-Summary.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-1-big-TOC.txt b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-307-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-1-big-TOC.txt rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_compact/mb-307-big-TOC.txt diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-1-big-CompressionInfo.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-307-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-1-big-CompressionInfo.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-307-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-1-big-Data.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-307-big-Data.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-1-big-Data.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-307-big-Data.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-1-big-Digest.crc32 b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-307-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-1-big-Digest.crc32 rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-307-big-Digest.crc32 diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-1-big-Filter.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-307-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-1-big-Filter.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-307-big-Filter.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-1-big-Index.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-307-big-Index.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-1-big-Index.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-307-big-Index.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-1-big-Statistics.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-307-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-1-big-Statistics.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-307-big-Statistics.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-1-big-Summary.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-307-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-1-big-Summary.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-307-big-Summary.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-1-big-TOC.txt b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-307-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-1-big-TOC.txt rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter/mb-307-big-TOC.txt diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-1-big-CompressionInfo.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-307-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-1-big-CompressionInfo.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-307-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-1-big-Data.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-307-big-Data.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-1-big-Data.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-307-big-Data.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-1-big-Digest.crc32 b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-307-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-1-big-Digest.crc32 rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-307-big-Digest.crc32 diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Filter.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-307-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Filter.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-307-big-Filter.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-1-big-Index.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-307-big-Index.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-1-big-Index.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-307-big-Index.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-1-big-Statistics.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-307-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-1-big-Statistics.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-307-big-Statistics.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Summary.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-307-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Summary.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-307-big-Summary.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-1-big-TOC.txt b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-307-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-1-big-TOC.txt rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_simple_counter_compact/mb-307-big-TOC.txt diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-CompressionInfo.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-CompressionInfo.db new file mode 100644 index 0000000000000000000000000000000000000000..53c889d0e457e35473bb0309ad74e4fa7eff0d72 GIT binary patch literal 43 gcmZSJ^@%cZ&d)6<N-ZwVFJb@zMlcB^Smz=00Jr1?x&QzG literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-Data.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-Data.db new file mode 100644 index 0000000000000000000000000000000000000000..ec7c382817d30c107ab8b76c7d7655deacaa6398 GIT binary patch literal 276 zcmWlQKTg9i97cbpR765hMAxbq35m7*7ZX<~Z~){uv15|huG6Nr7sAYfH~^R62B^S& zV1X$(!;{|kN$<4}@B(P^(6(&@2!5Up9ybRNHBDytGRrf^ol$veHRVJts?rh`GwW-b zaI5R3m)bKiC)%Y#8CA;K^IYfDa%#$|2>o0Y#0$x13sY#h@FJZhnX47AVn+ot_ly}P z6XuDf%o<uaE2(uNx&VMtpchyVpa_O*5!`lrhyf$A4?A%m_UI)XPItq*eFGB`(+%)R u$`GKVF}m9tyuW1_{f&+>x>}v%PEUe<te!hP4f+MY7=2HBU35J@`}_yC<X2Gu literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-Digest.crc32 b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-Digest.crc32 new file mode 100644 index 000000000000..e86fd57a347b --- /dev/null +++ b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-Digest.crc32 @@ -0,0 +1 @@ +768627854 \ No newline at end of file diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Filter.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Filter.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-Filter.db diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-Index.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-Index.db new file mode 100644 index 0000000000000000000000000000000000000000..3e735baf9ae0498e90d261ae49388ca581c28b22 GIT binary patch literal 29 gcmZQzG+<z0U^Hx$1CmBfc0kg&ISxpgwA3*G05PWoLI3~& literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-Statistics.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-Statistics.db new file mode 100644 index 0000000000000000000000000000000000000000..8ebf44bc222978021dcc970e9d3bc9798389d4f6 GIT binary patch literal 5275 zcmeI$e@q)y902gU{saacEsPD8RkwkogcPLoI?T2T>jpI@AO%~0EO4|pVCjIgbpmb~ znan?kB2E#PY~VB@3oIiJb%1j-nZi_*jhNvtCMFfljMgn1GKbVF>$&guY0@VCXL`xY zz0do;`|*C<-D@d|l2CWHRDMVEX$PM~eA>$=F-0klQCan_YAxHq+G{wioprm}I){tZ zI%+)Hf+i1F>~e4}&QZd;Jhh(M`Z~^KY;FF&@MPjM{0Ub3{r<(cWLkRY`a!y5?w%Wi zeWFwEFGyz=rsMf=7oP%K;JR=l9k?I3@%M!`p$$TzRLBJPGQx7gQG{a%#}ZZ$RuWDm zyo&G|!s`iV5Y8dIiEt6&M+sLG4va0_IwAP{AmJ%&S{RRDKhq2m_YgjdEm}$V0Jc~- zf3R451ji-i*is#~Y%{i;Xm8YQ9FG~pj`d+LKZvb*9Xsi9?379Dd$X|b>%rFAunkAC z3(jDdtiY~J#&$n~-Ns_~67zS=gyTLnGOfObZ7ah*isvWoyNu!@3x1wOpJ0n^pQHK` z37-FwOXDamz0i&<`>hW-%Hl_kUgVIMEhZo<9C$t|O+TS{;&2l3sw*kTYo<z(*H7T_ z&X~vJt?M|4>TjO;0r{bPe1DIncu{=UFDm4g;sNBtw>BaVeTe&KdI<F=9TR^O)~Ahy z&m-?&cO3cC^V!g%4X4rjRrIRj3m6xtG@|(-9^MfR<C4bcEzr^@r%-!k^|g&KE+6vF zK}Vam-GE-E9zoAjoK1BR#^c52QRoCi8k)~56}Mi4@zwn=U4>qoTXYI~!<W`g&{-F2 zPD1BZ?(2p&eVuKEF74T^hqkA^`ylk5KjLPg+dt|ngzmQ-UI+bISL=(=bMH)U1E$mC zs)nKMTergJpYAI;4WED7|B3=W|02$N<vnn`Xy(M0W?-?0_8tP3q)w>J(DU<q;rK{@ z@92cz54k2`HXqcF>UyjBZQz*VW1}wM*oyXFVfz(dDX+rzD=!rdszCh}Y+D2T9wZ+( zj%Yz#{qd{{j#ujSLUS94XO{cY;Ceu>`*~CW;>M{TN1g_L_{hKsct210^7L>%*Oi)j zJ3#%`s-dwN;O=jy&-4PHedEmmc)#BB8Iy*)aa#54_(B1&dDqR`2kx432lMzH{LeL* z2pkj03TrNDuv|jKg>s=7y%YGt$eK&&nLGPV%&^y7!NVeJF5G{SH5clS$eJs%<^mHU zvgZ0PYcBfbeb#f|P)Sw`B}-r1^IU1=(*6aDZ9o72Mj#1YZM?v!yT>b>yd8X^^S|SR z9|%#3_?`B5`BEk^Nd7|_ydY&w`cr=STK{AvMWz4ypD<P0-K@)9!#cI)W!pEFH8pVj zKRQyb(c@ynK0>^Qb+$}|I)B+y4Nfj_fsp65aGtR0nd>~doIp)YW~hsYdrF}bG#Uxt zsGR?0t8^{iP~BDT=3Gld;N+@VdsBsnt#WeiOvcXW83S`aqch~`cQOtpN59jMYtS3A j0}(#2XL1d>nGDBQaWMJ^LR}{ODIttY$bl9rYBYZVoAq=c literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Summary.db b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Summary.db rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-Summary.db diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-TOC.txt b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-TOC.txt rename to test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-TOC.txt index 6ea912e92c15..8ea9eedf3bba 100644 --- a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-TOC.txt +++ b/test/data/legacy-sstables/mb/legacy_tables/legacy_mb_tuple/mb-307-big-TOC.txt @@ -1,8 +1,8 @@ +CompressionInfo.db +Digest.crc32 +Statistics.db TOC.txt Data.db -Statistics.db -Summary.db -Filter.db -Digest.crc32 Index.db -CompressionInfo.db +Filter.db +Summary.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-3113-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-CompressionInfo.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-3113-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-3113-big-Data.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Data.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-3113-big-Data.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-3113-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Digest.crc32 rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-3113-big-Digest.crc32 diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-3113-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Filter.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-3113-big-Filter.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-3113-big-Index.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Index.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-3113-big-Index.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-3113-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Statistics.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-3113-big-Statistics.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-3113-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Summary.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-3113-big-Summary.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-3113-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-TOC.txt rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-3113-big-TOC.txt diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-3113-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-CompressionInfo.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-3113-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-3113-big-Data.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Data.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-3113-big-Data.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-3113-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Digest.crc32 rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-3113-big-Digest.crc32 diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-3113-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Filter.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-3113-big-Filter.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-3113-big-Index.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Index.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-3113-big-Index.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-3113-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Statistics.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-3113-big-Statistics.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-3113-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Summary.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-3113-big-Summary.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-3113-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-TOC.txt rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-3113-big-TOC.txt diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-3113-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-CompressionInfo.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-3113-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-3113-big-Data.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Data.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-3113-big-Data.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-3113-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Digest.crc32 rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-3113-big-Digest.crc32 diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-3113-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Filter.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-3113-big-Filter.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-3113-big-Index.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Index.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-3113-big-Index.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-3113-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Statistics.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-3113-big-Statistics.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-3113-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Summary.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-3113-big-Summary.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-3113-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-TOC.txt rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-3113-big-TOC.txt diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-3113-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-CompressionInfo.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-3113-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-3113-big-Data.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Data.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-3113-big-Data.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-3113-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Digest.crc32 rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-3113-big-Digest.crc32 diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-3113-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Filter.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-3113-big-Filter.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-3113-big-Index.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Index.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-3113-big-Index.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-3113-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Statistics.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-3113-big-Statistics.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-3113-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Summary.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-3113-big-Summary.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-3113-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-TOC.txt rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-3113-big-TOC.txt diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-3113-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-1-big-CompressionInfo.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-3113-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-3113-big-Data.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-1-big-Data.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-3113-big-Data.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-3113-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-1-big-Digest.crc32 rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-3113-big-Digest.crc32 diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-3113-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-1-big-Filter.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-3113-big-Filter.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-3113-big-Index.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-1-big-Index.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-3113-big-Index.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-3113-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-1-big-Statistics.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-3113-big-Statistics.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-3113-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-1-big-Summary.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-3113-big-Summary.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-3113-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-1-big-TOC.txt rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_inaccurate_min_max/mc-3113-big-TOC.txt diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-3113-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-CompressionInfo.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-3113-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-3113-big-Data.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Data.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-3113-big-Data.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-3113-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Digest.crc32 rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-3113-big-Digest.crc32 diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-3113-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Filter.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-3113-big-Filter.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-3113-big-Index.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Index.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-3113-big-Index.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-3113-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Statistics.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-3113-big-Statistics.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-3113-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Summary.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-3113-big-Summary.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-3113-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-TOC.txt rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-3113-big-TOC.txt diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-3113-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-CompressionInfo.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-3113-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-3113-big-Data.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Data.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-3113-big-Data.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-3113-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Digest.crc32 rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-3113-big-Digest.crc32 diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-3113-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Filter.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-3113-big-Filter.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-3113-big-Index.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Index.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-3113-big-Index.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-3113-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Statistics.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-3113-big-Statistics.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-3113-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Summary.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-3113-big-Summary.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-3113-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-TOC.txt rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-3113-big-TOC.txt diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-3113-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-CompressionInfo.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-3113-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-3113-big-Data.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Data.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-3113-big-Data.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-3113-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Digest.crc32 rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-3113-big-Digest.crc32 diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-3113-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-1-big-Filter.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-3113-big-Filter.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-3113-big-Index.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Index.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-3113-big-Index.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-3113-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Statistics.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-3113-big-Statistics.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-3113-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-1-big-Summary.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-3113-big-Summary.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-3113-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-TOC.txt rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-3113-big-TOC.txt diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-3113-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-CompressionInfo.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-3113-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-3113-big-Data.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Data.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-3113-big-Data.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-3113-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Digest.crc32 rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-3113-big-Digest.crc32 diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-3113-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-1-big-Filter.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-3113-big-Filter.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-3113-big-Index.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Index.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-3113-big-Index.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-3113-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Statistics.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-3113-big-Statistics.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-3113-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-1-big-Summary.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-3113-big-Summary.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-3113-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-TOC.txt rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-3113-big-TOC.txt diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-CompressionInfo.db new file mode 100644 index 0000000000000000000000000000000000000000..8e08a5dca913775242791d79b15823b407030b47 GIT binary patch literal 43 fcmZSJ^@%cZ&d)6<N-ZwVFJb@zMlcCtbRzQrwwnc^ literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-Data.db new file mode 100644 index 0000000000000000000000000000000000000000..73694b2f46ea7a4d7acc7b1f20919c86d9872b80 GIT binary patch literal 121 zcmeBVWMG)Yz-Una|Ns9621bU@>?+p$3>@YR3=A?r%nZaVK+MR&Ak4(Uz#zdOEUdtw zz?i7WAdtk$5Sh%#z-Z{p5O2$WD0GECgQm2RKZE^Y_ZuLlF_3x0$O6ceG4W^6zh&sp PpcDW!Dw(m)WXl==EI1dL literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-Digest.crc32 new file mode 100644 index 000000000000..b57ebd32f353 --- /dev/null +++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-Digest.crc32 @@ -0,0 +1 @@ +527605309 \ No newline at end of file diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-1-big-Filter.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-Filter.db diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-Index.db new file mode 100644 index 0000000000000000000000000000000000000000..0398b1f8a0daf8ad065034f4b71658c0cb693a9e GIT binary patch literal 28 fcmZQzG+<z0U^Mgv5=ISkfTVH5YanUTXvF{kGaLku literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-Statistics.db new file mode 100644 index 0000000000000000000000000000000000000000..fd80793b5b094e4877506d2bc8407596f0eb6d9f GIT binary patch literal 4791 zcmeI$drVVT90&0G=mRS4Xh9T=Y*QG*OffQsCTbO{(8UDCL)pX33%wBx0%@y5rqMb4 zhb)Zg6ij@~EkT11GTA0D=S+0qHaC<#*xWEI>Da>DI*lR@q`TdE!|!{t1^?V1J;}+r zpZh!a-t)S>C*?R!!~K&Il!90CS|tlg-l}Ag<IE?y)JoTSt5hYGl*-l;$?cXZoG!`g zEcIA3w|L|nms57h&RogmDf5(7R>&?}ZOxBar=nj{ZZN~|_m3N6dF}37yZDCThi|vF z2&X?B(+-UFM=Ie)B?nUAxiqm3w1+08U69m1s3*1RL9o;@)-w)cY+xM0*u>b(IGXWv z#xohuW}L!!KI27<vx8Rqy_$)yXB?PYn)XsC$8C)J&^$9A{F6-F!}uIpn8tV~T4e4o z9>BP!5Uou^>lUN+On<{hF>dHaNBGbW?Lu4LM8`afj_XA~l8S!pFxpy#&Nzt9Jd4hq zie49sc0Y}-lhDn~`t7!3+!qh#<8Pvi3eX2}ee%9<ATH$N`z(Bl7K^@s`Wg+cf6dj) z5Z7L+2kZW50f*)L!QtaFc*=Mb*yP0ZXtw_X@#u~i@bv3(;F*1S;MqMm-zg(F-)RjW zLH)%8KY^cGhV6SM&I|DkzgfULa$3Rr?=1kgU&Qh0Z-?>Z4Uu<fecqP!DtP;>W8lv& z%%c{P&cN@hu+Q`rjf-*Bus+0&wc#|bsSaO4t$n@^`m3uftEO>%yLXs6+`i&A^_2Kd zc%O_#mMb(KDLT5SqcRd<Jx?>;YohTP$6p_!e)Ng#)6_}du31E#dZqLf^}==A4pG~$ z&09mAcX(qubxGp;OQ|>iX&j`k|F|WK`gs2SS=665*1kqP{9f-0Vt$UXxP!W6`EvUH z^DVh&==;z6-!RelUy!}mKOol&11FZ$5Q`q(yPH^(&|`5>kBn@k^P|1nu$O*6^vO|! z%SipO#&>GoB{t+7>2eWAyior;?Z4?e^APR7`D%8Xh18!a)m72&LF_SGr<KIxKN+;p z`AWEz<)|a^xrM$&x*teS`?bqN;<mn@J6|MTb)fYGZO?PQh3RxXSLE578%X`y;`Z(V z;zK|5pKT^S_tx93w7uR7DZLpJao+OE<*`g+$A&wjJ15M!gU9M`uz|5!RZtItWoXZZ z1ETJ^)U&JAgJ2ojbHRXw_FOO=Txib);}P0(h4x%zLHxh=T>PHOi%s{rn7fa2x}+OT zUmrdHcX{pQo7<KDHv(C0y6sEj$s3!NZd2@3&;3`a4F;4l_yF$vOUgOr!%RK#@ca5B zL!(LOId0Cs{~=Rs-6*--rBb=Iuwdnaf-P0Dawl=KRKBBUvI`0}Rh7$uD<-{lzU+CR YdX5TD+WbJx<hheQ{6TN2_9{8~F9;~kCIA2c literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-1-big-Summary.db rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-Summary.db diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple/nc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple/nc-1-big-TOC.txt rename to test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-TOC.txt index 6ea912e92c15..9e670e31e346 100644 --- a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple/nc-1-big-TOC.txt +++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_tuple/mc-3113-big-TOC.txt @@ -1,8 +1,8 @@ -TOC.txt -Data.db Statistics.db -Summary.db +Data.db Filter.db +CompressionInfo.db +TOC.txt +Summary.db Digest.crc32 Index.db -CompressionInfo.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-1-big-CompressionInfo.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-31110-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-1-big-CompressionInfo.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-31110-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-1-big-Data.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-31110-big-Data.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-1-big-Data.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-31110-big-Data.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-1-big-Digest.crc32 b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-31110-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-1-big-Digest.crc32 rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-31110-big-Digest.crc32 diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-1-big-Filter.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-31110-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-1-big-Filter.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-31110-big-Filter.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-1-big-Index.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-31110-big-Index.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-1-big-Index.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-31110-big-Index.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-1-big-Statistics.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-31110-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-1-big-Statistics.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-31110-big-Statistics.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-1-big-Summary.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-31110-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-1-big-Summary.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-31110-big-Summary.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-1-big-TOC.txt b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-31110-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-1-big-TOC.txt rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust/md-31110-big-TOC.txt diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-1-big-CompressionInfo.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-31110-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-1-big-CompressionInfo.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-31110-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-1-big-Data.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-31110-big-Data.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-1-big-Data.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-31110-big-Data.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-1-big-Digest.crc32 b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-31110-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-1-big-Digest.crc32 rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-31110-big-Digest.crc32 diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-1-big-Filter.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-31110-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-1-big-Filter.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-31110-big-Filter.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-1-big-Index.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-31110-big-Index.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-1-big-Index.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-31110-big-Index.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-1-big-Statistics.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-31110-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-1-big-Statistics.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-31110-big-Statistics.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-1-big-Summary.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-31110-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-1-big-Summary.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-31110-big-Summary.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-1-big-TOC.txt b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-31110-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-1-big-TOC.txt rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_compact/md-31110-big-TOC.txt diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-1-big-CompressionInfo.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-31110-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-1-big-CompressionInfo.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-31110-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-1-big-Data.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-31110-big-Data.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-1-big-Data.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-31110-big-Data.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-1-big-Digest.crc32 b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-31110-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-1-big-Digest.crc32 rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-31110-big-Digest.crc32 diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-1-big-Filter.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-31110-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-1-big-Filter.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-31110-big-Filter.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-1-big-Index.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-31110-big-Index.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-1-big-Index.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-31110-big-Index.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-1-big-Statistics.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-31110-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-1-big-Statistics.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-31110-big-Statistics.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-1-big-Summary.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-31110-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-1-big-Summary.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-31110-big-Summary.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-1-big-TOC.txt b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-31110-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-1-big-TOC.txt rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter/md-31110-big-TOC.txt diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-1-big-CompressionInfo.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-31110-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-1-big-CompressionInfo.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-31110-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-1-big-Data.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-31110-big-Data.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-1-big-Data.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-31110-big-Data.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-1-big-Digest.crc32 b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-31110-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-1-big-Digest.crc32 rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-31110-big-Digest.crc32 diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-1-big-Filter.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-31110-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-1-big-Filter.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-31110-big-Filter.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-1-big-Index.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-31110-big-Index.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-1-big-Index.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-31110-big-Index.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-1-big-Statistics.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-31110-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-1-big-Statistics.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-31110-big-Statistics.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-1-big-Summary.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-31110-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-1-big-Summary.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-31110-big-Summary.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-1-big-TOC.txt b/test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-31110-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-1-big-TOC.txt rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_clust_counter_compact/md-31110-big-TOC.txt diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-1-big-CompressionInfo.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-31110-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-1-big-CompressionInfo.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-31110-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-1-big-Data.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-31110-big-Data.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-1-big-Data.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-31110-big-Data.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-1-big-Digest.crc32 b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-31110-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-1-big-Digest.crc32 rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-31110-big-Digest.crc32 diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-1-big-Filter.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-31110-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-1-big-Filter.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-31110-big-Filter.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-1-big-Index.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-31110-big-Index.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-1-big-Index.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-31110-big-Index.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-1-big-Statistics.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-31110-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-1-big-Statistics.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-31110-big-Statistics.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-1-big-Summary.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-31110-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-1-big-Summary.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-31110-big-Summary.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-1-big-TOC.txt b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-31110-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-1-big-TOC.txt rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple/md-31110-big-TOC.txt diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-1-big-CompressionInfo.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-31110-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-1-big-CompressionInfo.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-31110-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-1-big-Data.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-31110-big-Data.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-1-big-Data.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-31110-big-Data.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-1-big-Digest.crc32 b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-31110-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-1-big-Digest.crc32 rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-31110-big-Digest.crc32 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-1-big-Filter.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-31110-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-1-big-Filter.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-31110-big-Filter.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-1-big-Index.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-31110-big-Index.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-1-big-Index.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-31110-big-Index.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-1-big-Statistics.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-31110-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-1-big-Statistics.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-31110-big-Statistics.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-1-big-Summary.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-31110-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-1-big-Summary.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-31110-big-Summary.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-1-big-TOC.txt b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-31110-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-1-big-TOC.txt rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_compact/md-31110-big-TOC.txt diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-1-big-CompressionInfo.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-31110-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-1-big-CompressionInfo.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-31110-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-1-big-Data.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-31110-big-Data.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-1-big-Data.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-31110-big-Data.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-1-big-Digest.crc32 b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-31110-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-1-big-Digest.crc32 rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-31110-big-Digest.crc32 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-1-big-Filter.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-31110-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-1-big-Filter.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-31110-big-Filter.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-1-big-Index.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-31110-big-Index.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-1-big-Index.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-31110-big-Index.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-1-big-Statistics.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-31110-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-1-big-Statistics.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-31110-big-Statistics.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-1-big-Summary.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-31110-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-1-big-Summary.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-31110-big-Summary.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-1-big-TOC.txt b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-31110-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-1-big-TOC.txt rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter/md-31110-big-TOC.txt diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-1-big-CompressionInfo.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-31110-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-1-big-CompressionInfo.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-31110-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-1-big-Data.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-31110-big-Data.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-1-big-Data.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-31110-big-Data.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-1-big-Digest.crc32 b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-31110-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-1-big-Digest.crc32 rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-31110-big-Digest.crc32 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-1-big-Filter.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-31110-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-1-big-Filter.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-31110-big-Filter.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-1-big-Index.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-31110-big-Index.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-1-big-Index.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-31110-big-Index.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-1-big-Statistics.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-31110-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-1-big-Statistics.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-31110-big-Statistics.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-1-big-Summary.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-31110-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-1-big-Summary.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-31110-big-Summary.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-1-big-TOC.txt b/test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-31110-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-1-big-TOC.txt rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_simple_counter_compact/md-31110-big-TOC.txt diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-CompressionInfo.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-CompressionInfo.db new file mode 100644 index 0000000000000000000000000000000000000000..8e08a5dca913775242791d79b15823b407030b47 GIT binary patch literal 43 fcmZSJ^@%cZ&d)6<N-ZwVFJb@zMlcCtbRzQrwwnc^ literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-Data.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-Data.db new file mode 100644 index 0000000000000000000000000000000000000000..0c874f65fab2ea4e38a68d85cb7cb2fb773233e2 GIT binary patch literal 121 zcmeBVWMG)Yz-Una|Ns9621bU@>?+p$3>@YR3=A?r%nZaVK+MR&Ak4(Uz#zdOEUdtw zz?i7WAdtk$5Sh%#z-Z{p5O2$WXupC#gQm2RKZE@dxeXwuF_3vQ=>>>s;?JOeOzSO! Oasbe%WX3Qyi$4H14j2sp literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-Digest.crc32 b/test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-Digest.crc32 new file mode 100644 index 000000000000..2fc48a9ce7f6 --- /dev/null +++ b/test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-Digest.crc32 @@ -0,0 +1 @@ +1520549740 \ No newline at end of file diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-1-big-Filter.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-1-big-Filter.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-Filter.db diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-Index.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-Index.db new file mode 100644 index 0000000000000000000000000000000000000000..0398b1f8a0daf8ad065034f4b71658c0cb693a9e GIT binary patch literal 28 fcmZQzG+<z0U^Mgv5=ISkfTVH5YanUTXvF{kGaLku literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-Statistics.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-Statistics.db new file mode 100644 index 0000000000000000000000000000000000000000..f4aa1057e0886a6cfc9a66b01dc30265efa9ebed GIT binary patch literal 4855 zcmeI$c}x^n90%|@_5fs8c0m*j)B}nTvlu8v<FSfsVNDY#htk8;ft?Blfh?m%s?k{d zhcpzgqKRjl5QA1yX)91`|F8vWZBg2kwkeh-TQpj`tyPMZvd*?U;rD%MgMT;sl9xB1 z`MsHW*X+D3$8kFDpOl~$yqedkSy1yfHH#c)xy_|kxi{FPYN@nLwv|dXHBzO^E!kXU ziY;raBImkYvRiiLNp7WFDX*%O-S+yrAG6Oyy`tV=rqAaaH^uP!y?1u=O(T!pZEqFM zemJHd8XJsI!_8{;C;xM4V(o7aO=`O!X?@U6>a~MFX<%$*9Ktw^aX4c$V+-Rb#xoel zF`mmfg>f3=C5&?d*807MiEm)+pIe#^P^ic4j0ey>GavkuOk81n0WC~tybCQd_ZJUi zTvvqFr=tx^(MG1fArCMf)`JfBqNnagTaTlopF_v?qi3a}A3uh+6{9ncptH`S^QNKK z$DnJTK{rU~7H0kSI56&w2lMg2p^FRAM{s@e-mf7p6yW<TTtbV*UqF4G4%ff#>Sc)Q zdmF)qUt7T;1wL@-xD1{$9tk$Pa6MWaHy|F>84aFsEfyR%kPn{Qhx45>iu0Y`^byow zI`k8G^$Kj?v#}nCZ@g^<@62rjAG*H~-0?Av&tM0PCm$AZkJjhy*{^|j%sC1E?BW7y zA?Y0az6x)czoKz5wiech*tss0#&xx!%c%7)3_yPkRpqrbZtU=kP=`8J-KCxq-v#fJ zsn~jj#v?>$H+5uYBCO}>=KBX}eCDY=H>qbok#m+h>6^7ns8g?$ouOW|e)~~s$9D_X zQs*DrltEpZ`2KS0Eq|JZsT)6O&89w8aA*$o=gsx6Q;)pYzlxZjXDaEWE?v2jzW;n{ z-Z}dI^S=FN`u+>D=h_G4dSU4FvN~c>;XQkabqRe|C-vy)Hab80Kbj8E?}srta(D%) zAJY6z-MhqLxhJ~a#NjVC{zCh2{?>An_TO?fr`<~GPm>y|>GvS!q`k{V;_;sjTj_iy z+{tz}kof!}ZzA0fWTfBfHj}u0;Ck1~#7`e?J5Ag3ymwIsUC)*Ij+Q1;zrLiSXNdUd z_k-tKh%dbPRvT@v=VD5K=0u#gzIu5qi`cpG-h*8e=CrnHzrp%IOtV&1&<+A+aL<JU zqV2h~vum`2KpEU~!GHw!TreD5aL)zf5!`bH_grK_{J-{G{JyGvFJ0%Ne_zZQl5YJl zy1Mu8a$LEwL;ZimpS7;rzcju5^XBE-RlBrv|5a)qZnX?NfZMZQt*m~?sV5%(m+?ee z+@)TQoA>X3=#<ztN$#35slrxN_|(F}t<|!6Cvl5Zv9oWo3ko+^SIGV=CcSlmtUOXZ XXQh&!=C7GNf3k-^>MgZiB_}@wHxtxQ literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-1-big-Summary.db b/test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-1-big-Summary.db rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-Summary.db diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-TOC.txt b/test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-TOC.txt rename to test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-TOC.txt index 6ea912e92c15..f2df26ca5694 100644 --- a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-TOC.txt +++ b/test/data/legacy-sstables/md/legacy_tables/legacy_md_tuple/md-31110-big-TOC.txt @@ -2,7 +2,7 @@ TOC.txt Data.db Statistics.db Summary.db -Filter.db -Digest.crc32 Index.db +Filter.db CompressionInfo.db +Digest.crc32 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-CompressionInfo.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-CompressionInfo.db new file mode 100644 index 0000000000000000000000000000000000000000..5f53ae065801789b3075eb4bdf63340df7161c6b GIT binary patch literal 83 zcmZSJ^@%cZ&d)6<N-ZwVFJb@zMli|1z^X7E$YO)?pcM0Y2!nkSgy!yn(EPa&TG$N$ D-(U#O literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-Data.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-Data.db new file mode 100644 index 0000000000000000000000000000000000000000..2c082def965b0be6523091e8b79184935e3b4676 GIT binary patch literal 5236 zcmd6r{Zmxe8OQGeE3%N>*hov1>9rY}F}B0uWr5Wyca6@dAqk`=rm2(2o3J3t%kIK1 z4Y1ai5YwcA)}m<~MTfmu%~UXLsFQ((ePOi?V~Jzcp<`2L8k5>KG#y_ugSXFf?%sRP z-SH3L7l!Y1o^#G~?|nYcbIx*{iR<M|IbE_W``~|>%i{J9<>ch%<y#91i)<?`9E`|) za))mp&n@)%yxVKsm9<qnUG475CRdHiSy^9G?eJDQb~<;qHG4d*JG?t<T3WYP)U|D| zb~~#b&92rqhuc%XtJ&eMZgsb~9QAFD6_pK5^}9T^4Q|H{uiH`Qae7_Vj*1GWyUy8I z-%wNA<ZiBT)_R-UnjJf8cQrKDS5<pp*TyCYZ#Z|28G|~}*u8J{c;`jC*-3_STP!rt z1^%QiU-v$H;Axu(_TgMR^&Kb`$M+wzo88~cyJqHN55@T`;`rN5U={KUsaqdRH`&eP zS9Wx@zRRhuXzag>o&`qFO9AvOG<x2-iJnD9Px%k%Y12J9yIGt_?glGAyb^;yv|uI$ zHWD8732q^8jM@fD?GsBE`7BOhNF?aw7XxR{&_|1T_{E8D-@`?G^bLS~{O>50`5>RT zV32vc4RTc(BgsUu__DS)%q{+b&!R3RkxjF@uS8xPw3}T*zOs)*Uh%+(cw#93kf^>i zu6R8O{6jRm$gt-1+DMlLK8vb_?oKDX{Q$2#+Qd6gG5ZP9*mE$bZV>sB86(pt*;u~F zVAa3jE$UIvet~gXW6KD0qsLvs&{!7Tkpynbi{0v=UksYw7~|EcAkEGCN+?|@xpXDk zV9JjTqbaW#WcnkFB*BdPz3fJrpXyKJC&6X+dm$=uNi3?bi`RoJrV?{ErGJ!%6j-Af zY(99e{A$!U$|$bCpNYQ80m8qf+)|T;drw9h#PKT6{9t!zo#{A#$JEb%fOW-Y63xV! zA)1?xn>bT$7nraJaiwM<=d(m~;5R19TN-_@^TFkZo-;RXqxg#&{>8z~JqrGX$?_K7 zVZu3-|Dd%jj{o51U&8kg@-kAig-t#S*;<a){JSP?KC<2dRRzA*6+H;FwvM2+AkMe_ z7+MSCd^-_rp(xJxr<hNIEzWoUkHE@5Btl>^03Y1H6*E?^0Ko%m;VIP$Ad%JIVGDpn zdTFL9{7dAd4_8yExoKeGKYwP&zQU5E777wd6bZwAD0MlTp&>qy)vEf6p%o6FMT6fj zhE^@45z0^EGf0A+r5=Y<S?s~E=D)bL%6(|<6+<M?A2^~H4e)+>qaKPRG!*Y)%7W(r zGe%x((HD{gU)Ls$IZ&e>oqRF4?#nTrsSA7OI$%$=Opv|n*=4(*#AQ|RLw!+Z7jpD4 zt4{1@l~iX?4Xir#L*=U~(2Kzh*N|xq`d~=_H8THR8STcIzq9pI#8+w!8d}*6*@uh= zOM>%+aobx7@-6<&B=~dR)Q^mp4&swcB+EDWqC^V+?F7**oQjd3*9A#K$$#4-<VXqf zZGK?_oH`Kv(Ai}V0zWierGVeM5FGhIm_xY_eV7u*edzMj@Y036E6M7zi2b!3H0Ij? zaAh{~{kcN@AuRwj&QiP!jGp(JaC)K9a}F^UigeEeXy$;jn@<zr0TJzR3Pn{zJ^aY0 zM%2TLA+;FP&&{%^`^E6$KjMOFuMcOOL8-tFqgf7v{QgRVv;w4zvluYQb-&T2MAp5} zstvt{UcccWLp$&}dX+}0wd#^~c5+DvDwXwcX(c;Yx}8;<HyPA3ta>SzRa+lp)i(H2 zI?H5R5{)m`<RQKp8lMJt<>&0=%3G|ep(Rp9JzR5%9jwO=pw-q+JFDKf$0*^gIY}=) zr02YHgi}x_p^1)erN6&eW<!{5dorJ)WXF!(V4{cA?<s7DB7^gU?V^vdC1&%f$#OZ8 zJ>>zi1)C{BF6UdY{bnQBKSA6}9tKG?&*}0Q?fG3`r?h{}3ptDQnTUJ1XX-(?hd-EA za8JYToy`{~%H@>z@WJ6Y-oqnT;X4R<PeFvSkP!HfFxiFtDJ6WIk8}pnXid;#K;i=@ z(OMAadK!&|ajv0vz{;nJ5ER8b>P4;NesLpe9rueRakyWBG-hx={4R^TL}oaguXgsC zL{w^LpIN}F>xPuRZb<1HhOYn5WBNS57)hiRNv$A}Z$Lp|8hd2LG6r>t9D#GEetw7P z*@l`D$xg+^W2}=%F66m(+V_*lvz@PMts7){msOWy#AD~u)#rnyeN1E$@ja@CEs@v% zW&r-@Lk8f({SE{0!Y@WW42^xe+1~!^I$T%<K63PVcAuj!u<FDrR!NT=)OT5R%A$PN zB@dU;tMrL`)psc1aHoznZ{-%+LJ7Br2g>FN;f3F#$7?TtG$oy09>N2gk@2DrxpaOx z{W-vk<i8JxP#|CR?oLWa>kzbG7k_Cn0@2Qk=A}Yf40~%Am?-vX=|aw=z8GR38NT`; z>?6~e@EYNEzlGjyy*%u~ozD^@=T62^AGr+V0`+&zdXPZ<WoQJDVHvSUg6T+oG8(NT zdI(7URk$Vs^>MEM0wZq`A#~(>a$gj+#`|QC54FboWX~Sd8t;=m`;EL$9;{dLg5KpN za_%(CyI-6<3}-?Wc#ub!gag^nWE{u=NHU$5$$^J-DUm^_nCc*?|FP=yHG}#qgS<q} zjMEU6qdZ_Z+RGv@kr7(+)G8Fs!KKvZKKaQ_cICeq3eR<>@JQsdCf4(-6|B0mh*e+T zHL!jk(1((#HuuN`v>+%+vF7d<qY3Br*(m>HkPFVBRB?|kIBOVPOAEGI4WrL;%qWSL z&3&Ymku>Gnee0k*uWuP=ymq@3sG~1joF7mRTQg1V{Ld)7lz>ktRt0C1FtOA5tRycd zn#D*2dbCR@Tx#;F&a?tVB2}~kFz(3ug-x^qFy3gL%Eb!s(o&=!S6@P=9}~?DW%%oz KX{zJGlK%sEOvIJ| literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-Digest.crc32 b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-Digest.crc32 new file mode 100644 index 000000000000..f4803c470ff2 --- /dev/null +++ b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-Digest.crc32 @@ -0,0 +1 @@ +3346976800 \ No newline at end of file diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-1-big-Filter.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-1-big-Filter.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-Filter.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-Index.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-Index.db new file mode 100644 index 0000000000000000000000000000000000000000..8c62daca3bbbc9b0340c48b84753a0ded04ce2a1 GIT binary patch literal 157553 zcmeI*F^^PL9ES0;5rJ7i66}nzvzD1zU}<e_u`;nWblTXMpoz8?hVUH-u~~Y;9W{Oc z68Hc%#P}6R);R|*hk}H}UB>(F{HSg=VfOCKJ<mS>_uRd2*|}VX@7}xk{o&!^59Rs4 z3#ZCY2bV9;7t6gXSFi0~cjP^A9asy~=D`|T_vdr0!YZu7s%f*TQ|jXC$*eSVE&3N7 zff$H^*t8+msC5OaunMcNYTB&ol)AIKdVKiYxUTlEIzllNL$PU7tQnU$-j;-I8Cz2^ z+jI9`8rQ<6B6eXHcEzx(8rMQHk6NgOS~1jmxVw6?(=x8bQnDr4l59z~rk!kcN;Tt> z%rh>=#kgXOYq&8Rw-p!HI?NoAFY-mcrY&Dpxt5}N%0;;-SB`Ss9F^-p$c0_lg<Uc1 zYQ}|HsD)an6+^Az)@<BYT<(dsL|dXQ(blx1txl<?T%vi(MY$+fj&j{TJMJqk_dCoS zkuUN^zNRf-YvnpHCo(R^#kg{e>+abx<l6HzLGS`E@QQ&~4Y~Fx7iysvYQ<1%cyMM6 zx%NCy5WK((ykg+hluI;^TBwCuG1M9!b>;FvLGS`E@QQ&~-B)xc7ifVNXvIKlczR~_ z<lXvEt^?B|c3~HG#jvYF%ajOOpaohn&>EhdTK%*0qSAWybt#wrD&@l)rF{HnDW6;~ zW&hVwKK-ba&weT8^LI-5^4n6r`n!~Gz9>8M)#)GAV!OrME#}j{#oQmlyy*{N>i(o! zY$QW6B+H@Yx)!@-W%uXGH_V;pPIG6>+}VW7ZLFw;TBsF6tvlz(Tkz+~H?Rx4uq%dL z&A9AOq84hQRt&Z7y)-Vh*y4Hdym&q*p0CEG78|ut3$=2n_3-@oWG}VY*o9r#mBX%P zT#|VimyAos6_ardH_naQifXYb7v-W{S<0mr8?{giwQ{I6+#HolEjD&x7k1^atA<=l zwb()~A(xOVC*-<yZroQ?i%q#G7v&05E)P?sT$GD)<tW$f-Em)0EjHz%T$C$Hxzu8V z7HEN14zz~5yJN_u78|_43%qjRRreLuVxtynp;ivHh6l4T<Wh?bUf=~@Iq+)ArRdDQ zqJ2gCiaGm=kGgVspdfgG7kI_MYpq;rvCWC*L~~-!oH#t4sl}$6RFi7Psb&=}OV`r1 zbmNw8jvyWPnGVlR|Myz##cvnRs>L>Anla6o#|@^<zFys*REv#dNQPvaCC^d5n_6ts zrfJi(8M7zZVb_F9EjE7P7k<U?OD(n$(THe7>>iVI)QrpiB;#USj4Q^t)M7I(#>KdD zJgc8+H7>Q-_=R8i)u}M`BemFwg;<C+bxytR{F-s8#g=i&xMW;08JAjY%hhs?S+1`W zkL~?hRW7yIMn)r}kuisYYOxUuu@EbVSp7_!ajC@?a+#V<%`sE+pcb2QQ7+2WMJDH{ zDVJJo%0;;-SB`S2#im@8i*n_NRzK6Va;e3JFZhD54uz>7sl_%R8IWQIq^aX%v*%Ys zF16T{i*iw}&64LR-%Twx<)U1aEAq)!O}W%!+gHSJ43A;>pcdPdXi79C9%XL(zy5FC zS9B*A<6>NlE62FhVj~t}Ayy2ry53dciVWxakKkzEC+HW@7X1P`QZ4rK_xFERi!CA( zk%`E%)uHZBs>Mb!Btx<slBvZut8CG%lCv<K>6>ztCR}Q<@e99%{V`#`T5Kbt5z&a4 z=y%Py>~1kG#>Kc|j7u#x<6>NlE7iEvV&fNn;a3j7)M6tRVj)%zvHF?TjBBA5TgD~h zl5xdkTxzi`SIaeKxpv)fw12%Sms)Hiqmj|bn8QQ0*ocK#h?PUEex}X1)M5*{OwFd| zn5kJUw&iNM#w^#a8;)LtTpp%MxiB2VV;DZD#g=kOxujf)3b#(V)MCRIe8E=^zSLqH zkPJvM15yrHHgTqPUr{YK<)U1aYZJUVp6{U+n{rVu$`$!!tEOCPvF$5jIEKeCTrIZc zYPrTN*W;P9PRgs)eMNV2F)j#)@EC*-YOxUuu@EbUSX~#Za7Bi5{YP;0raC9@kay@L z?VKN}7JKiH4=$<2K1r7YyusTxUe9%ZQZ4p2e)5Qic&XRDT5MsNuuNE%8J69jtHs7I z{KBsoeyPPyRb^wIF>kxC-4=1#pJZH&i*dymms;%Y{;kE^cw6cUsK%uhJN45;9^~1^ zYgaAyHh$`8hj#hbeKRh#*fK5|my9bW<5G)lxmvC<%atm>s>)Tl)MDqWv#a{mtJ=G9 zvfmB0*ofuv&mRAr$oXbmYO#e}re;%f%+#zF+j6yBW0q^87`?r7Zp!6hs+0@EF+7Ih zYOyU>%Qa@XzCC?OeC4%rsl^u33+aXQIrvhG&A1pB<BELDLJhgpVpA^4MY)2M>%naG z_^>p1d7hwzo`4e27y)h6Qj3jRsD)ZF)KZIWxmvC<%Qa_~+r*jHeMNV2F)j#)@EC-v o#TLtp<;C)w5TxV&9#yy^_d}2SjqrLW`<Vhh-;?wC9-$WdUmiv8M*si- literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust/nc-1-big-Statistics.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-Statistics.db similarity index 64% rename from test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust/nc-1-big-Statistics.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-Statistics.db index ef1355cb6e1d7bb5066bf18a7c4aaa0c789f11db..78bfef8ec52879118a35bd73f589c7db45a8189f 100644 GIT binary patch delta 192 zcmez7{oY)Sfq{Vqh(SOFh#7%67KoXExCV%s85pEf81yF>s)|i?QJk154^ljR<Afs4 z%|TqH?50yvxdUG^NdCCMz^1)wD@$txnC7{qUH||8e-H*L2Lt>5BWluf&3u5Il+Awx zCvn<Bv@+ON1BHOrG9vMzY@o4>Z!f7aU2=4=YWuUoCUe5&*MGSa*tJ+ACg-WP0078U BI0FCx delta 328 zcmaEF{>@vDfq{WVO;ef`NP~b55HkXCCJ-|LaSsqPGcc&8AHS+6!Jt1mPg-oEjUr3J zmDAHF#>#VKANh9=sAA5>h9XWk#%^DA-Sa>$6foY3nVJPM5DJ(qt)JCrz<5COSBtDO zszvabjW=a2`m^~3XFa=VTXcl(GY0NOpBY%cT)8meg9n(NqB5iY|Ns9W4AW*m%lr4* zJy``H{^mJ?lQ^v)S{aV&0!4scVMO9X*&t&X7<DrByt~T%jxKq^$dV@U_Y?!;<XmYj z7NDM%%^L+bv9Pg$m>V}w6zgP}yg^KZpV5HPM5OQCl|NUyl(rRkOwN{WVfp!MpA!Hi CcVBA& diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-1-big-Summary.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-1-big-Summary.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-Summary.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-TOC.txt b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-TOC.txt new file mode 100644 index 000000000000..aaf69afa269f --- /dev/null +++ b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-3025-big-TOC.txt @@ -0,0 +1,8 @@ +CompressionInfo.db +TOC.txt +Filter.db +Statistics.db +Data.db +Summary.db +Index.db +Digest.crc32 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-1-big-CompressionInfo.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-31111-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-1-big-CompressionInfo.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-31111-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-1-big-Data.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-31111-big-Data.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-1-big-Data.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-31111-big-Data.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-1-big-Digest.crc32 b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-31111-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-1-big-Digest.crc32 rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-31111-big-Digest.crc32 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-1-big-Filter.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-31111-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-1-big-Filter.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-31111-big-Filter.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-1-big-Index.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-31111-big-Index.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-1-big-Index.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-31111-big-Index.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-1-big-Statistics.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-31111-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-1-big-Statistics.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-31111-big-Statistics.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-1-big-Summary.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-31111-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-1-big-Summary.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-31111-big-Summary.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-1-big-TOC.txt b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-31111-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-1-big-TOC.txt rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust/me-31111-big-TOC.txt diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-CompressionInfo.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-CompressionInfo.db new file mode 100644 index 0000000000000000000000000000000000000000..dfbc93bfef7fe78be6b6e6595b5bac45a4237431 GIT binary patch literal 83 zcmZSJ^@%cZ&d)6<N-ZwVFJb@zMli|1z^X6}$YO)?pcL~V2!nk-gyyb>(EK3~T38<d D*@_5l literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-Data.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-Data.db new file mode 100644 index 0000000000000000000000000000000000000000..4b3dc5714cae62564f3728f424aefe330fdcd346 GIT binary patch literal 5213 zcmd6r{c}{+8OQHUa*bKQL$s?=m|j$LAY-|keM@jc?wYnvFo>a65Gg`lNH!smm)#}V z%p|)qK#^kV3j-N2K#Z6>Wu`WvEm~U0?#{G>Rccd*0VmB+w6vv^B2}TC#@pvP_wAgN zKOn#5`<(AN``mNS=XuVJAc(@Ng6K);x}Jo;bzz>cZP?=}^7{ONU~$P}mjEOBKD|4+ zod`lQ8Q)MJt7)j+6zz)Dv_|Wrk(#FZx^TQEyeYD&qph=Z%f|So`t~gwsvA2t)Wsro z;kM|Oj&Q8AX>(gRR<|YA9t}5jv{ctLw>E9=Y-o;!H^yV(#?DAQS{JUaj>H-xElth! z4Xv@Z>PSPpt)nfxv0-y_OH*xKC;VzlE2KA4v_!(B1`|8CmkOgBUJgm^%5YJ;i)MP; zTRDm3_qT_Hq1j~+A1P98bD1z2^Ms`6-QM#OF(2%Q$(`UG-S#2GWnYkb_rV%jAxYUi zfM)-G!7^j8?^$#Pc<1Ynp)<%k4;P}d*l}iuB>5X?Ea8oB-U&f6ycqNEyKOuVeeRGH zk-aBr?8rLEaOfsHraY`F=+cz(MirmVd&+dE!sy%Y;WA0O4VrRn1eGxi>SvsKhgHY- zbE=<Hhgo%EE31Ax4|DWcKQZ<hO;BHGB>2Q)nxHfMIXn04c~qms@SYuK$0jsZUFu}j z<!V-4t>9E8qvY#7{ZZL(C3oGIbiwYOns5Bz^80CW&Tx>Ks(g7btNsITJxT<(F>~@V zJ2v?bM-8bdag32l%02l@I|UeZuSvSBH!`H8?p>0O!27(<Sob@w1SVKU<&kypI-j|w zN)fWe39KE(Kq&@3t)hUw*81Qj0jPZ&Ni~>t2?z89oM~8~TALQQ4u4a8gWM1Y$WO79 zlw_-td{mWM-Vg<`C;<^HJHk_v?D<PJj+2UPeI*b~gw*}BmrE_{DDf=Bsfq^-;)A*R zN)i@vmVt3<!Jp`REecVNaT`+=55YT;y#;yp;#o<TQn3J|zMG=G0<yg<52OBbP9lin zObiAXD+UP$gNzgBKXo0W#f%lBPvcpaOBgRMK#%+^6|&Ni@6@VMO1#Ol0FDd?yl0pN zUQ<^68dkMvm)BITj@Z>elh-U;-70+bv}~3KO=<Cr66@?~N(U^@C`iXqU4P@$6;Az= zBfbZUCY)J|c>jK$_?|zob9>IBGKrUadY*^vx9HsZ3x^C%+4?JHp1k#BE#urVz&dxl z$vBlALztb-^sb#WI}3S#Kac#bAFy+G^>XSJR2K5RZR}WYlv5p?>SEQa&v9z4@j6Z3 zOI|>j9o$o;VGH;ED%yG51My}GH`k0W>Pbp>DzsvH&|UIUmVSl2kfUEDpNmJcR7v?3 zRQ?D_=jxx3$Fh%-lY)K)p)WO|_a@-TQ|NnLvZo(se+3K=7|ic2zcuskhGF}l%O`IK zD;a@VCi43i{vYzG64-dzdnVUf8i_wm+4hwK+s1rE^L+}ezt>Iz$@~Im2YBO$A7e1c z8~=#0V#kP+FF=gURN)Z~tUpR&fZ_a{HkI?E#=$Tf`@z{f=i~G&T3vm74=!hMGro^g zTpz!{oy7~?tn*W@i(cc}-3{R1?&?EoI`>7Io(1{AyI9C!d<H`YN{WYpIY_eF`CXLX z<K;R=D(Snrp=Fyz0nGOlI;^9_tCRWYDB&jO&@RtWH$yiyV256N@a}}&Yc=KTaCfq% zYRbc>IrTm&gLPQ8@F(nG;W1AAoKx?zYEd7j4luAouRl214cP(hE30UqzOn52Cd#&P z$wtEIeYDV`d*I*GgYM!Fuoo^OGjjFy2=RXbZMj6u($|xAY`lvQ_#+n(_{uv#E4K3! z6L((%!W8!pi0sLCW>VhM-fg$!Jzb_6ywAkGy@-@$>+22R(?R3rTjcckboo5keA!zL zNm^+E@#PO<)OW^s0$c$BeHerOEGG{PF2rDfabj_J{C126r&wn|kjzjaD++k}@jEbJ z5|3>=AGHH~dg-;P!2gQ_KG@6yzw}F7zyw}idb*Zr?8=kp9VRqo1uZUCHF)MnIMGt8 zU*^<noZ8E(wNNQ+;x%RWvmEfj78ZC--oy3dJ&#~M);+%OPIlcs_=a)v=__EAeBjh_ z8+A=S@ZtzOq<tMYbdkNR9Qrd0x^n2RG(D@WKe~dZXLtQU4(>q;x3wjMzrz%)wm!s> zehm7HoveI3DaY)cJh7PF^An|vQ91R1@h<NpRrG_oVX$C~f^FT$gT=;&W*a3tB%~kx z?(`r#cn`fJ$H}2QH<G17YO9d$;^TswETEtLaYX((ioE6Cnd3&}%V>}P=xc~U7g43T zc6;|HAVhhe>6Sgean>?;fA8F_d7r3<jfDQbl77C%3I5LM`O29TeZ4H~6OHf!WbaMM z&Jx@EL^DQx1?EHHbU^sx6BzZs<79!fKZ(J>6ze7UjVN)cki7x+B|ioj(l66K$&fx5 z4JPSGzC0Cl^Kh7*eN5v)zX}=Jpet7;RMt_winYg9pTU`D)Zbb4#qU`)d6-j!oO+Al zd`QiRyExFt;HwP&qhR;njQEl>8|}29Se-qS$948h9@p73d2DA_GkG=KQnSi=`E6cS zaKO*1WPpbbJ@#Xs_<Sth_EzOT!YKun)!Z}rUT!eJDmPcV+)VA7WqfQ`E(sssJ5S(0 zwj(Ci8F#_7z<R_#SL`CYDZP|{0=W7BSAk<Wc(C1MeokCeB{hBa6~sC?M~qu;vj}|L rMT-FA$0E>8ivZ)sA~3NYqGZ(^EC9IrT)9@2Hdf)k>BPc)%bNZNenf|$ literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-Digest.crc32 b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-Digest.crc32 new file mode 100644 index 000000000000..afe9d4969179 --- /dev/null +++ b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-Digest.crc32 @@ -0,0 +1 @@ +2310485613 \ No newline at end of file diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-1-big-Filter.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-1-big-Filter.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-Filter.db diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust/nc-1-big-Index.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-Index.db similarity index 99% rename from test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust/nc-1-big-Index.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-Index.db index 8b23beed9049e49e2182187f19e855f527005e72..a4f797dfc9fcdcbfd0c517c416aac5d81be81e9d 100644 GIT binary patch delta 130 zcmex(j`QO=&J8zJH-A`Az{$w5IdN4Hh}vsW2&A^hS}+2+?AtFmG6MPR+Y20-fPA*i z^KODE!N)*Rw(T1qF&1#LFdux)IQc=96=T!(gg1<W)ocumMu&LW;T+?`!aZ<~$q^|* G2nPTNRxE=6 delta 140 zcmex(j`QO=&J8zJ8JRbKT#(Pn$i6voRS}5VYf;F_!hY}?^Y#P_MxX@i_6v@TKp|G3 z&}O|0KoRE6^KJr_GH(`q3>0TRc#V5=eqjzL>+in@U$ahrP-VqfwLRetqhK`~1EbL) Up8s%;@nPW>ILG9O6eolO0Jah{w*UYD diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-Statistics.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-Statistics.db similarity index 64% rename from test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-Statistics.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-Statistics.db index 4a3ab075fc509ccea6cd53a5ad6e61b304f51bc3..a2aaf2853b6b480d82a5c91487fbb1327902e908 100644 GIT binary patch delta 196 zcmezG{oY)Sfq{Vqh(SOFh#7%67KoXExCV%s85pEf81yF>s)|i?QJk154^ljR<Afs4 z%|TqH?50yvxdUG^NdCCMz^1)wE3Zldm=>^&sQ>@}KL`VrgMoej0?)*Qo96&IDVzTY zPU5tMXk|#v2MPhLWkljb*+63%-(FH<y5#6!)%ItFP3DBlum5r<up2E&n4G5?J-I+t F69Cr-I*|YX delta 351 zcmaEF{@+`Vfq{WVO;ef`NP~b55HkXCCJ-|LaSsqPGcc&8AHS+6!Jt1mPg-oEjUr3J zmDAHF#>#VKANh9=sAA5>h9XWk#%^DA-Sa>$6foY3nVJPM5DJ(qt)JCrz<5COSBtDO zszvabjW=a2`m^~3XFa=VTXcl(GY0NOpBY%cT)8meivgIPqB67o|Ns9W4AW*mBO~QS zT7n^nzj=<}Bu*=cR)$jrKoOu<7?Jo;Hpo~8Mx9JO@2+ycqf4GJvZP7;J;lH{IagYX z1*oTG^G3l<ENpBb=Eltv#X4CgZxGYqXEb0m+4^D6^FLR*l(v-^OwN{$W;6tbh=Qnd YeraAwYLRn(PHAplNM%9l+k<l~0MLPFwg3PC diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-1-big-Summary.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-1-big-Summary.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-Summary.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-TOC.txt b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-TOC.txt new file mode 100644 index 000000000000..aaf69afa269f --- /dev/null +++ b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-3025-big-TOC.txt @@ -0,0 +1,8 @@ +CompressionInfo.db +TOC.txt +Filter.db +Statistics.db +Data.db +Summary.db +Index.db +Digest.crc32 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-1-big-CompressionInfo.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-31111-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-1-big-CompressionInfo.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-31111-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-1-big-Data.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-31111-big-Data.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-1-big-Data.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-31111-big-Data.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-1-big-Digest.crc32 b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-31111-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-1-big-Digest.crc32 rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-31111-big-Digest.crc32 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-1-big-Filter.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-31111-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-1-big-Filter.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-31111-big-Filter.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-1-big-Index.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-31111-big-Index.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-1-big-Index.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-31111-big-Index.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-1-big-Statistics.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-31111-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-1-big-Statistics.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-31111-big-Statistics.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-1-big-Summary.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-31111-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-1-big-Summary.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-31111-big-Summary.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-1-big-TOC.txt b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-31111-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-1-big-TOC.txt rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_compact/me-31111-big-TOC.txt diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-1-big-CompressionInfo.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-31111-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-1-big-CompressionInfo.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-31111-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-1-big-Data.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-31111-big-Data.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-1-big-Data.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-31111-big-Data.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-1-big-Digest.crc32 b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-31111-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-1-big-Digest.crc32 rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-31111-big-Digest.crc32 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-1-big-Filter.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-31111-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-1-big-Filter.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-31111-big-Filter.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-1-big-Index.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-31111-big-Index.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-1-big-Index.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-31111-big-Index.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-1-big-Statistics.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-31111-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-1-big-Statistics.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-31111-big-Statistics.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Summary.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-31111-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Summary.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-31111-big-Summary.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-1-big-TOC.txt b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-31111-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-1-big-TOC.txt rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter/me-31111-big-TOC.txt diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-CompressionInfo.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-CompressionInfo.db new file mode 100644 index 0000000000000000000000000000000000000000..c0a4633f440ab66c945023af012bd9cc8beb480d GIT binary patch literal 75 zcmZSJ^@%cZ&d)6<N-ZwVFJb@zMli|1z;fgSki`n+K`G`n5C*#ogy#AQq50+l0J=d5 AQvd(} literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-Data.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-Data.db new file mode 100644 index 0000000000000000000000000000000000000000..5f592c4e84d1152e79b6327a490dcdfc7cf5983c GIT binary patch literal 4482 zcma)<e^69a6vyv_MO}1w1__j-FB!%km|k{SVBIv78dF2XAN|m=$gwar2aAe{3P>46 zD<Z`oqtlE6?$gmCDI4=w>@JNzMvbT((-~rS$|))mM@uq&opaybefMttA%8G@&;8zW z&)&VCdoF}1WFt{BD-FY_g1-%NAE`c&ndxw5xw3O|-4o&oj2K(5#*bDBNheAb`MqOg zOvjYTp0)q%nYgU(n_s`RL3jWuf{87TA8crEs2p3}Nm>_N2-$|v1CGz^EJr`Dtt6v5 zM0Pyc1>qJ)FyieZtt+1i*;107!{XHNijX2iT3>NMSj)P=tXm<260*@ZW6<lGNVpe+ zHJ_t1TXxpfpfksE!c6ULH0H|2`c-Ij%f?cOsd_dRzpY<|9p}CpvL$QIPg$&~2nyUe z8a|_Tpm!{~8px;F^s|rI#|xWu+B6u`1lS%lC`}2pDG38+4j4`rm3~7~HzjpjRQtLF zrF}AZRB9yU7tgoZleH|qLSQZ{Q5_X2@$hz5qUCr@$VE?hQ0-1&c4do)o|q&Ke`Ocd z`R_$_(J!jby^`85s?htmlP+FksGik0E)5sewfwTY6wP%oyv;$<8rmAL!<h+(bgkRi z9Wm&%XIh&*MXgC<lWEVRxR5znr&lutrTsnjWMrO~`r-o3mdx|T9rY&gnch5~y@3_A zEY-}LzanT$R-b38sR)XiItK$wt-!1i7$`Oav%xX9adEPC;5jouf;MfC-<~*IA&MjY z8YXQi%20`K$g|z4YWzG5G!=CbpL<P*J&|0#rH-CuTq$HAo+-W1M}W^}_XN%+a`N8Q z1jNL5PyYX+2@?{P{^UKPFgP6-V*q7!PmJ*=&=rJ**23OsP7egs0XU|#*{ORy2D2<L z(lv2823?jDVOWW=Y}q)u5Q90gaURBUW#hwm$~3oZWCyK#4DhC|jx+wmm8Xq>gF>B| zvN2ttPN(TNf_S4i`FB2T6y#Rh7;-u-U<a0u6%4{Lu8QCoT59QHNj)p76*D9?Q&8H9 zNz-A+c+L_>n;otMlQ$^tCM(Vwb_r8yNiHfcgH5NUsp44aP)Q9JlvdidNLaEg1_9s& zl*I{7S~h@@#s|xP5lN%zw^?-@EL#4XIJf){NnJ(7F{|hh$11*)6x{q)J*y8%>WHL{ zON@;U$&C5#KP5Bf%WM%C0}6ev?HJ&U)pu!CL0jS`V9Z$wKvk{#X*RaBF|!b_h7#m$ zZQ~VNqEg(giXF)J;m52JU=DupZuf220DGg3VWjTW4sK3LQg$bVLG4$_`lIu!o$oU) zBUrlQNqLycV}*A@W!6cuM%}{N>_DW7!Qv`l9dDp|uWb!d;B&*KY1vWM2QZi;SaJS? z7|e}wPQjpCaAMLg(;%wG4=-2eE8!9<Fn$VvLF8IvVJ9lo+E`;SCI)bzn>*(z#$Yu~ z!67ARqqQ%>Cgrs_9fI4uUCyxM!YMp+9T(%L@XQTSwM)mP{gO1+DXMS%qWTdZ4UwEz zQt_ZiIQy%ceV-h|+LvVvZ%W7aCOf_yL*uZR*EnV=Z=OGPrzHF16C+#$jP}RZg&=EX zX?8tx7rfLzaD6fKhj7=UwPk?_#gHm#vNSg(Lzd%EnlOTS=v~&RPKVwtsT4`wE2`mr zqDq5D1zcTb$Ab=7TONhu#)Sfo*wCI5XoQaE+p_ey-0>cUj)yd6Rl<p>3S5e&(VzAc z(%8_&X)GCA-9`N4UtR%uEExSGPvho<7#7%l{m!?ra<gsf)LAX5b&}eEYL~}w6|y(& z<>gC8Flc!z4L@@WOm6)-%kG5R9N@w*3e-ihMn2;40xY8Cu2qbDzzIh%Sb{-!l=CGl z$CU`4nfQSop`JM=aKdEl#Gd&kaqn6SA#o$mhKUpGj5xpN3Czri^DM;0vsMj=l&N?V z=!2~9x`eri{8VTXaKA(qIuO%wWT*E^>YAu#N&wGkl|ZZlV1#}+vRwwTnteNbm6}zU zmxEZtp5T!&czs*-)VwAr?H#EF*G(4tVckQ5lh#ee>LLxVZaS-uGnfJGMR;h^dVxBf z)-&jMJ@xEq$@}3B=}K!nC9J9G1;bJ!8@X@0SX^_JV5H4!1ORoKaZu*ZpK&OPzauh# z{_)d>$^3l~=Fe3L$DnH1+5hl&WCHT{&)1wkI7-0Zxf84q0l+WP*v>cv1katu+Bt$9 zdnC0_QXioL1mQ=g=QpBiU<x~}|NLeQG-6<SA^zK$x%%wU-6z`!X^#9Snh6iy+%TaR z%kP9x_Zy1gR%alLk0Xq4;v!G?GbWhV$RZ849{b!j<m+*@IM#3@cAd|_M~6oDx-}cI z3)sWh33d~^&Lx<bW7k;>O3Q+E_}YkF=Q51Cgt>@a2o4`$7gc8g26LjUmz@~R6|9(` z2Yy*d^w>nz`?G{$@2~6|<^3=)NXD?YN1S`T9Tms0PwIz#LO;}LU#mFp)=o(^O3Ev# zy%L7E_Ot3Zh5<N@7KYxufFj%yDBrbmp{zs6n_)oNB#p+gJH@0H%yGh{f>O9uPzu)y vNc#d-7w?Gv(-)$MJbfvO$WDPs>{I7_7~t<yXqjz6TS6g$2Bfs_()Rxg5`w8c literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-Digest.crc32 b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-Digest.crc32 new file mode 100644 index 000000000000..17149972ebc6 --- /dev/null +++ b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-Digest.crc32 @@ -0,0 +1 @@ +704145780 \ No newline at end of file diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-1-big-Filter.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-1-big-Filter.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-Filter.db diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-Index.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-Index.db similarity index 99% rename from test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-Index.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-Index.db index a8fdb3cb11064754fb053a6a0547390b65fdb58f..baadf5f36441aa851b8ab3a9d20ee120bba1f02a 100644 GIT binary patch delta 93 zcmex(j`QO=&JALr+lw5T3OE_hY@T<sh?6m7v*6=GPR2XiH$Gx4;ABZW_@;95gDNXV hmF)>{7zL}@7#NKXzVL-}j1S!@gL6y{UziW!002YfCK~_% delta 96 zcmex(j`QO=&JALrjGs5_UC8HTJiU3|%_2_5;LU=M3prVCAH4B<bADkiCrj+XH?@-= kR9P{~ZBKZ^C|J$Lz-V;vg*%*MeCSRAoMUqM!b}JU06U5)6aWAK diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-Statistics.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-Statistics.db new file mode 100644 index 0000000000000000000000000000000000000000..c481e489b67349a6b26034f4ac5b88f868595d42 GIT binary patch literal 7160 zcmeI%c}yHt7y$71X7=FP6&88`DQXLCMYB+j0tUJTwgrp{l*3ZBk%65Q8rUVnwo<IA zX{jbeacx>@lO7%kA={>;U@c&+f1seXTDlrhBMl}iYMZ*+v`AZIoe?&C-_u0nUpCH5 zCNn?2x3lxk`LYl~G-S5K%O@%yo8^O(kDc<t6B279Ne*d?S!fh$YDII6(9|T@ZIWQN z)jG|2JDg&%WD_OPRw78wI%l23E=rcxmTwDAMm;0nAlC2q&lsYqcHi~A)HQkUjb1m` z^X{~Ee0nTg4tL0hQk3t7g|+fLu*lDaN}vsaKWYOPb7vi6J>ww8A&kQq8yQD1j$*u+ z@e;<%7$-7LVVuplknv-Tw=h=57M24L@^vrcQEbYL2VKp?os7?6bBh@7#^#y(^M`R< zQ-Q5b#n$Cu>zVcj-NEsYVeBv;_C0&CO|M}ud>lK*gMD8T_VQ!c=1T00BiMOou}c<U zS4CqtZNP35usfOgJ8Z>qU)-E$*Ab2T1;uG)Irb4eKdJ8v6z9tDdgjh!^Oc{X`Wg+M z|C&n|P+a?AJF@O)H*!#!A31nNL=K&aL^j&+e2lREfZ|d83y~LJjzL~BT8g~vA|CI= zDLme(uJ=&=obm6FAIZhfw<*Sp;#;qoke@5=Mm}^a6S?nw+&^P|s6S~)_)S=!S_)o7 z-nFy~`QyQ4Xl}(Bw7+t%8b60|KBgJX559kMFpO)OgV#W7pBzQ))j8^#VO-zmorDgy zuDbyp8aIH}lcCZy0^{L)(GYZGMgp48i;TAp!g%cQSFS+cpH|odz2eJ_+0aQNwI`vo zs-8UxZT&iVBXsGphIHtfgtym1Z@+DrfNuZ5T>yQ&?9fu^PdZv(hMqj_SqDs48mjxD zYx48q`ls%aGlBK5RHOd=M!5bt(R=wF@Of_h#F`dh-buatfHm<KO-0aCQ#;}KXn%Db zfbS3e1CbNCpng!tsg}2ZLyBJ?l7PdWYX1qg-}qI;71;iWONG5AP=A5Y)(GDR(Os4S zGl<81G+~0{6@R^;s13v)s_-Sid>}pb>W~q{Eu-HLJPrKl;qDXgd7kiPrNjAbFST~M zK>gP0zTt7;qu-33?F2se#+%*ndA);)o{af8HSM`Dod;aB_2!-3^Cp33!+Y?bb1)8k zM!EdI<Xp4wJVH{K_M<f}$JCq)bKq3$66I_H|bjhb_*ITx5vRL!}1KzES4E9av7 z#n25ulIW=sqI=MN+K@*6IEUwK+9m%!q8x#?SzgN7Gk89#L$)pOH*lt!Q_0DzQc}~> zS7&5qsaI+rsPjNw3;*r?K;FLOZA;y()D2sGX{j&nyC!)FWL>xCF68rE8b=&2D0>?A ze2Pb}d6sjY+_eusKi!pG=lbmGZ=qHC6y+DJm4E$zv}$vMAT`wr_2!E5^_k^68b$h# dYZA(D7wT92-Hr1d+w4wJ%6HUnYp^Td@H><BnX&)? literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-1-big-Summary.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-1-big-Summary.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-Summary.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-TOC.txt b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-TOC.txt new file mode 100644 index 000000000000..aaf69afa269f --- /dev/null +++ b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-3025-big-TOC.txt @@ -0,0 +1,8 @@ +CompressionInfo.db +TOC.txt +Filter.db +Statistics.db +Data.db +Summary.db +Index.db +Digest.crc32 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-1-big-CompressionInfo.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-31111-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-1-big-CompressionInfo.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-31111-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-1-big-Data.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-31111-big-Data.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-1-big-Data.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-31111-big-Data.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-1-big-Digest.crc32 b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-31111-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-1-big-Digest.crc32 rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-31111-big-Digest.crc32 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-1-big-Filter.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-31111-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-1-big-Filter.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-31111-big-Filter.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-1-big-Index.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-31111-big-Index.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-1-big-Index.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-31111-big-Index.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-1-big-Statistics.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-31111-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-1-big-Statistics.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-31111-big-Statistics.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Summary.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-31111-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Summary.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-31111-big-Summary.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-1-big-TOC.txt b/test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-31111-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-1-big-TOC.txt rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_clust_counter_compact/me-31111-big-TOC.txt diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-1-big-CompressionInfo.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-1-big-CompressionInfo.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-Data.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-Data.db new file mode 100644 index 0000000000000000000000000000000000000000..a630c4ff9c88d04bbf9c7db8c8999dfcdc997ef8 GIT binary patch literal 89 zcmZ3>z`!txfzhD;|Ns9D42%pv*;M%X892Dp^79pv5{rPiijjfQP>~@<kpJ*rPGtsZ lBV`7gqwW)cEMp++n3f8VWunYrdfeSZnIQnILC0jf2>>j77MuV8 literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-Digest.crc32 b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-Digest.crc32 new file mode 100644 index 000000000000..a51df1bb9234 --- /dev/null +++ b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-Digest.crc32 @@ -0,0 +1 @@ +1415075435 \ No newline at end of file diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-1-big-Filter.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-1-big-Filter.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-Filter.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-1-big-Index.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-Index.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-1-big-Index.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-Index.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-Statistics.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-Statistics.db new file mode 100644 index 0000000000000000000000000000000000000000..5db2740efe37d936412d826da75f5c7c9d1c4404 GIT binary patch literal 4704 zcmeI$drVVT90&0G=mRKpwBQ4bY*X-oOEE)47qo(Pz_JA7G1<e63%wDHJZu|9=HlXT ze~^WBM#oHi?2!;sjbt(>Fz5bogUu&)i>MK0Nyjz~R*f>8NO$Xc!|!{tE&F@vNlwoF z^mlsiIrp4%?`b)X({O*KWToJhyjIDAlD8^Z<T%p=msaOkXOZfq$|~7XDLI`|t=%D6 z>{TvH&K8$k;IPXM*<L6)T-C1Xx?0&`ZEE}}_i+3h$_Zu!0)dH`1YW!I?hf8P`sBS% zk8tFxaqZCfV5|~uRdO%|uS*l_V0&m%+677NgZiOXJqVRL#(KsPjH4JwGd40dF^*?E zo$<_&)ppKi;;D?EWt_!0kMT>4*D(%`Els;Al;cju18ANZ4}LKdcQHPJ7N#-Yjux5o zi+eGyDMf43(Yj@5J=5QahZv9QLr443PwYUOKSED^1)b<cKb3}_a{z5ALuc+o=Nv~D zPC=I^pq;DH%@VqenZJEDjQf+oe9|3sSqXX{&QIQd5#mBIu4myCS}Z#U^)(ut|C$SD zAg=wc1+2U60Y?-Ez>yO&*f0?XHrjDMnry#9JidD>c>1+O@XUcC@a*$A-l=0a-s$eI zq5iU=pTRF=WBXoC^g(?6h#CA=K?iuxgC*duZ}9#McESDRqhjyV`n)ywUGTP9?cmcV z7g7szkHPa*_`vuhjf;s5Fh9iZwUIQgX^31-t$l3(`m3v}ZlG~}mv59h(zfazwIQho z)>BNGxu3>k#r$6CxXcuo&(n+#KBVy(2j9Oz{q&-|Bh+&*t;wQJ>#sUYy|nzz{nWNA z3)fH=9jM8mu1xu21@)#sVuq<(zV+l%A1vN8i~4M9(|gpTN4=|v`FSxF-PDyUSJL&* zdkT-y_0I=(8R_~LWZ$(f$@hh!L(3b9MHlbeNvugeZ_cM48{109NBg^bH+_HT=f@3a zlll>@pEZ6?998gfuY)-H^_E+-|HjLv8?^tX3wfPpQh$onTu<Kz3GLP%3yCLvKWwJs zm3%ihznR3JDfOq&{Xj<g&0ZskTL*sW*-ZT6-i|}GJ+JzgX3+UuTV!i<lln~+U428u z`>zfjZzDeO$)_E(y}pyF-pt83Z+`pCcn)#?`uh*JPnuKPraptUp_pp5s-PZ(%Kyur zOYK&8&!ygp$7OiW1-F9>@44W9g!f$Z9l~+pJy&?oMHa}v@45J0vSIaaT*6pCr(57T zDvofEOT(hPZOZ>6!K|LjdVS-a^iw@f#dh_xe=5}%w^D{Kz@-I$h$=5~>dA}qBi=cJ zH+#i%?~J!+Rl9$<`N&YNpI@VVW!~Ta!&70YksQt{X@jM-<oP8fTk2)y)Z!*-!+);- ECpRgVaR2}S literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-1-big-Summary.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-1-big-Summary.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-Summary.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-TOC.txt b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-TOC.txt new file mode 100644 index 000000000000..aaf69afa269f --- /dev/null +++ b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-3025-big-TOC.txt @@ -0,0 +1,8 @@ +CompressionInfo.db +TOC.txt +Filter.db +Statistics.db +Data.db +Summary.db +Index.db +Digest.crc32 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-31111-big-CompressionInfo.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-31111-big-CompressionInfo.db new file mode 100644 index 0000000000000000000000000000000000000000..0b7faea4a8a443b914a30cadb6eaba6c821efcb7 GIT binary patch literal 43 fcmZSJ^@%cZ&d)6<N-ZwVFJb@zMlcB?)*|x&xM2mv literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-1-big-Data.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-31111-big-Data.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-1-big-Data.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-31111-big-Data.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-1-big-Digest.crc32 b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-31111-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-1-big-Digest.crc32 rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-31111-big-Digest.crc32 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-31111-big-Filter.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-31111-big-Filter.db new file mode 100644 index 0000000000000000000000000000000000000000..2e1d5d29ca0683d58365c1bda385a89ced35f1f9 GIT binary patch literal 24 fcmZQzU|?lnU|?ckVPs%abZX%^z`&xww2T1&5k3Nr literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Index.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-31111-big-Index.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Index.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-31111-big-Index.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-1-big-Statistics.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-31111-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-1-big-Statistics.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-31111-big-Statistics.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Summary.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-31111-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Summary.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-31111-big-Summary.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-1-big-TOC.txt b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-31111-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-1-big-TOC.txt rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple/me-31111-big-TOC.txt diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-1-big-CompressionInfo.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-1-big-CompressionInfo.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-Data.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-Data.db new file mode 100644 index 0000000000000000000000000000000000000000..4243803bc10752e5ea10c837baace82f44eaf985 GIT binary patch literal 89 zcmdnQz`!txfzhD;|Ns9D42%rF*cUMJ1I4)0^79pv5{rPiijjfQP>CU1kb&W#{}&Yo nIU^MY?K2-#7-WrA7_1Lv0=aS~Dhw)Luc|NvfYo!DE;|4KU85ML literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-Digest.crc32 b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-Digest.crc32 new file mode 100644 index 000000000000..99ffe738abee --- /dev/null +++ b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-Digest.crc32 @@ -0,0 +1 @@ +2721811463 \ No newline at end of file diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-Filter.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-Filter.db new file mode 100644 index 0000000000000000000000000000000000000000..2e1d5d29ca0683d58365c1bda385a89ced35f1f9 GIT binary patch literal 24 fcmZQzU|?lnU|?ckVPs%abZX%^z`&xww2T1&5k3Nr literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-1-big-Index.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-Index.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-1-big-Index.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-Index.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-Statistics.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-Statistics.db new file mode 100644 index 0000000000000000000000000000000000000000..ae651c4e084dc7b893eeb1246e32ae5160150099 GIT binary patch literal 4745 zcmeI$YfMvD902fp?(KtTr3D{gWK+QhnqpuKUC;_v3yTTJL+dMYp*Mn&M@r)mT{I5% zfh^b=9h&&smJm{nsQ3VUeNe%-&@E#cA((Y+YFM37hRWz}wI}@lCtKoY*OQ!_`@8?s z-t+7|DaUaN?rn;Z3nJ&Wa^~f{Rn7v(X-ByDGV6N1SS}V8N%}&uqCzaSSVg_1$fi%* zY?Csr7Rf4EvP7$`*j8LtDp?KHRS!+aBDTn1Fwx`jjD$uK<*sKn#6I}J^JWKs^1`sP zZ@4E+4%f@so4n73iM6*qFv;ygiS~isRMHDysbZ{V9Kbk;aR_4#V=dzd#uFG%@|pH? zs?W3^aZLRL#wm=`882tNp0RgsVcG*BUpF)E#wN^skdK+TjqycnemvtH*aGwY!hRfA z<Y6llu~qZ2)l7c_UgLOB2X=@XdrS?s?lAVaRoGEZ><{Cyryau9o3WD*V5gnO&KirI zABkPD47*muZeZ4LhY`o!(a0qFPi%88_5r*;iTfuM=X2;fe|ww1fi0Ntp!y00UjK@& zTPUu)R)?&5>Oc<2@gN6|NXWq>;m8^bUXNPi?<gM8Iu3b4Zxr&R?rh|#ckz734dMAt zv|mE?=lA`J{Ans~-^wT#if<UuA#cxYLf-dsE^^BiJU%@wXgo<!*b7*n7))OwZ=2GH zeDm@gXny)x^!v(xqxliWg{VrjK7`h_fiSM93|s)MT+@yEt12t5gmHC?YY;lnxcE7A zaC95mPoZXACya*)8ST*F$+2iXkJr513*!@yeEk@D@<-_>p{M`6A_Y3Wv*;M~y!_7( zLL2YTSpl7Ws3ZxxF!szs=uNLe`=RTuI!w?<a`sJuzFl9v6MFEpb1^WP5n9j+UASlw z-2cRpbr$Y_;@Pc%`=6Iwz30H^`M#qIs(=L>aqR+D#N5?oKo1RVh4Z8Q%f1J`Kh!hB z`%^*vfcg_vr+|Ypziqbyhpemn1NL8YPx~15U)z=5tONDOinZnNeGu7bXw!pu^!0un zoUfQ?ri@w;pPlE9h3A2!#3$_<5I1!H*7h0j()~?GVS84)=Ow}QT$*icu!H*51uY$Y zzy}}noNoZW_}%wSu)VI!an9t?IMID^Yd8%!W5bKrJ4VgXw$a~Ur7uQJtMc^1SH3T2 zF4`@6=Aw?H7rruqG1}vD%zx%W1LQw*q2b{CXD&1z{xg^V%mo(2|Le>}c1yv_9&wRF zot$cx<8;Wt@i*m|Uuw3=|BrYx9U4RL&aPJm(<9jydhfqV`r?*L-vc=FQ@OJIB8MJ* zI2mwG<DIDsKlyUFF{Rl4!;`;*^VJC@$2o4szyHHipf3@v6-DAkeO~U8xw)ImCGzey Mg-zndcdpI-14;C%Z2$lO literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-1-big-Summary.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-1-big-Summary.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-Summary.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-TOC.txt b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-TOC.txt new file mode 100644 index 000000000000..aaf69afa269f --- /dev/null +++ b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-3025-big-TOC.txt @@ -0,0 +1,8 @@ +CompressionInfo.db +TOC.txt +Filter.db +Statistics.db +Data.db +Summary.db +Index.db +Digest.crc32 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-31111-big-CompressionInfo.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-31111-big-CompressionInfo.db new file mode 100644 index 0000000000000000000000000000000000000000..adb7fc457930e947bac28ae21637d92b336e2d84 GIT binary patch literal 43 fcmZSJ^@%cZ&d)6<N-ZwVFJb@zMlcB?HX-u>xS|Eg literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-1-big-Data.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-31111-big-Data.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-1-big-Data.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-31111-big-Data.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-1-big-Digest.crc32 b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-31111-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-1-big-Digest.crc32 rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-31111-big-Digest.crc32 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-31111-big-Filter.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-31111-big-Filter.db new file mode 100644 index 0000000000000000000000000000000000000000..2e1d5d29ca0683d58365c1bda385a89ced35f1f9 GIT binary patch literal 24 fcmZQzU|?lnU|?ckVPs%abZX%^z`&xww2T1&5k3Nr literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-1-big-Index.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-31111-big-Index.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-1-big-Index.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-31111-big-Index.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-1-big-Statistics.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-31111-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-1-big-Statistics.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-31111-big-Statistics.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Summary.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-31111-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Summary.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-31111-big-Summary.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-1-big-TOC.txt b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-31111-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-1-big-TOC.txt rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_compact/me-31111-big-TOC.txt diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-1-big-CompressionInfo.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-31111-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-1-big-CompressionInfo.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-31111-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-1-big-Data.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-31111-big-Data.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-1-big-Data.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-31111-big-Data.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-1-big-Digest.crc32 b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-31111-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-1-big-Digest.crc32 rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-31111-big-Digest.crc32 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-31111-big-Filter.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-31111-big-Filter.db new file mode 100644 index 0000000000000000000000000000000000000000..2e1d5d29ca0683d58365c1bda385a89ced35f1f9 GIT binary patch literal 24 fcmZQzU|?lnU|?ckVPs%abZX%^z`&xww2T1&5k3Nr literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-1-big-Index.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-31111-big-Index.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-1-big-Index.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-31111-big-Index.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-1-big-Statistics.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-31111-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-1-big-Statistics.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-31111-big-Statistics.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-1-big-Summary.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-31111-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-1-big-Summary.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-31111-big-Summary.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-1-big-TOC.txt b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-31111-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-1-big-TOC.txt rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter/me-31111-big-TOC.txt diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-1-big-CompressionInfo.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-1-big-CompressionInfo.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-Data.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-Data.db new file mode 100644 index 0000000000000000000000000000000000000000..b8a8f0e6a043bf14a9c8c7d01e43eece9752b015 GIT binary patch literal 110 zcmdOAWMG)Yz-Una|Ns9621bTYJPQ~#7#J8-7#JHE-d<8;y5#6!)%ItFP3DBluN6VU z91LEJ42*`>4Eb6N3<phS*fNO98QC)EuiXe_$r{@-*dOxz3=%W}3f`))W#9_{n#ahP I`OA?T0K_^SPyhe` literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-Digest.crc32 b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-Digest.crc32 new file mode 100644 index 000000000000..80dc60278f12 --- /dev/null +++ b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-Digest.crc32 @@ -0,0 +1 @@ +3723331170 \ No newline at end of file diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-Filter.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-Filter.db new file mode 100644 index 0000000000000000000000000000000000000000..2e1d5d29ca0683d58365c1bda385a89ced35f1f9 GIT binary patch literal 24 fcmZQzU|?lnU|?ckVPs%abZX%^z`&xww2T1&5k3Nr literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-1-big-Index.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-Index.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-1-big-Index.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-Index.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-Statistics.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-Statistics.db new file mode 100644 index 0000000000000000000000000000000000000000..8bdebb654758f621db600e8f1ad6bd4b840a567b GIT binary patch literal 4754 zcmeI$YfMvD902fp?(KtDD=qi{Bij_zv8AYtp$l4pvcj?i<T2Tt8JG6PVxT3L>X5m( zINS$fSZ8$1#K#^9A=StRa{_bjgB#4v2Ua87G7KgiZW>o+l;LD_x6%{-|C24-cYBhP zbAR`LdhdDNdr!)7oQj)HsY*eVyiv)#lD8{a;5fq+m*J8(nZ$b0=8#M_v7te9+GWvX zceqV?+uTyIY?oxoULwlw8h4G$Daq#Mrfb$?iLWUam>&!Vr*ug~z328W;u(MBPPdou z`eIT&GC3Ttgxi!HN}>D0#5&X-n3Q&*MEgKbs_8+v)G*dEj$$0cIF7NNv4L?S<9Uo1 zFkZwso$-pW>A2*CP5WKQ)PI5TCdQ$;g=rsza@@^$2%9kTL7ryfZpNpu`MHdDVhham zg#$RQs=!ufVry1oYnlE=-N*5m0qnQ{_N-ml#`m%3yo8<X!+taad+}jxQzdrxLF~Mf z*d?>EtCFxA)?v4Z*zL^v9kAedAO)GE{El5&j(w1>&*{@7a0SKrGP-W3PxIem3zg?l zeU%EYf7QjaD6amd6<IUpMUE;9B1cb2$gxui$a*_oj|R&xD4y6m2YKGrWaI@yrO1me z;Q3CU!1JBy`4ZJ%J@Pa1bGf*EFDCm@eDf_M@*BmS$oucDMDF<tkI!%q8cz}ve;3v# zX6tU`9Sb{<zdOAQnqP7Py<hov^_O8>NNz;yL+ITY4dbfD=rz#lmxoY)HLjXQ7}xgr z$DyMw1$UrhQ~J>Jq^mRz!g#z;)DNAIorc!)T>ZWGV0`|OcWyvG_EcdP^pfw_=Rju+ zI*viFs(SqpwB^TT>!C{z*JVN5(mq=Yz4cGsD0J)BUMuvGvi%F8&$Ttb4LyF`R{%_w z>Z*I8ZTb1|`6u3z6Y%*b!M%F;{PU9k>gVA1{K(NYO~8Vi`1b&-QZE>bpeH7_!}(GF z;n@ejAKJ$gMsq>^sJ2g<J_U{`{-9q5j(er`H`ssu4~84C|Avc&-9}J<w%Ae+zXwSj z<~|dMr<@rz!ud+QZ7pg6@h2++X>dP~m3gyY58~#b>wT{RKYyU}C~VJ$z^W{`o}HzZ zb`PlET-`G;0(|JF;gjvar#}3s6SmiXI^CB&6DP(u&raq67j3?Kf9H%j+BW(QR)=HM zv?@;z!sWxd=c2=ao^j=5WY0xM=0O?RbD`njB6}`09+5p)WX}Z_#KX1cB73FSb-!{+ z6N8-Q3GZ=T+}MM1)^*1Y<^PdTrbA=CV(Tif4t}iIPS5>YNk8058NLBm724P;A9CoK zo0D6<#k?<f?K5vpcI4D}F5Ubqwo1EV_ZY`5{pWvps!esG+~5#vO%>(Ot}NeHFOmOW cQ`jojF8{CV=DW5y-IAQ|s@+oO4BdeH8|@#hPXGV_ literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-1-big-Summary.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-1-big-Summary.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-Summary.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-TOC.txt b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-TOC.txt new file mode 100644 index 000000000000..aaf69afa269f --- /dev/null +++ b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-3025-big-TOC.txt @@ -0,0 +1,8 @@ +CompressionInfo.db +TOC.txt +Filter.db +Statistics.db +Data.db +Summary.db +Index.db +Digest.crc32 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-31111-big-CompressionInfo.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-31111-big-CompressionInfo.db new file mode 100644 index 0000000000000000000000000000000000000000..56c95a8a36330ee3340d0eea70a0c0f94f7b75e6 GIT binary patch literal 43 fcmZSJ^@%cZ&d)6<N-ZwVFJb@zMlcCt=p*w0vIzx8 literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-1-big-Data.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-31111-big-Data.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-1-big-Data.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-31111-big-Data.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-1-big-Digest.crc32 b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-31111-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-1-big-Digest.crc32 rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-31111-big-Digest.crc32 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-31111-big-Filter.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-31111-big-Filter.db new file mode 100644 index 0000000000000000000000000000000000000000..2e1d5d29ca0683d58365c1bda385a89ced35f1f9 GIT binary patch literal 24 fcmZQzU|?lnU|?ckVPs%abZX%^z`&xww2T1&5k3Nr literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-1-big-Index.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-31111-big-Index.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-1-big-Index.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-31111-big-Index.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-1-big-Statistics.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-31111-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-1-big-Statistics.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-31111-big-Statistics.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-1-big-Summary.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-31111-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-1-big-Summary.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-31111-big-Summary.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-1-big-TOC.txt b/test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-31111-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-1-big-TOC.txt rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_simple_counter_compact/me-31111-big-TOC.txt diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-CompressionInfo.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-CompressionInfo.db new file mode 100644 index 0000000000000000000000000000000000000000..916b491e39ce12e92b875ba07a92071483c7b0c9 GIT binary patch literal 43 gcmZSJ^@%cZ&d)6<N-ZwVFJb@zMlcB^*jSKx0I(nhApigX literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-Data.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-Data.db new file mode 100644 index 0000000000000000000000000000000000000000..a05ba8371a7e394206b688327253e6734bfe480f GIT binary patch literal 284 zcmW;FJx;?g6u|Lk8fd0~C}OHaD<QEKh=iCrFv1YQ1+e2dj`Qg{P8=8r2VmhmOsF6R z&cG!&KrkKt((gU#={Y+9uYeX$!!Y!K;P>VDdA|GpbMN+j7couZFs~mpscKd$A{gZz z;kn9NX-t!LIa@UmPTGiSMY*h-mTMyl$u({Cs^+3CG07@Xm^jls?KDq}>T1d{Ryt9o z$XHxyiB;UmR`N71vQmW9*so>fUI4(<HnKT3<icdjg}eWGGJ_eybJ!!>a1>s`Nw9CX z-YqOJ3CFOc5jy~Mg8-fSXt|CY9}V90%=+W{apQCAPx`j8Z~b)`SdW8|Jw@L7>iZA( CnOP11 literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-Digest.crc32 b/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-Digest.crc32 new file mode 100644 index 000000000000..6a41d2d577f6 --- /dev/null +++ b/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-Digest.crc32 @@ -0,0 +1 @@ +416438660 \ No newline at end of file diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-Filter.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-Filter.db new file mode 100644 index 0000000000000000000000000000000000000000..2e1d5d29ca0683d58365c1bda385a89ced35f1f9 GIT binary patch literal 24 fcmZQzU|?lnU|?ckVPs%abZX%^z`&xww2T1&5k3Nr literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-Index.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-Index.db new file mode 100644 index 0000000000000000000000000000000000000000..7cb91dc9b3815095523e5066ef8a1a589532f1b7 GIT binary patch literal 29 gcmZQzG+<z0U^Hwr0g^^d89>swc@B^?X}Q1v060Gd5&!@I literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-Statistics.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-Statistics.db new file mode 100644 index 0000000000000000000000000000000000000000..4fd6530c8610dc4e4ef61b8e43966bbad063bb7d GIT binary patch literal 5603 zcmeI$ZA=?w902gUYfE{pP}mqOL)~B-ND5MV9Sd6rD+AOB<Hg|%8Ap2q76zm}C*aJ; zWPTtE)@8_+>`hGw>6Q_PbAWSxn1U};UNoXc7M4`hEwj38*f|96MlbyTk1pk#iS|h@ zcfb2TchBqHJuO91N@_mEi#uA}%EgTlw>{#fps0vhDy805rQ;enOEs^vaCSRaXSH!U zYqg-uZW8zco0Yfm)<VuE)Ce{8b-c~g+I%(lOw^0w4VKT%&CQ0z(5izs_tTE)2XFN{ znX~WDs3vD7!o_f>xJgMW7benDeP9yng_7I``J_rd@a_W$s|g1Z4kjE*SVK61a1`NX zgjWz=ML3CY8sSXBd4!)LTt!$KTbMc^#N%GV<Jh!39^QVYSt2eFzKG346W)idknit3 ztvHI~%2I5V9y?$Iwwh>f;9VRK9>NZFV?VGTTYD0F=`+}|W7rR+V6Q%btt-Pe9>dPQ zfL*u*yCMeLz7@NT!|o#H?~obC-Eqit+;7-rCD_OC{G{Dqqc~HHpJ(P%Y(?1@D8EvP z=f86FGK#A{ZbuII)rlNfJck@K%Oi)(Mj~shcs@p$e?sx7{-wyvuEZj*7%xIzHH^nQ z$%DsR@Av@a-!S<D@?%-Jeox1`P<-bNEpkgi5Au;a>yZ0C#QihThx(Ha4!;fa)27@5 z$a`0IBY$=&6`EOd9=%_g*EL_kxFWU@%@0L?c@T^%8-q4NtDYN2?G31}X@qfgpKBUA z$eeQvIwWoYJx^g}+7TEJSL6>uM;a5*e2&)KISk{=PrY^x`r-7vv(Rh4*_H{NGE#j8 zI-}yH<Iv{sQnx`Do!Di7wj{i}33~S*VN=lUA31ZOPZb|o3H^Cz>#NYy=f-k?>9t{% z{m_=po8j|MI}6Xl=bxTCq=C;r!@I7$2d-x(Pj74nRtU7~Ah0riSep;+@$7-)qx#*^ z0ly#W#K@^EkUy~V?dErYgA3jmv;l{1Z~q0hU-NCmHQ4@$(Y#(Q$iIYZYk=Q_m~PX6 z4#eX=nbN}Xiocng-v;8zrS1f{9x&*y4{AW%H2&kj3&2kt?KutW^Q=3=0OxaEk-5tO z^0!v@4NU?c|9;{^7x2Y5-|B(&bzMptGcLqw?aP;EvVrq=-oCqU!IWG#{0{#68jJzg zNQd`ZbE&-f<h^_udMAh{{cA3{XXf`En2TO>c`x&?xp4pa*IcMS{A(`%nhQ(_|C;OH zTyxQf_>iqXQ!$<qD&P_4IkhwJ-ahoIXRrAGh_uN=Wg5yE70RQxh_=e-?zfkJ^obu{ zZvoVk(hm#q!w$VrIDKPmH8YmAY5glR-I+CxFR%X@QlU<ITwL6*{pWvpDs{U!o4uN= z)s>cPSy$53z*Ce;#WxBzZqZjLc5}5Y!@llc@_a)rFBRbPzQw$-sC@Z#f<8^knV9UW z@WtLz?u5idrP!#n|75H5EiPZ(RchyL_lBUBui`9C+Xb$&mbWLf7S_NT*+*HuF~hKf xwXz1o4r98}U`&-F;<15EH>M}EyqHA{uJ={U;-`c>E<OiZ&Nx4S>Hm+W{skW<*y#WO literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-1-big-Summary.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-1-big-Summary.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-Summary.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-TOC.txt b/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-TOC.txt new file mode 100644 index 000000000000..aaf69afa269f --- /dev/null +++ b/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple/me-31111-big-TOC.txt @@ -0,0 +1,8 @@ +CompressionInfo.db +TOC.txt +Filter.db +Statistics.db +Data.db +Summary.db +Index.db +Digest.crc32 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-CompressionInfo.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-CompressionInfo.db new file mode 100644 index 0000000000000000000000000000000000000000..916b491e39ce12e92b875ba07a92071483c7b0c9 GIT binary patch literal 43 gcmZSJ^@%cZ&d)6<N-ZwVFJb@zMlcB^*jSKx0I(nhApigX literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-Data.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-Data.db new file mode 100644 index 0000000000000000000000000000000000000000..af0f76965a48999c0eeaaa6d1154fa8f819ff7a3 GIT binary patch literal 282 zcmW;Fy-ve05Ww*>Ei_Y$C>uqz5)vZ?B2}>T1sE!r80|QY^BvcA?643XfQ6Np0Um)@ zK`>PcF2i5?-ASkG?*Y65nm=`2*8+m?m(%Cj=>7M|YuhGdnnXda9yO^HQ<5Z%@`mtS z=0;dsr%le*b%<jVQY|SLsy1BPq!3)wTCWvPOo>TWC54SL&C^En*vdvxj<M9SER&2y zl@?e=jWB|zQIVBNK&|^)R^9>tCa#{V??N7ocRaY?&f_Ue5uU+5*@ctf8qQV+Zach# zIVQmXmNaCCfNl|>3m+{%%7u@*;pEc!bM&zF@s*Fx-!}jB9e=(Y2dkbtMCN8g{s15D BSrPyM literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-Digest.crc32 b/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-Digest.crc32 new file mode 100644 index 000000000000..48b3a9b8234e --- /dev/null +++ b/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-Digest.crc32 @@ -0,0 +1 @@ +601622112 \ No newline at end of file diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-Filter.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-Filter.db new file mode 100644 index 0000000000000000000000000000000000000000..2e1d5d29ca0683d58365c1bda385a89ced35f1f9 GIT binary patch literal 24 fcmZQzU|?lnU|?ckVPs%abZX%^z`&xww2T1&5k3Nr literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-Index.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-Index.db new file mode 100644 index 0000000000000000000000000000000000000000..7cb91dc9b3815095523e5066ef8a1a589532f1b7 GIT binary patch literal 29 gcmZQzG+<z0U^Hwr0g^^d89>swc@B^?X}Q1v060Gd5&!@I literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-Statistics.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-Statistics.db new file mode 100644 index 0000000000000000000000000000000000000000..f4d51e7127017739380dc5e944794c2dcaa50356 GIT binary patch literal 5603 zcmeI$e@q)y902gUYfJfAX<<Kr40VGIkQAi!Iu;neI-$k{#t(<nWgP7dSQwDDPQaOw z$&4X})n$lF_Dh!#(k&wn=K$yYV@mxog;gVJWWl7OZkgz^A#+H*8@=%TK3&RxCfb+0 z+<or*?%us0clTO~qU6*<N)R@*khck$5%L}(%P2}cPo*_kYqeZ6XRhP5X3l2g8ZA~% zYpJtq3tH`biPgeec}pp0wb$G0n;Lm*Zb#eqMQ5U)7cLOz^?K(cVra#|oBL_U%!9WE zoXpwx<`k226Olr=N62Cl>xGH5ct0=+_l1(w2I-(e+VO3J2rCH(6AmRDPFO`)O*oqH zO2VrMuOXa5ID>FD;bOv160RjIjx9{x5W@Zd;c;wQ8V_GT(<~9U6TXPetRTD(TP9uK zcUX1=$K{pS3LSP(4z`kLZ}43l4|QROPh&r@A6s(*d-+!ESU2`VY1nIzV{5Ch4M(vH zE?}1~!>*3Owr$4l<gj~*`Rg*`_;frn9se74RR#7@JU{8_uTY#R$ImnK3AU{2b5vd~ z$Mau)<uZyZKI%da`qhaXT<%2<ndgzi=A)2R7Cax-#-C6;dT2TF%B!)+tH#Ta*No!v zPMO8yt#iDO%I8e}fc$tqzQ3npJt)5Oh6cI4q#yb4o%P6rAK?C(7)1R^heqCp<>}m_ zSCIFv?nD0cQaUuV?mT+GGOww=gmGDH3z{FYp=}{BE^i6h2(5T#9JM#7slElqm4luc z=n&(kThL+g!{~X6sM3tVc%;lU0v%;YMDuxt>dxyh9(U^1YtRp87N3P)_w|-+=(MrA zGtgPpFC2q5ew)4py6pHaJ+wLTojmB>e?&|{cYWwAf<9G#cs2BAJsk(2XU@4d0n<ql zHAB$m!b14`)6UZK@cE~`hg9(SXL!%mcftA0<mruVz%o1SIS4FI7}c1dXJ_}o@lpKl z=!V}9WpdP1J}4jD^H$s2z@a5?j97ugw|D&l+pqdYeGRr>eWiFn1IjPsI-BA5Af_*O zSPSCuA5UrEcqQB{GIfG@YUOkyTo34V*GE(!o;&{I@N>XV9O*v|@8{|1EIpjhjb+AO z2PofBGw7NGKK9+jg<jx`Z=URj_v^Wo;x;VCY0Zn5=L&#LJ8$3Jw`fYbH~bF%`x=Y^ z=ZL%aTXQLV<)p228hR%P2Lo#^sb?0p-Iz;WbNNmSthsRi1=d`sKLTs6z?ut8h`^fb z-&}Lihxo9~KT|QYV^q*1&U31qllQifIa~J%|Br}S8me4ZTw&6I6M2Fy(y{w(r5}C5 zhu2pD)gmqogbzFPV&(J=_gcoCpSR(qxxVar#~0WC469aVxL&5Hq<{X0r$)Prv)byo z25n`<W9utgn|X>-DEJn;m0R)|vfW%m`>4O`S3KL?z>5|5y>2;gUs5?!qg|IFmP}6d zSNT#eDRn|}vRr6X#(%O^x^|Dh?y9u$)_X(Hz}Ir-*6nt#rh&JmvSwD#8rVl!ogquV ygSD`F{SHH>L2pPGBf`F(%`{}Dvb<142yXCK&C;iYG%kJzS}M3OfSLb~rv3$hgx4nk literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-1-big-Summary.db b/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-1-big-Summary.db rename to test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-Summary.db diff --git a/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-TOC.txt b/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-TOC.txt new file mode 100644 index 000000000000..aaf69afa269f --- /dev/null +++ b/test/data/legacy-sstables/me/legacy_tables/legacy_me_tuple_compact/me-31111-big-TOC.txt @@ -0,0 +1,8 @@ +CompressionInfo.db +TOC.txt +Filter.db +Statistics.db +Data.db +Summary.db +Index.db +Digest.crc32 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-CompressionInfo.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-400-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-CompressionInfo.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-400-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Data.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-400-big-Data.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Data.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-400-big-Data.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Digest.crc32 b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-400-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Digest.crc32 rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-400-big-Digest.crc32 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Filter.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-400-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Filter.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-400-big-Filter.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Index.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-400-big-Index.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Index.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-400-big-Index.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Statistics.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-400-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-Statistics.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-400-big-Statistics.db diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust/nc-1-big-Summary.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-400-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust/nc-1-big-Summary.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-400-big-Summary.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-TOC.txt b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-400-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-1-big-TOC.txt rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust/na-400-big-TOC.txt diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-1-big-CompressionInfo.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-400-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-1-big-CompressionInfo.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-400-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-1-big-Data.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-400-big-Data.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-1-big-Data.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-400-big-Data.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-1-big-Digest.crc32 b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-400-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-1-big-Digest.crc32 rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-400-big-Digest.crc32 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-400-big-Filter.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-400-big-Filter.db new file mode 100644 index 0000000000000000000000000000000000000000..2e1d5d29ca0683d58365c1bda385a89ced35f1f9 GIT binary patch literal 24 fcmZQzU|?lnU|?ckVPs%abZX%^z`&xww2T1&5k3Nr literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-1-big-Index.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-400-big-Index.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-1-big-Index.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-400-big-Index.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-1-big-Statistics.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-400-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-1-big-Statistics.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-400-big-Statistics.db diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-Summary.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-400-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-Summary.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-400-big-Summary.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-1-big-TOC.txt b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-400-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-1-big-TOC.txt rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_compact/na-400-big-TOC.txt diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-CompressionInfo.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-400-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-CompressionInfo.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-400-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Data.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-400-big-Data.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Data.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-400-big-Data.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Digest.crc32 b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-400-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Digest.crc32 rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-400-big-Digest.crc32 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Filter.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-400-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Filter.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-400-big-Filter.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Index.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-400-big-Index.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Index.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-400-big-Index.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Statistics.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-400-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-Statistics.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-400-big-Statistics.db diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple/nc-1-big-Summary.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-400-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple/nc-1-big-Summary.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-400-big-Summary.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-TOC.txt b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-400-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-1-big-TOC.txt rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter/na-400-big-TOC.txt diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-1-big-CompressionInfo.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-400-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-1-big-CompressionInfo.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-400-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-1-big-Data.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-400-big-Data.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-1-big-Data.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-400-big-Data.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-1-big-Digest.crc32 b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-400-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-1-big-Digest.crc32 rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-400-big-Digest.crc32 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-400-big-Filter.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-400-big-Filter.db new file mode 100644 index 0000000000000000000000000000000000000000..2e1d5d29ca0683d58365c1bda385a89ced35f1f9 GIT binary patch literal 24 fcmZQzU|?lnU|?ckVPs%abZX%^z`&xww2T1&5k3Nr literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-1-big-Index.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-400-big-Index.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-1-big-Index.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-400-big-Index.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-1-big-Statistics.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-400-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-1-big-Statistics.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-400-big-Statistics.db diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-Summary.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-400-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-Summary.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-400-big-Summary.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-1-big-TOC.txt b/test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-400-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-1-big-TOC.txt rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_clust_counter_compact/na-400-big-TOC.txt diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-CompressionInfo.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-400-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-CompressionInfo.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-400-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Data.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-400-big-Data.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Data.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-400-big-Data.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Digest.crc32 b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-400-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Digest.crc32 rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-400-big-Digest.crc32 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Filter.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-400-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Filter.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-400-big-Filter.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-1-big-Index.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-400-big-Index.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-1-big-Index.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-400-big-Index.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Statistics.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-400-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-Statistics.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-400-big-Statistics.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-1-big-Summary.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-400-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-1-big-Summary.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-400-big-Summary.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-TOC.txt b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-400-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-1-big-TOC.txt rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple/na-400-big-TOC.txt diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-1-big-CompressionInfo.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-400-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-1-big-CompressionInfo.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-400-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-1-big-Data.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-400-big-Data.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-1-big-Data.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-400-big-Data.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-1-big-Digest.crc32 b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-400-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-1-big-Digest.crc32 rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-400-big-Digest.crc32 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-400-big-Filter.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-400-big-Filter.db new file mode 100644 index 0000000000000000000000000000000000000000..2e1d5d29ca0683d58365c1bda385a89ced35f1f9 GIT binary patch literal 24 fcmZQzU|?lnU|?ckVPs%abZX%^z`&xww2T1&5k3Nr literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-400-big-Index.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-400-big-Index.db new file mode 100644 index 0000000000000000000000000000000000000000..56f29df1cb9be118b9993063de9405194522e236 GIT binary patch literal 26 dcmZQzG+<z0U^G+$5=L%7!Z-^^m^Ab;000@i0^k4u literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-1-big-Statistics.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-400-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-1-big-Statistics.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-400-big-Statistics.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-1-big-Summary.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-400-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-1-big-Summary.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-400-big-Summary.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-1-big-TOC.txt b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-400-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-1-big-TOC.txt rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_compact/na-400-big-TOC.txt diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-CompressionInfo.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-400-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-CompressionInfo.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-400-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Data.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-400-big-Data.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Data.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-400-big-Data.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Digest.crc32 b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-400-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Digest.crc32 rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-400-big-Digest.crc32 diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-1-big-Filter.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-400-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-1-big-Filter.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-400-big-Filter.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Index.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-400-big-Index.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Index.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-400-big-Index.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Statistics.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-400-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-Statistics.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-400-big-Statistics.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-1-big-Summary.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-400-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-1-big-Summary.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-400-big-Summary.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-TOC.txt b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-400-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-1-big-TOC.txt rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter/na-400-big-TOC.txt diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-1-big-CompressionInfo.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-400-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-1-big-CompressionInfo.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-400-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-1-big-Data.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-400-big-Data.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-1-big-Data.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-400-big-Data.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-1-big-Digest.crc32 b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-400-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-1-big-Digest.crc32 rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-400-big-Digest.crc32 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-400-big-Filter.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-400-big-Filter.db new file mode 100644 index 0000000000000000000000000000000000000000..2e1d5d29ca0683d58365c1bda385a89ced35f1f9 GIT binary patch literal 24 fcmZQzU|?lnU|?ckVPs%abZX%^z`&xww2T1&5k3Nr literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-400-big-Index.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-400-big-Index.db new file mode 100644 index 0000000000000000000000000000000000000000..d094f73e2135ffcf478c5bda9adef5a7c030292b GIT binary patch literal 27 ecmZQzG+<z0U^KJ_5=Iq3!nk28kThxd!~g&%Cj?pm literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-1-big-Statistics.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-400-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-1-big-Statistics.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-400-big-Statistics.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-1-big-Summary.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-400-big-Summary.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-1-big-Summary.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-400-big-Summary.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-1-big-TOC.txt b/test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-400-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-1-big-TOC.txt rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_simple_counter_compact/na-400-big-TOC.txt diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-CompressionInfo.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-CompressionInfo.db new file mode 100644 index 0000000000000000000000000000000000000000..f69c841bc29231a879f8d9caa341f305d6dd6f1d GIT binary patch literal 47 pcmZSJ^@%cZ&d)6<N-ZwVFJb@#2Zs9p|NleSY$8CK5yEC*000?c2<!j= literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-Data.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-Data.db new file mode 100644 index 0000000000000000000000000000000000000000..722922d2cf27a6ef21e10a85ac5874100a4c4b22 GIT binary patch literal 307 zcmWlRKTpCy7>C~%M7tnH(ZOJZ!8jREiI$1oiRj2+u049Ccl~!~OWQ@<G{(gbz^vg* zz&Fv)V4@=)!!tZj^83{{0p0;SkE_*c3J5>1^r=-tV>*p9$76wSE8BaZGGUQRIrk^j z30=-e8U%sjEH6|*qlCvf5x(V#%!+d2x}l}4kSkFLDU(!+IQ7zKGM36Kn9wU8=3WuA z2_aFUBpVkrWX>ofb{spYBqGXuX<P$9LsQmlXwZPYk^y&X-)lk>NegzY3LM)#I34Wj zx_t{BV%b~pMn}$iP&$Ac1U&EJ{Yh;x|GvWC{$l=RVx@6mVt-kEF?Cs1pLIF9G_gJW cH1V%J{8)Qe`)ptP*{p8jpsYVJSRD5L0SH=M^#A|> literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-Digest.crc32 b/test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-Digest.crc32 new file mode 100644 index 000000000000..1e72b0fd436b --- /dev/null +++ b/test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-Digest.crc32 @@ -0,0 +1 @@ +2035855651 \ No newline at end of file diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-1-big-Filter.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-1-big-Filter.db rename to test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-Filter.db diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-Index.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-Index.db new file mode 100644 index 0000000000000000000000000000000000000000..f33dab8bba2558deb3791ec8a2e432e187ff1161 GIT binary patch literal 29 gcmZQzG+<z0U^Hwr0g^^dIY835c?pm-X}QAy065PDDgXcg literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-Statistics.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-Statistics.db new file mode 100644 index 0000000000000000000000000000000000000000..b7a6da2db11226ed419238d30ca5734c2c09f021 GIT binary patch literal 5676 zcmeI$drT8|902gUJ}5=#Xu$^%*`}hFH04p+%BC_9c`Qp{JSOfDIocC&2;|y1UJH!j zKg0l<F=m1~_aFp~WU@^_$NtcPQ#X{Dq01mFscc~noKXf7VRzDl@Av1H_K*E#ddcPP z)9?3w{k`t)*HRQETdxg~Qwy7pKhXSM$nO$<Kfv!&iqZs7z0WGCh!QSe&y+Hj0#<Kf zY&NFY$}xItfnA?aZfDatE6cIgbcVAR+6zmHSuU=w;-}<Oft9(}FSqaoh0M>-&-n$> z@<Vs_(@j%P-tFp;oc?lFJ~lhz|N4cwQ9eaT6E>j^a3d|$4Q_lL(I!KpD6xIw$6n%z z^Qa)~P1uL9FJTp7HQ_+Qs|c?qyq0h{;b_8}38xT#iEuvQD#Fc#y9f_s)8cqI$B8yM z<IZuQ?L>YbwqzyYT5PFUzVl;gD~`*uvE>GAuPxXLqP^btaolGB+jkQCiT&7`53pCf zj2%3L{Zs_@`eWGo9BlIu?1XdJ>C3TmgRpHcV%Ia+ZN&T?NW}5UP-HsvPwbp5>?3%7 z(v#n#xFi$b&yq{n(wy&5zD$PazwFu-6qjFYK=!)bf$W_*kF1<yk(bQ{Agio+KB^Oc zLGi$z709bbgOOJcXCSW~#N!=4jmO*2^cBkAGIk63`FLEvoxu(i-}Ac$xhAa>`S8pp z<n9Z&e@41df6_kw_h3F9mn^&o7B{+jP5UDF6Z-P`NNCB1GwAs$c~5m6#-+h!Xnsh0 zUQ@!jtW3ETTK?)VYOhyGVHu1ox*b!{%Eaw=p_hgBqWj4&M{^a%{iUh>&;jN!G@n<h zX5NSKkP`<dpr4LOISswx#*WR<5myUNLC5C4brd@BhsYh!8OQb-p)FxwJO{n+Z@+Qq zhOax4p-*HUUIYDYOI;)M)aOInf$5EYc|FjUZQJ1ePj{rBf%iW>|E>z&{}R?QdKO$S z89TYP0$6IN9fyEr>jpKc(9_dZaD3!{H8qRx4<SvV4H%CH`Q9y`R(uBRlXkqH1NPn3 z@CR(a>ZW=EwqJcMrAq_ymoxRH@O==}9@nb}@z8I^HE_Jv-APWZ2XS5YWEflz7!AMn zt3W(%_~+g?fnR9tJPGUb%4Dn&&gbHc#I`1oUzgWCFa~_|$B}bwz<nQn)CueBI3GS_ zUW(J2cdpDP0H^M`cfWSYlvp?X41RPCCIQz7r~kF)k~{Onqj(v5Ch#A6)?8xGEF7CL zDR<Xg&dV0pT%zr`Jh~6tv*yD6;aPK`{_w21JZmm6Av|lY|8vbnH{bBDK1=z0l0<n$ zWl^;ajSr7?x{vd3Su3mg|4GEIkGuHWM2b}*toMZXGaY|(W@oq<eDr^fr3il}{s`n> z#Lh3IG9CzpV1vp~$AhX`AE=cX?W)GxYR9EYYU98E<&&r1%W$>=rbwTim9#0Vyp-kZ zB4^9&9OL#C(tS)(&7iCDv)(8zVudeoS#Bn4cPlTo*lvgxGPOEaU+!*^Vkc;|GQLsK zk7cV2Z~MFIt|YFcf-QDy5VCD7_b{=D&1Wp-yX;I}5o^<#EGDDLY>F}&%(2GZCe{>X z+-;6A8_khIgg-Z$V$3l*Az-qajL*2Lk^8ep9730KLQK>y40cS7!&N07hqfNzV$J^m DM&Rw1 literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-Summary.db b/test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-Summary.db new file mode 100644 index 0000000000000000000000000000000000000000..9b24e0450c732103bdd22afb63db34808745ed6f GIT binary patch literal 47 jcmZQzU}#`qU|<Ad5a0#k1|Vi+0TKp43<HcH4x<SGD_8-l literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-TOC.txt b/test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-TOC.txt new file mode 100644 index 000000000000..ed377e244ebf --- /dev/null +++ b/test/data/legacy-sstables/na/legacy_tables/legacy_na_tuple/na-400-big-TOC.txt @@ -0,0 +1,8 @@ +Data.db +CompressionInfo.db +Digest.crc32 +Statistics.db +Filter.db +Index.db +Summary.db +TOC.txt diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-1-big-CompressionInfo.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-400-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-1-big-CompressionInfo.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-400-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-1-big-Data.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-400-big-Data.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-1-big-Data.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-400-big-Data.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-1-big-Digest.crc32 b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-400-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-1-big-Digest.crc32 rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-400-big-Digest.crc32 diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-1-big-Filter.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-400-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-1-big-Filter.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-400-big-Filter.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-1-big-Index.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-400-big-Index.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-1-big-Index.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-400-big-Index.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-1-big-Statistics.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-400-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-1-big-Statistics.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-400-big-Statistics.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-400-big-Summary.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-400-big-Summary.db new file mode 100644 index 0000000000000000000000000000000000000000..9b24e0450c732103bdd22afb63db34808745ed6f GIT binary patch literal 47 jcmZQzU}#`qU|<Ad5a0#k1|Vi+0TKp43<HcH4x<SGD_8-l literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-1-big-TOC.txt b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-400-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-1-big-TOC.txt rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust/nb-400-big-TOC.txt diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-1-big-CompressionInfo.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-400-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-1-big-CompressionInfo.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-400-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-1-big-Data.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-400-big-Data.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-1-big-Data.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-400-big-Data.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-1-big-Digest.crc32 b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-400-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-1-big-Digest.crc32 rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-400-big-Digest.crc32 diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-1-big-Filter.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-400-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-1-big-Filter.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-400-big-Filter.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-1-big-Index.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-400-big-Index.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-1-big-Index.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-400-big-Index.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-1-big-Statistics.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-400-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-1-big-Statistics.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-400-big-Statistics.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-400-big-Summary.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-400-big-Summary.db new file mode 100644 index 0000000000000000000000000000000000000000..9b24e0450c732103bdd22afb63db34808745ed6f GIT binary patch literal 47 jcmZQzU}#`qU|<Ad5a0#k1|Vi+0TKp43<HcH4x<SGD_8-l literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-1-big-TOC.txt b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-400-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-1-big-TOC.txt rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_clust_counter/nb-400-big-TOC.txt diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-1-big-CompressionInfo.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-1-big-CompressionInfo.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-Data.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-Data.db new file mode 100644 index 0000000000000000000000000000000000000000..a72b4ee380178e5534ee75b0ec646b6fcd294e11 GIT binary patch literal 89 zcmZ3>z`!txfzhD;|Ns9D42%pv*;M%X892Dp^79pv5{rPiijjfQP>~@<kpJK`2W19n lBV`7gLytUwEMp++@R3_UmWeWh>5<SxWrhH-h74P)dH`BS7=8c% literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-Digest.crc32 b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-Digest.crc32 new file mode 100644 index 000000000000..682a656d0108 --- /dev/null +++ b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-Digest.crc32 @@ -0,0 +1 @@ +626505151 \ No newline at end of file diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust/nc-1-big-Filter.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust/nc-1-big-Filter.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-Filter.db diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple/nc-1-big-Index.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-Index.db similarity index 100% rename from test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple/nc-1-big-Index.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-Index.db diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple/nc-1-big-Statistics.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-Statistics.db similarity index 84% rename from test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple/nc-1-big-Statistics.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-Statistics.db index f9940d0cf8198ac643cadb2c364f80702a904558..836b7091cc7318f2962010e8fb084bfdebd24daa 100644 GIT binary patch delta 171 zcmcbn`b%YkI;+UMd+SzB)JbNXzHvek=jI@;Qg*}1`JM@i47?Wi7}(TbZMS*&157)b zN7n!U{~v@Ipuqk?S>h_4lm(L+g`_772ysr95#n)!=w>+m1gMA+h>-YDHppZK#;<!e uzuxo3Ev50l`hDe_>{t4K=lcI;U%;7^_=AB#W1^ne<OxDfEY}xYSPKB1H$cJw delta 248 zcmeyRa!qxDI%~ldM()=Wb&?t9Y@ATU>BiXYtFC*V0S*{%#Z1l8fpZy{EUll_XCU!c zi>x!MMev!8H)SpQv-t;SJ-cCBbcF6R2JS_l8CbtuxiH~i2AH0_pR4}=|NkHi(`J8f zONVmvX5Psg1*In+5agVEMv%uJqMKp208kYp5FzoQY>>$ej5?Wm-d*K>N0&TdWJ#0w idkU<Y4M-X=nt0bW@%*{crL@g4W3seRG|SQr=>-6pR!o2Z diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-Summary.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-Summary.db new file mode 100644 index 0000000000000000000000000000000000000000..9b24e0450c732103bdd22afb63db34808745ed6f GIT binary patch literal 47 jcmZQzU}#`qU|<Ad5a0#k1|Vi+0TKp43<HcH4x<SGD_8-l literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-TOC.txt b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-TOC.txt new file mode 100644 index 000000000000..8484db90983c --- /dev/null +++ b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-3gll_1kke_3qdyo2i4kzfpt3cqcy-big-TOC.txt @@ -0,0 +1,8 @@ +Filter.db +Statistics.db +TOC.txt +Index.db +Summary.db +Digest.crc32 +Data.db +CompressionInfo.db diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple/nc-1-big-CompressionInfo.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-400-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple/nc-1-big-CompressionInfo.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-400-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-1-big-Data.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-400-big-Data.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-1-big-Data.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-400-big-Data.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-1-big-Digest.crc32 b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-400-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-1-big-Digest.crc32 rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-400-big-Digest.crc32 diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-Filter.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-400-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-Filter.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-400-big-Filter.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-400-big-Index.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-400-big-Index.db new file mode 100644 index 0000000000000000000000000000000000000000..b3094bffbad9302ccd45282d2175bbeb41e862ea GIT binary patch literal 26 dcmZQzG+<z0U^G+&5=JgS!Z;mBm^5@T000?r0@eTk literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-1-big-Statistics.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-400-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-1-big-Statistics.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-400-big-Statistics.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-400-big-Summary.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-400-big-Summary.db new file mode 100644 index 0000000000000000000000000000000000000000..9b24e0450c732103bdd22afb63db34808745ed6f GIT binary patch literal 47 jcmZQzU}#`qU|<Ad5a0#k1|Vi+0TKp43<HcH4x<SGD_8-l literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-1-big-TOC.txt b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-400-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-1-big-TOC.txt rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-400-big-TOC.txt diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-1-big-CompressionInfo.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-400-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-1-big-CompressionInfo.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-400-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-1-big-Data.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-400-big-Data.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-1-big-Data.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-400-big-Data.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-1-big-Digest.crc32 b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-400-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-1-big-Digest.crc32 rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-400-big-Digest.crc32 diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple/nc-1-big-Filter.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-400-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple/nc-1-big-Filter.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-400-big-Filter.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-1-big-Index.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-400-big-Index.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-1-big-Index.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-400-big-Index.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-1-big-Statistics.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-400-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-1-big-Statistics.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-400-big-Statistics.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-400-big-Summary.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-400-big-Summary.db new file mode 100644 index 0000000000000000000000000000000000000000..9b24e0450c732103bdd22afb63db34808745ed6f GIT binary patch literal 47 jcmZQzU}#`qU|<Ad5a0#k1|Vi+0TKp43<HcH4x<SGD_8-l literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-1-big-TOC.txt b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-400-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-1-big-TOC.txt rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple_counter/nb-400-big-TOC.txt diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-CompressionInfo.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-CompressionInfo.db new file mode 100644 index 0000000000000000000000000000000000000000..361f57d7d4dfccfe921532b33363024c684e52b2 GIT binary patch literal 47 pcmZSJ^@%cZ&d)6<N-ZwVFJb@#2Zs9p|NleSjGaK55yEC*0018i30wdG literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-Data.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-Data.db new file mode 100644 index 0000000000000000000000000000000000000000..e97f414ff026c96d063d9bc34e4cd7babeb0dc49 GIT binary patch literal 122 zcmeBVWMG)Yz-Una|Ns9621bU@>?+p$3>@YR3=A?r%nZaVK+MR&Ak4(Uz#zdOEUdtw zz?i7WAdtk$5Sh%#z-Z{p5O2$WFyFwRK~vhupTYjn_8<_`7|1+4_XLP(0%RW1%Jyea O3ILjw%qVX8bRz&sD;Q1y literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-Digest.crc32 b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-Digest.crc32 new file mode 100644 index 000000000000..7b1035c7ce92 --- /dev/null +++ b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-Digest.crc32 @@ -0,0 +1 @@ +3757037919 \ No newline at end of file diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-Filter.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-Filter.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-Filter.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-Index.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-Index.db new file mode 100644 index 0000000000000000000000000000000000000000..0398b1f8a0daf8ad065034f4b71658c0cb693a9e GIT binary patch literal 28 fcmZQzG+<z0U^Mgv5=ISkfTVH5YanUTXvF{kGaLku literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-Statistics.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-Statistics.db similarity index 81% rename from test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-Statistics.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-Statistics.db index ead79f6f295c72384167f8557cea21a3f4aa1f87..6dc52a3c8b8f92bd2bba510d4b2b5bebff4a3691 100644 GIT binary patch delta 283 zcmcbwdQNSEI;+UMd+SzB)JbNXzHx#+<K$*;1xD7*4;Ym=HV1K)vKvm$_e@Y^;I+8N zz^49cyDiTLFzsj_UH||8e-LJX0{hGN6ehmBWigpiNP4n>5a(nWAs#n~ZicrpKt+r| zgv5ukK_)XWe%-VA^`0+oDUJWt?<?PAzvB4XVftV81)NDe9~c-kChCbzo**R2)bJa^ z;^KBI%CAbz3#lwfg-UX9g_IWLq#}!O1gDlD^SM3qN{o#_BAPl-bI_Csv1@8->fg?M F4*-HuVb%Zu delta 301 zcmX@7c3*XZI(xwtM()=P`jh>I#3s6UGtSvKL7$P4b@Bm5`OU%H%5007IGEfRyM5Jl z&ojUQ<E@yfSvqho1Cyoov-%7q{%VnRMzsh&v+<^^MSnK`;H+miY>ST2ea67O=raTB zmn#=0oK*nRlMj8U|Ns9#2*b45-#P9Za3J^F<c)&TlMe`TPCg^Z6A01Ga6kg6iV=vA z_)s>;WCli^Og-<ea=)WXo-necN&GzpR?P+^4H!+fO*uQ|&y_BvZN3T&43p)AH93Id j%w>r=hQL5j5OvNk%_~VQa?Z~w&CLs`EJ$6~+vN`cv>jZ; diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-Summary.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-Summary.db new file mode 100644 index 0000000000000000000000000000000000000000..9b24e0450c732103bdd22afb63db34808745ed6f GIT binary patch literal 47 jcmZQzU}#`qU|<Ad5a0#k1|Vi+0TKp43<HcH4x<SGD_8-l literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-TOC.txt b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-TOC.txt new file mode 100644 index 000000000000..8484db90983c --- /dev/null +++ b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-3gll_1kke_4abk02i4kzfpt3cqcy-big-TOC.txt @@ -0,0 +1,8 @@ +Filter.db +Statistics.db +TOC.txt +Index.db +Summary.db +Digest.crc32 +Data.db +CompressionInfo.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-CompressionInfo.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-CompressionInfo.db new file mode 100644 index 0000000000000000000000000000000000000000..f69c841bc29231a879f8d9caa341f305d6dd6f1d GIT binary patch literal 47 pcmZSJ^@%cZ&d)6<N-ZwVFJb@#2Zs9p|NleSY$8CK5yEC*000?c2<!j= literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-Data.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-Data.db new file mode 100644 index 0000000000000000000000000000000000000000..eda05d21c56d6b3edf8be762d4594879624334f1 GIT binary patch literal 309 zcmWlRF;Buk9L4|tVzHALF-}c{!8nRav<!w>C!!y~T-&?$O0T^ZOM5Kh;$mF<8qV%0 z-+)gdn&61X@Ov-sCGYLm0p0*RkL&e%3JAY1{!_bw)^r+VDd)M%3#K@YV^LHU6{*aN zybvJ|Q#A=yL?xkPo^X{!K`b5=DWjbEi8qRss)|6cv7mktrnyfzQ-W25WPUg-Q%?nP zoGKxRq-q?oayT4?RJoI|%y`D!k+3cSz}D0>TN<>WX;^T#$;}qDkhEdfHQ~_f!SP^E zw>!7cA+EO#uRdj)fpGvg2zc7X`=iEevAn=lf42B`(6rASJozjW<4>zJ>h`&V{jcjE eX{@v@rD57#-OSC*!PV8=lnxGR`W4?lo;&|e`C^{{ literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-Digest.crc32 b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-Digest.crc32 new file mode 100644 index 000000000000..3b8ed43b90e9 --- /dev/null +++ b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-Digest.crc32 @@ -0,0 +1 @@ +4271820172 \ No newline at end of file diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-1-big-Filter.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-1-big-Filter.db rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-Filter.db diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-Index.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-Index.db new file mode 100644 index 0000000000000000000000000000000000000000..f33dab8bba2558deb3791ec8a2e432e187ff1161 GIT binary patch literal 29 gcmZQzG+<z0U^Hwr0g^^dIY835c?pm-X}QAy065PDDgXcg literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-Statistics.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-Statistics.db new file mode 100644 index 0000000000000000000000000000000000000000..a6348d64d015e98b61deecf7f1135031990a47eb GIT binary patch literal 5773 zcmeI$eNYr-7y$6y``}AB;6M-<$y7jy4&=ByDV0DJa4L|Gq|%Y)SiXXTj@5v4K*Y(R zLUOu*8I+<KV!(k3DuImtKq1pa$cc1<p&6n^r%(n7lb}7hh39##IsWJ`hn?Bk{oV8I zd+*oozIPNwNxXeMq}0OZ&mU-h&*XOzzrVxpVv3S;-)HwvQUSU8bOn>gXfs%amdVd& zbg6nqk($9N!V5VzMxV;++0<A@&t-C%xjI%KT2}OPWQ+5g$=5Ge@dbIz&(F`<xzW;7 zlgH`G=_jVzn?!A2%}PgShaFzKFgL=d)bhJ=Gu8oaq>Xih8(&AXNsuT?XrJ)0g>Ygz zS`xM*Y(v<Vu#B)HVQ0c?2(Kl)fv_LpAi`S-M-hIBa5~|`gbjq-2@he@!g!d*iB_B9 zrg5M-BEK73<Vv^{TP&1s`dBQ?6O$#0IA5y7w%CSkNwnAM0gl`BVcSk%+aJf4e~9h! z3by+o_PPLU?>cNn61L_vcKCVh*wxs{ZrJ&Iu*(_j24ep9Md0{^Co=7M7dt5d`!t@P z^u!M+E{ezZv*=rFane<kFOlH+FB!Ok;?i#_kS*>sAzQ`IBU{h0$gAd@kY%ZOJ~~GH zhT_hhF34*}+>zG~#UXF#$K&lcgU4H0`8CSlHu?+l?l4@xm)*xuJnMHka!E`pa?SlM z$Q>7O{|tAa{-kXj?!kOIG}8DUSlsC1^^J?*Pw2}%fzTqKbLjah`apIa#>MUhXnu%0 z_gllbq`-PRv~=GPYOh6ZW&w;_c8pC!TSx4if?nm>h3+T2B>6QMcMwPSLOW@^(0q24 z-LHmmkLLGoK|dKB)dubJ<KC^%0oO8GphJ?6oPmzG8MqfZt}aIft@ZliIp{-w*^NV2 zTx^PjZjP^65B*(L*-7Z>&j)t`)0^y4I-#{YcEJ0eZi+nz?|*v!eHpy}MeNwfm*9HQ z=-KT>z+#RbI|VG+*e{QUo|!oe$4B~Sr9pUq7}G3$oyNmJzE#zyMV|rN#C+7N2ev&> z@ds?b?1tkl*nY=>sCGHXU(J-~!S{h%V`!HG#62&K%i(x!oQ#Yv2XX(z2`{)FP$_Ts z%0N7H=+~|{fS<2zJqzpe>O_bN&Sza*L_;OWFH7m@8wEb|)A0EQ;O_d5TVZ|2di(}8 zOL1C$?8<C7aCFwa2c^Jozu@z9Kk&_chAjUji$b0qFU|mNc5VqevShKcEvtTZ$&}Dm z{7ipzO(z1^7*GFe4JS3_2}j{F^o-#@Tv@{jJ+p8$U{cFn!<jByT*C?GaCvk$_R1O# z_s7Z_4)w>%8g69`2PVYI8t(sG!_kKQ?|yQEvUOfZSp*z>JuA8H;V~!8A-&hKx|sjp zMd<p_ORiQG-cRN7Hol+fup4)e-WGz7{!g+L;m?F0mHdm?^o6v?TgF1LL1idfmp%}- zJN8BE8kf?>6#HmH?VhXj-}ZhzCr9h<I9|-DrZ)Zie@ZEe97dm?!DK5E6LxJ$D9mH| z%1PM*PR}g+3h^N(yQJS-`3VQ}vRUI7m@PM+<(8Eft>ctI#!O#-b6>vPB85)y^_B3A z3VJMCr9A3juDf>WbBkErvIZeBpVdE1%x2RWZQ%iqNy%pO{nc8vO07|ERx33jsx&pL z4pOCQf;B2lpfSRqtJJ}oV1GWaS)Hm@J!7s$%bz{M5SpD6LZa`&U<a3snXAO(&{pk< H`l){aE&%_# literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-Summary.db b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-Summary.db new file mode 100644 index 0000000000000000000000000000000000000000..9b24e0450c732103bdd22afb63db34808745ed6f GIT binary patch literal 47 jcmZQzU}#`qU|<Ad5a0#k1|Vi+0TKp43<HcH4x<SGD_8-l literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-1-big-TOC.txt b/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-1-big-TOC.txt rename to test/data/legacy-sstables/nb/legacy_tables/legacy_nb_tuple/nb-400-big-TOC.txt diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust/nc-1-big-CompressionInfo.db b/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust/nc-1-big-CompressionInfo.db deleted file mode 100644 index 2abf0a8898f0ddb1eda4d572f2bd3ea8e5668cff..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 207 zcmZSJ^@%cZ&d)6<N-ZwVFJb@#2Zs9p|Nn!TtO|WVS`;D<BN<mgSj;XEn&lycW-Ecv z96Au1^9+RMPKVGu-yk&KY6vah3ZVsGLTKR@2rX(2p~W6UXo)rmEd>>qegoml_CjcR JQwXha8vw|A7D503 diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust/nc-1-big-Data.db b/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust/nc-1-big-Data.db deleted file mode 100644 index 8db5dadcd0f1535fc14c32802594b18d90cc5893..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8713 zcmd5?Yj9NM89tk2b5r=Z1u)p2NK=|b%kG{%dkMwtVl9n`y0%_WN+E<uHV}46Hka*K zveKY2T1t^h#41LFsHv5NK>-zZH(C*BMawXV4g=8<N*M<!Rq(>;`+R42&pB)V>5u)P zGkNlSxA*zp_j$h)1g{{h6-?F+MNuMxNl=7Rp>x1$wMll_;dHqx3kCX(vR?UZWEB(g zBay|m%j%k%Yl02UVgEhNZHq!Ri<<m(HO=AHnq`ZFjV-Nh&CC3C4UOR?&7sDye__}k z3WUP#HB0?9;nv#LrpCsmhERh)xTq}<TDqjBrY+!Kv}6&@wXC77J#bIBwrO#Ddwb3D z+WO^-8e9DJP3^U9fkm`VdsC=o;gVX~-k|MTGd8s;(6wrM^tyVlx!E#cYbxX$&E1j{ zh&-^$`@q2}f(LD$6tyZk%k4F{jF;Xsv&5-7B3-Mz(VI&M7VS>XTu&32yk^VoXTg;> z2uUu$c~LMOMrO^kU^<OVe><2iBXfB*m~JDp#Y!L>sKoAjaz7}*I^l_4b4ZkSS0432 z1M4I8|NenMO1+qp#mY~xS&8ry{5C_B8IXRKL&w;{zRWwxRd{Fzso)uuY!8}FvM+Fz zwjkzJ(a0&Bx`S)BN!ZOSod*#x`~3xc{7cy<O_#=rT~XF5%H654SzfcaD^f@j%v9F_ zkK*-kF9tg`BEU5d0(2Rf>w*B?MrO|(0$Hb>mhNL;<?$*1>?H{-cB_&3R2f}1aX=|o zj`?_cj9GSII#<S+F})-VUBXAYG9+W{wi~k*rJsFmR)$ppo*9KeNS#^AFgtIe->JFK zcLA_{HfO&|zeFVaPXLrPF2HSD0XmG#mwNy@jm%df09{7ro_c_8BXh_{Ae-@HMR}TC zGAn1W&YNxsAzN2xDQDOv+G#J){gksL#GX>pOxZM_V!b}d6JFj;nKbO<#q06EglBSb zDpTrE+B~7$$FJbq`57XPELCSw&g`JGom5qL_bh^A?;;gWvJ3P}9Qp7OfU?E~cvP)O zM&_|+!E|a&<jwQITt?#X9|5{GBBngOfIxO>it+?Q`8t_<Aq(&))jZ;!;#9;pdaKMK zc5Mo%>Kg^%CmQhA;{YlFoc5uR*}^UEpL0o){$jNi3V%{oP_|UPv^6sX8E~6=*z+h> zp9xiF%dVa(bMR@CV6t@(1e2W4J@OfppwC!t&Uz|!EP775*&M`g44xA!x=XdE_T2S} zIaP^2yTZf$$;(}5^DUm(6<&xT9YzJ@(+ZJ7PwabCap)>cGbM*Zi>r9@%8#aY0D;GV z=r9sXF9LLGM2s#w0nlY+Ub7dV+rSj9mwBf6xOxA|_xc6srp){fZQPKSbm|_(0XvIX z2fx)t@S0r~$!fG-`7x4V%7FA(%FhLl*%ST3L@L0aG&bb%#N78$I9P=H(-u!`dOiI( z-hMRIIS5haiFr}3L|Ge)=w46ts%5-GGkIt;4MeArSbY_UE{%xMv-81p8=1GD9OP+4 z%y<qRf_+crlWK=hv+?i-o>yrd!U?rQi2d@tZxzz-O!NW!eYmv4tU14*EAaKdxB_<D z&8B{~8%f6v$`iYjS2k&-N<Wv~&(mS<vlI#zWyDiRm$|&E$*XgdV@;7l%Y0td9NI_( zwl<8wI<-{*E=H*nT}I}TYJhGdGhoHKF|-IDvQU0MMt66q&SYe+os)E?Ue$%RkK6I~ zH(f|M#%;(OGrf};$_&aj%^cJucvP)OI(wk#ZYV=|bl!y9SOF}}nw}gD(4kF=nDwd! zNjD{dn~#H;=1tG-1(@bdFKi`{t)K?s7;DW{wqor;oiFuS%sV|~IaS<)P*n?_!p|_N z%|8C58M(Nunacq?QeB9ef}=c7Jh2fO(y<M=e}~SjEr+*f+o?=hMh9_{cphEA8?(2w zy@E2F<uQv*9@U+s4!WCgCt-D=XniEr*TI}(+av$ioDNZr;qBC&kLD2V9yh1h&iZt7 zIuwT*((8H2b`gno<T-$8&Ef|<`_h`l<2?J)n#GTJ_N6t8r+D_In#K2hT&8(fPG84G zOT0=+xkBNiL<XIjl_42qOM6$GvEjDv%RR-o%lBK(PDI6!thzS|h4H+=Rgknd66D_5 zkgz6=tY+nTy7WnF>a%=)ifgbhH~GFYMVaNxg8*f1BvMb=0nnj6XplU+fH^fHZN(GM z?gi@7m>8bd1<;+M4yzBK&b9sTLm8zT)*Zhrm?2t7sqA+1r?S$6Loh1Nc-3kCiXhxg z@t-naH}(dx*KSQXlSbwO#K+E{NQ{>>KZY~KCzKJIa;EqMwIsHtzDgirCZQzidjst0 z`xg+{wB}&)aWHkw0roizVp?x7WiNrc-T*UJP?Zdc@}A1O)$AFec;<W~d#G9{Q`GJt zUh(Z3uJ5ipkl%#eE9FeduS0tfi@CW7vMt|K$J+U~sB%?>snpR?McSJf;LfSzGDs1? zb{!p&TS*`jiZJ&UPu%GT$C1yqDQ7ek;!N`ab?Qb^EXAsb*ebeot6;im_y*xNGA-9# zg+=mYEK<RQf;vbsGhgRU)Udo!lCY;=tM8%xKJP{@js87E6HHF{(HoO9BcBFbL-ZI` zO4;YRN(0K%TLp9w(S2E#(!lPem#b0=^uVV$-i(p-^AA(hgGUk;W$z%ITh!l~Xq}`D zolhPWzv1_VA>#+Tc+JAqg+XCOp=is(tEiO8*-8VqsQ3;4NwcVU-3`!Bnw_WZh*b4k zf!beJcaQN9Py#4luI&I?HC_SKp`!v_P9>49MS^=+Z6Z!gI&u$}&Co<BSF4T`51q;k z)dp>@Ls^PqXV=sJ=i)T&+!@?o;b*oKzXN(NtbyG&LQ!NkkshQNP=q*<J;1Ma2t}-n zCtn!!NUv&G6ZqRT%(`-lApTA7B^*mfS97%!-jR4s^;x=th^T!IPd{}uc~(~wP+2z? zOFc{ln>4|u6h-9{y2cCCnZ{j*3M{%f_YNO#?$&jm{gms(Kh|-G+@L%mQ_rXjkx6nN zG$~V=7kRV*aUVk`VWaXCi`qHVSstgp8O9}`+$~_iY-K8H)<&`kYT)c|Yr~<$t<i*a z>FBBw8;21iNw*xP+4L6HQ(skmmcFgv$X=wJwp1VW*`$PiBq`C)tFTkI4pe)Q7h#u< zi$kr%1nv~|BeeonUf1`)H!9$d62`R6pCJm_4+s@(I!^&!*I)KQ580Z-FJ>JJ<&@)y zzu<Dn`-Sop8FIuECZ=Pna=cQBm`De7e3D~N2~fL^jkzZinDDSZ%TNI*j>~hkg=pa4 z&H{C&4Lk?XrC)<2+IIw~JB@mjK=#nI3YKq5PFxN_{}D~t*aRPcGhJD|?L#ga|MLWg z)Go`P5bpNpGDEll=LoJ)fIU<osxiN74V=pr6GSF)K?8V3t>t!>B(KO3kw^2wu(N&R zC=rw(^O#F5H4746CU+3Vz3lQL(fW=y8TYbxXBDS=*~e1S%VzV!7hqWAty1xDw+L?P zLty;$#EwGxC}oqfa#LGGmYyz#WcHDX17SEx^N9ddj%z!>WugD*dJuJ_+5)mK?(}iN z^baUH!SX~Y{4`@6%Q+4uF2p*sm13OaG*04r)ua@&Yfv9U0M7t)>zIhdf8QaL9U4od z7SzoW@t>-F=>{W?7k8@mHC9^)Vjrd}z`q6R=<1ER_zEUgKL++$zV5~1_S;Y%)otQq z;E)L(C(63P*l0IEhi;<)9|q`5Fca^n^Yia`lnca7D)pz=fU3`sd3k2!)1c%UMXgkT zMW}bcEfbq@7<UvuGAsQ*04WOYB9yJ6h9bz;(0%31xvGVwQAf;BEsSZ?qVta*ZBKZZ zj?d*78u3#sDe3ls&VOihIo3>m8XOvZHCVDhpEi;kBkhau0uvp&u{gt6fKJ^)IK#w? zKvT#4gO9i9<5td7xK8}f862X@*O-uL;+GjBlifW&i*D!r$e3cB=E@Uv8g@sXni>iv z$A_we?Luc<%qp^#5R#*WS^?@P<RWdjD#vdU9;V}~a+K^xoFv_PT#nKKz;+!S+bHY7 zHj*!~p|TAiIcQs{z0w}8;q8M{HxweR1l6Tm2J>0#fx1)FkA3{zKljANZL3S@59aDW zVn9w63QJUP8nQNIh)D7nQ;@CX9>hzzOicyF-tiiL(~4|rPJqY$7@1B;bdEL!l5Dju v9b3qqOh-o@8?NS7renk3X2HpBngkaky_HhS$5xMdsSp38!c_D^$&CL2<_ggw diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust/nc-1-big-Digest.crc32 b/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust/nc-1-big-Digest.crc32 deleted file mode 100644 index a3554fabfcf3..000000000000 --- a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust/nc-1-big-Digest.crc32 +++ /dev/null @@ -1 +0,0 @@ -516351458 \ No newline at end of file diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-CompressionInfo.db b/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-CompressionInfo.db deleted file mode 100644 index d0ab8d26692709bbc822531138f8df4087cdc3f2..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 199 zcmZSJ^@%cZ&d)6<N-ZwVFJb@#2Zs9p|Nn!TEJqFjX%UDxjAWbzVKM7KXqF8Sn#}`3 zvtNVIoM{l6>nDWfnGT`(Bp@{ZLI^FW2%&}cKxh$X2rarELW^rbXo-CgTFM+kOV5VT GvI+oAk`#FW diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-Data.db b/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-Data.db deleted file mode 100644 index c212d3df38a5d052d5c1c934fa727db82e56dfe3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 7529 zcmaJ`eQ;FO6@M>zN%p~zn~;ViXn4sI-Q7Uh?+>VwK&>T!nrNj^Yaw;8q#X)aY@wwj zV#Y40G#^x>_>u4xELzwg1*J-|8x*KRV}@Y@LvgdEG%(gCQCpBQufKEOe!N|4_#+eU zIp>~pf9IaxIqycO6oeK*wAHJs+8~I6DwGI~0h`V4aJt+cuW#l!0TT83(TQdz2xWrU zAWW~Dv0%$D?mfQik|;e|c=MEy@!6Tarq65_q<TSwb(j4kiEMV%Piwp=bkAK8k+Qta zp0<c&4cPxE^X)SCNUO>lBf^tc!cx}MrvH8<r3Ym;7`F=x-FMf(&e=Sx;crsfuIx#U z;tUpz1^Y&X?t4ojlEE^!IwC2~0dDv_b`{~}!C{ijhkTOj4k2bmk|pOzQUkf-PPVDt zOOgi%VB<-)sar>qhap$I$u=u%NwOUBNxo#8HB}_3TNaU6r<)Gkmbp=;03NqPU|u;7 zn>pU(l3h!2mi0vuDPIYUOslQtpbvf9r3%Q*{?t4uqxl5e(?`b0ggS&g*7Hm_>`OT^ z3Y#IV0&IZw*9ara(Y(esD5di#gjH-4&#|K6+Hi$pf4mSv=dy$~Mc?=#M0x9B*()oq zHxrf+Dt>>Sp}cv6KTmO|m_5_pD;uUZ^c|PeAob|%(qOlP+%+D-Z{8HUrp+Ysw?pL9 z8BrwL)<mQNwrUz-Tf(*)R?%YH66l+oR`<$Qr!B?cr8$t9wk^h90cIkz9m1P#>u)>4 zlA$PlezD@z($Q&cw~%IAWl*+uVex7;uY`sb9fLe0D2ZrN_-*{%=p=p{JFk6pMw(wT z$1~z|UQCDcm)Ps+BNX$#FuWPtbF^%@JmLtGL{cE({*w?|c3wl!?qsur0g`mWSn(v= z9Dac$`wu|kO}07u1W69ILE=lcIaWt#mY2fs%WO*)4=hvzZS&y<fa$U&tG_=JBGXaS zwG#Nc5k~3`Ud+$02AHPt?O~~+%VmGpK*ykSY8UU%rR4q?)vH6d->z)yUu2Sme5jUK z68cAI>HQKc{U%TSSv3cJ|9_wQnCBo$O~?b!!GvfI&Yl}12Y)uibKr8{PdUh8CF$x$ z!ufm;yctgqYf10~MRg-<qAcW)^dhH*{sBT%Z@4Gh8fl_1&vp}}DOQ))ljuBb+4)J> zDwjQ$sd>9v3yJ9>OqD=gAw*I5UabUt=7PmxiTx<P92PorX!#Zh*>Y`J2us#KglTmu zWbBc|YS&3xEr*QNCRX2W!)n=#6Je=U6hwPHJT~MCOF7ZoC&$mas+<YkqqbqmF!l1X zk0f!xrr7n2+>`qlF?oyZ2Pgr%!=Qd%&1+%p`G>W<XdQ#fgYvRDnwEbTj**tD^Wtf- z+xyZ~^;z}`eu}s=gF;U}Dmu#nk5RI^i$rW9sR_MuMNiQj0(rt}K<U&OQ`U;9j4io~ zA#&-=NK~3bqWnsT+$mON*GN=khRCC{BA)t!B*idQyeUTJL6nc;)9I6!>AZMLuJy_+ zY*E#%?57zBo+P#d*vedn;<Y+#DORpF$V}!FaD#S0mt#*w_ZPw-wa`z%+14({A>KDS zt*#dV&diqS*p2m~t!}L5`2VCu{3oxH7*Bb9l#Mhk8$G@!zB(J&E1nI9^IpI_p52O{ zPg2x(fFuk?TU(T=-lKK4jdVy_NpxXxH6U(xis4;868Tm@<k49Xrn3k*Z;JlBQqsSp z8a>9R(<d(rIa5?(L)N6=OpH1P#mM>pzF`=e?<#>}691E@QHV_Q4EQZd|A6)85QlZN za1ey$w9nzANvu`k_C0V80FMjF5IXejWOugpyFC=+f?*_gPqvn&o`?3*+I<5Ej;AP7 ztB(GaFKhWos*Ybq)3LN^jC4HE9Z!eTt>YIuPz#(Hzq>a|S$bZ=@gD*C09c&~q4%R& z3h<#O0yf3qCsia`x(>0=&sXai#GC>-J#O(sXx?}PTexf~dv09=nQ2oyWC2%=j#hD1 z#XjIWwfWy+DQ!Wq)n(V2YUSNR<?U>F%KtY>{JPQVa_FN%<9TilMYUx$j_cGJ(^+0L zlW6N=1nkeF1aw+o$)Tl|${7B%G?A8`gRYj6XSEXf^=-Y<mr7(bnvss{V`SvHL??H7 zE5#cuo7zMRMYMYmzKkzMJGt)+VbQ{Nq~9RnYn%*TzJ@@)7&pr9O4BT#CD|J&V1ub6 zj9&xpzo4L^6etpe{eaNdd>uDc@q81qDEn#nWxO>gO^P;)T}yFe<&HsQ>SiBi=|?&L z^VJEPX_FT5$Y896@a5dqP%g`8ALMPpQIB}zZDCVxJPc|JJ%JafE$lvlwoqhBX$yTQ z6NBlvf#SE9WyNC@y+-SyC*Z^M+5sXidxTGu-1nJoO3Voqtm5c`2|x0WtNCQ*&DlPg zXNQl%d}*ApHkzG*y<=qOgMoOx*q!eR>)A7Wu+1lwFEmkl`nsd^wTQ=(&&s6r(P}DC z+$knuRFT1QW-sc+qtmC|F4dA~0DCIl6sw_mB)Wi7QhYiq@-m;stZ`F*$>xjE*!;BF z$3Vtl4ll!vci2wRz45cQbTyM?!$pvpE@4k=z)>ZtnXHg*oS!boTPq0MH^9ghT@Jg> zP`e$&w@@HMHrzRC4^$$bVM~MM<HA@yVCg?sC@8I`v}#`;8~;$coD%S<!olZhpdq*C z#)o5oKMd@Du6=d5lfQ=4BjokdAt2yMd_fT)Z<F9ZefWU#CQO8gR1r+?AO`a#)Cr8N z&7^<hNUxlgP$$87Vj;8Ij$fU&WE-{^9OG?G_eZ4L1I};6KL^oLD|+7gF?E2JK!er+ z5|$jnAp4%y06b^_35PU!1oP%lC|UT{h&O;>ftl9E!=d7On<LJ@NRifpa+0vhIPQBY zg16)*fX9EQTql)0Gd4Wrc4?LH=If;=zf=j7$%R@q1gG5i>V&iw<ixAN>1*Nly#@{Q zP6Et0NY2;tAn#`pX|DBlVkqO}Yauy}kNlg2J6Sa^pBfPh<p&@(O$oy`;A&26#$kM? zMhBzIt~1qeEbodnQkO#)2nTY;3v_Pd2^5)AXGlRh8c1|IJV!YmFgZ5Sy>M%hYsW!! zo;uPjZ=*_0-b9(Tu-H)EI%8q6;?=np54o+geZ8{5;#k2E*Xm<74J5$-s8$F~m+Z4S z{)sx^9~`|=_~(|60l%jj@UNnR2)Hu{uwHg1{Upb$XD0A+v>aN2&Pc>QK|IMW5Z1j~ z<sL6VVHdp>lofBnk}CeB0Rr<oK1h5CBidse33_11HWy%{<Lq44NwWB0WKfz&W52T= z%MGKF@OYtOD~>x{3U&|P{h@3yx6bOy21)BD{$TC0emTq8`e0L!EG{Zm?3d}APCJL! zSP`rmhS$Q`+wzjgJy`ogWJ9L8$?h1=h#ie?(ef%tOiR}7&W6mH8ApY(T?eAVJIgYv zp6qyh#1ak?)Pe$8$1HVLHGi7Q+y1)_o*H_i($d3Ib9Xc~D~rZRO|9;u!?TqLcOm<J zx;mG_Sc4x1V;yEfdLHpUC8~4TcZhNql637Ax_-VCn{^+;CU0d(#=8h}O@#72s44Z^ z)(ymbQ&$gRzQNa`!=~R2aBLv{Gpz;xf!v?f#eu|CjRITKY-E(kU1zA3R6_?$fib=Z zV&vYq8B3ORU`NHdni3T~6g)RRO`;9$gspBXI*}jLiGoTw(KAcPppob;_PNnChnD<c zH(Khc=D@e@Uibk|L;StaMrn9AnuTqaF|y!a9Ooy8FAb+Z#_(Zvnu2P}fiL4Tl{^XT z5(b2B0`!c6d2SLY9$$c`Sr}P4Mz?A^QKzX1)M*Vk#l%KH@t$*p;#`ysP522j4a8SL zX9mkQI8z+D0I<)i5t2NA4Tsg{`%df5lLVCOvrN6~&I6>kJq*m<i5=+Dg7OL!Pr@S| zYai#jSBdMq32l<>=Tx^7yT$R|zX?0=|5a&M{=|g^@BE)D?~P)8ux<>jJKN$|yKD`Z zBkyGN)m1~j4_D(1#(fQg{P8s4Y9#^tAt@1q)N~Vs-$JR=tB%fA!|?F^u;#_?a?sac z%n~3Tok`&A2f->%ogMi+RzjkK@EjlCj~V$z{5g`X+an~C;$cv=;?eCv2#+Uyuiod# zH0ygKq<^#o`o0wXPYKTloT@sT?vlF5dg>f>O~(w-jXuP5T5Ezht+{FdM7jbKbielU z)IBY~jYFUG^7IBFKPUtrd~FQmt7s<G+JN^{Q9Z!$A*244fL+oASH^p~wI<+GaB=_2 zK1ov#5TCcuWPfP98eoUgHKWO7^DrJ?4P`ZZgEDcts~%O3nO!)Tt=LwI>SQ*Zm})JJ z?7`DP8rg%xA4$TOi9ND_j(&C>QW3B}M`W><HKwbRSJ9jo>|azY7rwWsIGNTj+e5O! zbbgse?3I~ucT|Iyos<pdABn`Yttib`izpix=K*2Y4-)ZF_As1+c2e{|fsw%yu~D{D z^ufjjZ09U&f1mauYk$Aw7|%((0eC<?_GmSX42_YK%Z2f3u=_TMst*G>HeN(=T_rz1 z$W&+YOQJqx{k#t`O}rW;atDQDFo!04G+p)4xj*m6%RSipSBYIHPyPFlr;_&uy5)?J ztk`u%TGd>q#!&jtp|B*O1Y!(+9sK|x!h0+tY8y8@pj$H1*CS`_$yQvty(r$D97o+D z1dPui5#z3*^z$P@S)wQR6q_w%<JH+vt}<4Wo9pdx<Ijt-(4LA=#$yBsZvtnlIn<?B z{<M(;ANv!L%|1?eO~XATFCGK;oBlZNPTR1skG)GnPlS$vNgFUar_x9eVWa8$1n^Dp zY%`U}scwKzFeXemilGUWm@8*)6A`)MC`*`PKo3orOL4~@XF|DrHqTg0H?cn>=9w~> zi`g7EF=0+%<mUhki(E+;G2xX0oyf6E)-FV!z%!&7+P-A%FDQtR{dMg!{lj`|7@FqX z910`WSZ|H+KmKSh=}W6Q#q@pc@BiQ!$A%k^cbrffz1wG7$H4eh;?2lq|EXYMckm&8 zxrSKD2_WX!_O)x;c;nbZpsCB*TFFAwc&OwQ9nYz=BQ(kabiP9BaxR?(!Eum-m!fax zeeTv1gkFmNb`GCTA8|#5(DWzBL&*U`Fva3Yl9aWfOg5Uz+3({gvQT8UI*y(8jE>KP z^WaB(?Va9>>k=RWS5<m%2%i5{@=HnZVl8f#xxXp#Qi2PoJVIK_3nh(VuD*EaUqK0f fg-c=>Y`I~03Mc+Z)fkC5yH3#liQ;Wr+8+NOMu9BH diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-Digest.crc32 b/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-Digest.crc32 deleted file mode 100644 index 3e0bf3e604f6..000000000000 --- a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_clust_counter/nc-1-big-Digest.crc32 +++ /dev/null @@ -1 +0,0 @@ -3677152410 \ No newline at end of file diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple/nc-1-big-Data.db b/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple/nc-1-big-Data.db deleted file mode 100644 index fe53589beb7c62878383e8507b07d8f444d0f2db..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 87 zcmZ3>z`!txfzhD;|Ns9D42%pv*;M%X892Dp^79pv5{rPiijjfQP>~@<kpG~ffHH%O mkurnMgGyxvX=7yun?oh9fI=q9461(@DKi9smD@=!Tn_*N=oeK0 diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple/nc-1-big-Digest.crc32 b/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple/nc-1-big-Digest.crc32 deleted file mode 100644 index 67f6298a97ca..000000000000 --- a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple/nc-1-big-Digest.crc32 +++ /dev/null @@ -1 +0,0 @@ -1155625239 \ No newline at end of file diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-CompressionInfo.db b/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-CompressionInfo.db deleted file mode 100644 index 1db9aa06b311ba8dae1bc2eaaeca080ba080a480..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 47 pcmZSJ^@%cZ&d)6<N-ZwVFJb@#2Zs9p|NleSj9NgN5yEC*000^~2>k#6 diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-Data.db b/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-Data.db deleted file mode 100644 index 1431e3e27d5573a09206afa2dfc71aa95d0792d3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 138 zcmdO5WMG)Yz-Una|Ns9621bTY!U`Jv3=Ap^j13GrnR?z`<$gz(JYi%>llXgzf%VIk z3lko!&|t`9U}R)qG_+#K*Wzb5C>m+Qz-~VAQH2eItdR|a{*^-@{-lKuY#3yWZ5Zqi i<w}4ACR<qm6_@}Oe7#`9z_H=b2L=dWWUM}>AOiq7yeF6d diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-Digest.crc32 b/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-Digest.crc32 deleted file mode 100644 index 18641177fbf2..000000000000 --- a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-Digest.crc32 +++ /dev/null @@ -1 +0,0 @@ -4045937701 \ No newline at end of file diff --git a/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-Index.db b/test/data/legacy-sstables/nc/legacy_tables/legacy_nc_simple_counter/nc-1-big-Index.db deleted file mode 100644 index 59e65cab85e287ef068ea9b6ef8bea46f0d1664b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 27 ecmZQzG+<z0U^KJ>5=Lb}!nk1*kThv{#{d8(*aT1j diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-1-big-CompressionInfo.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-500-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-1-big-CompressionInfo.db rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-500-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-1-big-Data.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-500-big-Data.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-1-big-Data.db rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-500-big-Data.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-1-big-Digest.crc32 b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-500-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-1-big-Digest.crc32 rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-500-big-Digest.crc32 diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-1-big-Filter.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-500-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-1-big-Filter.db rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-500-big-Filter.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-1-big-Index.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-500-big-Index.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-1-big-Index.db rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-500-big-Index.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-1-big-Statistics.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-500-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-1-big-Statistics.db rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-500-big-Statistics.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-500-big-Summary.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-500-big-Summary.db new file mode 100644 index 0000000000000000000000000000000000000000..9b24e0450c732103bdd22afb63db34808745ed6f GIT binary patch literal 47 jcmZQzU}#`qU|<Ad5a0#k1|Vi+0TKp43<HcH4x<SGD_8-l literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-1-big-TOC.txt b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-500-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-1-big-TOC.txt rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust/oa-500-big-TOC.txt diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-1-big-CompressionInfo.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-500-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-1-big-CompressionInfo.db rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-500-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-1-big-Data.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-500-big-Data.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-1-big-Data.db rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-500-big-Data.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-1-big-Digest.crc32 b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-500-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-1-big-Digest.crc32 rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-500-big-Digest.crc32 diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-1-big-Filter.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-500-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-1-big-Filter.db rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-500-big-Filter.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-1-big-Index.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-500-big-Index.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-1-big-Index.db rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-500-big-Index.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-1-big-Statistics.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-500-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-1-big-Statistics.db rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-500-big-Statistics.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-500-big-Summary.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-500-big-Summary.db new file mode 100644 index 0000000000000000000000000000000000000000..9b24e0450c732103bdd22afb63db34808745ed6f GIT binary patch literal 47 jcmZQzU}#`qU|<Ad5a0#k1|Vi+0TKp43<HcH4x<SGD_8-l literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-1-big-TOC.txt b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-500-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-1-big-TOC.txt rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_clust_counter/oa-500-big-TOC.txt diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-1-big-CompressionInfo.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-500-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-1-big-CompressionInfo.db rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-500-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-1-big-Data.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-500-big-Data.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-1-big-Data.db rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-500-big-Data.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-1-big-Digest.crc32 b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-500-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-1-big-Digest.crc32 rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-500-big-Digest.crc32 diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-1-big-Filter.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-500-big-Filter.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-1-big-Filter.db rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-500-big-Filter.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-1-big-Index.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-500-big-Index.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-1-big-Index.db rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-500-big-Index.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-1-big-Statistics.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-500-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-1-big-Statistics.db rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-500-big-Statistics.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-500-big-Summary.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-500-big-Summary.db new file mode 100644 index 0000000000000000000000000000000000000000..9b24e0450c732103bdd22afb63db34808745ed6f GIT binary patch literal 47 jcmZQzU}#`qU|<Ad5a0#k1|Vi+0TKp43<HcH4x<SGD_8-l literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-1-big-TOC.txt b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-500-big-TOC.txt similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-1-big-TOC.txt rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple/oa-500-big-TOC.txt diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-1-big-CompressionInfo.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-500-big-CompressionInfo.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-1-big-CompressionInfo.db rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-500-big-CompressionInfo.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-1-big-Data.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-500-big-Data.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-1-big-Data.db rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-500-big-Data.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-1-big-Digest.crc32 b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-500-big-Digest.crc32 similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-1-big-Digest.crc32 rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-500-big-Digest.crc32 diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-500-big-Filter.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-500-big-Filter.db new file mode 100644 index 0000000000000000000000000000000000000000..8868e5c18008783c106cc95dcb2a5942cf662a48 GIT binary patch literal 24 fcmZQzU|?lnU|?!-Qe<ObWMN=f#-zZ)aDW2<72X1l literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-1-big-Index.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-500-big-Index.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-1-big-Index.db rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-500-big-Index.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-1-big-Statistics.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-500-big-Statistics.db similarity index 100% rename from test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-1-big-Statistics.db rename to test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-500-big-Statistics.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-500-big-Summary.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-500-big-Summary.db new file mode 100644 index 0000000000000000000000000000000000000000..9b24e0450c732103bdd22afb63db34808745ed6f GIT binary patch literal 47 jcmZQzU}#`qU|<Ad5a0#k1|Vi+0TKp43<HcH4x<SGD_8-l literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-500-big-TOC.txt b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-500-big-TOC.txt new file mode 100644 index 000000000000..576c5e0598c5 --- /dev/null +++ b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_simple_counter/oa-500-big-TOC.txt @@ -0,0 +1,8 @@ +Data.db +Statistics.db +Digest.crc32 +TOC.txt +CompressionInfo.db +Filter.db +Index.db +Summary.db diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-CompressionInfo.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-CompressionInfo.db new file mode 100644 index 0000000000000000000000000000000000000000..5a82349dd2e22fcbf900172acbc7f84c8e531651 GIT binary patch literal 47 pcmZSJ^@%cZ&d)6<N-ZwVFJb@#2Zs9p|NleStapJlBZSSs001M23A6wJ literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-Data.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-Data.db new file mode 100644 index 0000000000000000000000000000000000000000..f605927b22c1e244293dc56257911bb58e58140f GIT binary patch literal 301 zcmWlU%SyvQ6hQB^31%hQg53x<f~ZJneSo--enN|WLFSRBd32JQq-JJSKR{gTcL?3K zg{)n>@E;UGw@w#laS!KSxSuw_&oiL@oIbi1a^`b^^)y#n(FF5{m=eiT8jG2TBN0+O z);vqIEU!uuE14;sQ!IE8`lTYFm<2iI8b_HDB&{N*r#w@VMPoWEd?^Xxk_nx#GD$G; zQ_bQ@k_VHVGg0D#R#Fy)a}5CdhNi(XAP3qF2ktj>y9-_9yRd_8I3PE0JZc%ceFuFX zlP!3nEZ8hG_TU`>IyX-pS<A)i8TuV87cU;NcP~6NSPj2DV^+g2V_KIU>eYuH`s>y1 ZrnoZ2x^rrZ+jZyU%^Ee$RaAe};XjEjUZns4 literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-Digest.crc32 b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-Digest.crc32 new file mode 100644 index 000000000000..c5c321ba1866 --- /dev/null +++ b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-Digest.crc32 @@ -0,0 +1 @@ +3312043736 \ No newline at end of file diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-Filter.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-Filter.db new file mode 100644 index 0000000000000000000000000000000000000000..8868e5c18008783c106cc95dcb2a5942cf662a48 GIT binary patch literal 24 fcmZQzU|?lnU|?!-Qe<ObWMN=f#-zZ)aDW2<72X1l literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-Index.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-Index.db new file mode 100644 index 0000000000000000000000000000000000000000..8d7a18f061e88d6a4dfe06cae9fc0985a8eb8f5f GIT binary patch literal 29 gcmZQzG+<z0U^Hyh1d>KgVL;Njxfw{Bv}|Ai05x9(yZ`_I literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-Statistics.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-Statistics.db new file mode 100644 index 0000000000000000000000000000000000000000..979a9a888ce472e729f7c47fb02437eb7806192b GIT binary patch literal 5785 zcmeI$dr%a0902g$dvH8YI8b?vR4T$TbRfswL8&O70zC%ekyJXe9Lp;_nBz1c9LUgS z$RV<bobi!X3<C@lR05&Rq(X*;k`v|xL(PQ^PGOEB1VMXn3*Yarn!Ep+ac*aJc0c$1 z?%waU`}^Hd6eU_G_Y+eKTL^ccxjmQL1>Am^+l3S*tH|x1x=)1`7_t>iA*0J-6*{J< zh|y;n7)54|Q4v#OWD^XTtbxr;WDLe!V{U<-HAGjI{uJBd^>X^PzB=wkesgnkvu-}L z_{g1ubj`#=ciS5UZD(i1BQrzpFJ71(=2DJvyLmI00dAztWrG`6Mzo2LD2lHi|FHvq zVmV3(I}&y#>`GWl*n_YaVQ<2#2(KX=OgNnI2EuWKpCz14xQuWu;da7<*fifCmUgDq zMBGUD9JXL3;RD!0{{EJah5R_NSd@zM#Y$|4jo1>RzK+v4?%a#*I*z^UAhzs1>=n;r z`wn0~9E!c}B(@?ATXP&crW-qPId-}acF{A~RSb4LF@AfsI6m%=O#A<VotA=q9FI?W z{CgA^B;)5<@HMtD?OT*D65;VLx_TMK#a~q;JKSzWc1)f_cA908U1mLzrI~m<dT4(| zaj(u5$lk-g$g2jEkk|C%{tlkR{jIF|0_ATU`33ojEx3Ho`I=BX?>8B8MM5j`(Wyw} zjtjVbhB{Dt($4PpU_KokYn}%dH@bXv!y>p7y6=1#v>@;ddcO+ZkzRvwp>Hu7AHvRE zPB1PicG?6j-Z_Zs>rjwe4C9gx(*(4WcH3QO7ymBwJh`RGuE4mvFun)cQxky3^GfN| zyD;w8^yV1!BN1_J(1F)?Y=92ElG6emmA?N3wDv~W4(OzldsNW6fX_BV@B7nj6uSCi zV=Q!2^3m1M-_%vU0X^~Qz&2ny$StE2TDNs8eE#Xi#53^ur{~_5!slPWnub3I*9%6P zH<bbljkM_quxM?+EFOAtvJCc*_<l_-oIfOT&(SR)-?8rF(ocY$6HfIQfL(W2{|@Uf z{lQ}l*5Bi5T)Pb9FK4O>;e6oJ5Z$E!asNxBGT2{h@5IJefp|#jcmP}vsFb&Qq#zzW zc(dyz;HQqYHpB9~Fdn6X<5{1it*-(3l^GqqBfuwq9O|wIK6m>4R#;xs`QQP~Qk<5( zdU+-WI6m**^a0@4pALMqANa=3+Psh@3!)w$EzbdN@@fg+zht4ZE${Tqk}1Bf_?`aG zHJt!lV?O<_HJsRz#~=C2&^w0v(7uM_TV~-{i%HqKhO=B|U&G<{v#;S$d)U`-_B9+B z5cW0P|G9>vYj?dR^`%_Hjg&*^-b07`B=g7WpTn|y9H+UzT^4`;L|;li`(0wwEw1a# z^Yi>1o!#T)#hv5%m5YE0mj6A;<ygLp_IqvNdT3A%iXLm$rnqd5Eo&I<5Bl(f@sB7v zjE*Ru<gT)8ry_55{k1IkdfStd+a4G7(^SyE|G$)>*uxl#a+rKYYRdM=l#)W0E3B9; zHX0b4uMqBI@+<nS-9Ke-VLogA0;~5;W{o!G#p{jAaC4?S#M+nJdP}|$<Z=;LsqhD~ zR?1i0tu@znLqRF4x2Y3Si&(>aVm_PA=t_1QnT&k4C`7GOtJE6xdbLs$rOHxgsn@Hr zG!Yt=Cd?e+&Q<CNO+<*A<x(=$s>iIA$oA30ccIlW!6(WWIy<7mWUUYnLS5&+)O-I8 D*n<N) literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-Summary.db b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-Summary.db new file mode 100644 index 0000000000000000000000000000000000000000..9b24e0450c732103bdd22afb63db34808745ed6f GIT binary patch literal 47 jcmZQzU}#`qU|<Ad5a0#k1|Vi+0TKp43<HcH4x<SGD_8-l literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-TOC.txt b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-TOC.txt new file mode 100644 index 000000000000..576c5e0598c5 --- /dev/null +++ b/test/data/legacy-sstables/oa/legacy_tables/legacy_oa_tuple/oa-500-big-TOC.txt @@ -0,0 +1,8 @@ +Data.db +Statistics.db +Digest.crc32 +TOC.txt +CompressionInfo.db +Filter.db +Index.db +Summary.db diff --git a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java index 3a1bf38e0349..8081b9bffbd8 100644 --- a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java @@ -19,13 +19,14 @@ import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Random; -import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import org.junit.After; @@ -53,6 +54,7 @@ import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.io.sstable.format.SSTableFormat; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.format.Version; @@ -95,13 +97,22 @@ public class LegacySSTableTest * When adding a new sstable version, add that one here. * See {@link #testGenerateSstables()} to generate sstables. * Take care on commit as you need to add the sstable files using {@code git add -f} + * + * There are two me sstables, where the sequence number indicates the C* version they come from. + * For example: + * me-3025-big-* sstables are generated from 3.0.25 + * me-31111-big-* sstables are generated from 3.11.11 + * Both exist because of differences introduced in 3.6 (and 3.11) in how frozen multi-cell headers are serialised + * without the sstable format `me` being bumped, ref CASSANDRA-15035 + * + * Sequence numbers represent the C* version used when creating the SSTable, i.e. with #testGenerateSstables() */ public static String[] legacyVersions = null; // Get all versions up to the current one. Useful for testing in compatibility mode C18301 private static String[] getValidLegacyVersions() { - String[] versions = {"oa", "da", "nb", "na", "me", "md", "mc", "mb", "ma"}; + String[] versions = {"da", "oa", "nb", "na", "me", "md", "mc", "mb", "ma"}; return Arrays.stream(versions).filter((v) -> v.compareTo(BigFormat.getInstance().getLatestVersion().toString()) <= 0).toArray(String[]::new); } @@ -138,7 +149,6 @@ public static void defineSchema() throws ConfigurationException { createTables(legacyVersion); } - } @After @@ -155,9 +165,11 @@ public void tearDown() */ protected Descriptor getDescriptor(String legacyVersion, String table) throws IOException { - File[] files = getTableDir(legacyVersion, table).list(); - Preconditions.checkArgument(files.length > 0, "No files for version=%s and table=%s", legacyVersion, table); - return Descriptor.fromFileWithComponent(files[0]).left; + Path file = Files.list(getTableDir(legacyVersion, table).toPath()) + .findFirst() + .orElseThrow(() -> new RuntimeException(String.format("No files for verion=%s and table=%s", legacyVersion, table))); + + return Descriptor.fromFile(new File(file)); } @Test @@ -317,7 +329,7 @@ public void testStreamLegacyCqlTables() throws Exception public void testInaccurateSSTableMinMax() throws Exception { QueryProcessor.executeInternal("CREATE TABLE legacy_tables.legacy_mc_inaccurate_min_max (k int, c1 int, c2 int, c3 int, v int, primary key (k, c1, c2, c3))"); - loadLegacyTable("legacy_%s_inaccurate_min_max", "mc"); + loadLegacyTable("mc", "inaccurate_min_max"); /* sstable has the following mutations: @@ -335,12 +347,65 @@ public void testInaccurateSSTableMinMax() throws Exception } @Test - public void testVerifyOldSSTables() throws IOException + public void testVerifyOldSimpleSSTables() throws IOException + { + verifyOldSSTables("simple"); + } + + @Test + public void testVerifyOldTupleSSTables() throws IOException + { + verifyOldSSTables("tuple"); + } + + @Test + public void testVerifyOldDroppedTupleSSTables() throws IOException + { + try { + for (String legacyVersion : legacyVersions) + { + QueryProcessor.executeInternal(String.format("ALTER TABLE legacy_tables.legacy_%s_tuple DROP val", legacyVersion)); + QueryProcessor.executeInternal(String.format("ALTER TABLE legacy_tables.legacy_%s_tuple DROP val2", legacyVersion)); + QueryProcessor.executeInternal(String.format("ALTER TABLE legacy_tables.legacy_%s_tuple DROP val3", legacyVersion)); + // dropping non-frozen UDTs disabled, see AlterTableStatement.DropColumns.dropColumn(..) + //QueryProcessor.executeInternal(String.format("ALTER TABLE legacy_tables.legacy_%s_tuple DROP val4", legacyVersion)); + } + + verifyOldSSTables("tuple"); + } + finally + { + for (String legacyVersion : legacyVersions) + { + alterTableAddColumn(legacyVersion, "val frozen<tuple<set<int>,set<text>>>"); + alterTableAddColumn(legacyVersion, "val2 tuple<set<int>,set<text>>"); + try + { + alterTableAddColumn(legacyVersion, String.format("val3 frozen<legacy_%s_tuple_udt>", legacyVersion)); + throw new AssertionError(String.format("Against legacyVersion %s expected InvalidRequestException: Cannot re-add previously dropped column 'val3' of type frozen<legacy_da_tuple_udt>, incompatible with previous type frozen<tuple<frozen<tuple<text, text>>>>", legacyVersion)); + } + catch (InvalidRequestException ex) + { + // expected + // InvalidRequestException: Cannot re-add previously dropped column 'val3' of type frozen<legacy_da_tuple_udt>, incompatible with previous type frozen<tuple<frozen<tuple<text, text>>>> + } + // dropping non-frozen UDTs disabled, see AlterTableStatement.DropColumns.dropColumn(..) + //alterTableAddColumn(legacyVersion, String.format("val4 legacy_%s_tuple_udt", legacyVersion)); + } + } + } + + private static void alterTableAddColumn(String legacyVersion, String column_definition) + { + QueryProcessor.executeInternal(String.format("ALTER TABLE legacy_tables.legacy_%s_tuple ADD IF NOT EXISTS %s", legacyVersion, column_definition)); + } + + private void verifyOldSSTables(String tableSuffix) throws IOException { for (String legacyVersion : legacyVersions) { - ColumnFamilyStore cfs = Keyspace.open(LEGACY_TABLES_KEYSPACE).getColumnFamilyStore(String.format("legacy_%s_simple", legacyVersion)); - loadLegacyTable("legacy_%s_simple", legacyVersion); + ColumnFamilyStore cfs = Keyspace.open(LEGACY_TABLES_KEYSPACE).getColumnFamilyStore(String.format("legacy_%s_%s", legacyVersion, tableSuffix)); + loadLegacyTable(legacyVersion, tableSuffix); for (SSTableReader sstable : cfs.getLiveSSTables()) { @@ -374,7 +439,7 @@ public void testPendingAntiCompactionOldSSTables() throws Exception for (String legacyVersion : legacyVersions) { ColumnFamilyStore cfs = Keyspace.open(LEGACY_TABLES_KEYSPACE).getColumnFamilyStore(String.format("legacy_%s_simple", legacyVersion)); - loadLegacyTable("legacy_%s_simple", legacyVersion); + loadLegacyTable(legacyVersion, "simple"); boolean shouldFail = !cfs.getLiveSSTables().stream().allMatch(sstable -> sstable.descriptor.version.hasPendingRepair()); IPartitioner p = Iterables.getFirst(cfs.getLiveSSTables(), null).getPartitioner(); @@ -422,27 +487,34 @@ public void testAutomaticUpgrade() throws Exception private void streamLegacyTables(String legacyVersion) throws Exception { - logger.info("Streaming legacy version {}", legacyVersion); - streamLegacyTable("legacy_%s_simple", legacyVersion); - streamLegacyTable("legacy_%s_simple_counter", legacyVersion); - streamLegacyTable("legacy_%s_clust", legacyVersion); - streamLegacyTable("legacy_%s_clust_counter", legacyVersion); + logger.info("Streaming legacy version {}", legacyVersion); + streamLegacyTable("legacy_%s_simple", legacyVersion); + streamLegacyTable("legacy_%s_simple_counter", legacyVersion); + streamLegacyTable("legacy_%s_clust", legacyVersion); + streamLegacyTable("legacy_%s_clust_counter", legacyVersion); + streamLegacyTable("legacy_%s_tuple", legacyVersion); } private void streamLegacyTable(String tablePattern, String legacyVersion) throws Exception { String table = String.format(tablePattern, legacyVersion); - SSTableReader sstable = SSTableReader.open(null, getDescriptor(legacyVersion, table)); - IPartitioner p = sstable.getPartitioner(); - List<Range<Token>> ranges = new ArrayList<>(); - ranges.add(new Range<>(p.getMinimumToken(), p.getToken(ByteBufferUtil.bytes("100")))); - ranges.add(new Range<>(p.getToken(ByteBufferUtil.bytes("100")), p.getMinimumToken())); - List<OutgoingStream> streams = Lists.newArrayList(new CassandraOutgoingFile(StreamOperation.OTHER, - sstable.ref(), - sstable.getPositionsForRanges(ranges), - ranges, - sstable.estimatedKeysForRanges(ranges))); - new StreamPlan(StreamOperation.OTHER).transferStreams(FBUtilities.getBroadcastAddressAndPort(), streams).execute().get(); + Descriptor descriptor = getDescriptor(legacyVersion, table); + if (null != descriptor) + { + SSTableReader sstable = SSTableReader.open(null, getDescriptor(legacyVersion, table)); + IPartitioner p = sstable.getPartitioner(); + List<Range<Token>> ranges = new ArrayList<>(); + ranges.add(new Range<>(p.getMinimumToken(), p.getToken(ByteBufferUtil.bytes("100")))); + ranges.add(new Range<>(p.getToken(ByteBufferUtil.bytes("100")), p.getMinimumToken())); + + List<OutgoingStream> streams = Lists.newArrayList(new CassandraOutgoingFile(StreamOperation.OTHER, + sstable.ref(), + sstable.getPositionsForRanges(ranges), + ranges, + sstable.estimatedKeysForRanges(ranges))); + + new StreamPlan(StreamOperation.OTHER).transferStreams(FBUtilities.getBroadcastAddressAndPort(), streams).execute().get(); + } } public static void truncateLegacyTables(String legacyVersion) throws Exception @@ -452,6 +524,9 @@ public static void truncateLegacyTables(String legacyVersion) throws Exception Keyspace.open(LEGACY_TABLES_KEYSPACE).getColumnFamilyStore(String.format("legacy_%s_simple_counter", legacyVersion)).truncateBlocking(); Keyspace.open(LEGACY_TABLES_KEYSPACE).getColumnFamilyStore(String.format("legacy_%s_clust", legacyVersion)).truncateBlocking(); Keyspace.open(LEGACY_TABLES_KEYSPACE).getColumnFamilyStore(String.format("legacy_%s_clust_counter", legacyVersion)).truncateBlocking(); + Keyspace.open(LEGACY_TABLES_KEYSPACE).getColumnFamilyStore(String.format("legacy_%s_tuple", legacyVersion)).truncateBlocking(); + CacheService.instance.invalidateCounterCache(); + CacheService.instance.invalidateKeyCache(); } private static void compactLegacyTables(String legacyVersion) throws Exception @@ -461,15 +536,17 @@ private static void compactLegacyTables(String legacyVersion) throws Exception Keyspace.open(LEGACY_TABLES_KEYSPACE).getColumnFamilyStore(String.format("legacy_%s_simple_counter", legacyVersion)).forceMajorCompaction(); Keyspace.open(LEGACY_TABLES_KEYSPACE).getColumnFamilyStore(String.format("legacy_%s_clust", legacyVersion)).forceMajorCompaction(); Keyspace.open(LEGACY_TABLES_KEYSPACE).getColumnFamilyStore(String.format("legacy_%s_clust_counter", legacyVersion)).forceMajorCompaction(); + Keyspace.open(LEGACY_TABLES_KEYSPACE).getColumnFamilyStore(String.format("legacy_%s_tuple", legacyVersion)).forceMajorCompaction(); } public static void loadLegacyTables(String legacyVersion) throws Exception { - logger.info("Preparing legacy version {}", legacyVersion); - loadLegacyTable("legacy_%s_simple", legacyVersion); - loadLegacyTable("legacy_%s_simple_counter", legacyVersion); - loadLegacyTable("legacy_%s_clust", legacyVersion); - loadLegacyTable("legacy_%s_clust_counter", legacyVersion); + logger.info("Preparing legacy version {}", legacyVersion); + loadLegacyTable(legacyVersion, "simple"); + loadLegacyTable(legacyVersion, "simple_counter"); + loadLegacyTable(legacyVersion, "clust"); + loadLegacyTable(legacyVersion, "clust_counter"); + loadLegacyTable(legacyVersion, "tuple"); } private static void verifyCache(String legacyVersion, long startCount) throws InterruptedException, java.util.concurrent.ExecutionException @@ -567,6 +644,21 @@ private static void createTables(String legacyVersion) QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_simple_counter (pk text PRIMARY KEY, val counter)", legacyVersion)); QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_clust (pk text, ck text, val text, PRIMARY KEY (pk, ck))", legacyVersion)); QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_clust_counter (pk text, ck text, val counter, PRIMARY KEY (pk, ck))", legacyVersion)); + + + QueryProcessor.executeInternal(String.format("CREATE TYPE legacy_tables.legacy_%s_tuple_udt (name tuple<text,text>)", legacyVersion)); + + if (legacyVersion.startsWith("m")) + { + // sstable formats possibly from 3.0.x would have had a schema with everything frozen + QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%1$s_tuple (pk text PRIMARY KEY, " + + "val frozen<tuple<set<int>,set<text>>>, val2 frozen<tuple<set<int>,set<text>>>, val3 frozen<legacy_%1$s_tuple_udt>, val4 frozen<legacy_%1$s_tuple_udt>, extra text)", legacyVersion)); + } + else + { + QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%1$s_tuple (pk text PRIMARY KEY, " + + "val frozen<tuple<set<int>,set<text>>>, val2 tuple<set<int>,set<text>>, val3 frozen<legacy_%1$s_tuple_udt>, val4 legacy_%1$s_tuple_udt, extra text)", legacyVersion)); + } } private static void truncateTables(String legacyVersion) @@ -592,10 +684,9 @@ private static void assertLegacyClustRows(int count, UntypedResultSet rs) } } - private static void loadLegacyTable(String tablePattern, String legacyVersion) throws IOException + private static void loadLegacyTable(String legacyVersion, String tableSuffix) throws IOException { - String table = String.format(tablePattern, legacyVersion); - + String table = String.format("legacy_%s_%s", legacyVersion, tableSuffix); logger.info("Loading legacy table {}", table); ColumnFamilyStore cfs = Keyspace.open(LEGACY_TABLES_KEYSPACE).getColumnFamilyStore(table); @@ -609,9 +700,11 @@ private static void loadLegacyTable(String tablePattern, String legacyVersion) t } /** - * Generates sstables for 8 CQL tables (see {@link #createTables(String)}) in <i>current</i> + * Generates sstables for CQL tables (see {@link #createTables(String)}) in <i>current</i> * sstable format (version) into {@code test/data/legacy-sstables/VERSION}, where * {@code VERSION} matches {@link Version#version BigFormat.latestVersion.getVersion()}. + * + * Sequence numbers are changed to represent the C* version used when creating the SSTable. * <p> * Run this test alone (e.g. from your IDE) when a new version is introduced or format changed * during development. I.e. remove the {@code @Ignore} annotation temporarily. @@ -639,6 +732,11 @@ public void testGenerateSstables() throws Throwable QueryProcessor.executeInternal(String.format("UPDATE legacy_tables.legacy_%s_simple_counter SET val = val + 1 WHERE pk = '%s'", format.getLatestVersion(), valPk)); + QueryProcessor.executeInternal( + String.format("INSERT INTO legacy_tables.legacy_%s_tuple (pk, val, val2, val3, val4, extra)" + + " VALUES ('%s', ({1,2,3},{'a','b','c'}), ({1,2,3},{'a','b','c'}), {name: ('abc','def')}, {name: ('abc','def')}, '%s')", + format.getLatestVersion(), valPk, randomString)); + for (int ck = 0; ck < 50; ck++) { String valCk = Integer.toString(ck); @@ -655,14 +753,21 @@ public void testGenerateSstables() throws Throwable File ksDir = new File(LEGACY_SSTABLE_ROOT, String.format("%s/legacy_tables", format.getLatestVersion())); ksDir.tryCreateDirectories(); - copySstablesFromTestData(String.format("legacy_%s_simple", format.getLatestVersion()), ksDir, LEGACY_TABLES_KEYSPACE); - copySstablesFromTestData(String.format("legacy_%s_simple_counter", format.getLatestVersion()), ksDir, LEGACY_TABLES_KEYSPACE); - copySstablesFromTestData(String.format("legacy_%s_clust", format.getLatestVersion()), ksDir, LEGACY_TABLES_KEYSPACE); - copySstablesFromTestData(String.format("legacy_%s_clust_counter", format.getLatestVersion()), ksDir, LEGACY_TABLES_KEYSPACE); + copySstablesFromTestData(format.getLatestVersion(), "legacy_%s_simple", ksDir); + copySstablesFromTestData(format.getLatestVersion(), "legacy_%s_simple_counter", ksDir); + copySstablesFromTestData(format.getLatestVersion(), "legacy_%s_clust", ksDir); + copySstablesFromTestData(format.getLatestVersion(), "legacy_%s_clust_counter", ksDir); + copySstablesFromTestData(format.getLatestVersion(), "legacy_%s_tuple", ksDir); + } + + public static void copySstablesFromTestData(Version legacyVersion, String tablePattern, File ksDir) throws IOException + { + copySstablesFromTestData(legacyVersion, tablePattern, ksDir, LEGACY_TABLES_KEYSPACE); } - public static void copySstablesFromTestData(String table, File ksDir, String ks) throws IOException + public static void copySstablesFromTestData(Version legacyVersion, String tablePattern, File ksDir, String ks) throws IOException { + String table = String.format(tablePattern, legacyVersion); File cfDir = new File(ksDir, table); cfDir.tryCreateDirectory(); @@ -670,7 +775,10 @@ public static void copySstablesFromTestData(String table, File ksDir, String ks) { for (File file : srcDir.tryList()) { - copyFile(cfDir, file); + // Sequence IDs represent the C* version used when creating the SSTable, i.e. with #testGenerateSstables() (if not uuid based) + String newSeqId = FBUtilities.getReleaseVersionString().split("-")[0].replaceAll("[^0-9]", ""); + File target = new File(cfDir, file.name().replace(legacyVersion + "-1-", legacyVersion + "-" + newSeqId + "-")); + copyFile(cfDir, file, target); } } } @@ -680,9 +788,7 @@ private static void copySstablesToTestData(String legacyVersion, String table, F File tableDir = getTableDir(legacyVersion, table); Assert.assertTrue("The table directory " + tableDir + " was not found", tableDir.isDirectory()); for (File file : tableDir.tryList()) - { copyFile(cfDir, file); - } } private static File getTableDir(String legacyVersion, String table) @@ -691,17 +797,22 @@ private static File getTableDir(String legacyVersion, String table) } public static void copyFile(File cfDir, File file) throws IOException + { + copyFile(cfDir, file, new File(cfDir, file.name())); + } + + public static void copyFile(File cfDir, File file, File target) throws IOException { byte[] buf = new byte[65536]; if (file.isFile()) { - File target = new File(cfDir, file.name()); int rd; try (FileInputStreamPlus is = new FileInputStreamPlus(file); - FileOutputStreamPlus os = new FileOutputStreamPlus(target);) { + FileOutputStreamPlus os = new FileOutputStreamPlus(target);) + { while ((rd = is.read(buf)) >= 0) os.write(buf, 0, rd); - } + } } } } diff --git a/test/unit/org/apache/cassandra/tools/SSTablePartitionsTest.java b/test/unit/org/apache/cassandra/tools/SSTablePartitionsTest.java index 7899a79341b3..bd38139001f9 100644 --- a/test/unit/org/apache/cassandra/tools/SSTablePartitionsTest.java +++ b/test/unit/org/apache/cassandra/tools/SSTablePartitionsTest.java @@ -41,12 +41,12 @@ public class SSTablePartitionsTest extends OfflineToolUtils { private static final String SSTABLE_1 = sstable("legacy_ma_simple"); private static final String SSTABLE_2 = sstable("legacy_ma_clust"); - private static final String HEADER_1 = "\nProcessing #1 (big-ma) (173 B uncompressed, 88 B on disk)\n"; - private static final String HEADER_2 = "\nProcessing #1 (big-ma) (328.145 KiB uncompressed, 5.096 KiB on disk)\n"; - private static final String BACKUPS_HEADER_1 = "\nProcessing Backup:backups #1 (big-ma) (173 B uncompressed, 88 B on disk)\n"; - private static final String BACKUPS_HEADER_2 = "\nProcessing Backup:backups #1 (big-ma) (328.145 KiB uncompressed, 5.096 KiB on disk)\n"; - private static final String SNAPSHOTS_HEADER_1 = "\nProcessing Snapshot:snapshot-1 #1 (big-ma) (173 B uncompressed, 88 B on disk)\n"; - private static final String SNAPSHOTS_HEADER_2 = "\nProcessing Snapshot:snapshot-1 #1 (big-ma) (328.145 KiB uncompressed, 5.096 KiB on disk)\n"; + private static final String HEADER_1 = "\nProcessing #306 (big-ma) (173 B uncompressed, 88 B on disk)\n"; + private static final String HEADER_2 = "\nProcessing #306 (big-ma) (328.145 KiB uncompressed, 5.096 KiB on disk)\n"; + private static final String BACKUPS_HEADER_1 = "\nProcessing Backup:backups #306 (big-ma) (173 B uncompressed, 88 B on disk)\n"; + private static final String BACKUPS_HEADER_2 = "\nProcessing Backup:backups #306 (big-ma) (328.145 KiB uncompressed, 5.096 KiB on disk)\n"; + private static final String SNAPSHOTS_HEADER_1 = "\nProcessing Snapshot:snapshot-1 #306 (big-ma) (173 B uncompressed, 88 B on disk)\n"; + private static final String SNAPSHOTS_HEADER_2 = "\nProcessing Snapshot:snapshot-1 #306 (big-ma) (328.145 KiB uncompressed, 5.096 KiB on disk)\n"; private static final String SUMMARY_1 = " Partition size Row count Cell count Tombstone count\n" + " ~p50 35 B 1 1 0\n" + " ~p75 35 B 1 1 0\n" + @@ -251,7 +251,7 @@ private static void testMinSize(String option) " Partition: '2' (32) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '3' (33) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '4' (34) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + - "Summary of #1 (big-ma):\n" + + "Summary of #306 (big-ma):\n" + " File: " + SSTABLE_2 + "\n" + " 5 partitions match\n" + " Keys: 0 1 2 3 4\n" + @@ -261,7 +261,7 @@ private static void testMinSize(String option) " Partition: '2' (32) live, size: 35 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '3' (33) live, size: 35 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '4' (34) live, size: 35 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + - "Summary of #1 (big-ma):\n" + + "Summary of #306 (big-ma):\n" + " File: " + SSTABLE_1 + "\n" + " 4 partitions match\n" + " Keys: 1 2 3 4\n" + @@ -274,7 +274,7 @@ private static void testMinSize(String option) " Partition: '2' (32) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '3' (33) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '4' (34) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + - "Summary of #1 (big-ma):\n" + + "Summary of #306 (big-ma):\n" + " File: " + SSTABLE_2 + "\n" + " 5 partitions match\n" + " Keys: 0 1 2 3 4\n" + @@ -286,7 +286,7 @@ private static void testMinSize(String option) " Partition: '2' (32) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '3' (33) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '4' (34) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + - "Summary of #1 (big-ma):\n" + + "Summary of #306 (big-ma):\n" + " File: " + SSTABLE_2 + "\n" + " 4 partitions match\n" + " Keys: 1 2 3 4\n" + @@ -298,7 +298,7 @@ private static void testMinSize(String option) " Partition: '2' (32) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '3' (33) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '4' (34) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + - "Summary of #1 (big-ma):\n" + + "Summary of #306 (big-ma):\n" + " File: " + SSTABLE_2 + "\n" + " 4 partitions match\n" + " Keys: 1 2 3 4\n" + @@ -311,7 +311,7 @@ private static void testMinSize(String option) " Partition: '2' (32) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '3' (33) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '4' (34) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + - "Summary of #1 (big-ma):\n" + + "Summary of #306 (big-ma):\n" + " File: " + SSTABLE_2 + "\n" + " 5 partitions match\n" + " Keys: 0 1 2 3 4\n" + @@ -337,7 +337,7 @@ private static void testMinCells(String option) " Partition: '2' (32) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '3' (33) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '4' (34) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + - "Summary of #1 (big-ma):\n" + + "Summary of #306 (big-ma):\n" + " File: " + SSTABLE_2 + "\n" + " 5 partitions match\n" + " Keys: 0 1 2 3 4\n" + @@ -348,7 +348,7 @@ private static void testMinCells(String option) " Partition: '2' (32) live, size: 35 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '3' (33) live, size: 35 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '4' (34) live, size: 35 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + - "Summary of #1 (big-ma):\n" + + "Summary of #306 (big-ma):\n" + " File: " + SSTABLE_1 + "\n" + " 5 partitions match\n" + " Keys: 0 1 2 3 4\n" + @@ -361,7 +361,7 @@ private static void testMinCells(String option) " Partition: '2' (32) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '3' (33) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '4' (34) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + - "Summary of #1 (big-ma):\n" + + "Summary of #306 (big-ma):\n" + " File: " + SSTABLE_2 + "\n" + " 5 partitions match\n" + " Keys: 0 1 2 3 4\n" + @@ -391,7 +391,7 @@ private static void testMinRows(String option) " Partition: '2' (32) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '3' (33) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '4' (34) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + - "Summary of #1 (big-ma):\n" + + "Summary of #306 (big-ma):\n" + " File: " + SSTABLE_2 + "\n" + " 5 partitions match\n" + " Keys: 0 1 2 3 4\n" + @@ -402,7 +402,7 @@ private static void testMinRows(String option) " Partition: '2' (32) live, size: 35 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '3' (33) live, size: 35 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '4' (34) live, size: 35 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + - "Summary of #1 (big-ma):\n" + + "Summary of #306 (big-ma):\n" + " File: " + SSTABLE_1 + "\n" + " 5 partitions match\n" + " Keys: 0 1 2 3 4\n" + @@ -415,7 +415,7 @@ private static void testMinRows(String option) " Partition: '2' (32) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '3' (33) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '4' (34) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + - "Summary of #1 (big-ma):\n" + + "Summary of #306 (big-ma):\n" + " File: " + SSTABLE_2 + "\n" + " 5 partitions match\n" + " Keys: 0 1 2 3 4\n" + @@ -445,7 +445,7 @@ private static void testMinTombstones(String option) " Partition: '2' (32) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '3' (33) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '4' (34) live, size: 65.630 KiB, rows: 50, cells: 50, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + - "Summary of #1 (big-ma):\n" + + "Summary of #306 (big-ma):\n" + " File: " + SSTABLE_2 + "\n" + " 5 partitions match\n" + " Keys: 0 1 2 3 4\n" + @@ -456,7 +456,7 @@ private static void testMinTombstones(String option) " Partition: '2' (32) live, size: 35 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '3' (33) live, size: 35 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '4' (34) live, size: 35 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + - "Summary of #1 (big-ma):\n" + + "Summary of #306 (big-ma):\n" + " File: " + SSTABLE_1 + "\n" + " 5 partitions match\n" + " Keys: 0 1 2 3 4\n" + @@ -536,7 +536,7 @@ private static void testIncludedKeys(String option) .contains(HEADER_1 + " Partition: '1' (31) live, size: 35 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '3' (33) live, size: 35 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + - "Summary of #1 (big-ma):\n" + + "Summary of #306 (big-ma):\n" + " File: " + SSTABLE_1 + "\n" + " 2 partitions match\n" + " Keys: 1 3\n") @@ -547,7 +547,7 @@ private static void testIncludedKeys(String option) " Partition: '0' (30) live, size: 33 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '2' (32) live, size: 35 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '4' (34) live, size: 35 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + - "Summary of #1 (big-ma):\n" + + "Summary of #306 (big-ma):\n" + " File: " + SSTABLE_1 + "\n" + " 3 partitions match\n" + " Keys: 0 2 4\n") @@ -558,7 +558,7 @@ private static void testIncludedKeys(String option) " Partition: '0' (30) live, size: 33 B\n" + " Partition: '2' (32) live, size: 35 B\n" + " Partition: '4' (34) live, size: 35 B\n" + - "Summary of #1 (big-ma):\n" + + "Summary of #306 (big-ma):\n" + " File: " + SSTABLE_1 + "\n" + " 3 partitions match\n" + " Keys: 0 2 4\n") @@ -582,7 +582,7 @@ private static void testExcludedKeys(String option) " Partition: '0' (30) live, size: 33 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '2' (32) live, size: 35 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '4' (34) live, size: 35 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + - "Summary of #1 (big-ma):\n" + + "Summary of #306 (big-ma):\n" + " File: " + SSTABLE_1 + "\n" + " 3 partitions match\n" + " Keys: 0 2 4\n") @@ -592,7 +592,7 @@ private static void testExcludedKeys(String option) .contains(HEADER_1 + " Partition: '1' (31) live, size: 35 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + " Partition: '3' (33) live, size: 35 B, rows: 1, cells: 1, tombstones: 0 (row:0, range:0, complex:0, cell:0, row-TTLd:0, cell-TTLd:0)\n" + - "Summary of #1 (big-ma):\n" + + "Summary of #306 (big-ma):\n" + " File: " + SSTABLE_1 + "\n" + " 2 partitions match\n" + " Keys: 1 3\n") @@ -616,15 +616,15 @@ private static void testCSV(String option) "rowTombstoneCount,rangeTombstoneCount,complexTombstoneCount,cellTombstoneCount," + "rowTtlExpired,cellTtlExpired,directory,keyspace,table,index,snapshot,backup," + "generation,format,version\n" + - "\"0\",30,true,0,67200,50,50,0,0,0,0,0,0,0,%s,,,,,,1,big,ma\n" + - "\"1\",31,true,67200,67205,50,50,0,0,0,0,0,0,0,%<s,,,,,,1,big,ma\n" + - "\"2\",32,true,134405,67205,50,50,0,0,0,0,0,0,0,%<s,,,,,,1,big,ma\n" + - "\"3\",33,true,201610,67205,50,50,0,0,0,0,0,0,0,%<s,,,,,,1,big,ma\n" + - "\"4\",34,true,268815,67205,50,50,0,0,0,0,0,0,0,%<s,,,,,,1,big,ma\n" + - "\"1\",31,true,33,35,1,1,0,0,0,0,0,0,0,%s,,,,,,1,big,ma\n" + - "\"2\",32,true,68,35,1,1,0,0,0,0,0,0,0,%<s,,,,,,1,big,ma\n" + - "\"3\",33,true,103,35,1,1,0,0,0,0,0,0,0,%<s,,,,,,1,big,ma\n" + - "\"4\",34,true,138,35,1,1,0,0,0,0,0,0,0,%<s,,,,,,1,big,ma\n", + "\"0\",30,true,0,67200,50,50,0,0,0,0,0,0,0,%s,,,,,,306,big,ma\n" + + "\"1\",31,true,67200,67205,50,50,0,0,0,0,0,0,0,%<s,,,,,,306,big,ma\n" + + "\"2\",32,true,134405,67205,50,50,0,0,0,0,0,0,0,%<s,,,,,,306,big,ma\n" + + "\"3\",33,true,201610,67205,50,50,0,0,0,0,0,0,0,%<s,,,,,,306,big,ma\n" + + "\"4\",34,true,268815,67205,50,50,0,0,0,0,0,0,0,%<s,,,,,,306,big,ma\n" + + "\"1\",31,true,33,35,1,1,0,0,0,0,0,0,0,%s,,,,,,306,big,ma\n" + + "\"2\",32,true,68,35,1,1,0,0,0,0,0,0,0,%<s,,,,,,306,big,ma\n" + + "\"3\",33,true,103,35,1,1,0,0,0,0,0,0,0,%<s,,,,,,306,big,ma\n" + + "\"4\",34,true,138,35,1,1,0,0,0,0,0,0,0,%<s,,,,,,306,big,ma\n", SSTABLE_2, SSTABLE_1)); } diff --git a/test/unit/org/apache/cassandra/utils/concurrent/RefCountedTest.java b/test/unit/org/apache/cassandra/utils/concurrent/RefCountedTest.java index d50e408d4d9c..c34d4a12ccb3 100644 --- a/test/unit/org/apache/cassandra/utils/concurrent/RefCountedTest.java +++ b/test/unit/org/apache/cassandra/utils/concurrent/RefCountedTest.java @@ -515,7 +515,7 @@ public void testSSTableReaderLeakIsDetected() { DatabaseDescriptor.clientInitialization(); DatabaseDescriptor.setPartitionerUnsafe(ByteOrderedPartitioner.instance); - Descriptor descriptor = Descriptor.fromFileWithComponent(new File("test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-1-big-Data.db"), false).left; + Descriptor descriptor = Descriptor.fromFileWithComponent(new File("test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-400-big-Data.db"), false).left; TableMetadata tm = TableMetadata.builder("legacy_tables", "legacy_nb_simple").addPartitionKeyColumn("pk", UTF8Type.instance).addRegularColumn("val", UTF8Type.instance).build(); AtomicBoolean leakDetected = new AtomicBoolean(); AtomicBoolean runOnCloseExecuted1 = new AtomicBoolean(); From 2a5749f203ac6626d153b91be555a89a19aeae7c Mon Sep 17 00:00:00 2001 From: Blake Eggleston <blake@ultrablake.com> Date: Thu, 27 Mar 2025 10:39:32 -0700 Subject: [PATCH 219/225] convert legacy zero ballots to 4.1 zero value on deserialization Patch by Blake Eggleston, reviewed by Alex Petrov and Ariel Weisberg for CASSANDRA-20493 --- CHANGES.txt | 1 + .../cassandra/service/paxos/Ballot.java | 15 +- .../org/apache/cassandra/utils/TimeUUID.java | 2 +- .../MixedModePaxosLegacyCoordinatorTest.java | 28 +++ .../upgrade/MixedModePaxosTestBase.java | 203 ++++++++++++++++++ ...MixedModePaxosUpgradedCoordinatorTest.java | 28 +++ 6 files changed, 275 insertions(+), 2 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/upgrade/MixedModePaxosLegacyCoordinatorTest.java create mode 100644 test/distributed/org/apache/cassandra/distributed/upgrade/MixedModePaxosTestBase.java create mode 100644 test/distributed/org/apache/cassandra/distributed/upgrade/MixedModePaxosUpgradedCoordinatorTest.java diff --git a/CHANGES.txt b/CHANGES.txt index e9a374bdfe42..d7b2baed6cec 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.1.9 + * Fix paxos mixed mode infinite loop (CASSANDRA-20493) * Optionally skip exception logging on invalid legacy protocol magic exception (CASSANDRA-19483) * Fix SimpleClient ability to release acquired capacity (CASSANDRA-20202) * Fix WaitQueue.Signal.awaitUninterruptibly may block forever if invoking thread is interrupted (CASSANDRA-20084) diff --git a/src/java/org/apache/cassandra/service/paxos/Ballot.java b/src/java/org/apache/cassandra/service/paxos/Ballot.java index 1787e34e0dca..01d2700bc916 100644 --- a/src/java/org/apache/cassandra/service/paxos/Ballot.java +++ b/src/java/org/apache/cassandra/service/paxos/Ballot.java @@ -77,7 +77,20 @@ public static Ballot fromUuid(UUID uuid) public static Ballot fromBytes(long msb, long lsb) { - return new Ballot(msbToRawTimestamp(msb), lsb); + msb = msbToRawTimestamp(msb); + + /** + * CEP-14 changed the lsb of Ballot#none, which caused some paxos coordination problems + * in mixed mode (see CASSANDRA-20493) so here we detect the old none value and convert + * it to the current one. + * + * See the comment on * {@link org.apache.cassandra.distributed.upgrade.MixedModePaxosTest} + * for a full explanation. + */ + if (lsb == TimeUUID.MIN_CLOCK_SEQ_AND_NODE && rawTimestampToUnixMicros(msb) == 0) + return Ballot.none(); + + return new Ballot(msb, lsb); } public static Ballot fromString(String uuidString) diff --git a/src/java/org/apache/cassandra/utils/TimeUUID.java b/src/java/org/apache/cassandra/utils/TimeUUID.java index 8d79096ecdc7..b6ea93205f5d 100644 --- a/src/java/org/apache/cassandra/utils/TimeUUID.java +++ b/src/java/org/apache/cassandra/utils/TimeUUID.java @@ -79,7 +79,7 @@ public class TimeUUID implements Serializable, Comparable<TimeUUID> * I don't think that has any practical consequence and is more robust in * case someone provides a UUID with a broken variant. */ - private static final long MIN_CLOCK_SEQ_AND_NODE = 0x8080808080808080L; + public static final long MIN_CLOCK_SEQ_AND_NODE = 0x8080808080808080L; private static final long MAX_CLOCK_SEQ_AND_NODE = 0x7f7f7f7f7f7f7f7fL; diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModePaxosLegacyCoordinatorTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModePaxosLegacyCoordinatorTest.java new file mode 100644 index 000000000000..c3263174b89b --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModePaxosLegacyCoordinatorTest.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.upgrade; + +public class MixedModePaxosLegacyCoordinatorTest extends MixedModePaxosTestBase +{ + @Override + boolean upgradedCoordinator() + { + return false; + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModePaxosTestBase.java b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModePaxosTestBase.java new file mode 100644 index 000000000000..12e7b9656112 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModePaxosTestBase.java @@ -0,0 +1,203 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.upgrade; + +import java.nio.ByteBuffer; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import org.junit.Assert; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.SimpleBuilders; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.api.ICoordinator; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.paxos.Ballot; +import org.apache.cassandra.utils.ByteBufferUtil; + +import static java.lang.String.format; + +public abstract class MixedModePaxosTestBase extends UpgradeTestBase +{ + private static final Logger logger = LoggerFactory.getLogger(MixedModePaxosTestBase.class); + + abstract boolean upgradedCoordinator(); + + /** + * Tests the mixed mode loop bug in CASSANDRA-20493 + * <p> + * Paxos uses a 'zero' ballot in place of null when it doesn't find a ballot in system.paxos. CEP-14 changed the lsb + * of the zero ballot uuid from the TimeUUID.MIN_CLOCK_SEQ_AND_NODE value of 0x8080808080808080 (-9187201950435737472) + * to 0. It also removed the check added in CASSANDRA-12043, since the way it read and filtered ttld paxos data had + * been improved. + * <p> + * In mixed mode with a 4.0 or lower replica and a 4.1 and higher paxos coordinator, and in the absence of existing + * paxos metadata for the key being queried, the prepare phase will interpret the mismatched ‘zero’ ballots as the + * 4.0 and lower nodes having missed the most recent commit and will attempt to update them using the 4.1 zero ballot + * and empty partition update. + * <p> + * In cases where this is the first paxos operation on a key, or the previously ttl'd paxos data on disk had been purged, + * this would just add a retry step as it updated the 4.0 and lower hosts with its zero ballot. + * <p> + * On nodes where there was ttl'd paxos data though, the ttl'd data on disk would shadow this update. This would + * happen because paxos commits are recorded to system.paxos using the ballot timestamp as the write timestamp, so + * the more recent write updating the commit with timestamp 0 would be shadowed by the now ttl’d write with a ‘real’ + * timestamp. When the prepare phase restarted it would again get the old zero value and cause the prepare phase to + * get into an infinite loop. + */ + private void ttldPaxosStateTest(boolean legacyAware, boolean upgradeAware) throws Throwable + { + String keyspace = KEYSPACE; + String table = "tbl"; + int gcGrace = 10; + new TestCase() + .withConfig(c -> c.with(Feature.GOSSIP, Feature.NETWORK)) + .nodes(2) + .nodesToUpgrade(1) + .singleUpgrade(v40) + .setup(cluster -> { + cluster.schemaChange(format("CREATE TABLE %s.%s (k int primary key, v int) " + + "WITH gc_grace_seconds=%s", keyspace, table, gcGrace)); + }) + .runAfterClusterUpgrade(cluster -> { + // disable compaction to prevent paxos state from being purged + cluster.forEach(instance -> instance.nodetool("disableautocompaction")); + + // insert a ttl'd committed paxos state + long ballotMicros = TimeUnit.NANOSECONDS.toMicros(System.currentTimeMillis()); + FakePaxosHelper helper = FakePaxosHelper.create(cluster.coordinator(1), keyspace, table, gcGrace, ballotMicros); + + // confirm none of the nodes have paxos state + for (int i = 1; i <= cluster.size(); i++) + Assert.assertEquals(0, cluster.coordinator(i).execute("SELECT * FROM system.paxos", ConsistencyLevel.ONE).length); + + + // save commit to both nodes + if (upgradeAware) + helper.saveCommit(cluster.coordinator(1)); + + if (legacyAware) + helper.saveCommit(cluster.coordinator(2)); + + // wait for the paxos state to expire + Thread.sleep(TimeUnit.SECONDS.toMillis(gcGrace * 2)); + + // confirm paxos state has ttld + Assert.assertEquals(0, cluster.coordinator(1).execute("SELECT * FROM system.paxos", ConsistencyLevel.ONE).length); + Assert.assertEquals(0, cluster.coordinator(2).execute("SELECT * FROM system.paxos", ConsistencyLevel.ONE).length); + + // paxos operation should not timeout + cluster.coordinator(upgradedCoordinator() ? 1 : 2).execute(format("SELECT * FROM %s.%s WHERE k=1", keyspace, table), ConsistencyLevel.SERIAL); + }) + .run(); + } + + @Test + public void upgradeAwareTTldPaxosStateTest() throws Throwable + { + ttldPaxosStateTest(false, true); + } + + @Test + public void legacyAwareTTldPaxosStateTest() throws Throwable + { + ttldPaxosStateTest(true, false); + } + + @Test + public void bothAwareTTldPaxosStateTest() throws Throwable + { + ttldPaxosStateTest(true, false); + } + + /** + * This is an upgrade test, and paxos internally limits ttls to 3 hours, so we have to manually save commits in + * the paxos table to get entries ttl'd in a reasonable amount of time + */ + private static class FakePaxosHelper + { + static final int current_version = MessagingService.current_version; + static final int version_40a = MessagingService.VERSION_40; + + final UUID cfId; + final ByteBuffer key; + final long ballotMicros; + final int ballotSeconds; + final int ttl; + final UUID ballot; + final PartitionUpdate update; + + public FakePaxosHelper(String keyspace, String table, UUID cfId, int key, int ttl, long ballotMicros) + { + this.cfId = cfId; + this.ttl = ttl; + TableId tableId = TableId.fromUUID(cfId); + TableMetadata metadata = TableMetadata.builder(keyspace, table) + .id(tableId) + .partitioner(Murmur3Partitioner.instance) + .addPartitionKeyColumn("k", Int32Type.instance) + .addRegularColumn("v", Int32Type.instance) + .build(); + + this.key = ByteBufferUtil.bytes(key); + Row row = new SimpleBuilders.RowBuilder(metadata).add("v", (int) key).build(); + this.update = PartitionUpdate.singleRowUpdate(metadata, this.key, row); + + + + this.ballotMicros = ballotMicros; + this.ballotSeconds = (int) TimeUnit.MICROSECONDS.toSeconds(ballotMicros); + this.ballot = Ballot.atUnixMicrosWithLsb(ballotMicros, 0, Ballot.Flag.GLOBAL).asUUID(); + } + + ByteBuffer updateBytes(int version) + { + return PartitionUpdate.toBytes(update, version); + } + + void saveCommit(ICoordinator coordinator) + { + String cql = "UPDATE system.paxos USING TIMESTAMP ? AND TTL ? SET proposal_ballot = null, proposal = null, most_recent_commit_at = ?, most_recent_commit = ?, most_recent_commit_version = ? WHERE row_key = ? AND cf_id = ?"; + coordinator.execute(cql, ConsistencyLevel.ONE, + ballotMicros, + ttl, + ballot, + updateBytes(version_40a), + version_40a, + key, + cfId); + } + + public static FakePaxosHelper create(ICoordinator coordinator, String keyspace, String table, int ttl, long ballotMicros) + { + UUID cfId = (UUID) coordinator.execute("SELECT id FROM system_schema.tables WHERE keyspace_name=? AND table_name=?", ConsistencyLevel.ONE, keyspace, table)[0][0]; + return new FakePaxosHelper(keyspace, table, cfId, 1, ttl, ballotMicros); + } + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModePaxosUpgradedCoordinatorTest.java b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModePaxosUpgradedCoordinatorTest.java new file mode 100644 index 000000000000..f04380fbb63b --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/upgrade/MixedModePaxosUpgradedCoordinatorTest.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.upgrade; + +public class MixedModePaxosUpgradedCoordinatorTest extends MixedModePaxosTestBase +{ + @Override + boolean upgradedCoordinator() + { + return true; + } +} From b56edf2a5df8c320b33e38116f2742ab69f7b4fd Mon Sep 17 00:00:00 2001 From: David Capwell <dcapwell@apache.org> Date: Fri, 28 Mar 2025 11:48:51 -0700 Subject: [PATCH 220/225] Add support in CAS for -= on numeric types, and fixed improper handling of empty bytes which lead to NPE patch by David Capwell; reviewed by Ariel Weisberg for CASSANDRA-20477 --- CHANGES.txt | 1 + .../org/apache/cassandra/cql3/Operation.java | 12 +- .../cql3/statements/BatchStatement.java | 12 +- .../cql3/statements/CQL3CasRequest.java | 12 +- .../statements/ModificationStatement.java | 12 +- .../cassandra/cql3/terms/Constants.java | 64 +++- .../cassandra/db/RegularAndStaticColumns.java | 7 + .../cassandra/db/marshal/AbstractType.java | 26 +- .../apache/cassandra/service/CASRequest.java | 3 + .../apache/cassandra/service/paxos/Paxos.java | 9 +- .../cassandra/transport/Dispatcher.java | 2 + .../test/cql3/CasMultiNodeTableWalkBase.java | 129 +++++++ .../test/cql3/MultiNodeTableWalkBase.java | 21 +- .../MultiNodeTableWalkWithReadRepairTest.java | 4 +- ...ltiNodeTableWalkWithoutReadRepairTest.java | 4 +- .../cql3/PaxosV1MultiNodeTableWalkTest.java | 42 ++ .../cql3/PaxosV2MultiNodeTableWalkTest.java | 42 ++ .../test/cql3/SingleNodeTableWalkTest.java | 26 +- .../test/cql3/StatefulASTBase.java | 79 +++- .../harry/model/ASTSingleTableModel.java | 358 ++++++++++++++++-- .../harry/model/BytesPartitionState.java | 17 +- .../org/apache/cassandra/cql3/KnownIssue.java | 4 +- .../cql3/ast/AssignmentOperator.java | 7 +- .../org/apache/cassandra/cql3/ast/Bind.java | 1 + .../cassandra/cql3/ast/CasCondition.java | 20 +- .../cassandra/cql3/ast/Conditional.java | 15 + .../cql3/ast/ExpressionEvaluator.java | 5 + .../apache/cassandra/cql3/ast/Literal.java | 6 + .../apache/cassandra/cql3/ast/Mutation.java | 53 ++- .../apache/cassandra/cql3/ast/Reference.java | 5 + .../cql3/ast/ReferenceExpression.java | 2 +- .../org/apache/cassandra/cql3/ast/Value.java | 4 + .../apache/cassandra/cql3/ast/Visitor.java | 5 + .../apache/cassandra/utils/ASTGenerators.java | 17 +- 34 files changed, 934 insertions(+), 92 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/test/cql3/CasMultiNodeTableWalkBase.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/cql3/PaxosV1MultiNodeTableWalkTest.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/cql3/PaxosV2MultiNodeTableWalkTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 80eabfd705b4..361fb3cd0755 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Add support in CAS for -= on numeric types, and fixed improper handling of empty bytes which lead to NPE (CASSANDRA-20477) * Do not fail to start a node with materialized views after they are turned off in config (CASSANDRA-20452) * Fix nodetool gcstats output, support human-readable units and more output formats (CASSANDRA-19022) * Various gossip to TCM upgrade fixes (CASSANDRA-20483) diff --git a/src/java/org/apache/cassandra/cql3/Operation.java b/src/java/org/apache/cassandra/cql3/Operation.java index 7a7c0e842070..7c5e02eb63e3 100644 --- a/src/java/org/apache/cassandra/cql3/Operation.java +++ b/src/java/org/apache/cassandra/cql3/Operation.java @@ -374,8 +374,16 @@ public Operation prepare(TableMetadata metadata, ColumnMetadata receiver, boolea { if (!(receiver.type instanceof CollectionType)) { - if (!(receiver.type instanceof CounterColumnType)) - throw new InvalidRequestException(String.format("Invalid operation (%s) for non counter column %s", toString(receiver), receiver.name)); + if (canReadExistingState) + { + if (!(receiver.type instanceof NumberType<?>)) + throw new InvalidRequestException(String.format("Invalid operation (%s) for non-numeric type %s", toString(receiver), receiver.name)); + } + else + { + if (!(receiver.type instanceof CounterColumnType)) + throw new InvalidRequestException(String.format("Invalid operation (%s) for non counter column %s", toString(receiver), receiver.name)); + } return new Constants.Substracter(receiver, value.prepare(metadata.keyspace, receiver)); } else if (!(receiver.type.isMultiCell())) diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java index bfec675464db..6c5b12199252 100644 --- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java @@ -468,7 +468,7 @@ private void updatePartitionsPerBatchMetrics(int updatedPartitions) private ResultMessage executeWithConditions(BatchQueryOptions options, QueryState state, Dispatcher.RequestTime requestTime) { - Pair<CQL3CasRequest, Set<ColumnMetadata>> p = makeCasRequest(options, state); + Pair<CQL3CasRequest, Set<ColumnMetadata>> p = makeCasRequest(options, state, requestTime); CQL3CasRequest casRequest = p.left; Set<ColumnMetadata> columnsWithConditions = p.right; @@ -495,7 +495,7 @@ private ResultMessage executeWithConditions(BatchQueryOptions options, QueryStat } } - private Pair<CQL3CasRequest,Set<ColumnMetadata>> makeCasRequest(BatchQueryOptions options, QueryState state) + private Pair<CQL3CasRequest,Set<ColumnMetadata>> makeCasRequest(BatchQueryOptions options, QueryState state, Dispatcher.RequestTime requestTime) { long batchTimestamp = options.getTimestamp(state); long nowInSeconds = options.getNowInSeconds(state); @@ -514,7 +514,7 @@ private Pair<CQL3CasRequest,Set<ColumnMetadata>> makeCasRequest(BatchQueryOption if (key == null) { key = statement.metadata().partitioner.decorateKey(pks.get(0)); - casRequest = new CQL3CasRequest(statement.metadata(), key, conditionColumns, updatesRegularRows, updatesStaticRow); + casRequest = new CQL3CasRequest(statement.metadata(), key, conditionColumns, updatesRegularRows, updatesStaticRow, requestTime); } else if (!key.getKey().equals(pks.get(0))) { @@ -570,7 +570,7 @@ public ResultMessage executeLocally(QueryState queryState, QueryOptions options) BatchQueryOptions batchOptions = BatchQueryOptions.withoutPerStatementVariables(options); if (hasConditions) - return executeInternalWithConditions(batchOptions, queryState); + return executeInternalWithConditions(batchOptions, queryState, Dispatcher.RequestTime.forImmediateExecution()); executeInternalWithoutCondition(queryState, batchOptions, Dispatcher.RequestTime.forImmediateExecution()); return new ResultMessage.Void(); @@ -586,9 +586,9 @@ private ResultMessage executeInternalWithoutCondition(QueryState queryState, Bat return null; } - private ResultMessage executeInternalWithConditions(BatchQueryOptions options, QueryState state) + private ResultMessage executeInternalWithConditions(BatchQueryOptions options, QueryState state, Dispatcher.RequestTime requestTime) { - Pair<CQL3CasRequest, Set<ColumnMetadata>> p = makeCasRequest(options, state); + Pair<CQL3CasRequest, Set<ColumnMetadata>> p = makeCasRequest(options, state, requestTime); CQL3CasRequest request = p.left; Set<ColumnMetadata> columnsWithConditions = p.right; diff --git a/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java b/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java index 0d322691c6e6..4db98459ec1f 100644 --- a/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java +++ b/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java @@ -34,6 +34,7 @@ import org.apache.cassandra.service.CASRequest; import org.apache.cassandra.service.ClientState; import org.apache.cassandra.service.paxos.Ballot; +import org.apache.cassandra.transport.Dispatcher; import org.apache.cassandra.utils.TimeUUID; import org.apache.commons.lang3.builder.ToStringBuilder; @@ -49,6 +50,7 @@ public class CQL3CasRequest implements CASRequest private final RegularAndStaticColumns conditionColumns; private final boolean updatesRegularRows; private final boolean updatesStaticRow; + private final Dispatcher.RequestTime requestTime; private boolean hasExists; // whether we have an exist or if not exist condition // Conditions on the static row. We keep it separate from 'conditions' as most things related to the static row are @@ -66,7 +68,8 @@ public CQL3CasRequest(TableMetadata metadata, DecoratedKey key, RegularAndStaticColumns conditionColumns, boolean updatesRegularRows, - boolean updatesStaticRow) + boolean updatesStaticRow, + Dispatcher.RequestTime requestTime) { this.metadata = metadata; this.key = key; @@ -74,6 +77,13 @@ public CQL3CasRequest(TableMetadata metadata, this.conditionColumns = conditionColumns; this.updatesRegularRows = updatesRegularRows; this.updatesStaticRow = updatesStaticRow; + this.requestTime = requestTime; + } + + @Override + public Dispatcher.RequestTime requestTime() + { + return requestTime; } void addRowUpdate(Clustering<?> clustering, ModificationStatement stmt, QueryOptions options, long timestamp, long nowInSeconds) diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java index ce9da9a538ee..21da99c14af4 100644 --- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java @@ -576,7 +576,7 @@ private ResultMessage executeWithoutCondition(QueryState queryState, QueryOption private ResultMessage executeWithCondition(QueryState queryState, QueryOptions options, Dispatcher.RequestTime requestTime) { - CQL3CasRequest request = makeCasRequest(queryState, options); + CQL3CasRequest request = makeCasRequest(queryState, options, requestTime); try (RowIterator result = StorageProxy.cas(keyspace(), table(), @@ -592,7 +592,7 @@ private ResultMessage executeWithCondition(QueryState queryState, QueryOptions o } } - private CQL3CasRequest makeCasRequest(QueryState queryState, QueryOptions options) + private CQL3CasRequest makeCasRequest(QueryState queryState, QueryOptions options, Dispatcher.RequestTime requestTime) { ClientState clientState = queryState.getClientState(); List<ByteBuffer> keys = buildPartitionKeyNames(options, clientState); @@ -610,7 +610,7 @@ private CQL3CasRequest makeCasRequest(QueryState queryState, QueryOptions option type.isUpdate()? "updates" : "deletions"); Clustering<?> clustering = Iterables.getOnlyElement(createClustering(options, clientState)); - CQL3CasRequest request = new CQL3CasRequest(metadata(), key, conditionColumns(), updatesRegularRows(), updatesStaticRow()); + CQL3CasRequest request = new CQL3CasRequest(metadata(), key, conditionColumns(), updatesRegularRows(), updatesStaticRow(), requestTime); addConditions(clustering, request, options); request.addRowUpdate(clustering, this, options, timestamp, nowInSeconds); @@ -718,7 +718,7 @@ private static ResultSet buildCasFailureResultSet(RowIterator partition, public ResultMessage executeLocally(QueryState queryState, QueryOptions options) throws RequestValidationException, RequestExecutionException { return hasConditions() - ? executeInternalWithCondition(queryState, options) + ? executeInternalWithCondition(queryState, options, Dispatcher.RequestTime.forImmediateExecution()) : executeInternalWithoutCondition(queryState, options, Dispatcher.RequestTime.forImmediateExecution()); } @@ -732,9 +732,9 @@ public ResultMessage executeInternalWithoutCondition(QueryState queryState, Quer return null; } - public ResultMessage executeInternalWithCondition(QueryState state, QueryOptions options) + public ResultMessage executeInternalWithCondition(QueryState state, QueryOptions options, Dispatcher.RequestTime requestTime) { - CQL3CasRequest request = makeCasRequest(state, options); + CQL3CasRequest request = makeCasRequest(state, options, requestTime); try (RowIterator result = casInternal(state.getClientState(), request, options.getTimestamp(state), options.getNowInSeconds(state))) { diff --git a/src/java/org/apache/cassandra/cql3/terms/Constants.java b/src/java/org/apache/cassandra/cql3/terms/Constants.java index f63d7ce28b84..3b9ae6b4c90e 100644 --- a/src/java/org/apache/cassandra/cql3/terms/Constants.java +++ b/src/java/org/apache/cassandra/cql3/terms/Constants.java @@ -44,6 +44,14 @@ */ public abstract class Constants { + + private static ByteBuffer getCurrentCellBuffer(ColumnMetadata column, DecoratedKey key, UpdateParameters params) + { + Row currentRow = params.getPrefetchedRow(key, column.isStatic() ? Clustering.STATIC_CLUSTERING : params.currentClustering()); + Cell<?> currentCell = currentRow == null ? null : currentRow.getCell(column); + return currentCell == null ? null : currentCell.buffer(); + } + public enum Type { STRING @@ -489,8 +497,10 @@ public void execute(DecoratedKey partitionKey, UpdateParameters params) throws I else if (column.type instanceof NumberType<?>) { @SuppressWarnings("unchecked") NumberType<Number> type = (NumberType<Number>) column.type; - ByteBuffer increment = t.bindAndGet(params.options); - ByteBuffer current = getCurrentCellBuffer(partitionKey, params); + ByteBuffer increment = type.sanitize(t.bindAndGet(params.options)); + if (increment == null) + return; + ByteBuffer current = type.sanitize(getCurrentCellBuffer(column, partitionKey, params)); if (current == null) return; ByteBuffer newValue = type.add(type.compose(current), type.compose(increment)); @@ -499,7 +509,9 @@ else if (column.type instanceof NumberType<?>) else if (column.type instanceof StringType) { ByteBuffer append = t.bindAndGet(params.options); - ByteBuffer current = getCurrentCellBuffer(partitionKey, params); + if (append == null) + return; + ByteBuffer current = getCurrentCellBuffer(column, partitionKey, params); if (current == null) return; ByteBuffer newValue = ByteBuffer.allocate(current.remaining() + append.remaining()); @@ -508,13 +520,6 @@ else if (column.type instanceof StringType) params.addCell(column, newValue); } } - - private ByteBuffer getCurrentCellBuffer(DecoratedKey key, UpdateParameters params) - { - Row currentRow = params.getPrefetchedRow(key, column.isStatic() ? Clustering.STATIC_CLUSTERING : params.currentClustering()); - Cell<?> currentCell = currentRow == null ? null : currentRow.getCell(column); - return currentCell == null ? null : currentCell.buffer(); - } } public static class Substracter extends Operation @@ -524,19 +529,40 @@ public Substracter(ColumnMetadata column, Term t) super(column, t); } + @Override + public boolean requiresRead() + { + return !column.type.isCounter(); + } + public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException { - ByteBuffer bytes = t.bindAndGet(params.options); - if (bytes == null) - throw new InvalidRequestException("Invalid null value for counter increment"); - if (bytes == ByteBufferUtil.UNSET_BYTE_BUFFER) - return; + if (column.type instanceof CounterColumnType) + { + ByteBuffer bytes = t.bindAndGet(params.options); + if (bytes == null) + throw new InvalidRequestException("Invalid null value for counter increment"); + if (bytes == ByteBufferUtil.UNSET_BYTE_BUFFER) + return; - long increment = ByteBufferUtil.toLong(bytes); - if (increment == Long.MIN_VALUE) - throw new InvalidRequestException("The negation of " + increment + " overflows supported counter precision (signed 8 bytes integer)"); + long increment = ByteBufferUtil.toLong(bytes); + if (increment == Long.MIN_VALUE) + throw new InvalidRequestException("The negation of " + increment + " overflows supported counter precision (signed 8 bytes integer)"); - params.addCounter(column, -increment); + params.addCounter(column, -increment); + } + else if (column.type instanceof NumberType<?>) + { + @SuppressWarnings("unchecked") NumberType<Number> type = (NumberType<Number>) column.type; + ByteBuffer increment = type.sanitize(t.bindAndGet(params.options)); + if (increment == null) + return; + ByteBuffer current = type.sanitize(getCurrentCellBuffer(column, partitionKey, params)); + if (current == null) + return; + ByteBuffer newValue = type.substract(type.compose(current), type.compose(increment)); + params.addCell(column, newValue); + } } } diff --git a/src/java/org/apache/cassandra/db/RegularAndStaticColumns.java b/src/java/org/apache/cassandra/db/RegularAndStaticColumns.java index 4daea2f4f7b4..2032fe6fe905 100644 --- a/src/java/org/apache/cassandra/db/RegularAndStaticColumns.java +++ b/src/java/org/apache/cassandra/db/RegularAndStaticColumns.java @@ -18,6 +18,8 @@ package org.apache.cassandra.db; import java.util.*; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; import com.google.common.collect.Iterators; @@ -58,6 +60,11 @@ public RegularAndStaticColumns without(ColumnMetadata column) column.isStatic() ? regulars : regulars.without(column)); } + public Stream<ColumnMetadata> stream() + { + return StreamSupport.stream(spliterator(), false); + } + public RegularAndStaticColumns mergeTo(RegularAndStaticColumns that) { if (this == that) diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractType.java b/src/java/org/apache/cassandra/db/marshal/AbstractType.java index c624ce505be0..5378a4cd3fba 100644 --- a/src/java/org/apache/cassandra/db/marshal/AbstractType.java +++ b/src/java/org/apache/cassandra/db/marshal/AbstractType.java @@ -28,6 +28,8 @@ import java.util.Map; import java.util.Objects; +import javax.annotation.Nullable; + import org.apache.cassandra.cql3.AssignmentTestable; import org.apache.cassandra.cql3.CQL3Type; import org.apache.cassandra.cql3.ColumnSpecification; @@ -474,14 +476,6 @@ public AbstractType<?> freezeNestedMulticellTypes() return this; } - /** - * Returns {@code true} for types where empty should be handled like {@code null} like {@link Int32Type}. - */ - public boolean isEmptyValueMeaningless() - { - return false; - } - /** * @param ignoreFreezing if true, the type string will not be wrapped with FrozenType(...), even if this type is frozen. */ @@ -537,6 +531,22 @@ public boolean allowsEmpty() return false; } + /** + * Returns {@code true} for types where empty should be handled like {@code null} like {@link Int32Type}. + */ + public boolean isEmptyValueMeaningless() + { + return false; + } + + @Nullable + public ByteBuffer sanitize(@Nullable ByteBuffer bb) + { + if (bb == null) return null; + // not checking allowsEmpty as this method assumes that the bb has already passed validation for the type + return bb.remaining() == 0 && isEmptyValueMeaningless() ? null : bb; + } + public boolean isNull(ByteBuffer bb) { return isNull(bb, ByteBufferAccessor.instance); diff --git a/src/java/org/apache/cassandra/service/CASRequest.java b/src/java/org/apache/cassandra/service/CASRequest.java index 6fb5eea20c60..50ea5852a63a 100644 --- a/src/java/org/apache/cassandra/service/CASRequest.java +++ b/src/java/org/apache/cassandra/service/CASRequest.java @@ -22,12 +22,15 @@ import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.service.paxos.Ballot; +import org.apache.cassandra.transport.Dispatcher; /** * Abstract the conditions and updates for a CAS operation. */ public interface CASRequest { + Dispatcher.RequestTime requestTime(); + /** * The command to use to fetch the value to compare for the CAS. */ diff --git a/src/java/org/apache/cassandra/service/paxos/Paxos.java b/src/java/org/apache/cassandra/service/paxos/Paxos.java index 3412add1a252..06f90907d502 100644 --- a/src/java/org/apache/cassandra/service/paxos/Paxos.java +++ b/src/java/org/apache/cassandra/service/paxos/Paxos.java @@ -737,7 +737,7 @@ private static RowIterator cas(DecoratedKey partitionKey, Tracing.trace("Reading existing values for CAS precondition"); BeginResult begin = begin(proposeDeadline, readCommand, consistencyForConsensus, - true, minimumBallot, failedAttemptsDueToContention); + true, minimumBallot, failedAttemptsDueToContention, request.requestTime()); Ballot ballot = begin.ballot; Participants participants = begin.participants; failedAttemptsDueToContention = begin.failedAttemptsDueToContention; @@ -914,7 +914,7 @@ private static PartitionIterator read(SinglePartitionReadCommand.Group group, Co while (true) { // does the work of applying in-progress writes; throws UAE or timeout if it can't - final BeginResult begin = begin(deadline, read, consistencyForConsensus, false, minimumBallot, failedAttemptsDueToContention); + final BeginResult begin = begin(deadline, read, consistencyForConsensus, false, minimumBallot, failedAttemptsDueToContention, requestTime); failedAttemptsDueToContention = begin.failedAttemptsDueToContention; switch (PAXOS_VARIANT) @@ -1034,7 +1034,8 @@ private static BeginResult begin(long deadline, ConsistencyLevel consistencyForConsensus, final boolean isWrite, Ballot minimumBallot, - int failedAttemptsDueToContention) + int failedAttemptsDueToContention, + Dispatcher.RequestTime requestTime) throws WriteTimeoutException, WriteFailureException, ReadTimeoutException, ReadFailureException { boolean acceptEarlyReadPermission = !isWrite; // if we're reading, begin by assuming a read permission is sufficient @@ -1111,7 +1112,7 @@ private static BeginResult begin(long deadline, PaxosPrepare.Success success = prepare.success(); Supplier<Participants> plan = () -> success.participants; - DataResolver<?, ?> resolver = new DataResolver<>(query, plan, NoopReadRepair.instance, new Dispatcher.RequestTime(query.creationTimeNanos())); + DataResolver<?, ?> resolver = new DataResolver<>(query, plan, NoopReadRepair.instance, requestTime); for (int i = 0 ; i < success.responses.size() ; ++i) resolver.preprocess(success.responses.get(i)); diff --git a/src/java/org/apache/cassandra/transport/Dispatcher.java b/src/java/org/apache/cassandra/transport/Dispatcher.java index d6cb5e822f9f..f701434d0068 100644 --- a/src/java/org/apache/cassandra/transport/Dispatcher.java +++ b/src/java/org/apache/cassandra/transport/Dispatcher.java @@ -24,6 +24,7 @@ import java.util.function.Consumer; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -140,6 +141,7 @@ public RequestTime(long createdAtNanos) public RequestTime(long enqueuedAtNanos, long startedAtNanos) { + Preconditions.checkArgument(enqueuedAtNanos != -1); this.enqueuedAtNanos = enqueuedAtNanos; this.startedAtNanos = startedAtNanos; } diff --git a/test/distributed/org/apache/cassandra/distributed/test/cql3/CasMultiNodeTableWalkBase.java b/test/distributed/org/apache/cassandra/distributed/test/cql3/CasMultiNodeTableWalkBase.java new file mode 100644 index 000000000000..bf8a44dcb946 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/cql3/CasMultiNodeTableWalkBase.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.cql3; + +import accord.utils.Gen; +import accord.utils.RandomSource; +import org.apache.cassandra.config.Config; +import org.apache.cassandra.cql3.KnownIssue; +import org.apache.cassandra.cql3.ast.CasCondition; +import org.apache.cassandra.cql3.ast.Conditional; +import org.apache.cassandra.cql3.ast.Mutation; +import org.apache.cassandra.cql3.ast.Value; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.IInstanceConfig; +import org.apache.cassandra.service.reads.repair.ReadRepairStrategy; +import org.apache.cassandra.utils.ASTGenerators; + +import static org.apache.cassandra.utils.Generators.toGen; + +public abstract class CasMultiNodeTableWalkBase extends MultiNodeTableWalkBase +{ + protected final Config.PaxosVariant paxos_variant; + + protected CasMultiNodeTableWalkBase(Config.PaxosVariant paxos_variant) + { + super(ReadRepairStrategy.NONE); + this.paxos_variant = paxos_variant; + } + + @Override + protected void clusterConfig(IInstanceConfig c) + { + super.clusterConfig(c); + c.set("paxos_variant", paxos_variant); + c.set("cas_contention_timeout", "180s"); + //TODO (now): should these be included? They are in the benchmark clusters +// c.set("paxos_contention_min_wait", 0); +// c.set("paxos_contention_max_wait", "100ms"); +// c.set("paxos_contention_min_delta", 0); + } + + @Override + protected SingleNodeTableWalkTest.State createState(RandomSource rs, Cluster cluster) + { + return new State(rs, cluster); + } + + private static boolean isValueUDTSafe(Value value) + { + var bb = value.valueEncoded(); + return bb == null ? true : bb.hasRemaining(); + } + + protected class State extends MultiNodeState + { + private State(RandomSource rs, Cluster cluster) + { + super(rs, cluster); + } + + @Override + protected Gen<Mutation> toMutationGen(ASTGenerators.MutationGenBuilder mutationGenBuilder) + { + mutationGenBuilder.withCasGen(i -> true); + // generator might not always generate a cas statement... should fix generator! + Gen<Mutation> gen = toGen(mutationGenBuilder.build()).filter(Mutation::isCas); + if (metadata.regularAndStaticColumns().stream().anyMatch(c -> c.type.isUDT()) + && IGNORED_ISSUES.contains(KnownIssue.CAS_CONDITION_ON_UDT_W_EMPTY_BYTES)) + { + gen = gen.filter(m -> { + CasCondition condition; + switch (m.kind) + { + case INSERT: + return true; + case DELETE: + condition = ((Mutation.Delete) m).casCondition.get(); + break; + case UPDATE: + condition = ((Mutation.Update) m).casCondition.get(); + break; + default: + throw new UnsupportedOperationException(m.kind.name()); + } + return !condition.streamRecursive(true).anyMatch(e -> { + if (!(e instanceof Conditional.Where)) return false; + var where = (Conditional.Where) e; + if (!where.lhs.type().isUDT()) return false; + if (where.lhs instanceof Value && !isValueUDTSafe((Value) where.lhs)) + return true; + if (where.rhs instanceof Value && !isValueUDTSafe((Value) where.rhs)) + return true; + return false; + }); + }); + } + return gen; + } + + @Override + protected ConsistencyLevel selectCl() + { + return ConsistencyLevel.SERIAL; + } + + @Override + protected ConsistencyLevel mutationCl() + { + return ConsistencyLevel.SERIAL; + } + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkBase.java b/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkBase.java index f10d0edcf4d8..3e9c1195f6a1 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkBase.java +++ b/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkBase.java @@ -23,6 +23,7 @@ import accord.utils.RandomSource; import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.IInstanceConfig; import org.apache.cassandra.distributed.api.IInvokableInstance; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.reads.repair.ReadRepairStrategy; @@ -52,13 +53,17 @@ protected TableMetadata defineTable(RandomSource rs, String ks) @Override protected Cluster createCluster() throws IOException { - return createCluster(mockMultiNode ? 1 : 3, c -> { - c.set("range_request_timeout", "180s") - .set("read_request_timeout", "180s") - .set("write_request_timeout", "180s") - .set("native_transport_timeout", "180s") - .set("slow_query_log_timeout", "180s"); - }); + return createCluster(mockMultiNode ? 1 : 3, this::clusterConfig); + } + + @Override + protected void clusterConfig(IInstanceConfig c) + { + c.set("range_request_timeout", "180s") + .set("read_request_timeout", "180s") + .set("write_request_timeout", "180s") + .set("native_transport_timeout", "180s") + .set("slow_query_log_timeout", "180s"); } @Override @@ -67,7 +72,7 @@ protected State createState(RandomSource rs, Cluster cluster) return new MultiNodeState(rs, cluster); } - private class MultiNodeState extends State + protected class MultiNodeState extends State { public MultiNodeState(RandomSource rs, Cluster cluster) { diff --git a/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkWithReadRepairTest.java b/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkWithReadRepairTest.java index e8b01f8c712a..7727e3a76ab3 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkWithReadRepairTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkWithReadRepairTest.java @@ -38,7 +38,9 @@ protected void preCheck(Cluster cluster, Property.StatefulBuilder builder) // if a failing seed is detected, populate here // Example: builder.withSeed(42L); // CQL operations may have opertors such as +, -, and / (example 4 + 4), to "apply" them to get a constant value -// CQL_DEBUG_APPLY_OPERATOR = true; + // CQL_DEBUG_APPLY_OPERATOR = true; + // When mutations look to be lost as seen by more complex SELECTs, it can be useful to just SELECT the partition/row right after to write to see if it was safe at the time. + // READ_AFTER_WRITE = true; // When an issue is found, it's a good idea to also run the same seed against MultiNodeTableWalkWithoutReadRepairTest; if Read Repair is given bad input, you should expect bad output! // This test needs to make sure it shares the same random history as MultiNodeTableWalkWithoutReadRepairTest to always allow the ability to maintain this property. } diff --git a/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkWithoutReadRepairTest.java b/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkWithoutReadRepairTest.java index a18b80d68ad2..5a0ce66ccca9 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkWithoutReadRepairTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/cql3/MultiNodeTableWalkWithoutReadRepairTest.java @@ -35,6 +35,8 @@ protected void preCheck(Cluster cluster, Property.StatefulBuilder builder) // if a failing seed is detected, populate here // Example: builder.withSeed(42L); // CQL operations may have opertors such as +, -, and / (example 4 + 4), to "apply" them to get a constant value -// CQL_DEBUG_APPLY_OPERATOR = true; + // CQL_DEBUG_APPLY_OPERATOR = true; + // When mutations look to be lost as seen by more complex SELECTs, it can be useful to just SELECT the partition/row right after to write to see if it was safe at the time. + // READ_AFTER_WRITE = true; } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/cql3/PaxosV1MultiNodeTableWalkTest.java b/test/distributed/org/apache/cassandra/distributed/test/cql3/PaxosV1MultiNodeTableWalkTest.java new file mode 100644 index 000000000000..0cf333d2ab84 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/cql3/PaxosV1MultiNodeTableWalkTest.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.cql3; + +import accord.utils.Property; +import org.apache.cassandra.config.Config; +import org.apache.cassandra.distributed.Cluster; + +public class PaxosV1MultiNodeTableWalkTest extends CasMultiNodeTableWalkBase +{ + public PaxosV1MultiNodeTableWalkTest() + { + super(Config.PaxosVariant.v1); + } + + @Override + protected void preCheck(Cluster cluster, Property.StatefulBuilder builder) + { + // if a failing seed is detected, populate here + // Example: builder.withSeed(42L); + // CQL operations may have opertors such as +, -, and / (example 4 + 4), to "apply" them to get a constant value + // CQL_DEBUG_APPLY_OPERATOR = true; + // When mutations look to be lost as seen by more complex SELECTs, it can be useful to just SELECT the partition/row right after to write to see if it was safe at the time. + // READ_AFTER_WRITE = true; + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/cql3/PaxosV2MultiNodeTableWalkTest.java b/test/distributed/org/apache/cassandra/distributed/test/cql3/PaxosV2MultiNodeTableWalkTest.java new file mode 100644 index 000000000000..fa098edaacbc --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/cql3/PaxosV2MultiNodeTableWalkTest.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test.cql3; + +import accord.utils.Property; +import org.apache.cassandra.config.Config; +import org.apache.cassandra.distributed.Cluster; + +public class PaxosV2MultiNodeTableWalkTest extends CasMultiNodeTableWalkBase +{ + public PaxosV2MultiNodeTableWalkTest() + { + super(Config.PaxosVariant.v2); + } + + @Override + protected void preCheck(Cluster cluster, Property.StatefulBuilder builder) + { + // if a failing seed is detected, populate here + // Example: builder.withSeed(42L); + // CQL operations may have opertors such as +, -, and / (example 4 + 4), to "apply" them to get a constant value + // CQL_DEBUG_APPLY_OPERATOR = true; + // When mutations look to be lost as seen by more complex SELECTs, it can be useful to just SELECT the partition/row right after to write to see if it was safe at the time. + // READ_AFTER_WRITE = true; + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTableWalkTest.java b/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTableWalkTest.java index 755d479e92e7..1feb9c5f8693 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTableWalkTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTableWalkTest.java @@ -59,6 +59,7 @@ import org.apache.cassandra.db.marshal.UTF8Type; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.IInstanceConfig; import org.apache.cassandra.distributed.test.sai.SAIUtil; import org.apache.cassandra.harry.model.BytesPartitionState; import org.apache.cassandra.schema.ColumnMetadata; @@ -92,12 +93,16 @@ public class SingleNodeTableWalkTest extends StatefulASTBase .collect(Collectors.toList())); private static final Logger logger = LoggerFactory.getLogger(SingleNodeTableWalkTest.class); + protected static boolean READ_AFTER_WRITE = false; + protected void preCheck(Cluster cluster, Property.StatefulBuilder builder) { // if a failing seed is detected, populate here // Example: builder.withSeed(42L); // CQL operations may have opertors such as +, -, and / (example 4 + 4), to "apply" them to get a constant value // CQL_DEBUG_APPLY_OPERATOR = true; + // When mutations look to be lost as seen by more complex SELECTs, it can be useful to just SELECT the partition/row right after to write to see if it was safe at the time. + // READ_AFTER_WRITE = true; } protected TypeGenBuilder supportedTypes(RandomSource rs) @@ -345,7 +350,12 @@ protected State createState(RandomSource rs, Cluster cluster) protected Cluster createCluster() throws IOException { - return createCluster(1, i -> {}); + return createCluster(1, this::clusterConfig); + } + + protected void clusterConfig(IInstanceConfig config) + { + } @Test @@ -460,7 +470,8 @@ public State(RandomSource rs, Cluster cluster) { model.factory.regularAndStaticColumns.forEach(mutationGenBuilder::allowEmpty); } - this.mutationGen = toGen(mutationGenBuilder.build()); + model.factory.regularAndStaticColumns.forEach(mutationGenBuilder::allowNull); + this.mutationGen = toMutationGen(mutationGenBuilder); var nonPartitionColumns = ImmutableList.<Symbol>builder() .addAll(model.factory.clusteringColumns) @@ -480,6 +491,17 @@ public State(RandomSource rs, Cluster cluster) .collect(Collectors.toList()); } + @Override + protected boolean readAfterWrite() + { + return READ_AFTER_WRITE; + } + + protected Gen<Mutation> toMutationGen(ASTGenerators.MutationGenBuilder mutationGenBuilder) + { + return toGen(mutationGenBuilder.build()); + } + private boolean isSearchable(Symbol symbol) { // See org.apache.cassandra.cql3.Operator.validateFor diff --git a/test/distributed/org/apache/cassandra/distributed/test/cql3/StatefulASTBase.java b/test/distributed/org/apache/cassandra/distributed/test/cql3/StatefulASTBase.java index ec530b97fdc5..a90b4679797c 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/cql3/StatefulASTBase.java +++ b/test/distributed/org/apache/cassandra/distributed/test/cql3/StatefulASTBase.java @@ -81,6 +81,7 @@ import org.apache.cassandra.utils.Generators; import org.quicktheories.generators.SourceDSL; +import static accord.utils.Property.multistep; import static org.apache.cassandra.distributed.test.JavaDriverUtils.toDriverCL; import static org.apache.cassandra.utils.AbstractTypeGenerators.overridePrimitiveTypeSupport; import static org.apache.cassandra.utils.AbstractTypeGenerators.stringComparator; @@ -179,7 +180,35 @@ protected <S extends BaseState> Property.StatefulSuccess<S, Void> onSuccess(Logg protected static <S extends CommonState> Property.Command<S, Void, ?> insert(RandomSource rs, S state) { int timestamp = ++state.operations; - return state.command(rs, state.mutationGen().next(rs).withTimestamp(timestamp)); + Mutation mutation = state.mutationGen().next(rs).withTimestamp(timestamp); + + if (!state.readAfterWrite()) + return state.command(rs, mutation); + + return multistep(state.command(rs, mutation), + state.commandSafeRandomHistory(selectForMutation(state, mutation), "Select for Mutation Validation")); + } + + private static <S extends CommonState> Select selectForMutation(S state, Mutation mutation) + { + var select = Select.builder(state.metadata).allowFiltering(); + switch (mutation.kind) + { + case INSERT: + { + var insert = (Mutation.Insert) mutation; + for (var c : state.model.factory.partitionColumns) + select.value(c, insert.values.get(c)); + } + break; + default: + { + select.where(mutation.kind == Mutation.Kind.UPDATE + ? ((Mutation.Update) mutation).where + : ((Mutation.Delete) mutation).where); + } + } + return select.build(); } protected static <S extends BaseState> Property.Command<S, Void, ?> fullTableScan(RandomSource rs, S state) @@ -244,6 +273,11 @@ protected BaseState(RandomSource rs, Cluster cluster, TableMetadata metadata) createTable(metadata); } + protected boolean readAfterWrite() + { + return false; + } + protected boolean isMultiNode() { return cluster.size() > 1; @@ -314,6 +348,17 @@ protected boolean allowPaging(Select select) }); } + protected <S extends BaseState> Property.Command<S, Void, ?> commandSafeRandomHistory(Select select, @Nullable String annotate) + { + var inst = cluster.firstAlive(); + String postfix = "on " + inst; + if (annotate == null) annotate = postfix; + else annotate += ", " + postfix; + return new Property.SimpleCommand<>(humanReadable(select, annotate), s -> { + s.model.validate(s.executeQuery(inst, Integer.MAX_VALUE, s.selectCl(), select), select); + }); + } + protected ConsistencyLevel selectCl() { return ConsistencyLevel.LOCAL_QUORUM; @@ -333,11 +378,18 @@ protected ConsistencyLevel mutationCl() { var inst = selectInstance(rs); String postfix = "on " + inst; + if (mutation.isCas()) + { + postfix += ", would apply " + model.shouldApply(mutation); + // CAS doesn't allow timestamps + mutation = mutation.withoutTimestamp(); + } if (annotate == null) annotate = postfix; else annotate += ", " + postfix; + Mutation finalMutation = mutation; return new Property.SimpleCommand<>(humanReadable(mutation, annotate), s -> { - s.executeQuery(inst, Integer.MAX_VALUE, s.mutationCl(), mutation); - s.model.update(mutation); + s.executeQuery(inst, Integer.MAX_VALUE, s.mutationCl(), finalMutation); + s.model.update(finalMutation); s.mutation(); }); } @@ -399,7 +451,26 @@ protected ByteBuffer[][] executeQuery(IInstance instance, int fetchSize, Consist SimpleStatement ss = new SimpleStatement(stmt.toCQL(), (Object[]) stmt.bindsEncoded()); if (fetchSize != Integer.MAX_VALUE) ss.setFetchSize(fetchSize); - ss.setConsistencyLevel(toDriverCL(cl)); + if (stmt instanceof Mutation) + { + switch (cl) + { + case SERIAL: + ss.setSerialConsistencyLevel(toDriverCL(cl)); + ss.setConsistencyLevel(com.datastax.driver.core.ConsistencyLevel.QUORUM); + break; + case LOCAL_SERIAL: + ss.setSerialConsistencyLevel(toDriverCL(cl)); + ss.setConsistencyLevel(com.datastax.driver.core.ConsistencyLevel.LOCAL_QUORUM); + break; + default: + ss.setConsistencyLevel(toDriverCL(cl)); + } + } + else + { + ss.setConsistencyLevel(toDriverCL(cl)); + } InetSocketAddress broadcastAddress = instance.config().broadcastAddress(); var host = client.getMetadata().getAllHosts().stream() diff --git a/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModel.java b/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModel.java index d2fbb6edcca0..f6a03bdd1784 100644 --- a/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModel.java +++ b/test/harry/main/org/apache/cassandra/harry/model/ASTSingleTableModel.java @@ -45,6 +45,7 @@ import accord.utils.Invariants; import org.apache.cassandra.cql3.ast.AssignmentOperator; +import org.apache.cassandra.cql3.ast.CasCondition; import org.apache.cassandra.cql3.ast.Conditional; import org.apache.cassandra.cql3.ast.Conditional.Where.Inequality; import org.apache.cassandra.cql3.ast.Element; @@ -54,10 +55,13 @@ import org.apache.cassandra.cql3.ast.Literal; import org.apache.cassandra.cql3.ast.Mutation; import org.apache.cassandra.cql3.ast.Operator; +import org.apache.cassandra.cql3.ast.Reference; +import org.apache.cassandra.cql3.ast.ReferenceExpression; import org.apache.cassandra.cql3.ast.Select; import org.apache.cassandra.cql3.ast.StandardVisitors; import org.apache.cassandra.cql3.ast.Symbol; import org.apache.cassandra.cql3.ast.Value; +import org.apache.cassandra.cql3.ast.Visitor; import org.apache.cassandra.db.BufferClustering; import org.apache.cassandra.db.Clustering; import org.apache.cassandra.db.marshal.AbstractType; @@ -75,6 +79,8 @@ public class ASTSingleTableModel { + private static final ByteBuffer[][] NO_ROWS = new ByteBuffer[0][]; + public final BytesPartitionState.Factory factory; private final TreeMap<BytesPartitionState.Ref, BytesPartitionState> partitions = new TreeMap<>(); @@ -184,6 +190,7 @@ private void indexRowColumn(TreeMap<ByteBuffer, List<PrimaryKey>> index, boolean public void update(Mutation mutation) { + if (!shouldApply(mutation)) return; switch (mutation.kind) { case INSERT: @@ -200,7 +207,7 @@ public void update(Mutation mutation) } } - public void update(Mutation.Insert insert) + private void update(Mutation.Insert insert) { Clustering<ByteBuffer> pd = pd(insert); BytesPartitionState partition = partitions.get(factory.createRef(pd)); @@ -229,7 +236,7 @@ public void update(Mutation.Insert insert) true); } - public void update(Mutation.Update update) + private void update(Mutation.Update update) { var split = splitOnPartition(update.where.simplify()); List<Clustering<ByteBuffer>> pks = split.left; @@ -250,9 +257,12 @@ public void update(Mutation.Update update) for (Symbol col : Sets.intersection(factory.staticColumns.asSet(), set.keySet())) { ByteBuffer current = partition.staticRow().get(col); - write.put(col, eval(col, current, set.get(col))); + EvalResult result = eval(col, current, set.get(col)); + if (result.kind == EvalResult.Kind.SKIP) continue; + write.put(col, result.value); } - partition.setStaticColumns(write); + if (!write.isEmpty()) + partition.setStaticColumns(write); } // table has clustering but non are in the write, so only pk/static can be updated if (!factory.clusteringColumns.isEmpty() && remaining.isEmpty()) @@ -263,10 +273,13 @@ public void update(Mutation.Update update) for (Symbol col : Sets.intersection(factory.regularColumns.asSet(), set.keySet())) { ByteBuffer current = partition.get(cd, col); - write.put(col, eval(col, current, set.get(col))); + EvalResult result = eval(col, current, set.get(col)); + if (result.kind == EvalResult.Kind.SKIP) continue; + write.put(col, result.value); } - partition.setColumns(cd, write, false); + if (!write.isEmpty()) + partition.setColumns(cd, write, false); } } } @@ -274,7 +287,7 @@ public void update(Mutation.Update update) private enum DeleteKind {PARTITION, ROW, COLUMN} - public void update(Mutation.Delete delete) + private void update(Mutation.Delete delete) { //TODO (coverage): range deletes var split = splitOnPartition(delete.where.simplify()); @@ -328,6 +341,168 @@ else if (!clusterings.isEmpty()) } } + public boolean shouldApply(Mutation mutation) + { + if (!mutation.isCas()) return true; + return shouldApply(mutation, selectPartitionForCAS(mutation)); + } + + private SelectResult selectPartitionForCAS(Mutation mutation) + { + var partition = partitions.get(factory.createRef(pd(mutation))); + if (partition == null) return SelectResult.ordered(factory.selectionOrder, NO_ROWS); + + var cd = cdOrNull(mutation); + var row = cd == null ? null : partition.get(cd); + ImmutableUniqueList<Symbol> columns = cd != null ? factory.selectionOrder : factory.partitionAndStaticColumns; + return SelectResult.ordered(columns, new ByteBuffer[][] { getRowAsByteBuffer(columns, partition, row)}); + } + + private boolean shouldApply(Mutation mutation, SelectResult current) + { + Preconditions.checkArgument(mutation.isCas()); + // process condition + CasCondition condition; + switch (mutation.kind) + { + case INSERT: + condition = CasCondition.Simple.NotExists; + break; + case UPDATE: + condition = ((Mutation.Update) mutation).casCondition.get(); + break; + case DELETE: + condition = ((Mutation.Delete) mutation).casCondition.get(); + break; + default: + throw new UnsupportedOperationException(mutation.kind.name()); + } + if (condition instanceof CasCondition.Simple) + { + boolean hasPartition = current.rows.length > 0; + boolean partitionOrRow = current.columns.equals(factory.partitionAndStaticColumns); + boolean hasRow = partitionOrRow ? hasPartition : current.isAllDefined(factory.clusteringColumns); + var simple = (CasCondition.Simple) condition; + switch (simple) + { + case Exists: + return hasRow; + case NotExists: + return !hasRow; + default: + throw new UnsupportedOperationException(simple.name()); + } + } + var ifCondition = (CasCondition.IfCondition) condition; + String letRow = "row"; + Symbol rowSymbol = Symbol.unknownType(letRow); + Map<String, SelectResult> lets = Map.of(letRow, current); + // point the columns to be row.column that way it matches LET clause in BEGIN TRANSACTION, allowing better reuse + var updatedCondition = ifCondition.conditional.visit(new Visitor() + { + @Override + public ReferenceExpression visit(ReferenceExpression r) + { + Preconditions.checkArgument(!(r instanceof Reference), "Unexpected reference detected: %s", r); + return Reference.of(rowSymbol, r); + } + }); + return process(updatedCondition, lets); + } + + private boolean process(Conditional condition, Map<String, SelectResult> lets) + { + if (condition.getClass() == Conditional.Is.class) + { + var is = (Conditional.Is) condition; + Object result = extract(is.reference, lets); + return result == null + ? is.kind == Conditional.Is.Kind.Null + : is.kind == Conditional.Is.Kind.NotNull; + } + else if (condition.getClass() == Conditional.Where.class) + { + var where = (Conditional.Where) (condition); + if (!where.lhs.type().equals(where.rhs.type())) + throw new UnsupportedOperationException("For now where clause must always have matching types: given " + where.lhs.type() + ' ' + where.rhs.type()); + ByteBuffer lhs = where.lhs instanceof ReferenceExpression + ? (ByteBuffer) extract((ReferenceExpression) where.lhs, lets) + : eval(where.lhs); + ByteBuffer rhs = where.rhs instanceof ReferenceExpression + ? (ByteBuffer) extract((ReferenceExpression) where.rhs, lets) + : eval(where.rhs); + // If anything is null avoid doing the test, but there is a special case where this returns true... both sides are null! + // This logic isn't consistent with other parts of the database and is local to CAS IF clause + // see ML@Inconsistent null handling between WHERE and IF clauses + if (lhs == null || rhs == null) + return lhs == rhs; + return where.kind.test(where.lhs.type(), lhs, rhs); + } + else if (condition.getClass() == Conditional.And.class) + { + var conditions = condition.simplify(); + for (var c : conditions) + { + if (!process(c, lets)) + return false; + } + return true; + } + else + { + throw new UnsupportedOperationException("Unsupported condition type: " + condition.getClass() + "; " + condition.toCQL()); + } + } + + // Either ByteBuffer (cell) or ByteBuffer[] (row) + private static Object extract(ReferenceExpression expr, Map<String, SelectResult> lets) + { + Object result = extract0(expr, lets); + if (result instanceof SelectResult) + { + var rows = ((SelectResult) result).rows; + result = rows.length == 0 ? null : rows[0]; + } + return result; + } + + // o can be Map<String, SelectResult> (lets), SelectResult (row), ByteBuffer (cell) + private static Object extract0(ReferenceExpression expr, @Nullable Object o) + { + if (o == null) return null; + if (expr instanceof Reference) + { + Reference ref = (Reference) expr; + for (var symbol : ref.path) + o = extract0(symbol, o); + return o; + } + else if (expr instanceof Symbol) + { + var symbol = (Symbol) expr; + if (o instanceof Map) + { + Map<String, SelectResult> lets = (Map<String, SelectResult>) o; + return lets.get(symbol.symbol); + } + else if (o instanceof SelectResult) + { + SelectResult result = (SelectResult) o; + if (result.rows.length == 0) + return null; + return result.rows[0][result.columns.indexOf(symbol)]; + } + else + { + throw new UnsupportedOperationException("Unexpected object type: " + o.getClass()); + } + } + else + { + throw new UnsupportedOperationException("Unsupported ref type: " + expr.getClass() + "; " + expr.toCQL()); + } + } + private List<Clustering<ByteBuffer>> clustering(List<Conditional> conditionals) { if (conditionals.isEmpty()) @@ -422,11 +597,74 @@ private List<Clustering<ByteBuffer>> keys(Collection<Symbol> columns, Map<Symbol return Collections.singletonList(BufferClustering.make(bbs)); } + private Clustering<ByteBuffer> pd(Mutation mutation) + { + switch (mutation.kind) + { + case INSERT: + return pd((Mutation.Insert) mutation); + case UPDATE: + return pd((Mutation.Update) mutation); + case DELETE: + return pd((Mutation.Delete) mutation); + default: + throw new UnsupportedOperationException(mutation.kind.name()); + } + } + private Clustering<ByteBuffer> pd(Mutation.Insert mutation) { return key(mutation.values, factory.partitionColumns); } + private Clustering<ByteBuffer> pd(Mutation.Update mutation) + { + return pd("Update", mutation.where.simplify()); + } + + private Clustering<ByteBuffer> pd(Mutation.Delete mutation) + { + return pd("Delete", mutation.where.simplify()); + } + + private Clustering<ByteBuffer> pd(String type, List<Conditional> conditionals) + { + var split = splitOnPartition(conditionals); + List<Clustering<ByteBuffer>> pks = split.left; + Preconditions.checkArgument(pks.size() == 1, "%s had more than 1 partition key! expected 1 but was %s", type, pks.size()); + return pks.get(0); + } + + @Nullable + private Clustering<ByteBuffer> cdOrNull(Mutation mutation) + { + if (factory.clusteringColumns.isEmpty()) return Clustering.EMPTY; + if (mutation.kind == Mutation.Kind.INSERT) + { + var insert = (Mutation.Insert) mutation; + return !insert.values.keySet().containsAll(factory.clusteringColumns) + ? null + : key(insert.values, factory.clusteringColumns); + } + Conditional where; + switch (mutation.kind) + { + case UPDATE: + where = ((Mutation.Update) mutation).where; + break; + case DELETE: + where = ((Mutation.Delete) mutation).where; + break; + default: + throw new UnsupportedOperationException("Unexpected mutation: " + mutation.kind); + } + var partitions = splitOnPartition(where.simplify()); + if (partitions.right.isEmpty()) return null; + var matches = clustering(partitions.right); + Preconditions.checkArgument(matches.size() == 1); + return matches.get(0); + } + public BytesPartitionState get(BytesPartitionState.Ref ref) { return partitions.get(ref); @@ -611,20 +849,53 @@ private static Set<Row> toRow(ImmutableUniqueList<Symbol> columns, ByteBuffer[][ private static class SelectResult { + private final ImmutableUniqueList<Symbol> columns; private final ByteBuffer[][] rows; private final boolean unordered; - private SelectResult(ByteBuffer[][] rows, boolean unordered) + private SelectResult(ImmutableUniqueList<Symbol> columns, ByteBuffer[][] rows, boolean unordered) { + this.columns = columns; this.rows = rows; this.unordered = unordered; } + + private static SelectResult ordered(ImmutableUniqueList<Symbol> columns, ByteBuffer[][] rows) + { + return new SelectResult(columns, rows, false); + } + + private static SelectResult unordered(ImmutableUniqueList<Symbol> columns, ByteBuffer[][] rows) + { + return new SelectResult(columns, rows, true); + } + + public boolean isAllDefined(ImmutableUniqueList<Symbol> selectColumns) + { + if (rows.length == 0) return false; + for (var row : rows) + { + for (var col : selectColumns) + { + if (row[columns.indexOf(col)] == null) + return false; + } + } + return true; + } + } + + public ImmutableUniqueList<Symbol> columns(Select select) + { + if (select.selections.isEmpty()) return factory.selectionOrder; + throw new UnsupportedOperationException("Getting columns from select other than SELECT * is currently not supported"); } private SelectResult getRowsAsByteBuffer(Select select) { + ImmutableUniqueList<Symbol> columns = columns(select); if (select.where.isEmpty()) - return new SelectResult(getRowsAsByteBuffer(applyLimits(all(), select.perPartitionLimit, select.limit)), false); + return SelectResult.ordered(columns, getRowsAsByteBuffer(applyLimits(all(), select.perPartitionLimit, select.limit))); LookupContext ctx = context(select); List<PrimaryKey> primaryKeys; if (ctx.unmatchable) @@ -652,7 +923,7 @@ else if (ctx.tokenLowerBound != null || ctx.tokenUpperBound != null) } primaryKeys = applyLimits(primaryKeys, select.perPartitionLimit, select.limit); //TODO (correctness): now that we have the rows we need to handle the selections/aggregation/limit/group-by/etc. - return new SelectResult(getRowsAsByteBuffer(primaryKeys), ctx.unordered); + return new SelectResult(columns, getRowsAsByteBuffer(primaryKeys), ctx.unordered); } private List<PrimaryKey> applyLimits(List<PrimaryKey> primaryKeys, Optional<Value> perPartitionLimitOpt, Optional<Value> limitOpt) @@ -709,20 +980,37 @@ public ByteBuffer[][] getRowsAsByteBuffer(List<PrimaryKey> primaryKeys) } private ByteBuffer[] getRowAsByteBuffer(BytesPartitionState partition, @Nullable BytesPartitionState.Row row) + { + return getRowAsByteBuffer(factory.selectionOrder, partition, row); + } + + private ByteBuffer[] getRowAsByteBuffer(ImmutableUniqueList<Symbol> columns, BytesPartitionState partition, @Nullable BytesPartitionState.Row row) { Clustering<ByteBuffer> pd = partition.key; BytesPartitionState.Row staticRow = partition.staticRow(); - ByteBuffer[] bbs = new ByteBuffer[factory.selectionOrder.size()]; + ByteBuffer[] bbs = new ByteBuffer[columns.size()]; for (Symbol col : factory.partitionColumns) - bbs[factory.selectionOrder.indexOf(col)] = pd.bufferAt(factory.partitionColumns.indexOf(col)); + { + if (!columns.contains(col)) continue; + bbs[columns.indexOf(col)] = pd.bufferAt(factory.partitionColumns.indexOf(col)); + } for (Symbol col : factory.staticColumns) - bbs[factory.selectionOrder.indexOf(col)] = staticRow.get(col); + { + if (!columns.contains(col)) continue; + bbs[columns.indexOf(col)] = staticRow.get(col); + } if (row != null) { for (Symbol col : factory.clusteringColumns) - bbs[factory.selectionOrder.indexOf(col)] = row.clustering.bufferAt(factory.clusteringColumns.indexOf(col)); + { + if (!columns.contains(col)) continue; + bbs[columns.indexOf(col)] = row.clustering.bufferAt(factory.clusteringColumns.indexOf(col)); + } for (Symbol col : factory.regularColumns) - bbs[factory.selectionOrder.indexOf(col)] = row.get(col); + { + if (!columns.contains(col)) continue; + bbs[columns.indexOf(col)] = row.get(col); + } } return bbs; } @@ -942,26 +1230,52 @@ private List<Clustering<ByteBuffer>> keys(Map<Symbol, List<? extends Expression> return current.stream().map(BufferClustering::new).collect(Collectors.toList()); } - private static ByteBuffer eval(Symbol col, @Nullable ByteBuffer current, Expression e) + private static class EvalResult + { + private static final EvalResult SKIP = new EvalResult(Kind.SKIP, null); + + private enum Kind { SKIP, ACCEPT } + + private final Kind kind; + private final @Nullable ByteBuffer value; + + private EvalResult(Kind kind, @Nullable ByteBuffer value) + { + this.kind = kind; + this.value = value; + } + + private static EvalResult accept(@Nullable ByteBuffer bb) + { + return new EvalResult(Kind.ACCEPT, bb); + } + } + + private static EvalResult eval(Symbol col, @Nullable ByteBuffer current, Expression e) { - if (!(e instanceof AssignmentOperator)) return eval(e); + if (!(e instanceof AssignmentOperator)) return EvalResult.accept(eval(e)); + current = col.type().sanitize(current); // multi cell collections have the property that they do update even if the current value is null boolean isFancy = col.type().isCollection() && col.type().isMultiCell(); - if (current == null && !isFancy) return null; // null + ? == null + if (current == null && !isFancy) return EvalResult.SKIP; // null + ? == null var assignment = (AssignmentOperator) e; if (isFancy && current == null) { return assignment.kind == AssignmentOperator.Kind.SUBTRACT // if it doesn't exist, then there is nothing to subtract - ? null - : eval(assignment.right); + ? EvalResult.SKIP + : EvalResult.accept(eval(assignment.right)); } + // validate your inputs... + ByteBuffer rhs = col.type().sanitize(eval(assignment.right)); + if (rhs == null) + return EvalResult.SKIP; switch (assignment.kind) { case ADD: - return eval(new Operator(Operator.Kind.ADD, new Literal(current, e.type()), assignment.right)); + return EvalResult.accept(eval(new Operator(Operator.Kind.ADD, new Literal(current, e.type()), assignment.right))); case SUBTRACT: - return eval(new Operator(Operator.Kind.SUBTRACT, new Literal(current, e.type()), assignment.right)); + return EvalResult.accept(eval(new Operator(Operator.Kind.SUBTRACT, new Literal(current, e.type()), assignment.right))); default: throw new UnsupportedOperationException(assignment.kind + ": " + assignment.toCQL()); } diff --git a/test/harry/main/org/apache/cassandra/harry/model/BytesPartitionState.java b/test/harry/main/org/apache/cassandra/harry/model/BytesPartitionState.java index a10524968ae7..c2d18e573d81 100644 --- a/test/harry/main/org/apache/cassandra/harry/model/BytesPartitionState.java +++ b/test/harry/main/org/apache/cassandra/harry/model/BytesPartitionState.java @@ -449,9 +449,10 @@ public static class Factory public final TableMetadata metadata; public final ImmutableUniqueList<Symbol> partitionColumns; public final ImmutableUniqueList<Symbol> clusteringColumns; + public final ImmutableUniqueList<Symbol> primaryColumns; public final ImmutableUniqueList<Symbol> staticColumns; public final ImmutableUniqueList<Symbol> regularColumns; - public final ImmutableUniqueList<Symbol> selectionOrder, regularAndStaticColumns; + public final ImmutableUniqueList<Symbol> selectionOrder, partitionAndStaticColumns, regularAndStaticColumns; public final ClusteringComparator clusteringComparator; @@ -471,9 +472,23 @@ public Factory(TableMetadata metadata) for (ColumnMetadata pk : metadata.clusteringColumns()) symbolListBuilder.add(Symbol.from(pk)); clusteringColumns = symbolListBuilder.buildAndClear(); + if (clusteringColumns.isEmpty()) primaryColumns = partitionColumns; + else + { + symbolListBuilder.addAll(partitionColumns); + symbolListBuilder.addAll(clusteringColumns); + primaryColumns = symbolListBuilder.buildAndClear(); + } for (ColumnMetadata pk : metadata.staticColumns()) symbolListBuilder.add(Symbol.from(pk)); staticColumns = symbolListBuilder.buildAndClear(); + if (staticColumns.isEmpty()) partitionAndStaticColumns = partitionColumns; + else + { + symbolListBuilder.addAll(partitionColumns); + symbolListBuilder.addAll(staticColumns); + partitionAndStaticColumns = symbolListBuilder.buildAndClear(); + } for (ColumnMetadata pk : metadata.regularColumns()) symbolListBuilder.add(Symbol.from(pk)); regularColumns = symbolListBuilder.buildAndClear(); diff --git a/test/unit/org/apache/cassandra/cql3/KnownIssue.java b/test/unit/org/apache/cassandra/cql3/KnownIssue.java index b1c2c09d6624..be2dfe75248c 100644 --- a/test/unit/org/apache/cassandra/cql3/KnownIssue.java +++ b/test/unit/org/apache/cassandra/cql3/KnownIssue.java @@ -40,7 +40,9 @@ public enum KnownIssue AF_MULTI_NODE_MULTI_COLUMN_AND_NODE_LOCAL_WRITES("https://issues.apache.org/jira/browse/CASSANDRA-19007", "When doing multi node/multi column queries, AF can miss data when the nodes are not in-sync"), SAI_AND_VECTOR_COLUMNS("https://issues.apache.org/jira/browse/CASSANDRA-20464", - "When doing an SAI query, if the where clause also contains a vector column bad results can be produced") + "When doing an SAI query, if the where clause also contains a vector column bad results can be produced"), + CAS_CONDITION_ON_UDT_W_EMPTY_BYTES("https://issues.apache.org/jira/browse/CASSANDRA-20479", + "WHERE clause blocks operations on UDTs but CAS allows in IF clause. During this path empty can be confused with null which allows non-existing rows to match empty bytes"), ; KnownIssue(String url, String description) diff --git a/test/unit/org/apache/cassandra/cql3/ast/AssignmentOperator.java b/test/unit/org/apache/cassandra/cql3/ast/AssignmentOperator.java index 0ffb65411b46..e3918f70da7c 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/AssignmentOperator.java +++ b/test/unit/org/apache/cassandra/cql3/ast/AssignmentOperator.java @@ -59,10 +59,15 @@ public static EnumSet<Kind> supportsOperators(AbstractType<?> type, boolean isTr EnumSet<Kind> result = EnumSet.noneOf(Kind.class); if (type instanceof CollectionType && type.isMultiCell()) { - if (type instanceof SetType || type instanceof ListType) + if (type instanceof SetType) return EnumSet.of(Kind.ADD, Kind.SUBTRACT); + if (type instanceof ListType) + return isTransaction + ? EnumSet.of(Kind.ADD, Kind.SUBTRACT) + : EnumSet.of(Kind.ADD); if (type instanceof MapType) { + //TODO (coverage): include SUBTRACT support // map supports subtract, but not map - map; only map - set! // since this is annoying to support, for now dropping - return EnumSet.of(Kind.ADD); diff --git a/test/unit/org/apache/cassandra/cql3/ast/Bind.java b/test/unit/org/apache/cassandra/cql3/ast/Bind.java index ac58dfc7df46..788d301dbf36 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/Bind.java +++ b/test/unit/org/apache/cassandra/cql3/ast/Bind.java @@ -54,6 +54,7 @@ public AbstractType<?> type() @Override public ByteBuffer valueEncoded() { + if (value == null) return null; return value instanceof ByteBuffer ? (ByteBuffer) value : ((AbstractType) type).decompose(value); } diff --git a/test/unit/org/apache/cassandra/cql3/ast/CasCondition.java b/test/unit/org/apache/cassandra/cql3/ast/CasCondition.java index 9f8d5867e0b9..d0d4d0e35b09 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/CasCondition.java +++ b/test/unit/org/apache/cassandra/cql3/ast/CasCondition.java @@ -22,6 +22,8 @@ public interface CasCondition extends Element { + CasCondition visit(Visitor v); + enum Simple implements CasCondition { NotExists("IF NOT EXISTS"), @@ -39,11 +41,17 @@ public void toCQL(StringBuilder sb, CQLFormatter formatter) { sb.append(cql); } + + @Override + public CasCondition visit(Visitor v) + { + return v.visit(this); + } } class IfCondition implements CasCondition { - private final Conditional conditional; + public final Conditional conditional; public IfCondition(Conditional conditional) { @@ -62,5 +70,15 @@ public Stream<? extends Element> stream() { return Stream.of(conditional); } + + @Override + public CasCondition visit(Visitor v) + { + var u = v.visit(this); + if (u != this) return u; + var c = conditional.visit(v); + if (c == conditional) return this; + return new IfCondition(c); + } } } diff --git a/test/unit/org/apache/cassandra/cql3/ast/Conditional.java b/test/unit/org/apache/cassandra/cql3/ast/Conditional.java index 52f79bb1dc95..66012c060cba 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/Conditional.java +++ b/test/unit/org/apache/cassandra/cql3/ast/Conditional.java @@ -72,6 +72,21 @@ public enum Inequality { this.value = value; } + + public boolean test(AbstractType<?> type, ByteBuffer a, ByteBuffer b) + { + int rc = type.compare(a, b); + switch (this) + { + case EQUAL: return rc == 0; + case NOT_EQUAL: return rc != 0; + case GREATER_THAN: return rc > 0; + case GREATER_THAN_EQ: return rc >=0; + case LESS_THAN: return rc < 0; + case LESS_THAN_EQ: return rc <=0; + default: throw new UnsupportedOperationException(this.name()); + } + } } public final Inequality kind; diff --git a/test/unit/org/apache/cassandra/cql3/ast/ExpressionEvaluator.java b/test/unit/org/apache/cassandra/cql3/ast/ExpressionEvaluator.java index 34acb843c7c6..a626dcd6b002 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/ExpressionEvaluator.java +++ b/test/unit/org/apache/cassandra/cql3/ast/ExpressionEvaluator.java @@ -55,6 +55,11 @@ public static Object eval(Operator e) Object rhs = eval(e.right); if (rhs instanceof ByteBuffer) rhs = e.right.type().compose((ByteBuffer) rhs); + // null + 42 = null + // 42 + null = null + // if anything is null, everything is null! + if (lhs == null || rhs == null) + return null; switch (e.kind) { case ADD: diff --git a/test/unit/org/apache/cassandra/cql3/ast/Literal.java b/test/unit/org/apache/cassandra/cql3/ast/Literal.java index 3886fe91b45f..4bd2f9b6319c 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/Literal.java +++ b/test/unit/org/apache/cassandra/cql3/ast/Literal.java @@ -56,6 +56,7 @@ public Object value() @Override public ByteBuffer valueEncoded() { + if (value == null) return null; return value instanceof ByteBuffer ? (ByteBuffer) value : ((AbstractType) type).decompose(value); } @@ -69,6 +70,11 @@ public Literal with(Object value, AbstractType<?> type) public void toCQL(StringBuilder sb, CQLFormatter formatter) { ByteBuffer bytes = valueEncoded(); + if (bytes == null) + { + sb.append("null"); + return; + } if (bytes.remaining() == 0 && !actuallySupportsEmpty(type)) { sb.append("<empty bytes>"); diff --git a/test/unit/org/apache/cassandra/cql3/ast/Mutation.java b/test/unit/org/apache/cassandra/cql3/ast/Mutation.java index 9126d8cbda76..95987dc657ac 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/Mutation.java +++ b/test/unit/org/apache/cassandra/cql3/ast/Mutation.java @@ -52,6 +52,8 @@ protected Mutation(Kind kind, TableReference table) public abstract boolean isCas(); + public abstract Mutation withoutTimestamp(); + public Mutation withTimestamp(long timestamp) { return withTimestamp(new Timestamp(new Literal(timestamp, LongType.instance))); @@ -172,6 +174,11 @@ public Using(Optional<TTL> ttl, Optional<Timestamp> timestamp) this.timestamp = timestamp; } + public Using withoutTimestamp() + { + return new Using(ttl, Optional.empty()); + } + public Using withTimestamp(Timestamp timestamp) { return new Using(ttl, Optional.of(timestamp)); @@ -299,6 +306,14 @@ public boolean isCas() return ifNotExists; } + @Override + public Mutation withoutTimestamp() + { + return new Insert(table, values, ifNotExists, using.isEmpty() + ? using + : using.map(u -> u.withoutTimestamp())); + } + @Override public Insert withTimestamp(Timestamp timestamp) { @@ -404,9 +419,20 @@ public Statement visit(Visitor v) Conditional copiedWhere = where.visit(v); if (where != copiedWhere) updated = true; + Optional<? extends CasCondition> updatedCasCondition = casCondition; + if (casCondition.isPresent()) + { + CasCondition original = casCondition.get(); + var casCopy = original.visit(v); + if (casCopy != original) + { + updatedCasCondition = Optional.ofNullable(casCopy); + updated = true; + } + } if (!updated) return this; - return new Update(table, using, copied, copiedWhere, casCondition); + return new Update(table, using, copied, copiedWhere, updatedCasCondition); } @Override @@ -415,6 +441,12 @@ public boolean isCas() return casCondition.isPresent(); } + @Override + public Mutation withoutTimestamp() + { + return new Update(table, using.isEmpty() ? using : using.map(u -> u.withoutTimestamp()), set, where, casCondition); + } + @Override public Update withTimestamp(Timestamp timestamp) { @@ -520,9 +552,20 @@ public Statement visit(Visitor v) var copiedWhere = where.visit(v); if (copiedWhere != where) updated = true; + Optional<? extends CasCondition> updatedCasCondition = casCondition; + if (casCondition.isPresent()) + { + CasCondition original = casCondition.get(); + var casCopy = original.visit(v); + if (casCopy != original) + { + updatedCasCondition = Optional.ofNullable(casCopy); + updated = true; + } + } if (!updated) return this; - return new Delete(copiedColumns, table, timestamp, copiedWhere, casCondition); + return new Delete(copiedColumns, table, timestamp, copiedWhere, updatedCasCondition); } @Override @@ -531,6 +574,12 @@ public boolean isCas() return casCondition.isPresent(); } + @Override + public Mutation withoutTimestamp() + { + return new Delete(columns, table, Optional.empty(), where, casCondition); + } + @Override public Delete withTimestamp(Timestamp timestamp) { diff --git a/test/unit/org/apache/cassandra/cql3/ast/Reference.java b/test/unit/org/apache/cassandra/cql3/ast/Reference.java index fe836a05ddc1..ee3b73d6fd9a 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/Reference.java +++ b/test/unit/org/apache/cassandra/cql3/ast/Reference.java @@ -48,6 +48,11 @@ public static Reference of(ReferenceExpression top) return new Reference(Collections.singletonList(Objects.requireNonNull(top))); } + public static Reference of(ReferenceExpression top, ReferenceExpression next) + { + return new Reference(List.of(top, next)); + } + @Override public AbstractType<?> type() { diff --git a/test/unit/org/apache/cassandra/cql3/ast/ReferenceExpression.java b/test/unit/org/apache/cassandra/cql3/ast/ReferenceExpression.java index 7a99e030a4e7..d1727fccdbb9 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/ReferenceExpression.java +++ b/test/unit/org/apache/cassandra/cql3/ast/ReferenceExpression.java @@ -26,6 +26,6 @@ public interface ReferenceExpression extends Expression @Override default ReferenceExpression visit(Visitor v) { - return this; + return v.visit(this); } } diff --git a/test/unit/org/apache/cassandra/cql3/ast/Value.java b/test/unit/org/apache/cassandra/cql3/ast/Value.java index f04b5fcd3638..92ef0101831d 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/Value.java +++ b/test/unit/org/apache/cassandra/cql3/ast/Value.java @@ -20,11 +20,15 @@ import java.nio.ByteBuffer; +import javax.annotation.Nullable; + import org.apache.cassandra.db.marshal.AbstractType; public interface Value extends Expression { + @Nullable Object value(); + @Nullable ByteBuffer valueEncoded(); Value with(Object value, AbstractType<?> type); diff --git a/test/unit/org/apache/cassandra/cql3/ast/Visitor.java b/test/unit/org/apache/cassandra/cql3/ast/Visitor.java index 71e4797c2595..c87415e98148 100644 --- a/test/unit/org/apache/cassandra/cql3/ast/Visitor.java +++ b/test/unit/org/apache/cassandra/cql3/ast/Visitor.java @@ -71,6 +71,11 @@ default ReferenceExpression visit(ReferenceExpression r) default Value visit(Value v) { return v; } + default CasCondition visit(CasCondition s) + { + return s; + } + class CompositeVisitor implements Visitor { private final List<Visitor> visitors; diff --git a/test/unit/org/apache/cassandra/utils/ASTGenerators.java b/test/unit/org/apache/cassandra/utils/ASTGenerators.java index 47b2267de3f9..cdf533cb420e 100644 --- a/test/unit/org/apache/cassandra/utils/ASTGenerators.java +++ b/test/unit/org/apache/cassandra/utils/ASTGenerators.java @@ -166,6 +166,7 @@ public static class ExpressionBuilder private Gen<?> valueGen; private Gen<Boolean> useOperator = SourceDSL.booleans().all(); private Gen<Boolean> useEmpty = SourceDSL.arbitrary().constant(false); + private Gen<Boolean> useNull = SourceDSL.arbitrary().constant(false); private BiFunction<Object, AbstractType<?>, Gen<Value>> literalOrBindGen = ASTGenerators::valueGen; public ExpressionBuilder(AbstractType<?> type) @@ -182,6 +183,12 @@ public ExpressionBuilder allowEmpty() return this; } + public ExpressionBuilder allowNull() + { + useNull = SourceDSL.integers().between(1, 100).map(i -> i < 10); + return this; + } + public ExpressionBuilder withOperators() { useOperator = i -> true; @@ -217,6 +224,8 @@ public Gen<Expression> build() //TODO (coverage): rather than single level operators, allow nested (a + b + c + d) Gen<Value> leaf = rs -> literalOrBindGen.apply(valueGen.generate(rs), type).generate(rs); return rs -> { + if (useNull.generate(rs)) + return new Bind(null, type); if (useEmpty.generate(rs)) return new Bind(ByteBufferUtil.EMPTY_BYTE_BUFFER, type); Expression e = leaf.generate(rs); @@ -395,6 +404,12 @@ public MutationGenBuilder allowEmpty(Symbol symbol) return this; } + public MutationGenBuilder allowNull(Symbol symbol) + { + columnExpressions.get(symbol).allowNull(); + return this; + } + public MutationGenBuilder withDeletionKind(Gen<DeleteKind> deleteKindGen) { this.deleteKindGen = deleteKindGen; @@ -432,7 +447,7 @@ public MutationGenBuilder withoutCas() public MutationGenBuilder withCasGen(Gen<Boolean> withCasGen) { - withCasGen = Objects.requireNonNull(withCasGen); + this.withCasGen = Objects.requireNonNull(withCasGen); return this; } From 9abed87ba4674864bb772bf6c8a0fdb2285b50c7 Mon Sep 17 00:00:00 2001 From: Mick Semb Wever <mck@apache.org> Date: Wed, 12 Feb 2025 18:33:44 +0100 Subject: [PATCH 221/225] Autogenerate toplevel .snyk file from owasp suppressions patch by Mick Semb Wever; reviewed by Brandon Williams for CASSANDRA-20319 --- .build/build-owasp.xml | 3 ++ .build/build-rat.xml | 1 + .build/generate-snyk-file | 75 +++++++++++++++++++++++++++++++++ .snyk | 50 ++++++++++++++++++++++ CHANGES.txt | 1 + build.xml | 3 +- debian/rules | 2 +- redhat/cassandra.spec | 2 +- redhat/noboolean/cassandra.spec | 2 +- 9 files changed, 135 insertions(+), 4 deletions(-) create mode 100755 .build/generate-snyk-file create mode 100644 .snyk diff --git a/.build/build-owasp.xml b/.build/build-owasp.xml index 35da8fdac7ac..d4b44c897b87 100644 --- a/.build/build-owasp.xml +++ b/.build/build-owasp.xml @@ -112,4 +112,7 @@ <antcall target="-run-owasp-scan" inheritrefs="true" inheritall="true"/> </target> + <target name="generate-snyk-file" unless="ant.gen-snyk.skip"> + <exec executable="${basedir}/.build/generate-snyk-file" failonerror="true"/> + </target> </project> diff --git a/.build/build-rat.xml b/.build/build-rat.xml index 09c7fd204120..210112da55ff 100644 --- a/.build/build-rat.xml +++ b/.build/build-rat.xml @@ -42,6 +42,7 @@ <fileset dir="." includesfile="${build.dir}/.ratinclude"> <!-- Config files with not much creativity --> <exclude name=".asf.yaml"/> + <exclude name=".snyk"/> <exclude name="**/cassandra*.yaml"/> <exclude name="conf/metrics-reporter-config-sample.yaml"/> <exclude NAME="doc/antora.yml"/> diff --git a/.build/generate-snyk-file b/.build/generate-snyk-file new file mode 100755 index 000000000000..bf9255ffd5bc --- /dev/null +++ b/.build/generate-snyk-file @@ -0,0 +1,75 @@ +#!/usr/bin/env python +# -*- coding: utf-8 -*- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +""" +Regenerates the toplevel .snyk file from the dependency-check-suppressions.xml found in this directory. +""" + +import re +import os + + +script_dir = os.path.dirname(os.path.abspath(__file__)) +with open(os.path.join(script_dir, "dependency-check-suppressions.xml"), "r", encoding="utf-8") as f: + xml_lines = [line.strip() for line in f.readlines()] + +snyk_data = {"ignore": {}} + +# Parse XML (keeping track of comments) +last_comment = None +i = 0 +while i < len(xml_lines): + # Detect XML comments (these contain the suppression reason) + if xml_lines[i].startswith("<!--") and xml_lines[i].endswith("-->"): + last_comment = xml_lines[i][4:-3].strip() + + # Collect CVE suppressions + if "<suppress>" in xml_lines[i]: + cve_ids = [] + packages = '' + i += 1 + while i < len(xml_lines): + if xml_lines[i].startswith("<!--") and xml_lines[i].endswith("-->"): + last_comment = xml_lines[i][4:-3].strip() + if "<cve>" in xml_lines[i] and "</cve>" in xml_lines[i]: + cve_ids.append(xml_lines[i].strip().replace("<cve>", "").replace("</cve>", "").strip()) + if "<packageUrl" in xml_lines[i] and "</packageUrl>" in xml_lines[i]: + packages = re.sub(r'<packageUrl(?: regex="(?:true|false)")?>|</packageUrl>', '', xml_lines[i]).strip() + if "</suppress>" in xml_lines[i]: + break + i += 1 + reason = f"{last_comment} -- {packages}" if last_comment else "Suppressed due to internal review, see project's .build/dependency-check-suppressions.xml" + + # Add suppressions + for cve_id in cve_ids: + snyk_data["ignore"][cve_id] = [{"reason": reason}] + + last_comment = None + else: + i += 1 + +with open(os.path.join(script_dir, "../.snyk"), "w") as snyk_file: + snyk_file.write("# Snyk (https://snyk.io) policy file, provides ignores for known false positives.\n") + snyk_file.write("# This file is autogenerated from .build/dependency-check-suppressions.xml\n") + snyk_file.write("version: v1.25.0\n") + snyk_file.write(f"ignore:\n") + for cve_id, ignores in snyk_data["ignore"].items(): + snyk_file.write(f" {cve_id}:\n") + for ignore in ignores: + snyk_file.write(f" - reason: {ignore['reason']}\n") + diff --git a/.snyk b/.snyk new file mode 100644 index 000000000000..68d9a43e949b --- /dev/null +++ b/.snyk @@ -0,0 +1,50 @@ +# Snyk (https://snyk.io) policy file, provides ignores for known false positives. +# This file is autogenerated from .build/dependency-check-suppressions.xml +version: v1.25.0 +ignore: + CVE-2022-1471: + - reason: https://issues.apache.org/jira/browse/CASSANDRA-17907 -- ^pkg:maven/org\.yaml/snakeyaml@.*$ + CVE-2022-25857: + - reason: https://issues.apache.org/jira/browse/CASSANDRA-17907 -- ^pkg:maven/org\.yaml/snakeyaml@.*$ + CVE-2022-38749: + - reason: https://issues.apache.org/jira/browse/CASSANDRA-17907 -- ^pkg:maven/org\.yaml/snakeyaml@.*$ + CVE-2022-38750: + - reason: https://issues.apache.org/jira/browse/CASSANDRA-17907 -- ^pkg:maven/org\.yaml/snakeyaml@.*$ + CVE-2022-38751: + - reason: https://issues.apache.org/jira/browse/CASSANDRA-17907 -- ^pkg:maven/org\.yaml/snakeyaml@.*$ + CVE-2022-38752: + - reason: https://issues.apache.org/jira/browse/CASSANDRA-17907 -- ^pkg:maven/org\.yaml/snakeyaml@.*$ + CVE-2022-41854: + - reason: https://issues.apache.org/jira/browse/CASSANDRA-17907 -- ^pkg:maven/org\.yaml/snakeyaml@.*$ + CVE-2020-8908: + - reason: not applicable https://nvd.nist.gov/vuln/detail/CVE-2020-8908 -- ^pkg:maven/com\.google\.guava/guava@.*$ + CVE-2023-2976: + - reason: not applicable https://nvd.nist.gov/vuln/detail/CVE-2020-8908 -- ^pkg:maven/com\.google\.guava/guava@.*$ + CVE-2021-21290: + - reason: netty's http stuff is not applicable here -- ^pkg:maven/io\.netty/netty\-all@.*$ + CVE-2021-21295: + - reason: netty's http stuff is not applicable here -- ^pkg:maven/io\.netty/netty\-all@.*$ + CVE-2021-21409: + - reason: netty's http stuff is not applicable here -- ^pkg:maven/io\.netty/netty\-all@.*$ + CVE-2021-37136: + - reason: netty's http stuff is not applicable here -- ^pkg:maven/io\.netty/netty\-all@.*$ + CVE-2021-37137: + - reason: netty's http stuff is not applicable here -- ^pkg:maven/io\.netty/netty\-all@.*$ + CVE-2021-43797: + - reason: netty's http stuff is not applicable here -- ^pkg:maven/io\.netty/netty\-all@.*$ + CVE-2022-24823: + - reason: netty's http stuff is not applicable here -- ^pkg:maven/io\.netty/netty\-all@.*$ + CVE-2022-41881: + - reason: netty's http stuff is not applicable here -- ^pkg:maven/io\.netty/netty\-all@.*$ + CVE-2023-34462: + - reason: netty's http stuff is not applicable here -- ^pkg:maven/io\.netty/netty\-all@.*$ + CVE-2023-44487: + - reason: netty's http stuff is not applicable here -- ^pkg:maven/io\.netty/netty\-all@.*$ + CVE-2022-42003: + - reason: https://issues.apache.org/jira/browse/CASSANDRA-17966 -- ^pkg:maven/com\.fasterxml\.jackson\.core/jackson\-databind@.*$ + CVE-2022-42004: + - reason: https://issues.apache.org/jira/browse/CASSANDRA-17966 -- ^pkg:maven/com\.fasterxml\.jackson\.core/jackson\-databind@.*$ + CVE-2023-35116: + - reason: https://issues.apache.org/jira/browse/CASSANDRA-17966 -- ^pkg:maven/com\.fasterxml\.jackson\.core/jackson\-databind@.*$ + CVE-2023-6378: + - reason: Suppressed due to internal review, see project's .build/dependency-check-suppressions.xml diff --git a/CHANGES.txt b/CHANGES.txt index a8a6062366fc..a1d17c3e5f5b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0.18 + * Include in source tree and build packages a Snyk policy file that lists known false positives (CASSANDRA-20319) * Update zstd-jni to 1.5.7-2 (CASSANDRA-20453) * Suppress CVE-2024-12801 (CASSANDRA-20412) * Suppress CVE-2024-12798 (CASSANDRA-20408) diff --git a/build.xml b/build.xml index 59749daa57f5..313a8caabbd7 100644 --- a/build.xml +++ b/build.xml @@ -943,7 +943,7 @@ </javac> </target> - <target depends="init,gen-cql3-grammar,generate-cql-html,generate-jflex-java,rat-check" + <target depends="init,gen-cql3-grammar,generate-cql-html,generate-jflex-java,rat-check,generate-snyk-file" name="build-project"> <echo message="${ant.project.name}: ${ant.file}"/> <!-- Order matters! --> @@ -1177,6 +1177,7 @@ <copy todir="${dist.dir}/"> <fileset dir="${basedir}"> <include name="*.txt" /> + <include name=".snyk" /> </fileset> </copy> <copy todir="${dist.dir}/tools/bin"> diff --git a/debian/rules b/debian/rules index 81812b180608..4923058537f6 100755 --- a/debian/rules +++ b/debian/rules @@ -82,7 +82,7 @@ binary-indep: build install dh_testroot dh_installchangelogs dh_installinit -u'start 50 2 3 4 5 . stop 50 0 1 6 .' - dh_installdocs README.asc CHANGES.txt NEWS.txt doc/cql3/CQL.css doc/cql3/CQL.html CASSANDRA-14092.txt + dh_installdocs README.asc CHANGES.txt NEWS.txt doc/cql3/CQL.css doc/cql3/CQL.html CASSANDRA-14092.txt .snyk dh_installexamples tools/*.yaml dh_bash-completion dh_compress diff --git a/redhat/cassandra.spec b/redhat/cassandra.spec index 775aa2a20844..a9fc164643b0 100644 --- a/redhat/cassandra.spec +++ b/redhat/cassandra.spec @@ -137,7 +137,7 @@ exit 0 %files %defattr(0644,root,root,0755) -%doc CHANGES.txt LICENSE.txt README.asc NEWS.txt NOTICE.txt CASSANDRA-14092.txt +%doc CHANGES.txt LICENSE.txt README.asc NEWS.txt NOTICE.txt CASSANDRA-14092.txt .snyk %attr(755,root,root) %{_bindir}/auditlogviewer %attr(755,root,root) %{_bindir}/jmxtool %attr(755,root,root) %{_bindir}/cassandra-stress diff --git a/redhat/noboolean/cassandra.spec b/redhat/noboolean/cassandra.spec index 269d9993a332..d3166e56b80f 100644 --- a/redhat/noboolean/cassandra.spec +++ b/redhat/noboolean/cassandra.spec @@ -140,7 +140,7 @@ exit 0 %files %defattr(0644,root,root,0755) -%doc CHANGES.txt LICENSE.txt README.asc NEWS.txt NOTICE.txt CASSANDRA-14092.txt +%doc CHANGES.txt LICENSE.txt README.asc NEWS.txt NOTICE.txt CASSANDRA-14092.txt .snyk %attr(755,root,root) %{_bindir}/auditlogviewer %attr(755,root,root) %{_bindir}/jmxtool %attr(755,root,root) %{_bindir}/cassandra-stress From e2a881959babc1ddf52821b55cbf940d58857e7c Mon Sep 17 00:00:00 2001 From: Mick Semb Wever <mck@apache.org> Date: Sun, 30 Mar 2025 10:58:07 +0200 Subject: [PATCH 222/225] =?UTF-8?q?ninja-fix=20=E2=80=93=20python2=20compa?= =?UTF-8?q?t=20on=20.build/generate-snyk-file?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit debian packaging still uses python2 redhat packaging clobbers the exec bit on git clones --- .build/build-owasp.xml | 1 + .build/generate-snyk-file | 12 +++++------ .snyk | 44 +++++++++++++++++++++------------------ 3 files changed, 31 insertions(+), 26 deletions(-) diff --git a/.build/build-owasp.xml b/.build/build-owasp.xml index d4b44c897b87..3b4a5a62988b 100644 --- a/.build/build-owasp.xml +++ b/.build/build-owasp.xml @@ -113,6 +113,7 @@ </target> <target name="generate-snyk-file" unless="ant.gen-snyk.skip"> + <chmod file="${basedir}/.build/generate-snyk-file" perm="+x"/> <exec executable="${basedir}/.build/generate-snyk-file" failonerror="true"/> </target> </project> diff --git a/.build/generate-snyk-file b/.build/generate-snyk-file index bf9255ffd5bc..8c556abf62fd 100755 --- a/.build/generate-snyk-file +++ b/.build/generate-snyk-file @@ -25,7 +25,7 @@ import os script_dir = os.path.dirname(os.path.abspath(__file__)) -with open(os.path.join(script_dir, "dependency-check-suppressions.xml"), "r", encoding="utf-8") as f: +with open(os.path.join(script_dir, "dependency-check-suppressions.xml"), "r") as f: xml_lines = [line.strip() for line in f.readlines()] snyk_data = {"ignore": {}} @@ -53,7 +53,7 @@ while i < len(xml_lines): if "</suppress>" in xml_lines[i]: break i += 1 - reason = f"{last_comment} -- {packages}" if last_comment else "Suppressed due to internal review, see project's .build/dependency-check-suppressions.xml" + reason = (last_comment + " -- " + packages) if last_comment else "Suppressed due to internal review, see project's .build/dependency-check-suppressions.xml" # Add suppressions for cve_id in cve_ids: @@ -67,9 +67,9 @@ with open(os.path.join(script_dir, "../.snyk"), "w") as snyk_file: snyk_file.write("# Snyk (https://snyk.io) policy file, provides ignores for known false positives.\n") snyk_file.write("# This file is autogenerated from .build/dependency-check-suppressions.xml\n") snyk_file.write("version: v1.25.0\n") - snyk_file.write(f"ignore:\n") - for cve_id, ignores in snyk_data["ignore"].items(): - snyk_file.write(f" {cve_id}:\n") + snyk_file.write("ignore:\n") + for cve_id, ignores in sorted(snyk_data["ignore"].items()): + snyk_file.write(" {}:\n".format(cve_id)) for ignore in ignores: - snyk_file.write(f" - reason: {ignore['reason']}\n") + snyk_file.write(" - reason: {}\n".format(ignore['reason'])) diff --git a/.snyk b/.snyk index 68d9a43e949b..66d72fb74c58 100644 --- a/.snyk +++ b/.snyk @@ -2,24 +2,8 @@ # This file is autogenerated from .build/dependency-check-suppressions.xml version: v1.25.0 ignore: - CVE-2022-1471: - - reason: https://issues.apache.org/jira/browse/CASSANDRA-17907 -- ^pkg:maven/org\.yaml/snakeyaml@.*$ - CVE-2022-25857: - - reason: https://issues.apache.org/jira/browse/CASSANDRA-17907 -- ^pkg:maven/org\.yaml/snakeyaml@.*$ - CVE-2022-38749: - - reason: https://issues.apache.org/jira/browse/CASSANDRA-17907 -- ^pkg:maven/org\.yaml/snakeyaml@.*$ - CVE-2022-38750: - - reason: https://issues.apache.org/jira/browse/CASSANDRA-17907 -- ^pkg:maven/org\.yaml/snakeyaml@.*$ - CVE-2022-38751: - - reason: https://issues.apache.org/jira/browse/CASSANDRA-17907 -- ^pkg:maven/org\.yaml/snakeyaml@.*$ - CVE-2022-38752: - - reason: https://issues.apache.org/jira/browse/CASSANDRA-17907 -- ^pkg:maven/org\.yaml/snakeyaml@.*$ - CVE-2022-41854: - - reason: https://issues.apache.org/jira/browse/CASSANDRA-17907 -- ^pkg:maven/org\.yaml/snakeyaml@.*$ CVE-2020-8908: - reason: not applicable https://nvd.nist.gov/vuln/detail/CVE-2020-8908 -- ^pkg:maven/com\.google\.guava/guava@.*$ - CVE-2023-2976: - - reason: not applicable https://nvd.nist.gov/vuln/detail/CVE-2020-8908 -- ^pkg:maven/com\.google\.guava/guava@.*$ CVE-2021-21290: - reason: netty's http stuff is not applicable here -- ^pkg:maven/io\.netty/netty\-all@.*$ CVE-2021-21295: @@ -32,19 +16,39 @@ ignore: - reason: netty's http stuff is not applicable here -- ^pkg:maven/io\.netty/netty\-all@.*$ CVE-2021-43797: - reason: netty's http stuff is not applicable here -- ^pkg:maven/io\.netty/netty\-all@.*$ + CVE-2022-1471: + - reason: https://issues.apache.org/jira/browse/CASSANDRA-17907 -- ^pkg:maven/org\.yaml/snakeyaml@.*$ CVE-2022-24823: - reason: netty's http stuff is not applicable here -- ^pkg:maven/io\.netty/netty\-all@.*$ + CVE-2022-25857: + - reason: https://issues.apache.org/jira/browse/CASSANDRA-17907 -- ^pkg:maven/org\.yaml/snakeyaml@.*$ + CVE-2022-38749: + - reason: https://issues.apache.org/jira/browse/CASSANDRA-17907 -- ^pkg:maven/org\.yaml/snakeyaml@.*$ + CVE-2022-38750: + - reason: https://issues.apache.org/jira/browse/CASSANDRA-17907 -- ^pkg:maven/org\.yaml/snakeyaml@.*$ + CVE-2022-38751: + - reason: https://issues.apache.org/jira/browse/CASSANDRA-17907 -- ^pkg:maven/org\.yaml/snakeyaml@.*$ + CVE-2022-38752: + - reason: https://issues.apache.org/jira/browse/CASSANDRA-17907 -- ^pkg:maven/org\.yaml/snakeyaml@.*$ + CVE-2022-41854: + - reason: https://issues.apache.org/jira/browse/CASSANDRA-17907 -- ^pkg:maven/org\.yaml/snakeyaml@.*$ CVE-2022-41881: - reason: netty's http stuff is not applicable here -- ^pkg:maven/io\.netty/netty\-all@.*$ - CVE-2023-34462: - - reason: netty's http stuff is not applicable here -- ^pkg:maven/io\.netty/netty\-all@.*$ - CVE-2023-44487: - - reason: netty's http stuff is not applicable here -- ^pkg:maven/io\.netty/netty\-all@.*$ CVE-2022-42003: - reason: https://issues.apache.org/jira/browse/CASSANDRA-17966 -- ^pkg:maven/com\.fasterxml\.jackson\.core/jackson\-databind@.*$ CVE-2022-42004: - reason: https://issues.apache.org/jira/browse/CASSANDRA-17966 -- ^pkg:maven/com\.fasterxml\.jackson\.core/jackson\-databind@.*$ + CVE-2023-2976: + - reason: not applicable https://nvd.nist.gov/vuln/detail/CVE-2020-8908 -- ^pkg:maven/com\.google\.guava/guava@.*$ + CVE-2023-34462: + - reason: netty's http stuff is not applicable here -- ^pkg:maven/io\.netty/netty\-all@.*$ CVE-2023-35116: - reason: https://issues.apache.org/jira/browse/CASSANDRA-17966 -- ^pkg:maven/com\.fasterxml\.jackson\.core/jackson\-databind@.*$ + CVE-2023-44487: + - reason: netty's http stuff is not applicable here -- ^pkg:maven/io\.netty/netty\-all@.*$ CVE-2023-6378: - reason: Suppressed due to internal review, see project's .build/dependency-check-suppressions.xml + CVE-2024-12798: + - reason: Suppressed due to internal review, see project's .build/dependency-check-suppressions.xml + CVE-2024-12801: + - reason: Suppressed due to internal review, see project's .build/dependency-check-suppressions.xml From d954c19b21bcf53da3f15fae40a0751c6dcc73df Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Fri, 28 Mar 2025 10:46:34 +0100 Subject: [PATCH 223/225] CASSANDRA-20296 follow-up: by default, explicitly set -XX:MaxDirectMemorySize to half of -Xmx patch by Stefan Miklosovic; reviewed by Michael Semb Wever for CASSANDRA-20296 --- NEWS.txt | 1 + conf/cassandra-env.sh | 14 ++++++++++++++ conf/jvm-server.options | 6 ++++++ 3 files changed, 21 insertions(+) diff --git a/NEWS.txt b/NEWS.txt index 5b5ec9b92a1a..efbc8d7ac86a 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -74,6 +74,7 @@ Upgrading storage-attached indexes (SAI) to make migration between the two safer. This behavior can be switched off via the flag `sai_options.prioritize_over_legacy_index` (which defaults to `false`) in `cassandra.yaml` or via `setPrioritizeSAIOverLegacyIndex(boolean)` in the JMX MBean `org.apache.cassandra.db:type=StorageService`. + - Java's -XX:MaxDirectMemorySize is by default explicitly half of max heap size (-Xmx) instead of implicitly equal to it. 5.0.1 ===== diff --git a/conf/cassandra-env.sh b/conf/cassandra-env.sh index 209816197955..2d83763b8994 100644 --- a/conf/cassandra-env.sh +++ b/conf/cassandra-env.sh @@ -56,11 +56,14 @@ calculate_heap_sizes() heap_limit="31744" fi half_system_memory_in_mb=`expr $system_memory_in_mb / 2` + quarter_system_memory_in_mb=`expr $system_memory_in_mb / 4` if [ "$half_system_memory_in_mb" -gt "$heap_limit" ] ; then CALCULATED_MAX_HEAP_SIZE="${heap_limit}M" + CALCULATED_MAX_DIRECT_MEMORY_SIZE="`expr $heap_limit / 2`M" CALCULATED_CMS_HEAP_NEWSIZE="8G" else CALCULATED_MAX_HEAP_SIZE="${half_system_memory_in_mb}M" + CALCULATED_MAX_DIRECT_MEMORY_SIZE="${quarter_system_memory_in_mb}M" CALCULATED_CMS_HEAP_NEWSIZE="`expr $half_system_memory_in_mb / 4`M" fi } @@ -87,6 +90,8 @@ echo $JVM_OPTS | grep -q Xmx DEFINED_XMX=$? echo $JVM_OPTS | grep -q Xms DEFINED_XMS=$? +echo $JVM_OPTS | grep -q MaxDirectMemorySize +DEFINED_MAX_DIRECT_MEMORY_SIZE=$? echo $JVM_OPTS | grep -q ParallelGCThreads DEFINED_PARALLEL_GC_THREADS=$? echo $JVM_OPTS | grep -q ConcGCThreads @@ -112,6 +117,7 @@ calculate_heap_sizes #MAX_HEAP_SIZE="20G" #HEAP_NEWSIZE="10G" +#MAX_DIRECT_MEMORY_SIZE="10G" # Set this to control the amount of arenas per-thread in glibc #export MALLOC_ARENA_MAX=4 @@ -130,6 +136,10 @@ elif [ "x$MAX_HEAP_SIZE" = "x" ] || [ "x$HEAP_NEWSIZE" = "x" -a $USING_G1 -ne 0 exit 1 fi +if [ "x$MAX_DIRECT_MEMORY_SIZE" = "x" ]; then + MAX_DIRECT_MEMORY_SIZE="$CALCULATED_MAX_DIRECT_MEMORY_SIZE" +fi + if [ "x$MALLOC_ARENA_MAX" = "x" ] ; then export MALLOC_ARENA_MAX=4 fi @@ -144,6 +154,10 @@ elif [ $DEFINED_XMX -ne 0 ] || [ $DEFINED_XMS -ne 0 ]; then exit 1 fi +if [ $DEFINED_MAX_DIRECT_MEMORY_SIZE -ne 0 ]; then + JVM_OPTS="$JVM_OPTS -XX:MaxDirectMemorySize=${MAX_DIRECT_MEMORY_SIZE}" +fi + # We only set -Xmn flag if it was not defined in jvm-server.options file # and CMS is being used. If defined, both Xmn and Xmx must be defined together. if [ $DEFINED_XMN -eq 0 ] && [ $DEFINED_XMX -ne 0 ]; then diff --git a/conf/jvm-server.options b/conf/jvm-server.options index 547a06dce3f7..f68d875a51e2 100644 --- a/conf/jvm-server.options +++ b/conf/jvm-server.options @@ -166,6 +166,12 @@ # For production use you may wish to adjust this for your environment. # If that's the case, see MAX_HEAP_SIZE (and HEAP_NEWSIZE for CMS) in cassandra-env.sh +##################### +# OFF-HEAP SETTINGS # +##################### + +# By default, this setting is half of max heap size +#-XX:MaxDirectMemorySize= ################################### # EXPIRATION DATE OVERFLOW POLICY # From bb66561142788270ab450c02de836b3952ed37b4 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Sun, 23 Mar 2025 23:37:11 +0100 Subject: [PATCH 224/225] Various fixes in constraint framework - fix some edge cases for NOT_NULL - ability to specify constraints when altering a column - ensure constraint is specified on a column it is bound to - fix nullity check on map type - fix satistfiability check on function constraints patch by Stefan Miklosovic; reviewed by Bernardo Botella for CASSANDRA-20481 --- CHANGES.txt | 1 + src/antlr/Parser.g | 10 +- ...AbstractFunctionSatisfiabilityChecker.java | 27 ++- .../cql3/constraints/ColumnConstraint.java | 5 +- .../cql3/constraints/ColumnConstraints.java | 18 +- .../cql3/constraints/ConstraintFunction.java | 10 +- .../cql3/constraints/JsonConstraint.java | 9 +- .../cql3/constraints/NotNullConstraint.java | 9 +- .../constraints/UnaryConstraintFunction.java | 37 ++++ .../cql3/statements/UpdateStatement.java | 4 +- .../schema/AlterTableStatement.java | 19 +- .../schema/CreateTableStatement.java | 2 +- .../cassandra/db/marshal/AbstractType.java | 14 +- .../cassandra/db/marshal/CollectionType.java | 6 + .../apache/cassandra/db/marshal/MapType.java | 6 - .../cassandra/schema/TableMetadata.java | 4 +- ...ableWithTableConstraintValidationTest.java | 42 +++- .../ConstraintsSatisfiabilityTest.java | 17 +- ...WithColumnCqlConstraintValidationTest.java | 18 ++ .../constraints/JsonConstraintTest.java | 2 +- .../constraints/NotNullConstraintTest.java | 165 ++++++++++++++ .../cql3/ColumnSpecificationTest.java | 206 ++++++++++++++++++ .../db/marshal/AbstractTypeTest.java | 9 +- 23 files changed, 568 insertions(+), 72 deletions(-) create mode 100644 src/java/org/apache/cassandra/cql3/constraints/UnaryConstraintFunction.java create mode 100644 test/unit/org/apache/cassandra/constraints/NotNullConstraintTest.java create mode 100644 test/unit/org/apache/cassandra/cql3/ColumnSpecificationTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 0e155821c0ab..c1ee79f162d1 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.1 + * Various fixes in constraint framework (CASSANDRA-20481) * Add support in CAS for -= on numeric types, and fixed improper handling of empty bytes which lead to NPE (CASSANDRA-20477) * Do not fail to start a node with materialized views after they are turned off in config (CASSANDRA-20452) * Fix nodetool gcstats output, support human-readable units and more output formats (CASSANDRA-19022) diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g index dd9eb181dfe7..c91dfe60d925 100644 --- a/src/antlr/Parser.g +++ b/src/antlr/Parser.g @@ -988,14 +988,14 @@ alterTableStatement returns [AlterTableStatement.Raw stmt] | K_ALTER ( K_IF K_EXISTS { $stmt.ifColumnExists(true); } )? id=cident ( mask=columnMask { $stmt.mask(id, mask); } + | constraints=columnConstraints { $stmt.constraint(id, constraints); } | K_DROP K_MASKED { $stmt.mask(id, null); } - | K_DROP K_CHECK { $stmt.constraint(id, null); } - | (constraints=columnConstraints) { $stmt.constraint(id, constraints); }) + | K_DROP K_CHECK { $stmt.constraint(id, null); }) | K_ADD ( K_IF K_NOT K_EXISTS { $stmt.ifColumnNotExists(true); } )? - ( id=ident v=comparatorType b=isStaticColumn (m=columnMask)? { $stmt.add(id, v, b, m); } - | ('(' id1=ident v1=comparatorType b1=isStaticColumn (m1=columnMask)? { $stmt.add(id1, v1, b1, m1); } - ( ',' idn=ident vn=comparatorType bn=isStaticColumn (mn=columnMask)? { $stmt.add(idn, vn, bn, mn); mn=null; } )* ')') ) + ( id=ident v=comparatorType b=isStaticColumn (m=columnMask)? (c=columnConstraints)? { $stmt.add(id, v, b, m, c); } + | ('(' id1=ident v1=comparatorType b1=isStaticColumn (m1=columnMask)? (c=columnConstraints)? { $stmt.add(id1, v1, b1, m1, c); } + ( ',' idn=ident vn=comparatorType bn=isStaticColumn (mn=columnMask)? (c=columnConstraints)? { $stmt.add(idn, vn, bn, mn, c); mn=null; c=null;} )* ')') ) | K_DROP ( K_IF K_EXISTS { $stmt.ifColumnExists(true); } )? ( id=ident { $stmt.drop(id); } diff --git a/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionSatisfiabilityChecker.java b/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionSatisfiabilityChecker.java index 91d448da012a..cebc5c36d3dd 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionSatisfiabilityChecker.java +++ b/src/java/org/apache/cassandra/cql3/constraints/AbstractFunctionSatisfiabilityChecker.java @@ -26,6 +26,8 @@ import org.apache.cassandra.cql3.Operator; import org.apache.cassandra.cql3.functions.types.ParseUtils; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.Int32Type; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.utils.Pair; @@ -70,6 +72,8 @@ public void check(String functionName, List<ColumnConstraint<?>> constraints, Co */ abstract Pair<List<CONSTRAINT_TYPE>, List<CONSTRAINT_TYPE>> categorizeConstraints(List<ColumnConstraint<?>> constraints, String functionName); + abstract AbstractType<?> returnType(ColumnMetadata columnMetadata); + private void checkSupportedOperators(List<CONSTRAINT_TYPE> allConstraints, String functionName) { for (CONSTRAINT_TYPE constraint : allConstraints) @@ -147,11 +151,12 @@ else if (firstRelation == NEQ && secondRelation == NEQ) } else { - ByteBuffer firstTermBuffer = columnMetadata.type.fromString(ParseUtils.unquote(firstTerm)); - ByteBuffer secondTermBuffer = columnMetadata.type.fromString(ParseUtils.unquote(secondTerm)); + AbstractType<?> returnType = returnType(columnMetadata); + ByteBuffer firstTermBuffer = returnType.fromString(ParseUtils.unquote(firstTerm)); + ByteBuffer secondTermBuffer = returnType.fromString(ParseUtils.unquote(secondTerm)); - boolean firstSatisfaction = firstRelation.isSatisfiedBy(columnMetadata.type, secondTermBuffer, firstTermBuffer); - boolean secondSatisfaction = secondRelation.isSatisfiedBy(columnMetadata.type, firstTermBuffer, secondTermBuffer); + boolean firstSatisfaction = firstRelation.isSatisfiedBy(returnType, secondTermBuffer, firstTermBuffer); + boolean secondSatisfaction = secondRelation.isSatisfiedBy(returnType, firstTermBuffer, secondTermBuffer); if (!firstSatisfaction || !secondSatisfaction) throw new InvalidConstraintDefinitionException(format("Constraints of %s are not satisfiable: %s %s %s, %s %s %s", @@ -186,6 +191,14 @@ public Pair<List<ScalarColumnConstraint>, List<ScalarColumnConstraint>> categori return Pair.create(scalars, notEqualScalars); } + + @Override + AbstractType<?> returnType(ColumnMetadata metadata) + { + // function constraints will always have terms of int32 type + // unlike scalar constraints where it will be a type of column + return metadata.type; + } }; public static final AbstractFunctionSatisfiabilityChecker<FunctionColumnConstraint> FUNCTION_SATISFIABILITY_CHECKER = new AbstractFunctionSatisfiabilityChecker<>() @@ -215,5 +228,11 @@ public Pair<List<FunctionColumnConstraint>, List<FunctionColumnConstraint>> cate return Pair.create(funnctionColumnConstraints, notEqualConstraints); } + + @Override + AbstractType<?> returnType(ColumnMetadata columnMetadata) + { + return Int32Type.instance; + } }; } diff --git a/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraint.java index ddcca653ea30..eecc0b8ecce1 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraint.java @@ -30,6 +30,7 @@ import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.tcm.serialization.MetadataSerializer; +import org.apache.cassandra.utils.ByteBufferUtil; import static java.lang.String.format; @@ -116,8 +117,10 @@ public String fullName() */ public void evaluate(AbstractType<?> valueType, ByteBuffer columnValue) throws ConstraintViolationException { - if (columnValue.capacity() == 0) + if (columnValue == ByteBufferUtil.EMPTY_BYTE_BUFFER) throw new ConstraintViolationException("Column value does not satisfy value constraint for column '" + columnName + "' as it is null."); + else if (valueType.isEmptyValueMeaningless() && columnValue.capacity() == 0) + throw new ConstraintViolationException("Column value does not satisfy value constraint for column '" + columnName + "' as it is empty."); internalEvaluate(valueType, columnValue); } diff --git a/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraints.java b/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraints.java index 0acace098f46..21b119522866 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraints.java +++ b/src/java/org/apache/cassandra/cql3/constraints/ColumnConstraints.java @@ -27,6 +27,7 @@ import java.util.Set; import java.util.TreeSet; +import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.CqlBuilder; import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.db.marshal.AbstractType; @@ -108,7 +109,7 @@ public int getSize() // Checks if there is at least one constraint that will perform checks public boolean hasRelevantConstraints() { - for (ColumnConstraint c : constraints) + for (ColumnConstraint<?> c : constraints) { if (c != ColumnConstraints.NO_OP) return true; @@ -120,9 +121,12 @@ public boolean hasRelevantConstraints() public void validate(ColumnMetadata columnMetadata) throws InvalidConstraintDefinitionException { if (!columnMetadata.type.isConstrainable()) + { throw new InvalidConstraintDefinitionException("Constraint cannot be defined on the column " + columnMetadata.name + " of type " + columnMetadata.type.asCQL3Type() - + " for the table " + columnMetadata.ksName + "." + columnMetadata.cfName); + + " for the table " + columnMetadata.ksName + '.' + columnMetadata.cfName + '.' + + (columnMetadata.type.isCollection() ? " When using collections, constraints can be used only of frozen collections." : "")); + } // this will look at constraints as a whole, // checking if combinations of a particular constraint make sense (duplicities, satisfiability etc.). @@ -207,10 +211,18 @@ public Raw() this.constraints = Collections.emptyList(); } - public ColumnConstraints prepare() + public ColumnConstraints prepare(ColumnIdentifier column) { if (constraints.isEmpty()) return NO_OP; + + for (ColumnConstraint<?> constraint : constraints) + { + if (constraint.columnName != null && !column.equals(constraint.columnName)) + throw new InvalidConstraintDefinitionException(format("Constraint %s was not specified on a column it operates on: %s but on: %s", + constraint, column.toCQLString(), constraint.columnName)); + } + return new ColumnConstraints(constraints); } } diff --git a/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java b/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java index 9952ab32d94f..ad8424c3f563 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java +++ b/src/java/org/apache/cassandra/cql3/constraints/ConstraintFunction.java @@ -25,6 +25,7 @@ import org.apache.cassandra.cql3.Operator; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.utils.ByteBufferUtil; import static org.apache.cassandra.cql3.Operator.EQ; import static org.apache.cassandra.cql3.Operator.GT; @@ -55,8 +56,10 @@ public ConstraintFunction(ColumnIdentifier columnName, String name) */ public void evaluate(AbstractType<?> valueType, Operator relationType, String term, ByteBuffer columnValue) throws ConstraintViolationException { - if (columnValue.capacity() == 0) + if (columnValue == ByteBufferUtil.EMPTY_BYTE_BUFFER) throw new ConstraintViolationException("Column value does not satisfy value constraint for column '" + columnName + "' as it is null."); + else if (valueType.isEmptyValueMeaningless() && columnValue.capacity() == 0) + throw new ConstraintViolationException("Column value does not satisfy value constraint for column '" + columnName + "' as it is empty."); internalEvaluate(valueType, relationType, term, columnValue); } @@ -88,10 +91,7 @@ public void validate(ColumnMetadata columnMetadata, String term) throws InvalidC * * @return list of operators this function is allowed to have. */ - public List<Operator> getSupportedOperators() - { - return List.of(); - } + public abstract List<Operator> getSupportedOperators(); /** * Tells what types of columns are supported by this constraint. diff --git a/src/java/org/apache/cassandra/cql3/constraints/JsonConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/JsonConstraint.java index 99aeb6734e63..95fbac5b3c0a 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/JsonConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/JsonConstraint.java @@ -31,7 +31,7 @@ import static java.lang.String.format; -public class JsonConstraint extends ConstraintFunction +public class JsonConstraint extends UnaryConstraintFunction { private static final List<AbstractType<?>> SUPPORTED_TYPES = List.of(UTF8Type.instance, AsciiType.instance); @@ -39,12 +39,7 @@ public class JsonConstraint extends ConstraintFunction public JsonConstraint(ColumnIdentifier columnName) { - this(columnName, FUNCTION_NAME); - } - - public JsonConstraint(ColumnIdentifier columnName, String name) - { - super(columnName, name); + super(columnName, FUNCTION_NAME); } @Override diff --git a/src/java/org/apache/cassandra/cql3/constraints/NotNullConstraint.java b/src/java/org/apache/cassandra/cql3/constraints/NotNullConstraint.java index fb9f7de95b2d..af79086701f0 100644 --- a/src/java/org/apache/cassandra/cql3/constraints/NotNullConstraint.java +++ b/src/java/org/apache/cassandra/cql3/constraints/NotNullConstraint.java @@ -28,18 +28,13 @@ import static java.lang.String.format; -public class NotNullConstraint extends ConstraintFunction +public class NotNullConstraint extends UnaryConstraintFunction { public static final String FUNCTION_NAME = "NOT_NULL"; public NotNullConstraint(ColumnIdentifier columnName) { - this(columnName, FUNCTION_NAME); - } - - public NotNullConstraint(ColumnIdentifier columnName, String name) - { - super(columnName, name); + super(columnName, FUNCTION_NAME); } @Override diff --git a/src/java/org/apache/cassandra/cql3/constraints/UnaryConstraintFunction.java b/src/java/org/apache/cassandra/cql3/constraints/UnaryConstraintFunction.java new file mode 100644 index 000000000000..8696e81a65a7 --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/constraints/UnaryConstraintFunction.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3.constraints; + +import java.util.List; + +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.Operator; + +public abstract class UnaryConstraintFunction extends ConstraintFunction +{ + public UnaryConstraintFunction(ColumnIdentifier columnName, String name) + { + super(columnName, name); + } + + public List<Operator> getSupportedOperators() + { + return List.of(); + } +} diff --git a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java index d8310cdd2f8d..f6ecda2b87e9 100644 --- a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java @@ -371,9 +371,7 @@ public static void evaluateConstraintsForRow(Row row, TableMetadata metadata) public static void evaluateConstraint(ColumnMetadata columnMetadata, ByteBuffer cellData) { - for (ColumnConstraint constraint : columnMetadata.getColumnConstraints().getConstraints()) - { + for (ColumnConstraint<?> constraint : columnMetadata.getColumnConstraints().getConstraints()) constraint.evaluate(columnMetadata.type, cellData); - } } } diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java index fc2ab582f754..94120ac63ce1 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java @@ -257,13 +257,16 @@ private static class Column private final boolean isStatic; @Nullable private final ColumnMask.Raw mask; + @Nullable + private final ColumnConstraints.Raw constraints; - Column(ColumnIdentifier name, CQL3Type.Raw type, boolean isStatic, @Nullable ColumnMask.Raw mask) + Column(ColumnIdentifier name, CQL3Type.Raw type, boolean isStatic, @Nullable ColumnMask.Raw mask, @Nullable ColumnConstraints.Raw constraints) { this.name = name; this.type = type; this.isStatic = isStatic; this.mask = mask; + this.constraints = constraints; } } @@ -311,6 +314,7 @@ private void addColumn(KeyspaceMetadata keyspace, AbstractType<?> type = column.type.prepare(keyspaceName, keyspace.types).getType(); boolean isStatic = column.isStatic; ColumnMask mask = column.mask == null ? null : column.mask.prepare(keyspaceName, tableName, name, type, keyspace.userFunctions); + ColumnConstraints columnConstraints = column.constraints == null ? ColumnConstraints.NO_OP : column.constraints.prepare(name); if (null != tableBuilder.getColumn(name)) { if (!ifColumnNotExists) @@ -361,9 +365,9 @@ private void addColumn(KeyspaceMetadata keyspace, } if (isStatic) - tableBuilder.addStaticColumn(name, type, mask); + tableBuilder.addStaticColumn(name, type, mask, columnConstraints); else - tableBuilder.addRegularColumn(name, type, mask); + tableBuilder.addRegularColumn(name, type, mask, columnConstraints); if (!isStatic) { @@ -372,7 +376,8 @@ private void addColumn(KeyspaceMetadata keyspace, if (view.includeAllColumns) { ColumnMetadata viewColumn = ColumnMetadata.regularColumn(view.metadata, name.bytes, type) - .withNewMask(mask); + .withNewMask(mask) + .withNewColumnConstraints(columnConstraints); viewsBuilder.put(viewsBuilder.get(view.name()).withAddedRegularColumn(viewColumn)); } } @@ -732,7 +737,7 @@ public KeyspaceMetadata apply(Epoch epoch, KeyspaceMetadata keyspace, TableMetad if (column != null) { ColumnConstraints oldConstraints = column.getColumnConstraints(); - ColumnConstraints newConstraints = constraints == null ? ColumnConstraints.NO_OP : constraints.prepare(); + ColumnConstraints newConstraints = constraints == null ? ColumnConstraints.NO_OP : constraints.prepare(columnName); if (Objects.equals(oldConstraints, newConstraints)) return keyspace; newConstraints.validate(column); @@ -837,10 +842,10 @@ public void mask(ColumnIdentifier name, ColumnMask.Raw mask) rawMask = mask; } - public void add(ColumnIdentifier name, CQL3Type.Raw type, boolean isStatic, @Nullable ColumnMask.Raw mask) + public void add(ColumnIdentifier name, CQL3Type.Raw type, boolean isStatic, @Nullable ColumnMask.Raw mask, @Nullable ColumnConstraints.Raw constraints) { kind = Kind.ADD_COLUMNS; - addedColumns.add(new AddColumns.Column(name, type, isStatic, mask)); + addedColumns.add(new AddColumns.Column(name, type, isStatic, mask, constraints)); } public void drop(ColumnIdentifier name) diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java index 997a40200077..b8e51d1286e5 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java @@ -578,7 +578,7 @@ public void addColumn(ColumnIdentifier column, CQL3Type.Raw type, boolean isStat if (null == constraints) columnConstraints.put(column, ColumnConstraints.NO_OP); else - columnConstraints.put(column, constraints.prepare()); + columnConstraints.put(column, constraints.prepare(column)); } public void setCompactStorage() diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractType.java b/src/java/org/apache/cassandra/db/marshal/AbstractType.java index 5378a4cd3fba..b5156c4fdb8f 100644 --- a/src/java/org/apache/cassandra/db/marshal/AbstractType.java +++ b/src/java/org/apache/cassandra/db/marshal/AbstractType.java @@ -211,20 +211,12 @@ public <V> void validate(V value, ValueAccessor<V> accessor) throws MarshalExcep public void checkConstraints(ByteBuffer bytes, ColumnConstraints constraints) throws ConstraintViolationException { - if (constraints.isEmpty()) - return; - - T value = getSerializer().deserialize(bytes); - constraints.evaluate(this, bytes); + checkConstraints(bytes, constraints.getConstraints()); } - public void checkConstraints(ByteBuffer bytes, List<ColumnConstraint> constraints) throws ConstraintViolationException + public void checkConstraints(ByteBuffer bytes, List<ColumnConstraint<?>> constraints) throws ConstraintViolationException { - if (constraints.isEmpty()) - return; - - T value = getSerializer().deserialize(bytes); - for (ColumnConstraint constraint : constraints) + for (ColumnConstraint<?> constraint : constraints) constraint.evaluate(this, bytes); } diff --git a/src/java/org/apache/cassandra/db/marshal/CollectionType.java b/src/java/org/apache/cassandra/db/marshal/CollectionType.java index 8c39dbab4f5b..3952ee137e0a 100644 --- a/src/java/org/apache/cassandra/db/marshal/CollectionType.java +++ b/src/java/org/apache/cassandra/db/marshal/CollectionType.java @@ -171,6 +171,12 @@ public boolean isFreezable() return true; } + @Override + public boolean isConstrainable() + { + return isFrozenCollection(); + } + public ByteBuffer serializeForNativeProtocol(Iterator<Cell<?>> cells) { assert isMultiCell(); diff --git a/src/java/org/apache/cassandra/db/marshal/MapType.java b/src/java/org/apache/cassandra/db/marshal/MapType.java index f8ac6c00680e..69ea6d17e1d3 100644 --- a/src/java/org/apache/cassandra/db/marshal/MapType.java +++ b/src/java/org/apache/cassandra/db/marshal/MapType.java @@ -456,10 +456,4 @@ public ByteBuffer getElement(@Nullable ColumnData columnData, ByteBuffer keyOrIn return getSerializer().getSerializedValue(((Cell<?>) columnData).buffer(), keyOrIndex, getValuesType()); } - - @Override - public boolean isConstrainable() - { - return false; - } } diff --git a/src/java/org/apache/cassandra/schema/TableMetadata.java b/src/java/org/apache/cassandra/schema/TableMetadata.java index 9fdf5e821542..268111abd68f 100644 --- a/src/java/org/apache/cassandra/schema/TableMetadata.java +++ b/src/java/org/apache/cassandra/schema/TableMetadata.java @@ -204,7 +204,7 @@ public enum Kind // We cache the columns with constraints to avoid iterations over columns // Partition keys columns are evaluated separately, so we keep the two of them in // two different variables. - public final List<ColumnConstraint> partitionKeyConstraints; + public final List<ColumnConstraint<?>> partitionKeyConstraints; public final List<ColumnMetadata> columnsWithConstraints; public final List<ColumnMetadata> notNullColumns; @@ -248,7 +248,7 @@ else if (isIndex()) else ref = TableMetadataRef.withInitialReference(new TableMetadataRef(Schema.instance, keyspace, name, id), this); - List<ColumnConstraint> pkConstraints = new ArrayList<>(this.partitionKeyColumns.size()); + List<ColumnConstraint<?>> pkConstraints = new ArrayList<>(this.partitionKeyColumns.size()); for (ColumnMetadata column : this.partitionKeyColumns) { if (column.hasConstraint()) diff --git a/test/unit/org/apache/cassandra/constraints/AlterTableWithTableConstraintValidationTest.java b/test/unit/org/apache/cassandra/constraints/AlterTableWithTableConstraintValidationTest.java index d5f3f4a32c50..91f21f8b0922 100644 --- a/test/unit/org/apache/cassandra/constraints/AlterTableWithTableConstraintValidationTest.java +++ b/test/unit/org/apache/cassandra/constraints/AlterTableWithTableConstraintValidationTest.java @@ -22,10 +22,8 @@ import org.apache.cassandra.exceptions.InvalidRequestException; - public class AlterTableWithTableConstraintValidationTest extends CqlConstraintValidationTester { - @Test public void testCreateTableWithColumnNamedConstraintDescribeTableNonFunction() throws Throwable { @@ -239,4 +237,44 @@ public void testCreateTableAddConstraintWithNonExistingColumn() throws Throwable String expectedErrorMessage = "Column 'foo' doesn't exist"; assertInvalidThrowMessage(expectedErrorMessage, InvalidRequestException.class, "ALTER TABLE %s ALTER foo CHECK foo < 100"); } + + @Test + public void testAlterTableAlterExistingColumnWithCheckOnNonExistingColumn() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 text, ck2 text, v int, PRIMARY KEY ((pk),ck1, ck2));"); + assertInvalidThrowMessage("Constraint ck3 < 100 was not specified on a column it operates on: ck1 but on: ck3", + InvalidRequestException.class, + "ALTER TABLE %s ALTER ck1 CHECK ck3 < 100"); + assertInvalidThrowMessage("Constraint NOT_NULL(ck3) was not specified on a column it operates on: ck1 but on: ck3", + InvalidRequestException.class, + "ALTER TABLE %s ALTER ck1 CHECK NOT_NULL(ck3)"); + assertInvalidThrowMessage("Constraint LENGTH(ck3) > 10 was not specified on a column it operates on: ck1 but on: ck3", + InvalidRequestException.class, + "ALTER TABLE %s ALTER ck1 CHECK LENGTH(ck3) > 10"); + } + + @Test + public void testAlterTableAddNewColumnWithCheckOnNonExistingColumn() throws Throwable + { + createTable("CREATE TABLE %s (pk int, ck1 text, ck2 text, v int, PRIMARY KEY ((pk),ck1, ck2));"); + + assertInvalidThrowMessage("Constraint v3 < 100 was not specified on a column it operates on: v2 but on: v3", + InvalidRequestException.class, + "ALTER TABLE %s ADD v2 int CHECK v3 < 100"); + + assertInvalidThrowMessage("Constraint NOT_NULL(v3) was not specified on a column it operates on: v2 but on: v3", + InvalidRequestException.class, + "ALTER TABLE %s ADD v2 int CHECK NOT_NULL(v3)"); + + assertInvalidThrowMessage("Constraint LENGTH(v3) > 10 was not specified on a column it operates on: v2 but on: v3", + InvalidRequestException.class, + "ALTER TABLE %s ADD v2 int CHECK LENGTH(v3) > 10"); + } + + @Test + public void testAlterTableAddColumnWithCheck() + { + createTable("CREATE TABLE %s (pk text, col1 int, primary key (pk));"); + execute("ALTER TABLE %s ADD col2 int CHECK col2 > 0"); + } } diff --git a/test/unit/org/apache/cassandra/constraints/ConstraintsSatisfiabilityTest.java b/test/unit/org/apache/cassandra/constraints/ConstraintsSatisfiabilityTest.java index 6f087c5850b5..148aaa48a67f 100644 --- a/test/unit/org/apache/cassandra/constraints/ConstraintsSatisfiabilityTest.java +++ b/test/unit/org/apache/cassandra/constraints/ConstraintsSatisfiabilityTest.java @@ -71,13 +71,13 @@ private <T> void run(QuadFunction<T> quadFunction, ColumnMetadata columnMetadata if (op1 == NEQ) { // a_column != 0 and a_column != 10 -> valid - check(op1, 0, op2, 100, quadFunction, null, columnMetadata); + check(op1, 50, op2, 100, quadFunction, null, columnMetadata); // does not make sense to check twice // check a_column != 0 and a_column != 0 check(op1, 0, op2, 0, quadFunction, "There are duplicate constraint definitions on column", columnMetadata); } else - check(op1, 0, op2, 100, quadFunction, "There are duplicate constraint definitions on column", columnMetadata); + check(op1, 50, op2, 100, quadFunction, "There are duplicate constraint definitions on column", columnMetadata); } else if ((op1 == GT && op2 == GTE) || (op1 == GTE && op2 == GT) || @@ -85,18 +85,25 @@ else if ((op1 == GT && op2 == GTE) || (op1 == LTE && op2 == LT) || (op1 == EQ || op2 == EQ)) { - check(op1, 0, op2, 100, quadFunction, "not supported", columnMetadata); + check(op1, 50, op2, 100, quadFunction, "not supported", columnMetadata); } else if ((op1 == LTE && op2 == GT) || (op1 == LT && op2 == GT) || (op1 == LTE && op2 == GTE) || (op1 == LT && op2 == GTE)) { - check(op1, 0, op2, 100, quadFunction, "are not satisfiable", columnMetadata); + check(op1, 50, op2, 100, quadFunction, "are not satisfiable", columnMetadata); + } + else if ((op1 == GT && op2 == LTE) || + (op1 == GT && op2 == LT) || + (op1 == GTE && op2 == LTE) || + (op1 == GTE && op2 == LT)) + { + check(op1, 50, op2, 100, quadFunction, null, columnMetadata); } else if (!(op1 == NEQ || op2 == NEQ)) { - check(op1, 0, op2, 100, quadFunction, null, columnMetadata); + check(op1, 50, op2, 100, quadFunction, null, columnMetadata); } else { diff --git a/test/unit/org/apache/cassandra/constraints/CreateTableWithColumnCqlConstraintValidationTest.java b/test/unit/org/apache/cassandra/constraints/CreateTableWithColumnCqlConstraintValidationTest.java index 6dc160468642..857ec85f408b 100644 --- a/test/unit/org/apache/cassandra/constraints/CreateTableWithColumnCqlConstraintValidationTest.java +++ b/test/unit/org/apache/cassandra/constraints/CreateTableWithColumnCqlConstraintValidationTest.java @@ -26,9 +26,11 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import org.apache.cassandra.cql3.constraints.InvalidConstraintDefinitionException; import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.utils.Generators; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static accord.utils.Property.qt; @@ -1438,4 +1440,20 @@ public void testCreateTableWithColumnWithClusteringColumnLessThanScalarConstrain } }); } + + @Test + public void testCreateTableAddConstraintWithCheckOnNonExistingColumn() throws Throwable + { + assertThatThrownBy(() -> createTable("CREATE TABLE %s (pk int, ck1 text CHECK NOT_NULL(ck3), ck2 text, v int, PRIMARY KEY ((pk),ck1, ck2));")) + .hasRootCauseMessage("Constraint NOT_NULL(ck3) was not specified on a column it operates on: ck1 but on: ck3") + .rootCause().isInstanceOf(InvalidConstraintDefinitionException.class); + + assertThatThrownBy(() -> createTable("CREATE TABLE %s (pk int, ck1 int CHECK ck3 > 5, ck2 text, v int, PRIMARY KEY ((pk),ck1, ck2));")) + .hasRootCauseMessage("Constraint ck3 > 5 was not specified on a column it operates on: ck1 but on: ck3") + .rootCause().isInstanceOf(InvalidConstraintDefinitionException.class); + + assertThatThrownBy(() -> createTable("CREATE TABLE %s (pk int, ck1 text CHECK LENGTH(ck3) > 10, ck2 text, v int, PRIMARY KEY ((pk),ck1, ck2));")) + .hasRootCauseMessage("Constraint LENGTH(ck3) > 10 was not specified on a column it operates on: ck1 but on: ck3") + .rootCause().isInstanceOf(InvalidConstraintDefinitionException.class); + } } diff --git a/test/unit/org/apache/cassandra/constraints/JsonConstraintTest.java b/test/unit/org/apache/cassandra/constraints/JsonConstraintTest.java index 95db5b7604de..adf86093bc42 100644 --- a/test/unit/org/apache/cassandra/constraints/JsonConstraintTest.java +++ b/test/unit/org/apache/cassandra/constraints/JsonConstraintTest.java @@ -50,7 +50,7 @@ public void testJsonConstraint() throws Throwable run("{}"); run("{\"a\": 5, \"b\": \"1\", \"c\": [1,2,3]}"); run("nonsense", "Value for column 'a_column' violated JSON constraint as it is not a valid JSON."); - run("", "Column value does not satisfy value constraint for column 'a_column' as it is null."); + run("", "Value for column 'a_column' violated JSON constraint as it is not a valid JSON."); } @Test diff --git a/test/unit/org/apache/cassandra/constraints/NotNullConstraintTest.java b/test/unit/org/apache/cassandra/constraints/NotNullConstraintTest.java new file mode 100644 index 000000000000..61d24d850683 --- /dev/null +++ b/test/unit/org/apache/cassandra/constraints/NotNullConstraintTest.java @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.constraints; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.junit.Test; + +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.constraints.ColumnConstraints; +import org.apache.cassandra.cql3.constraints.ConstraintViolationException; +import org.apache.cassandra.cql3.constraints.FunctionColumnConstraint; +import org.apache.cassandra.cql3.constraints.InvalidConstraintDefinitionException; +import org.apache.cassandra.cql3.constraints.NotNullConstraint; +import org.apache.cassandra.cql3.constraints.ScalarColumnConstraint; +import org.apache.cassandra.cql3.constraints.UnaryFunctionColumnConstraint; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.ListType; +import org.apache.cassandra.db.marshal.MapType; +import org.apache.cassandra.db.marshal.SetType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.marshal.UUIDType; +import org.apache.cassandra.schema.ColumnMetadata; + +import static java.util.List.of; +import static org.apache.cassandra.cql3.Operator.GT; +import static org.apache.cassandra.schema.ColumnMetadata.Kind.REGULAR; +import static org.apache.cassandra.utils.ByteBufferUtil.EMPTY_BYTE_BUFFER; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** + * TODO - UDTs are not supported yet in constraints as such + */ +public class NotNullConstraintTest +{ + private static final ColumnIdentifier columnIdentifier = new ColumnIdentifier("a_column", false); + private static final ColumnConstraints unaryConstraint = new ColumnConstraints(of(new UnaryFunctionColumnConstraint.Raw(new ColumnIdentifier(NotNullConstraint.FUNCTION_NAME, false), columnIdentifier).prepare())); + private static final ColumnConstraints scalarConstraint = new ColumnConstraints(of(new ScalarColumnConstraint.Raw(columnIdentifier, GT, "5").prepare())); + private static final ColumnConstraints functionConstraint = new ColumnConstraints(of(new FunctionColumnConstraint.Raw(new ColumnIdentifier("LENGTH", false), columnIdentifier, GT, "5").prepare())); + + @Test + public void testNotNullConstraintValidation() + { + // unary + unaryConstraint.validate(getColumnOfType(UTF8Type.instance)); + assertThatThrownBy(() -> unaryConstraint.evaluate(UTF8Type.instance, EMPTY_BYTE_BUFFER)) + .hasMessage("Column value does not satisfy value constraint for column 'a_column' as it is null.") + .isInstanceOf(ConstraintViolationException.class); + + // not null / empty + unaryConstraint.evaluate(UTF8Type.instance, UTF8Type.instance.fromString("a value")); + + // scalar + scalarConstraint.validate(getColumnOfType(Int32Type.instance)); + assertThatThrownBy(() -> scalarConstraint.evaluate(Int32Type.instance, EMPTY_BYTE_BUFFER)) + .hasMessage("Column value does not satisfy value constraint for column 'a_column' as it is null.") + .isInstanceOf(ConstraintViolationException.class); + + // function, e.g. length + functionConstraint.validate(getColumnOfType(UTF8Type.instance)); + assertThatThrownBy(() -> functionConstraint.evaluate(UTF8Type.instance, EMPTY_BYTE_BUFFER)) + .hasMessage("Column value does not satisfy value constraint for column 'a_column' as it is null.") + .isInstanceOf(ConstraintViolationException.class); + + // empty string is not _null_ string so this passes + unaryConstraint.evaluate(UTF8Type.instance, UTF8Type.instance.fromString("")); + + // test a type for which empty value is meaningless + + assertThatThrownBy(() -> unaryConstraint.evaluate(UUIDType.instance, ByteBuffer.allocate(0))) + .hasMessage("Column value does not satisfy value constraint for column 'a_column' as it is empty.") + .isInstanceOf(ConstraintViolationException.class); + } + + @Test + public void testCollections() + { + checkList(false); + checkSet(false); + checkMap(false); + + checkList(true); + checkSet(true); + checkMap(true); + } + + private static ColumnMetadata getColumnOfType(AbstractType<?> type) + { + return new ColumnMetadata("a", "b", columnIdentifier, type, -1, REGULAR, null); + } + + private void checkList(boolean frozen) + { + if (frozen) + { + ListType<Integer> listType = ListType.getInstance(Int32Type.instance, false); + ByteBuffer payload = listType.getSerializer().serialize(List.of(1, 2, 3)); + checkFrozenCollection(listType, payload); + } + else + checkUnfrozenCollection(ListType.getInstance(Int32Type.instance, true)); + } + + private void checkMap(boolean frozen) + { + if (frozen) + { + MapType<Integer, Integer> mapType = MapType.getInstance(Int32Type.instance, Int32Type.instance, false); + ByteBuffer payload = mapType.getSerializer().serialize(Map.of(1, 1, 2, 2, 3, 3)); + checkFrozenCollection(mapType, payload); + } + else + checkUnfrozenCollection(MapType.getInstance(Int32Type.instance, Int32Type.instance, true)); + } + + private void checkSet(boolean frozen) + { + if (frozen) + { + SetType<Integer> setType = SetType.getInstance(Int32Type.instance, false); + ByteBuffer payload = setType.getSerializer().serialize(Set.of(1, 2, 3)); + checkFrozenCollection(setType, payload); + } + else + checkUnfrozenCollection(SetType.getInstance(Int32Type.instance, true)); + } + + private void checkFrozenCollection(AbstractType<?> type, ByteBuffer payload) + { + unaryConstraint.validate(getColumnOfType(type)); + unaryConstraint.evaluate(type, payload); + + assertThatThrownBy(() -> unaryConstraint.evaluate(type, EMPTY_BYTE_BUFFER)) + .hasMessage("Column value does not satisfy value constraint for column 'a_column' as it is null.") + .isInstanceOf(ConstraintViolationException.class); + } + + private void checkUnfrozenCollection(AbstractType<?> type) + { + assertThatThrownBy(() -> unaryConstraint.validate(getColumnOfType(type))) + .hasMessageContaining("Constraint cannot be defined on the column") + .hasMessageContaining("When using collections, constraints can be used only of frozen collections") + .isInstanceOf(InvalidConstraintDefinitionException.class); + } +} diff --git a/test/unit/org/apache/cassandra/cql3/ColumnSpecificationTest.java b/test/unit/org/apache/cassandra/cql3/ColumnSpecificationTest.java new file mode 100644 index 000000000000..8513a5440023 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/ColumnSpecificationTest.java @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.cql3; + +import java.util.Map; + +import org.junit.Before; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.schema.CompactionParams; + +/** + * Test various "extensions" to a column spec when altering / creating a table + */ +public class ColumnSpecificationTest extends CQLTester +{ + @Before + public void before() + { + DatabaseDescriptor.setDynamicDataMaskingEnabled(true); + } + + @Test + public void testCreateTableWithColumnHavingMaskBeforeCheck() + { + createTable("CREATE TABLE %s (pk text primary key, name text MASKED WITH system.mask_default() CHECK NOT_NULL(name) AND LENGTH(name) > 1);"); + verifyColumnSpec("name text MASKED WITH system.mask_default() CHECK NOT_NULL(name) AND LENGTH(name) > 1"); + } + + @Test + public void testAlterTableAlterColumnWithMaskAndCheckStandalone() + { + createTable("CREATE TABLE %s (pk text, name text, primary key (pk));"); + execute("ALTER TABLE %s ALTER name MASKED WITH system.mask_default()"); + execute("ALTER TABLE %s ALTER name CHECK NOT_NULL(name) AND LENGTH(name) > 1;"); + verifyColumnSpec("name text MASKED WITH system.mask_default() CHECK NOT_NULL(name) AND LENGTH(name) > 1"); + } + + @Test + public void testAlterTableAlterColumnWithMask() + { + createTable("CREATE TABLE %s (pk text, name text, primary key (pk));"); + execute("ALTER TABLE %s ALTER name MASKED WITH system.mask_default()"); + verifyColumnSpec("name text MASKED WITH system.mask_default()"); + } + + @Test + public void testAlterTableAlterColumnWithCheck() + { + createTable("CREATE TABLE %s (pk text, name text, primary key (pk));"); + execute("ALTER TABLE %s ALTER name CHECK NOT_NULL(name) AND LENGTH(name) > 1;"); + verifyColumnSpec("name text CHECK NOT_NULL(name) AND LENGTH(name) > 1"); + } + + @Test + public void testAddingCheckToColumnWithMask() + { + createTable("CREATE TABLE %s (pk text primary key, name text MASKED WITH system.mask_default());"); + execute("ALTER TABLE %s ALTER name CHECK NOT_NULL(name) AND LENGTH(name) > 1"); + verifyColumnSpec("name text MASKED WITH system.mask_default() CHECK NOT_NULL(name) AND LENGTH(name) > 1"); + } + + @Test + public void testAddingMaskToColumnWithCheck() + { + createTable("CREATE TABLE %s (pk text primary key, name text CHECK NOT_NULL(name) AND LENGTH(name) > 1);"); + execute("ALTER TABLE %s ALTER name MASKED WITH system.mask_default()"); + verifyColumnSpec("name text MASKED WITH system.mask_default() CHECK NOT_NULL(name) AND LENGTH(name) > 1"); + } + + @Test + public void testDroppingCheckKeepsMask() + { + createTable("CREATE TABLE %s (pk text primary key, name text MASKED WITH system.mask_default() CHECK NOT_NULL(name) AND LENGTH(name) > 1);"); + execute("ALTER TABLE %s ALTER name DROP CHECK"); + verifyColumnSpec("name text MASKED WITH system.mask_default()"); + } + + @Test + public void droppingMaskKeepsCheck() + { + createTable("CREATE TABLE %s (pk text primary key, name text MASKED WITH system.mask_default() CHECK NOT_NULL(name) AND LENGTH(name) > 1);"); + execute("ALTER TABLE %s ALTER name DROP MASKED"); + verifyColumnSpec("name text CHECK NOT_NULL(name) AND LENGTH(name) > 1"); + } + + @Test + public void testAlterTableAddColumnWithCheck() + { + createTable("CREATE TABLE %s (pk text primary key);"); + execute("ALTER TABLE %s ADD name text CHECK NOT_NULL(name) AND LENGTH(name) > 1"); + verifyColumnSpec("name text CHECK NOT_NULL(name) AND LENGTH(name) > 1"); + } + + @Test + public void testAlterTableAddColumnWithMask() + { + createTable("CREATE TABLE %s (pk text primary key);"); + execute("ALTER TABLE %s ADD name text MASKED WITH system.mask_default()"); + verifyColumnSpec("name text MASKED WITH system.mask_default()"); + } + + @Test + public void testAlterTableAddColumnWithMaskAndCheck() + { + createTable("CREATE TABLE %s (pk text primary key);"); + execute("ALTER TABLE %s ADD name text MASKED WITH system.mask_default() CHECK NOT_NULL(name)"); + verifyColumnSpec("name text MASKED WITH system.mask_default() CHECK NOT_NULL(name)"); + } + + @Test + public void testAlterTableAddColumnWithMaskAndMultipleChecks() + { + createTable("CREATE TABLE %s (pk text primary key);"); + execute("ALTER TABLE %s ADD name text MASKED WITH system.mask_default() CHECK NOT_NULL(name) AND LENGTH(name) > 1"); + verifyColumnSpec("name text MASKED WITH system.mask_default() CHECK NOT_NULL(name) AND LENGTH(name) > 1"); + } + + /** + * TODO - investigate if it is possible to specify checks before mask when creating a table + */ + @Test(expected = RuntimeException.class) + public void testFailingCreateTableWithColumnHavingMaskAfterCheck() + { + createTable("CREATE TABLE %s (pk text primary key, name text CHECK NOT_NULL(name) AND LENGTH(name) > 1 MASKED WITH system.mask_default());"); + } + + /** + * TODO - investigate if it is possible to specify both check and mask, check being first + */ + @Test(expected = RuntimeException.class) + public void testFailingAlterTableAlterColumnWithCheckAndMask() + { + createTable("CREATE TABLE %s (pk text, name text, primary key (pk));"); + execute("ALTER TABLE %s ALTER name CHECK NOT_NULL(name) AND LENGTH(name) > 1 MASKED WITH system.mask_default();"); + verifyColumnSpec("name text MASKED WITH system.mask_default() CHECK NOT_NULL(name) AND LENGTH(name) > 1"); + } + + /** + * TODO - investigate if it is possible to specify both check and mask, mask being first + */ + @Test(expected = RuntimeException.class) + public void testFailingAlterTableAlterColumnWithMaskAndCheck() + { + createTable("CREATE TABLE %s (pk text, name text, primary key (pk));"); + execute("ALTER TABLE %s ALTER name MASKED WITH system.mask_default() CHECK NOT_NULL(name) AND LENGTH(name) > 1"); + verifyColumnSpec("name text MASKED WITH system.mask_default() CHECK NOT_NULL(name) AND LENGTH(name) > 1"); + } + + private void verifyColumnSpec(String modifiedColumn) + { + assertRowsContains(executeNetWithoutPaging("DESCRIBE TABLE " + KEYSPACE + '.' + currentTable()), + row(KEYSPACE, + "table", + currentTable(), + "CREATE TABLE " + KEYSPACE + '.' + currentTable() + " (\n" + + " pk text PRIMARY KEY,\n" + + " " + modifiedColumn + '\n' + + ") WITH " + tableParametersCql())); + } + + static String tableParametersCql() + { + return "additional_write_policy = '99p'\n" + + " AND allow_auto_snapshot = true\n" + + " AND bloom_filter_fp_chance = 0.01\n" + + " AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'}\n" + + " AND cdc = false\n" + + " AND comment = ''\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 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(); + } +} diff --git a/test/unit/org/apache/cassandra/db/marshal/AbstractTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/AbstractTypeTest.java index 5cb051b35e77..630c322fce3a 100644 --- a/test/unit/org/apache/cassandra/db/marshal/AbstractTypeTest.java +++ b/test/unit/org/apache/cassandra/db/marshal/AbstractTypeTest.java @@ -263,10 +263,15 @@ private boolean isTestType(Class<? extends AbstractType> klass) public void isConstrainedTest() { qt().forAll(genBuilder().build()).checkAssert(type -> { - if (type instanceof MapType || type instanceof TupleType || type instanceof AbstractCompositeType) + if (type instanceof TupleType || type instanceof AbstractCompositeType) assertThat(type.isConstrainable()).isEqualTo(false); else - assertThat(type.isConstrainable()).isEqualTo(true); + { + if (type.isCollection() && !type.isFrozenCollection()) + assertThat(type.isConstrainable()).isEqualTo(false); + else + assertThat(type.isConstrainable()).isEqualTo(true); + } }); } From 73f0e2e4017397bfdcf16b7e333c1279ada57c74 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic <smiklosovic@apache.org> Date: Tue, 1 Apr 2025 13:03:24 +0200 Subject: [PATCH 225/225] Suppress CVE-2025-25193 patch by Stefan Miklosovic; reviewed by Michael Semb Wever, Brandon Williams for CASSANDRA-20504 --- .build/dependency-check-suppressions.xml | 1 + .snyk | 2 ++ CHANGES.txt | 1 + 3 files changed, 4 insertions(+) diff --git a/.build/dependency-check-suppressions.xml b/.build/dependency-check-suppressions.xml index 4cdcc9149c7c..70e1f8716493 100644 --- a/.build/dependency-check-suppressions.xml +++ b/.build/dependency-check-suppressions.xml @@ -50,6 +50,7 @@ <cve>CVE-2022-41881</cve> <cve>CVE-2023-34462</cve> <cve>CVE-2023-44487</cve> + <cve>CVE-2025-25193</cve> </suppress> <!-- https://issues.apache.org/jira/browse/CASSANDRA-17966 --> <suppress> diff --git a/.snyk b/.snyk index 66d72fb74c58..50af01c673d7 100644 --- a/.snyk +++ b/.snyk @@ -52,3 +52,5 @@ ignore: - reason: Suppressed due to internal review, see project's .build/dependency-check-suppressions.xml CVE-2024-12801: - reason: Suppressed due to internal review, see project's .build/dependency-check-suppressions.xml + CVE-2025-25193: + - reason: netty's http stuff is not applicable here -- ^pkg:maven/io\.netty/netty\-all@.*$ diff --git a/CHANGES.txt b/CHANGES.txt index a1d17c3e5f5b..da5ad11d3aec 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0.18 + * Suppress CVE-2025-25193 (CASSANDRA-20504) * Include in source tree and build packages a Snyk policy file that lists known false positives (CASSANDRA-20319) * Update zstd-jni to 1.5.7-2 (CASSANDRA-20453) * Suppress CVE-2024-12801 (CASSANDRA-20412)